mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Fix sticking mutations
This commit is contained in:
parent
c66dc23a47
commit
e0bf5913e7
@ -1309,7 +1309,7 @@ void ExpressionActionsChain::finalize()
|
||||
}
|
||||
}
|
||||
|
||||
std::string ExpressionActionsChain::dumpChain()
|
||||
std::string ExpressionActionsChain::dumpChain() const
|
||||
{
|
||||
std::stringstream ss;
|
||||
|
||||
|
@ -347,7 +347,7 @@ struct ExpressionActionsChain
|
||||
return steps.back();
|
||||
}
|
||||
|
||||
std::string dumpChain();
|
||||
std::string dumpChain() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -764,4 +764,23 @@ std::optional<SortDescription> MutationsInterpreter::getStorageSortDescriptionIf
|
||||
return sort_description;
|
||||
}
|
||||
|
||||
bool MutationsInterpreter::Stage::isAffectingAllColumns(const Names & storage_columns) const
|
||||
{
|
||||
/// is subset
|
||||
for (const auto & storage_column : storage_columns)
|
||||
if (!output_columns.contains(storage_column))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MutationsInterpreter::isAffectingAllColumns() const
|
||||
{
|
||||
auto storage_columns = metadata_snapshot->getColumns().getNamesOfPhysical();
|
||||
for (const auto & stage : stages)
|
||||
if (stage.isAffectingAllColumns(storage_columns))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -42,6 +42,8 @@ public:
|
||||
/// Only changed columns.
|
||||
const Block & getUpdatedHeader() const;
|
||||
|
||||
bool isAffectingAllColumns() const;
|
||||
|
||||
private:
|
||||
ASTPtr prepare(bool dry_run);
|
||||
|
||||
@ -86,8 +88,8 @@ private:
|
||||
ASTs filters;
|
||||
std::unordered_map<String, ASTPtr> column_to_updated;
|
||||
|
||||
/// Contains columns that are changed by this stage,
|
||||
/// columns changed by the previous stages and also columns needed by the next stages.
|
||||
/// Contains columns that are changed by this stage, columns changed by
|
||||
/// the previous stages and also columns needed by the next stages.
|
||||
NameSet output_columns;
|
||||
|
||||
std::unique_ptr<ExpressionAnalyzer> analyzer;
|
||||
@ -97,6 +99,8 @@ private:
|
||||
/// then there is (possibly) an UPDATE step, and finally a projection step.
|
||||
ExpressionActionsChain expressions_chain;
|
||||
Names filter_column_names;
|
||||
|
||||
bool isAffectingAllColumns(const Names & storage_columns) const;
|
||||
};
|
||||
|
||||
std::unique_ptr<Block> updated_header;
|
||||
|
@ -1092,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
need_remove_expired_values = true;
|
||||
|
||||
/// All columns from part are changed and may be some more that were missing before in part
|
||||
if (!isWidePart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
|
||||
if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns()))
|
||||
{
|
||||
auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames);
|
||||
mutateAllPartColumns(
|
||||
|
@ -1 +1,14 @@
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value1` UInt64,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value1` UInt64,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value1` String,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `renamed_value1` String,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nSETTINGS index_granularity = 8192
|
||||
1
|
||||
CREATE TABLE default.sticking_mutations\n(\n `date` Date,\n `key` UInt64,\n `value1` UInt64,\n `value2` UInt8\n)\nENGINE = MergeTree()\nORDER BY key\nTTL date + toIntervalDay(1)\nSETTINGS index_granularity = 8192
|
||||
|
@ -1,11 +1,16 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations"
|
||||
|
||||
function check_sticky_mutations()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations (
|
||||
date Date,
|
||||
key UInt64,
|
||||
value1 String,
|
||||
value2 UInt8
|
||||
@ -13,16 +18,17 @@ $CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations (
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY key;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO sticking_mutations SELECT number, toString(number), number % 128 FROM numbers(1000)"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO sticking_mutations SELECT toDate('2020-07-10'), number, toString(number), number % 128 FROM numbers(1000)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO sticking_mutations SELECT toDate('2100-01-10'), number, toString(number), number % 128 FROM numbers(1000)"
|
||||
|
||||
# if merges stopped for normal merge tree mutations will stick
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES sticking_mutations"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, MODIFY COLUMN value1 UInt64;" &
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "$1" &
|
||||
|
||||
##### wait mutation to start #####
|
||||
check_query="SELECT count() FROM system.mutations WHERE table='sticking_mutations' and database='$CLICKHOUSE_DATABASE'"
|
||||
check_query="SELECT count() FROM system.mutations WHERE table='sticking_mutations' and database='$CLICKHOUSE_DATABASE' and is_done = 0"
|
||||
|
||||
query_result=`$CLICKHOUSE_CLIENT --query="$check_query" 2>&1`
|
||||
|
||||
@ -38,10 +44,27 @@ done
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM START MERGES sticking_mutations"
|
||||
|
||||
# just to be sure, that previous mutations finished
|
||||
$CLICKHOUSE_CLIENT --query "ALTER TABLE sticking_mutations DELETE WHERE value % 31 == 0 SETTINGS mutations_sync = 1"
|
||||
$CLICKHOUSE_CLIENT --query "ALTER TABLE sticking_mutations DELETE WHERE value2 % 31 == 0 SETTINGS mutations_sync = 1"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE sticking_mutations FINAL"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT 1"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT sum(cityHash64(*)) > 1 FROM sticking_mutations WHERE key > 10"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE sticking_mutations"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations"
|
||||
}
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, MODIFY COLUMN value1 UInt64"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations MODIFY COLUMN value1 UInt64, DELETE WHERE value2 % 32 == 0"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations UPDATE value1 = 15 WHERE key < 2000, DELETE WHERE value2 % 32 == 0"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, UPDATE value1 = 15 WHERE key < 2000"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, DROP COLUMN value1"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, RENAME COLUMN value1 TO renamed_value1"
|
||||
|
||||
check_sticky_mutations "ALTER TABLE sticking_mutations MODIFY COLUMN value1 UInt64, MODIFY TTL date + INTERVAL 1 DAY"
|
||||
|
Loading…
Reference in New Issue
Block a user