mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #31260 from azat/external-cleanup
Cleanup extern ProfileEvents/CurrentMetrics and add a style check
This commit is contained in:
commit
2313981fd7
@ -60,10 +60,7 @@
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric Revision;
|
||||
extern const Metric VersionInteger;
|
||||
extern const Metric MemoryTracking;
|
||||
extern const Metric MaxDDLEntryID;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
@ -7,6 +7,8 @@
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event HedgedRequestsChangeReplica;
|
||||
extern const Event DistributedConnectionFailTry;
|
||||
extern const Event DistributedConnectionFailAtAll;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
|
@ -15,7 +15,7 @@
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern Event DNSError;
|
||||
extern const Event DNSError;
|
||||
}
|
||||
|
||||
namespace std
|
||||
|
@ -30,6 +30,8 @@
|
||||
M(UncompressedCacheWeightLost, "") \
|
||||
M(MMappedFileCacheHits, "") \
|
||||
M(MMappedFileCacheMisses, "") \
|
||||
M(OpenedFileCacheHits, "") \
|
||||
M(OpenedFileCacheMisses, "") \
|
||||
M(AIOWrite, "Number of writes with Linux or FreeBSD AIO interface") \
|
||||
M(AIOWriteBytes, "Number of bytes written with Linux or FreeBSD AIO interface") \
|
||||
M(AIORead, "Number of reads with Linux or FreeBSD AIO interface") \
|
||||
@ -187,8 +189,6 @@
|
||||
M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
|
||||
M(SoftPageFaults, "") \
|
||||
M(HardPageFaults, "") \
|
||||
M(VoluntaryContextSwitches, "") \
|
||||
M(InvoluntaryContextSwitches, "") \
|
||||
\
|
||||
M(OSIOWaitMicroseconds, "Total time a thread spent waiting for a result of IO operation, from the OS point of view. This is real IO that doesn't include page cache.") \
|
||||
M(OSCPUWaitMicroseconds, "Total time a thread was ready for execution but waiting to be scheduled by OS, from the OS point of view.") \
|
||||
|
@ -24,6 +24,45 @@
|
||||
#include <base/errnoToString.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
#if defined(__linux__)
|
||||
extern const Event OSIOWaitMicroseconds;
|
||||
extern const Event OSCPUWaitMicroseconds;
|
||||
extern const Event OSCPUVirtualTimeMicroseconds;
|
||||
extern const Event OSReadChars;
|
||||
extern const Event OSWriteChars;
|
||||
extern const Event OSReadBytes;
|
||||
extern const Event OSWriteBytes;
|
||||
|
||||
extern const Event PerfCpuCycles;
|
||||
extern const Event PerfInstructions;
|
||||
extern const Event PerfCacheReferences;
|
||||
extern const Event PerfCacheMisses;
|
||||
extern const Event PerfBranchInstructions;
|
||||
extern const Event PerfBranchMisses;
|
||||
extern const Event PerfBusCycles;
|
||||
extern const Event PerfStalledCyclesFrontend;
|
||||
extern const Event PerfStalledCyclesBackend;
|
||||
extern const Event PerfRefCpuCycles;
|
||||
|
||||
extern const Event PerfCpuClock;
|
||||
extern const Event PerfTaskClock;
|
||||
extern const Event PerfContextSwitches;
|
||||
extern const Event PerfCpuMigrations;
|
||||
extern const Event PerfAlignmentFaults;
|
||||
extern const Event PerfEmulationFaults;
|
||||
extern const Event PerfMinEnabledTime;
|
||||
extern const Event PerfMinEnabledRunningTime;
|
||||
extern const Event PerfDataTLBReferences;
|
||||
extern const Event PerfDataTLBMisses;
|
||||
extern const Event PerfInstructionTLBReferences;
|
||||
extern const Event PerfInstructionTLBMisses;
|
||||
extern const Event PerfLocalMemoryReferences;
|
||||
extern const Event PerfLocalMemoryMisses;
|
||||
#endif
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -24,44 +24,6 @@ namespace ProfileEvents
|
||||
extern const Event SystemTimeMicroseconds;
|
||||
extern const Event SoftPageFaults;
|
||||
extern const Event HardPageFaults;
|
||||
extern const Event VoluntaryContextSwitches;
|
||||
extern const Event InvoluntaryContextSwitches;
|
||||
|
||||
#if defined(__linux__)
|
||||
extern const Event OSIOWaitMicroseconds;
|
||||
extern const Event OSCPUWaitMicroseconds;
|
||||
extern const Event OSCPUVirtualTimeMicroseconds;
|
||||
extern const Event OSReadChars;
|
||||
extern const Event OSWriteChars;
|
||||
extern const Event OSReadBytes;
|
||||
extern const Event OSWriteBytes;
|
||||
|
||||
extern const Event PerfCpuCycles;
|
||||
extern const Event PerfInstructions;
|
||||
extern const Event PerfCacheReferences;
|
||||
extern const Event PerfCacheMisses;
|
||||
extern const Event PerfBranchInstructions;
|
||||
extern const Event PerfBranchMisses;
|
||||
extern const Event PerfBusCycles;
|
||||
extern const Event PerfStalledCyclesFrontend;
|
||||
extern const Event PerfStalledCyclesBackend;
|
||||
extern const Event PerfRefCpuCycles;
|
||||
|
||||
extern const Event PerfCpuClock;
|
||||
extern const Event PerfTaskClock;
|
||||
extern const Event PerfContextSwitches;
|
||||
extern const Event PerfCpuMigrations;
|
||||
extern const Event PerfAlignmentFaults;
|
||||
extern const Event PerfEmulationFaults;
|
||||
extern const Event PerfMinEnabledTime;
|
||||
extern const Event PerfMinEnabledRunningTime;
|
||||
extern const Event PerfDataTLBReferences;
|
||||
extern const Event PerfDataTLBMisses;
|
||||
extern const Event PerfInstructionTLBReferences;
|
||||
extern const Event PerfInstructionTLBMisses;
|
||||
extern const Event PerfLocalMemoryReferences;
|
||||
extern const Event PerfLocalMemoryMisses;
|
||||
#endif
|
||||
}
|
||||
|
||||
namespace DB
|
||||
|
@ -23,6 +23,11 @@
|
||||
#include <Dictionaries/ICacheDictionaryStorage.h>
|
||||
#include <Dictionaries/DictionaryHelpers.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric Write;
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event FileOpen;
|
||||
|
@ -12,6 +12,11 @@
|
||||
#include <unistd.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -28,11 +28,6 @@ namespace Poco
|
||||
}
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -12,6 +12,11 @@
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -46,8 +46,14 @@ public:
|
||||
|
||||
auto [it, inserted] = files.emplace(key, OpenedFilePtr{});
|
||||
if (!inserted)
|
||||
{
|
||||
if (auto res = it->second.lock())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::OpenedFileCacheHits);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
ProfileEvents::increment(ProfileEvents::OpenedFileCacheMisses);
|
||||
|
||||
OpenedFilePtr res
|
||||
{
|
||||
|
@ -17,7 +17,6 @@ namespace ProfileEvents
|
||||
extern const Event ReadBufferFromFileDescriptorReadFailed;
|
||||
extern const Event ReadBufferFromFileDescriptorReadBytes;
|
||||
extern const Event DiskReadElapsedMicroseconds;
|
||||
extern const Event Seek;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
|
@ -33,11 +33,6 @@ namespace ProfileEvents
|
||||
extern const Event ExternalAggregationUncompressedBytes;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric QueryThread;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -94,7 +94,6 @@ namespace fs = std::filesystem;
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ContextLock;
|
||||
extern const Event CompiledCacheSizeBytes;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -105,14 +104,9 @@ namespace CurrentMetrics
|
||||
extern const Metric BackgroundBufferFlushSchedulePoolTask;
|
||||
extern const Metric BackgroundDistributedSchedulePoolTask;
|
||||
extern const Metric BackgroundMessageBrokerSchedulePoolTask;
|
||||
|
||||
|
||||
extern const Metric DelayedInserts;
|
||||
extern const Metric BackgroundMergesAndMutationsPoolTask;
|
||||
extern const Metric BackgroundMovePoolTask;
|
||||
extern const Metric BackgroundFetchesPoolTask;
|
||||
extern const Metric BackgroundCommonPoolTask;
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
|
@ -16,6 +16,11 @@
|
||||
#include <chrono>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric Query;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -26,10 +26,8 @@
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event SelectedRows;
|
||||
extern const Event SelectedBytes;
|
||||
extern const Event InsertedRows;
|
||||
extern const Event InsertedBytes;
|
||||
extern const Event InsertedRows;
|
||||
extern const Event InsertedBytes;
|
||||
}
|
||||
|
||||
|
||||
|
@ -13,15 +13,9 @@
|
||||
#include <Formats/NativeWriter.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ExternalSortWritePart;
|
||||
extern const Event ExternalSortMerge;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
|
@ -24,6 +24,7 @@ namespace fs = std::filesystem;
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric ReplicatedSend;
|
||||
extern const Metric ReplicatedFetch;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
|
@ -87,7 +87,6 @@ namespace ProfileEvents
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DelayedInserts;
|
||||
extern const Metric BackgroundMovePoolTask;
|
||||
}
|
||||
|
||||
|
||||
|
@ -41,18 +41,9 @@
|
||||
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MergedRows;
|
||||
extern const Event MergedUncompressedBytes;
|
||||
extern const Event MergesTimeMilliseconds;
|
||||
extern const Event Merge;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundMergesAndMutationsPoolTask;
|
||||
extern const Metric PartMutation;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
|
@ -42,14 +42,6 @@
|
||||
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event SelectedParts;
|
||||
extern const Event SelectedRanges;
|
||||
extern const Event SelectedMarks;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -10,11 +10,6 @@
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundPoolTask;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -81,14 +81,10 @@ namespace fs = std::filesystem;
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ReplicatedPartMerges;
|
||||
extern const Event ReplicatedPartMutations;
|
||||
extern const Event ReplicatedPartFailedFetches;
|
||||
extern const Event ReplicatedPartFetchesOfMerged;
|
||||
extern const Event ObsoleteReplicatedParts;
|
||||
extern const Event ReplicatedPartFetches;
|
||||
extern const Event DataAfterMergeDiffersFromReplica;
|
||||
extern const Event DataAfterMutationDiffersFromReplica;
|
||||
extern const Event CreatedLogEntryForMerge;
|
||||
extern const Event NotCreatedLogEntryForMerge;
|
||||
extern const Event CreatedLogEntryForMutation;
|
||||
|
@ -13,7 +13,21 @@
|
||||
# and then to run formatter only for the specified files.
|
||||
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New'
|
||||
EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing'
|
||||
|
||||
# From [1]:
|
||||
# But since array_to_string_internal() in array.c still loops over array
|
||||
# elements and concatenates them into a string, it's probably not more
|
||||
# efficient than the looping solutions proposed, but it's more readable.
|
||||
#
|
||||
# [1]: https://stackoverflow.com/a/15394738/328260
|
||||
function in_array()
|
||||
{
|
||||
local IFS="|"
|
||||
local value=$1 && shift
|
||||
|
||||
[[ "${IFS}${*}${IFS}" =~ "${IFS}${value}${IFS}" ]]
|
||||
}
|
||||
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
@ -40,24 +54,78 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/n
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done
|
||||
|
||||
# Unused ErrorCodes
|
||||
# NOTE: to fix automatically, replace echo with:
|
||||
# sed -i "/extern const int $code/d" $file
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
xargs grep -l -P 'extern const int [_A-Z]+' | while read file; do grep -P 'extern const int [_A-Z]+;' $file | sed -r -e 's/^.*?extern const int ([_A-Z]+);.*?$/\1/' | while read code; do grep -q "ErrorCodes::$code" $file || echo "ErrorCode $code is defined but not used in file $file"; done; done
|
||||
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
||||
declare -A EXTERN_TYPES
|
||||
EXTERN_TYPES[ErrorCodes]=int
|
||||
EXTERN_TYPES[ProfileEvents]=Event
|
||||
EXTERN_TYPES[CurrentMetrics]=Metric
|
||||
declare -A EXTERN_ALLOWED_CHARS
|
||||
EXTERN_ALLOWED_CHARS[ErrorCodes]='_A-Z'
|
||||
EXTERN_ALLOWED_CHARS[ProfileEvents]='_A-Za-z'
|
||||
EXTERN_ALLOWED_CHARS[CurrentMetrics]='_A-Za-z'
|
||||
EXTERN_TYPES_EXCLUDES=(
|
||||
ProfileEvents::global_counters
|
||||
ProfileEvents::Event
|
||||
ProfileEvents::Count
|
||||
ProfileEvents::Counters
|
||||
ProfileEvents::end
|
||||
ProfileEvents::increment
|
||||
ProfileEvents::getName
|
||||
ProfileEvents::Type
|
||||
ProfileEvents::TypeEnum
|
||||
ProfileEvents::dumpToMapColumn
|
||||
ProfileEvents::LOCAL_NAME
|
||||
|
||||
# Undefined ErrorCodes
|
||||
# NOTE: to fix automatically, replace echo with:
|
||||
# ( grep -q -F 'namespace ErrorCodes' $file && sed -i -r "0,/(\s*)extern const int [_A-Z]+/s//\1extern const int $code;\n&/" $file || awk '{ print; if (ns == 1) { ns = 2 }; if (ns == 2) { ns = 0; print "namespace ErrorCodes\n{\n extern const int '$code';\n}" } }; /namespace DB/ { ns = 1; };' < $file > ${file}.tmp && mv ${file}.tmp $file )
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
xargs grep -l -P 'ErrorCodes::[_A-Z]+' | while read file; do grep -P 'ErrorCodes::[_A-Z]+' $file | sed -r -e 's/^.*?ErrorCodes::([_A-Z]+).*?$/\1/' | while read code; do grep -q "extern const int $code" $file || echo "ErrorCode $code is used in file $file but not defined"; done; done
|
||||
CurrentMetrics::add
|
||||
CurrentMetrics::sub
|
||||
CurrentMetrics::set
|
||||
CurrentMetrics::end
|
||||
CurrentMetrics::Increment
|
||||
CurrentMetrics::Metric
|
||||
CurrentMetrics::values
|
||||
)
|
||||
for extern_type in ${!EXTERN_TYPES[@]}; do
|
||||
type_of_extern=${EXTERN_TYPES[$extern_type]}
|
||||
allowed_chars=${EXTERN_ALLOWED_CHARS[$extern_type]}
|
||||
|
||||
# Duplicate ErrorCodes
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
xargs grep -l -P 'ErrorCodes::[_A-Z]+' | while read file; do grep -P 'extern const int [_A-Z]+;' $file | sort | uniq -c | grep -v -P ' +1 ' && echo "Duplicate ErrorCode in file $file"; done
|
||||
# Unused
|
||||
# NOTE: to fix automatically, replace echo with:
|
||||
# sed -i "/extern const $type_of_extern $val/d" $file
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
|
||||
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "extern const $type_of_extern [$allowed_chars]+"
|
||||
} | while read file; do
|
||||
grep -P "extern const $type_of_extern [$allowed_chars]+;" $file | sed -r -e "s/^.*?extern const $type_of_extern ([$allowed_chars]+);.*?$/\1/" | while read val; do
|
||||
if ! grep -q "$extern_type::$val" $file; then
|
||||
# Excludes for SOFTWARE_EVENT/HARDWARE_EVENT/CACHE_EVENT in ThreadProfileEvents.cpp
|
||||
if [[ ! $extern_type::$val =~ ProfileEvents::Perf.* ]]; then
|
||||
echo "$extern_type::$val is defined but not used in file $file"
|
||||
fi
|
||||
fi
|
||||
done
|
||||
done
|
||||
|
||||
# Undefined
|
||||
# NOTE: to fix automatically, replace echo with:
|
||||
# ( grep -q -F 'namespace $extern_type' $file && sed -i -r "0,/(\s*)extern const $type_of_extern [$allowed_chars]+/s//\1extern const $type_of_extern $val;\n&/" $file || awk '{ print; if (ns == 1) { ns = 2 }; if (ns == 2) { ns = 0; print "namespace $extern_type\n{\n extern const $type_of_extern '$val';\n}" } }; /namespace DB/ { ns = 1; };' < $file > ${file}.tmp && mv ${file}.tmp $file )
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
|
||||
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "$extern_type::[$allowed_chars]+"
|
||||
} | while read file; do
|
||||
grep -P "$extern_type::[$allowed_chars]+" $file | sed -r -e "s/^.*?$extern_type::([$allowed_chars]+).*?$/\1/" | while read val; do
|
||||
if ! grep -q "extern const $type_of_extern $val" $file; then
|
||||
if ! in_array "$extern_type::$val" "${EXTERN_TYPES_EXCLUDES[@]}"; then
|
||||
echo "$extern_type::$val is used in file $file but not defined"
|
||||
fi
|
||||
fi
|
||||
done
|
||||
done
|
||||
|
||||
# Duplicates
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
|
||||
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "$extern_type::[$allowed_chars]+"
|
||||
} | while read file; do
|
||||
grep -P "extern const $type_of_extern [$allowed_chars]+;" $file | sort | uniq -c | grep -v -P ' +1 ' && echo "Duplicate $extern_type in file $file"
|
||||
done
|
||||
done
|
||||
|
||||
# Three or more consecutive empty lines
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
|
||||
|
Loading…
Reference in New Issue
Block a user