|
1 | 1 | #include <Common/ProfileEvents.h> |
2 | 2 | #include <Common/CurrentThread.h> |
3 | 3 | #include <Common/TraceSender.h> |
| 4 | +#include <Interpreters/Context.h> |
4 | 5 |
|
5 | 6 |
|
6 | 7 | // clang-format off |
|
11 | 12 | M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.", ValueType::Number) \ |
12 | 13 | M(SelectQuery, "Same as Query, but only for SELECT queries.", ValueType::Number) \ |
13 | 14 | M(InsertQuery, "Same as Query, but only for INSERT queries.", ValueType::Number) \ |
14 | | - M(InitialQuery, "Same as Query, but only counts initial queries (see is_initial_query).", ValueType::Number)\ |
| 15 | + M(InitialQuery, "Same as Query, but only counts initial queries (see is_initial_query).", ValueType::Number) \ |
15 | 16 | M(QueriesWithSubqueries, "Count queries with all subqueries", ValueType::Number) \ |
16 | 17 | M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries", ValueType::Number) \ |
17 | 18 | M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries", ValueType::Number) \ |
@@ -712,7 +713,7 @@ The server successfully detected this situation and will download merged part fr |
712 | 713 | M(KeeperLatency, "Keeper latency", ValueType::Milliseconds) \ |
713 | 714 | M(KeeperTotalElapsedMicroseconds, "Keeper total latency for a single request", ValueType::Microseconds) \ |
714 | 715 | M(KeeperProcessElapsedMicroseconds, "Keeper commit latency for a single request", ValueType::Microseconds) \ |
715 | | - M(KeeperPreprocessElapsedMicroseconds, "Keeper preprocessing latency for a single reuquest", ValueType::Microseconds)\ |
| 716 | + M(KeeperPreprocessElapsedMicroseconds, "Keeper preprocessing latency for a single reuquest", ValueType::Microseconds) \ |
716 | 717 | M(KeeperStorageLockWaitMicroseconds, "Time spent waiting for acquiring Keeper storage lock", ValueType::Microseconds) \ |
717 | 718 | M(KeeperCommitWaitElapsedMicroseconds, "Time spent waiting for certain log to be committed", ValueType::Microseconds) \ |
718 | 719 | M(KeeperBatchMaxCount, "Number of times the size of batch was limited by the amount", ValueType::Number) \ |
@@ -743,29 +744,29 @@ The server successfully detected this situation and will download merged part fr |
743 | 744 | M(S3QueueSetFileProcessingMicroseconds, "Time spent to set file as processing", ValueType::Microseconds) \ |
744 | 745 | M(S3QueueSetFileProcessedMicroseconds, "Time spent to set file as processed", ValueType::Microseconds) \ |
745 | 746 | M(S3QueueSetFileFailedMicroseconds, "Time spent to set file as failed", ValueType::Microseconds) \ |
746 | | - M(ObjectStorageQueueFailedFiles, "Number of files which failed to be processed", ValueType::Number)\ |
747 | | - M(ObjectStorageQueueProcessedFiles, "Number of files which were processed", ValueType::Number)\ |
| 747 | + M(ObjectStorageQueueFailedFiles, "Number of files which failed to be processed", ValueType::Number) \ |
| 748 | + M(ObjectStorageQueueProcessedFiles, "Number of files which were processed", ValueType::Number) \ |
748 | 749 | M(ObjectStorageQueueCleanupMaxSetSizeOrTTLMicroseconds, "Time spent to set file as failed", ValueType::Microseconds) \ |
749 | 750 | M(ObjectStorageQueuePullMicroseconds, "Time spent to read file data", ValueType::Microseconds) \ |
750 | 751 | M(ObjectStorageQueueLockLocalFileStatusesMicroseconds, "Time spent to lock local file statuses", ValueType::Microseconds) \ |
751 | | - M(ObjectStorageQueueFailedToBatchSetProcessing, "Number of times batched set processing request failed", ValueType::Number)\ |
752 | | - M(ObjectStorageQueueTrySetProcessingRequests, "The number of times we tried to make set processing request", ValueType::Number)\ |
753 | | - M(ObjectStorageQueueTrySetProcessingSucceeded, "The number of times we successfully set file as processing", ValueType::Number)\ |
754 | | - M(ObjectStorageQueueTrySetProcessingFailed, "The number of times we unsuccessfully set file as processing", ValueType::Number)\ |
755 | | - M(ObjectStorageQueueListedFiles, "Number of listed files in StorageS3(Azure)Queue", ValueType::Number)\ |
756 | | - M(ObjectStorageQueueFilteredFiles, "Number of filtered files in StorageS3(Azure)Queue", ValueType::Number)\ |
757 | | - M(ObjectStorageQueueReadFiles, "Number of read files (not equal to the number of actually inserted files)", ValueType::Number)\ |
758 | | - M(ObjectStorageQueueReadRows, "Number of read rows (not equal to the number of actually inserted rows)", ValueType::Number)\ |
759 | | - M(ObjectStorageQueueReadBytes, "Number of read bytes (not equal to the number of actually inserted bytes)", ValueType::Number)\ |
760 | | - M(ObjectStorageQueueExceptionsDuringRead, "Number of exceptions during read in S3(Azure)Queue", ValueType::Number)\ |
761 | | - M(ObjectStorageQueueExceptionsDuringInsert, "Number of exceptions during insert in S3(Azure)Queue", ValueType::Number)\ |
762 | | - M(ObjectStorageQueueRemovedObjects, "Number of objects removed as part of after_processing = delete", ValueType::Number)\ |
763 | | - M(ObjectStorageQueueInsertIterations, "Number of insert iterations", ValueType::Number)\ |
764 | | - M(ObjectStorageQueueCommitRequests, "Number of keeper requests to commit files as either failed or processed", ValueType::Number)\ |
765 | | - M(ObjectStorageQueueSuccessfulCommits, "Number of successful keeper commits", ValueType::Number)\ |
766 | | - M(ObjectStorageQueueUnsuccessfulCommits, "Number of unsuccessful keeper commits", ValueType::Number)\ |
767 | | - M(ObjectStorageQueueCancelledFiles, "Number cancelled files in StorageS3(Azure)Queue", ValueType::Number)\ |
768 | | - M(ObjectStorageQueueProcessedRows, "Number of processed rows in StorageS3(Azure)Queue", ValueType::Number)\ |
| 752 | + M(ObjectStorageQueueFailedToBatchSetProcessing, "Number of times batched set processing request failed", ValueType::Number) \ |
| 753 | + M(ObjectStorageQueueTrySetProcessingRequests, "The number of times we tried to make set processing request", ValueType::Number) \ |
| 754 | + M(ObjectStorageQueueTrySetProcessingSucceeded, "The number of times we successfully set file as processing", ValueType::Number) \ |
| 755 | + M(ObjectStorageQueueTrySetProcessingFailed, "The number of times we unsuccessfully set file as processing", ValueType::Number) \ |
| 756 | + M(ObjectStorageQueueListedFiles, "Number of listed files in StorageS3(Azure)Queue", ValueType::Number) \ |
| 757 | + M(ObjectStorageQueueFilteredFiles, "Number of filtered files in StorageS3(Azure)Queue", ValueType::Number) \ |
| 758 | + M(ObjectStorageQueueReadFiles, "Number of read files (not equal to the number of actually inserted files)", ValueType::Number) \ |
| 759 | + M(ObjectStorageQueueReadRows, "Number of read rows (not equal to the number of actually inserted rows)", ValueType::Number) \ |
| 760 | + M(ObjectStorageQueueReadBytes, "Number of read bytes (not equal to the number of actually inserted bytes)", ValueType::Number) \ |
| 761 | + M(ObjectStorageQueueExceptionsDuringRead, "Number of exceptions during read in S3(Azure)Queue", ValueType::Number) \ |
| 762 | + M(ObjectStorageQueueExceptionsDuringInsert, "Number of exceptions during insert in S3(Azure)Queue", ValueType::Number) \ |
| 763 | + M(ObjectStorageQueueRemovedObjects, "Number of objects removed as part of after_processing = delete", ValueType::Number) \ |
| 764 | + M(ObjectStorageQueueInsertIterations, "Number of insert iterations", ValueType::Number) \ |
| 765 | + M(ObjectStorageQueueCommitRequests, "Number of keeper requests to commit files as either failed or processed", ValueType::Number) \ |
| 766 | + M(ObjectStorageQueueSuccessfulCommits, "Number of successful keeper commits", ValueType::Number) \ |
| 767 | + M(ObjectStorageQueueUnsuccessfulCommits, "Number of unsuccessful keeper commits", ValueType::Number) \ |
| 768 | + M(ObjectStorageQueueCancelledFiles, "Number cancelled files in StorageS3(Azure)Queue", ValueType::Number) \ |
| 769 | + M(ObjectStorageQueueProcessedRows, "Number of processed rows in StorageS3(Azure)Queue", ValueType::Number) \ |
769 | 770 | \ |
770 | 771 | M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds", ValueType::Milliseconds) \ |
771 | 772 | M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted", ValueType::Number) \ |
@@ -938,7 +939,7 @@ The server successfully detected this situation and will download merged part fr |
938 | 939 | M(StorageConnectionsPreserved, "Number of preserved connections for storages", ValueType::Number) \ |
939 | 940 | M(StorageConnectionsExpired, "Number of expired connections for storages", ValueType::Number) \ |
940 | 941 | M(StorageConnectionsErrors, "Number of cases when creation of a connection for storage is failed", ValueType::Number) \ |
941 | | - M(StorageConnectionsElapsedMicroseconds, "Total time spend on creating connections for storages", ValueType::Microseconds) \ |
| 942 | + M(StorageConnectionsElapsedMicroseconds, "Total time spend on creating connections for storages", ValueType::Microseconds) \ |
942 | 943 | \ |
943 | 944 | M(DiskConnectionsCreated, "Number of created connections for disk", ValueType::Number) \ |
944 | 945 | M(DiskConnectionsReused, "Number of reused connections for disk", ValueType::Number) \ |
@@ -1122,6 +1123,30 @@ void incrementNoTrace(Event event, Count amount) |
1122 | 1123 | DB::CurrentThread::getProfileEvents().incrementNoTrace(event, amount); |
1123 | 1124 | } |
1124 | 1125 |
|
| 1126 | +double Counters::getCPUOverload(Int64 os_cpu_busy_time_threshold, bool reset) |
| 1127 | +{ |
| 1128 | + /// It's possible that we'll have slightly inconsistent values between wait time and busy time. But since we take the value of CPU wait time first, |
| 1129 | + /// it should not affect the situation a lot. In the worst case scenario we will have a slightly lower CPU overload value than it should be, but it's fine. |
| 1130 | + Int64 curr_cpu_wait_microseconds = counters[OSCPUWaitMicroseconds]; |
| 1131 | + Int64 curr_cpu_virtual_time_microseconds = counters[OSCPUVirtualTimeMicroseconds]; |
| 1132 | + |
| 1133 | + Int64 os_cpu_wait_microseconds = curr_cpu_wait_microseconds - prev_cpu_wait_microseconds.load(std::memory_order_acquire); |
| 1134 | + Int64 os_cpu_virtual_time_microseconds = curr_cpu_virtual_time_microseconds - prev_cpu_virtual_time_microseconds.load(std::memory_order_acquire); |
| 1135 | + |
| 1136 | + if (reset) |
| 1137 | + { |
| 1138 | + /// It's important to update wait time first, since the atomicity is not guaranteed for both counters at the same time. |
| 1139 | + /// So in the worst case scenario, we'll update prev wait time first, which will result in an underestimated wait time and lower CPU overload value. |
| 1140 | + prev_cpu_wait_microseconds.store(curr_cpu_wait_microseconds, std::memory_order_release); |
| 1141 | + prev_cpu_virtual_time_microseconds.store(curr_cpu_virtual_time_microseconds, std::memory_order_release); |
| 1142 | + } |
| 1143 | + |
| 1144 | + if (os_cpu_virtual_time_microseconds <= os_cpu_busy_time_threshold || os_cpu_wait_microseconds <= 0) |
| 1145 | + return 0; |
| 1146 | + |
| 1147 | + return static_cast<double>(os_cpu_wait_microseconds) / os_cpu_virtual_time_microseconds; |
| 1148 | +} |
| 1149 | + |
1125 | 1150 | void Counters::increment(Event event, Count amount) |
1126 | 1151 | { |
1127 | 1152 | Counters * current = this; |
|
0 commit comments