Write the documentation for all ProfileEvents. Enforce that documentation exists.

This commit is contained in:
Alexey Milovidov 2022-08-26 01:34:51 +02:00
parent c9dea66f8d
commit 91bd6f1d46
3 changed files with 100 additions and 85 deletions

View File

@ -31,30 +31,30 @@
M(ReadCompressedBytes, "Number of bytes (the number of bytes before decompression) read from compressed sources (files, network).") \
M(CompressedReadBufferBlocks, "Number of compressed blocks (the blocks of data that are compressed independent of each other) read from compressed sources (files, network).") \
M(CompressedReadBufferBytes, "Number of uncompressed bytes (the number of bytes after decompression) read from compressed sources (files, network).") \
M(UncompressedCacheHits, "") \
M(UncompressedCacheMisses, "") \
M(UncompressedCacheWeightLost, "") \
M(MMappedFileCacheHits, "") \
M(MMappedFileCacheMisses, "") \
M(OpenedFileCacheHits, "") \
M(OpenedFileCacheMisses, "") \
M(UncompressedCacheHits, "Number of times, a block of data has been found in the uncompressed cache (and decompression was avoided).") \
M(UncompressedCacheMisses, "Number of times, a block of data has not been found in the uncompressed cache (and required decompression).") \
M(UncompressedCacheWeightLost, "Number of bytes evicted from the uncompressed cache.") \
M(MMappedFileCacheHits, "Number of times a file has been found in the MMap cache (for the 'mmap' read_method), so we didn't have to mmap it again.") \
M(MMappedFileCacheMisses, "Number of times a file has not been found in the MMap cache (for the 'mmap' read_method), so we had to mmap it again.") \
M(OpenedFileCacheHits, "Number of times a file has been found in the opened file cache, so we didn't have to open it again.") \
M(OpenedFileCacheMisses, "Number of times a file has been found in the opened file cache, so we had to open it again.") \
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") \
M(AIOReadBytes, "Number of bytes read with Linux or FreeBSD AIO interface") \
M(IOBufferAllocs, "") \
M(IOBufferAllocBytes, "") \
M(ArenaAllocChunks, "") \
M(ArenaAllocBytes, "") \
M(FunctionExecute, "") \
M(TableFunctionExecute, "") \
M(MarkCacheHits, "") \
M(MarkCacheMisses, "") \
M(CreatedReadBufferOrdinary, "") \
M(CreatedReadBufferDirectIO, "") \
M(CreatedReadBufferDirectIOFailed, "") \
M(CreatedReadBufferMMap, "") \
M(CreatedReadBufferMMapFailed, "") \
M(IOBufferAllocs, "Number of allocations of IO buffers (for ReadBuffer/WriteBuffer).") \
M(IOBufferAllocBytes, "Number of bytes allocated for IO buffers (for ReadBuffer/WriteBuffer).") \
M(ArenaAllocChunks, "Number of chunks allocated for memory Arena (used for GROUP BY and similar operations)") \
M(ArenaAllocBytes, "Number of bytes allocated for memory Arena (used for GROUP BY and similar operations)") \
M(FunctionExecute, "Number of SQL ordinary function calls (SQL functions are called on per-block basis, so this number represents the number of blocks).") \
M(TableFunctionExecute, "Number of table function calls.") \
M(MarkCacheHits, "Number of times, an entry has been found in the mark cache, so we didn't have to load a mark file.") \
M(MarkCacheMisses, "Number of times, an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.") \
M(CreatedReadBufferOrdinary, "Number of times, ordinary read buffer was created for reading data (while choosing among other read methods).") \
M(CreatedReadBufferDirectIO, "Number of times, a read buffer with O_DIRECT was created for reading data (while choosing among other read methods).") \
M(CreatedReadBufferDirectIOFailed, "Number of times, a read buffer with O_DIRECT was attempted to create for reading data (while choosing among other read methods), but the OS did not allow it (due to lack of filesystem support or other reasons) and we fallen back to the ordinary reading method.") \
M(CreatedReadBufferMMap, "Number of times, a read buffer using 'mmap' was created for reading data (while choosing among other read methods).") \
M(CreatedReadBufferMMapFailed, "Number of times, a read buffer with 'mmap' was attempted to create for reading data (while choosing among other read methods), but the OS did not allow it (due to lack of filesystem support or other reasons) and we fallen back to the ordinary reading method.") \
M(DiskReadElapsedMicroseconds, "Total time spent waiting for read syscall. This include reads from page cache.") \
M(DiskWriteElapsedMicroseconds, "Total time spent waiting for write syscall. This include writes to page cache.") \
M(NetworkReceiveElapsedMicroseconds, "Total time spent waiting for data to receive or receiving data from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
@ -67,12 +67,12 @@
\
M(ReplicatedPartFetches, "Number of times a data part was downloaded from replica of a ReplicatedMergeTree table.") \
M(ReplicatedPartFailedFetches, "Number of times a data part was failed to download from replica of a ReplicatedMergeTree table.") \
M(ObsoleteReplicatedParts, "") \
M(ObsoleteReplicatedParts, "Number of times a data part was covered by another data part that has been fetched from a replica (so, we have marked a covered data part as obsolete and no longer needed).") \
M(ReplicatedPartMerges, "Number of times data parts of ReplicatedMergeTree tables were successfully merged.") \
M(ReplicatedPartFetchesOfMerged, "Number of times we prefer to download already merged part from replica of ReplicatedMergeTree table instead of performing a merge ourself (usually we prefer doing a merge ourself to save network traffic). This happens when we have not all source parts to perform a merge or when the data part is old enough.") \
M(ReplicatedPartMutations, "") \
M(ReplicatedPartChecks, "") \
M(ReplicatedPartChecksFailed, "") \
M(ReplicatedPartMutations, "Number of times data parts of ReplicatedMergeTree tables were successfully mutated.") \
M(ReplicatedPartChecks, "Number of times we had to perform advanced search for a data part on replicas or to clarify the need of an existing data part.") \
M(ReplicatedPartChecksFailed, "Number of times the advanced search for a data part on replicas did not give result or when unexpected part has been found and moved away.") \
M(ReplicatedDataLoss, "Number of times a data part that we wanted doesn't exist on any replica (even on replicas that are offline right now). That data parts are definitely lost. This is normal due to asynchronous replication (if quorum inserts were not enabled), when the replica on which the data part was written was failed and when it became online after fail it doesn't contain that data part.") \
\
M(InsertedRows, "Number of rows INSERTed to all tables.") \
@ -85,30 +85,30 @@
M(DistributedDelayedInsertsMilliseconds, "Total number of milliseconds spent while the INSERT of a block to a Distributed table was throttled due to high number of pending bytes.") \
M(DuplicatedInsertedBlocks, "Number of times the INSERTed block to a ReplicatedMergeTree table was deduplicated.") \
\
M(ZooKeeperInit, "") \
M(ZooKeeperTransactions, "") \
M(ZooKeeperList, "") \
M(ZooKeeperCreate, "") \
M(ZooKeeperRemove, "") \
M(ZooKeeperExists, "") \
M(ZooKeeperGet, "") \
M(ZooKeeperSet, "") \
M(ZooKeeperMulti, "") \
M(ZooKeeperCheck, "") \
M(ZooKeeperSync, "") \
M(ZooKeeperClose, "") \
M(ZooKeeperWatchResponse, "") \
M(ZooKeeperUserExceptions, "") \
M(ZooKeeperHardwareExceptions, "") \
M(ZooKeeperOtherExceptions, "") \
M(ZooKeeperWaitMicroseconds, "") \
M(ZooKeeperBytesSent, "") \
M(ZooKeeperBytesReceived, "") \
M(ZooKeeperInit, "Number of times connection with ZooKeeper has been established.") \
M(ZooKeeperTransactions, "Number of ZooKeeper operations, which include both read and write operations as well as multi-transactions.") \
M(ZooKeeperList, "Number of 'list' (getChildren) requests to ZooKeeper.") \
M(ZooKeeperCreate, "Number of 'create' requests to ZooKeeper.") \
M(ZooKeeperRemove, "Number of 'remove' requests to ZooKeeper.") \
M(ZooKeeperExists, "Number of 'exists' requests to ZooKeeper.") \
M(ZooKeeperGet, "Number of 'get' requests to ZooKeeper.") \
M(ZooKeeperSet, "Number of 'set' requests to ZooKeeper.") \
M(ZooKeeperMulti, "Number of 'multi' requests to ZooKeeper (compound transactions).") \
M(ZooKeeperCheck, "Number of 'check' requests to ZooKeeper. Usually they don't make sense in isolation, only as part of a complex transaction.") \
M(ZooKeeperSync, "Number of 'sync' requests to ZooKeeper. These requests are rarely needed or usable.") \
M(ZooKeeperClose, "Number of times connection with ZooKeeper has been closed voluntary.") \
M(ZooKeeperWatchResponse, "Number of times watch notification has been received from ZooKeeper.") \
M(ZooKeeperUserExceptions, "Number of exceptions while working with ZooKeeper related to the data (no node, bad version or similar).") \
M(ZooKeeperHardwareExceptions, "Number of exceptions while working with ZooKeeper related to network (connection loss or similar).") \
M(ZooKeeperOtherExceptions, "Number of exceptions while working with ZooKeeper other than ZooKeeperUserExceptions and ZooKeeperHardwareExceptions.") \
M(ZooKeeperWaitMicroseconds, "Number of microseconds spent waiting for responses from ZooKeeper after creating a request, summed across all the requesting threads.") \
M(ZooKeeperBytesSent, "Number of bytes send over network while communicating with ZooKeeper.") \
M(ZooKeeperBytesReceived, "Number of bytes received over network while communicating with ZooKeeper.") \
\
M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry") \
M(DistributedConnectionMissingTable, "") \
M(DistributedConnectionStaleReplica, "") \
M(DistributedConnectionFailAtAll, "Total count when distributed connection fails after all retries finished") \
M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \
M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \
M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \
M(DistributedConnectionFailAtAll, "Total count when distributed connection fails after all retries finished.") \
\
M(HedgedRequestsChangeReplica, "Total count when timeout for changing replica expired in hedged requests.") \
\
@ -119,12 +119,12 @@
\
M(ExecuteShellCommand, "Number of shell command executions.") \
\
M(ExternalSortWritePart, "") \
M(ExternalSortMerge, "") \
M(ExternalAggregationWritePart, "") \
M(ExternalAggregationMerge, "") \
M(ExternalAggregationCompressedBytes, "") \
M(ExternalAggregationUncompressedBytes, "") \
M(ExternalSortWritePart, "Number of times a temporary file was written to disk for sorting in external memory.") \
M(ExternalSortMerge, "Number of times temporary files were merged for sorting in external memory.") \
M(ExternalAggregationWritePart, "Number of times a temporary file was written to disk for aggregation in external memory.") \
M(ExternalAggregationMerge, "Number of times temporary files were merged for aggregation in external memory.") \
M(ExternalAggregationCompressedBytes, "Number of bytes written to disk for aggregation in external memory.") \
M(ExternalAggregationUncompressedBytes, "Amount of data (uncompressed, before compression) written to disk for aggregation in external memory.") \
\
M(SlowRead, "Number of reads from a file that were slow. This indicate system overload. Thresholds are controlled by read_backoff_* settings.") \
M(ReadBackoff, "Number of times the number of query processing threads was lowered due to slow reads.") \
@ -166,46 +166,59 @@
M(RegexpCreated, "Compiled regular expressions. Identical regular expressions compiled just once and cached forever.") \
M(ContextLock, "Number of times the lock of Context was acquired or tried to acquire. This is global lock.") \
\
M(StorageBufferFlush, "") \
M(StorageBufferErrorOnFlush, "") \
M(StorageBufferPassedAllMinThresholds, "") \
M(StorageBufferPassedTimeMaxThreshold, "") \
M(StorageBufferPassedRowsMaxThreshold, "") \
M(StorageBufferPassedBytesMaxThreshold, "") \
M(StorageBufferPassedTimeFlushThreshold, "") \
M(StorageBufferPassedRowsFlushThreshold, "") \
M(StorageBufferPassedBytesFlushThreshold, "") \
M(StorageBufferLayerLockReadersWaitMilliseconds, "Time for waiting for Buffer layer during reading") \
M(StorageBufferLayerLockWritersWaitMilliseconds, "Time for waiting free Buffer layer to write to (can be used to tune Buffer layers)") \
M(StorageBufferFlush, "Number of times a buffer in a 'Buffer' table was flushed.") \
M(StorageBufferErrorOnFlush, "Number of times a buffer in the 'Buffer' table has not been able to flush due to error writing in the destination table.") \
M(StorageBufferPassedAllMinThresholds, "Number of times a criteria on min thresholds has been reached to flush a buffer in a 'Buffer' table.") \
M(StorageBufferPassedTimeMaxThreshold, "Number of times a criteria on max time threshold has been reached to flush a buffer in a 'Buffer' table.") \
M(StorageBufferPassedRowsMaxThreshold, "Number of times a criteria on max rows threshold has been reached to flush a buffer in a 'Buffer' table.") \
M(StorageBufferPassedBytesMaxThreshold, "Number of times a criteria on max bytes threshold has been reached to flush a buffer in a 'Buffer' table.") \
M(StorageBufferPassedTimeFlushThreshold, "Number of times background-only flush threshold on time has been reached to flush a buffer in a 'Buffer' table. This is expert-only metric. If you read this and you are not an expert, stop reading.") \
M(StorageBufferPassedRowsFlushThreshold, "Number of times background-only flush threshold on rows has been reached to flush a buffer in a 'Buffer' table. This is expert-only metric. If you read this and you are not an expert, stop reading.") \
M(StorageBufferPassedBytesFlushThreshold, "Number of times background-only flush threshold on bytes has been reached to flush a buffer in a 'Buffer' table. This is expert-only metric. If you read this and you are not an expert, stop reading.") \
M(StorageBufferLayerLockReadersWaitMilliseconds, "Time for waiting for Buffer layer during reading.") \
M(StorageBufferLayerLockWritersWaitMilliseconds, "Time for waiting free Buffer layer to write to (can be used to tune Buffer layers).") \
\
M(DictCacheKeysRequested, "") \
M(DictCacheKeysRequestedMiss, "") \
M(DictCacheKeysRequestedFound, "") \
M(DictCacheKeysExpired, "") \
M(DictCacheKeysNotFound, "") \
M(DictCacheKeysHit, "") \
M(DictCacheRequestTimeNs, "") \
M(DictCacheRequests, "") \
M(DictCacheLockWriteNs, "") \
M(DictCacheLockReadNs, "") \
M(DictCacheKeysRequested, "Number of keys requested from the data source for the dictionaries of 'cache' types.") \
M(DictCacheKeysRequestedMiss, "Number of keys requested from the data source for dictionaries of 'cache' types but not found in the data source.") \
M(DictCacheKeysRequestedFound, "Number of keys requested from the data source for dictionaries of 'cache' types and found in the data source.") \
M(DictCacheKeysExpired, "Number of keys looked up in the dictionaries of 'cache' types and found in the cache but they were obsolete.") \
M(DictCacheKeysNotFound, "Number of keys looked up in the dictionaries of 'cache' types and not found.") \
M(DictCacheKeysHit, "Number of keys looked up in the dictionaries of 'cache' types and found in the cache.") \
M(DictCacheRequestTimeNs, "Number of nanoseconds spend in querying the external data sources for the dictionaries of 'cache' types.") \
M(DictCacheRequests, "Number of bulk requests to the external data sources for the dictionaries of 'cache' types.") \
M(DictCacheLockWriteNs, "Number of nanoseconds spend in waiting for write lock to update the data for the dictionaries of 'cache' types.") \
M(DictCacheLockReadNs, "Number of nanoseconds spend in waiting for read lock to lookup the data for the dictionaries of 'cache' types.") \
\
M(DistributedSyncInsertionTimeoutExceeded, "") \
M(DataAfterMergeDiffersFromReplica, "") \
M(DataAfterMutationDiffersFromReplica, "") \
M(PolygonsAddedToPool, "") \
M(PolygonsInPoolAllocatedBytes, "") \
M(RWLockAcquiredReadLocks, "") \
M(RWLockAcquiredWriteLocks, "") \
M(RWLockReadersWaitMilliseconds, "") \
M(RWLockWritersWaitMilliseconds, "") \
M(DistributedSyncInsertionTimeoutExceeded, "A timeout has exceeded while waiting for shards during synchronous insertion into a Distributed table (with 'insert_distributed_sync' = 1)") \
M(DataAfterMergeDiffersFromReplica, R"(
Number of times data after merge is not byte-identical to the data on another replicas. There could be several reasons:
1. Using newer version of compression library after server update.
2. Using another compression method.
3. Non-deterministic compression algorithm (highly unlikely).
4. Non-deterministic merge algorithm due to logical error in code.
5. Data corruption in memory due to bug in code.
6. Data corruption in memory due to hardware issue.
7. Manual modification of source data after server startup.
8. Manual modification of checksums stored in ZooKeeper.
9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas.
The server successfully detected this situation and will download merged part from replica to force byte-identical result.
)") \
M(DataAfterMutationDiffersFromReplica, "Number of times data after mutation is not byte-identical to the data on another replicas. In addition to the reasons described in 'DataAfterMergeDiffersFromReplica', it is also possible due to non-deterministic mutation.") \
M(PolygonsAddedToPool, "A polygon has been added to the cache (pool) for the 'pointInPolygon' function.") \
M(PolygonsInPoolAllocatedBytes, "The number of bytes for polygons added to the cache (pool) for the 'pointInPolygon' function.") \
\
M(RWLockAcquiredReadLocks, "Number of times a read lock were acquired (in a heavy RWLock).") \
M(RWLockAcquiredWriteLocks, "Number of times a write lock were acquired (in a heavy RWLock).") \
M(RWLockReadersWaitMilliseconds, "Total time spent waiting for a read lock to be acquired (in a heavy RWLock).") \
M(RWLockWritersWaitMilliseconds, "Total time spent waiting for a write lock to be acquired (in a heavy RWLock).") \
M(DNSError, "Total count of errors in DNS resolution") \
\
M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (not that this is a sum).") \
M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
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(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \
M(SoftPageFaults, "") \
M(HardPageFaults, "") \
M(SoftPageFaults, "The number of soft page faults in query execution threads. Soft page fault usually means a miss in the memory allocator cache which required a new memory mapping from the OS and subsequent allocation of a page of physical memory.") \
M(HardPageFaults, "The number of hard page faults in query execution threads. High values indicate either that you forgot to turn off swap on your server, or eviction of memory pages of the ClickHouse binary during very high memory pressure, or successful usage of the 'mmap' read method for the tables data.") \
\
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.") \

View File

@ -0,0 +1,2 @@
SET system_events_show_zero_values = true;
SELECT event FROM system.events WHERE length(description) < 10;