ClickHouse/src/Storages/System/StorageSystemPartsBase.h
Ivan 495c6e03aa
Replace all Context references with std::weak_ptr (#22297)
* Replace all Context references with std::weak_ptr

* Fix shared context captured by value

* Fix build

* Fix Context with named sessions

* Fix copy context

* Fix gcc build

* Merge with master and fix build

* Fix gcc-9 build
2021-04-11 02:33:54 +03:00

82 lines
2.0 KiB
C++

#pragma once
#include <ext/shared_ptr_helper.h>
#include <Formats/FormatSettings.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
namespace DB
{
class Context;
struct StoragesInfo
{
StoragePtr storage = nullptr;
TableLockHolder table_lock;
String database;
String table;
String engine;
bool need_inactive_parts = false;
MergeTreeData * data = nullptr;
operator bool() const { return storage != nullptr; }
MergeTreeData::DataPartsVector getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const;
};
/** A helper class that enumerates the storages that match given query. */
class StoragesInfoStream
{
public:
StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context);
StoragesInfo next();
private:
String query_id;
Settings settings;
ColumnPtr database_column;
ColumnPtr table_column;
ColumnPtr active_column;
size_t next_row;
size_t rows;
using StoragesMap = std::map<std::pair<String, String>, StoragePtr>;
StoragesMap storages;
};
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
*/
class StorageSystemPartsBase : public IStorage
{
public:
Pipe read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
NamesAndTypesList getVirtuals() const override;
private:
bool hasStateColumn(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const;
protected:
const FormatSettings format_settings;
StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_);
virtual void
processNextStorage(MutableColumns & columns, std::vector<UInt8> & columns_mask, const StoragesInfo & info, bool has_state_column) = 0;
};
}