From 974a1e4fa503cf693ebdc6f3b8e38688bf739b8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 4 Nov 2017 00:50:22 +0300 Subject: [PATCH] Attempt to change ext::shared_ptr_helper (incomplete) [#CLICKHOUSE-2]. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- .../tests/in_join_subqueries_preprocessor.cpp | 2 +- dbms/src/Storages/StorageBuffer.h | 25 ++++---- dbms/src/Storages/StorageDictionary.cpp | 10 +-- dbms/src/Storages/StorageDictionary.h | 20 +++--- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 50 +++++++-------- dbms/src/Storages/StorageFile.h | 36 ++++++----- dbms/src/Storages/StorageJoin.h | 25 ++------ dbms/src/Storages/StorageKafka.h | 10 +-- dbms/src/Storages/StorageLog.h | 2 +- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMemory.cpp | 8 --- dbms/src/Storages/StorageMemory.h | 6 +- dbms/src/Storages/StorageMerge.h | 2 +- dbms/src/Storages/StorageMergeTree.h | 2 +- dbms/src/Storages/StorageNull.h | 2 +- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- dbms/src/Storages/StorageSet.h | 15 +---- dbms/src/Storages/StorageStripeLog.h | 2 +- dbms/src/Storages/StorageTinyLog.h | 2 +- dbms/src/Storages/StorageView.h | 2 +- .../System/StorageSystemAsynchronousMetrics.h | 2 +- .../System/StorageSystemBuildOptions.h | 2 +- .../Storages/System/StorageSystemClusters.h | 2 +- .../Storages/System/StorageSystemColumns.h | 2 +- .../Storages/System/StorageSystemDatabases.h | 2 +- .../System/StorageSystemDictionaries.h | 2 +- .../src/Storages/System/StorageSystemEvents.h | 2 +- .../Storages/System/StorageSystemFunctions.h | 2 +- .../Storages/System/StorageSystemGraphite.h | 2 +- .../src/Storages/System/StorageSystemMerges.h | 2 +- .../Storages/System/StorageSystemMetrics.h | 2 +- .../Storages/System/StorageSystemNumbers.h | 2 +- dbms/src/Storages/System/StorageSystemOne.h | 2 +- dbms/src/Storages/System/StorageSystemParts.h | 2 +- .../Storages/System/StorageSystemProcesses.h | 2 +- .../Storages/System/StorageSystemReplicas.h | 2 +- .../System/StorageSystemReplicationQueue.h | 2 +- .../Storages/System/StorageSystemSettings.h | 2 +- .../src/Storages/System/StorageSystemTables.h | 2 +- .../Storages/System/StorageSystemZooKeeper.h | 2 +- .../libcommon/include/ext/shared_ptr_helper.h | 61 ++----------------- 43 files changed, 123 insertions(+), 207 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 1487c8c2512..0c4b278dda0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -796,7 +796,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t Block sample = interpreter->getSampleBlock(); NamesAndTypesListPtr columns = std::make_shared(sample.getColumnsList()); - StoragePtr external_storage = StorageMemory::create(external_table_name, columns); + StoragePtr external_storage = StorageMemory::create(external_table_name, columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}); external_storage->startup(); /** There are two ways to perform distributed GLOBAL subqueries. diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index da452cec5ca..6dc12ad1b16 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -30,7 +30,7 @@ namespace DB /// Simplified version of the StorageDistributed class. class StorageDistributedFake : public ext::shared_ptr_helper, public DB::IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "DistributedFake"; } diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 0e67dc4fb02..efef203b6f3 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -38,7 +38,7 @@ class Context; */ class StorageBuffer : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class BufferBlockInputStream; friend class BufferBlockOutputStream; @@ -112,17 +112,6 @@ private: /// Resets data by timeout. std::thread flush_thread; - /** 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 std::string & name_, NamesAndTypesListPtr columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_, - Context & context_, - size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, - const String & destination_database_, const String & destination_table_); - void flushAllBuffers(bool check_thresholds = true); /// Reset the buffer. If check_thresholds is set - resets only if thresholds are exceeded. void flushBuffer(Buffer & buffer, bool check_thresholds); @@ -133,6 +122,18 @@ private: void writeBlockToDestination(const Block & block, StoragePtr table); void flushThread(); + +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 std::string & name_, NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + Context & context_, + size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, + const String & destination_database_, const String & destination_table_); }; } diff --git a/dbms/src/Storages/StorageDictionary.cpp b/dbms/src/Storages/StorageDictionary.cpp index 2746b6a9861..f32447143af 100644 --- a/dbms/src/Storages/StorageDictionary.cpp +++ b/dbms/src/Storages/StorageDictionary.cpp @@ -33,8 +33,9 @@ StoragePtr StorageDictionary::create( const auto & dictionary = context.getExternalDictionaries().getDictionary(dictionary_name); const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - return make_shared(table_name, columns, materialized_columns, alias_columns, - column_defaults, dictionary_structure, dictionary_name); + return ext::shared_ptr_helper::create( + table_name, columns, materialized_columns, alias_columns, + column_defaults, dictionary_structure, dictionary_name); } StoragePtr StorageDictionary::create( @@ -46,8 +47,9 @@ StoragePtr StorageDictionary::create( const DictionaryStructure & dictionary_structure, const String & dictionary_name) { - return make_shared(table_name, columns, materialized_columns, alias_columns, - column_defaults, dictionary_structure, dictionary_name); + return ext::shared_ptr_helper::create( + table_name, columns, materialized_columns, alias_columns, + column_defaults, dictionary_structure, dictionary_name); } StorageDictionary::StorageDictionary( diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index 2e0a2c43d83..4c0d8ae6b93 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -19,7 +19,7 @@ class ExternalDictionaries; class StorageDictionary : private ext::shared_ptr_helper, public IStorage { - friend class ext::shared_ptr_helper; + friend struct ext::shared_ptr_helper; public: static StoragePtr create(const String & table_name_, @@ -59,14 +59,6 @@ private: String dictionary_name; Poco::Logger * logger; - StorageDictionary(const String & table_name_, - NamesAndTypesListPtr columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_, - const DictionaryStructure & dictionary_structure_, - const String & dictionary_name_); - void checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionaryStructure) const; template @@ -86,5 +78,15 @@ private: } return description.substr(2, description.size()); } + +protected: + StorageDictionary(const String & table_name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + const DictionaryStructure & dictionary_structure_, + const String & dictionary_name_); }; + } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4db43cb17bf..f053bba8679 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -177,7 +177,7 @@ StoragePtr StorageDistributed::createWithOwnCluster( ClusterPtr & owned_cluster_, const Context & context_) { - auto res = make_shared( + auto res = ext::shared_ptr_helper::create( name_, columns_, remote_database_, remote_table_, String{}, context_); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 6c6fd662eff..a8b5f25e276 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -27,7 +27,7 @@ class StorageDistributedDirectoryMonitor; */ class StorageDistributed : public ext::shared_ptr_helper, public IStorage { - friend class ext::shared_ptr_helper; + friend struct ext::shared_ptr_helper; friend class DistributedBlockOutputStream; friend class StorageDistributedDirectoryMonitor; @@ -92,30 +92,6 @@ public: std::string getRemoteTableName() const { return remote_table; } std::string getClusterName() const { return cluster_name; } /// Returns empty string if tables is used by TableFunctionRemote -private: - StorageDistributed( - const std::string & name_, - NamesAndTypesListPtr columns_, - const String & remote_database_, - const String & remote_table_, - const String & cluster_name_, - const Context & context_, - const ASTPtr & sharding_key_ = nullptr, - const String & data_path_ = String{}); - - StorageDistributed( - const std::string & name_, - NamesAndTypesListPtr columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_, - const String & remote_database_, - const String & remote_table_, - const String & cluster_name_, - const Context & context_, - const ASTPtr & sharding_key_ = nullptr, - const String & data_path_ = String{}); - /// create directory monitors for each existing subdirectory void createDirectoryMonitors(); /// ensure directory monitor thread creation by subdirectory name @@ -159,6 +135,30 @@ private: /// Used for global monotonic ordering of files to send. SimpleIncrement file_names_increment; + +protected: + StorageDistributed( + const std::string & name_, + NamesAndTypesListPtr columns_, + const String & remote_database_, + const String & remote_table_, + const String & cluster_name_, + const Context & context_, + const ASTPtr & sharding_key_ = nullptr, + const String & data_path_ = String{}); + + StorageDistributed( + const std::string & name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + const String & remote_database_, + const String & remote_table_, + const String & cluster_name_, + const Context & context_, + const ASTPtr & sharding_key_ = nullptr, + const String & data_path_ = String{}); }; } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 3b14145dfe2..6901af0d3ef 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -20,24 +20,6 @@ class StorageFileBlockOutputStream; class StorageFile : public IStorage { public: - - /** there are three options (ordered by priority): - - use specified file descriptor if (fd >= 0) - - use specified table_path if it isn't empty - - create own tabale inside data/db/table/ - */ - StorageFile( - const std::string & table_path_, - int table_fd_, - const std::string & db_dir_path, - const std::string & table_name_, - const std::string & format_name_, - const NamesAndTypesListPtr & columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_, - Context & context_); - static StoragePtr create( const std::string & table_path, int table_fd, @@ -89,10 +71,26 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; protected: - friend class StorageFileBlockInputStream; friend class StorageFileBlockOutputStream; + /** there are three options (ordered by priority): + - use specified file descriptor if (fd >= 0) + - use specified table_path if it isn't empty + - create own tabale inside data/db/table/ + */ + StorageFile( + const std::string & table_path_, + int table_fd_, + const std::string & db_dir_path, + const std::string & table_name_, + const std::string & format_name_, + const NamesAndTypesListPtr & columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + Context & context_); + private: std::string table_name; diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index fa83dbd846f..749b5107bbb 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -22,25 +22,9 @@ using JoinPtr = std::shared_ptr; */ class StorageJoin : public ext::shared_ptr_helper, public StorageSetOrJoinBase { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: - static StoragePtr create( - const String & path_, - const String & name_, - const Names & key_names_, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, - NamesAndTypesListPtr columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_) - { - return ext::shared_ptr_helper::make_shared( - path_, name_, key_names_, kind_, strictness_, - columns_, materialized_columns_, alias_columns_, column_defaults_ - ); - } - String getName() const override { return "Join"; } /// Access the innards. @@ -56,6 +40,10 @@ private: JoinPtr join; + void insertBlock(const Block & block) override; + size_t getSize() const override; + +protected: StorageJoin( const String & path_, const String & name_, @@ -65,9 +53,6 @@ private: const NamesAndTypesList & materialized_columns_, const NamesAndTypesList & alias_columns_, const ColumnDefaults & column_defaults_); - - void insertBlock(const Block & block) override; - size_t getSize() const override; }; } diff --git a/dbms/src/Storages/StorageKafka.h b/dbms/src/Storages/StorageKafka.h index 096a90f21a5..0612d5740cf 100644 --- a/dbms/src/Storages/StorageKafka.h +++ b/dbms/src/Storages/StorageKafka.h @@ -23,7 +23,7 @@ class StorageKafka; */ class StorageKafka : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class KafkaBlockInputStream; friend class KafkaBlockOutputStream; @@ -53,7 +53,6 @@ public: void updateDependencies() override; - private: String table_name; String database_name; @@ -71,6 +70,10 @@ private: std::atomic is_cancelled{false}; std::thread stream_thread; + void streamThread(); + void streamToViews(); + +protected: StorageKafka( const std::string & table_name_, const std::string & database_name_, @@ -81,9 +84,6 @@ private: const ColumnDefaults & column_defaults_, const String & brokers_, const String & group_, const Names & topics_, const String & format_name_, const String & schema_name_); - - void streamThread(); - void streamToViews(); }; } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index 71c743fa3dd..d8c18458cea 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -39,7 +39,7 @@ using Marks = std::vector; */ class StorageLog : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class LogBlockInputStream; friend class LogBlockOutputStream; diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index c4a47e31671..202372272fc 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -14,7 +14,7 @@ using ASTPtr = std::shared_ptr; class StorageMaterializedView : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "MaterializedView"; } diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index e3b3d53d6f8..81ecfac68a9 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -74,14 +74,6 @@ private: }; -StorageMemory::StorageMemory( - const std::string & name_, - NamesAndTypesListPtr columns_) - : name(name_), columns(columns_) -{ -} - - StorageMemory::StorageMemory( const std::string & name_, NamesAndTypesListPtr columns_, diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 32e476e3973..64fde3076a3 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -22,7 +22,7 @@ class StorageMemory; */ class StorageMemory : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class MemoryBlockInputStream; friend class MemoryBlockOutputStream; @@ -56,10 +56,6 @@ private: std::mutex mutex; - StorageMemory( - const std::string & name_, - NamesAndTypesListPtr columns_); - StorageMemory( const std::string & name_, NamesAndTypesListPtr columns_, diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 835f60a586e..869e3aa4bc8 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -14,7 +14,7 @@ namespace DB */ class StorageMerge : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Merge"; } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index f98425c12e5..235850ffa83 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -18,7 +18,7 @@ namespace DB */ class StorageMergeTree : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class MergeTreeBlockOutputStream; public: diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index fba3e8bd8eb..28600f20477 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -16,7 +16,7 @@ namespace DB */ class StorageNull : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Null"; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 1cfea25a948..9ed28d32fb2 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -72,7 +72,7 @@ namespace DB class StorageReplicatedMergeTree : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 52fff963484..48fd59129e8 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -16,7 +16,7 @@ using SetPtr = std::shared_ptr; */ class StorageSetOrJoinBase : public ext::shared_ptr_helper, public IStorage { - friend class ext::shared_ptr_helper; + friend struct ext::shared_ptr_helper; friend class SetOrJoinBlockOutputStream; public: @@ -61,20 +61,9 @@ private: */ class StorageSet : public ext::shared_ptr_helper, public StorageSetOrJoinBase { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: - static StoragePtr create( - const String & path_, - const String & name_, - NamesAndTypesListPtr columns_, - const NamesAndTypesList & materialized_columns_, - const NamesAndTypesList & alias_columns_, - const ColumnDefaults & column_defaults_) - { - return ext::shared_ptr_helper::make_shared(path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_); - } - String getName() const override { return "Set"; } /// Access the insides. diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 206b140bd9d..d87da8e46b3 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -21,7 +21,7 @@ namespace DB */ class StorageStripeLog : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class StripeLogBlockInputStream; friend class StripeLogBlockOutputStream; diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 2540642a8da..c956c996fcf 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -20,7 +20,7 @@ namespace DB */ class StorageTinyLog : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; friend class TinyLogBlockInputStream; friend class TinyLogBlockOutputStream; diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index cffac728aff..51cbc197511 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -13,7 +13,7 @@ using ASTPtr = std::shared_ptr; class StorageView : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "View"; } diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h index 7224c158b9c..99a47ca80b4 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -15,7 +15,7 @@ class Context; */ class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemAsynchronousMetrics"; } diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.h b/dbms/src/Storages/System/StorageSystemBuildOptions.h index 5f212bb79a2..f10e743128b 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.h +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemBuildOptions : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemBuildOptions"; } diff --git a/dbms/src/Storages/System/StorageSystemClusters.h b/dbms/src/Storages/System/StorageSystemClusters.h index 24c28221392..88b1710aa8c 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.h +++ b/dbms/src/Storages/System/StorageSystemClusters.h @@ -15,7 +15,7 @@ class Context; */ class StorageSystemClusters : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: StorageSystemClusters(const std::string & name_); diff --git a/dbms/src/Storages/System/StorageSystemColumns.h b/dbms/src/Storages/System/StorageSystemColumns.h index 3e00f380375..ec1b7432b9c 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.h +++ b/dbms/src/Storages/System/StorageSystemColumns.h @@ -13,7 +13,7 @@ class Context; */ class StorageSystemColumns : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemColumns"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index a93b107b132..b761981365d 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemDatabases : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDatabases"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.h b/dbms/src/Storages/System/StorageSystemDictionaries.h index 3838681ee45..09d6a89a1d6 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.h +++ b/dbms/src/Storages/System/StorageSystemDictionaries.h @@ -12,7 +12,7 @@ class Context; class StorageSystemDictionaries : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDictionaries"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index 05ab28b83af..1cc9d025539 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemEvents : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemEvents"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index ae4769eb1f9..a88de4a3f43 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -15,7 +15,7 @@ class Context; */ class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemFunctions"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.h b/dbms/src/Storages/System/StorageSystemGraphite.h index b587a6a823c..154743b38d8 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.h +++ b/dbms/src/Storages/System/StorageSystemGraphite.h @@ -9,7 +9,7 @@ namespace DB /// Provides information about Graphite configuration. class StorageSystemGraphite : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemGraphite"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemMerges.h b/dbms/src/Storages/System/StorageSystemMerges.h index 3c8b1b01019..a1e50e0721f 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.h +++ b/dbms/src/Storages/System/StorageSystemMerges.h @@ -12,7 +12,7 @@ class Context; class StorageSystemMerges : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMerges"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemMetrics.h b/dbms/src/Storages/System/StorageSystemMetrics.h index c19f7ef18ae..6d7a12f329f 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.h +++ b/dbms/src/Storages/System/StorageSystemMetrics.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemMetrics : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMetrics"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index d151af9cddf..7170c605a9c 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -21,7 +21,7 @@ class Context; */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemNumbers"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemOne.h b/dbms/src/Storages/System/StorageSystemOne.h index 6134789cbed..e3c38dcb119 100644 --- a/dbms/src/Storages/System/StorageSystemOne.h +++ b/dbms/src/Storages/System/StorageSystemOne.h @@ -17,7 +17,7 @@ class Context; */ class StorageSystemOne : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemOne"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemParts.h b/dbms/src/Storages/System/StorageSystemParts.h index 17c6a7f4e5c..733ecb790d1 100644 --- a/dbms/src/Storages/System/StorageSystemParts.h +++ b/dbms/src/Storages/System/StorageSystemParts.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemParts : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemParts"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index c86aacdf990..2675aeda127 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemProcesses"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemReplicas.h b/dbms/src/Storages/System/StorageSystemReplicas.h index 437cdcbeea8..8067b3ea55c 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.h +++ b/dbms/src/Storages/System/StorageSystemReplicas.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemReplicas : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicas"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.h b/dbms/src/Storages/System/StorageSystemReplicationQueue.h index 82f621f13eb..11e5e823ca6 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.h +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemReplicationQueue : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicationQueue"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemSettings.h b/dbms/src/Storages/System/StorageSystemSettings.h index e81e80b2814..a7739d03c0e 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.h +++ b/dbms/src/Storages/System/StorageSystemSettings.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemSettings : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemSettings"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index 324e5da8454..04ad0665f2d 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemTables : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemTables"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.h b/dbms/src/Storages/System/StorageSystemZooKeeper.h index 445f04f51f7..5347c95e0a3 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.h +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.h @@ -14,7 +14,7 @@ class Context; */ class StorageSystemZooKeeper : public ext::shared_ptr_helper, public IStorage { -friend class ext::shared_ptr_helper; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemZooKeeper"; } std::string getTableName() const override { return name; } diff --git a/libs/libcommon/include/ext/shared_ptr_helper.h b/libs/libcommon/include/ext/shared_ptr_helper.h index 6f85fcf980a..7393865872f 100644 --- a/libs/libcommon/include/ext/shared_ptr_helper.h +++ b/libs/libcommon/include/ext/shared_ptr_helper.h @@ -5,66 +5,17 @@ namespace ext { -/** Class AllocateShared allow to make std::shared_ptr from T with private constructor. - * Derive your T class from shared_ptr_helper, define him as friend and call allocate_shared()/make_shared() method. +/** Allows to make std::shared_ptr from T with private constructor. + * Derive your T class from shared_ptr_helper and define him as friend. */ template -class shared_ptr_helper +struct shared_ptr_helper { -protected: - typedef typename std::remove_const::type TNoConst; - - template - struct Deleter - { - void operator()(typename TAlloc::value_type * ptr) - { - using AllocTraits = std::allocator_traits; - ptr->~TNoConst(); - AllocTraits::deallocate(alloc, ptr, 1); - } - - TAlloc alloc; - }; - - /// see std::allocate_shared - template - static std::shared_ptr allocate_shared(const TAlloc & alloc, TArgs &&... args) - { - using AllocTraits = std::allocator_traits; - TAlloc alloc_copy(alloc); - - auto ptr = AllocTraits::allocate(alloc_copy, 1); - - try - { - new (ptr) TNoConst(std::forward(args)...); - } - catch (...) - { - AllocTraits::deallocate(alloc_copy, ptr, 1); - throw; - } - - return std::shared_ptr( - ptr, - Deleter(), - alloc_copy); - } - template - static std::shared_ptr make_shared(TArgs &&... args) + static auto create(TArgs &&... args) { - return allocate_shared(std::allocator(), std::forward(args)...); - } - -public: - - /// Default implementation of 'create' method just use make_shared. - template - static std::shared_ptr create(TArgs &&... args) - { - return make_shared(std::forward(args)...); + struct Local : T { using T::T; }; + return std::make_shared(std::forward(args)...); } };