mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 01:12:12 +00:00
330212e0f4
The original motivation for this commit was that shared_ptr_helper used std::shared_ptr<>() which does two heap allocations instead of make_shared<>() which does a single allocation. Turned out that 1. the affected code (--> Storages/) is not on a hot path (rendering the performance argument moot ...) 2. yet copying Storage objects is potentially dangerous and was previously allowed. Hence, this change - removes shared_ptr_helper and as a result all inherited create() methods, - instead, Storage objects are now created using make_shared<>() by the caller (for that to work, many constructors had to be made public), and - all Storage classes were marked as noncopyable using boost::noncopyable. In sum, we are (likely) not making things faster but the code becomes cleaner and harder to misuse.
68 lines
2.4 KiB
C++
68 lines
2.4 KiB
C++
#pragma once
|
|
|
|
#include <Core/NamesAndAliases.h>
|
|
#include <DataTypes/DataTypeString.h>
|
|
#include <Storages/ColumnsDescription.h>
|
|
#include <Storages/IStorage.h>
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
|
#include <QueryPipeline/Pipe.h>
|
|
|
|
namespace DB
|
|
{
|
|
|
|
class Context;
|
|
|
|
|
|
/** IStorageSystemOneBlock is base class for system tables whose all columns can be synchronously fetched.
|
|
*
|
|
* Client class need to provide static method static NamesAndTypesList getNamesAndTypes() that will return list of column names and
|
|
* their types. IStorageSystemOneBlock during read will create result columns in same order as result of getNamesAndTypes
|
|
* and pass it with fillData method.
|
|
*
|
|
* Client also must override fillData and fill result columns.
|
|
*
|
|
* If subclass want to support virtual columns, it should override getVirtuals method of IStorage interface.
|
|
* IStorageSystemOneBlock will add virtuals columns at the end of result columns of fillData method.
|
|
*/
|
|
template <typename Self>
|
|
class IStorageSystemOneBlock : public IStorage
|
|
{
|
|
protected:
|
|
virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0;
|
|
|
|
public:
|
|
explicit IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_)
|
|
{
|
|
StorageInMemoryMetadata storage_metadata;
|
|
storage_metadata.setColumns(ColumnsDescription(Self::getNamesAndTypes(), Self::getNamesAndAliases()));
|
|
setInMemoryMetadata(storage_metadata);
|
|
}
|
|
|
|
Pipe read(
|
|
const Names & column_names,
|
|
const StorageSnapshotPtr & storage_snapshot,
|
|
SelectQueryInfo & query_info,
|
|
ContextPtr context,
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
size_t /*max_block_size*/,
|
|
unsigned /*num_streams*/) override
|
|
{
|
|
storage_snapshot->check(column_names);
|
|
|
|
Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals());
|
|
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
|
fillData(res_columns, context, query_info);
|
|
|
|
UInt64 num_rows = res_columns.at(0)->size();
|
|
Chunk chunk(std::move(res_columns), num_rows);
|
|
|
|
return Pipe(std::make_shared<SourceFromSingleChunk>(sample_block, std::move(chunk)));
|
|
}
|
|
|
|
bool isSystemStorage() const override { return true; }
|
|
|
|
static NamesAndAliases getNamesAndAliases() { return {}; }
|
|
};
|
|
|
|
}
|