diff --git a/base/base/shared_ptr_helper.h b/base/base/shared_ptr_helper.h deleted file mode 100644 index c1214a20c85..00000000000 --- a/base/base/shared_ptr_helper.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once - -#include - - -/** Allows to make std::shared_ptr from T with protected constructor. - * - * Derive your T class from shared_ptr_helper and add shared_ptr_helper as a friend - * and you will have static 'create' method in your class. - */ -template -struct shared_ptr_helper -{ - template - static std::shared_ptr create(TArgs &&... args) - { - return std::shared_ptr(new T(std::forward(args)...)); - } -}; - - -template -struct is_shared_ptr -{ - static constexpr bool value = false; -}; - - -template -struct is_shared_ptr> -{ - static constexpr bool value = true; -}; - -template -inline constexpr bool is_shared_ptr_v = is_shared_ptr::value; diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index d0742e0c691..44073320a4c 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 432106faf29..977d6de988a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1146,7 +1146,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des ConstraintsDescription{}, String{}, }; - StoragePtr storage = StorageFile::create(in_file, global_context->getUserFilesPath(), args); + StoragePtr storage = std::make_shared(in_file, global_context->getUserFilesPath(), args); storage->startup(); SelectQueryInfo query_info; diff --git a/src/Common/typeid_cast.h b/src/Common/typeid_cast.h index 465de425532..3c3f236f740 100644 --- a/src/Common/typeid_cast.h +++ b/src/Common/typeid_cast.h @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -60,9 +59,26 @@ To typeid_cast(From * from) } } +namespace detail +{ + +template +struct is_shared_ptr : std::false_type +{ +}; + +template +struct is_shared_ptr> : std::true_type +{ +}; + +template +inline constexpr bool is_shared_ptr_v = is_shared_ptr::value; + +} template -requires is_shared_ptr_v +requires detail::is_shared_ptr_v To typeid_cast(const std::shared_ptr & from) { try diff --git a/src/Core/MySQL/MySQLClient.cpp b/src/Core/MySQL/MySQLClient.cpp index 98797b3d284..b83acfe5e98 100644 --- a/src/Core/MySQL/MySQLClient.cpp +++ b/src/Core/MySQL/MySQLClient.cpp @@ -56,7 +56,7 @@ void MySQLClient::connect() in = std::make_shared(*socket); out = std::make_shared(*socket); - packet_endpoint = MySQLProtocol::PacketEndpoint::create(*in, *out, sequence_id); + packet_endpoint = std::make_shared(*in, *out, sequence_id); handshake(); } diff --git a/src/Core/MySQL/MySQLClient.h b/src/Core/MySQL/MySQLClient.h index 9fa3ace6baa..1dd8eda1575 100644 --- a/src/Core/MySQL/MySQLClient.h +++ b/src/Core/MySQL/MySQLClient.h @@ -56,7 +56,7 @@ private: std::shared_ptr out; std::unique_ptr socket; std::optional address; - std::shared_ptr packet_endpoint; + MySQLProtocol::PacketEndpointPtr packet_endpoint; void handshake(); void registerSlaveOnMaster(UInt32 slave_id); diff --git a/src/Core/MySQL/PacketEndpoint.h b/src/Core/MySQL/PacketEndpoint.h index b1897958ced..5e1a6a31048 100644 --- a/src/Core/MySQL/PacketEndpoint.h +++ b/src/Core/MySQL/PacketEndpoint.h @@ -1,11 +1,11 @@ #pragma once +#include #include #include #include "IMySQLReadPacket.h" #include "IMySQLWritePacket.h" #include "IO/MySQLPacketPayloadReadBuffer.h" -#include namespace DB { @@ -16,13 +16,19 @@ namespace MySQLProtocol /* Writes and reads packets, keeping sequence-id. * Throws ProtocolError, if packet with incorrect sequence-id was received. */ -class PacketEndpoint : public shared_ptr_helper +class PacketEndpoint : boost::noncopyable { public: uint8_t & sequence_id; ReadBuffer * in; WriteBuffer * out; + /// For writing. + PacketEndpoint(WriteBuffer & out_, uint8_t & sequence_id_); + + /// For reading and writing. + PacketEndpoint(ReadBuffer & in_, WriteBuffer & out_, uint8_t & sequence_id_); + MySQLPacketPayloadReadBuffer getPayload(); void receivePacket(IMySQLReadPacket & packet); @@ -43,15 +49,6 @@ public: /// Converts packet to text. Is used for debug output. static String packetToText(const String & payload); - -protected: - /// For writing. - PacketEndpoint(WriteBuffer & out_, uint8_t & sequence_id_); - - /// For reading and writing. - PacketEndpoint(ReadBuffer & in_, WriteBuffer & out_, uint8_t & sequence_id_); - - friend struct shared_ptr_helper; }; using PacketEndpointPtr = std::shared_ptr; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 3c7dfd2b9f9..3a403fdb5ab 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -31,7 +31,7 @@ namespace DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); auto comment = load_result.config->config->getString("dictionary.comment", ""); - return StorageDictionary::create( + return std::make_shared( StorageID(database_name, load_result.name), load_result.name, dictionary_structure, diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 13f55eab9e2..d97cd35c169 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -137,7 +137,7 @@ StoragePtr DatabaseMaterializedMySQL::tryGetTable(const String & name, ContextPt if (context_->isInternalQuery()) return nested_storage; if (nested_storage) - return std::make_shared(std::move(nested_storage), this); + return StorageMaterializedMySQL::create(std::move(nested_storage), this); return nullptr; } diff --git a/src/Databases/MySQL/DatabaseMaterializedTablesIterator.h b/src/Databases/MySQL/DatabaseMaterializedTablesIterator.h index 8a7dbacf4a2..2a96b0cfa8f 100644 --- a/src/Databases/MySQL/DatabaseMaterializedTablesIterator.h +++ b/src/Databases/MySQL/DatabaseMaterializedTablesIterator.h @@ -23,7 +23,7 @@ public: const StoragePtr & table() const override { - StoragePtr storage = std::make_shared(nested_iterator->table(), database); + StoragePtr storage = StorageMaterializedMySQL::create(nested_iterator->table(), database); return tables.emplace_back(storage); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 5ce96cc37ad..60d2fa0d2c8 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -94,12 +94,12 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() if (storage) { /// Nested table was already created and synchronized. - storage = StorageMaterializedPostgreSQL::create(storage, getContext(), remote_database_name, table_name); + storage = std::make_shared(storage, getContext(), remote_database_name, table_name); } else { /// Nested table does not exist and will be created by replication thread. - storage = StorageMaterializedPostgreSQL::create(StorageID(database_name, table_name), getContext(), remote_database_name, table_name); + storage = std::make_shared(StorageID(database_name, table_name), getContext(), remote_database_name, table_name); } /// Cache MaterializedPostgreSQL wrapper over nested table. @@ -210,7 +210,7 @@ ASTPtr DatabaseMaterializedPostgreSQL::getCreateTableQueryImpl(const String & ta std::lock_guard lock(handler_mutex); - auto storage = StorageMaterializedPostgreSQL::create(StorageID(database_name, table_name), getContext(), remote_database_name, table_name); + auto storage = std::make_shared(StorageID(database_name, table_name), getContext(), remote_database_name, table_name); auto ast_storage = replication_handler->getCreateNestedTableQuery(storage.get(), table_name); assert_cast(ast_storage.get())->uuid = UUIDHelpers::generateV4(); return ast_storage; @@ -291,7 +291,7 @@ void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const Stri InterpreterAlterQuery(alter_query, current_context).execute(); - auto storage = StorageMaterializedPostgreSQL::create(table, getContext(), remote_database_name, table_name); + auto storage = std::make_shared(table, getContext(), remote_database_name, table_name); materialized_tables[table_name] = storage; std::lock_guard lock(handler_mutex); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 7e2be7e324a..1bcc203beb9 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -187,7 +187,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, if (!columns_info) return StoragePtr{}; - auto storage = StoragePostgreSQL::create( + auto storage = std::make_shared( StorageID(database_name, table_name), pool, table_name, ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict); diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 5ca3a6f3d5d..7a4844e4d69 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -145,7 +145,7 @@ StoragePtr DatabaseSQLite::fetchTable(const String & table_name, ContextPtr loca if (!columns) return StoragePtr{}; - auto storage = StorageSQLite::create( + auto storage = std::make_shared( StorageID(database_name, table_name), sqlite_db, database_path, diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 8b8d0a57cc7..0293288d9de 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -504,7 +504,7 @@ Pipe CacheDictionary::read(const Names & column_names, size } std::shared_ptr dictionary = shared_from_this(); - auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); auto result = coordinator->read(num_streams); return result; diff --git a/src/Dictionaries/DictionarySource.h b/src/Dictionaries/DictionarySource.h index 7809c958419..b54a8da6596 100644 --- a/src/Dictionaries/DictionarySource.h +++ b/src/Dictionaries/DictionarySource.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -13,18 +14,14 @@ namespace DB class DictionarySource; -class DictionarySourceCoordinator final : public shared_ptr_helper, public std::enable_shared_from_this +class DictionarySourceCoordinator final : public std::enable_shared_from_this + , private boost::noncopyable { - friend struct shared_ptr_helper; - public: - using ReadColumnsFunc = std::function; Pipe read(size_t num_streams); -private: - explicit DictionarySourceCoordinator( std::shared_ptr dictionary_, const Names & column_names, @@ -85,6 +82,8 @@ private: initialize(column_names); } +private: + friend class DictionarySource; bool getKeyColumnsNextRangeToRead(ColumnsWithTypeAndName & key_columns, ColumnsWithTypeAndName & data_columns); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index cb2419633bf..2fe9f09741b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -573,7 +573,7 @@ Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size, siz ColumnsWithTypeAndName key_columns = {ColumnWithTypeAndName(keys_column, std::make_shared(), dict_struct.id->name)}; std::shared_ptr dictionary = shared_from_this(); - auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size); + auto coordinator =std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); auto result = coordinator->read(num_streams); return result; diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 65d9b3e7d42..5a2586147b8 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -758,7 +758,7 @@ Pipe HashedArrayDictionary::read(const Names & column_names } std::shared_ptr dictionary = shared_from_this(); - auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); auto result = coordinator->read(num_streams); return result; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index ad1323c6ddf..7025922da12 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -745,7 +745,7 @@ Pipe HashedDictionary::read(const Names & column_na } std::shared_ptr dictionary = shared_from_this(); - auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns), max_block_size); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); auto result = coordinator->read(num_streams); return result; diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 0babecab5a5..28aab8ed982 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -873,7 +873,7 @@ Pipe IPAddressDictionary::read(const Names & column_names, size_t max_block_size } std::shared_ptr dictionary = shared_from_this(); - auto coordinator = DictionarySourceCoordinator::create(dictionary, column_names, std::move(key_columns_with_type), std::move(view_columns), max_block_size); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns_with_type), std::move(view_columns), max_block_size); auto result = coordinator->read(num_streams); return result; diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index e82fcd580e2..20230b1bd32 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -1005,7 +1005,7 @@ Pipe RangeHashedDictionary::read(const Names & column_names return result; }; - auto coordinator = DictionarySourceCoordinator::create( + auto coordinator = std::make_shared( dictionary, column_names, std::move(key_columns), diff --git a/src/IO/Archives/ZipArchiveReader.h b/src/IO/Archives/ZipArchiveReader.h index 7236b0b660c..262680317cd 100644 --- a/src/IO/Archives/ZipArchiveReader.h +++ b/src/IO/Archives/ZipArchiveReader.h @@ -3,8 +3,8 @@ #include #if USE_MINIZIP +#include #include -#include #include #include @@ -16,9 +16,16 @@ class ReadBufferFromFileBase; class SeekableReadBuffer; /// Implementation of IArchiveReader for reading zip archives. -class ZipArchiveReader : public shared_ptr_helper, public IArchiveReader +class ZipArchiveReader : public IArchiveReader, boost::noncopyable { public: + /// Constructs an archive's reader that will read from a file in the local filesystem. + explicit ZipArchiveReader(const String & path_to_archive_); + + /// Constructs an archive's reader that will read by making a read buffer by using + /// a specified function. + ZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + ~ZipArchiveReader() override; /// Returns true if there is a specified file in the archive. @@ -43,14 +50,6 @@ public: void setPassword(const String & password_) override; private: - /// Constructs an archive's reader that will read from a file in the local filesystem. - explicit ZipArchiveReader(const String & path_to_archive_); - - /// Constructs an archive's reader that will read by making a read buffer by using - /// a specified function. - ZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); - - friend struct shared_ptr_helper; class ReadBufferFromZipArchive; class FileEnumeratorImpl; class HandleHolder; diff --git a/src/IO/Archives/ZipArchiveWriter.h b/src/IO/Archives/ZipArchiveWriter.h index 58df4902434..6c0b8f426fd 100644 --- a/src/IO/Archives/ZipArchiveWriter.h +++ b/src/IO/Archives/ZipArchiveWriter.h @@ -3,8 +3,8 @@ #include #if USE_MINIZIP +#include #include -#include #include @@ -14,9 +14,15 @@ class WriteBuffer; class WriteBufferFromFileBase; /// Implementation of IArchiveWriter for writing zip archives. -class ZipArchiveWriter : public shared_ptr_helper, public IArchiveWriter +class ZipArchiveWriter : public IArchiveWriter, boost::noncopyable { public: + /// Constructs an archive that will be written as a file in the local filesystem. + explicit ZipArchiveWriter(const String & path_to_archive_); + + /// Constructs an archive that will be written by using a specified `archive_write_buffer_`. + ZipArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_); + /// Destructors finalizes writing the archive. ~ZipArchiveWriter() override; @@ -63,13 +69,6 @@ public: static void checkEncryptionIsEnabled(); private: - /// Constructs an archive that will be written as a file in the local filesystem. - explicit ZipArchiveWriter(const String & path_to_archive_); - - /// Constructs an archive that will be written by using a specified `archive_write_buffer_`. - ZipArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_); - - friend struct shared_ptr_helper; class WriteBufferFromZipArchive; class HandleHolder; using RawHandle = void *; diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 6ebab000a18..ea51d44399e 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -26,7 +26,7 @@ std::shared_ptr createArchiveReader( if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { #if USE_MINIZIP - return ZipArchiveReader::create(path_to_archive, archive_read_function, archive_size); + return std::make_shared(path_to_archive, archive_read_function, archive_size); #else throw Exception("minizip library is disabled", ErrorCodes::SUPPORT_IS_DISABLED); #endif diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 26cbde8c363..573e36861ed 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -26,7 +26,7 @@ std::shared_ptr createArchiveWriter( if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { #if USE_MINIZIP - return ZipArchiveWriter::create(path_to_archive, std::move(archive_write_buffer)); + return std::make_shared(path_to_archive, std::move(archive_write_buffer)); #else throw Exception("minizip library is disabled", ErrorCodes::SUPPORT_IS_DISABLED); #endif diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 84f18d66196..2a58acdc8ad 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3314,7 +3314,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() background_common_pool_size = config.getUInt64("profiles.default.background_common_pool_size"); /// With this executor we can execute more tasks than threads we have - shared->merge_mutate_executor = MergeMutateBackgroundExecutor::create + shared->merge_mutate_executor = std::make_shared ( "MergeMutate", /*max_threads_count*/background_pool_size, @@ -3324,7 +3324,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() LOG_INFO(shared->log, "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}", background_pool_size, background_pool_size * background_merges_mutations_concurrency_ratio); - shared->moves_executor = OrdinaryBackgroundExecutor::create + shared->moves_executor = std::make_shared ( "Move", background_move_pool_size, @@ -3333,7 +3333,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() ); LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size); - shared->fetch_executor = OrdinaryBackgroundExecutor::create + shared->fetch_executor = std::make_shared ( "Fetch", background_fetches_pool_size, @@ -3342,7 +3342,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() ); LOG_INFO(shared->log, "Initialized background executor for fetches with num_threads={}, num_tasks={}", background_fetches_pool_size, background_fetches_pool_size); - shared->common_executor = OrdinaryBackgroundExecutor::create + shared->common_executor = std::make_shared ( "Common", background_common_pool_size, diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index efa693564c3..5af4d4941e8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -92,7 +92,7 @@ TemporaryTableHolder::TemporaryTableHolder( context_, [&](const StorageID & table_id) { - auto storage = StorageMemory::create(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}, String{}); + auto storage = std::make_shared(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}, String{}); if (create_for_global_subquery) storage->delayReadForGlobalSubqueries(); @@ -250,7 +250,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( /// It's definitely not the best place for this logic, but behaviour must be consistent with DatabaseMaterializedMySQL::tryGetTable(...) if (!context_->isInternalQuery() && db_and_table.first->getEngineName() == "MaterializedMySQL") { - db_and_table.second = std::make_shared(std::move(db_and_table.second), db_and_table.first.get()); + db_and_table.second = StorageMaterializedMySQL::create(std::move(db_and_table.second), db_and_table.first.get()); } #endif return db_and_table; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 74070252ebb..344c5c179db 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -32,7 +32,7 @@ MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, for (const auto & type : data_types) serializations.emplace_back(type->getDefaultSerialization()); - packet_endpoint = MySQLProtocol::PacketEndpoint::create(out, *sequence_id); + packet_endpoint = std::make_shared(out, *sequence_id); } void MySQLOutputFormat::setContext(ContextPtr context_) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 1f127603870..8abbb63d418 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -420,7 +420,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// but it will contain single block (that is INSERT-ed into main table). /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); - local_context->addViewSource(StorageValues::create( + local_context->addViewSource(std::make_shared( views_data.source_storage_id, views_data.source_metadata_snapshot->getColumns(), std::move(block), diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 2ec7c5752c4..b4d94d8a78c 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -94,7 +94,7 @@ void MySQLHandler::run() in = std::make_shared(socket()); out = std::make_shared(socket()); - packet_endpoint = MySQLProtocol::PacketEndpoint::create(*in, *out, sequence_id); + packet_endpoint = std::make_shared(*in, *out, sequence_id); try { @@ -403,7 +403,7 @@ void MySQLHandlerSSL::finishHandshakeSSL( in = std::make_shared(*ss); out = std::make_shared(*ss); sequence_id = 2; - packet_endpoint = MySQLProtocol::PacketEndpoint::create(*in, *out, sequence_id); + packet_endpoint = std::make_shared(*in, *out, sequence_id); packet_endpoint->receivePacket(packet); /// Reading HandshakeResponse from secure socket. } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c50425343b2..4bf77792559 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -808,7 +808,7 @@ void registerStorageFileLog(StorageFactory & factory) auto path = path_ast->as().value.safeGet(); auto format = format_ast->as().value.safeGet(); - return StorageFileLog::create( + return std::make_shared( args.table_id, args.getContext(), args.columns, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ded97ecbd8c..2ec1f592ebd 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -8,8 +9,6 @@ #include #include -#include - #include #include #include @@ -26,11 +25,19 @@ namespace ErrorCodes class FileLogDirectoryWatcher; -class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext +class StorageFileLog final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; - public: + StorageFileLog( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & path_, + const String & metadata_base_path_, + const String & format_name_, + std::unique_ptr settings, + const String & comment, + bool attach); using Files = std::vector; @@ -125,18 +132,6 @@ public: const auto & getFileLogSettings() const { return filelog_settings; } -protected: - StorageFileLog( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - const String & path_, - const String & metadata_base_path_, - const String & format_name_, - std::unique_ptr settings, - const String & comment, - bool attach); - private: std::unique_ptr filelog_settings; diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 7340ce3fa71..2f0d458e94c 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -4,11 +4,11 @@ #if USE_HDFS +#include #include #include #include #include -#include namespace DB { @@ -16,10 +16,21 @@ namespace DB * This class represents table engine for external hdfs files. * Read method is supported for now. */ -class StorageHDFS final : public shared_ptr_helper, public IStorage, WithContext +class StorageHDFS final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageHDFS( + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + const String & compression_method_ = "", + bool distributed_processing_ = false, + ASTPtr partition_by = nullptr); + String getName() const override { return "HDFS"; } Pipe read( @@ -57,17 +68,6 @@ public: protected: friend class HDFSSource; - StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context_, - const String & compression_method_ = "", - bool distributed_processing_ = false, - ASTPtr partition_by = nullptr); private: std::vector uris; diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 953311de056..5d7bd070d53 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -7,8 +7,7 @@ #include #include -#include - +#include #include #include #include @@ -18,10 +17,18 @@ namespace DB class Context; -class StorageHDFSCluster : public shared_ptr_helper, public IStorage +class StorageHDFSCluster : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageHDFSCluster( + String cluster_name_, + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & compression_method_); + std::string getName() const override { return "HDFSCluster"; } Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, @@ -32,16 +39,6 @@ public: NamesAndTypesList getVirtuals() const override; -protected: - StorageHDFSCluster( - String cluster_name_, - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & compression_method_); - private: String cluster_name; String uri; diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 978c6685814..ebd13919512 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -7,8 +7,8 @@ #include #include +#include #include -#include #include #include #include @@ -23,10 +23,23 @@ class HiveSettings; * This class represents table engine for external hdfs files. * Read method is supported for now. */ -class StorageHive final : public shared_ptr_helper, public IStorage, WithContext +class StorageHive final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; public: + friend class StorageHiveSource; + + StorageHive( + const String & hive_metastore_url_, + const String & hive_database_, + const String & hive_table_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_, + const ASTPtr & partition_by_ast_, + std::unique_ptr storage_settings_, + ContextPtr context_); + String getName() const override { return "Hive"; } bool supportsIndexForIn() const override { return true; } @@ -56,20 +69,6 @@ public: std::optional totalRows(const Settings & settings) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; -protected: - friend class StorageHiveSource; - StorageHive( - const String & hive_metastore_url_, - const String & hive_database_, - const String & hive_table_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment_, - const ASTPtr & partition_by_ast_, - std::unique_ptr storage_settings_, - ContextPtr context_); - private: using FileFormat = IHiveFile::FileFormat; using FileInfo = HiveMetastoreClient::FileInfo; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index f85c004ab4d..c985cf49ae6 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -842,7 +842,7 @@ void registerStorageKafka(StorageFactory & factory) throw Exception("kafka_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - return StorageKafka::create(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); }; factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 707db7a798e..0cdab0a72f3 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -7,7 +8,6 @@ #include #include -#include #include #include @@ -28,12 +28,18 @@ struct StorageKafkaInterceptors; /** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, * or as a basic building block for creating pipelines with a continuous insertion / ETL. */ -class StorageKafka final : public shared_ptr_helper, public IStorage, WithContext +class StorageKafka final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; friend struct StorageKafkaInterceptors; public: + StorageKafka( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr kafka_settings_, + const String & collection_name_); + std::string getName() const override { return "Kafka"; } bool noPushingToViews() const override { return true; } @@ -66,13 +72,6 @@ public: NamesAndTypesList getVirtuals() const override; Names getVirtualColumnNames() const; HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } -protected: - StorageKafka( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - std::unique_ptr kafka_settings_, - const String & collection_name_); private: // Configuration and state @@ -117,7 +116,7 @@ private: std::list> thread_statuses; /// Handle error mode - HandleKafkaErrorMode handle_error_mode; + // HandleKafkaErrorMode handle_error_mode; // NOLINT -- actually not used, needs checing SettingsChanges createSettingsAdjustments(); ConsumerBufferPtr createReadBuffer(size_t consumer_number); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index d11c5b64c57..edd062aa592 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -625,7 +625,7 @@ void registerStorageLiveView(StorageFactory & factory) "Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); - return StorageLiveView::create(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment); + return std::make_shared(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment); }); } diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 2fb2ec509fa..d9d3032df23 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -11,7 +11,7 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include +#include #include #include @@ -49,14 +49,20 @@ class Pipe; using Pipes = std::vector; -class StorageLiveView final : public shared_ptr_helper, public IStorage, WithContext +class StorageLiveView final : public IStorage, WithContext, boost::noncopyable { -friend struct shared_ptr_helper; friend class LiveViewSource; friend class LiveViewEventsSource; friend class LiveViewSink; public: + StorageLiveView( + const StorageID & table_id_, + ContextPtr context_, + const ASTCreateQuery & query, + const ColumnsDescription & columns, + const String & comment); + ~StorageLiveView() override; String getName() const override { return "LiveView"; } bool isView() const override { return true; } @@ -227,13 +233,6 @@ private: /// Must be called with mutex locked void scheduleNextPeriodicRefresh(); - - StorageLiveView( - const StorageID & table_id_, - ContextPtr context_, - const ASTCreateQuery & query, - const ColumnsDescription & columns, - const String & comment); }; } diff --git a/src/Storages/MergeTree/IExecutableTask.h b/src/Storages/MergeTree/IExecutableTask.h index 00ae60ffe17..d665a19d395 100644 --- a/src/Storages/MergeTree/IExecutableTask.h +++ b/src/Storages/MergeTree/IExecutableTask.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include namespace DB @@ -42,10 +42,9 @@ using ExecutableTaskPtr = std::shared_ptr; /** * Some background operations won't represent a coroutines (don't want to be executed step-by-step). For this we have this wrapper. */ -class ExecutableLambdaAdapter : public shared_ptr_helper, public IExecutableTask +class ExecutableLambdaAdapter : public IExecutableTask, boost::noncopyable { public: - template explicit ExecutableLambdaAdapter( Job && job_to_execute_, diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.h b/src/Storages/MergeTree/MergeFromLogEntryTask.h index 250086e0f7d..7b10b632282 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.h +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.h @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -14,7 +15,7 @@ namespace DB { -class MergeFromLogEntryTask : public shared_ptr_helper, public ReplicatedMergeMutateTaskBase +class MergeFromLogEntryTask : public ReplicatedMergeMutateTaskBase, boost::noncopyable { public: template diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 8976e4f656b..d02494ed4ca 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -10,8 +10,8 @@ #include #include +#include -#include #include #include #include @@ -156,7 +156,7 @@ private: * So, when a Storage want to shutdown, it must wait until all its background operaions are finished. */ template -class MergeTreeBackgroundExecutor final : public shared_ptr_helper> +class MergeTreeBackgroundExecutor final : boost::noncopyable { public: MergeTreeBackgroundExecutor( diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d84fb9d30d3..6d1a034a0ed 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6185,7 +6185,7 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) if (moving_tagger->parts_to_move.empty()) return false; - assignee.scheduleMoveTask(ExecutableLambdaAdapter::create( + assignee.scheduleMoveTask(std::make_shared( [this, moving_tagger] () mutable { return moveParts(moving_tagger); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 952a60f09dd..cff4b34e7ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -223,7 +223,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( if (query_info.projection->merge_tree_normal_select_result_ptr) { auto storage_from_base_parts_of_projection - = StorageFromMergeTreeDataPart::create(data, query_info.projection->merge_tree_normal_select_result_ptr); + = std::make_shared(data, query_info.projection->merge_tree_normal_select_result_ptr); auto interpreter = InterpreterSelectQuery( query_info.query, context, diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index fdcee9fb8eb..ff08ba0c062 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -88,7 +88,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() } new_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, storage.format_version); - commands = MutationCommands::create(storage.queue.getMutationCommands(source_part, new_part_info.mutation)); + commands = std::make_shared(storage.queue.getMutationCommands(source_part, new_part_info.mutation)); /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. /// Can throw an exception. diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.h b/src/Storages/MergeTree/MutateFromLogEntryTask.h index 6c13401b290..db453c4d927 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.h +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -12,7 +11,7 @@ namespace DB { -class MutateFromLogEntryTask : public shared_ptr_helper, public ReplicatedMergeMutateTaskBase +class MutateFromLogEntryTask : public ReplicatedMergeMutateTaskBase, boost::noncopyable { public: template @@ -22,6 +21,7 @@ public: Callback && task_result_callback_) : ReplicatedMergeMutateTaskBase(&Poco::Logger::get("MutateFromLogEntryTask"), storage_, selected_entry_, task_result_callback_) {} + UInt64 getPriority() override { return priority; } private: diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 429a66a6af2..554afd6e494 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1406,7 +1406,7 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); ctx->source_part = ctx->future_part->parts[0]; - auto storage_from_source_part = StorageFromMergeTreeDataPart::create(ctx->source_part); + auto storage_from_source_part = std::make_shared(ctx->source_part); auto context_for_reading = Context::createCopy(ctx->context); context_for_reading->setSetting("max_streams_to_max_threads_ratio", 1); diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 854f070d0e0..ec055b3fb85 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -9,17 +10,31 @@ #include #include -#include - namespace DB { /// A Storage that allows reading from a single MergeTree data part. -class StorageFromMergeTreeDataPart final : public shared_ptr_helper, public IStorage +class StorageFromMergeTreeDataPart final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + /// Used in part mutation. + explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) + : IStorage(getIDFromPart(part_)) + , parts({part_}) + , storage(part_->storage) + , partition_id(part_->info.partition_id) + { + setInMemoryMetadata(storage.getInMemoryMetadata()); + } + + /// Used in queries with projection. + StorageFromMergeTreeDataPart(const MergeTreeData & storage_, MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr_) + : IStorage(storage_.getStorageID()), storage(storage_), analysis_result_ptr(analysis_result_ptr_) + { + setInMemoryMetadata(storage.getInMemoryMetadata()); + } + String getName() const override { return "FromMergeTreeDataPart"; } Pipe read( @@ -77,24 +92,6 @@ public: return parts.front()->storage.getSettings()->materialize_ttl_recalculate_only; } -protected: - /// Used in part mutation. - explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) - : IStorage(getIDFromPart(part_)) - , parts({part_}) - , storage(part_->storage) - , partition_id(part_->info.partition_id) - { - setInMemoryMetadata(storage.getInMemoryMetadata()); - } - - /// Used in queries with projection. - StorageFromMergeTreeDataPart(const MergeTreeData & storage_, MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr_) - : IStorage(storage_.getStorageID()), storage(storage_), analysis_result_ptr(analysis_result_ptr_) - { - setInMemoryMetadata(storage.getInMemoryMetadata()); - } - private: MergeTreeData::DataPartsVector parts; const MergeTreeData & storage; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b2c41c7df68..6d8c3b313d4 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -672,7 +672,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); if (replicated) - return StorageReplicatedMergeTree::create( + return std::make_shared( zookeeper_path, replica_name, args.attach, @@ -686,7 +686,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) args.has_force_restore_data_flag, renaming_restrictions); else - return StorageMergeTree::create( + return std::make_shared( args.table_id, args.relative_data_path, metadata, diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index d2895215ebe..b89692869fd 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -64,7 +64,7 @@ TEST(Executor, RemoveTasks) const size_t tasks_kinds = 25; const size_t batch = 100; - auto executor = DB::OrdinaryBackgroundExecutor::create + auto executor = std::make_shared ( "GTest", tasks_kinds, @@ -105,7 +105,7 @@ TEST(Executor, RemoveTasksStress) const size_t schedulers_count = 5; const size_t removers_count = 5; - auto executor = DB::OrdinaryBackgroundExecutor::create + auto executor = std::make_shared ( "GTest", tasks_kinds, diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index cf1116f547a..1512608a926 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include #include @@ -70,7 +70,7 @@ struct MutationCommand }; /// Multiple mutation commands, possible from different ALTER queries -class MutationCommands : public shared_ptr_helper, public std::vector +class MutationCommands : public std::vector { public: std::shared_ptr ast() const; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 29969bc5aa4..a440584d353 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -147,7 +147,7 @@ StoragePtr StorageMaterializedPostgreSQL::createTemporary() const } auto new_context = Context::createCopy(context); - return StorageMaterializedPostgreSQL::create(tmp_table_id, new_context, "temporary", table_id.table_name); + return std::make_shared(tmp_table_id, new_context, "temporary", table_id.table_name); } @@ -569,7 +569,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) if (has_settings) postgresql_replication_settings->loadFromQuery(*args.storage_def); - return StorageMaterializedPostgreSQL::create( + return std::make_shared( args.table_id, args.attach, configuration.database, configuration.table, connection_info, metadata, args.getContext(), std::move(postgresql_replication_settings)); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index e41eb8ee98f..6ea0165c200 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -6,13 +6,13 @@ #include "PostgreSQLReplicationHandler.h" #include "MaterializedPostgreSQLSettings.h" +#include #include #include #include #include #include #include -#include #include @@ -61,10 +61,8 @@ namespace DB * **/ -class StorageMaterializedPostgreSQL final : public shared_ptr_helper, public IStorage, WithContext +class StorageMaterializedPostgreSQL final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; - public: StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_, const String & postgres_database_name, const String & postgres_table_name); @@ -72,6 +70,16 @@ public: StorageMaterializedPostgreSQL(StoragePtr nested_storage_, ContextPtr context_, const String & postgres_database_name, const String & postgres_table_name); + StorageMaterializedPostgreSQL( + const StorageID & table_id_, + bool is_attach_, + const String & remote_database_name, + const String & remote_table_name, + const postgres::ConnectionInfo & connection_info, + const StorageInMemoryMetadata & storage_metadata, + ContextPtr context_, + std::unique_ptr replication_settings); + String getName() const override { return "MaterializedPostgreSQL"; } void shutdown() override; @@ -122,17 +130,6 @@ public: bool supportsFinal() const override { return true; } -protected: - StorageMaterializedPostgreSQL( - const StorageID & table_id_, - bool is_attach_, - const String & remote_database_name, - const String & remote_table_name, - const postgres::ConnectionInfo & connection_info, - const StorageInMemoryMetadata & storage_metadata, - ContextPtr context_, - std::unique_ptr replication_settings); - private: static std::shared_ptr getMaterializedColumnsDeclaration( String name, String type, UInt64 default_value); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index b4c6d73d32f..678416ddd42 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1164,7 +1164,7 @@ void registerStorageRabbitMQ(StorageFactory & factory) if (!rabbitmq_settings->rabbitmq_format.changed) throw Exception("You must specify `rabbitmq_format` setting", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageRabbitMQ::create(args.table_id, args.getContext(), args.columns, std::move(rabbitmq_settings), args.attach); + return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(rabbitmq_settings), args.attach); }; factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 394845bbc2f..868020613e2 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include #include -#include #include #include #include @@ -18,11 +18,16 @@ namespace DB { -class StorageRabbitMQ final: public shared_ptr_helper, public IStorage, WithContext +class StorageRabbitMQ final: public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; - public: + StorageRabbitMQ( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + std::unique_ptr rabbitmq_settings_, + bool is_attach_); + std::string getName() const override { return "RabbitMQ"; } bool noPushingToViews() const override { return true; } @@ -71,14 +76,6 @@ public: void incrementReader(); void decrementReader(); -protected: - StorageRabbitMQ( - const StorageID & table_id_, - ContextPtr context_, - const ColumnsDescription & columns_, - std::unique_ptr rabbitmq_settings_, - bool is_attach_); - private: ContextMutablePtr rabbitmq_context; std::unique_ptr rabbitmq_settings; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index e2b3995a5aa..64746000363 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -508,7 +508,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); } - return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0]); + return std::make_shared(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0]); } std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 52a08cbefd4..12882d6b70f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -19,11 +19,17 @@ namespace DB class Context; -class StorageEmbeddedRocksDB final : public shared_ptr_helper, public IStorage, WithContext +class StorageEmbeddedRocksDB final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; friend class EmbeddedRocksDBSink; public: + StorageEmbeddedRocksDB(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + ContextPtr context_, + const String & primary_key_); + std::string getName() const override { return "EmbeddedRocksDB"; } Pipe read( @@ -53,14 +59,6 @@ public: std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; const String & getPrimaryKey() const { return primary_key; } -protected: - StorageEmbeddedRocksDB(const StorageID & table_id_, - const String & relative_data_path_, - const StorageInMemoryMetadata & metadata, - bool attach, - ContextPtr context_, - const String & primary_key_); - private: const String primary_key; using RocksDBPtr = std::unique_ptr; diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.h b/src/Storages/RocksDB/StorageSystemRocksDB.h index 8394fc3dd3a..85773367c2c 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.h +++ b/src/Storages/RocksDB/StorageSystemRocksDB.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements the `rocksdb` system table, which expose various rocksdb metrics. */ -class StorageSystemRocksDB final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemRocksDB final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemRocksDB"; } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 1aea6386e75..e551abe0065 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1145,7 +1145,7 @@ void registerStorageBuffer(StorageFactory & factory) destination_id.table_name = destination_table; } - return StorageBuffer::create( + return std::make_shared( args.table_id, args.columns, args.constraints, diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index f589560008a..5967ab3f35c 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include #include -#include #include @@ -41,9 +41,8 @@ namespace DB * When you destroy a Buffer table, all remaining data is flushed to the subordinate table. * The data in the buffer is not replicated, not logged to disk, not indexed. With a rough restart of the server, the data is lost. */ -class StorageBuffer final : public shared_ptr_helper, public IStorage, WithContext +class StorageBuffer final : public IStorage, WithContext, boost::noncopyable { -friend struct shared_ptr_helper; friend class BufferSource; friend class BufferSink; @@ -55,6 +54,22 @@ public: size_t bytes = 0; /// The number of (uncompressed) bytes in the block. }; + /** num_shards - the level of internal parallelism (the number of independent buffers) + * The buffer is flushed if all minimum thresholds or at least one of the maximum thresholds are exceeded. + */ + StorageBuffer( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + size_t num_shards_, + const Thresholds & min_thresholds_, + const Thresholds & max_thresholds_, + const Thresholds & flush_thresholds_, + const StorageID & destination_id, + bool allow_materialized_); + std::string getName() const override { return "Buffer"; } QueryProcessingStage::Enum @@ -166,23 +181,6 @@ private: BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder flush_handle; - -protected: - /** num_shards - the level of internal parallelism (the number of independent buffers) - * The buffer is flushed if all minimum thresholds or at least one of the maximum thresholds are exceeded. - */ - StorageBuffer( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context_, - size_t num_shards_, - const Thresholds & min_thresholds_, - const Thresholds & max_thresholds_, - const Thresholds & flush_thresholds_, - const StorageID & destination_id, - bool allow_materialized_); }; } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index b3d0c1f17de..fda6da6c1ff 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -318,7 +318,7 @@ void registerStorageDictionary(StorageFactory & factory) /// Create dictionary storage that owns underlying dictionary auto abstract_dictionary_configuration = getDictionaryConfigurationFromAST(args.query, local_context, dictionary_id.database_name); - auto result_storage = StorageDictionary::create(dictionary_id, abstract_dictionary_configuration, local_context); + auto result_storage = std::make_shared(dictionary_id, abstract_dictionary_configuration, local_context); bool lazy_load = local_context->getConfigRef().getBool("dictionaries_lazy_load", true); if (!args.attach && !lazy_load) @@ -348,7 +348,7 @@ void registerStorageDictionary(StorageFactory & factory) checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure); } - return StorageDictionary::create( + return std::make_shared( args.table_id, dictionary_name, args.columns, args.comment, StorageDictionary::Location::Custom, local_context); } }); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index bf9e6853233..4b7656f4a52 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -1,8 +1,8 @@ #pragma once #include -#include +#include #include #include @@ -12,11 +12,50 @@ namespace DB struct DictionaryStructure; class TableFunctionDictionary; -class StorageDictionary final : public shared_ptr_helper, public IStorage, public WithContext +class StorageDictionary final : public IStorage, public WithContext, boost::noncopyable { - friend struct shared_ptr_helper; - friend class TableFunctionDictionary; +friend class TableFunctionDictionary; + public: + /// Specifies where the table is located relative to the dictionary. + enum class Location + { + /// Table was created automatically as an element of a database with the Dictionary engine. + DictionaryDatabase, + + /// Table was created automatically along with a dictionary + /// and has the same database and name as the dictionary. + /// It provides table-like access to the dictionary. + /// User cannot drop that table. + SameDatabaseAndNameAsDictionary, + + /// Table was created explicitly by a statement like + /// CREATE TABLE ... ENGINE=Dictionary + /// User chose the table's database and name and can drop that table. + Custom, + }; + + StorageDictionary( + const StorageID & table_id_, + const String & dictionary_name_, + const ColumnsDescription & columns_, + const String & comment, + Location location_, + ContextPtr context_); + + StorageDictionary( + const StorageID & table_id_, + const String & dictionary_name_, + const DictionaryStructure & dictionary_structure, + const String & comment, + Location location_, + ContextPtr context_); + + StorageDictionary( + const StorageID & table_id_, + LoadablesConfigurationPtr dictionary_configuration_, + ContextPtr context_); + std::string getName() const override { return "Dictionary"; } ~StorageDictionary() override; @@ -51,24 +90,6 @@ public: String getDictionaryName() const { return dictionary_name; } - /// Specifies where the table is located relative to the dictionary. - enum class Location - { - /// Table was created automatically as an element of a database with the Dictionary engine. - DictionaryDatabase, - - /// Table was created automatically along with a dictionary - /// and has the same database and name as the dictionary. - /// It provides table-like access to the dictionary. - /// User cannot drop that table. - SameDatabaseAndNameAsDictionary, - - /// Table was created explicitly by a statement like - /// CREATE TABLE ... ENGINE=Dictionary - /// User chose the table's database and name and can drop that table. - Custom, - }; - private: String dictionary_name; const Location location; @@ -80,27 +101,6 @@ private: scope_guard remove_repository_callback; void removeDictionaryConfigurationFromRepository(); - - StorageDictionary( - const StorageID & table_id_, - const String & dictionary_name_, - const ColumnsDescription & columns_, - const String & comment, - Location location_, - ContextPtr context_); - - StorageDictionary( - const StorageID & table_id_, - const String & dictionary_name_, - const DictionaryStructure & dictionary_structure, - const String & comment, - Location location_, - ContextPtr context_); - - StorageDictionary( - const StorageID & table_id_, - LoadablesConfigurationPtr dictionary_configuration_, - ContextPtr context_); }; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ae67a67af7f..333e35ff842 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1498,7 +1498,7 @@ void registerStorageDistributed(StorageFactory & factory) if (!distributed_settings.monitor_max_sleep_time_ms.changed) distributed_settings.monitor_max_sleep_time_ms = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms); - return StorageDistributed::create( + return std::make_shared( args.table_id, args.columns, args.constraints, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index dc8635a7550..be4ff190c86 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -36,14 +35,44 @@ using ExpressionActionsPtr = std::shared_ptr; * You can pass one address, not several. * In this case, the table can be considered remote, rather than distributed. */ -class StorageDistributed final : public shared_ptr_helper, public IStorage, WithContext +class StorageDistributed final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; friend class DistributedSink; friend class StorageDistributedDirectoryMonitor; friend class StorageSystemDistributionQueue; public: + StorageDistributed( + const StorageID & id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + const String & remote_database_, + const String & remote_table_, + const String & cluster_name_, + ContextPtr context_, + const ASTPtr & sharding_key_, + const String & storage_policy_name_, + const String & relative_data_path_, + const DistributedSettings & distributed_settings_, + bool attach_, + ClusterPtr owned_cluster_ = {}, + ASTPtr remote_table_function_ptr_ = {}); + + StorageDistributed( + const StorageID & id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ASTPtr remote_table_function_ptr_, + const String & cluster_name_, + ContextPtr context_, + const ASTPtr & sharding_key_, + const String & storage_policy_name_, + const String & relative_data_path_, + const DistributedSettings & distributed_settings_, + bool attach, + ClusterPtr owned_cluster_ = {}); + ~StorageDistributed() override; std::string getName() const override { return "Distributed"; } @@ -137,37 +166,6 @@ public: size_t getShardCount() const; private: - StorageDistributed( - const StorageID & id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - const String & remote_database_, - const String & remote_table_, - const String & cluster_name_, - ContextPtr context_, - const ASTPtr & sharding_key_, - const String & storage_policy_name_, - const String & relative_data_path_, - const DistributedSettings & distributed_settings_, - bool attach_, - ClusterPtr owned_cluster_ = {}, - ASTPtr remote_table_function_ptr_ = {}); - - StorageDistributed( - const StorageID & id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ASTPtr remote_table_function_ptr_, - const String & cluster_name_, - ContextPtr context_, - const ASTPtr & sharding_key_, - const String & storage_policy_name_, - const String & relative_data_path_, - const DistributedSettings & distributed_settings_, - bool attach, - ClusterPtr owned_cluster_ = {}); - void renameOnDisk(const String & new_path_to_table_data); const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 577b4cab585..4a4317c9aab 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -217,7 +217,7 @@ void registerStorageExecutable(StorageFactory & factory) settings.loadFromQuery(*args.storage_def); auto global_context = args.getContext()->getGlobalContext(); - return StorageExecutable::create(args.table_id, format, settings, input_queries, columns, constraints); + return std::make_shared(args.table_id, format, settings, input_queries, columns, constraints); }; StorageFactory::StorageFeatures storage_features; diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 8465c88c793..9e5eee56530 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include #include #include #include @@ -15,11 +15,16 @@ namespace DB * Executable storage that will start process for read. * ExecutablePool storage maintain pool of processes and take process from pool for read. */ -class StorageExecutable final : public shared_ptr_helper, public IStorage +class StorageExecutable final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; - public: + StorageExecutable( + const StorageID & table_id, + const String & format, + const ExecutableSettings & settings, + const std::vector & input_queries, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints); String getName() const override { @@ -38,16 +43,6 @@ public: size_t max_block_size, unsigned threads) override; -protected: - - StorageExecutable( - const StorageID & table_id, - const String & format, - const ExecutableSettings & settings, - const std::vector & input_queries, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints); - private: ExecutableSettings settings; std::vector input_queries; diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 2f3a28f8ac9..24d39bac589 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -97,7 +97,7 @@ StorageExternalDistributed::StorageExternalDistributed( context->getSettingsRef().postgresql_connection_pool_size, context->getSettingsRef().postgresql_connection_pool_wait_timeout); - shard = StoragePostgreSQL::create(table_id_, std::move(pool), configuration.table, columns_, constraints_, String{}); + shard = std::make_shared(table_id_, std::move(pool), configuration.table, columns_, constraints_, String{}); break; } #endif @@ -257,7 +257,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) auto format_settings = StorageURL::getFormatSettingsFromArgs(args); - return StorageExternalDistributed::create( + return std::make_shared( cluster_description, args.table_id, configuration.format, @@ -304,7 +304,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) } - return StorageExternalDistributed::create( + return std::make_shared( args.table_id, table_engine, cluster_description, diff --git a/src/Storages/StorageExternalDistributed.h b/src/Storages/StorageExternalDistributed.h index 57767db10b0..da5c7cec2b1 100644 --- a/src/Storages/StorageExternalDistributed.h +++ b/src/Storages/StorageExternalDistributed.h @@ -2,7 +2,7 @@ #include "config_core.h" -#include +#include #include @@ -16,10 +16,8 @@ struct ExternalDataSourceConfiguration; /// A query to external database is passed to one replica on each shard, the result is united. /// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue. /// Similar approach is used for URL storage. -class StorageExternalDistributed final : public shared_ptr_helper, public DB::IStorage +class StorageExternalDistributed final : public DB::IStorage, boost::noncopyable { - friend struct shared_ptr_helper; - public: enum class ExternalStorageEngine { @@ -28,18 +26,6 @@ public: URL }; - std::string getName() const override { return "ExternalDistributed"; } - - Pipe read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -protected: StorageExternalDistributed( const StorageID & table_id_, ExternalStorageEngine table_engine, @@ -60,6 +46,17 @@ protected: const ConstraintsDescription & constraints, ContextPtr context); + std::string getName() const override { return "ExternalDistributed"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + private: using Shards = std::unordered_set; Shards shards; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 1d8bcdc60ed..c460b8a4c67 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1091,7 +1091,7 @@ void registerStorageFile(StorageFactory & factory) } if (engine_args_ast.size() == 1) /// Table in database - return StorageFile::create(factory_args.relative_data_path, storage_args); + return std::make_shared(factory_args.relative_data_path, storage_args); /// Will use FD if engine_args[1] is int literal or identifier with std* name int source_fd = -1; @@ -1131,9 +1131,9 @@ void registerStorageFile(StorageFactory & factory) storage_args.compression_method = "auto"; if (0 <= source_fd) /// File descriptor - return StorageFile::create(source_fd, storage_args); + return std::make_shared(source_fd, storage_args); else /// User's file - return StorageFile::create(source_path, factory_args.getContext()->getUserFilesPath(), storage_args); + return std::make_shared(source_path, factory_args.getContext()->getUserFilesPath(), storage_args); }, storage_features); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 7b29fa04c4d..5c25a917e43 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -3,21 +3,42 @@ #include #include +#include #include #include -#include namespace DB { -class StorageFile final : public shared_ptr_helper, public IStorage +class StorageFile final : public IStorage { -friend struct shared_ptr_helper; -friend class PartitionedStorageFileSink; +friend class partitionedstoragefilesink; public: + struct CommonArguments : public WithContext + { + StorageID table_id; + std::string format_name; + std::optional format_settings; + std::string compression_method; + const ColumnsDescription & columns; + const ConstraintsDescription & constraints; + const String & comment; + }; + + /// From file descriptor + StorageFile(int table_fd_, CommonArguments args); + + /// From user's file + StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args); + + /// From table in database + StorageFile(const std::string & relative_table_dir_path, CommonArguments args); + + explicit StorageFile(CommonArguments args); + std::string getName() const override { return "File"; } Pipe read( @@ -45,17 +66,6 @@ public: bool storesDataOnDisk() const override; Strings getDataPaths() const override; - struct CommonArguments : public WithContext - { - StorageID table_id; - std::string format_name; - std::optional format_settings; - std::string compression_method; - const ColumnsDescription & columns; - const ConstraintsDescription & constraints; - const String & comment; - }; - NamesAndTypesList getVirtuals() const override; static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read); @@ -81,18 +91,7 @@ protected: friend class StorageFileSource; friend class StorageFileSink; - /// From file descriptor - StorageFile(int table_fd_, CommonArguments args); - - /// From user's file - StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args); - - /// From table in database - StorageFile(const std::string & relative_table_dir_path, CommonArguments args); - private: - explicit StorageFile(CommonArguments args); - void setStorageMetadata(CommonArguments args); std::string format_name; diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index aa7b17191b6..ae452bbf146 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -480,7 +480,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) if (engine_args.size() == 3) max_array_length = engine_args[2]->as().value.safeGet(); - return StorageGenerateRandom::create(args.table_id, args.columns, args.comment, max_array_length, max_string_length, random_seed); + return std::make_shared(args.table_id, args.columns, args.comment, max_array_length, max_string_length, random_seed); }); } diff --git a/src/Storages/StorageGenerateRandom.h b/src/Storages/StorageGenerateRandom.h index ca12d9c2841..906f3b99801 100644 --- a/src/Storages/StorageGenerateRandom.h +++ b/src/Storages/StorageGenerateRandom.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -9,10 +9,17 @@ namespace DB { /* Generates random data for given schema. */ -class StorageGenerateRandom final : public shared_ptr_helper, public IStorage +class StorageGenerateRandom final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageGenerateRandom( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const String & comment, + UInt64 max_array_length, + UInt64 max_string_length, + std::optional random_seed); + std::string getName() const override { return "GenerateRandom"; } Pipe read( @@ -29,15 +36,6 @@ private: UInt64 max_array_length = 10; UInt64 max_string_length = 10; UInt64 random_seed = 0; - -protected: - StorageGenerateRandom( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const String & comment, - UInt64 max_array_length, - UInt64 max_string_length, - std::optional random_seed); }; } diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index 4c44213a06b..405cb092c00 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -1,18 +1,19 @@ #pragma once -#include -#include +#include #include +#include namespace DB { /** Internal temporary storage for table function input(...) */ -class StorageInput final : public shared_ptr_helper, public IStorage +class StorageInput final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageInput(const StorageID & table_id, const ColumnsDescription & columns_); + String getName() const override { return "Input"; } /// A table will read from this stream. @@ -29,8 +30,5 @@ public: private: Pipe pipe; - -protected: - StorageInput(const StorageID & table_id, const ColumnsDescription & columns_); }; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index ecd182457e2..4781d3fe3ed 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -337,7 +337,7 @@ void registerStorageJoin(StorageFactory & factory) key_names.push_back(*opt_key); } - return StorageJoin::create( + return std::make_shared( disk, args.relative_data_path, args.table_id, diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index ea71ff4be8f..968b27e4c25 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -23,10 +22,24 @@ using HashJoinPtr = std::shared_ptr; * * When using, JOIN must be of the appropriate type (ANY|ALL LEFT|INNER ...). */ -class StorageJoin final : public shared_ptr_helper, public StorageSetOrJoinBase +class StorageJoin final : public StorageSetOrJoinBase, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageJoin( + DiskPtr disk_, + const String & relative_path_, + const StorageID & table_id_, + const Names & key_names_, + bool use_nulls_, + SizeLimits limits_, + ASTTableJoin::Kind kind_, + ASTTableJoin::Strictness strictness_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + bool overwrite, + bool persistent_); + String getName() const override { return "Join"; } void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; @@ -98,22 +111,6 @@ private: void finishInsert() override {} size_t getSize(ContextPtr context) const override; RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const; - -protected: - StorageJoin( - DiskPtr disk_, - const String & relative_path_, - const StorageID & table_id_, - const Names & key_names_, - bool use_nulls_, - SizeLimits limits_, - ASTTableJoin::Kind kind_, - ASTTableJoin::Strictness strictness_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool overwrite, - bool persistent_); }; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f84c356d66e..ea4a855ad34 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -1089,7 +1089,7 @@ void registerStorageLog(StorageFactory & factory) String disk_name = getDiskName(*args.storage_def); DiskPtr disk = args.getContext()->getDisk(disk_name); - return StorageLog::create( + return std::make_shared( args.engine_name, disk, args.relative_data_path, diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 6692fbe0524..534914c9725 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -2,8 +2,8 @@ #include #include -#include +#include #include #include #include @@ -19,14 +19,28 @@ namespace DB * Also implements TinyLog - a table engine that is suitable for small chunks of the log. * It differs from Log in the absence of mark files. */ -class StorageLog final : public shared_ptr_helper, public IStorage +class StorageLog final : public IStorage, boost::noncopyable { friend class LogSource; friend class LogSink; friend class LogRestoreTask; - friend struct shared_ptr_helper; public: + /** Attach the table with the appropriate name, along the appropriate path (with / at the end), + * (the correctness of names and paths is not verified) + * consisting of the specified columns; Create files if they do not exist. + */ + StorageLog( + const String & engine_name_, + DiskPtr disk_, + const std::string & relative_path_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + bool attach, + size_t max_compress_block_size_); + ~StorageLog() override; String getName() const override { return engine_name; } @@ -56,22 +70,6 @@ public: BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings, const std::shared_ptr & restore_coordination) override; -protected: - /** Attach the table with the appropriate name, along the appropriate path (with / at the end), - * (the correctness of names and paths is not verified) - * consisting of the specified columns; Create files if they do not exist. - */ - StorageLog( - const String & engine_name_, - DiskPtr disk_, - const std::string & relative_path_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool attach, - size_t max_compress_block_size_); - private: using ReadLock = std::shared_lock; using WriteLock = std::unique_lock; diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 953d83360fd..dca00f81281 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -4,6 +4,7 @@ #if USE_MYSQL +#include #include namespace DB @@ -14,14 +15,13 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -class StorageMaterializedMySQL final : public shared_ptr_helper, public StorageProxy +class StorageMaterializedMySQL final : public StorageProxy, boost::noncopyable { - friend struct shared_ptr_helper; public: - String getName() const override { return "MaterializedMySQL"; } - StorageMaterializedMySQL(const StoragePtr & nested_storage_, const IDatabase * database_); + String getName() const override { return "MaterializedMySQL"; } + bool needRewriteQueryWithFinal(const Names & column_names) const override; Pipe read( diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 063953161ea..ab78d1cfb80 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -462,7 +462,7 @@ void registerStorageMaterializedView(StorageFactory & factory) factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args) { /// Pass local_context here to convey setting for inner table - return StorageMaterializedView::create( + return std::make_shared( args.table_id, args.getLocalContext(), args.query, args.columns, args.attach, args.comment); }); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index c0038ed5190..7400ee55130 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -1,9 +1,8 @@ #pragma once -#include - #include +#include #include #include @@ -11,10 +10,17 @@ namespace DB { -class StorageMaterializedView final : public shared_ptr_helper, public IStorage, WithMutableContext +class StorageMaterializedView final : public IStorage, WithMutableContext, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageMaterializedView( + const StorageID & table_id_, + ContextPtr local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns_, + bool attach_, + const String & comment); + std::string getName() const override { return "MaterializedView"; } bool isView() const override { return true; } @@ -109,15 +115,6 @@ private: bool has_inner_table = false; void checkStatementCanBeForwarded() const; - -protected: - StorageMaterializedView( - const StorageID & table_id_, - ContextPtr local_context, - const ASTCreateQuery & query, - const ColumnsDescription & columns_, - bool attach_, - const String & comment); }; } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c660195c368..e7911125383 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -377,11 +378,9 @@ void StorageMemory::truncate( } -class MemoryBackupEntriesBatch : public shared_ptr_helper, public IBackupEntriesBatch +class MemoryBackupEntriesBatch : public IBackupEntriesBatch, boost::noncopyable { -private: - friend struct shared_ptr_helper; - +public: MemoryBackupEntriesBatch( const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr blocks_, UInt64 max_compress_block_size_) : IBackupEntriesBatch({"data.bin", "index.mrk", "sizes.json"}) @@ -391,6 +390,7 @@ private: { } +private: static constexpr const size_t kDataBinPos = 0; static constexpr const size_t kIndexMrkPos = 1; static constexpr const size_t kSizesJsonPos = 2; @@ -475,7 +475,7 @@ BackupEntries StorageMemory::backupData(ContextPtr context, const ASTs & partiti if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); - return MemoryBackupEntriesBatch::create(getInMemoryMetadataPtr(), data.get(), context->getSettingsRef().max_compress_block_size) + return std::make_shared(getInMemoryMetadataPtr(), data.get(), context->getSettingsRef().max_compress_block_size) ->getBackupEntries(); } @@ -589,7 +589,7 @@ void registerStorageMemory(StorageFactory & factory) if (has_settings) settings.loadFromQuery(*args.storage_def); - return StorageMemory::create(args.table_id, args.columns, args.constraints, args.comment, settings.compress); + return std::make_shared(args.table_id, args.columns, args.constraints, args.comment, settings.compress); }, { .supports_settings = true, diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 70da733668d..7918568bb71 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -4,8 +4,7 @@ #include #include -#include - +#include #include #include @@ -19,13 +18,19 @@ namespace DB * It does not support keys. * Data is stored as a set of blocks and is not stored anywhere else. */ -class StorageMemory final : public shared_ptr_helper, public IStorage +class StorageMemory final : public IStorage, boost::noncopyable { friend class MemorySink; friend class MemoryRestoreTask; -friend struct shared_ptr_helper; public: + StorageMemory( + const StorageID & table_id_, + ColumnsDescription columns_description_, + ConstraintsDescription constraints_, + const String & comment, + bool compress_ = false); + String getName() const override { return "Memory"; } size_t getSize() const { return data.get()->size(); } @@ -123,14 +128,6 @@ private: std::atomic total_size_rows = 0; bool compress; - -protected: - StorageMemory( - const StorageID & table_id_, - ColumnsDescription columns_description_, - ConstraintsDescription constraints_, - const String & comment, - bool compress_ = false); }; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5b1586040df..93da9a25d01 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -854,7 +854,7 @@ void registerStorageMerge(StorageFactory & factory) engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext()); String table_name_regexp = engine_args[1]->as().value.safeGet(); - return StorageMerge::create( + return std::make_shared( args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getContext()); }, { diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index b7bdd957164..c481dda605e 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include @@ -12,10 +11,29 @@ namespace DB /** A table that represents the union of an arbitrary number of other tables. * All tables must have the same structure. */ -class StorageMerge final : public shared_ptr_helper, public IStorage, WithContext +class StorageMerge final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; public: + using DBToTableSetMap = std::map>; + + StorageMerge( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const String & comment, + const String & source_database_name_or_regexp_, + bool database_is_regexp_, + const DBToTableSetMap & source_databases_and_tables_, + ContextPtr context_); + + StorageMerge( + const StorageID & table_id_, + const ColumnsDescription & columns_, + const String & comment, + const String & source_database_name_or_regexp_, + bool database_is_regexp_, + const String & source_table_regexp_, + ContextPtr context_); + std::string getName() const override { return "Merge"; } bool isRemote() const override; @@ -54,8 +72,6 @@ public: static std::tuple evaluateDatabaseName(const ASTPtr & node, ContextPtr context); private: - using DBToTableSetMap = std::map>; - std::optional source_database_regexp; std::optional source_table_regexp; std::optional source_databases_and_tables; @@ -88,24 +104,6 @@ private: ColumnSizeByName getColumnSizes() const override; protected: - StorageMerge( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const String & comment, - const String & source_database_name_or_regexp_, - bool database_is_regexp_, - const DBToTableSetMap & source_databases_and_tables_, - ContextPtr context_); - - StorageMerge( - const StorageID & table_id_, - const ColumnsDescription & columns_, - const String & comment, - const String & source_database_name_or_regexp_, - bool database_is_regexp_, - const String & source_table_regexp_, - ContextPtr context_); - struct AliasData { String name; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f7db56acd75..b685162f826 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -889,7 +889,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMerge( getContext()->getMergeList().bookMergeWithTTL(); merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts), *this, metadata_snapshot, false); - return std::make_shared(future_part, std::move(merging_tagger), MutationCommands::create()); + return std::make_shared(future_part, std::move(merging_tagger), std::make_shared()); } bool StorageMergeTree::merge( @@ -1012,7 +1012,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( continue; } - auto commands = MutationCommands::create(); + auto commands = std::make_shared(); size_t current_ast_elements = 0; auto last_mutation_to_apply = mutations_end_it; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) @@ -1183,7 +1183,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (auto lock = time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred( getSettings()->merge_tree_clear_old_temporary_directories_interval_seconds)) { - assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( + assignee.scheduleCommonTask(std::make_shared( [this, share_lock] () { return clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); @@ -1193,7 +1193,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (auto lock = time_after_previous_cleanup_parts.compareAndRestartDeferred( getSettings()->merge_tree_clear_old_parts_interval_seconds)) { - assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( + assignee.scheduleCommonTask(std::make_shared( [this, share_lock] () { /// All use relative_data_path which changes during rename diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6609bae40fb..4d2ec4edb5c 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -26,13 +25,30 @@ namespace DB /** See the description of the data structure in MergeTreeData. */ -class StorageMergeTree final : public shared_ptr_helper, public MergeTreeData +class StorageMergeTree final : public MergeTreeData, boost::noncopyable { - friend struct shared_ptr_helper; public: + /** Attach the table with the appropriate name, along the appropriate path (with / at the end), + * (correctness of names and paths are not checked) + * consisting of the specified columns. + * + * See MergeTreeData constructor for comments on parameters. + */ + StorageMergeTree( + const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + ContextMutablePtr context_, + const String & date_column_name, + const MergingParams & merging_params_, + std::unique_ptr settings_, + bool has_force_restore_data_flag); + void startup() override; void flush() override; void shutdown() override; + ~StorageMergeTree() override; std::string getName() const override { return merging_params.getModeName() + "MergeTree"; } @@ -257,23 +273,6 @@ private: protected: - /** Attach the table with the appropriate name, along the appropriate path (with / at the end), - * (correctness of names and paths are not checked) - * consisting of the specified columns. - * - * See MergeTreeData constructor for comments on parameters. - */ - StorageMergeTree( - const StorageID & table_id_, - const String & relative_data_path_, - const StorageInMemoryMetadata & metadata, - bool attach, - ContextMutablePtr context_, - const String & date_column_name, - const MergingParams & merging_params_, - std::unique_ptr settings_, - bool has_force_restore_data_flag); - MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index def02a8eb3a..680d1bda461 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -39,7 +39,7 @@ StorageMongoDB::StorageMongoDB( const String & comment) : IStorage(table_id_) , host(host_) - , port(port_) + // , port(port_) , database_name(database_name_) , collection_name(collection_name_) , username(username_) @@ -168,7 +168,7 @@ void registerStorageMongoDB(StorageFactory & factory) { auto configuration = StorageMongoDB::getConfiguration(args.engine_args, args.getLocalContext()); - return StorageMongoDB::create( + return std::make_shared( args.table_id, configuration.host, configuration.port, diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 549d444d7bb..57fc8e45588 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -2,12 +2,10 @@ #include -#include - +#include #include #include - namespace DB { /* Implements storage in the MongoDB database. @@ -15,9 +13,8 @@ namespace DB * Read only. */ -class StorageMongoDB final : public shared_ptr_helper, public IStorage +class StorageMongoDB final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: StorageMongoDB( const StorageID & table_id_, @@ -49,7 +46,7 @@ private: void connectIfNotConnected(); const std::string host; - const uint16_t port; /// NOLINT + // const uint16_t port; /// NOLINT -- actually not used, needs checking const std::string database_name; const std::string collection_name; const std::string username; diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 03ebaaf87d7..0caa48202f7 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -4,8 +4,7 @@ #if USE_MYSQL -#include - +#include #include #include #include @@ -23,9 +22,8 @@ namespace DB * Use ENGINE = mysql(host_port, database_name, table_name, user_name, password) * Read only. */ -class StorageMySQL final : public shared_ptr_helper, public IStorage, WithContext +class StorageMySQL final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; public: StorageMySQL( const StorageID & table_id_, diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 094e345e433..ada2521c43c 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -29,7 +29,7 @@ void registerStorageNull(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageNull::create(args.table_id, args.columns, args.constraints, args.comment); + return std::make_shared(args.table_id, args.columns, args.constraints, args.comment); }, { .supports_parallel_insert = true, diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index c5b2e2bf161..374bdead4a5 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -15,10 +14,20 @@ namespace DB /** When writing, does nothing. * When reading, returns nothing. */ -class StorageNull final : public shared_ptr_helper, public IStorage +class StorageNull final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageNull( + const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_, const String & comment) + : IStorage(table_id_) + { + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_description_); + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + } + std::string getName() const override { return "Null"; } Pipe read( @@ -54,19 +63,6 @@ public: return {0}; } -private: - -protected: - StorageNull( - const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_, const String & comment) - : IStorage(table_id_) - { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_description_); - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - } }; } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5f217cedbe4..69b6339881b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -451,7 +451,7 @@ void registerStoragePostgreSQL(StorageFactory & factory) args.getContext()->getSettingsRef().postgresql_connection_pool_size, args.getContext()->getSettingsRef().postgresql_connection_pool_wait_timeout); - return StoragePostgreSQL::create( + return std::make_shared( args.table_id, std::move(pool), configuration.table, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index ae41a713285..b3f1ea02b55 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -3,7 +3,7 @@ #include "config_core.h" #if USE_LIBPQXX -#include +#include #include #include #include @@ -17,9 +17,8 @@ class Logger; namespace DB { -class StoragePostgreSQL final : public shared_ptr_helper, public IStorage +class StoragePostgreSQL final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: StoragePostgreSQL( const StorageID & table_id_, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 56cb03d09b6..2cba93edc00 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3042,7 +3042,7 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne /// Depending on entry type execute in fetches (small) pool or big merge_mutate pool if (job_type == LogEntry::GET_PART) { - assignee.scheduleFetchTask(ExecutableLambdaAdapter::create( + assignee.scheduleFetchTask(std::make_shared( [this, selected_entry] () mutable { return processQueueEntry(selected_entry); @@ -3051,19 +3051,19 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne } else if (job_type == LogEntry::MERGE_PARTS) { - auto task = MergeFromLogEntryTask::create(selected_entry, *this, common_assignee_trigger); + auto task = std::make_shared(selected_entry, *this, common_assignee_trigger); assignee.scheduleMergeMutateTask(task); return true; } else if (job_type == LogEntry::MUTATE_PART) { - auto task = MutateFromLogEntryTask::create(selected_entry, *this, common_assignee_trigger); + auto task = std::make_shared(selected_entry, *this, common_assignee_trigger); assignee.scheduleMergeMutateTask(task); return true; } else { - assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( + assignee.scheduleCommonTask(std::make_shared( [this, selected_entry] () mutable { return processQueueEntry(selected_entry); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e589d40783c..8b4304185d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -1,9 +1,9 @@ #pragma once -#include #include #include #include +#include #include #include #include @@ -81,10 +81,32 @@ namespace DB * as the time will take the time of creation the appropriate part on any of the replicas. */ -class StorageReplicatedMergeTree final : public shared_ptr_helper, public MergeTreeData +class StorageReplicatedMergeTree final : public MergeTreeData, boost::noncopyable { - friend struct shared_ptr_helper; public: + enum RenamingRestrictions + { + ALLOW_ANY, + ALLOW_PRESERVING_UUID, + DO_NOT_ALLOW, + }; + + /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. + */ + StorageReplicatedMergeTree( + const String & zookeeper_path_, + const String & replica_name_, + bool attach, + const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata_, + ContextMutablePtr context_, + const String & date_column_name, + const MergingParams & merging_params_, + std::unique_ptr settings_, + bool has_force_restore_data_flag, + RenamingRestrictions renaming_restrictions_); + void startup() override; void shutdown() override; void flush() override; @@ -143,13 +165,6 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr query_context, TableExclusiveLockHolder &) override; - enum RenamingRestrictions - { - ALLOW_ANY, - ALLOW_PRESERVING_UUID, - DO_NOT_ALLOW, - }; - void checkTableCanBeRenamed(const StorageID & new_name) const override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; @@ -793,23 +808,6 @@ private: /// Create ephemeral lock in zookeeper for part and disk which support zero copy replication. /// If somebody already holding the lock -- return std::nullopt. std::optional tryCreateZeroCopyExclusiveLock(const String & part_name, const DiskPtr & disk) override; - -protected: - /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. - */ - StorageReplicatedMergeTree( - const String & zookeeper_path_, - const String & replica_name_, - bool attach, - const StorageID & table_id_, - const String & relative_data_path_, - const StorageInMemoryMetadata & metadata_, - ContextMutablePtr context_, - const String & date_column_name, - const MergingParams & merging_params_, - std::unique_ptr settings_, - bool has_force_restore_data_flag, - RenamingRestrictions renaming_restrictions_); }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e43fedebd86..217c5422030 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1018,7 +1018,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - return StorageS3::create( + return std::make_shared( s3_uri, configuration.auth_settings.access_key_id, configuration.auth_settings.secret_access_key, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index ce78ec5ec85..98fd9d8c654 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -11,10 +11,10 @@ #include #include +#include #include #include #include -#include #include #include #include @@ -120,7 +120,7 @@ private: * It sends HTTP GET to server when select is called and * HTTP PUT when insert is called. */ -class StorageS3 : public shared_ptr_helper, public IStorage, WithContext +class StorageS3 : public IStorage, WithContext, boost::noncopyable { public: StorageS3( diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 52b32e1217f..71308d8966a 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -7,8 +7,7 @@ #include #include -#include - +#include #include "Client/Connection.h" #include #include @@ -19,21 +18,9 @@ namespace DB class Context; -class StorageS3Cluster : public shared_ptr_helper, public IStorage +class StorageS3Cluster : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: - std::string getName() const override { return "S3Cluster"; } - - Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, - ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override; - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; - - NamesAndTypesList getVirtuals() const override; - -protected: StorageS3Cluster( const String & filename_, const String & access_key_id_, @@ -46,6 +33,16 @@ protected: ContextPtr context_, const String & compression_method_); + std::string getName() const override { return "S3Cluster"; } + + Pipe read(const Names &, const StorageSnapshotPtr &, SelectQueryInfo &, + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override; + + QueryProcessingStage::Enum + getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + + NamesAndTypesList getVirtuals() const override; + private: StorageS3::S3Configuration s3_configuration; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 199deb8a2ac..9b203967edd 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -172,7 +172,7 @@ void registerStorageSQLite(StorageFactory & factory) auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */!args.attach); - return StorageSQLite::create(args.table_id, sqlite_db, database_path, + return std::make_shared(args.table_id, sqlite_db, database_path, table_name, args.columns, args.constraints, args.getContext()); }, { diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index 367e6ee9e80..e986b11518d 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -3,7 +3,7 @@ #include "config_core.h" #if USE_SQLITE -#include +#include #include #include @@ -16,10 +16,8 @@ class Logger; namespace DB { -class StorageSQLite final : public shared_ptr_helper, public IStorage, public WithContext +class StorageSQLite final : public IStorage, public WithContext, boost::noncopyable { -friend struct shared_ptr_helper; - public: using SQLitePtr = std::shared_ptr; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 533ed6913af..ad63499acfa 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -258,7 +258,7 @@ void registerStorageSet(StorageFactory & factory) set_settings.loadFromQuery(*args.storage_def); DiskPtr disk = args.getContext()->getDisk(set_settings.disk); - return StorageSet::create( + return std::make_shared( disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.comment, set_settings.persistent); }, StorageFactory::StorageFeatures{ .supports_settings = true, }); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index a84bdea1430..56628c77a9d 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include #include #include #include @@ -63,11 +62,18 @@ private: * and also written to a file-backup, for recovery after a restart. * Reading from the table is not possible directly - it is possible to specify only the right part of the IN statement. */ -class StorageSet final : public shared_ptr_helper, public StorageSetOrJoinBase +class StorageSet final : public StorageSetOrJoinBase, boost::noncopyable { -friend struct shared_ptr_helper; - public: + StorageSet( + DiskPtr disk_, + const String & relative_path_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + bool persistent_); + String getName() const override { return "Set"; } /// Access the insides. @@ -84,16 +90,6 @@ private: void insertBlock(const Block & block, ContextPtr) override; void finishInsert() override; size_t getSize(ContextPtr) const override; - -protected: - StorageSet( - DiskPtr disk_, - const String & relative_path_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool persistent_); }; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 55da256a866..198bf9c01f9 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -663,7 +663,7 @@ void registerStorageStripeLog(StorageFactory & factory) String disk_name = getDiskName(*args.storage_def); DiskPtr disk = args.getContext()->getDisk(disk_name); - return StorageStripeLog::create( + return std::make_shared( disk, args.relative_data_path, args.table_id, diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index dbc69fa1553..6f586d68740 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -3,8 +3,7 @@ #include #include -#include - +#include #include #include #include @@ -20,14 +19,23 @@ struct IndexForNativeFormat; /** Implements a table engine that is suitable for small chunks of the log. * In doing so, stores all the columns in a single Native file, with a nearby index. */ -class StorageStripeLog final : public shared_ptr_helper, public IStorage +class StorageStripeLog final : public IStorage, boost::noncopyable { - friend class StripeLogSource; - friend class StripeLogSink; - friend class StripeLogRestoreTask; - friend struct shared_ptr_helper; +friend class StripeLogSource; +friend class StripeLogSink; +friend class StripeLogRestoreTask; public: + StorageStripeLog( + DiskPtr disk_, + const String & relative_path_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + bool attach, + size_t max_compress_block_size_); + ~StorageStripeLog() override; String getName() const override { return "StripeLog"; } @@ -56,17 +64,6 @@ public: BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings, const std::shared_ptr & restore_coordination) override; -protected: - StorageStripeLog( - DiskPtr disk_, - const String & relative_path_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool attach, - size_t max_compress_block_size_); - private: using ReadLock = std::shared_lock; using WriteLock = std::unique_lock; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 48fde50c5ee..35752835581 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -919,7 +919,7 @@ void registerStorageURL(StorageFactory & factory) if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - return StorageURL::create( + return std::make_shared( configuration.url, args.table_id, configuration.format, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index a035b1bb93d..feac2471155 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -1,8 +1,8 @@ #pragma once +#include #include #include -#include #include #include #include @@ -122,9 +122,8 @@ private: OutputFormatPtr writer; }; -class StorageURL : public shared_ptr_helper, public IStorageURLBase +class StorageURL : public IStorageURLBase, boost::noncopyable { - friend struct shared_ptr_helper; public: StorageURL( const String & uri_, diff --git a/src/Storages/StorageValues.h b/src/Storages/StorageValues.h index a4e18657f9f..836495ae4ba 100644 --- a/src/Storages/StorageValues.h +++ b/src/Storages/StorageValues.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,10 +9,12 @@ namespace DB /* One block storage used for values table function * It's structure is similar to IStorageSystemOneBlock */ -class StorageValues final : public shared_ptr_helper, public IStorage +class StorageValues final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageValues( + const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, const NamesAndTypesList & virtuals_ = {}); + std::string getName() const override { return "Values"; } Pipe read( @@ -38,10 +40,6 @@ public: private: Block res_block; NamesAndTypesList virtuals; - -protected: - StorageValues( - const StorageID & table_id_, const ColumnsDescription & columns_, const Block & res_block_, const NamesAndTypesList & virtuals_ = {}); }; } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 68b16de5a80..98d07686796 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -241,7 +241,7 @@ void registerStorageView(StorageFactory & factory) if (args.query.storage) throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY); - return StorageView::create(args.table_id, args.query, args.columns, args.comment); + return std::make_shared(args.table_id, args.query, args.columns, args.comment); }); } diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index f49736afe4a..9894c657d5b 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -1,19 +1,23 @@ #pragma once +#include #include #include #include -#include - namespace DB { -class StorageView final : public shared_ptr_helper, public IStorage +class StorageView final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageView( + const StorageID & table_id_, + const ASTCreateQuery & query, + const ColumnsDescription & columns_, + const String & comment); + std::string getName() const override { return "View"; } bool isView() const override { return true; } @@ -47,13 +51,6 @@ public: static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name); static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name); - -protected: - StorageView( - const StorageID & table_id_, - const ASTCreateQuery & query, - const ColumnsDescription & columns_, - const String & comment); }; } diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 7cc94a3f2f6..2cfe2de05db 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 97b84a49cde..b039d856e58 100644 --- a/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -1,16 +1,14 @@ #pragma once +#include #include #include #include -#include namespace DB { -class StorageSystemAggregateFunctionCombinators final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemAggregateFunctionCombinators final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.h b/src/Storages/System/StorageSystemAsynchronousInserts.h index d25217006db..52e0e758e1f 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.h +++ b/src/Storages/System/StorageSystemAsynchronousInserts.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -9,16 +9,13 @@ namespace DB /** Implements the system table `asynhronous_inserts`, * which contains information about pending asynchronous inserts in queue. */ -class StorageSystemAsynchronousInserts final : - public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemAsynchronousInserts"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.h b/src/Storages/System/StorageSystemAsynchronousMetrics.h index 2efe008ef3a..fb72e0d6c4f 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -12,11 +12,11 @@ class Context; /** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics. */ -class StorageSystemAsynchronousMetrics final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemAsynchronousMetrics final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: + StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); + std::string getName() const override { return "SystemAsynchronousMetrics"; } static NamesAndTypesList getNamesAndTypes(); @@ -25,8 +25,6 @@ private: const AsynchronousMetrics & async_metrics; protected: - StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_); - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/StorageSystemBackups.h b/src/Storages/System/StorageSystemBackups.h index e31c7b0a994..7b683ec8aa9 100644 --- a/src/Storages/System/StorageSystemBackups.h +++ b/src/Storages/System/StorageSystemBackups.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -8,14 +8,13 @@ namespace DB { /// Implements `grants` system table, which allows you to get information about grants. -class StorageSystemBackups final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemBackups final : public boost::noncopyable, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemBackups"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemBuildOptions.h b/src/Storages/System/StorageSystemBuildOptions.h index 4ecc1b21426..a6a7f65c66d 100644 --- a/src/Storages/System/StorageSystemBuildOptions.h +++ b/src/Storages/System/StorageSystemBuildOptions.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** System table "build_options" with many params used for clickhouse building */ -class StorageSystemBuildOptions final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemBuildOptions final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 1e6bfff8a85..35349bd38e6 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -1,8 +1,8 @@ #pragma once +#include #include #include -#include #include @@ -16,9 +16,8 @@ class Cluster; * that allows to obtain information about available clusters * (which may be specified in Distributed tables). */ -class StorageSystemClusters final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemClusters final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemClusters"; } diff --git a/src/Storages/System/StorageSystemCollations.h b/src/Storages/System/StorageSystemCollations.h index 1a58deba989..bd2d1055275 100644 --- a/src/Storages/System/StorageSystemCollations.h +++ b/src/Storages/System/StorageSystemCollations.h @@ -1,20 +1,19 @@ #pragma once + +#include #include -#include namespace DB { -class StorageSystemCollations final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemCollations final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; -public: +public: std::string getName() const override { return "SystemTableCollations"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index 126deef1921..3bf3241acab 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -11,10 +11,11 @@ class Context; /** Implements system table 'columns', that allows to get information about columns for every table. */ -class StorageSystemColumns final : public shared_ptr_helper, public IStorage +class StorageSystemColumns final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemColumns(const StorageID & table_id_); + std::string getName() const override { return "SystemColumns"; } Pipe read( @@ -27,9 +28,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - StorageSystemColumns(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemContributors.h b/src/Storages/System/StorageSystemContributors.h index 04a70a840f8..c0b667a3fdc 100644 --- a/src/Storages/System/StorageSystemContributors.h +++ b/src/Storages/System/StorageSystemContributors.h @@ -1,8 +1,7 @@ #pragma once +#include #include -#include - namespace DB { @@ -11,10 +10,8 @@ class Context; /** System table "contributors" with list of clickhouse contributors */ -class StorageSystemContributors final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemContributors final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemCurrentRoles.h b/src/Storages/System/StorageSystemCurrentRoles.h index 4b0547b7f0c..af7c37e806d 100644 --- a/src/Storages/System/StorageSystemCurrentRoles.h +++ b/src/Storages/System/StorageSystemCurrentRoles.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `current_roles` system table, which allows you to get information about current roles. -class StorageSystemCurrentRoles final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemCurrentRoles final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemCurrentRoles"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.h b/src/Storages/System/StorageSystemDDLWorkerQueue.h index 50eb2e10c7c..46517318b00 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.h +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.h @@ -1,7 +1,8 @@ #pragma once + +#include #include #include -#include #include namespace DB @@ -11,11 +12,8 @@ class Context; /** System table "distributed_ddl_queue" with list of queries that are currently in the DDL worker queue. */ -class StorageSystemDDLWorkerQueue final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemDDLWorkerQueue final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; - protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.h b/src/Storages/System/StorageSystemDataSkippingIndices.h index 93511d0d591..c07f185c21f 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.h +++ b/src/Storages/System/StorageSystemDataSkippingIndices.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -8,10 +8,11 @@ namespace DB { /// For system.data_skipping_indices table - describes the data skipping indices in tables, similar to system.columns. -class StorageSystemDataSkippingIndices : public shared_ptr_helper, public IStorage +class StorageSystemDataSkippingIndices : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemDataSkippingIndices(const StorageID & table_id_); + std::string getName() const override { return "SystemDataSkippingIndices"; } Pipe read( @@ -24,9 +25,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - explicit StorageSystemDataSkippingIndices(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemDataTypeFamilies.h b/src/Storages/System/StorageSystemDataTypeFamilies.h index 2ff04039471..44bfc8fb5f0 100644 --- a/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -1,15 +1,13 @@ #pragma once -#include +#include #include namespace DB { -class StorageSystemDataTypeFamilies final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemDataTypeFamilies final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemDatabases.h b/src/Storages/System/StorageSystemDatabases.h index 3de0da126d4..dd7d2cd1004 100644 --- a/src/Storages/System/StorageSystemDatabases.h +++ b/src/Storages/System/StorageSystemDatabases.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements `databases` system table, which allows you to get information about all databases. */ -class StorageSystemDatabases final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemDatabases final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 4797dff2fd1..3549085f319 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index 8ed11eb306c..0ae1738a3d5 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -1,5 +1,6 @@ #pragma once +#include #include namespace DB @@ -11,18 +12,15 @@ namespace DB * We don't use StorageSystemPartsBase, because it introduces virtual _state * column and column aliases which we don't need. */ -class StorageSystemDetachedParts final : - public shared_ptr_helper, - public IStorage +class StorageSystemDetachedParts final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemDetachedParts(const StorageID & table_id_); + std::string getName() const override { return "SystemDetachedParts"; } bool isSystemStorage() const override { return true; } protected: - explicit StorageSystemDetachedParts(const StorageID & table_id_); - Pipe read( const Names & /* column_names */, const StorageSnapshotPtr & storage_snapshot, diff --git a/src/Storages/System/StorageSystemDictionaries.h b/src/Storages/System/StorageSystemDictionaries.h index f862748789a..4011f7ffe90 100644 --- a/src/Storages/System/StorageSystemDictionaries.h +++ b/src/Storages/System/StorageSystemDictionaries.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -10,9 +10,8 @@ namespace DB class Context; -class StorageSystemDictionaries final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemDictionaries final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemDictionaries"; } diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index 2640ab7149b..6ec50252ca0 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -14,10 +14,11 @@ class Context; /** Implements the system table `disks`, which allows you to get information about all disks. */ -class StorageSystemDisks final : public shared_ptr_helper, public IStorage +class StorageSystemDisks final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemDisks(const StorageID & table_id_); + std::string getName() const override { return "SystemDisks"; } Pipe read( @@ -30,9 +31,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - explicit StorageSystemDisks(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemDistributionQueue.h b/src/Storages/System/StorageSystemDistributionQueue.h index 97c327827dc..f212980b020 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.h +++ b/src/Storages/System/StorageSystemDistributionQueue.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables. */ -class StorageSystemDistributionQueue final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemDistributionQueue final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemDistributionQueue"; } diff --git a/src/Storages/System/StorageSystemEnabledRoles.h b/src/Storages/System/StorageSystemEnabledRoles.h index 9d78b20109b..d683445c499 100644 --- a/src/Storages/System/StorageSystemEnabledRoles.h +++ b/src/Storages/System/StorageSystemEnabledRoles.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `enabled_roles` system table, which allows you to get information about enabled roles. -class StorageSystemEnabledRoles final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemEnabledRoles final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemEnabledRoles"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemErrors.h b/src/Storages/System/StorageSystemErrors.h index f0ef0f7228a..5bfc8aa721f 100644 --- a/src/Storages/System/StorageSystemErrors.h +++ b/src/Storages/System/StorageSystemErrors.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -14,9 +14,8 @@ class Context; * Implements the `errors` system table, which shows the error code and the number of times it happens * (i.e. Exception with this code had been thrown). */ -class StorageSystemErrors final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemErrors final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemErrors"; } diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index e765c60b6fa..adae1a7ca79 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -11,9 +11,8 @@ class Context; /** Implements `events` system table, which allows you to obtain information for profiling. */ -class StorageSystemEvents final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemEvents final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemEvents"; } diff --git a/src/Storages/System/StorageSystemFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h index 1d9d28d7b50..0bb9d5a433b 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -29,18 +29,16 @@ namespace DB * FORMAT Vertical */ -class StorageSystemFilesystemCache final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemFilesystemCache final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemFilesystemCache(const StorageID & table_id_); + std::string getName() const override { return "SystemFilesystemCache"; } static NamesAndTypesList getNamesAndTypes(); protected: - explicit StorageSystemFilesystemCache(const StorageID & table_id_); - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/StorageSystemFormats.h b/src/Storages/System/StorageSystemFormats.h index 5e0dfeeefb9..11a7c00e4c4 100644 --- a/src/Storages/System/StorageSystemFormats.h +++ b/src/Storages/System/StorageSystemFormats.h @@ -1,19 +1,18 @@ #pragma once +#include #include -#include namespace DB { -class StorageSystemFormats final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemFormats final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; -public: +public: std::string getName() const override { return "SystemFormats"; diff --git a/src/Storages/System/StorageSystemFunctions.h b/src/Storages/System/StorageSystemFunctions.h index bdb2c9ca274..6cedf490cb5 100644 --- a/src/Storages/System/StorageSystemFunctions.h +++ b/src/Storages/System/StorageSystemFunctions.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -13,9 +13,8 @@ class Context; /** Implements `functions`system table, which allows you to get a list * all normal and aggregate functions. */ -class StorageSystemFunctions final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemFunctions final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemFunctions"; } diff --git a/src/Storages/System/StorageSystemGrants.h b/src/Storages/System/StorageSystemGrants.h index 0cc6cff3e84..4362036c893 100644 --- a/src/Storages/System/StorageSystemGrants.h +++ b/src/Storages/System/StorageSystemGrants.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `grants` system table, which allows you to get information about grants. -class StorageSystemGrants final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemGrants final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemGrants"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemGraphite.h b/src/Storages/System/StorageSystemGraphite.h index 6c529558471..eb527607d96 100644 --- a/src/Storages/System/StorageSystemGraphite.h +++ b/src/Storages/System/StorageSystemGraphite.h @@ -1,19 +1,18 @@ #pragma once +#include #include #include #include #include #include -#include namespace DB { /// Provides information about Graphite configuration. -class StorageSystemGraphite final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemGraphite final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemGraphite"; } diff --git a/src/Storages/System/StorageSystemLicenses.h b/src/Storages/System/StorageSystemLicenses.h index 0876d01c311..6d246ec248f 100644 --- a/src/Storages/System/StorageSystemLicenses.h +++ b/src/Storages/System/StorageSystemLicenses.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include namespace DB @@ -11,11 +11,8 @@ class Context; /** System table "licenses" with list of licenses of 3rd party libraries */ -class StorageSystemLicenses final : - public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemLicenses final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemMacros.h b/src/Storages/System/StorageSystemMacros.h index 90032957c57..48cdcdddf00 100644 --- a/src/Storages/System/StorageSystemMacros.h +++ b/src/Storages/System/StorageSystemMacros.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include #include @@ -13,9 +13,8 @@ class Context; /** Information about macros for introspection. */ -class StorageSystemMacros final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMacros final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemMacros"; } diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h index 8169d1a83fb..d4885f0c274 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.h +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.h @@ -3,7 +3,7 @@ #include "config_core.h" #if USE_ROCKSDB -#include +#include #include @@ -13,10 +13,8 @@ class Context; /// Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for testing purposes. -class StorageSystemMergeTreeMetadataCache : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMergeTreeMetadataCache : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; - public: std::string getName() const override { return "SystemMergeTreeMetadataCache"; } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.h b/src/Storages/System/StorageSystemMergeTreeSettings.h index 0c9e857b8e6..5252f10820b 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.h +++ b/src/Storages/System/StorageSystemMergeTreeSettings.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -15,11 +15,8 @@ class Context; * which allows to get information about the current MergeTree settings. */ template -class SystemMergeTreeSettings final : public shared_ptr_helper>, - public IStorageSystemOneBlock> +class SystemMergeTreeSettings final : public IStorageSystemOneBlock>, boost::noncopyable { - friend struct shared_ptr_helper>; - public: std::string getName() const override { return replicated ? "SystemReplicatedMergeTreeSettings" : "SystemMergeTreeSettings"; } diff --git a/src/Storages/System/StorageSystemMerges.h b/src/Storages/System/StorageSystemMerges.h index 48a07620d19..f198b2e694d 100644 --- a/src/Storages/System/StorageSystemMerges.h +++ b/src/Storages/System/StorageSystemMerges.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include #include -#include #include @@ -13,9 +13,8 @@ namespace DB class Context; -class StorageSystemMerges final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMerges final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemMerges"; } diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index 107f2c05fb1..a656c706226 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements `metrics` system table, which provides information about the operation of the server. */ -class StorageSystemMetrics final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMetrics final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemMetrics"; } diff --git a/src/Storages/System/StorageSystemModels.h b/src/Storages/System/StorageSystemModels.h index 491fa63ca5a..8f3b11d37f2 100644 --- a/src/Storages/System/StorageSystemModels.h +++ b/src/Storages/System/StorageSystemModels.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -10,9 +10,8 @@ namespace DB class Context; -class StorageSystemModels final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemModels final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemModels"; } diff --git a/src/Storages/System/StorageSystemMutations.h b/src/Storages/System/StorageSystemMutations.h index f3c77e349b0..c40b2a04ffb 100644 --- a/src/Storages/System/StorageSystemMutations.h +++ b/src/Storages/System/StorageSystemMutations.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /// Implements the `mutations` system table, which provides information about the status of mutations /// in the MergeTree tables. -class StorageSystemMutations final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemMutations final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: String getName() const override { return "SystemMutations"; } diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index 043a0e7c0c2..1296137a238 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -23,10 +23,13 @@ class Context; * In multithreaded case, if even_distributed is False, implementation with atomic is used, * and result is always in [0 ... limit - 1] range. */ -class StorageSystemNumbers final : public shared_ptr_helper, public IStorage +class StorageSystemNumbers final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + /// If even_distribution is true, numbers are distributed evenly between streams. + /// Otherwise, streams concurrently increment atomic. + StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, bool even_distribution_ = true); + std::string getName() const override { return "SystemNumbers"; } Pipe read( @@ -47,11 +50,6 @@ private: bool even_distribution; std::optional limit; UInt64 offset; - -protected: - /// If even_distribution is true, numbers are distributed evenly between streams. - /// Otherwise, streams concurrently increment atomic. - StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, bool even_distribution_ = true); }; } diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 1c3d5c9ab80..1378dcae184 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -15,10 +15,11 @@ class Context; * Used when the table is not specified in the query. * Analog of the DUAL table in Oracle and MySQL. */ -class StorageSystemOne final : public shared_ptr_helper, public IStorage +class StorageSystemOne final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemOne(const StorageID & table_id_); + std::string getName() const override { return "SystemOne"; } Pipe read( @@ -33,9 +34,6 @@ public: bool isSystemStorage() const override { return true; } bool supportsTransactions() const override { return true; } - -protected: - explicit StorageSystemOne(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.h b/src/Storages/System/StorageSystemPartMovesBetweenShards.h index a73b594f1c8..bca32de0b93 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.h +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -10,9 +10,8 @@ namespace DB class Context; -class StorageSystemPartMovesBetweenShards final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemPartMovesBetweenShards final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemShardMoves"; } diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h index f7b069c9516..b1a683b2f59 100644 --- a/src/Storages/System/StorageSystemParts.h +++ b/src/Storages/System/StorageSystemParts.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,14 +12,14 @@ class Context; /** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family. */ -class StorageSystemParts final : public shared_ptr_helper, public StorageSystemPartsBase +class StorageSystemParts final : public StorageSystemPartsBase, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemParts(const StorageID & table_id_); + std::string getName() const override { return "SystemParts"; } protected: - explicit StorageSystemParts(const StorageID & table_id_); void processNextStorage( ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index 3eb8c7c8711..477261ad7ad 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/System/StorageSystemPartsColumns.h b/src/Storages/System/StorageSystemPartsColumns.h index 9cdd2befb40..fcc4a7d7c77 100644 --- a/src/Storages/System/StorageSystemPartsColumns.h +++ b/src/Storages/System/StorageSystemPartsColumns.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -13,15 +13,14 @@ class Context; /** Implements system table 'parts_columns' which allows to get information about * columns in data parts for tables of MergeTree family. */ -class StorageSystemPartsColumns final - : public shared_ptr_helper, public StorageSystemPartsBase +class StorageSystemPartsColumns final : public StorageSystemPartsBase, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemPartsColumns(const StorageID & table_id_); + std::string getName() const override { return "SystemPartsColumns"; } protected: - explicit StorageSystemPartsColumns(const StorageID & table_id_); void processNextStorage( ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemPrivileges.h b/src/Storages/System/StorageSystemPrivileges.h index 5eaba9bed79..09777fff66e 100644 --- a/src/Storages/System/StorageSystemPrivileges.h +++ b/src/Storages/System/StorageSystemPrivileges.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,7 +9,7 @@ namespace DB class Context; /// Implements `privileges` system table, which allows you to get information about access types. -class StorageSystemPrivileges final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemPrivileges final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemPrivileges"; } @@ -17,7 +17,6 @@ public: static const std::vector> & getAccessTypeEnumValues(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemProcesses.h b/src/Storages/System/StorageSystemProcesses.h index c22b83df6d0..0c67cfcd4ea 100644 --- a/src/Storages/System/StorageSystemProcesses.h +++ b/src/Storages/System/StorageSystemProcesses.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemProcesses final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemProcesses final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemProcesses"; } diff --git a/src/Storages/System/StorageSystemProjectionParts.h b/src/Storages/System/StorageSystemProjectionParts.h index be31d08b24e..ba4165e2647 100644 --- a/src/Storages/System/StorageSystemProjectionParts.h +++ b/src/Storages/System/StorageSystemProjectionParts.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,14 +12,14 @@ class Context; /** Implements system table 'projection_parts' which allows to get information about projection parts for tables of MergeTree family. */ -class StorageSystemProjectionParts final : public shared_ptr_helper, public StorageSystemPartsBase +class StorageSystemProjectionParts final : public StorageSystemPartsBase, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemProjectionParts(const StorageID & table_id_); + std::string getName() const override { return "SystemProjectionParts"; } protected: - explicit StorageSystemProjectionParts(const StorageID & table_id_); void processNextStorage( ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.h b/src/Storages/System/StorageSystemProjectionPartsColumns.h index ade07b70a23..50c04dcb612 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.h +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -13,15 +13,14 @@ class Context; /** Implements system table 'projection_parts_columns' which allows to get information about * columns in projection parts for tables of MergeTree family. */ -class StorageSystemProjectionPartsColumns final - : public shared_ptr_helper, public StorageSystemPartsBase +class StorageSystemProjectionPartsColumns final : public StorageSystemPartsBase, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemProjectionPartsColumns(const StorageID & table_id_); + std::string getName() const override { return "SystemProjectionPartsColumns"; } protected: - explicit StorageSystemProjectionPartsColumns(const StorageID & table_id_); void processNextStorage( ContextPtr context, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override; }; diff --git a/src/Storages/System/StorageSystemQuotaLimits.h b/src/Storages/System/StorageSystemQuotaLimits.h index 0fae259f4fc..071f588b4ae 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.h +++ b/src/Storages/System/StorageSystemQuotaLimits.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `quota_limits` system table, which allows you to get information about the limits set for quotas. -class StorageSystemQuotaLimits final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemQuotaLimits final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemQuotaLimits"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemQuotaUsage.h b/src/Storages/System/StorageSystemQuotaUsage.h index 98fa6692628..4b93b7faada 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.h +++ b/src/Storages/System/StorageSystemQuotaUsage.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -13,7 +13,7 @@ struct QuotaUsage; /** Implements the `quota_usage` system table, which allows you to get information about * how the current user uses the quota. */ -class StorageSystemQuotaUsage final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemQuotaUsage final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemQuotaUsage"; } @@ -23,7 +23,6 @@ public: static void fillDataImpl(MutableColumns & res_columns, ContextPtr context, bool add_column_is_current, const std::vector & quotas_usage); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemQuotas.h b/src/Storages/System/StorageSystemQuotas.h index 9dac02ccf56..63fd7349f8c 100644 --- a/src/Storages/System/StorageSystemQuotas.h +++ b/src/Storages/System/StorageSystemQuotas.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -10,14 +10,13 @@ class Context; /** Implements the `quotas` system tables, which allows you to get information about quotas. */ -class StorageSystemQuotas final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemQuotas final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemQuotas"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemQuotasUsage.h b/src/Storages/System/StorageSystemQuotasUsage.h index 3f7aa237fe0..c41531136d4 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.h +++ b/src/Storages/System/StorageSystemQuotasUsage.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -11,14 +11,13 @@ class Context; /** Implements the `quotas_usage` system table, which allows you to get information about * how all users use the quotas. */ -class StorageSystemQuotasUsage final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemQuotasUsage final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemQuotasUsage"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.h b/src/Storages/System/StorageSystemRemoteDataPaths.h index 0057dcf12f2..db20da8317f 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.h +++ b/src/Storages/System/StorageSystemRemoteDataPaths.h @@ -1,15 +1,16 @@ #pragma once -#include +#include #include namespace DB { -class StorageSystemRemoteDataPaths : public shared_ptr_helper, public IStorage +class StorageSystemRemoteDataPaths : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemRemoteDataPaths(const StorageID & table_id_); + std::string getName() const override { return "SystemRemoteDataPaths"; } bool isSystemStorage() const override { return true; } @@ -22,9 +23,6 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - -protected: - explicit StorageSystemRemoteDataPaths(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index 1b93d10367b..d66322a2ea6 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,10 +12,11 @@ class Context; /** Implements `replicas` system table, which provides information about the status of the replicated tables. */ -class StorageSystemReplicas final : public shared_ptr_helper, public IStorage +class StorageSystemReplicas final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemReplicas(const StorageID & table_id_); + std::string getName() const override { return "SystemReplicas"; } Pipe read( @@ -28,9 +29,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - explicit StorageSystemReplicas(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemReplicatedFetches.h b/src/Storages/System/StorageSystemReplicatedFetches.h index ccdf99b9ce1..471e81b88a3 100644 --- a/src/Storages/System/StorageSystemReplicatedFetches.h +++ b/src/Storages/System/StorageSystemReplicatedFetches.h @@ -1,7 +1,7 @@ #pragma once -#include +#include #include @@ -11,9 +11,8 @@ namespace DB class Context; /// system.replicated_fetches table. Takes data from context.getReplicatedFetchList() -class StorageSystemReplicatedFetches final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemReplicatedFetches final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemReplicatedFetches"; } diff --git a/src/Storages/System/StorageSystemReplicationQueue.h b/src/Storages/System/StorageSystemReplicationQueue.h index 64466d4c3e0..d279e830a89 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.h +++ b/src/Storages/System/StorageSystemReplicationQueue.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables. */ -class StorageSystemReplicationQueue final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemReplicationQueue final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemReplicationQueue"; } @@ -22,7 +21,6 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; }; diff --git a/src/Storages/System/StorageSystemRoleGrants.h b/src/Storages/System/StorageSystemRoleGrants.h index bd9c1086013..2de9680a6ef 100644 --- a/src/Storages/System/StorageSystemRoleGrants.h +++ b/src/Storages/System/StorageSystemRoleGrants.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `role_grants` system table, which allows you to get information about granted roles. -class StorageSystemRoleGrants final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemRoleGrants final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemRoleGrants"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemRoles.h b/src/Storages/System/StorageSystemRoles.h index ef5a5381ec3..fcf60588618 100644 --- a/src/Storages/System/StorageSystemRoles.h +++ b/src/Storages/System/StorageSystemRoles.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `roles` system table, which allows you to get information about roles. -class StorageSystemRoles final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemRoles final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemRoles"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemRowPolicies.h b/src/Storages/System/StorageSystemRowPolicies.h index f310dfe0927..dd8fb91ff22 100644 --- a/src/Storages/System/StorageSystemRowPolicies.h +++ b/src/Storages/System/StorageSystemRowPolicies.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -11,14 +11,13 @@ class Context; /// Implements `row_policies` system table, which allows you to get information about row policies. -class StorageSystemRowPolicies final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemRowPolicies final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemRowPolicies"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemSettings.h b/src/Storages/System/StorageSystemSettings.h index d323908b7cf..a998f5c01d4 100644 --- a/src/Storages/System/StorageSystemSettings.h +++ b/src/Storages/System/StorageSystemSettings.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** implements system table "settings", which allows to get information about the current settings. */ -class StorageSystemSettings final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemSettings final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemSettings"; } diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.h b/src/Storages/System/StorageSystemSettingsProfileElements.h index 752cdfbf5dc..a2c8966d8bf 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.h +++ b/src/Storages/System/StorageSystemSettingsProfileElements.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `settings_profile_elements` system table, which allows you to get information about elements of settings profiles. -class StorageSystemSettingsProfileElements final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemSettingsProfileElements final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemSettingsProfileElements"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemSettingsProfiles.h b/src/Storages/System/StorageSystemSettingsProfiles.h index f1fdbe34321..b40e8e1c241 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.h +++ b/src/Storages/System/StorageSystemSettingsProfiles.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `settings_profiles` system table, which allows you to get information about profiles. -class StorageSystemSettingsProfiles final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemSettingsProfiles final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemSettingsProfiles"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemStackTrace.h b/src/Storages/System/StorageSystemStackTrace.h index da4315d3ffa..1c4465fd832 100644 --- a/src/Storages/System/StorageSystemStackTrace.h +++ b/src/Storages/System/StorageSystemStackTrace.h @@ -3,7 +3,7 @@ #ifdef OS_LINUX /// Because of 'sigqueue' functions and RT signals. #include -#include +#include #include namespace Poco @@ -20,15 +20,14 @@ class Context; /// Allows to introspect stack trace of all server threads. /// It acts like an embedded debugger. /// More than one instance of this table cannot be used. -class StorageSystemStackTrace final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemStackTrace final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemStackTrace(const StorageID & table_id_); + String getName() const override { return "SystemStackTrace"; } static NamesAndTypesList getNamesAndTypes(); - explicit StorageSystemStackTrace(const StorageID & table_id_); - protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index e2890c42897..14b392d27a3 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -14,10 +14,11 @@ class Context; /** Implements the system table `storage`, which allows you to get information about all disks. */ -class StorageSystemStoragePolicies final : public shared_ptr_helper, public IStorage +class StorageSystemStoragePolicies final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemStoragePolicies(const StorageID & table_id_); + std::string getName() const override { return "SystemStoragePolicies"; } Pipe read( @@ -30,9 +31,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - explicit StorageSystemStoragePolicies(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemTableEngines.h b/src/Storages/System/StorageSystemTableEngines.h index ac3be81c019..cfe02850320 100644 --- a/src/Storages/System/StorageSystemTableEngines.h +++ b/src/Storages/System/StorageSystemTableEngines.h @@ -1,16 +1,14 @@ #pragma once +#include #include #include -#include namespace DB { -class StorageSystemTableEngines final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemTableEngines final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; diff --git a/src/Storages/System/StorageSystemTableFunctions.h b/src/Storages/System/StorageSystemTableFunctions.h index 882a1254ee0..73c704182f5 100644 --- a/src/Storages/System/StorageSystemTableFunctions.h +++ b/src/Storages/System/StorageSystemTableFunctions.h @@ -1,23 +1,20 @@ #pragma once +#include #include #include -#include + namespace DB { -class StorageSystemTableFunctions final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemTableFunctions final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; public: - std::string getName() const override { return "SystemTableFunctions"; diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 7f6a099a824..4fcb4b35e88 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,10 +12,11 @@ class Context; /** Implements the system table `tables`, which allows you to get information about all tables. */ -class StorageSystemTables final : public shared_ptr_helper, public IStorage +class StorageSystemTables final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + explicit StorageSystemTables(const StorageID & table_id_); + std::string getName() const override { return "SystemTables"; } Pipe read( @@ -28,9 +29,6 @@ public: unsigned num_streams) override; bool isSystemStorage() const override { return true; } - -protected: - explicit StorageSystemTables(const StorageID & table_id_); }; } diff --git a/src/Storages/System/StorageSystemTimeZones.h b/src/Storages/System/StorageSystemTimeZones.h index adfae5d760b..0b392f17363 100644 --- a/src/Storages/System/StorageSystemTimeZones.h +++ b/src/Storages/System/StorageSystemTimeZones.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include namespace DB @@ -11,12 +11,9 @@ class Context; /** System table "time_zones" with list of timezones pulled from /contrib/cctz/testdata/zoneinfo */ -class StorageSystemTimeZones final : public shared_ptr_helper, - public IStorageSystemOneBlock +class StorageSystemTimeZones final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; - -protected: +public: void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemTransactions.h b/src/Storages/System/StorageSystemTransactions.h index 38244815549..f1a39c29c33 100644 --- a/src/Storages/System/StorageSystemTransactions.h +++ b/src/Storages/System/StorageSystemTransactions.h @@ -1,5 +1,6 @@ #pragma once -#include + +#include #include @@ -8,9 +9,8 @@ namespace DB class Context; -class StorageSystemTransactions final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemTransactions final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: String getName() const override { return "SystemTransactions"; } diff --git a/src/Storages/System/StorageSystemUserDirectories.h b/src/Storages/System/StorageSystemUserDirectories.h index 905eae7a7b9..c61d3e0055c 100644 --- a/src/Storages/System/StorageSystemUserDirectories.h +++ b/src/Storages/System/StorageSystemUserDirectories.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `users_directories` system table, which allows you to get information about user directories. -class StorageSystemUserDirectories final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemUserDirectories final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemUserDirectories"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemUsers.h b/src/Storages/System/StorageSystemUsers.h index b72ad612c65..f939d482766 100644 --- a/src/Storages/System/StorageSystemUsers.h +++ b/src/Storages/System/StorageSystemUsers.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,14 +9,13 @@ namespace DB class Context; /// Implements `users` system table, which allows you to get information about users. -class StorageSystemUsers final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemUsers final : public IStorageSystemOneBlock, boost::noncopyable { public: std::string getName() const override { return "SystemUsers"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemWarnings.h b/src/Storages/System/StorageSystemWarnings.h index 087c4016aff..0a81e25dcd6 100644 --- a/src/Storages/System/StorageSystemWarnings.h +++ b/src/Storages/System/StorageSystemWarnings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -11,15 +12,14 @@ class Context; /** Implements system.warnings table that contains warnings about server configuration * to be displayed in clickhouse-client. */ -class StorageSystemWarnings final : public shared_ptr_helper, - public IStorageSystemOneBlock { +class StorageSystemWarnings final : public IStorageSystemOneBlock, boost::noncopyable +{ public: std::string getName() const override { return "SystemWarnings"; } static NamesAndTypesList getNamesAndTypes(); protected: - friend struct shared_ptr_helper; using IStorageSystemOneBlock::IStorageSystemOneBlock; void fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const override; diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index 067e6c7217d..198abcf8d44 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -14,10 +14,13 @@ namespace DB * You could also specify a limit (how many zeros to give). * If multithreaded is specified, zeros will be generated in several streams. */ -class StorageSystemZeros final : public shared_ptr_helper, public IStorage +class StorageSystemZeros final : public IStorage, boost::noncopyable { - friend struct shared_ptr_helper; public: + /// If even_distribution is true, numbers are distributed evenly between streams. + /// Otherwise, streams concurrently increment atomic. + StorageSystemZeros(const StorageID & table_id_, bool multithreaded_, std::optional limit_ = std::nullopt); + std::string getName() const override { return "SystemZeros"; } Pipe read( @@ -36,11 +39,6 @@ public: private: bool multithreaded; std::optional limit; - -protected: - /// If even_distribution is true, numbers are distributed evenly between streams. - /// Otherwise, streams concurrently increment atomic. - StorageSystemZeros(const StorageID & table_id_, bool multithreaded_, std::optional limit_ = std::nullopt); }; } diff --git a/src/Storages/System/StorageSystemZooKeeper.h b/src/Storages/System/StorageSystemZooKeeper.h index 26bcaccba6a..3edbd15f839 100644 --- a/src/Storages/System/StorageSystemZooKeeper.h +++ b/src/Storages/System/StorageSystemZooKeeper.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -12,9 +12,8 @@ class Context; /** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes. */ -class StorageSystemZooKeeper final : public shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemZooKeeper final : public IStorageSystemOneBlock, boost::noncopyable { - friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemZooKeeper"; } diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index b6080d15f2c..fcc1ab43a64 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -14,7 +14,7 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl { /// Attach to Ordinary database. auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - system_database.attachTable(context, table_name, StorageT::create(table_id, std::forward(args)...)); + system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...)); } else { @@ -23,7 +23,7 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl /// and path is actually not used auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - system_database.attachTable(context, table_name, StorageT::create(table_id, std::forward(args)...), path); + system_database.attachTable(context, table_name, std::make_shared(table_id, std::forward(args)...), path); } } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e1798ffbad4..8669406f2ce 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1516,7 +1516,7 @@ void registerStorageWindowView(StorageFactory & factory) "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')", ErrorCodes::SUPPORT_IS_DISABLED); - return StorageWindowView::create(args.table_id, args.getLocalContext(), args.query, args.columns, args.attach); + return std::make_shared(args.table_id, args.getLocalContext(), args.query, args.columns, args.attach); }); } diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 4e1ca87a8da..9c340d1e45a 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -1,11 +1,11 @@ #pragma once +#include #include #include #include #include #include -#include #include @@ -99,14 +99,20 @@ using ASTPtr = std::shared_ptr; * Users need to take these duplicated results into account. */ -class StorageWindowView final : public shared_ptr_helper, public IStorage, WithContext +class StorageWindowView final : public IStorage, WithContext, boost::noncopyable { - friend struct shared_ptr_helper; friend class TimestampTransformation; friend class WindowViewSource; friend class WatermarkTransform; public: + StorageWindowView( + const StorageID & table_id_, + ContextPtr context_, + const ASTCreateQuery & query, + const ColumnsDescription & columns, + bool attach_); + String getName() const override { return "WindowView"; } bool isView() const override { return true; } @@ -241,12 +247,5 @@ private: StoragePtr getTargetStorage() const; Block & getHeader() const; - - StorageWindowView( - const StorageID & table_id_, - ContextPtr context_, - const ASTCreateQuery & query, - const ColumnsDescription & columns, - bool attach_); }; } diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 66922afdd9c..30dd63a5948 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -34,9 +34,9 @@ DB::StoragePtr createStorage(DB::DiskPtr & disk) NamesAndTypesList names_and_types; names_and_types.emplace_back("a", std::make_shared()); - StoragePtr table = StorageLog::create( + StoragePtr table = std::make_shared( "Log", disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types}, - ConstraintsDescription{}, String{}, false, 1048576); + ConstraintsDescription{}, String{}, false, static_cast(1048576)); table->startup(); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 57b9e73bbbd..3248f790bdb 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -86,7 +86,7 @@ private: database->attachTable( context, table_name, - StorageMemory::create( + std::make_shared( StorageID(db_name, table_name), ColumnsDescription{getColumns()}, ConstraintsDescription{}, String{})); } DatabaseCatalog::instance().attachDatabase(database->getDatabaseName(), database); diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 0eb3c286be0..c251b2703e1 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -53,7 +53,7 @@ StoragePtr TableFunctionDictionary::executeImpl( StorageID dict_id(getDatabaseName(), table_name); auto dictionary_table_structure = getActualTableStructure(context); - auto result = StorageDictionary::create( + auto result = std::make_shared( dict_id, dictionary_name, std::move(dictionary_table_structure), String{}, StorageDictionary::Location::Custom, context); return result; diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 18d7d8867e8..dc88cca51e6 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -80,7 +80,7 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, settings.script_name = script_name; settings.script_arguments = arguments; - auto storage = StorageExecutable::create(storage_id, format, settings, input_queries, getActualTableStructure(context), ConstraintsDescription{}); + auto storage = std::make_shared(storage_id, format, settings, input_queries, getActualTableStructure(context), ConstraintsDescription{}); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index b09bb8b6ae1..cd489e2e698 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -29,7 +29,7 @@ StoragePtr TableFunctionFile::getStorage(const String & source, String{}, }; - return StorageFile::create(source, global_context->getUserFilesPath(), args); + return std::make_shared(source, global_context->getUserFilesPath(), args); } ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context) const diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index 9a2de735353..83e6b07ea25 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -85,7 +85,7 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con { auto columns = getActualTableStructure(context); Block res_block = parseData(columns, context); - auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), columns, res_block); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index ff9630eabd8..ad766c6c66e 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -82,7 +82,7 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextP StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); - auto res = StorageGenerateRandom::create( + auto res = std::make_shared( StorageID(getDatabaseName(), table_name), columns, String{}, max_array_length, max_string_length, random_seed); res->startup(); return res; diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 9c434444314..0ff56fefb68 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -60,7 +60,7 @@ ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr contex StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); + auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index f1ef4262d08..28aed2f03ed 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -116,7 +116,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto res = StorageMerge::create( + auto res = std::make_shared( StorageID(getDatabaseName(), table_name), getActualTableStructure(context), String{}, diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index cb51799a4a7..dea95b86ffd 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -45,7 +45,7 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, Conte columns = getActualTableStructure(context); else if (!structure_hint.empty()) columns = structure_hint; - auto res = StorageNull::create(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription(), String{}); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription(), String{}); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 38ac0238362..0ef1d3815d9 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -42,7 +42,7 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = StorageSystemNumbers::create(StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 90fbb079bb6..1f8d72b68d7 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -267,7 +267,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con assert(cluster); StoragePtr res = remote_table_function_ptr - ? StorageDistributed::create( + ? std::make_shared( StorageID(getDatabaseName(), table_name), cached_columns, ConstraintsDescription{}, @@ -280,7 +280,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con DistributedSettings{}, false, cluster) - : StorageDistributed::create( + : std::make_shared( StorageID(getDatabaseName(), table_name), cached_columns, ConstraintsDescription{}, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 19674072099..d081ec4319d 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -156,7 +156,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context else if (!structure_hint.empty()) columns = structure_hint; - StoragePtr storage = StorageS3::create( + StoragePtr storage = std::make_shared( s3_uri, configuration.auth_settings.access_key_id, configuration.auth_settings.secret_access_key, diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index b72ac991350..2f558c58352 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -115,7 +115,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( /// On worker node this filename won't contains globs Poco::URI uri (configuration.url); S3::URI s3_uri (uri); - storage = StorageS3::create( + storage = std::make_shared( s3_uri, configuration.auth_settings.access_key_id, configuration.auth_settings.secret_access_key, @@ -133,7 +133,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl( } else { - storage = StorageS3Cluster::create( + storage = std::make_shared( configuration.url, configuration.auth_settings.access_key_id, configuration.auth_settings.secret_access_key, diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index a2038725d07..fb2dc90a1f7 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -35,7 +35,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, { auto columns = getActualTableStructure(context); - auto storage = StorageSQLite::create(StorageID(getDatabaseName(), table_name), + auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), sqlite_db, database_path, remote_table_name, diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 0a794831d01..6f9c4d8820b 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -66,7 +66,7 @@ StoragePtr TableFunctionURL::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const { - return StorageURL::create( + return std::make_shared( source, StorageID(getDatabaseName(), table_name), format_, diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 595e8f9cf41..3f16b35fce5 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -140,7 +140,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context Block res_block = sample_block.cloneWithColumns(std::move(res_columns)); - auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), columns, res_block); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index e9fcbb219a3..c53d26a794b 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -48,7 +48,7 @@ StoragePtr TableFunctionView::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto columns = getActualTableStructure(context); - auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns, ""); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), create, columns, ""); res->startup(); return res; } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 9fd14eec4af..fdc8c4ac911 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -39,7 +39,7 @@ StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_fun UInt64 length = evaluateArgument(context, arguments[0]); - auto res = StorageSystemZeros::create(StorageID(getDatabaseName(), table_name), multithreaded, length); + auto res = std::make_shared(StorageID(getDatabaseName(), table_name), multithreaded, length); res->startup(); return res; }