2020-07-30 01:17:33 +00:00
|
|
|
#include <Storages/System/StorageSystemStoragePolicies.h>
|
|
|
|
|
2019-05-24 19:03:07 +00:00
|
|
|
#include <Columns/ColumnArray.h>
|
2020-07-30 01:17:33 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
2019-05-24 19:03:07 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2020-07-30 01:17:33 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2020-01-24 18:17:02 +00:00
|
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/EnumReflection.h>
|
2022-05-24 19:29:00 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2019-05-24 19:03:07 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-12 20:40:13 +00:00
|
|
|
StorageSystemStoragePolicies::StorageSystemStoragePolicies(const StorageID & table_id_)
|
|
|
|
: IStorage(table_id_)
|
2019-05-24 19:03:07 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(
|
2019-09-09 13:50:19 +00:00
|
|
|
ColumnsDescription({
|
|
|
|
{"policy_name", std::make_shared<DataTypeString>()},
|
|
|
|
{"volume_name", std::make_shared<DataTypeString>()},
|
|
|
|
{"volume_priority", std::make_shared<DataTypeUInt64>()},
|
|
|
|
{"disks", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
2020-05-28 05:38:55 +00:00
|
|
|
{"volume_type", std::make_shared<DataTypeString>()},
|
2020-07-30 10:04:49 +00:00
|
|
|
{"max_data_part_size", std::make_shared<DataTypeUInt64>()},
|
2020-07-30 01:17:33 +00:00
|
|
|
{"move_factor", std::make_shared<DataTypeFloat32>()},
|
2020-10-20 15:10:24 +00:00
|
|
|
{"prefer_not_to_merge", std::make_shared<DataTypeUInt8>()}
|
2019-05-24 19:03:07 +00:00
|
|
|
}));
|
2020-07-08 15:52:23 +00:00
|
|
|
// TODO: Add string column with custom volume-type-specific options
|
2020-06-19 15:39:41 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2019-05-24 19:03:07 +00:00
|
|
|
}
|
|
|
|
|
2020-08-06 12:24:05 +00:00
|
|
|
Pipe StorageSystemStoragePolicies::read(
|
2020-06-15 19:08:58 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-06-15 19:08:58 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
const size_t /*max_block_size*/,
|
|
|
|
const unsigned /*num_streams*/)
|
2019-05-24 19:03:07 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2019-05-24 19:03:07 +00:00
|
|
|
|
2019-09-09 13:50:19 +00:00
|
|
|
MutableColumnPtr col_policy_name = ColumnString::create();
|
|
|
|
MutableColumnPtr col_volume_name = ColumnString::create();
|
|
|
|
MutableColumnPtr col_priority = ColumnUInt64::create();
|
|
|
|
MutableColumnPtr col_disks = ColumnArray::create(ColumnString::create());
|
2020-05-28 05:38:55 +00:00
|
|
|
MutableColumnPtr col_volume_type = ColumnString::create();
|
2020-07-30 10:04:49 +00:00
|
|
|
MutableColumnPtr col_max_part_size = ColumnUInt64::create();
|
2019-09-09 13:50:19 +00:00
|
|
|
MutableColumnPtr col_move_factor = ColumnFloat32::create();
|
2020-10-20 15:10:24 +00:00
|
|
|
MutableColumnPtr col_prefer_not_to_merge = ColumnUInt8::create();
|
2019-05-24 19:03:07 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
for (const auto & [policy_name, policy_ptr] : context->getPoliciesMap())
|
2019-05-24 19:03:07 +00:00
|
|
|
{
|
|
|
|
const auto & volumes = policy_ptr->getVolumes();
|
|
|
|
for (size_t i = 0; i != volumes.size(); ++i)
|
|
|
|
{
|
2019-09-09 13:50:19 +00:00
|
|
|
col_policy_name->insert(policy_name);
|
|
|
|
col_volume_name->insert(volumes[i]->getName());
|
|
|
|
col_priority->insert(i + 1);
|
2019-05-24 19:03:07 +00:00
|
|
|
Array disks;
|
2020-05-09 21:24:15 +00:00
|
|
|
disks.reserve(volumes[i]->getDisks().size());
|
|
|
|
for (const auto & disk_ptr : volumes[i]->getDisks())
|
2019-05-24 19:03:07 +00:00
|
|
|
disks.push_back(disk_ptr->getName());
|
2019-09-09 13:50:19 +00:00
|
|
|
col_disks->insert(disks);
|
2021-09-06 14:24:03 +00:00
|
|
|
col_volume_type->insert(magic_enum::enum_name(volumes[i]->getType()));
|
2020-07-30 10:04:49 +00:00
|
|
|
col_max_part_size->insert(volumes[i]->max_data_part_size);
|
2019-09-09 13:50:19 +00:00
|
|
|
col_move_factor->insert(policy_ptr->getMoveFactor());
|
2020-10-20 15:10:24 +00:00
|
|
|
col_prefer_not_to_merge->insert(volumes[i]->areMergesAvoided() ? 1 : 0);
|
2019-05-24 19:03:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-24 18:33:47 +00:00
|
|
|
Columns res_columns;
|
|
|
|
res_columns.emplace_back(std::move(col_policy_name));
|
|
|
|
res_columns.emplace_back(std::move(col_volume_name));
|
|
|
|
res_columns.emplace_back(std::move(col_priority));
|
|
|
|
res_columns.emplace_back(std::move(col_disks));
|
2020-05-28 05:38:55 +00:00
|
|
|
res_columns.emplace_back(std::move(col_volume_type));
|
2020-07-30 01:17:33 +00:00
|
|
|
res_columns.emplace_back(std::move(col_max_part_size));
|
2020-01-24 18:33:47 +00:00
|
|
|
res_columns.emplace_back(std::move(col_move_factor));
|
2020-10-20 15:10:24 +00:00
|
|
|
res_columns.emplace_back(std::move(col_prefer_not_to_merge));
|
2019-05-24 19:03:07 +00:00
|
|
|
|
2020-01-24 18:33:47 +00:00
|
|
|
UInt64 num_rows = res_columns.at(0)->size();
|
|
|
|
Chunk chunk(std::move(res_columns), num_rows);
|
2019-09-09 13:50:19 +00:00
|
|
|
|
2021-07-09 03:15:41 +00:00
|
|
|
return Pipe(std::make_shared<SourceFromSingleChunk>(storage_snapshot->metadata->getSampleBlock(), std::move(chunk)));
|
2019-05-24 19:03:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|