ClickHouse/src/Storages/System/StorageSystemRemoteDataPaths.cpp

112 lines
4.4 KiB
C++
Raw Normal View History

2022-03-23 12:01:18 +00:00
#include "StorageSystemRemoteDataPaths.h"
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
2022-09-05 16:55:00 +00:00
#include <DataTypes/DataTypesNumber.h>
2022-08-26 17:23:46 +00:00
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileCacheFactory.h>
2022-03-23 12:01:18 +00:00
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Interpreters/Context.h>
#include <Disks/IDisk.h>
namespace DB
{
StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & table_id_)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(
{
{"disk_name", std::make_shared<DataTypeString>()},
{"path", std::make_shared<DataTypeString>()},
2022-03-23 17:11:52 +00:00
{"cache_base_path", std::make_shared<DataTypeString>()},
2022-03-23 12:01:18 +00:00
{"local_path", std::make_shared<DataTypeString>()},
{"remote_path", std::make_shared<DataTypeString>()},
2022-09-05 16:55:00 +00:00
{"size", std::make_shared<DataTypeUInt64>()},
{"common_prefix_for_blobs", std::make_shared<DataTypeString>()},
2022-03-23 12:01:18 +00:00
{"cache_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
}));
setInMemoryMetadata(storage_metadata);
}
Pipe StorageSystemRemoteDataPaths::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & /*query_info*/,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
storage_snapshot->check(column_names);
MutableColumnPtr col_disk_name = ColumnString::create();
MutableColumnPtr col_base_path = ColumnString::create();
MutableColumnPtr col_cache_base_path = ColumnString::create();
MutableColumnPtr col_local_path = ColumnString::create();
MutableColumnPtr col_remote_path = ColumnString::create();
2022-09-05 16:55:00 +00:00
MutableColumnPtr col_size = ColumnUInt64::create();
MutableColumnPtr col_namespace = ColumnString::create();
2022-03-23 12:01:18 +00:00
MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create());
auto disks = context->getDisksMap();
for (const auto & [disk_name, disk] : disks)
{
if (disk->isRemote())
{
std::vector<IDisk::LocalPathWithObjectStoragePaths> remote_paths_by_local_path;
2022-03-23 12:01:18 +00:00
disk->getRemotePathsRecursive("store", remote_paths_by_local_path);
2022-04-11 20:24:45 +00:00
disk->getRemotePathsRecursive("data", remote_paths_by_local_path);
2022-03-23 12:01:18 +00:00
FileCachePtr cache;
2022-07-13 14:50:31 +00:00
auto cache_base_path = disk->supportsCache() ? disk->getCacheBasePath() : "";
2022-04-12 08:52:37 +00:00
2022-03-23 12:01:18 +00:00
if (!cache_base_path.empty())
cache = FileCacheFactory::instance().get(cache_base_path);
2022-09-05 16:55:00 +00:00
for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path)
2022-03-23 12:01:18 +00:00
{
for (const auto & object : storage_objects)
2022-03-23 12:01:18 +00:00
{
col_disk_name->insert(disk_name);
col_base_path->insert(disk->getPath());
col_cache_base_path->insert(cache_base_path);
col_local_path->insert(local_path);
2022-07-09 17:35:07 +00:00
col_remote_path->insert(object.absolute_path);
2022-09-05 16:55:00 +00:00
col_size->insert(object.bytes_size);
col_namespace->insert(common_prefox_for_objects);
2022-03-23 12:01:18 +00:00
if (cache)
{
2022-07-09 17:35:07 +00:00
auto cache_paths = cache->tryGetCachePaths(cache->hash(object.getPathKeyForCache()));
2022-03-23 12:01:18 +00:00
col_cache_paths->insert(Array(cache_paths.begin(), cache_paths.end()));
}
else
{
col_cache_paths->insertDefault();
}
}
}
}
}
Columns res_columns;
res_columns.emplace_back(std::move(col_disk_name));
res_columns.emplace_back(std::move(col_base_path));
res_columns.emplace_back(std::move(col_cache_base_path));
res_columns.emplace_back(std::move(col_local_path));
res_columns.emplace_back(std::move(col_remote_path));
2022-09-05 16:55:00 +00:00
res_columns.emplace_back(std::move(col_size));
res_columns.emplace_back(std::move(col_namespace));
2022-03-23 12:01:18 +00:00
res_columns.emplace_back(std::move(col_cache_paths));
UInt64 num_rows = res_columns.at(0)->size();
Chunk chunk(std::move(res_columns), num_rows);
return Pipe(std::make_shared<SourceFromSingleChunk>(storage_snapshot->metadata->getSampleBlock(), std::move(chunk)));
}
}