2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2018-05-31 15:54:08 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2018-05-17 16:01:41 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
2014-01-03 08:20:13 +00:00
|
|
|
|
2016-10-24 02:02:37 +00:00
|
|
|
/// Available events. Add something here as you wish.
|
|
|
|
#define APPLY_FOR_EVENTS(M) \
|
2017-04-01 07:20:54 +00:00
|
|
|
M(Query) \
|
|
|
|
M(SelectQuery) \
|
|
|
|
M(InsertQuery) \
|
|
|
|
M(FileOpen) \
|
|
|
|
M(FileOpenFailed) \
|
|
|
|
M(Seek) \
|
|
|
|
M(ReadBufferFromFileDescriptorRead) \
|
|
|
|
M(ReadBufferFromFileDescriptorReadFailed) \
|
|
|
|
M(ReadBufferFromFileDescriptorReadBytes) \
|
|
|
|
M(WriteBufferFromFileDescriptorWrite) \
|
|
|
|
M(WriteBufferFromFileDescriptorWriteFailed) \
|
|
|
|
M(WriteBufferFromFileDescriptorWriteBytes) \
|
|
|
|
M(ReadBufferAIORead) \
|
|
|
|
M(ReadBufferAIOReadBytes) \
|
|
|
|
M(WriteBufferAIOWrite) \
|
|
|
|
M(WriteBufferAIOWriteBytes) \
|
|
|
|
M(ReadCompressedBytes) \
|
|
|
|
M(CompressedReadBufferBlocks) \
|
|
|
|
M(CompressedReadBufferBytes) \
|
|
|
|
M(UncompressedCacheHits) \
|
|
|
|
M(UncompressedCacheMisses) \
|
|
|
|
M(UncompressedCacheWeightLost) \
|
|
|
|
M(IOBufferAllocs) \
|
|
|
|
M(IOBufferAllocBytes) \
|
|
|
|
M(ArenaAllocChunks) \
|
|
|
|
M(ArenaAllocBytes) \
|
|
|
|
M(FunctionExecute) \
|
2018-03-02 05:03:28 +00:00
|
|
|
M(TableFunctionExecute) \
|
2017-04-01 07:20:54 +00:00
|
|
|
M(MarkCacheHits) \
|
|
|
|
M(MarkCacheMisses) \
|
|
|
|
M(CreatedReadBufferOrdinary) \
|
|
|
|
M(CreatedReadBufferAIO) \
|
|
|
|
M(CreatedWriteBufferOrdinary) \
|
|
|
|
M(CreatedWriteBufferAIO) \
|
2018-05-28 19:53:03 +00:00
|
|
|
M(DiskReadElapsedMicroseconds) \
|
|
|
|
M(DiskWriteElapsedMicroseconds) \
|
|
|
|
M(NetworkReceiveElapsedMicroseconds) \
|
|
|
|
M(NetworkSendElapsedMicroseconds) \
|
|
|
|
M(ThrottlerSleepMicroseconds) \
|
2017-04-01 07:20:54 +00:00
|
|
|
\
|
|
|
|
M(ReplicatedPartFetches) \
|
|
|
|
M(ReplicatedPartFailedFetches) \
|
|
|
|
M(ObsoleteReplicatedParts) \
|
|
|
|
M(ReplicatedPartMerges) \
|
|
|
|
M(ReplicatedPartFetchesOfMerged) \
|
2018-05-25 19:44:14 +00:00
|
|
|
M(ReplicatedPartMutations) \
|
2017-04-01 07:20:54 +00:00
|
|
|
M(ReplicatedPartChecks) \
|
|
|
|
M(ReplicatedPartChecksFailed) \
|
|
|
|
M(ReplicatedDataLoss) \
|
|
|
|
\
|
|
|
|
M(InsertedRows) \
|
|
|
|
M(InsertedBytes) \
|
|
|
|
M(DelayedInserts) \
|
|
|
|
M(RejectedInserts) \
|
|
|
|
M(DelayedInsertsMilliseconds) \
|
2018-01-25 18:46:24 +00:00
|
|
|
M(DuplicatedInsertedBlocks) \
|
2017-04-01 07:20:54 +00:00
|
|
|
\
|
|
|
|
M(ZooKeeperInit) \
|
|
|
|
M(ZooKeeperTransactions) \
|
2018-04-03 19:54:35 +00:00
|
|
|
M(ZooKeeperList) \
|
2017-04-01 07:20:54 +00:00
|
|
|
M(ZooKeeperCreate) \
|
|
|
|
M(ZooKeeperRemove) \
|
|
|
|
M(ZooKeeperExists) \
|
|
|
|
M(ZooKeeperGet) \
|
|
|
|
M(ZooKeeperSet) \
|
|
|
|
M(ZooKeeperMulti) \
|
2018-04-03 19:54:35 +00:00
|
|
|
M(ZooKeeperCheck) \
|
|
|
|
M(ZooKeeperClose) \
|
2018-04-03 21:14:10 +00:00
|
|
|
M(ZooKeeperWatchResponse) \
|
2018-08-21 01:27:22 +00:00
|
|
|
M(ZooKeeperUserExceptions) \
|
|
|
|
M(ZooKeeperHardwareExceptions) \
|
|
|
|
M(ZooKeeperOtherExceptions) \
|
2018-04-03 21:07:22 +00:00
|
|
|
M(ZooKeeperWaitMicroseconds) \
|
2018-04-03 21:11:54 +00:00
|
|
|
M(ZooKeeperBytesSent) \
|
|
|
|
M(ZooKeeperBytesReceived) \
|
2017-04-01 07:20:54 +00:00
|
|
|
\
|
|
|
|
M(DistributedConnectionFailTry) \
|
2017-04-20 16:19:30 +00:00
|
|
|
M(DistributedConnectionMissingTable) \
|
|
|
|
M(DistributedConnectionStaleReplica) \
|
2017-04-01 07:20:54 +00:00
|
|
|
M(DistributedConnectionFailAtAll) \
|
|
|
|
\
|
|
|
|
M(CompileAttempt) \
|
|
|
|
M(CompileSuccess) \
|
|
|
|
\
|
2018-05-07 06:23:18 +00:00
|
|
|
M(CompileFunction) \
|
|
|
|
\
|
2017-04-01 07:20:54 +00:00
|
|
|
M(ExternalSortWritePart) \
|
|
|
|
M(ExternalSortMerge) \
|
|
|
|
M(ExternalAggregationWritePart) \
|
|
|
|
M(ExternalAggregationMerge) \
|
|
|
|
M(ExternalAggregationCompressedBytes) \
|
|
|
|
M(ExternalAggregationUncompressedBytes) \
|
|
|
|
\
|
|
|
|
M(SlowRead) \
|
|
|
|
M(ReadBackoff) \
|
|
|
|
\
|
|
|
|
M(ReplicaYieldLeadership) \
|
|
|
|
M(ReplicaPartialShutdown) \
|
|
|
|
\
|
|
|
|
M(SelectedParts) \
|
|
|
|
M(SelectedRanges) \
|
|
|
|
M(SelectedMarks) \
|
|
|
|
\
|
|
|
|
M(MergedRows) \
|
|
|
|
M(MergedUncompressedBytes) \
|
|
|
|
M(MergesTimeMilliseconds)\
|
|
|
|
\
|
|
|
|
M(MergeTreeDataWriterRows) \
|
|
|
|
M(MergeTreeDataWriterUncompressedBytes) \
|
|
|
|
M(MergeTreeDataWriterCompressedBytes) \
|
|
|
|
M(MergeTreeDataWriterBlocks) \
|
|
|
|
M(MergeTreeDataWriterBlocksAlreadySorted) \
|
|
|
|
\
|
|
|
|
M(ObsoleteEphemeralNode) \
|
|
|
|
M(CannotRemoveEphemeralNode) \
|
|
|
|
M(LeaderElectionAcquiredLeadership) \
|
|
|
|
\
|
|
|
|
M(RegexpCreated) \
|
|
|
|
M(ContextLock) \
|
|
|
|
\
|
|
|
|
M(StorageBufferFlush) \
|
|
|
|
M(StorageBufferErrorOnFlush) \
|
|
|
|
M(StorageBufferPassedAllMinThresholds) \
|
|
|
|
M(StorageBufferPassedTimeMaxThreshold) \
|
|
|
|
M(StorageBufferPassedRowsMaxThreshold) \
|
|
|
|
M(StorageBufferPassedBytesMaxThreshold) \
|
|
|
|
\
|
|
|
|
M(DictCacheKeysRequested) \
|
|
|
|
M(DictCacheKeysRequestedMiss) \
|
|
|
|
M(DictCacheKeysRequestedFound) \
|
|
|
|
M(DictCacheKeysExpired) \
|
|
|
|
M(DictCacheKeysNotFound) \
|
|
|
|
M(DictCacheKeysHit) \
|
|
|
|
M(DictCacheRequestTimeNs) \
|
|
|
|
M(DictCacheRequests) \
|
|
|
|
M(DictCacheLockWriteNs) \
|
|
|
|
M(DictCacheLockReadNs) \
|
2017-07-27 15:24:39 +00:00
|
|
|
\
|
|
|
|
M(DistributedSyncInsertionTimeoutExceeded) \
|
2017-09-20 02:30:44 +00:00
|
|
|
M(DataAfterMergeDiffersFromReplica) \
|
2018-05-25 19:44:14 +00:00
|
|
|
M(DataAfterMutationDiffersFromReplica) \
|
2017-09-20 02:30:44 +00:00
|
|
|
M(PolygonsAddedToPool) \
|
|
|
|
M(PolygonsInPoolAllocatedBytes) \
|
2017-09-26 17:19:16 +00:00
|
|
|
M(RWLockAcquiredReadLocks) \
|
|
|
|
M(RWLockAcquiredWriteLocks) \
|
|
|
|
M(RWLockReadersWaitMilliseconds) \
|
2018-02-01 17:55:08 +00:00
|
|
|
M(RWLockWritersWaitMilliseconds) \
|
2018-05-14 16:09:00 +00:00
|
|
|
M(NetworkErrors) \
|
2018-05-17 16:01:41 +00:00
|
|
|
\
|
2018-02-01 17:55:08 +00:00
|
|
|
M(RealTimeMicroseconds) \
|
2018-05-31 15:54:08 +00:00
|
|
|
M(UserTimeMicroseconds) \
|
|
|
|
M(SystemTimeMicroseconds) \
|
|
|
|
M(SoftPageFaults) \
|
|
|
|
M(HardPageFaults) \
|
|
|
|
M(VoluntaryContextSwitches) \
|
|
|
|
M(InvoluntaryContextSwitches) \
|
|
|
|
\
|
|
|
|
M(OSIOWaitMicroseconds) \
|
2018-06-20 15:21:42 +00:00
|
|
|
M(OSCPUWaitMicroseconds) \
|
|
|
|
M(OSCPUVirtualTimeMicroseconds) \
|
2018-05-17 16:01:41 +00:00
|
|
|
M(OSReadBytes) \
|
|
|
|
M(OSWriteBytes) \
|
|
|
|
M(OSReadChars) \
|
2018-05-31 15:54:08 +00:00
|
|
|
M(OSWriteChars) \
|
2016-10-24 02:02:37 +00:00
|
|
|
|
|
|
|
|
2014-01-03 08:20:13 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
2016-10-24 02:02:37 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
#define M(NAME) extern const Event NAME = __COUNTER__;
|
|
|
|
APPLY_FOR_EVENTS(M)
|
|
|
|
#undef M
|
|
|
|
constexpr Event END = __COUNTER__;
|
|
|
|
|
|
|
|
/// Global variable, initialized by zeros.
|
|
|
|
Counter global_counters_array[END] {};
|
|
|
|
/// Initialize global counters statically
|
|
|
|
Counters global_counters(global_counters_array);
|
|
|
|
|
|
|
|
const Event Counters::num_counters = END;
|
2016-10-24 02:02:37 +00:00
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
|
2018-06-09 15:29:08 +00:00
|
|
|
Counters::Counters(VariableContext level, Counters * parent)
|
2018-06-01 19:39:32 +00:00
|
|
|
: counters_holder(new Counter[num_counters] {}),
|
|
|
|
parent(parent),
|
2018-05-31 15:54:08 +00:00
|
|
|
level(level)
|
2018-02-01 17:55:08 +00:00
|
|
|
{
|
|
|
|
counters = counters_holder.get();
|
|
|
|
}
|
|
|
|
|
|
|
|
void Counters::resetCounters()
|
|
|
|
{
|
|
|
|
if (counters)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-01 17:55:08 +00:00
|
|
|
for (Event i = 0; i < num_counters; ++i)
|
|
|
|
counters[i].store(0, std::memory_order_relaxed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-02-01 17:55:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void Counters::reset()
|
|
|
|
{
|
|
|
|
parent = nullptr;
|
|
|
|
resetCounters();
|
|
|
|
}
|
|
|
|
|
2018-06-01 19:39:32 +00:00
|
|
|
Counters Counters::getPartiallyAtomicSnapshot() const
|
2018-03-01 16:52:24 +00:00
|
|
|
{
|
2018-06-09 15:29:08 +00:00
|
|
|
Counters res(VariableContext::Snapshot, nullptr);
|
2018-03-01 16:52:24 +00:00
|
|
|
for (Event i = 0; i < num_counters; ++i)
|
2018-05-31 15:54:08 +00:00
|
|
|
res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed);
|
2018-06-01 19:39:32 +00:00
|
|
|
return res;
|
2018-03-01 16:52:24 +00:00
|
|
|
}
|
|
|
|
|
2018-02-01 17:55:08 +00:00
|
|
|
const char * getDescription(Event event)
|
|
|
|
{
|
|
|
|
static const char * descriptions[] =
|
|
|
|
{
|
|
|
|
#define M(NAME) #NAME,
|
|
|
|
APPLY_FOR_EVENTS(M)
|
|
|
|
#undef M
|
|
|
|
};
|
|
|
|
|
|
|
|
return descriptions[event];
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Event end() { return END; }
|
|
|
|
|
|
|
|
|
|
|
|
void increment(Event event, Count amount)
|
|
|
|
{
|
2018-05-31 15:54:08 +00:00
|
|
|
DB::CurrentThread::getProfileEvents().increment(event, amount);
|
2018-02-01 17:55:08 +00:00
|
|
|
}
|
2016-10-24 02:02:37 +00:00
|
|
|
|
2014-01-03 08:20:13 +00:00
|
|
|
}
|
2016-10-24 02:02:37 +00:00
|
|
|
|
|
|
|
#undef APPLY_FOR_EVENTS
|