Attempt to change ext::shared_ptr_helper (incomplete) [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-11-04 00:50:22 +03:00 committed by alexey-milovidov
parent 03b6aa5d21
commit 974a1e4fa5
43 changed files with 123 additions and 207 deletions

View File

@ -796,7 +796,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
Block sample = interpreter->getSampleBlock();
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>(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.

View File

@ -30,7 +30,7 @@ namespace DB
/// Simplified version of the StorageDistributed class.
class StorageDistributedFake : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
{
friend class ext::shared_ptr_helper<StorageDistributedFake>;
friend struct ext::shared_ptr_helper<StorageDistributedFake>;
public:
std::string getName() const override { return "DistributedFake"; }

View File

@ -38,7 +38,7 @@ class Context;
*/
class StorageBuffer : public ext::shared_ptr_helper<StorageBuffer>, public IStorage
{
friend class ext::shared_ptr_helper<StorageBuffer>;
friend struct ext::shared_ptr_helper<StorageBuffer>;
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_);
};
}

View File

@ -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<StorageDictionary>::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<StorageDictionary>::create(
table_name, columns, materialized_columns, alias_columns,
column_defaults, dictionary_structure, dictionary_name);
}
StorageDictionary::StorageDictionary(

View File

@ -19,7 +19,7 @@ class ExternalDictionaries;
class StorageDictionary : private ext::shared_ptr_helper<StorageDictionary>, public IStorage
{
friend class ext::shared_ptr_helper<StorageDictionary>;
friend struct ext::shared_ptr_helper<StorageDictionary>;
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 <typename ForwardIterator>
@ -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_);
};
}

View File

@ -177,7 +177,7 @@ StoragePtr StorageDistributed::createWithOwnCluster(
ClusterPtr & owned_cluster_,
const Context & context_)
{
auto res = make_shared(
auto res = ext::shared_ptr_helper<StorageDictionary>::create(
name_, columns_, remote_database_,
remote_table_, String{}, context_);

View File

@ -27,7 +27,7 @@ class StorageDistributedDirectoryMonitor;
*/
class StorageDistributed : public ext::shared_ptr_helper<StorageDistributed>, public IStorage
{
friend class ext::shared_ptr_helper<StorageDistributed>;
friend struct ext::shared_ptr_helper<StorageDistributed>;
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{});
};
}

View File

@ -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;

View File

@ -22,25 +22,9 @@ using JoinPtr = std::shared_ptr<Join>;
*/
class StorageJoin : public ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
{
friend class ext::shared_ptr_helper<StorageJoin>;
friend struct ext::shared_ptr_helper<StorageJoin>;
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<StorageJoin>::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;
};
}

View File

@ -23,7 +23,7 @@ class StorageKafka;
*/
class StorageKafka : public ext::shared_ptr_helper<StorageKafka>, public IStorage
{
friend class ext::shared_ptr_helper<StorageKafka>;
friend struct ext::shared_ptr_helper<StorageKafka>;
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<bool> 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();
};
}

View File

@ -39,7 +39,7 @@ using Marks = std::vector<Mark>;
*/
class StorageLog : public ext::shared_ptr_helper<StorageLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageLog>;
friend struct ext::shared_ptr_helper<StorageLog>;
friend class LogBlockInputStream;
friend class LogBlockOutputStream;

View File

@ -14,7 +14,7 @@ using ASTPtr = std::shared_ptr<IAST>;
class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterializedView>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMaterializedView>;
friend struct ext::shared_ptr_helper<StorageMaterializedView>;
public:
std::string getName() const override { return "MaterializedView"; }

View File

@ -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_,

View File

@ -22,7 +22,7 @@ class StorageMemory;
*/
class StorageMemory : public ext::shared_ptr_helper<StorageMemory>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMemory>;
friend struct ext::shared_ptr_helper<StorageMemory>;
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_,

View File

@ -14,7 +14,7 @@ namespace DB
*/
class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMerge>;
friend struct ext::shared_ptr_helper<StorageMerge>;
public:
std::string getName() const override { return "Merge"; }

View File

@ -18,7 +18,7 @@ namespace DB
*/
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMergeTree>;
friend struct ext::shared_ptr_helper<StorageMergeTree>;
friend class MergeTreeBlockOutputStream;
public:

View File

@ -16,7 +16,7 @@ namespace DB
*/
class StorageNull : public ext::shared_ptr_helper<StorageNull>, public IStorage
{
friend class ext::shared_ptr_helper<StorageNull>;
friend struct ext::shared_ptr_helper<StorageNull>;
public:
std::string getName() const override { return "Null"; }

View File

@ -72,7 +72,7 @@ namespace DB
class StorageReplicatedMergeTree : public ext::shared_ptr_helper<StorageReplicatedMergeTree>, public IStorage
{
friend class ext::shared_ptr_helper<StorageReplicatedMergeTree>;
friend struct ext::shared_ptr_helper<StorageReplicatedMergeTree>;
public:
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.

View File

@ -16,7 +16,7 @@ using SetPtr = std::shared_ptr<Set>;
*/
class StorageSetOrJoinBase : public ext::shared_ptr_helper<StorageSetOrJoinBase>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSetOrJoinBase>;
friend struct ext::shared_ptr_helper<StorageSetOrJoinBase>;
friend class SetOrJoinBlockOutputStream;
public:
@ -61,20 +61,9 @@ private:
*/
class StorageSet : public ext::shared_ptr_helper<StorageSet>, public StorageSetOrJoinBase
{
friend class ext::shared_ptr_helper<StorageSet>;
friend struct ext::shared_ptr_helper<StorageSet>;
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<StorageSet>::make_shared(path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
}
String getName() const override { return "Set"; }
/// Access the insides.

View File

@ -21,7 +21,7 @@ namespace DB
*/
class StorageStripeLog : public ext::shared_ptr_helper<StorageStripeLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageStripeLog>;
friend struct ext::shared_ptr_helper<StorageStripeLog>;
friend class StripeLogBlockInputStream;
friend class StripeLogBlockOutputStream;

View File

@ -20,7 +20,7 @@ namespace DB
*/
class StorageTinyLog : public ext::shared_ptr_helper<StorageTinyLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageTinyLog>;
friend struct ext::shared_ptr_helper<StorageTinyLog>;
friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream;

View File

@ -13,7 +13,7 @@ using ASTPtr = std::shared_ptr<IAST>;
class StorageView : public ext::shared_ptr_helper<StorageView>, public IStorage
{
friend class ext::shared_ptr_helper<StorageView>;
friend struct ext::shared_ptr_helper<StorageView>;
public:
std::string getName() const override { return "View"; }

View File

@ -15,7 +15,7 @@ class Context;
*/
class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>;
friend struct ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>;
public:
std::string getName() const override { return "SystemAsynchronousMetrics"; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemBuildOptions : public ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemBuildOptions>;
friend struct ext::shared_ptr_helper<StorageSystemBuildOptions>;
public:
std::string getName() const override { return "SystemBuildOptions"; }

View File

@ -15,7 +15,7 @@ class Context;
*/
class StorageSystemClusters : public ext::shared_ptr_helper<StorageSystemClusters>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemClusters>;
friend struct ext::shared_ptr_helper<StorageSystemClusters>;
public:
StorageSystemClusters(const std::string & name_);

View File

@ -13,7 +13,7 @@ class Context;
*/
class StorageSystemColumns : public ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemColumns>;
friend struct ext::shared_ptr_helper<StorageSystemColumns>;
public:
std::string getName() const override { return "SystemColumns"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemDatabases : public ext::shared_ptr_helper<StorageSystemDatabases>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemDatabases>;
friend struct ext::shared_ptr_helper<StorageSystemDatabases>;
public:
std::string getName() const override { return "SystemDatabases"; }
std::string getTableName() const override { return name; }

View File

@ -12,7 +12,7 @@ class Context;
class StorageSystemDictionaries : public ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemDictionaries>;
friend struct ext::shared_ptr_helper<StorageSystemDictionaries>;
public:
std::string getName() const override { return "SystemDictionaries"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemEvents : public ext::shared_ptr_helper<StorageSystemEvents>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemEvents>;
friend struct ext::shared_ptr_helper<StorageSystemEvents>;
public:
std::string getName() const override { return "SystemEvents"; }
std::string getTableName() const override { return name; }

View File

@ -15,7 +15,7 @@ class Context;
*/
class StorageSystemFunctions : public ext::shared_ptr_helper<StorageSystemFunctions>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemFunctions>;
friend struct ext::shared_ptr_helper<StorageSystemFunctions>;
public:
std::string getName() const override { return "SystemFunctions"; }
std::string getTableName() const override { return name; }

View File

@ -9,7 +9,7 @@ namespace DB
/// Provides information about Graphite configuration.
class StorageSystemGraphite : public ext::shared_ptr_helper<StorageSystemGraphite>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemGraphite>;
friend struct ext::shared_ptr_helper<StorageSystemGraphite>;
public:
std::string getName() const override { return "SystemGraphite"; }
std::string getTableName() const override { return name; }

View File

@ -12,7 +12,7 @@ class Context;
class StorageSystemMerges : public ext::shared_ptr_helper<StorageSystemMerges>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemMerges>;
friend struct ext::shared_ptr_helper<StorageSystemMerges>;
public:
std::string getName() const override { return "SystemMerges"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemMetrics : public ext::shared_ptr_helper<StorageSystemMetrics>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemMetrics>;
friend struct ext::shared_ptr_helper<StorageSystemMetrics>;
public:
std::string getName() const override { return "SystemMetrics"; }
std::string getTableName() const override { return name; }

View File

@ -21,7 +21,7 @@ class Context;
*/
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemNumbers>;
friend struct ext::shared_ptr_helper<StorageSystemNumbers>;
public:
std::string getName() const override { return "SystemNumbers"; }
std::string getTableName() const override { return name; }

View File

@ -17,7 +17,7 @@ class Context;
*/
class StorageSystemOne : public ext::shared_ptr_helper<StorageSystemOne>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemOne>;
friend struct ext::shared_ptr_helper<StorageSystemOne>;
public:
std::string getName() const override { return "SystemOne"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemParts : public ext::shared_ptr_helper<StorageSystemParts>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemParts>;
friend struct ext::shared_ptr_helper<StorageSystemParts>;
public:
std::string getName() const override { return "SystemParts"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemProcesses : public ext::shared_ptr_helper<StorageSystemProcesses>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemProcesses>;
friend struct ext::shared_ptr_helper<StorageSystemProcesses>;
public:
std::string getName() const override { return "SystemProcesses"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemReplicas : public ext::shared_ptr_helper<StorageSystemReplicas>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemReplicas>;
friend struct ext::shared_ptr_helper<StorageSystemReplicas>;
public:
std::string getName() const override { return "SystemReplicas"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemReplicationQueue : public ext::shared_ptr_helper<StorageSystemReplicationQueue>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemReplicationQueue>;
friend struct ext::shared_ptr_helper<StorageSystemReplicationQueue>;
public:
std::string getName() const override { return "SystemReplicationQueue"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemSettings : public ext::shared_ptr_helper<StorageSystemSettings>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemSettings>;
friend struct ext::shared_ptr_helper<StorageSystemSettings>;
public:
std::string getName() const override { return "SystemSettings"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemTables : public ext::shared_ptr_helper<StorageSystemTables>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemTables>;
friend struct ext::shared_ptr_helper<StorageSystemTables>;
public:
std::string getName() const override { return "SystemTables"; }
std::string getTableName() const override { return name; }

View File

@ -14,7 +14,7 @@ class Context;
*/
class StorageSystemZooKeeper : public ext::shared_ptr_helper<StorageSystemZooKeeper>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemZooKeeper>;
friend struct ext::shared_ptr_helper<StorageSystemZooKeeper>;
public:
std::string getName() const override { return "SystemZooKeeper"; }
std::string getTableName() const override { return name; }

View File

@ -5,66 +5,17 @@
namespace ext
{
/** Class AllocateShared allow to make std::shared_ptr<T> from T with private constructor.
* Derive your T class from shared_ptr_helper<T>, define him as friend and call allocate_shared()/make_shared() method.
/** Allows to make std::shared_ptr<T> from T with private constructor.
* Derive your T class from shared_ptr_helper<T> and define him as friend.
*/
template <typename T>
class shared_ptr_helper
struct shared_ptr_helper
{
protected:
typedef typename std::remove_const<T>::type TNoConst;
template <typename TAlloc>
struct Deleter
{
void operator()(typename TAlloc::value_type * ptr)
{
using AllocTraits = std::allocator_traits<TAlloc>;
ptr->~TNoConst();
AllocTraits::deallocate(alloc, ptr, 1);
}
TAlloc alloc;
};
/// see std::allocate_shared
template <typename TAlloc, typename... TArgs>
static std::shared_ptr<T> allocate_shared(const TAlloc & alloc, TArgs &&... args)
{
using AllocTraits = std::allocator_traits<TAlloc>;
TAlloc alloc_copy(alloc);
auto ptr = AllocTraits::allocate(alloc_copy, 1);
try
{
new (ptr) TNoConst(std::forward<TArgs>(args)...);
}
catch (...)
{
AllocTraits::deallocate(alloc_copy, ptr, 1);
throw;
}
return std::shared_ptr<TNoConst>(
ptr,
Deleter<TAlloc>(),
alloc_copy);
}
template <typename... TArgs>
static std::shared_ptr<T> make_shared(TArgs &&... args)
static auto create(TArgs &&... args)
{
return allocate_shared(std::allocator<TNoConst>(), std::forward<TArgs>(args)...);
}
public:
/// Default implementation of 'create' method just use make_shared.
template <typename... TArgs>
static std::shared_ptr<T> create(TArgs &&... args)
{
return make_shared(std::forward<TArgs>(args)...);
struct Local : T { using T::T; };
return std::make_shared<Local>(std::forward<TArgs>(args)...);
}
};