ClickHouse/src/Parsers/ParserOptimizeQuery.cpp
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

108 lines
3.1 KiB
C++

#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserPartition.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ExpressionListParsers.h>
namespace DB
{
bool ParserOptimizeQueryColumnsSpecification::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
// Do not allow APPLY and REPLACE transformers.
// Since we use Columns Transformers only to get list of columns,
// we can't actually modify content of the columns for deduplication.
const auto allowed_transformers = ParserColumnsTransformers::ColumnTransformers{ParserColumnsTransformers::ColumnTransformer::EXCEPT};
return ParserColumnsMatcher(allowed_transformers).parse(pos, node, expected)
|| ParserAsterisk(allowed_transformers).parse(pos, node, expected)
|| ParserIdentifier(false).parse(pos, node, expected);
}
bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_optimize_table("OPTIMIZE TABLE");
ParserKeyword s_partition("PARTITION");
ParserKeyword s_final("FINAL");
ParserKeyword s_deduplicate("DEDUPLICATE");
ParserKeyword s_cleanup("CLEANUP");
ParserKeyword s_by("BY");
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p(true);
ParserPartition partition_p;
ASTPtr database;
ASTPtr table;
ASTPtr partition;
bool final = false;
bool deduplicate = false;
bool cleanup = false;
String cluster_str;
if (!s_optimize_table.ignore(pos, expected))
return false;
if (!name_p.parse(pos, table, expected))
return false;
if (s_dot.ignore(pos, expected))
{
database = table;
if (!name_p.parse(pos, table, expected))
return false;
}
if (ParserKeyword{"ON"}.ignore(pos, expected) && !ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
if (s_partition.ignore(pos, expected))
{
if (!partition_p.parse(pos, partition, expected))
return false;
}
if (s_final.ignore(pos, expected))
final = true;
if (s_deduplicate.ignore(pos, expected))
deduplicate = true;
if (s_cleanup.ignore(pos, expected))
cleanup = true;
ASTPtr deduplicate_by_columns;
if (deduplicate && s_by.ignore(pos, expected))
{
if (!ParserList(std::make_unique<ParserOptimizeQueryColumnsSpecification>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, deduplicate_by_columns, expected))
return false;
}
auto query = std::make_shared<ASTOptimizeQuery>();
node = query;
query->cluster = cluster_str;
if ((query->partition = partition))
query->children.push_back(partition);
query->final = final;
query->deduplicate = deduplicate;
query->deduplicate_by_columns = deduplicate_by_columns;
query->cleanup = cleanup;
query->database = database;
query->table = table;
if (database)
query->children.push_back(database);
if (table)
query->children.push_back(table);
return true;
}
}