Fix sticking mutations

This commit is contained in:
alesapin 2020-07-26 17:21:57 +03:00
parent c66dc23a47
commit e0bf5913e7
7 changed files with 91 additions and 32 deletions

View File

@ -1309,7 +1309,7 @@ void ExpressionActionsChain::finalize()
}
}
std::string ExpressionActionsChain::dumpChain()
std::string ExpressionActionsChain::dumpChain() const
{
std::stringstream ss;

View File

@ -347,7 +347,7 @@ struct ExpressionActionsChain
return steps.back();
}
std::string dumpChain();
std::string dumpChain() const;
};
}

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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(

View File

@ -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

View File

@ -1,47 +1,70 @@
#!/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"
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations (
key UInt64,
value1 String,
value2 UInt8
)
ENGINE = MergeTree()
ORDER BY key;"
function check_sticky_mutations()
{
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE sticking_mutations (
date Date,
key UInt64,
value1 String,
value2 UInt8
)
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)"
# if merges stopped for normal merge tree mutations will stick
$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES sticking_mutations"
$CLICKHOUSE_CLIENT --query "INSERT INTO sticking_mutations SELECT toDate('2100-01-10'), number, toString(number), number % 128 FROM numbers(1000)"
$CLICKHOUSE_CLIENT --query "ALTER TABLE sticking_mutations DELETE WHERE value2 % 32 == 0, MODIFY COLUMN value1 UInt64;" &
# if merges stopped for normal merge tree mutations will stick
$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES sticking_mutations"
$CLICKHOUSE_CLIENT --query "$1" &
##### wait mutation to start #####
check_query="SELECT count() FROM system.mutations WHERE table='sticking_mutations' and database='$CLICKHOUSE_DATABASE'"
##### wait mutation to start #####
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`
while [ "$query_result" == "0" ]
do
query_result=`$CLICKHOUSE_CLIENT --query="$check_query" 2>&1`
sleep 0.5
done
##### wait mutation to start #####
# Starting merges to execute sticked mutations
while [ "$query_result" == "0" ]
do
query_result=`$CLICKHOUSE_CLIENT --query="$check_query" 2>&1`
sleep 0.5
done
##### wait mutation to start #####
$CLICKHOUSE_CLIENT --query "SYSTEM START MERGES sticking_mutations"
# Starting merges to execute sticked 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 "SYSTEM START MERGES sticking_mutations"
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE sticking_mutations FINAL"
# just to be sure, that previous mutations finished
$CLICKHOUSE_CLIENT --query "ALTER TABLE sticking_mutations DELETE WHERE value2 % 31 == 0 SETTINGS mutations_sync = 1"
$CLICKHOUSE_CLIENT --query "SELECT 1"
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE sticking_mutations FINAL"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS sticking_mutations"
$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"