Revert "Add metrics for part number in MergeTree in ClickHouse"

This commit is contained in:
alexey-milovidov 2021-01-07 16:40:52 +03:00 committed by GitHub
parent 9b4160bbd0
commit a08db94343
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 2 additions and 133 deletions

View File

@ -5,9 +5,6 @@
#define APPLY_FOR_METRICS(M) \
M(Query, "Number of executing queries") \
M(Merge, "Number of executing background merges") \
M(Parts, "Total number of data parts") \
M(PartsActive, "Number of active data parts") \
M(PartsInactive, "Number of inactive data parts") \
M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \
M(ReplicatedFetch, "Number of data parts being fetched from replica") \
M(ReplicatedSend, "Number of data parts being sent to replicas") \

View File

@ -74,9 +74,6 @@ namespace CurrentMetrics
{
extern const Metric DelayedInserts;
extern const Metric BackgroundMovePoolTask;
extern const Metric Parts;
extern const Metric PartsActive;
extern const Metric PartsInactive;
}
@ -884,8 +881,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART);
addPartContributionToDataVolume(part);
CurrentMetrics::add(CurrentMetrics::Parts);
CurrentMetrics::add(CurrentMetrics::PartsActive);
});
}
@ -902,10 +897,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (!data_parts_indexes.insert(part).second)
throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART);
addPartContributionToDataVolume(part);
CurrentMetrics::add(CurrentMetrics::Parts);
CurrentMetrics::add(CurrentMetrics::PartsActive);
addPartContributionToDataVolume(part);
}
if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts)
@ -938,9 +931,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
(*it)->remove_time.store((*it)->modification_time, std::memory_order_relaxed);
modifyPartState(it, DataPartState::Outdated);
removePartContributionToDataVolume(*it);
CurrentMetrics::sub(CurrentMetrics::PartsActive);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
};
(*prev_jt)->assertState({DataPartState::Committed});
@ -1108,9 +1098,6 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa
(*it)->assertState({DataPartState::Deleting});
data_parts_indexes.erase(it);
CurrentMetrics::sub(CurrentMetrics::Parts);
CurrentMetrics::sub(CurrentMetrics::PartsInactive);
}
}
@ -2011,9 +1998,6 @@ bool MergeTreeData::renameTempPartAndReplace(
auto part_it = data_parts_indexes.insert(part).first;
CurrentMetrics::add(CurrentMetrics::Parts);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
if (out_transaction)
{
out_transaction->precommitted_parts.insert(part);
@ -2029,13 +2013,9 @@ bool MergeTreeData::renameTempPartAndReplace(
covered_part->remove_time.store(current_time, std::memory_order_relaxed);
modifyPartState(covered_part, DataPartState::Outdated);
removePartContributionToColumnSizes(covered_part);
reduce_bytes += covered_part->getBytesOnDisk();
reduce_rows += covered_part->rows_count;
++reduce_parts;
CurrentMetrics::sub(CurrentMetrics::PartsActive);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
}
decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);
@ -2043,9 +2023,6 @@ bool MergeTreeData::renameTempPartAndReplace(
modifyPartState(part_it, DataPartState::Committed);
addPartContributionToColumnSizes(part);
addPartContributionToDataVolume(part);
CurrentMetrics::add(CurrentMetrics::PartsActive);
CurrentMetrics::sub(CurrentMetrics::PartsInactive);
}
auto part_in_memory = asInMemoryPart(part);
@ -2089,9 +2066,6 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect
{
removePartContributionToColumnSizes(part);
removePartContributionToDataVolume(part);
CurrentMetrics::sub(CurrentMetrics::PartsActive);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
}
if (part->state == IMergeTreeDataPart::State::Committed || clear_without_timeout)
@ -2118,9 +2092,6 @@ void MergeTreeData::removePartsFromWorkingSetImmediatelyAndSetTemporaryState(con
modifyPartState(part, IMergeTreeDataPart::State::Temporary);
/// Erase immediately
data_parts_indexes.erase(it_part);
CurrentMetrics::sub(CurrentMetrics::Parts);
CurrentMetrics::sub(CurrentMetrics::PartsInactive);
}
}
@ -2211,17 +2182,12 @@ restore_covered)
{
removePartContributionToDataVolume(part);
removePartContributionToColumnSizes(part);
CurrentMetrics::sub(CurrentMetrics::PartsActive);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
}
modifyPartState(it_part, DataPartState::Deleting);
part->renameToDetached(prefix);
data_parts_indexes.erase(it_part);
CurrentMetrics::sub(CurrentMetrics::Parts);
CurrentMetrics::sub(CurrentMetrics::PartsInactive);
data_parts_indexes.erase(it_part);
if (restore_covered && part->info.level == 0)
{
@ -3431,9 +3397,6 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
data.modifyPartState(covered_part, DataPartState::Outdated);
data.removePartContributionToColumnSizes(covered_part);
CurrentMetrics::sub(CurrentMetrics::PartsActive);
CurrentMetrics::add(CurrentMetrics::PartsInactive);
}
reduce_parts += covered_parts.size();
@ -3443,9 +3406,6 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
data.modifyPartState(part, DataPartState::Committed);
data.addPartContributionToColumnSizes(part);
CurrentMetrics::add(CurrentMetrics::PartsActive);
CurrentMetrics::sub(CurrentMetrics::PartsInactive);
}
}
data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts);

View File

@ -31,9 +31,6 @@
namespace CurrentMetrics
{
extern const Metric BackgroundPoolTask;
extern const Metric Parts;
extern const Metric PartsActive;
extern const Metric PartsInactive;
}
namespace DB
@ -162,22 +159,6 @@ void StorageMergeTree::shutdown()
/// may have race condition between our remove call and background
/// process.
clearOldPartsFromFilesystem(true);
auto lock = lockParts();
DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end());
size_t committed_parts_count = 0;
for (const auto & parts_info : all_parts)
{
if (parts_info->state == DataPartState::Committed)
{
++committed_parts_count;
}
}
CurrentMetrics::sub(CurrentMetrics::Parts, all_parts.size());
CurrentMetrics::sub(CurrentMetrics::PartsActive, committed_parts_count);
CurrentMetrics::sub(CurrentMetrics::PartsInactive, all_parts.size() - committed_parts_count);
}
catch (...)
{

View File

@ -81,9 +81,6 @@ namespace ProfileEvents
namespace CurrentMetrics
{
extern const Metric BackgroundFetchesPoolTask;
extern const Metric Parts;
extern const Metric PartsActive;
extern const Metric PartsInactive;
}
namespace DB
@ -3686,22 +3683,6 @@ void StorageReplicatedMergeTree::shutdown()
/// which will remove themselves in their descrutors. If so, we may have
/// race condition between our remove call and background process.
clearOldPartsFromFilesystem(true);
auto lock = lockParts();
DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end());
size_t committed_parts_count = 0;
for (const auto & parts_info : all_parts)
{
if (parts_info->state == DataPartState::Committed)
{
committed_parts_count++;
}
}
CurrentMetrics::sub(CurrentMetrics::Parts, all_parts.size());
CurrentMetrics::sub(CurrentMetrics::PartsActive, committed_parts_count);
CurrentMetrics::sub(CurrentMetrics::PartsInactive, all_parts.size() - committed_parts_count);
}

View File

@ -1,46 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
verify_sql="SELECT COUNT(1)
FROM
(SELECT
SUM(IF(metric = 'Parts', value, 0)) AS Parts,
SUM(IF(metric = 'PartsActive', value, 0)) AS PartsActive,
SUM(IF(metric = 'PartsInactive', value, 0)) AS PartsInactive
FROM system.metrics) as a INNER JOIN
(SELECT
toInt64(SUM(1)) AS Parts,
toInt64(SUM(IF(active = 1, 1, 0))) AS PartsActive,
toInt64(SUM(IF(active = 0, 1, 0))) AS PartsInactive
FROM system.parts
) as b USING (Parts,PartsActive,PartsInactive)"
verify(){
for _ in $(seq 1 10)
do
result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" )
if [ "$result" = "1" ];then
echo 1
return
fi
done
echo 0
}
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_table" 2>/dev/null
$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" 2>/dev/null
$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" 2>/dev/null
verify
$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" 2>/dev/null
verify
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" 2>/dev/null
verify
$CLICKHOUSE_CLIENT --query="DROP TABLE test_table" 2>/dev/null
verify