Merge pull request #70908 from ClickHouse/merge-source-parts

Add a profile event about the number of merged source parts
This commit is contained in:
Ilya Yatsishin 2024-10-24 17:39:31 +00:00 committed by GitHub
commit 1a1bd26d76
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 24 additions and 8 deletions

View File

@ -231,6 +231,7 @@
M(LoadedMarksMemoryBytes, "Size of in-memory representations of loaded marks.", ValueType::Bytes) \
\
M(Merge, "Number of launched background merges.", ValueType::Number) \
M(MergeSourceParts, "Number of source parts scheduled for merges.", ValueType::Number) \
M(MergedRows, "Rows read for background merges. This is the number of rows before merge.", ValueType::Number) \
M(MergedColumns, "Number of columns merged during the horizontal stage of merges.", ValueType::Number) \
M(GatheredColumns, "Number of columns gathered during the vertical stage of merges.", ValueType::Number) \

View File

@ -6,11 +6,8 @@
#include <fmt/format.h>
#include <Common/logger_useful.h>
#include <Common/ActionBlocker.h>
#include <Core/Settings.h>
#include <Common/ProfileEvents.h>
#include <Processors/Transforms/CheckSortedTransform.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Compression/CompressedWriteBuffer.h>
#include <DataTypes/ObjectUtils.h>
#include <DataTypes/Serializations/SerializationInfo.h>
@ -20,10 +17,8 @@
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergeProjectionPartsTask.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Merges/MergingSortedTransform.h>
@ -34,9 +29,6 @@
#include <Processors/Merges/AggregatingSortedTransform.h>
#include <Processors/Merges/VersionedCollapsingTransform.h>
#include <Processors/Transforms/TTLTransform.h>
#include <Processors/Transforms/TTLCalcTransform.h>
#include <Processors/Transforms/DistinctSortedTransform.h>
#include <Processors/Transforms/DistinctTransform.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Processors/QueryPlan/DistinctStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
@ -48,9 +40,22 @@
#include <Interpreters/MergeTreeTransaction.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#ifndef NDEBUG
#include <Processors/Transforms/CheckSortedTransform.h>
#endif
#ifdef CLICKHOUSE_CLOUD
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Storages/MergeTree/DataPartStorageOnDiskPacked.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#endif
namespace ProfileEvents
{
extern const Event Merge;
extern const Event MergeSourceParts;
extern const Event MergedColumns;
extern const Event GatheredColumns;
extern const Event MergeTotalMilliseconds;
@ -302,6 +307,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu
bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
{
ProfileEvents::increment(ProfileEvents::Merge);
ProfileEvents::increment(ProfileEvents::MergeSourceParts, global_ctx->future_part->parts.size());
String local_tmp_prefix;
if (global_ctx->need_prefix)

View File

@ -0,0 +1 @@
1 2 2 1

View File

@ -0,0 +1,8 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8) ORDER BY x;
INSERT INTO test VALUES (1);
INSERT INTO test VALUES (2);
OPTIMIZE TABLE test FINAL;
SYSTEM FLUSH LOGS;
SELECT ProfileEvents['Merge'], ProfileEvents['MergeSourceParts'], ProfileEvents['MergedRows'], ProfileEvents['MergedColumns'] FROM system.part_log WHERE database = currentDatabase() AND table = 'test' AND event_type = 'MergeParts';
DROP TABLE test;