ClickHouse/src/Storages/StorageMaterializedView.h
youennL-cs 6526c2a8ab
[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 b65d64c05e.

* 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 16:03:16 +03:00

116 lines
4.6 KiB
C++

#pragma once
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage.h>
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
{
class StorageMaterializedView final : public IStorage, WithMutableContext
{
public:
StorageMaterializedView(
const StorageID & table_id_,
ContextPtr local_context,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
bool attach_,
const String & comment);
std::string getName() const override { return "MaterializedView"; }
bool isView() const override { return true; }
bool hasInnerTable() const { return has_inner_table; }
bool supportsSampling() const override { return getTargetTable()->supportsSampling(); }
bool supportsPrewhere() const override { return getTargetTable()->supportsPrewhere(); }
bool supportsFinal() const override { return getTargetTable()->supportsFinal(); }
bool supportsIndexForIn() const override { return getTargetTable()->supportsIndexForIn(); }
bool supportsParallelInsert() const override { return getTargetTable()->supportsParallelInsert(); }
bool supportsSubcolumns() const override { return getTargetTable()->supportsSubcolumns(); }
bool supportsTransactions() const override { return getTargetTable()->supportsTransactions(); }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /* metadata_snapshot */) const override
{
auto target_table = getTargetTable();
auto metadata_snapshot = target_table->getInMemoryMetadataPtr();
return target_table->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
}
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
void drop() override;
void dropInnerTableIfAny(bool sync, ContextPtr local_context) override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
bool optimize(
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
const ASTPtr & partition,
bool final,
bool deduplicate,
const Names & deduplicate_by_columns,
bool cleanup,
ContextPtr context) override;
void alter(const AlterCommands & params, ContextPtr context, AlterLockHolder & table_lock_holder) override;
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override;
Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, ContextPtr context) override;
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
void mutate(const MutationCommands & commands, ContextPtr context) override;
void renameInMemory(const StorageID & new_table_id) override;
void startup() override;
void shutdown() override;
QueryProcessingStage::Enum
getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override;
StoragePtr getTargetTable() const;
StoragePtr tryGetTargetTable() const;
/// Get the virtual column of the target table;
NamesAndTypesList getVirtuals() const override;
ActionLock getActionLock(StorageActionBlockType type) override;
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
Strings getDataPaths() const override;
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
bool supportsBackupPartition() const override;
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;
private:
/// Will be initialized in constructor
StorageID target_table_id = StorageID::createEmpty();
bool has_inner_table = false;
void checkStatementCanBeForwarded() const;
};
}