2020-10-14 12:19:29 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Storages/IStorage.h>
|
2022-02-02 18:31:24 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2020-10-14 12:19:29 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class StorageProxy : public IStorage
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
2021-03-16 18:41:29 +00:00
|
|
|
explicit StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {}
|
2020-10-14 12:19:29 +00:00
|
|
|
|
|
|
|
virtual StoragePtr getNested() const = 0;
|
|
|
|
|
|
|
|
String getName() const override { return "StorageProxy"; }
|
|
|
|
|
|
|
|
bool isRemote() const override { return getNested()->isRemote(); }
|
|
|
|
bool isView() const override { return getNested()->isView(); }
|
|
|
|
bool supportsSampling() const override { return getNested()->supportsSampling(); }
|
|
|
|
bool supportsFinal() const override { return getNested()->supportsFinal(); }
|
|
|
|
bool supportsPrewhere() const override { return getNested()->supportsPrewhere(); }
|
|
|
|
bool supportsReplication() const override { return getNested()->supportsReplication(); }
|
|
|
|
bool supportsParallelInsert() const override { return getNested()->supportsParallelInsert(); }
|
|
|
|
bool supportsDeduplication() const override { return getNested()->supportsDeduplication(); }
|
|
|
|
bool noPushingToViews() const override { return getNested()->noPushingToViews(); }
|
|
|
|
bool hasEvenlyDistributedRead() const override { return getNested()->hasEvenlyDistributedRead(); }
|
|
|
|
|
|
|
|
ColumnSizeByName getColumnSizes() const override { return getNested()->getColumnSizes(); }
|
2020-11-07 21:28:39 +00:00
|
|
|
|
|
|
|
QueryProcessingStage::Enum getQueryProcessingStage(
|
2021-04-22 13:32:17 +00:00
|
|
|
ContextPtr context,
|
|
|
|
QueryProcessingStage::Enum to_stage,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr &,
|
2021-04-22 13:32:17 +00:00
|
|
|
SelectQueryInfo & info) const override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
const auto & nested_metadata = getNested()->getInMemoryMetadataPtr();
|
2022-03-17 17:26:18 +00:00
|
|
|
return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata, context), info);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
2021-08-11 17:28:54 +00:00
|
|
|
Pipe watch(
|
2020-10-14 12:19:29 +00:00
|
|
|
const Names & column_names,
|
|
|
|
const SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-10-14 12:19:29 +00:00
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
|
|
|
return getNested()->watch(column_names, query_info, context, processed_stage, max_block_size, num_streams);
|
|
|
|
}
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
void read(
|
|
|
|
QueryPlan & query_plan,
|
2020-10-14 12:19:29 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-11-07 21:28:39 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-10-14 12:19:29 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2022-05-20 19:49:31 +00:00
|
|
|
return getNested()->read(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
2023-06-07 18:33:08 +00:00
|
|
|
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2023-06-07 18:33:08 +00:00
|
|
|
return getNested()->write(query, metadata_snapshot, context, async_insert);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void drop() override { getNested()->drop(); }
|
|
|
|
|
|
|
|
void truncate(
|
|
|
|
const ASTPtr & query,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-10-14 12:19:29 +00:00
|
|
|
TableExclusiveLockHolder & lock) override
|
|
|
|
{
|
|
|
|
getNested()->truncate(query, metadata_snapshot, context, lock);
|
|
|
|
}
|
|
|
|
|
|
|
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override
|
|
|
|
{
|
|
|
|
getNested()->rename(new_path_to_table_data, new_table_id);
|
|
|
|
IStorage::renameInMemory(new_table_id);
|
|
|
|
}
|
|
|
|
|
|
|
|
void renameInMemory(const StorageID & new_table_id) override
|
|
|
|
{
|
|
|
|
getNested()->renameInMemory(new_table_id);
|
|
|
|
IStorage::renameInMemory(new_table_id);
|
|
|
|
}
|
|
|
|
|
2021-10-25 17:49:49 +00:00
|
|
|
void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & alter_lock_holder) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
|
|
|
getNested()->alter(params, context, alter_lock_holder);
|
|
|
|
IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata());
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2021-02-28 05:24:39 +00:00
|
|
|
getNested()->checkAlterIsPossible(commands, context);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Pipe alterPartition(
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const PartitionCommands & commands,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2020-11-02 16:18:18 +00:00
|
|
|
return getNested()->alterPartition(metadata_snapshot, commands, context);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
2023-10-13 14:22:18 +00:00
|
|
|
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings, ContextPtr context) const override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2023-10-13 14:22:18 +00:00
|
|
|
getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings, context);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool optimize(
|
2023-12-28 13:07:59 +00:00
|
|
|
const ASTPtr & query,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const ASTPtr & partition,
|
|
|
|
bool final,
|
|
|
|
bool deduplicate,
|
|
|
|
const Names & deduplicate_by_columns,
|
|
|
|
bool cleanup,
|
|
|
|
ContextPtr context) override
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2023-12-28 13:07:59 +00:00
|
|
|
return getNested()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, context);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
2023-01-30 17:38:28 +00:00
|
|
|
void mutate(const MutationCommands & commands, ContextPtr context) override { getNested()->mutate(commands, context); }
|
2020-10-14 12:19:29 +00:00
|
|
|
|
|
|
|
CancellationCode killMutation(const String & mutation_id) override { return getNested()->killMutation(mutation_id); }
|
|
|
|
|
|
|
|
void startup() override { getNested()->startup(); }
|
2023-11-06 14:40:01 +00:00
|
|
|
void shutdown(bool is_drop) override { getNested()->shutdown(is_drop); }
|
2023-07-05 16:11:25 +00:00
|
|
|
void flushAndPrepareForShutdown() override { getNested()->flushAndPrepareForShutdown(); }
|
2020-10-14 12:19:29 +00:00
|
|
|
|
|
|
|
ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); }
|
|
|
|
|
2023-10-24 12:50:24 +00:00
|
|
|
DataValidationTasksPtr getCheckTaskList(const CheckTaskFilter & check_task_filter, ContextPtr context) override
|
|
|
|
{
|
|
|
|
return getNested()->getCheckTaskList(check_task_filter, context);
|
|
|
|
}
|
|
|
|
|
|
|
|
std::optional<CheckResult> checkDataNext(DataValidationTasksPtr & check_task_list) override
|
|
|
|
{
|
|
|
|
return getNested()->checkDataNext(check_task_list);
|
|
|
|
}
|
2023-08-14 09:58:08 +00:00
|
|
|
|
2023-08-29 14:26:48 +00:00
|
|
|
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { getNested()->checkTableCanBeDropped(query_context); }
|
2023-08-14 09:58:08 +00:00
|
|
|
|
2020-11-01 17:38:43 +00:00
|
|
|
bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); }
|
2020-10-14 12:19:29 +00:00
|
|
|
Strings getDataPaths() const override { return getNested()->getDataPaths(); }
|
|
|
|
StoragePolicyPtr getStoragePolicy() const override { return getNested()->getStoragePolicy(); }
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> totalRows(const Settings & settings) const override { return getNested()->totalRows(settings); }
|
|
|
|
std::optional<UInt64> totalBytes(const Settings & settings) const override { return getNested()->totalBytes(settings); }
|
2020-10-14 12:19:29 +00:00
|
|
|
std::optional<UInt64> lifetimeRows() const override { return getNested()->lifetimeRows(); }
|
|
|
|
std::optional<UInt64> lifetimeBytes() const override { return getNested()->lifetimeBytes(); }
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
}
|