mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
use std::enable_shared_from_this for IStorage
This commit is contained in:
parent
065b02494a
commit
e5825f790f
@ -47,7 +47,7 @@ using StoragePtr = std::shared_ptr<IStorage>;
|
||||
* - структура хранения данных (сжатие, etc.)
|
||||
* - конкуррентный доступ к данным (блокировки, etc.)
|
||||
*/
|
||||
class IStorage : private boost::noncopyable, public ITableDeclaration
|
||||
class IStorage : public std::enable_shared_from_this<IStorage>, private boost::noncopyable, public ITableDeclaration
|
||||
{
|
||||
public:
|
||||
/// Основное имя типа таблицы (например, StorageMergeTree).
|
||||
@ -106,7 +106,7 @@ public:
|
||||
*/
|
||||
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)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
return res;
|
||||
@ -270,19 +270,6 @@ public:
|
||||
*/
|
||||
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};
|
||||
|
||||
/// Поддерживается ли индекс в секции IN
|
||||
@ -293,10 +280,9 @@ public:
|
||||
|
||||
protected:
|
||||
using ITableDeclaration::ITableDeclaration;
|
||||
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
||||
|
||||
private:
|
||||
std::weak_ptr<IStorage> this_ptr;
|
||||
|
||||
/// Брать следующие два лока всегда нужно в этом порядке.
|
||||
|
||||
/** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree).
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <mutex>
|
||||
#include <thread>
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
@ -31,8 +33,9 @@ namespace DB
|
||||
* При уничтожении таблицы типа 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 BufferBlockOutputStream;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.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:
|
||||
static StoragePtr create(
|
||||
DatabasePtr & database_ptr_,
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Client/ConnectionPool.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 DirectoryMonitor;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/StorageSet.h>
|
||||
#include <DB/Interpreters/Join.h>
|
||||
|
||||
@ -14,8 +16,10 @@ namespace DB
|
||||
*
|
||||
* При использовании, 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:
|
||||
static StoragePtr create(
|
||||
const String & path_,
|
||||
@ -27,11 +31,10 @@ public:
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_)
|
||||
{
|
||||
return (new StorageJoin{
|
||||
path_, name_,
|
||||
key_names_, kind_, strictness_,
|
||||
columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_})->thisPtr();
|
||||
return ext::share_ptr_helper<StorageJoin>::make_shared(
|
||||
path_, name_, key_names_, kind_, strictness_,
|
||||
columns_, materialized_columns_, alias_columns_, column_defaults_
|
||||
);
|
||||
}
|
||||
|
||||
String getName() const override { return "Join"; }
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <map>
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <Poco/File.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 LogBlockOutputStream;
|
||||
|
||||
|
@ -1,12 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/StorageView.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageMaterializedView : public StorageView {
|
||||
class StorageMaterializedView : private ext::share_ptr_helper<StorageMaterializedView>, public StorageView
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageMaterializedView>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <mutex>
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Storages/IStorage.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 MemoryBlockOutputStream;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Common/OptimizedRegularExpression.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:
|
||||
static StoragePtr create(
|
||||
const std::string & name_, /// Имя таблицы.
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
@ -14,8 +16,9 @@ namespace DB
|
||||
|
||||
/** См. описание структуры данных в 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;
|
||||
|
||||
public:
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Storages/IStorage.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:
|
||||
static StoragePtr create(
|
||||
const std::string & name_,
|
||||
@ -22,7 +26,7 @@ public:
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
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"; }
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.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:
|
||||
/** Если !attach, либо создает новую таблицу в ZK, либо добавляет реплику в существующую таблицу.
|
||||
*/
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/IO/WriteBufferFromFile.h>
|
||||
#include <DB/IO/CompressedWriteBuffer.h>
|
||||
#include <DB/DataStreams/NativeBlockOutputStream.h>
|
||||
@ -14,8 +16,9 @@ namespace DB
|
||||
|
||||
/** Общая часть 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;
|
||||
|
||||
public:
|
||||
@ -78,8 +81,10 @@ private:
|
||||
* а также записаны в файл-бэкап, для восстановления после перезапуска.
|
||||
* Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части оператора IN.
|
||||
*/
|
||||
class StorageSet : public StorageSetOrJoinBase
|
||||
class StorageSet : private ext::share_ptr_helper<StorageSet>, public StorageSetOrJoinBase
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSet>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(
|
||||
const String & path_,
|
||||
@ -89,9 +94,7 @@ public:
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_)
|
||||
{
|
||||
return (new StorageSet{
|
||||
path_, name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_})->thisPtr();
|
||||
return ext::share_ptr_helper<StorageSet>::make_shared(path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
|
||||
}
|
||||
|
||||
String getName() const override { return "Set"; }
|
||||
|
@ -1,6 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <map>
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
@ -14,8 +17,9 @@ namespace DB
|
||||
/** Реализует хранилище, подходящее для маленьких кусочков лога.
|
||||
* При этом, хранит все столбцы в одном файле формата 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 StripeLogBlockOutputStream;
|
||||
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <map>
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
@ -15,8 +17,9 @@ namespace DB
|
||||
/** Реализует хранилище, подходящее для маленьких кусочков лога.
|
||||
* Отличается от 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 TinyLogBlockOutputStream;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
@ -7,8 +9,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageView : public IStorage
|
||||
class StorageView : private ext::share_ptr_helper<StorageView>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageView>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
@ -11,8 +13,10 @@ class Context;
|
||||
* that allows to obtain information about available clusters
|
||||
* (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:
|
||||
StorageSystemClusters(const std::string & name_, Context & context_);
|
||||
static StoragePtr create(const std::string & name_, Context & context_);
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,9 @@ namespace DB
|
||||
/** Реализует системную таблицу columns, которая позволяет получить информацию
|
||||
* о столбцах каждой таблицы для всех баз данных.
|
||||
*/
|
||||
class StorageSystemColumns : public IStorage
|
||||
class StorageSystemColumns : private ext::share_ptr_helper<StorageSystemColumns>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemColumns>;
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД.
|
||||
*/
|
||||
class StorageSystemDatabases : public IStorage
|
||||
class StorageSystemDatabases : private ext::share_ptr_helper<StorageSystemDatabases>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemDatabases>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,12 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageSystemDictionaries : public IStorage
|
||||
class StorageSystemDictionaries : private ext::share_ptr_helper<StorageSystemDictionaries>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemDictionaries>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
@ -8,8 +10,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу events, которая позволяет получить информацию для профайлинга.
|
||||
*/
|
||||
class StorageSystemEvents : public IStorage
|
||||
class StorageSystemEvents : private ext::share_ptr_helper<StorageSystemEvents>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemEvents>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
/** Реализует системную таблицу functions, которая позволяет получить список
|
||||
* всех обычных и агрегатных функций.
|
||||
*/
|
||||
class StorageSystemFunctions : public IStorage
|
||||
class StorageSystemFunctions : private ext::share_ptr_helper<StorageSystemFunctions>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemFunctions>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,12 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageSystemMerges : public IStorage
|
||||
class StorageSystemMerges : private ext::share_ptr_helper<StorageSystemMerges>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemMerges>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
@ -8,8 +10,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу metrics, которая позволяет получить информацию о работе сервера.
|
||||
*/
|
||||
class StorageSystemMetrics : public IStorage
|
||||
class StorageSystemMetrics : private ext::share_ptr_helper<StorageSystemMetrics>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemMetrics>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
@ -10,8 +12,10 @@ namespace DB
|
||||
* Таблица содержит единственный столбец number UInt64.
|
||||
* Из этой таблицы можно прочитать все натуральные числа, начиная с 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:
|
||||
static StoragePtr create(const std::string & name_, bool multithreaded_ = false);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
@ -12,8 +14,10 @@ namespace DB
|
||||
* Используется, если в запросе не указана таблица.
|
||||
* Аналог таблицы 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:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,12 +11,13 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
|
||||
*/
|
||||
class StorageSystemParts : public IStorage
|
||||
class StorageSystemParts : private ext::share_ptr_helper<StorageSystemParts>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemParts>;
|
||||
public:
|
||||
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; }
|
||||
|
||||
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент.
|
||||
*/
|
||||
class StorageSystemProcesses : public IStorage
|
||||
class StorageSystemProcesses : private ext::share_ptr_helper<StorageSystemProcesses>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemProcesses>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц.
|
||||
*/
|
||||
class StorageSystemReplicas : public IStorage
|
||||
class StorageSystemReplicas : private ext::share_ptr_helper<StorageSystemReplicas>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemReplicas>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
|
||||
*/
|
||||
class StorageSystemReplicationQueue : public IStorage
|
||||
class StorageSystemReplicationQueue : private ext::share_ptr_helper<StorageSystemReplicationQueue>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemReplicationQueue>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках.
|
||||
*/
|
||||
class StorageSystemSettings : public IStorage
|
||||
class StorageSystemSettings : private ext::share_ptr_helper<StorageSystemSettings>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemSettings>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
|
||||
*/
|
||||
class StorageSystemTables : public IStorage
|
||||
class StorageSystemTables : private ext::share_ptr_helper<StorageSystemTables>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemTables>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,8 +11,10 @@ namespace DB
|
||||
|
||||
/** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки.
|
||||
*/
|
||||
class StorageSystemZooKeeper : public IStorage
|
||||
class StorageSystemZooKeeper : private ext::share_ptr_helper<StorageSystemZooKeeper>, public IStorage
|
||||
{
|
||||
friend class ext::share_ptr_helper<StorageSystemZooKeeper>;
|
||||
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include <ext/share_ptr_helper.hpp>
|
||||
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
@ -15,12 +17,14 @@
|
||||
#include <string>
|
||||
|
||||
/// Упрощённый вариант класса 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:
|
||||
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"; }
|
||||
|
@ -30,9 +30,9 @@ StoragePtr StorageBuffer::create(const std::string & name_, NamesAndTypesListPtr
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_)
|
||||
{
|
||||
return (new StorageBuffer{
|
||||
return make_shared(
|
||||
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_);;
|
||||
}
|
||||
|
||||
|
||||
|
@ -37,7 +37,7 @@ StoragePtr StorageCloud::create(
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
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_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -131,13 +131,13 @@ StoragePtr StorageDistributed::create(
|
||||
const ASTPtr & sharding_key_,
|
||||
const String & data_path_)
|
||||
{
|
||||
return (new StorageDistributed{
|
||||
return make_shared(
|
||||
name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_,
|
||||
remote_database_, remote_table_,
|
||||
context_.getCluster(cluster_name), context_,
|
||||
sharding_key_, data_path_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@ -149,15 +149,15 @@ StoragePtr StorageDistributed::create(
|
||||
std::shared_ptr<Cluster> & owned_cluster_,
|
||||
Context & context_)
|
||||
{
|
||||
auto res = new StorageDistributed{
|
||||
auto res = make_shared(
|
||||
name_, columns_, remote_database_,
|
||||
remote_table_, *owned_cluster_, context_
|
||||
};
|
||||
);
|
||||
|
||||
/// Захватываем владение объектом-кластером.
|
||||
res->owned_cluster = owned_cluster_;
|
||||
|
||||
return res->thisPtr();
|
||||
return res;
|
||||
}
|
||||
|
||||
BlockInputStreams StorageDistributed::read(
|
||||
|
@ -496,11 +496,11 @@ StoragePtr StorageLog::create(
|
||||
const ColumnDefaults & column_defaults_,
|
||||
size_t max_compress_block_size_)
|
||||
{
|
||||
return (new StorageLog{
|
||||
return make_shared(
|
||||
path_, name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_,
|
||||
max_compress_block_size_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
StoragePtr StorageLog::create(
|
||||
@ -509,11 +509,11 @@ StoragePtr StorageLog::create(
|
||||
NamesAndTypesListPtr columns_,
|
||||
size_t max_compress_block_size_)
|
||||
{
|
||||
return (new StorageLog{
|
||||
return make_shared(
|
||||
path_, name_, columns_,
|
||||
{}, {}, ColumnDefaults{},
|
||||
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{},
|
||||
max_compress_block_size_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,11 +28,10 @@ StoragePtr StorageMaterializedView::create(
|
||||
const ColumnDefaults & column_defaults_,
|
||||
bool attach_)
|
||||
{
|
||||
return (new StorageMaterializedView{
|
||||
table_name_, database_name_, context_, query_,
|
||||
columns_, materialized_columns_, alias_columns_, column_defaults_,
|
||||
attach_
|
||||
})->thisPtr();
|
||||
return ext::share_ptr_helper<StorageMaterializedView>::make_shared(
|
||||
table_name_, database_name_, context_, query_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_, attach_
|
||||
);
|
||||
}
|
||||
|
||||
StorageMaterializedView::StorageMaterializedView(
|
||||
|
@ -98,9 +98,7 @@ StoragePtr StorageMemory::create(
|
||||
const std::string & name_,
|
||||
NamesAndTypesListPtr columns_)
|
||||
{
|
||||
return (new StorageMemory{
|
||||
name_, columns_
|
||||
})->thisPtr();
|
||||
return make_shared(name_, columns_);
|
||||
}
|
||||
|
||||
StoragePtr StorageMemory::create(
|
||||
@ -110,10 +108,7 @@ StoragePtr StorageMemory::create(
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_)
|
||||
{
|
||||
return (new StorageMemory{
|
||||
name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_
|
||||
})->thisPtr();
|
||||
return make_shared(name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -53,10 +53,10 @@ StoragePtr StorageMerge::create(
|
||||
const String & table_name_regexp_,
|
||||
const Context & context_)
|
||||
{
|
||||
return (new StorageMerge{
|
||||
return make_shared(
|
||||
name_, columns_,
|
||||
source_database_, table_name_regexp_, context_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
StoragePtr StorageMerge::create(
|
||||
@ -69,10 +69,10 @@ StoragePtr StorageMerge::create(
|
||||
const String & table_name_regexp_,
|
||||
const Context & context_)
|
||||
{
|
||||
return (new StorageMerge{
|
||||
return make_shared(
|
||||
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
|
||||
source_database_, table_name_regexp_, context_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
NameAndTypePair StorageMerge::getColumn(const String & column_name) const
|
||||
|
@ -90,17 +90,15 @@ StoragePtr StorageMergeTree::create(
|
||||
bool has_force_restore_data_flag_,
|
||||
const MergeTreeSettings & settings_)
|
||||
{
|
||||
auto res = new StorageMergeTree{
|
||||
auto res = make_shared(
|
||||
path_, database_name_, table_name_,
|
||||
columns_, materialized_columns_, alias_columns_, column_defaults_,
|
||||
context_, primary_expr_ast_, date_column_name_,
|
||||
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_ptr;
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
|
@ -345,15 +345,14 @@ StoragePtr StorageReplicatedMergeTree::create(
|
||||
bool has_force_restore_data_flag_,
|
||||
const MergeTreeSettings & settings_)
|
||||
{
|
||||
auto res = new StorageReplicatedMergeTree{
|
||||
auto res = make_shared(
|
||||
zookeeper_path_, replica_name_, attach,
|
||||
path_, database_name_, name_,
|
||||
columns_, materialized_columns_, alias_columns_, column_defaults_,
|
||||
context_, primary_expr_ast_, date_column_name_,
|
||||
sampling_expression_, index_granularity_,
|
||||
merging_params_, has_force_restore_data_flag_, settings_};
|
||||
|
||||
StoragePtr res_ptr = res->thisPtr();
|
||||
merging_params_, has_force_restore_data_flag_, settings_);
|
||||
StoragePtr res_ptr = res;
|
||||
|
||||
auto get_endpoint_holder = [&res](InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
@ -393,7 +392,7 @@ StoragePtr StorageReplicatedMergeTree::create(
|
||||
}
|
||||
}
|
||||
|
||||
return res_ptr;
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
|
@ -211,11 +211,11 @@ StoragePtr StorageStripeLog::create(
|
||||
bool attach,
|
||||
size_t max_compress_block_size_)
|
||||
{
|
||||
return (new StorageStripeLog{
|
||||
return make_shared(
|
||||
path_, name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_,
|
||||
attach, max_compress_block_size_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
|
@ -394,11 +394,11 @@ StoragePtr StorageTinyLog::create(
|
||||
bool attach,
|
||||
size_t max_compress_block_size_)
|
||||
{
|
||||
return (new StorageTinyLog{
|
||||
return make_shared(
|
||||
path_, name_, columns_,
|
||||
materialized_columns_, alias_columns_, column_defaults_,
|
||||
attach, max_compress_block_size_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
|
@ -25,10 +25,10 @@ StoragePtr StorageView::create(
|
||||
const NamesAndTypesList & alias_columns_,
|
||||
const ColumnDefaults & column_defaults_)
|
||||
{
|
||||
return (new StorageView{
|
||||
return make_shared(
|
||||
table_name_, database_name_, context_, query_,
|
||||
columns_, materialized_columns_, alias_columns_, column_defaults_
|
||||
})->thisPtr();
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
|
@ -31,7 +31,7 @@ StorageSystemClusters::StorageSystemClusters(const std::string & name_, Context
|
||||
|
||||
StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_)
|
||||
{
|
||||
return (new StorageSystemClusters{name_, context_})->thisPtr();
|
||||
return make_shared(name_, context_);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemClusters::read(
|
||||
|
@ -29,7 +29,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemColumns::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemColumns{name_})->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemColumns::read(
|
||||
|
@ -21,7 +21,7 @@ StorageSystemDatabases::StorageSystemDatabases(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemDatabases::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemDatabases(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -40,7 +40,7 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
|
||||
|
||||
StoragePtr StorageSystemDictionaries::create(const std::string & name)
|
||||
{
|
||||
return (new StorageSystemDictionaries{name})->thisPtr();
|
||||
return make_shared(name);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemDictionaries::read(
|
||||
|
@ -22,7 +22,7 @@ StorageSystemEvents::StorageSystemEvents(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemEvents::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemEvents(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -21,7 +21,7 @@ StorageSystemFunctions::StorageSystemFunctions(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemFunctions::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemFunctions{name_})->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemFunctions::read(
|
||||
|
@ -31,7 +31,7 @@ StorageSystemMerges::StorageSystemMerges(const std::string & name)
|
||||
|
||||
StoragePtr StorageSystemMerges::create(const std::string & name)
|
||||
{
|
||||
return (new StorageSystemMerges{name})->thisPtr();
|
||||
return make_shared(name);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemMerges::read(
|
||||
|
@ -22,7 +22,7 @@ StorageSystemMetrics::StorageSystemMetrics(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemMetrics::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemMetrics(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -55,7 +55,7 @@ StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multi
|
||||
|
||||
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
|
||||
{
|
||||
return (new StorageSystemNumbers(name_, multithreaded_))->thisPtr();
|
||||
return make_shared(name_, multithreaded_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -17,7 +17,7 @@ StorageSystemOne::StorageSystemOne(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemOne::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemOne(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -43,7 +43,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemParts::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemParts(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,7 +28,7 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemProcesses::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemProcesses(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -47,7 +47,7 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemReplicas::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemReplicas(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -49,7 +49,7 @@ StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string &
|
||||
|
||||
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemReplicationQueue(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -21,7 +21,7 @@ StorageSystemSettings::StorageSystemSettings(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemSettings::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemSettings(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,7 +28,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemTables::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemTables(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
@ -37,7 +37,7 @@ StorageSystemZooKeeper::StorageSystemZooKeeper(const std::string & name_)
|
||||
|
||||
StoragePtr StorageSystemZooKeeper::create(const std::string & name_)
|
||||
{
|
||||
return (new StorageSystemZooKeeper(name_))->thisPtr();
|
||||
return make_shared(name_);
|
||||
}
|
||||
|
||||
|
||||
|
44
libs/libcommon/include/ext/share_ptr_helper.hpp
Normal file
44
libs/libcommon/include/ext/share_ptr_helper.hpp
Normal 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)...);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user