mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
6526c2a8ab
* 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>
108 lines
3.1 KiB
C++
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;
|
|
}
|
|
|
|
|
|
}
|