mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Attempt to change ext::shared_ptr_helper (incomplete) [#CLICKHOUSE-2].
This commit is contained in:
parent
03b6aa5d21
commit
974a1e4fa5
@ -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.
|
||||
|
@ -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"; }
|
||||
|
@ -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_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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_);
|
||||
|
||||
|
@ -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{});
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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"; }
|
||||
|
@ -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_,
|
||||
|
@ -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_,
|
||||
|
@ -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"; }
|
||||
|
@ -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:
|
||||
|
@ -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"; }
|
||||
|
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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"; }
|
||||
|
@ -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"; }
|
||||
|
@ -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"; }
|
||||
|
@ -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_);
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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)...);
|
||||
}
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user