ClickHouse/src/Storages/StorageProxy.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

166 lines
7.0 KiB
C++
Raw Normal View History

#pragma once
#include <Storages/IStorage.h>
#include <Storages/SelectQueryInfo.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.h>
namespace DB
{
class StorageProxy : public IStorage
{
public:
2021-03-16 18:41:29 +00:00
explicit StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {}
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(); }
NamesAndTypesList getVirtuals() const override { return getNested()->getVirtuals(); }
2020-11-07 21:28:39 +00:00
QueryProcessingStage::Enum getQueryProcessingStage(
ContextPtr context,
QueryProcessingStage::Enum to_stage,
const StorageSnapshotPtr &,
SelectQueryInfo & info) const override
{
/// TODO: Find a way to support projections for StorageProxy
info.ignore_projections = true;
const auto & nested_metadata = getNested()->getInMemoryMetadataPtr();
return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata, context), info);
}
2021-08-11 17:28:54 +00:00
Pipe watch(
const Names & column_names,
const SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,
size_t num_streams) override
{
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,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
2020-11-07 21:28:39 +00:00
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override
{
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);
}
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override
{
return getNested()->write(query, metadata_snapshot, context, async_insert);
}
void drop() override { getNested()->drop(); }
void truncate(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
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
{
getNested()->alter(params, context, alter_lock_holder);
IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata());
}
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override
{
getNested()->checkAlterIsPossible(commands, context);
}
Pipe alterPartition(
const StorageMetadataPtr & metadata_snapshot,
const PartitionCommands & commands,
ContextPtr context) override
{
2020-11-02 16:18:18 +00:00
return getNested()->alterPartition(metadata_snapshot, commands, context);
}
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override
{
getNested()->checkAlterPartitionIsPossible(commands, metadata_snapshot, settings);
}
bool optimize(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
const ASTPtr & partition,
bool final,
bool deduplicate,
const Names & deduplicate_by_columns,
[RFC] Replacing merge tree new engine (#41005) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add keyword 'CLEANUP' when OPTIMIZE * Cleanup uniquely when it's a replacingMergeTree * Propagate CLEANUP information and change from 'with_cleanup' to 'cleanup' * Cleanup data flagged as 'is_deleted' * Fix merge when optimize and add a test * Fix OPTIMIZE and INSERT + add tests * New fix for cleanup at the merge * Cleanup debug logs * Add the SETTINGS option 'clean_deleted_rows' that can be 'never' or 'always' * Fix regression bug; Now REplicatedMergeTree can be called as before without 'is_deleted' * Add Replicated tests * Disable tag 'long' for our test and cleanup some white spaces * Update tests * Fix tests and remove additional useless whitespace * Fix replica test * Style clean && add condition check for is_deleted values * clean_deleted_rows settings is nom an enum * Add valid default value to the clean_deleted_rows settings * Update cleanup checkers to use the enum and fix typos in the test * Fix submodule contrib/AMQP-CPP pointer * Add missing messages in test reference and remove a print with non derterministic order * fix replica test reference * Fix edge case * Fix a typo for the spell checker * Fix reference * Fix a condition to raise an error if is_deleted differ from 0/1 and cleanup * Change tests file name and update number * This should fix the ReplacingMergeTree parameter set * Fix replicated parameters * Disable allow_deprecated_syntax_for_merge_tree for our new column * Fix a test * Remove non deterministic order print in the test * Test on replicas * Remove a condition, when checking optional parameters, that should not be sueful since we disabled the deprected_syntaxe * Revert "Remove a condition, when checking optional parameters, that should not be useful since we disabled the deprected_syntaxe" This reverts commit b65d64c05e482945ac20fcfcf0311e1b028ea137. * Fix replica management and limit the number of argument to two maximum, due to the possiblity of deprecated table create/attach failing otherwise * Test a fix for replicated log information error * Try to add sync to have consistent results * Change path of replicas that should cause one issue and add few prints in case it's not that * Get cleanup info on replicas only if information found * Fix style issues * Try to avoid replication error 'cannot select parts...' and and replica read/write field order * Cleanup according to PR reviews and add tests on error raised. * Update src/Storages/MergeTree/registerStorageMergeTree.cpp Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com> * Select ... FINAL don't show rows with is_deleted = true * Update and fix SELECT ... FINAL merge parameter * Remove is_deleted rows only on the version inserted when merge * Fix (master) updates issues * Revert changes that should not be commited * Add changes according to review * Revert changes that should not be commited - part 2 --------- Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com>
2023-02-16 13:03:16 +00:00
bool cleanup,
ContextPtr context) override
{
[RFC] Replacing merge tree new engine (#41005) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add keyword 'CLEANUP' when OPTIMIZE * Cleanup uniquely when it's a replacingMergeTree * Propagate CLEANUP information and change from 'with_cleanup' to 'cleanup' * Cleanup data flagged as 'is_deleted' * Fix merge when optimize and add a test * Fix OPTIMIZE and INSERT + add tests * New fix for cleanup at the merge * Cleanup debug logs * Add the SETTINGS option 'clean_deleted_rows' that can be 'never' or 'always' * Fix regression bug; Now REplicatedMergeTree can be called as before without 'is_deleted' * Add Replicated tests * Disable tag 'long' for our test and cleanup some white spaces * Update tests * Fix tests and remove additional useless whitespace * Fix replica test * Style clean && add condition check for is_deleted values * clean_deleted_rows settings is nom an enum * Add valid default value to the clean_deleted_rows settings * Update cleanup checkers to use the enum and fix typos in the test * Fix submodule contrib/AMQP-CPP pointer * Add missing messages in test reference and remove a print with non derterministic order * fix replica test reference * Fix edge case * Fix a typo for the spell checker * Fix reference * Fix a condition to raise an error if is_deleted differ from 0/1 and cleanup * Change tests file name and update number * This should fix the ReplacingMergeTree parameter set * Fix replicated parameters * Disable allow_deprecated_syntax_for_merge_tree for our new column * Fix a test * Remove non deterministic order print in the test * Test on replicas * Remove a condition, when checking optional parameters, that should not be sueful since we disabled the deprected_syntaxe * Revert "Remove a condition, when checking optional parameters, that should not be useful since we disabled the deprected_syntaxe" This reverts commit b65d64c05e482945ac20fcfcf0311e1b028ea137. * Fix replica management and limit the number of argument to two maximum, due to the possiblity of deprecated table create/attach failing otherwise * Test a fix for replicated log information error * Try to add sync to have consistent results * Change path of replicas that should cause one issue and add few prints in case it's not that * Get cleanup info on replicas only if information found * Fix style issues * Try to avoid replication error 'cannot select parts...' and and replica read/write field order * Cleanup according to PR reviews and add tests on error raised. * Update src/Storages/MergeTree/registerStorageMergeTree.cpp Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com> * Select ... FINAL don't show rows with is_deleted = true * Update and fix SELECT ... FINAL merge parameter * Remove is_deleted rows only on the version inserted when merge * Fix (master) updates issues * Revert changes that should not be commited * Add changes according to review * Revert changes that should not be commited - part 2 --------- Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com>
2023-02-16 13:03:16 +00:00
return getNested()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, context);
}
2023-01-30 17:38:28 +00:00
void mutate(const MutationCommands & commands, ContextPtr context) override { getNested()->mutate(commands, context); }
CancellationCode killMutation(const String & mutation_id) override { return getNested()->killMutation(mutation_id); }
void startup() override { getNested()->startup(); }
void shutdown() override { getNested()->shutdown(); }
void flushAndPrepareForShutdown() override { getNested()->flushAndPrepareForShutdown(); }
ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); }
bool supportsIndexForIn() const override { return getNested()->supportsIndexForIn(); }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override
{
return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
}
CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); }
2023-08-29 14:26:48 +00:00
void checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const override { getNested()->checkTableCanBeDropped(query_context); }
2020-11-01 17:38:43 +00:00
bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); }
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); }
std::optional<UInt64> lifetimeRows() const override { return getNested()->lifetimeRows(); }
std::optional<UInt64> lifetimeBytes() const override { return getNested()->lifetimeBytes(); }
};
}