提交 0d7d379b 编写于 作者: A Alexey Milovidov

Removed duplicate code [#CLICKHOUSE-2].

上级 2799fbf9
......@@ -36,7 +36,7 @@ try
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
......
......@@ -33,7 +33,7 @@ try
context.addDatabase("system", system_database);
context.setCurrentDatabase("system");
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
AnalyzeResultOfQuery analyzer;
analyzer.process(ast, context);
......
......@@ -34,7 +34,7 @@ try
context.addDatabase("system", system_database);
context.setCurrentDatabase("system");
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
CollectAliases collect_aliases;
collect_aliases.process(ast);
......
......@@ -39,7 +39,7 @@ try
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
......
......@@ -40,7 +40,7 @@ try
auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas;
......
......@@ -42,7 +42,7 @@ try
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......
......@@ -48,7 +48,7 @@ try
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......
......@@ -65,7 +65,7 @@ try
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......
......@@ -25,7 +25,7 @@ using namespace DB;
void test1()
{
Context context;
StoragePtr table = StorageSystemNumbers::create("numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......@@ -55,7 +55,7 @@ void test1()
void test2()
{
Context context;
StoragePtr table = StorageSystemNumbers::create("numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......
......@@ -27,16 +27,11 @@ namespace DB
/// Simplified version of the StorageDistributed class.
class StorageDistributedFake : private ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
class StorageDistributedFake : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
{
friend class ext::shared_ptr_helper<StorageDistributedFake>;
public:
static DB::StoragePtr create(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_)
{
return make_shared(remote_database_, remote_table_, shard_count_);
}
std::string getName() const override { return "DistributedFake"; }
bool isRemote() const override { return true; }
size_t getShardCount() const { return shard_count; }
......
......@@ -41,7 +41,7 @@ try
context.addDatabase("system", system);
system->loadTables(context, nullptr, false);
system->attachTable("one", StorageSystemOne::create("one"));
system->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
context.setCurrentDatabase("default");
ReadBufferFromIStream in(std::cin);
......
......@@ -43,20 +43,6 @@ namespace ErrorCodes
}
StoragePtr StorageBuffer::create(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_)
{
return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_);
}
StorageBuffer::StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
......
......@@ -36,7 +36,7 @@ class Context;
* When you destroy a Buffer table, all remaining data is flushed to the subordinate table.
* The data in the buffer is not replicated, not logged to disk, not indexed. With a rough restart of the server, the data is lost.
*/
class StorageBuffer : private ext::shared_ptr_helper<StorageBuffer>, public IStorage
class StorageBuffer : public ext::shared_ptr_helper<StorageBuffer>, public IStorage
{
friend class ext::shared_ptr_helper<StorageBuffer>;
friend class BufferBlockInputStream;
......@@ -51,17 +51,6 @@ public:
size_t bytes; /// The number of (uncompressed) bytes in the block.
};
/** 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.
*/
static StoragePtr create(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_);
std::string getName() const override { return "Buffer"; }
std::string getTableName() const override { return name; }
......@@ -123,6 +112,9 @@ 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_,
......
......@@ -28,17 +28,4 @@ StorageCloud::StorageCloud(
DatabaseCloud & db = static_cast<DatabaseCloud &>(*owned_db);
}
StoragePtr StorageCloud::create(
DatabasePtr & database_ptr_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
{
return make_shared(database_ptr_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
}
}
......@@ -16,19 +16,11 @@ class Context;
/** Cloud table. It can only be in the cloud database.
* When writing to a table, data is written to local tables on multiple cloud servers.
*/
class StorageCloud : private ext::shared_ptr_helper<StorageCloud>, public IStorage
class StorageCloud : public ext::shared_ptr_helper<StorageCloud>, public IStorage
{
friend class ext::shared_ptr_helper<StorageCloud>;
public:
static StoragePtr create(
DatabasePtr & database_ptr_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_);
std::string getName() const override { return "Cloud"; }
std::string getTableName() const override { return name; }
......
......@@ -25,7 +25,7 @@ class StorageDistributedDirectoryMonitor;
* You can pass one address, not several.
* In this case, the table can be considered remote, rather than distributed.
*/
class StorageDistributed : private ext::shared_ptr_helper<StorageDistributed>, public IStorage
class StorageDistributed : public ext::shared_ptr_helper<StorageDistributed>, public IStorage
{
friend class ext::shared_ptr_helper<StorageDistributed>;
friend class DistributedBlockOutputStream;
......
......@@ -554,7 +554,9 @@ StoragePtr StorageFactory::get(
table_name, columns,
materialized_columns, alias_columns, column_defaults,
context,
num_buckets, {min_time, min_rows, min_bytes}, {max_time, max_rows, max_bytes},
num_buckets,
StorageBuffer::Thresholds{min_time, min_rows, min_bytes},
StorageBuffer::Thresholds{max_time, max_rows, max_bytes},
destination_database, destination_table);
}
else if (name == "TrivialBuffer")
......@@ -602,7 +604,8 @@ StoragePtr StorageFactory::get(
table_name, columns,
materialized_columns, alias_columns, column_defaults,
context, num_blocks_to_deduplicate, path_in_zk_for_deduplication,
{min_time, min_rows, min_bytes}, {max_time, max_rows, max_bytes},
StorageTrivialBuffer::Thresholds{min_time, min_rows, min_bytes},
StorageTrivialBuffer::Thresholds{max_time, max_rows, max_bytes},
destination_database, destination_table);
}
else if (endsWith(name, "MergeTree"))
......
......@@ -20,7 +20,7 @@ using JoinPtr = std::shared_ptr<Join>;
*
* When using, JOIN must be of the appropriate type (ANY|ALL LEFT|INNER ...).
*/
class StorageJoin : private ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
class StorageJoin : public ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
{
friend class ext::shared_ptr_helper<StorageJoin>;
......
......@@ -589,35 +589,6 @@ StorageLog::StorageLog(
null_marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_NULL_MARKS_FILE_NAME);
}
StoragePtr StorageLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_,
max_compress_block_size_
);
}
StoragePtr StorageLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{},
max_compress_block_size_
);
}
void StorageLog::addFile(const String & column_name, const IDataType & type, size_t level)
{
......
......@@ -38,32 +38,13 @@ using Marks = std::vector<Mark>;
* Keys are not supported.
* The data is stored in a compressed form.
*/
class StorageLog : private ext::shared_ptr_helper<StorageLog>, public IStorage
class StorageLog : public ext::shared_ptr_helper<StorageLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageLog>;
friend class LogBlockInputStream;
friend class LogBlockOutputStream;
public:
/** hook the table with the appropriate name, along the appropriate path (with / at the end),
* (the correctness of names and paths is not verified)
* consisting of the specified columns; Create files if they do not exist.
*/
static StoragePtr create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
static StoragePtr create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
std::string getName() const override { return "Log"; }
std::string getTableName() const override { return name; }
......@@ -102,6 +83,10 @@ protected:
Poco::RWLock rwlock;
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
* (the correctness of names and paths is not verified)
* consisting of the specified columns; Create files if they do not exist.
*/
StorageLog(
const std::string & path_,
const std::string & name_,
......
......@@ -8,7 +8,7 @@
namespace DB
{
class StorageMaterializedView : private ext::shared_ptr_helper<StorageMaterializedView>, public StorageView
class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterializedView>, public StorageView
{
friend class ext::shared_ptr_helper<StorageMaterializedView>;
......
......@@ -94,24 +94,6 @@ StorageMemory::StorageMemory(
}
StoragePtr StorageMemory::create(
const std::string & name_,
NamesAndTypesListPtr columns_)
{
return make_shared(name_, columns_);
}
StoragePtr StorageMemory::create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
{
return make_shared(name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
}
BlockInputStreams StorageMemory::read(
const Names & column_names,
const ASTPtr & query,
......
......@@ -20,24 +20,13 @@ class StorageMemory;
* It does not support keys.
* Data is stored as a set of blocks and is not stored anywhere else.
*/
class StorageMemory : private ext::shared_ptr_helper<StorageMemory>, public IStorage
class StorageMemory : public ext::shared_ptr_helper<StorageMemory>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMemory>;
friend class MemoryBlockInputStream;
friend class MemoryBlockOutputStream;
public:
static StoragePtr create(
const std::string & name_,
NamesAndTypesListPtr columns_);
static StoragePtr create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_);
std::string getName() const override { return "Memory"; }
std::string getTableName() const override { return name; }
......
......@@ -50,35 +50,6 @@ StorageMerge::StorageMerge(
{
}
StoragePtr StorageMerge::create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
{
return make_shared(
name_, columns_,
source_database_, table_name_regexp_, context_
);
}
StoragePtr StorageMerge::create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
{
return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
source_database_, table_name_regexp_, context_
);
}
NameAndTypePair StorageMerge::getColumn(const String & column_name) const
{
auto type = VirtualColumnFactory::tryGetType(column_name);
......
......@@ -12,28 +12,11 @@ namespace DB
/** A table that represents the union of an arbitrary number of other tables.
* All tables must have the same structure.
*/
class StorageMerge : private ext::shared_ptr_helper<StorageMerge>, public IStorage
class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMerge>;
public:
static StoragePtr create(
const std::string & name_, /// The name of the table.
NamesAndTypesListPtr columns_, /// List of columns.
const String & source_database_, /// In which database to look for source tables.
const String & table_name_regexp_, /// Regex names of source tables.
const Context & context_); /// Known tables.
static StoragePtr create(
const std::string & name_, /// The name of the table.
NamesAndTypesListPtr columns_, /// List of columns.
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
const String & source_database_, /// In which database to look for source tables.
const String & table_name_regexp_, /// Regex names of source tables.
const Context & context_); /// Known tables.
std::string getName() const override { return "Merge"; }
std::string getTableName() const override { return name; }
......@@ -70,11 +53,11 @@ private:
const Context & context;
StorageMerge(
const std::string & name_,
NamesAndTypesListPtr columns_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_);
const std::string & name_, /// The name of the table.
NamesAndTypesListPtr columns_, /// List of columns.
const String & source_database_, /// In which database to look for source tables.
const String & table_name_regexp_, /// Regex names of source tables.
const Context & context_); /// Known tables.
StorageMerge(
const std::string & name_,
......
......@@ -65,30 +65,6 @@ StorageMergeTree::StorageMergeTree(
increment.set(data.getMaxDataPartIndex());
}
StoragePtr StorageMergeTree::create(
const String & path_, const String & database_name_, const String & table_name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
Context & context_,
ASTPtr & primary_expr_ast_,
const String & date_column_name_,
const ASTPtr & sampling_expression_,
size_t index_granularity_,
const MergeTreeData::MergingParams & merging_params_,
bool has_force_restore_data_flag_,
const MergeTreeSettings & settings_)
{
return make_shared(
path_, database_name_, table_name_,
columns_, materialized_columns_, alias_columns_, column_defaults_, attach,
context_, primary_expr_ast_, date_column_name_,
sampling_expression_, index_granularity_, merging_params_, has_force_restore_data_flag_, settings_
);
}
void StorageMergeTree::startup()
{
......
......@@ -16,38 +16,12 @@ namespace DB
/** See the description of the data structure in MergeTreeData.
*/
class StorageMergeTree : private ext::shared_ptr_helper<StorageMergeTree>, public IStorage
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public IStorage
{
friend class ext::shared_ptr_helper<StorageMergeTree>;
friend class MergeTreeBlockOutputStream;
public:
/** hook the table with the appropriate name, along the appropriate path (with / at the end),
* (correctness of names and paths are not checked)
* consisting of the specified columns.
*
* primary_expr_ast - expression for sorting;
* date_column_name - the name of the column with the date;
* index_granularity - fow how many rows one index value is written.
*/
static StoragePtr create(
const String & path_,
const String & database_name_,
const String & table_name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
Context & context_,
ASTPtr & primary_expr_ast_,
const String & date_column_name_,
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
size_t index_granularity_,
const MergeTreeData::MergingParams & merging_params_,
bool has_force_restore_data_flag,
const MergeTreeSettings & settings_);
void startup() override;
void shutdown() override;
~StorageMergeTree() override;
......@@ -141,6 +115,14 @@ private:
friend struct CurrentlyMergingPartsTagger;
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
* (correctness of names and paths are not checked)
* consisting of the specified columns.
*
* primary_expr_ast - expression for sorting;
* date_column_name - the name of the column with the date;
* index_granularity - fow how many rows one index value is written.
*/
StorageMergeTree(
const String & path_,
const String & database_name_,
......
......@@ -14,21 +14,11 @@ namespace DB
/** When writing, does nothing.
* When reading, returns nothing.
*/
class StorageNull : private ext::shared_ptr_helper<StorageNull>, public IStorage
class StorageNull : public ext::shared_ptr_helper<StorageNull>, public IStorage
{
friend class ext::shared_ptr_helper<StorageNull>;
public:
static StoragePtr create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
{
return make_shared(name_, columns_, materialized_columns_, alias_columns_, column_defaults_);
}
std::string getName() const override { return "Null"; }
std::string getTableName() const override { return name; }
......
......@@ -69,7 +69,7 @@ namespace DB
* as the time will take the time of creation the appropriate part on any of the replicas.
*/
class StorageReplicatedMergeTree : private ext::shared_ptr_helper<StorageReplicatedMergeTree>, public IStorage
class StorageReplicatedMergeTree : public ext::shared_ptr_helper<StorageReplicatedMergeTree>, public IStorage
{
friend class ext::shared_ptr_helper<StorageReplicatedMergeTree>;
......
......@@ -14,7 +14,7 @@ using SetPtr = std::shared_ptr<Set>;
/** Common part of StorageSet and StorageJoin.
*/
class StorageSetOrJoinBase : private ext::shared_ptr_helper<StorageSetOrJoinBase>, public IStorage
class StorageSetOrJoinBase : public ext::shared_ptr_helper<StorageSetOrJoinBase>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSetOrJoinBase>;
friend class SetOrJoinBlockOutputStream;
......@@ -59,7 +59,7 @@ private:
* and also written to a file-backup, for recovery after a restart.
* Reading from the table is not possible directly - it is possible to specify only the right part of the IN statement.
*/
class StorageSet : private ext::shared_ptr_helper<StorageSet>, public StorageSetOrJoinBase
class StorageSet : public ext::shared_ptr_helper<StorageSet>, public StorageSetOrJoinBase
{
friend class ext::shared_ptr_helper<StorageSet>;
......
......@@ -206,23 +206,6 @@ StorageStripeLog::StorageStripeLog(
}
}
StoragePtr StorageStripeLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_,
attach, max_compress_block_size_
);
}
void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
......
......@@ -17,28 +17,13 @@ namespace DB
/** Implements a repository that is suitable for small pieces of the log.
* In doing so, stores all the columns in a single Native file, with a nearby index.
*/
class StorageStripeLog : private ext::shared_ptr_helper<StorageStripeLog>, public IStorage
class StorageStripeLog : public ext::shared_ptr_helper<StorageStripeLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageStripeLog>;
friend class StripeLogBlockInputStream;
friend class StripeLogBlockOutputStream;
public:
/** hook the table with the appropriate name, along the appropriate path (with / at the end),
* (the correctness of names and paths is not checked)
* consisting of the specified columns.
* If not specified `attach` - create a directory if it does not exist.
*/
static StoragePtr create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
std::string getName() const override { return "StripeLog"; }
std::string getTableName() const override { return name; }
......@@ -87,7 +72,7 @@ private:
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_);
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
};
}
......@@ -464,24 +464,6 @@ StorageTinyLog::StorageTinyLog(
}
StoragePtr StorageTinyLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_,
attach, max_compress_block_size_
);
}
void StorageTinyLog::addFile(const String & column_name, const IDataType & type, size_t level)
{
if (files.end() != files.find(column_name))
......
......@@ -17,28 +17,13 @@ namespace DB
/** Implements a repository that is suitable for small pieces of the log.
* It differs from StorageLog in the absence of mark files.
*/
class StorageTinyLog : private ext::shared_ptr_helper<StorageTinyLog>, public IStorage
class StorageTinyLog : public ext::shared_ptr_helper<StorageTinyLog>, public IStorage
{
friend class ext::shared_ptr_helper<StorageTinyLog>;
friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream;
public:
/** hook the table with the appropriate name, along the appropriate path (with / at the end),
* (the correctness of names and paths is not verified)
* consisting of the specified columns.
* If not specified `attach` - create a directory if it does not exist.
*/
static StoragePtr create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
std::string getName() const override { return "TinyLog"; }
std::string getTableName() const override { return name; }
......@@ -90,7 +75,7 @@ private:
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_);
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
};
......
......@@ -43,23 +43,6 @@ namespace ErrorCodes
}
StoragePtr StorageTrivialBuffer::create(const std::string & name_, NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
Context & context_, const size_t num_blocks_to_deduplicate_,
const String & path_in_zk_for_deduplication_,
const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_)
{
return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
context_, num_blocks_to_deduplicate_, path_in_zk_for_deduplication_,
min_thresholds_, max_thresholds_,
destination_database_, destination_table_);
}
StorageTrivialBuffer::StorageTrivialBuffer(const std::string & name_, NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
......
......@@ -31,7 +31,7 @@ class Context;
* The data in the buffer is not replicated, logged or stored. After hard reset of the
* server, the data is lost.
*/
class StorageTrivialBuffer : private ext::shared_ptr_helper<StorageTrivialBuffer>, public IStorage
class StorageTrivialBuffer : public ext::shared_ptr_helper<StorageTrivialBuffer>, public IStorage
{
friend class ext::shared_ptr_helper<StorageTrivialBuffer>;
friend class TrivialBufferBlockInputStream;
......@@ -45,15 +45,6 @@ public:
size_t bytes; /// Number of bytes (incompressed) in buffer.
};
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
Context & context_, size_t num_blocks_to_deduplicate_,
const String & path_in_zk_for_deduplication_,
const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_);
std::string getName() const override { return "TrivialBuffer"; }
std::string getTableName() const override { return name; }
......
......@@ -15,23 +15,6 @@ namespace ErrorCodes
}
StoragePtr StorageView::create(
const String & table_name_,
const String & database_name_,
Context & context_,
ASTPtr & query_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
{
return make_shared(
table_name_, database_name_, context_, query_,
columns_, materialized_columns_, alias_columns_, column_defaults_
);
}
StorageView::StorageView(
const String & table_name_,
const String & database_name_,
......
......@@ -9,21 +9,11 @@
namespace DB
{
class StorageView : private ext::shared_ptr_helper<StorageView>, public IStorage
class StorageView : public ext::shared_ptr_helper<StorageView>, public IStorage
{
friend class ext::shared_ptr_helper<StorageView>;
public:
static StoragePtr create(
const String & table_name_,
const String & database_name_,
Context & context_,
ASTPtr & query_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_);
std::string getName() const override { return "View"; }
std::string getTableName() const override { return table_name; }
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
......
......@@ -23,11 +23,6 @@ StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::st
{
}
StoragePtr StorageSystemAsynchronousMetrics::create(const std::string & name_, const AsynchronousMetrics & async_metrics_)
{
return make_shared(name_, async_metrics_);
}
BlockInputStreams StorageSystemAsynchronousMetrics::read(
const Names & column_names,
......
......@@ -13,13 +13,11 @@ class Context;
/** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics.
*/
class StorageSystemAsynchronousMetrics : private ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>, public IStorage
class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>;
public:
static StoragePtr create(const std::string & name_, const AsynchronousMetrics & async_metrics_);
std::string getName() const override { return "SystemAsynchronousMetrics"; }
std::string getTableName() const override { return name; }
......
......@@ -19,11 +19,6 @@ StorageSystemBuildOptions::StorageSystemBuildOptions(const std::string & name_)
{
}
StoragePtr StorageSystemBuildOptions::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemBuildOptions::read(
const Names & column_names,
......
......@@ -12,13 +12,11 @@ class Context;
/** System table "build_options" with many params used for clickhouse building
*/
class StorageSystemBuildOptions : private ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorage
class StorageSystemBuildOptions : public ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemBuildOptions>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemBuildOptions"; }
std::string getTableName() const override { return name; }
......
......@@ -29,10 +29,6 @@ StorageSystemClusters::StorageSystemClusters(const std::string & name_, Context
{
}
StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_)
{
return make_shared(name_, context_);
}
BlockInputStreams StorageSystemClusters::read(
const Names & column_names,
......
......@@ -13,13 +13,12 @@ class Context;
* that allows to obtain information about available clusters
* (which may be specified in Distributed tables).
*/
class StorageSystemClusters : private ext::shared_ptr_helper<StorageSystemClusters>, public IStorage
class StorageSystemClusters : public ext::shared_ptr_helper<StorageSystemClusters>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemClusters>;
public:
StorageSystemClusters(const std::string & name_, Context & context_);
static StoragePtr create(const std::string & name_, Context & context_);
std::string getName() const override { return "SystemClusters"; }
std::string getTableName() const override { return name; }
......
......@@ -31,10 +31,6 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
{
}
StoragePtr StorageSystemColumns::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemColumns::read(
const Names & column_names,
......
......@@ -11,12 +11,10 @@ class Context;
/** Implements system table 'columns', that allows to get information about columns for every table.
*/
class StorageSystemColumns : private ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
class StorageSystemColumns : public ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemColumns>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemColumns"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
......
......@@ -20,11 +20,6 @@ StorageSystemDatabases::StorageSystemDatabases(const std::string & name_)
{
}
StoragePtr StorageSystemDatabases::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemDatabases::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements `databases` system table, which allows you to get information about all databases.
*/
class StorageSystemDatabases : private ext::shared_ptr_helper<StorageSystemDatabases>, public IStorage
class StorageSystemDatabases : public ext::shared_ptr_helper<StorageSystemDatabases>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemDatabases>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemDatabases"; }
std::string getTableName() const override { return name; }
......
......@@ -39,10 +39,6 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
{
}
StoragePtr StorageSystemDictionaries::create(const std::string & name)
{
return make_shared(name);
}
BlockInputStreams StorageSystemDictionaries::read(
const Names & column_names,
......
......@@ -10,13 +10,10 @@ namespace DB
class Context;
class StorageSystemDictionaries : private ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorage
class StorageSystemDictionaries : public ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemDictionaries>;
public:
static StoragePtr create(const std::string & name);
std::string getName() const override { return "SystemDictionaries"; }
std::string getTableName() const override { return name; }
......
......@@ -20,11 +20,6 @@ StorageSystemEvents::StorageSystemEvents(const std::string & name_)
{
}
StoragePtr StorageSystemEvents::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemEvents::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements `events` system table, which allows you to obtain information for profiling.
*/
class StorageSystemEvents : private ext::shared_ptr_helper<StorageSystemEvents>, public IStorage
class StorageSystemEvents : public ext::shared_ptr_helper<StorageSystemEvents>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemEvents>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemEvents"; }
std::string getTableName() const override { return name; }
......
......@@ -22,10 +22,6 @@ StorageSystemFunctions::StorageSystemFunctions(const std::string & name_)
{
}
StoragePtr StorageSystemFunctions::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemFunctions::read(
const Names & column_names,
......
......@@ -13,13 +13,10 @@ class Context;
/** Implements `functions`system table, which allows you to get a list
* all normal and aggregate functions.
*/
class StorageSystemFunctions : private ext::shared_ptr_helper<StorageSystemFunctions>, public IStorage
class StorageSystemFunctions : public ext::shared_ptr_helper<StorageSystemFunctions>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemFunctions>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemFunctions"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
......
......@@ -124,10 +124,6 @@ StorageSystemGraphite::StorageSystemGraphite(const std::string & name_)
{
}
StoragePtr StorageSystemGraphite::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemGraphite::read(
const Names & column_names,
......
......@@ -6,16 +6,11 @@
namespace DB
{
/// Provides information about graphite configuration.
class StorageSystemGraphite
: private ext::shared_ptr_helper<StorageSystemGraphite>
, public IStorage
/// Provides information about Graphite configuration.
class StorageSystemGraphite : public ext::shared_ptr_helper<StorageSystemGraphite>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemGraphite>;
friend class ext::shared_ptr_helper<StorageSystemGraphite>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemGraphite"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
......
......@@ -34,10 +34,6 @@ StorageSystemMerges::StorageSystemMerges(const std::string & name)
{
}
StoragePtr StorageSystemMerges::create(const std::string & name)
{
return make_shared(name);
}
BlockInputStreams StorageSystemMerges::read(
const Names & column_names,
......
......@@ -10,13 +10,10 @@ namespace DB
class Context;
class StorageSystemMerges : private ext::shared_ptr_helper<StorageSystemMerges>, public IStorage
class StorageSystemMerges : public ext::shared_ptr_helper<StorageSystemMerges>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemMerges>;
public:
static StoragePtr create(const std::string & name);
std::string getName() const override { return "SystemMerges"; }
std::string getTableName() const override { return name; }
......
......@@ -21,11 +21,6 @@ StorageSystemMetrics::StorageSystemMetrics(const std::string & name_)
{
}
StoragePtr StorageSystemMetrics::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemMetrics::read(
const Names & column_names,
......
......@@ -11,14 +11,11 @@ class Context;
/** Implements `metrics` system table, which provides information about the operation of the server.
*/
class StorageSystemMetrics : private ext::shared_ptr_helper<StorageSystemMetrics>, public IStorage
*/
class StorageSystemMetrics : public ext::shared_ptr_helper<StorageSystemMetrics>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemMetrics>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemMetrics"; }
std::string getTableName() const override { return name; }
......
......@@ -53,11 +53,6 @@ StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multi
{
}
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
{
return make_shared(name_, multithreaded_);
}
BlockInputStreams StorageSystemNumbers::read(
const Names & column_names,
......
......@@ -14,13 +14,10 @@ class Context;
* The table contains the only column number UInt64.
* From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again).
*/
class StorageSystemNumbers : private ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemNumbers>;
public:
static StoragePtr create(const std::string & name_, bool multithreaded_ = false);
std::string getName() const override { return "SystemNumbers"; }
std::string getTableName() const override { return name; }
......
......@@ -15,11 +15,6 @@ StorageSystemOne::StorageSystemOne(const std::string & name_)
{
}
StoragePtr StorageSystemOne::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemOne::read(
const Names & column_names,
......
......@@ -15,13 +15,10 @@ class Context;
* Used when the table is not specified in the query.
* Analog of the DUAL table in Oracle and MySQL.
*/
class StorageSystemOne : private ext::shared_ptr_helper<StorageSystemOne>, public IStorage
class StorageSystemOne : public ext::shared_ptr_helper<StorageSystemOne>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemOne>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemOne"; }
std::string getTableName() const override { return name; }
......
......@@ -44,11 +44,6 @@ StorageSystemParts::StorageSystemParts(const std::string & name_)
{
}
StoragePtr StorageSystemParts::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemParts::read(
const Names & column_names,
......
......@@ -12,12 +12,10 @@ class Context;
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
*/
class StorageSystemParts : private ext::shared_ptr_helper<StorageSystemParts>, public IStorage
class StorageSystemParts : public ext::shared_ptr_helper<StorageSystemParts>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemParts>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemParts"; }
std::string getTableName() const override { return name; }
......
......@@ -52,11 +52,6 @@ StorageSystemProcesses::StorageSystemProcesses(const std::string & name_)
{
}
StoragePtr StorageSystemProcesses::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemProcesses::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements `processes` system table, which allows you to get information about the queries that are currently executing.
*/
class StorageSystemProcesses : private ext::shared_ptr_helper<StorageSystemProcesses>, public IStorage
class StorageSystemProcesses : public ext::shared_ptr_helper<StorageSystemProcesses>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemProcesses>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemProcesses"; }
std::string getTableName() const override { return name; }
......
......@@ -46,11 +46,6 @@ StorageSystemReplicas::StorageSystemReplicas(const std::string & name_)
{
}
StoragePtr StorageSystemReplicas::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemReplicas::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements `replicas` system table, which provides information about the status of the replicated tables.
*/
class StorageSystemReplicas : private ext::shared_ptr_helper<StorageSystemReplicas>, public IStorage
class StorageSystemReplicas : public ext::shared_ptr_helper<StorageSystemReplicas>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemReplicas>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemReplicas"; }
std::string getTableName() const override { return name; }
......
......@@ -46,11 +46,6 @@ StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string &
{
}
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemReplicationQueue::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables.
*/
class StorageSystemReplicationQueue : private ext::shared_ptr_helper<StorageSystemReplicationQueue>, public IStorage
class StorageSystemReplicationQueue : public ext::shared_ptr_helper<StorageSystemReplicationQueue>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemReplicationQueue>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemReplicationQueue"; }
std::string getTableName() const override { return name; }
......
......@@ -21,11 +21,6 @@ StorageSystemSettings::StorageSystemSettings(const std::string & name_)
{
}
StoragePtr StorageSystemSettings::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemSettings::read(
const Names & column_names,
......
......@@ -12,13 +12,10 @@ class Context;
/** implements system table "settings", which allows to get information about the current settings.
*/
class StorageSystemSettings : private ext::shared_ptr_helper<StorageSystemSettings>, public IStorage
class StorageSystemSettings : public ext::shared_ptr_helper<StorageSystemSettings>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemSettings>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemSettings"; }
std::string getTableName() const override { return name; }
......
......@@ -30,11 +30,6 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
{
}
StoragePtr StorageSystemTables::create(const std::string & name_)
{
return make_shared(name_);
}
static ColumnWithTypeAndName getFilteredDatabases(const ASTPtr & query, const Context & context)
{
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements the system table `tables`, which allows you to get information about all tables.
*/
class StorageSystemTables : private ext::shared_ptr_helper<StorageSystemTables>, public IStorage
class StorageSystemTables : public ext::shared_ptr_helper<StorageSystemTables>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemTables>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemTables"; }
std::string getTableName() const override { return name; }
......
......@@ -21,29 +21,24 @@ namespace DB
StorageSystemZooKeeper::StorageSystemZooKeeper(const std::string & name_)
: name(name_)
, columns{
{ "name", std::make_shared<DataTypeString>() },
{ "value", std::make_shared<DataTypeString>() },
{ "czxid", std::make_shared<DataTypeInt64>() },
{ "mzxid", std::make_shared<DataTypeInt64>() },
{ "ctime", std::make_shared<DataTypeDateTime>()},
{ "mtime", std::make_shared<DataTypeDateTime>()},
{ "version", std::make_shared<DataTypeInt32>() },
{ "cversion", std::make_shared<DataTypeInt32>() },
{ "aversion", std::make_shared<DataTypeInt32>() },
{ "ephemeralOwner", std::make_shared<DataTypeInt64>() },
{ "dataLength", std::make_shared<DataTypeInt32>() },
{ "numChildren", std::make_shared<DataTypeInt32>() },
{ "pzxid", std::make_shared<DataTypeInt64>() },
{ "path", std::make_shared<DataTypeString>() },
{ "name", std::make_shared<DataTypeString>() },
{ "value", std::make_shared<DataTypeString>() },
{ "czxid", std::make_shared<DataTypeInt64>() },
{ "mzxid", std::make_shared<DataTypeInt64>() },
{ "ctime", std::make_shared<DataTypeDateTime>() },
{ "mtime", std::make_shared<DataTypeDateTime>() },
{ "version", std::make_shared<DataTypeInt32>() },
{ "cversion", std::make_shared<DataTypeInt32>() },
{ "aversion", std::make_shared<DataTypeInt32>() },
{ "ephemeralOwner", std::make_shared<DataTypeInt64>() },
{ "dataLength", std::make_shared<DataTypeInt32>() },
{ "numChildren", std::make_shared<DataTypeInt32>() },
{ "pzxid", std::make_shared<DataTypeInt64>() },
{ "path", std::make_shared<DataTypeString>() },
}
{
}
StoragePtr StorageSystemZooKeeper::create(const std::string & name_)
{
return make_shared(name_);
}
static bool extractPathImpl(const IAST & elem, String & res)
{
......
......@@ -12,13 +12,10 @@ class Context;
/** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes.
*/
class StorageSystemZooKeeper : private ext::shared_ptr_helper<StorageSystemZooKeeper>, public IStorage
class StorageSystemZooKeeper : public ext::shared_ptr_helper<StorageSystemZooKeeper>, public IStorage
{
friend class ext::shared_ptr_helper<StorageSystemZooKeeper>;
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemZooKeeper"; }
std::string getTableName() const override { return name; }
......
......@@ -26,7 +26,7 @@ namespace DB
void attachSystemTablesLocal(DatabasePtr system_database)
{
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
system_database->attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true));
system_database->attachTable("databases", StorageSystemDatabases::create("databases"));
system_database->attachTable("tables", StorageSystemTables::create("tables"));
......
......@@ -15,7 +15,7 @@ try
{
using namespace DB;
StoragePtr table = StorageSystemNumbers::create("Numbers");
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
......
......@@ -28,7 +28,7 @@ protected:
};
/// see std::allocate_shared
template <typename TAlloc, typename ... TArgs>
template <typename TAlloc, typename... TArgs>
static std::shared_ptr<T> allocate_shared(const TAlloc & alloc, TArgs &&... args)
{
using AllocTraits = std::allocator_traits<TAlloc>;
......@@ -52,11 +52,20 @@ protected:
alloc_copy);
}
template <typename ... TArgs>
template <typename... TArgs>
static std::shared_ptr<T> make_shared(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)...);
}
};
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册