use std::enable_shared_from_this for IStorage

This commit is contained in:
Yuri Dyachenko 2016-08-27 00:25:05 +03:00
parent 065b02494a
commit e5825f790f
64 changed files with 267 additions and 125 deletions

View File

@ -47,7 +47,7 @@ using StoragePtr = std::shared_ptr<IStorage>;
* - структура хранения данных (сжатие, etc.) * - структура хранения данных (сжатие, etc.)
* - конкуррентный доступ к данным (блокировки, etc.) * - конкуррентный доступ к данным (блокировки, etc.)
*/ */
class IStorage : private boost::noncopyable, public ITableDeclaration class IStorage : public std::enable_shared_from_this<IStorage>, private boost::noncopyable, public ITableDeclaration
{ {
public: public:
/// Основное имя типа таблицы (например, StorageMergeTree). /// Основное имя типа таблицы (например, StorageMergeTree).
@ -106,7 +106,7 @@ public:
*/ */
TableStructureReadLockPtr lockStructure(bool will_modify_data) TableStructureReadLockPtr lockStructure(bool will_modify_data)
{ {
TableStructureReadLockPtr res = std::make_shared<TableStructureReadLock>(thisPtr(), true, will_modify_data); TableStructureReadLockPtr res = std::make_shared<TableStructureReadLock>(shared_from_this(), true, will_modify_data);
if (is_dropped) if (is_dropped)
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
return res; return res;
@ -270,19 +270,6 @@ public:
*/ */
virtual void shutdown() {} virtual void shutdown() {}
/** Возвращает владеющий указатель на себя.
*/
std::shared_ptr<IStorage> thisPtr()
{
std::shared_ptr<IStorage> res = this_ptr.lock();
if (!res)
{
res.reset(this);
this_ptr = res;
}
return res;
}
bool is_dropped{false}; bool is_dropped{false};
/// Поддерживается ли индекс в секции IN /// Поддерживается ли индекс в секции IN
@ -293,10 +280,9 @@ public:
protected: protected:
using ITableDeclaration::ITableDeclaration; using ITableDeclaration::ITableDeclaration;
using std::enable_shared_from_this<IStorage>::shared_from_this;
private: private:
std::weak_ptr<IStorage> this_ptr;
/// Брать следующие два лока всегда нужно в этом порядке. /// Брать следующие два лока всегда нужно в этом порядке.
/** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree). /** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree).

View File

@ -3,6 +3,8 @@
#include <mutex> #include <mutex>
#include <thread> #include <thread>
#include <ext/share_ptr_helper.hpp>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/DataStreams/IBlockOutputStream.h> #include <DB/DataStreams/IBlockOutputStream.h>
@ -31,8 +33,9 @@ namespace DB
* При уничтожении таблицы типа Buffer и при завершении работы, все данные сбрасываются. * При уничтожении таблицы типа Buffer и при завершении работы, все данные сбрасываются.
* Данные в буфере не реплицируются, не логгируются на диск, не индексируются. При грубом перезапуске сервера, данные пропадают. * Данные в буфере не реплицируются, не логгируются на диск, не индексируются. При грубом перезапуске сервера, данные пропадают.
*/ */
class StorageBuffer : public IStorage class StorageBuffer : private ext::share_ptr_helper<StorageBuffer>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageBuffer>;
friend class BufferBlockInputStream; friend class BufferBlockInputStream;
friend class BufferBlockOutputStream; friend class BufferBlockOutputStream;

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Databases/IDatabase.h> #include <DB/Databases/IDatabase.h>
@ -13,8 +15,10 @@ class DatabaseCloud;
/** Облачная таблица. Может находиться только в облачной базе данных. /** Облачная таблица. Может находиться только в облачной базе данных.
* При записи в таблицу, данные записываются в локальные таблицы на нескольких серверах облака. * При записи в таблицу, данные записываются в локальные таблицы на нескольких серверах облака.
*/ */
class StorageCloud : public IStorage class StorageCloud : private ext::share_ptr_helper<StorageCloud>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageCloud>;
public: public:
static StoragePtr create( static StoragePtr create(
DatabasePtr & database_ptr_, DatabasePtr & database_ptr_,

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Client/ConnectionPool.h> #include <DB/Client/ConnectionPool.h>
#include <DB/Client/ConnectionPoolWithFailover.h> #include <DB/Client/ConnectionPoolWithFailover.h>
@ -18,8 +20,9 @@ namespace DB
* Можно передать один адрес, а не несколько. * Можно передать один адрес, а не несколько.
* В этом случае, таблицу можно считать удалённой, а не распределённой. * В этом случае, таблицу можно считать удалённой, а не распределённой.
*/ */
class StorageDistributed : public IStorage class StorageDistributed : private ext::share_ptr_helper<StorageDistributed>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageDistributed>;
friend class DistributedBlockOutputStream; friend class DistributedBlockOutputStream;
friend class DirectoryMonitor; friend class DirectoryMonitor;

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/StorageSet.h> #include <DB/Storages/StorageSet.h>
#include <DB/Interpreters/Join.h> #include <DB/Interpreters/Join.h>
@ -14,8 +16,10 @@ namespace DB
* *
* При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER ...). * При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER ...).
*/ */
class StorageJoin : public StorageSetOrJoinBase class StorageJoin : private ext::share_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
{ {
friend class ext::share_ptr_helper<StorageJoin>;
public: public:
static StoragePtr create( static StoragePtr create(
const String & path_, const String & path_,
@ -27,11 +31,10 @@ public:
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageJoin{ return ext::share_ptr_helper<StorageJoin>::make_shared(
path_, name_, path_, name_, key_names_, kind_, strictness_,
key_names_, kind_, strictness_, columns_, materialized_columns_, alias_columns_, column_defaults_
columns_, );
materialized_columns_, alias_columns_, column_defaults_})->thisPtr();
} }
String getName() const override { return "Join"; } String getName() const override { return "Join"; }

View File

@ -2,6 +2,8 @@
#include <map> #include <map>
#include <ext/share_ptr_helper.hpp>
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/RWLock.h> #include <Poco/RWLock.h>
@ -35,8 +37,9 @@ using Marks = std::vector<Mark>;
* Ключи не поддерживаются. * Ключи не поддерживаются.
* Данные хранятся в сжатом виде. * Данные хранятся в сжатом виде.
*/ */
class StorageLog : public IStorage class StorageLog : private ext::share_ptr_helper<StorageLog>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageLog>;
friend class LogBlockInputStream; friend class LogBlockInputStream;
friend class LogBlockOutputStream; friend class LogBlockOutputStream;

View File

@ -1,12 +1,16 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/StorageView.h> #include <DB/Storages/StorageView.h>
namespace DB namespace DB
{ {
class StorageMaterializedView : public StorageView { class StorageMaterializedView : private ext::share_ptr_helper<StorageMaterializedView>, public StorageView
{
friend class ext::share_ptr_helper<StorageMaterializedView>;
public: public:
static StoragePtr create( static StoragePtr create(

View File

@ -2,6 +2,8 @@
#include <mutex> #include <mutex>
#include <ext/share_ptr_helper.hpp>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/DataStreams/IBlockOutputStream.h> #include <DB/DataStreams/IBlockOutputStream.h>
@ -18,8 +20,9 @@ class StorageMemory;
* В нём не поддерживаются ключи. * В нём не поддерживаются ключи.
* Данные хранятся в виде набора блоков и никуда дополнительно не сохраняются. * Данные хранятся в виде набора блоков и никуда дополнительно не сохраняются.
*/ */
class StorageMemory : public IStorage class StorageMemory : private ext::share_ptr_helper<StorageMemory>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageMemory>;
friend class MemoryBlockInputStream; friend class MemoryBlockInputStream;
friend class MemoryBlockOutputStream; friend class MemoryBlockOutputStream;

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Common/OptimizedRegularExpression.h> #include <DB/Common/OptimizedRegularExpression.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -10,8 +12,10 @@ namespace DB
/** Таблица, представляющая собой объединение произвольного количества других таблиц. /** Таблица, представляющая собой объединение произвольного количества других таблиц.
* У всех таблиц должна быть одинаковая структура. * У всех таблиц должна быть одинаковая структура.
*/ */
class StorageMerge : public IStorage class StorageMerge : private ext::share_ptr_helper<StorageMerge>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageMerge>;
public: public:
static StoragePtr create( static StoragePtr create(
const std::string & name_, /// Имя таблицы. const std::string & name_, /// Имя таблицы.

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/MergeTree/MergeTreeData.h> #include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h> #include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h> #include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
@ -14,8 +16,9 @@ namespace DB
/** См. описание структуры данных в MergeTreeData. /** См. описание структуры данных в MergeTreeData.
*/ */
class StorageMergeTree : public IStorage class StorageMergeTree : private ext::share_ptr_helper<StorageMergeTree>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageMergeTree>;
friend class MergeTreeBlockOutputStream; friend class MergeTreeBlockOutputStream;
public: public:

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/DataStreams/NullBlockInputStream.h> #include <DB/DataStreams/NullBlockInputStream.h>
@ -12,8 +14,10 @@ namespace DB
/** При записи, ничего не делает. /** При записи, ничего не делает.
* При чтении, возвращает пустоту. * При чтении, возвращает пустоту.
*/ */
class StorageNull : public IStorage class StorageNull : private ext::share_ptr_helper<StorageNull>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageNull>;
public: public:
static StoragePtr create( static StoragePtr create(
const std::string & name_, const std::string & name_,
@ -22,7 +26,7 @@ public:
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageNull{name_, columns_, materialized_columns_, alias_columns_, column_defaults_})->thisPtr(); return make_shared(name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
} }
std::string getName() const override { return "Null"; } std::string getName() const override { return "Null"; }

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Storages/MergeTree/MergeTreeData.h> #include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Storages/MergeTree/MergeTreeDataMerger.h> #include <DB/Storages/MergeTree/MergeTreeDataMerger.h>
@ -67,8 +69,10 @@ namespace DB
* в качестве времени будет браться время создания соответствующего куска на какой-либо из реплик. * в качестве времени будет браться время создания соответствующего куска на какой-либо из реплик.
*/ */
class StorageReplicatedMergeTree : public IStorage class StorageReplicatedMergeTree : private ext::share_ptr_helper<StorageReplicatedMergeTree>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageReplicatedMergeTree>;
public: public:
/** Если !attach, либо создает новую таблицу в ZK, либо добавляет реплику в существующую таблицу. /** Если !attach, либо создает новую таблицу в ZK, либо добавляет реплику в существующую таблицу.
*/ */

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/IO/WriteBufferFromFile.h> #include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/CompressedWriteBuffer.h> #include <DB/IO/CompressedWriteBuffer.h>
#include <DB/DataStreams/NativeBlockOutputStream.h> #include <DB/DataStreams/NativeBlockOutputStream.h>
@ -14,8 +16,9 @@ namespace DB
/** Общая часть StorageSet и StorageJoin. /** Общая часть StorageSet и StorageJoin.
*/ */
class StorageSetOrJoinBase : public IStorage class StorageSetOrJoinBase : private ext::share_ptr_helper<StorageSetOrJoinBase>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSetOrJoinBase>;
friend class SetOrJoinBlockOutputStream; friend class SetOrJoinBlockOutputStream;
public: public:
@ -78,8 +81,10 @@ private:
* а также записаны в файл-бэкап, для восстановления после перезапуска. * а также записаны в файл-бэкап, для восстановления после перезапуска.
* Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части оператора IN. * Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части оператора IN.
*/ */
class StorageSet : public StorageSetOrJoinBase class StorageSet : private ext::share_ptr_helper<StorageSet>, public StorageSetOrJoinBase
{ {
friend class ext::share_ptr_helper<StorageSet>;
public: public:
static StoragePtr create( static StoragePtr create(
const String & path_, const String & path_,
@ -89,9 +94,7 @@ public:
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageSet{ return ext::share_ptr_helper<StorageSet>::make_shared(path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_})->thisPtr();
} }
String getName() const override { return "Set"; } String getName() const override { return "Set"; }

View File

@ -1,6 +1,9 @@
#pragma once #pragma once
#include <map> #include <map>
#include <ext/share_ptr_helper.hpp>
#include <Poco/File.h> #include <Poco/File.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -14,8 +17,9 @@ namespace DB
/** Реализует хранилище, подходящее для маленьких кусочков лога. /** Реализует хранилище, подходящее для маленьких кусочков лога.
* При этом, хранит все столбцы в одном файле формата Native, с расположенным рядом индексом. * При этом, хранит все столбцы в одном файле формата Native, с расположенным рядом индексом.
*/ */
class StorageStripeLog : public IStorage class StorageStripeLog : private ext::share_ptr_helper<StorageStripeLog>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageStripeLog>;
friend class StripeLogBlockInputStream; friend class StripeLogBlockInputStream;
friend class StripeLogBlockOutputStream; friend class StripeLogBlockOutputStream;

View File

@ -2,6 +2,8 @@
#include <map> #include <map>
#include <ext/share_ptr_helper.hpp>
#include <Poco/File.h> #include <Poco/File.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -15,8 +17,9 @@ namespace DB
/** Реализует хранилище, подходящее для маленьких кусочков лога. /** Реализует хранилище, подходящее для маленьких кусочков лога.
* Отличается от StorageLog отсутствием файлов с засечками. * Отличается от StorageLog отсутствием файлов с засечками.
*/ */
class StorageTinyLog : public IStorage class StorageTinyLog : private ext::share_ptr_helper<StorageTinyLog>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageTinyLog>;
friend class TinyLogBlockInputStream; friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream; friend class TinyLogBlockOutputStream;

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Parsers/ASTSelectQuery.h> #include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -7,8 +9,9 @@
namespace DB namespace DB
{ {
class StorageView : public IStorage class StorageView : private ext::share_ptr_helper<StorageView>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageView>;
public: public:
static StoragePtr create( static StoragePtr create(

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
namespace DB namespace DB
@ -11,8 +13,10 @@ class Context;
* that allows to obtain information about available clusters * that allows to obtain information about available clusters
* (which may be specified in Distributed tables). * (which may be specified in Distributed tables).
*/ */
class StorageSystemClusters : public IStorage class StorageSystemClusters : private ext::share_ptr_helper<StorageSystemClusters>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemClusters>;
public: public:
StorageSystemClusters(const std::string & name_, Context & context_); StorageSystemClusters(const std::string & name_, Context & context_);
static StoragePtr create(const std::string & name_, Context & context_); static StoragePtr create(const std::string & name_, Context & context_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,9 @@ namespace DB
/** Реализует системную таблицу columns, которая позволяет получить информацию /** Реализует системную таблицу columns, которая позволяет получить информацию
* о столбцах каждой таблицы для всех баз данных. * о столбцах каждой таблицы для всех баз данных.
*/ */
class StorageSystemColumns : public IStorage class StorageSystemColumns : private ext::share_ptr_helper<StorageSystemColumns>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemColumns>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД. /** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД.
*/ */
class StorageSystemDatabases : public IStorage class StorageSystemDatabases : private ext::share_ptr_helper<StorageSystemDatabases>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemDatabases>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,12 +1,16 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
namespace DB namespace DB
{ {
class StorageSystemDictionaries : public IStorage class StorageSystemDictionaries : private ext::share_ptr_helper<StorageSystemDictionaries>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemDictionaries>;
public: public:
static StoragePtr create(const std::string & name); static StoragePtr create(const std::string & name);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -8,8 +10,10 @@ namespace DB
/** Реализует системную таблицу events, которая позволяет получить информацию для профайлинга. /** Реализует системную таблицу events, которая позволяет получить информацию для профайлинга.
*/ */
class StorageSystemEvents : public IStorage class StorageSystemEvents : private ext::share_ptr_helper<StorageSystemEvents>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemEvents>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу functions, которая позволяет получить список /** Реализует системную таблицу functions, которая позволяет получить список
* всех обычных и агрегатных функций. * всех обычных и агрегатных функций.
*/ */
class StorageSystemFunctions : public IStorage class StorageSystemFunctions : private ext::share_ptr_helper<StorageSystemFunctions>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemFunctions>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,12 +1,16 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
namespace DB namespace DB
{ {
class StorageSystemMerges : public IStorage class StorageSystemMerges : private ext::share_ptr_helper<StorageSystemMerges>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemMerges>;
public: public:
static StoragePtr create(const std::string & name); static StoragePtr create(const std::string & name);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -8,8 +10,10 @@ namespace DB
/** Реализует системную таблицу metrics, которая позволяет получить информацию о работе сервера. /** Реализует системную таблицу metrics, которая позволяет получить информацию о работе сервера.
*/ */
class StorageSystemMetrics : public IStorage class StorageSystemMetrics : private ext::share_ptr_helper<StorageSystemMetrics>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemMetrics>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -10,8 +12,10 @@ namespace DB
* Таблица содержит единственный столбец number UInt64. * Таблица содержит единственный столбец number UInt64.
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново). * Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).
*/ */
class StorageSystemNumbers : public IStorage class StorageSystemNumbers : private ext::share_ptr_helper<StorageSystemNumbers>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemNumbers>;
public: public:
static StoragePtr create(const std::string & name_, bool multithreaded_ = false); static StoragePtr create(const std::string & name_, bool multithreaded_ = false);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
@ -12,8 +14,10 @@ namespace DB
* Используется, если в запросе не указана таблица. * Используется, если в запросе не указана таблица.
* Аналог таблицы DUAL в Oracle и MySQL. * Аналог таблицы DUAL в Oracle и MySQL.
*/ */
class StorageSystemOne : public IStorage class StorageSystemOne : private ext::share_ptr_helper<StorageSystemOne>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemOne>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,12 +11,13 @@ namespace DB
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах. /** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
*/ */
class StorageSystemParts : public IStorage class StorageSystemParts : private ext::share_ptr_helper<StorageSystemParts>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemParts>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);
std::string getName() const override{ return "SystemParts"; } std::string getName() const override { return "SystemParts"; }
std::string getTableName() const override { return name; } std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; } const NamesAndTypesList & getColumnsListImpl() const override { return columns; }

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент. /** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент.
*/ */
class StorageSystemProcesses : public IStorage class StorageSystemProcesses : private ext::share_ptr_helper<StorageSystemProcesses>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemProcesses>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц. /** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц.
*/ */
class StorageSystemReplicas : public IStorage class StorageSystemReplicas : private ext::share_ptr_helper<StorageSystemReplicas>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemReplicas>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц. /** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
*/ */
class StorageSystemReplicationQueue : public IStorage class StorageSystemReplicationQueue : private ext::share_ptr_helper<StorageSystemReplicationQueue>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemReplicationQueue>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках. /** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках.
*/ */
class StorageSystemSettings : public IStorage class StorageSystemSettings : private ext::share_ptr_helper<StorageSystemSettings>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemSettings>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах. /** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
*/ */
class StorageSystemTables : public IStorage class StorageSystemTables : private ext::share_ptr_helper<StorageSystemTables>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemTables>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <ext/share_ptr_helper.hpp>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,8 +11,10 @@ namespace DB
/** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки. /** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки.
*/ */
class StorageSystemZooKeeper : public IStorage class StorageSystemZooKeeper : private ext::share_ptr_helper<StorageSystemZooKeeper>, public IStorage
{ {
friend class ext::share_ptr_helper<StorageSystemZooKeeper>;
public: public:
static StoragePtr create(const std::string & name_); static StoragePtr create(const std::string & name_);

View File

@ -1,3 +1,5 @@
#include <ext/share_ptr_helper.hpp>
#include <DB/Parsers/ASTSelectQuery.h> #include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ParserSelectQuery.h> #include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/parseQuery.h> #include <DB/Parsers/parseQuery.h>
@ -15,12 +17,14 @@
#include <string> #include <string>
/// Упрощённый вариант класса StorageDistributed. /// Упрощённый вариант класса StorageDistributed.
class StorageDistributedFake : public DB::IStorage class StorageDistributedFake : private ext::share_ptr_helper<StorageDistributedFake>, public DB::IStorage
{ {
friend class ext::share_ptr_helper<StorageDistributedFake>;
public: public:
static DB::StoragePtr create(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_) static DB::StoragePtr create(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_)
{ {
return (new StorageDistributedFake{remote_database_, remote_table_, shard_count_})->thisPtr(); return make_shared(remote_database_, remote_table_, shard_count_);
} }
std::string getName() const override { return "DistributedFake"; } std::string getName() const override { return "DistributedFake"; }

View File

@ -30,9 +30,9 @@ StoragePtr StorageBuffer::create(const std::string & name_, NamesAndTypesListPtr
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_) const String & destination_database_, const String & destination_table_)
{ {
return (new StorageBuffer{ return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_})->thisPtr(); context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_);;
} }

View File

@ -37,7 +37,7 @@ StoragePtr StorageCloud::create(
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageCloud{database_ptr_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_})->thisPtr(); return make_shared(database_ptr_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
} }

View File

@ -131,13 +131,13 @@ StoragePtr StorageDistributed::create(
const ASTPtr & sharding_key_, const ASTPtr & sharding_key_,
const String & data_path_) const String & data_path_)
{ {
return (new StorageDistributed{ return make_shared(
name_, columns_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_, materialized_columns_, alias_columns_, column_defaults_,
remote_database_, remote_table_, remote_database_, remote_table_,
context_.getCluster(cluster_name), context_, context_.getCluster(cluster_name), context_,
sharding_key_, data_path_ sharding_key_, data_path_
})->thisPtr(); );
} }
@ -149,15 +149,15 @@ StoragePtr StorageDistributed::create(
std::shared_ptr<Cluster> & owned_cluster_, std::shared_ptr<Cluster> & owned_cluster_,
Context & context_) Context & context_)
{ {
auto res = new StorageDistributed{ auto res = make_shared(
name_, columns_, remote_database_, name_, columns_, remote_database_,
remote_table_, *owned_cluster_, context_ remote_table_, *owned_cluster_, context_
}; );
/// Захватываем владение объектом-кластером. /// Захватываем владение объектом-кластером.
res->owned_cluster = owned_cluster_; res->owned_cluster = owned_cluster_;
return res->thisPtr(); return res;
} }
BlockInputStreams StorageDistributed::read( BlockInputStreams StorageDistributed::read(

View File

@ -496,11 +496,11 @@ StoragePtr StorageLog::create(
const ColumnDefaults & column_defaults_, const ColumnDefaults & column_defaults_,
size_t max_compress_block_size_) size_t max_compress_block_size_)
{ {
return (new StorageLog{ return make_shared(
path_, name_, columns_, path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_, materialized_columns_, alias_columns_, column_defaults_,
max_compress_block_size_ max_compress_block_size_
})->thisPtr(); );
} }
StoragePtr StorageLog::create( StoragePtr StorageLog::create(
@ -509,11 +509,11 @@ StoragePtr StorageLog::create(
NamesAndTypesListPtr columns_, NamesAndTypesListPtr columns_,
size_t max_compress_block_size_) size_t max_compress_block_size_)
{ {
return (new StorageLog{ return make_shared(
path_, name_, columns_, path_, name_, columns_,
{}, {}, ColumnDefaults{}, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{},
max_compress_block_size_ max_compress_block_size_
})->thisPtr(); );
} }

View File

@ -28,11 +28,10 @@ StoragePtr StorageMaterializedView::create(
const ColumnDefaults & column_defaults_, const ColumnDefaults & column_defaults_,
bool attach_) bool attach_)
{ {
return (new StorageMaterializedView{ return ext::share_ptr_helper<StorageMaterializedView>::make_shared(
table_name_, database_name_, context_, query_, table_name_, database_name_, context_, query_, columns_,
columns_, materialized_columns_, alias_columns_, column_defaults_, materialized_columns_, alias_columns_, column_defaults_, attach_
attach_ );
})->thisPtr();
} }
StorageMaterializedView::StorageMaterializedView( StorageMaterializedView::StorageMaterializedView(

View File

@ -98,9 +98,7 @@ StoragePtr StorageMemory::create(
const std::string & name_, const std::string & name_,
NamesAndTypesListPtr columns_) NamesAndTypesListPtr columns_)
{ {
return (new StorageMemory{ return make_shared(name_, columns_);
name_, columns_
})->thisPtr();
} }
StoragePtr StorageMemory::create( StoragePtr StorageMemory::create(
@ -110,10 +108,7 @@ StoragePtr StorageMemory::create(
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageMemory{ return make_shared(name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
name_, columns_,
materialized_columns_, alias_columns_, column_defaults_
})->thisPtr();
} }

View File

@ -53,10 +53,10 @@ StoragePtr StorageMerge::create(
const String & table_name_regexp_, const String & table_name_regexp_,
const Context & context_) const Context & context_)
{ {
return (new StorageMerge{ return make_shared(
name_, columns_, name_, columns_,
source_database_, table_name_regexp_, context_ source_database_, table_name_regexp_, context_
})->thisPtr(); );
} }
StoragePtr StorageMerge::create( StoragePtr StorageMerge::create(
@ -69,10 +69,10 @@ StoragePtr StorageMerge::create(
const String & table_name_regexp_, const String & table_name_regexp_,
const Context & context_) const Context & context_)
{ {
return (new StorageMerge{ return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
source_database_, table_name_regexp_, context_ source_database_, table_name_regexp_, context_
})->thisPtr(); );
} }
NameAndTypePair StorageMerge::getColumn(const String & column_name) const NameAndTypePair StorageMerge::getColumn(const String & column_name) const

View File

@ -90,17 +90,15 @@ StoragePtr StorageMergeTree::create(
bool has_force_restore_data_flag_, bool has_force_restore_data_flag_,
const MergeTreeSettings & settings_) const MergeTreeSettings & settings_)
{ {
auto res = new StorageMergeTree{ auto res = make_shared(
path_, database_name_, table_name_, path_, database_name_, table_name_,
columns_, materialized_columns_, alias_columns_, column_defaults_, columns_, materialized_columns_, alias_columns_, column_defaults_,
context_, primary_expr_ast_, date_column_name_, context_, primary_expr_ast_, date_column_name_,
sampling_expression_, index_granularity_, merging_params_, has_force_restore_data_flag_, settings_ sampling_expression_, index_granularity_, merging_params_, has_force_restore_data_flag_, settings_
}; );
StoragePtr res_ptr = res->thisPtr(); res->merge_task_handle = res->background_pool.addTask(std::bind(&StorageMergeTree::mergeTask, res.get(), std::placeholders::_1));
res->merge_task_handle = res->background_pool.addTask(std::bind(&StorageMergeTree::mergeTask, res, std::placeholders::_1)); return res;
return res_ptr;
} }

View File

@ -345,15 +345,14 @@ StoragePtr StorageReplicatedMergeTree::create(
bool has_force_restore_data_flag_, bool has_force_restore_data_flag_,
const MergeTreeSettings & settings_) const MergeTreeSettings & settings_)
{ {
auto res = new StorageReplicatedMergeTree{ auto res = make_shared(
zookeeper_path_, replica_name_, attach, zookeeper_path_, replica_name_, attach,
path_, database_name_, name_, path_, database_name_, name_,
columns_, materialized_columns_, alias_columns_, column_defaults_, columns_, materialized_columns_, alias_columns_, column_defaults_,
context_, primary_expr_ast_, date_column_name_, context_, primary_expr_ast_, date_column_name_,
sampling_expression_, index_granularity_, sampling_expression_, index_granularity_,
merging_params_, has_force_restore_data_flag_, settings_}; merging_params_, has_force_restore_data_flag_, settings_);
StoragePtr res_ptr = res;
StoragePtr res_ptr = res->thisPtr();
auto get_endpoint_holder = [&res](InterserverIOEndpointPtr endpoint) auto get_endpoint_holder = [&res](InterserverIOEndpointPtr endpoint)
{ {
@ -393,7 +392,7 @@ StoragePtr StorageReplicatedMergeTree::create(
} }
} }
return res_ptr; return res;
} }

View File

@ -211,11 +211,11 @@ StoragePtr StorageStripeLog::create(
bool attach, bool attach,
size_t max_compress_block_size_) size_t max_compress_block_size_)
{ {
return (new StorageStripeLog{ return make_shared(
path_, name_, columns_, path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_, materialized_columns_, alias_columns_, column_defaults_,
attach, max_compress_block_size_ attach, max_compress_block_size_
})->thisPtr(); );
} }

View File

@ -394,11 +394,11 @@ StoragePtr StorageTinyLog::create(
bool attach, bool attach,
size_t max_compress_block_size_) size_t max_compress_block_size_)
{ {
return (new StorageTinyLog{ return make_shared(
path_, name_, columns_, path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_, materialized_columns_, alias_columns_, column_defaults_,
attach, max_compress_block_size_ attach, max_compress_block_size_
})->thisPtr(); );
} }

View File

@ -25,10 +25,10 @@ StoragePtr StorageView::create(
const NamesAndTypesList & alias_columns_, const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_) const ColumnDefaults & column_defaults_)
{ {
return (new StorageView{ return make_shared(
table_name_, database_name_, context_, query_, table_name_, database_name_, context_, query_,
columns_, materialized_columns_, alias_columns_, column_defaults_ columns_, materialized_columns_, alias_columns_, column_defaults_
})->thisPtr(); );
} }

View File

@ -31,7 +31,7 @@ StorageSystemClusters::StorageSystemClusters(const std::string & name_, Context
StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_) StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_)
{ {
return (new StorageSystemClusters{name_, context_})->thisPtr(); return make_shared(name_, context_);
} }
BlockInputStreams StorageSystemClusters::read( BlockInputStreams StorageSystemClusters::read(

View File

@ -29,7 +29,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
StoragePtr StorageSystemColumns::create(const std::string & name_) StoragePtr StorageSystemColumns::create(const std::string & name_)
{ {
return (new StorageSystemColumns{name_})->thisPtr(); return make_shared(name_);
} }
BlockInputStreams StorageSystemColumns::read( BlockInputStreams StorageSystemColumns::read(

View File

@ -21,7 +21,7 @@ StorageSystemDatabases::StorageSystemDatabases(const std::string & name_)
StoragePtr StorageSystemDatabases::create(const std::string & name_) StoragePtr StorageSystemDatabases::create(const std::string & name_)
{ {
return (new StorageSystemDatabases(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -40,7 +40,7 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
StoragePtr StorageSystemDictionaries::create(const std::string & name) StoragePtr StorageSystemDictionaries::create(const std::string & name)
{ {
return (new StorageSystemDictionaries{name})->thisPtr(); return make_shared(name);
} }
BlockInputStreams StorageSystemDictionaries::read( BlockInputStreams StorageSystemDictionaries::read(

View File

@ -22,7 +22,7 @@ StorageSystemEvents::StorageSystemEvents(const std::string & name_)
StoragePtr StorageSystemEvents::create(const std::string & name_) StoragePtr StorageSystemEvents::create(const std::string & name_)
{ {
return (new StorageSystemEvents(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -21,7 +21,7 @@ StorageSystemFunctions::StorageSystemFunctions(const std::string & name_)
StoragePtr StorageSystemFunctions::create(const std::string & name_) StoragePtr StorageSystemFunctions::create(const std::string & name_)
{ {
return (new StorageSystemFunctions{name_})->thisPtr(); return make_shared(name_);
} }
BlockInputStreams StorageSystemFunctions::read( BlockInputStreams StorageSystemFunctions::read(

View File

@ -31,7 +31,7 @@ StorageSystemMerges::StorageSystemMerges(const std::string & name)
StoragePtr StorageSystemMerges::create(const std::string & name) StoragePtr StorageSystemMerges::create(const std::string & name)
{ {
return (new StorageSystemMerges{name})->thisPtr(); return make_shared(name);
} }
BlockInputStreams StorageSystemMerges::read( BlockInputStreams StorageSystemMerges::read(

View File

@ -22,7 +22,7 @@ StorageSystemMetrics::StorageSystemMetrics(const std::string & name_)
StoragePtr StorageSystemMetrics::create(const std::string & name_) StoragePtr StorageSystemMetrics::create(const std::string & name_)
{ {
return (new StorageSystemMetrics(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -55,7 +55,7 @@ StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multi
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_) StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
{ {
return (new StorageSystemNumbers(name_, multithreaded_))->thisPtr(); return make_shared(name_, multithreaded_);
} }

View File

@ -17,7 +17,7 @@ StorageSystemOne::StorageSystemOne(const std::string & name_)
StoragePtr StorageSystemOne::create(const std::string & name_) StoragePtr StorageSystemOne::create(const std::string & name_)
{ {
return (new StorageSystemOne(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -43,7 +43,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_)
StoragePtr StorageSystemParts::create(const std::string & name_) StoragePtr StorageSystemParts::create(const std::string & name_)
{ {
return (new StorageSystemParts(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -28,7 +28,7 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_)
StoragePtr StorageSystemProcesses::create(const std::string & name_) StoragePtr StorageSystemProcesses::create(const std::string & name_)
{ {
return (new StorageSystemProcesses(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -47,7 +47,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_)
StoragePtr StorageSystemReplicas::create(const std::string & name_) StoragePtr StorageSystemReplicas::create(const std::string & name_)
{ {
return (new StorageSystemReplicas(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -49,7 +49,7 @@ StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string &
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_) StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
{ {
return (new StorageSystemReplicationQueue(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -21,7 +21,7 @@ StorageSystemSettings::StorageSystemSettings(const std::string & name_)
StoragePtr StorageSystemSettings::create(const std::string & name_) StoragePtr StorageSystemSettings::create(const std::string & name_)
{ {
return (new StorageSystemSettings(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -28,7 +28,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
StoragePtr StorageSystemTables::create(const std::string & name_) StoragePtr StorageSystemTables::create(const std::string & name_)
{ {
return (new StorageSystemTables(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -37,7 +37,7 @@ StorageSystemZooKeeper::StorageSystemZooKeeper(const std::string & name_)
StoragePtr StorageSystemZooKeeper::create(const std::string & name_) StoragePtr StorageSystemZooKeeper::create(const std::string & name_)
{ {
return (new StorageSystemZooKeeper(name_))->thisPtr(); return make_shared(name_);
} }

View File

@ -0,0 +1,44 @@
#pragma once
#include <memory>
namespace ext
{
/**
* Class AllocateShared allow to make std::shared_ptr<T> from T with private constructor.
* Derive you T class from AllocateShared<T>, define him as friend and call allocate_shared()/make_shared() method.
**/
template <class T>
class share_ptr_helper
{
protected:
typedef typename std::remove_const<T>::type TNoConst;
template <class TAlloc>
struct Deleter
{
void operator()(typename TAlloc::value_type * ptr)
{
std::allocator_traits<TAlloc>::destroy(alloc, ptr);
}
TAlloc alloc;
};
///see std::allocate_shared
template <class TAlloc, class ... TArgs>
static std::shared_ptr<T> allocate_shared(const TAlloc & alloc, TArgs && ... args)
{
TAlloc alloc_copy(alloc);
return std::shared_ptr<TNoConst>(new (std::allocator_traits<TAlloc>::allocate(alloc_copy, 1)) TNoConst(std::forward<TArgs>(args)...), Deleter<TAlloc>(), alloc_copy);
}
template <class ... TArgs>
static std::shared_ptr<T> make_shared(TArgs && ... args)
{
return allocate_shared(std::allocator<TNoConst>(), std::forward<TArgs>(args)...);
}
};
}