aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorOleg Shatov <olegsh@ydb.tech>2024-03-05 13:19:09 +0100
committerGitHub <noreply@github.com>2024-03-05 13:19:09 +0100
commit34372a661327f9e196003eff0da15acff92dbaa3 (patch)
tree7aba1486d0a1837de83f255ec7af7098be647f2c
parent2624e3de5af5111e83518d90e9f324a01afd57bf (diff)
downloadydb-34372a661327f9e196003eff0da15acff92dbaa3.tar.gz
Tracing levels proposal implementation (#2415)
-rw-r--r--ydb/core/keyvalue/keyvalue_intermediate.cpp4
-rw-r--r--ydb/core/keyvalue/keyvalue_storage_read_request.cpp2
-rw-r--r--ydb/core/keyvalue/keyvalue_storage_request.cpp2
-rw-r--r--ydb/core/tablet/tablet_req_writelog.cpp8
-rw-r--r--ydb/core/tablet_flat/flat_exec_seat.h4
-rw-r--r--ydb/core/tablet_flat/flat_executor.cpp2
-rw-r--r--ydb/core/tablet_flat/tablet_flat_executor.h6
-rw-r--r--ydb/core/tx/datashard/datashard.cpp24
-rw-r--r--ydb/core/tx/datashard/datashard__read_iterator.cpp2
-rw-r--r--ydb/core/tx/datashard/datashard_pipeline.cpp2
-rw-r--r--ydb/library/wilson_ids/wilson.h89
11 files changed, 93 insertions, 52 deletions
diff --git a/ydb/core/keyvalue/keyvalue_intermediate.cpp b/ydb/core/keyvalue/keyvalue_intermediate.cpp
index ed8fa1c355..b48d35ef62 100644
--- a/ydb/core/keyvalue/keyvalue_intermediate.cpp
+++ b/ydb/core/keyvalue/keyvalue_intermediate.cpp
@@ -80,7 +80,7 @@ TIntermediate::TIntermediate(TActorId respondTo, TActorId keyValueActorId, ui64
, CreatedAtGeneration(channelGeneration)
, CreatedAtStep(channelStep)
, IsReplied(false)
- , Span(TWilsonTablet::Tablet, std::move(traceId), "KeyValue.Intermediate", NWilson::EFlags::AUTO_END)
+ , Span(TWilsonTablet::TabletTopLevel, std::move(traceId), "KeyValue.Intermediate", NWilson::EFlags::AUTO_END)
{
Stat.IntermediateCreatedAt = TAppData::TimeProvider->Now();
Stat.RequestType = requestType;
@@ -106,7 +106,7 @@ void TIntermediate::UpdateStat() {
}
}
} else {
- Stat.IndexRangeRead++;
+ Stat.IndexRangeRead++;
}
};
diff --git a/ydb/core/keyvalue/keyvalue_storage_read_request.cpp b/ydb/core/keyvalue/keyvalue_storage_read_request.cpp
index 4f2eff31c6..d93ac00aa0 100644
--- a/ydb/core/keyvalue/keyvalue_storage_read_request.cpp
+++ b/ydb/core/keyvalue/keyvalue_storage_read_request.cpp
@@ -511,7 +511,7 @@ public:
: IntermediateResult(std::move(intermediate))
, TabletInfo(const_cast<TTabletStorageInfo*>(tabletInfo))
, TabletGeneration(tabletGeneration)
- , Span(TWilsonTablet::Tablet, IntermediateResult->Span.GetTraceId(), "KeyValue.StorageReadRequest")
+ , Span(TWilsonTablet::TabletBasic, IntermediateResult->Span.GetTraceId(), "KeyValue.StorageReadRequest")
{}
};
diff --git a/ydb/core/keyvalue/keyvalue_storage_request.cpp b/ydb/core/keyvalue/keyvalue_storage_request.cpp
index f725b9f459..7ae8ea9617 100644
--- a/ydb/core/keyvalue/keyvalue_storage_request.cpp
+++ b/ydb/core/keyvalue/keyvalue_storage_request.cpp
@@ -78,7 +78,7 @@ public:
, TabletGeneration(tabletGeneration)
, IntermediateResults(std::move(intermediate))
, TabletInfo(const_cast<TTabletStorageInfo*>(tabletInfo))
- , Span(TWilsonTablet::Tablet, IntermediateResults->Span.GetTraceId(), "KeyValue.StorageRequest")
+ , Span(TWilsonTablet::TabletBasic, IntermediateResults->Span.GetTraceId(), "KeyValue.StorageRequest")
{
IntermediateResults->Stat.KeyvalueStorageRequestSentAt = TAppData::TimeProvider->Now();
}
diff --git a/ydb/core/tablet/tablet_req_writelog.cpp b/ydb/core/tablet/tablet_req_writelog.cpp
index 864755528b..1e7bbbb293 100644
--- a/ydb/core/tablet/tablet_req_writelog.cpp
+++ b/ydb/core/tablet/tablet_req_writelog.cpp
@@ -154,7 +154,7 @@ public:
, CommitTactic(commitTactic)
, Info(info)
, RepliesToWait(Max<ui32>())
- , RequestSpan(TWilsonTablet::Tablet, std::move(traceId), "Tablet.WriteLog")
+ , RequestSpan(TWilsonTablet::TabletDetailed, std::move(traceId), "Tablet.WriteLog")
{
References.swap(refs);
Y_ABORT_UNLESS(Info);
@@ -171,9 +171,9 @@ public:
NWilson::TTraceId innerTraceId;
if (RequestSpan) {
- auto res = BlobSpans.try_emplace(ref.Id, TWilsonTablet::Tablet, RequestSpan.GetTraceId(), "Tablet.WriteLog.Reference");
+ auto res = BlobSpans.try_emplace(ref.Id, TWilsonTablet::TabletFull, RequestSpan.GetTraceId(), "Tablet.WriteLog.Reference");
- innerTraceId = std::move(res.first->second.GetTraceId());
+ innerTraceId = res.first->second.GetTraceId();
}
SendToBS(ref.Id, ref.Buffer, ctx, handleClass, ref.Tactic ? *ref.Tactic : CommitTactic, std::move(innerTraceId));
@@ -191,7 +191,7 @@ public:
NWilson::TTraceId traceId;
if (RequestSpan) {
- auto res = BlobSpans.try_emplace(actualLogEntryId, TWilsonTablet::Tablet, RequestSpan.GetTraceId(), "Tablet.WriteLog.LogEntry");
+ auto res = BlobSpans.try_emplace(actualLogEntryId, TWilsonTablet::TabletFull, RequestSpan.GetTraceId(), "Tablet.WriteLog.LogEntry");
traceId = std::move(res.first->second.GetTraceId());
}
diff --git a/ydb/core/tablet_flat/flat_exec_seat.h b/ydb/core/tablet_flat/flat_exec_seat.h
index 500b711fad..2eef8f0c59 100644
--- a/ydb/core/tablet_flat/flat_exec_seat.h
+++ b/ydb/core/tablet_flat/flat_exec_seat.h
@@ -35,7 +35,7 @@ namespace NTabletFlatExecutor {
void Terminate(ETerminationReason reason, const TActorContext& ctx) noexcept;
void StartEnqueuedSpan() noexcept {
- WaitingSpan = NWilson::TSpan(TWilsonTablet::Tablet, Self->TxSpan.GetTraceId(), "Tablet.Transaction.Enqueued");
+ WaitingSpan = NWilson::TSpan(TWilsonTablet::TabletDetailed, Self->TxSpan.GetTraceId(), "Tablet.Transaction.Enqueued");
}
void FinishEnqueuedSpan() noexcept {
@@ -43,7 +43,7 @@ namespace NTabletFlatExecutor {
}
void CreatePendingSpan() noexcept {
- WaitingSpan = NWilson::TSpan(TWilsonTablet::Tablet, Self->TxSpan.GetTraceId(), "Tablet.Transaction.Pending");
+ WaitingSpan = NWilson::TSpan(TWilsonTablet::TabletDetailed, Self->TxSpan.GetTraceId(), "Tablet.Transaction.Pending");
}
void FinishPendingSpan() noexcept {
diff --git a/ydb/core/tablet_flat/flat_executor.cpp b/ydb/core/tablet_flat/flat_executor.cpp
index 7e74e12d18..0a9e407ed7 100644
--- a/ydb/core/tablet_flat/flat_executor.cpp
+++ b/ydb/core/tablet_flat/flat_executor.cpp
@@ -4234,7 +4234,7 @@ TString TExecutor::CheckBorrowConsistency() {
TTransactionWaitPad::TTransactionWaitPad(THolder<TSeat> seat)
: Seat(std::move(seat))
- , WaitingSpan(NWilson::TSpan(TWilsonTablet::Tablet, Seat->GetTxTraceId(), "Tablet.Transaction.Wait"))
+ , WaitingSpan(NWilson::TSpan(TWilsonTablet::TabletDetailed, Seat->GetTxTraceId(), "Tablet.Transaction.Wait"))
{}
TTransactionWaitPad::~TTransactionWaitPad()
diff --git a/ydb/core/tablet_flat/tablet_flat_executor.h b/ydb/core/tablet_flat/tablet_flat_executor.h
index d5d1155d4c..d13d723413 100644
--- a/ydb/core/tablet_flat/tablet_flat_executor.h
+++ b/ydb/core/tablet_flat/tablet_flat_executor.h
@@ -229,7 +229,7 @@ public:
}
void StartExecutionSpan() noexcept {
- TransactionExecutionSpan = NWilson::TSpan(TWilsonTablet::Tablet, TransactionSpan.GetTraceId(), "Tablet.Transaction.Execute");
+ TransactionExecutionSpan = NWilson::TSpan(TWilsonTablet::TabletDetailed, TransactionSpan.GetTraceId(), "Tablet.Transaction.Execute");
}
void FinishExecutionSpan() noexcept {
@@ -289,7 +289,7 @@ public:
{ }
ITransaction(NWilson::TTraceId &&traceId)
- : TxSpan(NWilson::TSpan(TWilsonTablet::Tablet, std::move(traceId), "Tablet.Transaction"))
+ : TxSpan(NWilson::TSpan(TWilsonTablet::TabletBasic, std::move(traceId), "Tablet.Transaction"))
{ }
virtual ~ITransaction() = default;
@@ -314,7 +314,7 @@ public:
}
void SetupTxSpan(NWilson::TTraceId traceId) noexcept {
- TxSpan = NWilson::TSpan(TWilsonTablet::Tablet, std::move(traceId), "Tablet.Transaction");
+ TxSpan = NWilson::TSpan(TWilsonTablet::TabletBasic, std::move(traceId), "Tablet.Transaction");
if (TxSpan) {
TxSpan.Attribute("Type", TypeName(*this));
}
diff --git a/ydb/core/tx/datashard/datashard.cpp b/ydb/core/tx/datashard/datashard.cpp
index 752b7473d7..0b3c72610a 100644
--- a/ydb/core/tx/datashard/datashard.cpp
+++ b/ydb/core/tx/datashard/datashard.cpp
@@ -2905,8 +2905,10 @@ void TDataShard::ProposeTransaction(TEvDataShard::TEvProposeTransaction::TPtr &&
UpdateProposeQueueSize();
} else {
// Prepare planned transactions as soon as possible
- NWilson::TSpan datashardTransactionSpan(TWilsonTablet::Tablet, std::move(ev->TraceId), "Datashard.Transaction", NWilson::EFlags::AUTO_END);
- datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ NWilson::TSpan datashardTransactionSpan(TWilsonTablet::TabletTopLevel, std::move(ev->TraceId), "Datashard.Transaction", NWilson::EFlags::AUTO_END);
+ if (datashardTransactionSpan) {
+ datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ }
Execute(new TTxProposeTransactionBase(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false, std::move(datashardTransactionSpan)), ctx);
}
@@ -2926,8 +2928,10 @@ void TDataShard::ProposeTransaction(NEvents::TDataEvents::TEvWrite::TPtr&& ev, c
UpdateProposeQueueSize();
} else {
// Prepare planned transactions as soon as possible
- NWilson::TSpan datashardTransactionSpan(TWilsonTablet::Tablet, std::move(ev->TraceId), "Datashard.WriteTransaction", NWilson::EFlags::AUTO_END);
- datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ NWilson::TSpan datashardTransactionSpan(TWilsonTablet::TabletTopLevel, std::move(ev->TraceId), "Datashard.WriteTransaction", NWilson::EFlags::AUTO_END);
+ if (datashardTransactionSpan) {
+ datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ }
Execute(new TTxWrite(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false, std::move(datashardTransactionSpan)), ctx);
}
@@ -2994,16 +2998,20 @@ void TDataShard::Handle(TEvPrivate::TEvDelayedProposeTransaction::TPtr &ev, cons
switch (item.Event->GetTypeRewrite()) {
case TEvDataShard::TEvProposeTransaction::EventType: {
auto event = IEventHandle::Downcast<TEvDataShard::TEvProposeTransaction>(std::move(item.Event));
- NWilson::TSpan datashardTransactionSpan(TWilsonTablet::Tablet, std::move(event->TraceId), "Datashard.Transaction", NWilson::EFlags::AUTO_END);
- datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ NWilson::TSpan datashardTransactionSpan(TWilsonTablet::TabletTopLevel, std::move(event->TraceId), "Datashard.Transaction", NWilson::EFlags::AUTO_END);
+ if (datashardTransactionSpan) {
+ datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ }
Execute(new TTxProposeTransactionBase(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true, std::move(datashardTransactionSpan)), ctx);
return;
}
case NEvents::TDataEvents::TEvWrite::EventType: {
auto event = IEventHandle::Downcast<NEvents::TDataEvents::TEvWrite>(std::move(item.Event));
- NWilson::TSpan datashardTransactionSpan(TWilsonTablet::Tablet, std::move(event->TraceId), "Datashard.WriteTransaction", NWilson::EFlags::AUTO_END);
- datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ NWilson::TSpan datashardTransactionSpan(TWilsonTablet::TabletTopLevel, std::move(event->TraceId), "Datashard.WriteTransaction", NWilson::EFlags::AUTO_END);
+ if (datashardTransactionSpan) {
+ datashardTransactionSpan.Attribute("Shard", std::to_string(TabletID()));
+ }
Execute(new TTxWrite(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true, std::move(datashardTransactionSpan)), ctx);
return;
diff --git a/ydb/core/tx/datashard/datashard__read_iterator.cpp b/ydb/core/tx/datashard/datashard__read_iterator.cpp
index 928592a569..1f3278e71c 100644
--- a/ydb/core/tx/datashard/datashard__read_iterator.cpp
+++ b/ydb/core/tx/datashard/datashard__read_iterator.cpp
@@ -2545,7 +2545,7 @@ void TDataShard::Handle(TEvDataShard::TEvRead::TPtr& ev, const TActorContext& ct
auto* request = ev->Get();
if (!request->ReadSpan) {
- request->ReadSpan = NWilson::TSpan(TWilsonTablet::Tablet, std::move(ev->TraceId), "Datashard.Read", NWilson::EFlags::AUTO_END);
+ request->ReadSpan = NWilson::TSpan(TWilsonTablet::TabletTopLevel, std::move(ev->TraceId), "Datashard.Read", NWilson::EFlags::AUTO_END);
request->ReadSpan.Attribute("Shard", std::to_string(TabletID()));
}
diff --git a/ydb/core/tx/datashard/datashard_pipeline.cpp b/ydb/core/tx/datashard/datashard_pipeline.cpp
index 1006ebc1a0..5213df62bb 100644
--- a/ydb/core/tx/datashard/datashard_pipeline.cpp
+++ b/ydb/core/tx/datashard/datashard_pipeline.cpp
@@ -1783,7 +1783,7 @@ EExecutionStatus TPipeline::RunExecutionPlan(TOperation::TPtr op,
return EExecutionStatus::Reschedule;
}
- NWilson::TSpan unitSpan(TWilsonTablet::Tablet, txc.TransactionExecutionSpan.GetTraceId(), "Datashard.Unit");
+ NWilson::TSpan unitSpan(TWilsonTablet::TabletFull, txc.TransactionExecutionSpan.GetTraceId(), "Datashard.Unit");
NCpuTime::TCpuTimer timer;
auto status = unit.Execute(op, txc, ctx);
diff --git a/ydb/library/wilson_ids/wilson.h b/ydb/library/wilson_ids/wilson.h
index 36d7bc1658..58885bb430 100644
--- a/ydb/library/wilson_ids/wilson.h
+++ b/ydb/library/wilson_ids/wilson.h
@@ -1,62 +1,95 @@
#pragma once
+#include <util/system/types.h>
+
namespace NKikimr {
+ struct TComponentTracingLevels {
+#ifdef DEFINE_TRACING_LEVELS
+#error "Macro collision: DEFINE_TRACING_LEVELS"
+#endif
+
+#define DEFINE_TRACING_LEVELS(COMPONENT, MINIMAL, BASIC, DETAILED, FULL, DIAGNOSTIC, TRACE) \
+ struct COMPONENT { \
+ enum : ui8 { \
+ TopLevel = MINIMAL, \
+ Basic = BASIC, \
+ Detailed = DETAILED, \
+ Full = FULL, \
+ Diagnostic = DIAGNOSTIC, \
+ Trace = TRACE, \
+ }; \
+ };
+
+
+ DEFINE_TRACING_LEVELS(TGrpcProxy, 0, 5, 9, 13, 14, 15)
+ DEFINE_TRACING_LEVELS(TQueryProcessor, 1, 5, 9, 13, 14, 15)
+ DEFINE_TRACING_LEVELS(TDistributedTransactions, 2, 6, 10, 13, 14, 15)
+ DEFINE_TRACING_LEVELS(TTablet, 3, 7, 11, 13, 14, 15)
+ DEFINE_TRACING_LEVELS(TDistributedStorage, 4, 8, 12, 13, 14, 15)
+
+#undef DEFINE_TRACING_LEVELS
+ };
+
struct TWilson {
enum {
- BlobStorage = 8, // DS proxy and lower levels
- DsProxyInternals = 9,
- VDiskTopLevel = 12,
- VDiskInternals = 13,
- PDisk = 14,
+ BlobStorage = TComponentTracingLevels::TDistributedStorage::TopLevel,
+ DsProxyInternals = TComponentTracingLevels::TDistributedStorage::Detailed,
+ VDiskTopLevel = TComponentTracingLevels::TDistributedStorage::Basic,
+ VDiskInternals = TComponentTracingLevels::TDistributedStorage::Detailed,
+ PDisk = TComponentTracingLevels::TDistributedStorage::Detailed,
+ PDiskInternals = TComponentTracingLevels::TDistributedStorage::Full,
};
};
struct TWilsonKqp {
enum {
- KqpSession = 8,
- CompileService = 9,
- CompileActor = 9,
- SessionAcquireSnapshot = 9,
+ KqpSession = TComponentTracingLevels::TQueryProcessor::TopLevel,
+ CompileService = TComponentTracingLevels::TQueryProcessor::Basic,
+ CompileActor = TComponentTracingLevels::TQueryProcessor::Basic,
+ SessionAcquireSnapshot = TComponentTracingLevels::TQueryProcessor::Basic,
- ExecuterTableResolve = 10,
- ExecuterShardsResolve = 10,
+ ExecuterTableResolve = TComponentTracingLevels::TQueryProcessor::Detailed,
+ ExecuterShardsResolve = TComponentTracingLevels::TQueryProcessor::Detailed,
- LiteralExecuter = 9,
+ LiteralExecuter = TComponentTracingLevels::TQueryProcessor::Basic,
- DataExecuter = 9,
- DataExecuterAcquireSnapshot = 10,
- DataExecuterRunTasks = 10,
+ DataExecuter = TComponentTracingLevels::TQueryProcessor::Basic,
+ DataExecuterAcquireSnapshot = TComponentTracingLevels::TQueryProcessor::Detailed,
+ DataExecuterRunTasks = TComponentTracingLevels::TQueryProcessor::Detailed,
- ScanExecuter = 9,
- ScanExecuterRunTasks = 10,
+ ScanExecuter = TComponentTracingLevels::TQueryProcessor::Basic,
+ ScanExecuterRunTasks = TComponentTracingLevels::TQueryProcessor::Detailed,
- KqpNodeSendTasks = 9,
+ KqpNodeSendTasks = TComponentTracingLevels::TQueryProcessor::Basic,
- ProposeTransaction = 9,
+ ProposeTransaction = TComponentTracingLevels::TQueryProcessor::Basic,
- ComputeActor = 9,
+ ComputeActor = TComponentTracingLevels::TQueryProcessor::Basic,
- ReadActor = 9,
- ReadActorShardsResolve = 10,
+ ReadActor = TComponentTracingLevels::TQueryProcessor::Basic,
+ ReadActorShardsResolve = TComponentTracingLevels::TQueryProcessor::Detailed,
- LookupActor = 9,
- LookupActorShardsResolve = 10,
+ LookupActor = TComponentTracingLevels::TQueryProcessor::Basic,
+ LookupActorShardsResolve = TComponentTracingLevels::TQueryProcessor::Detailed,
- BulkUpsertActor = 9,
+ BulkUpsertActor = TComponentTracingLevels::TQueryProcessor::TopLevel,
};
};
struct TWilsonTablet {
enum {
- Tablet = 15
+ TabletTopLevel = TComponentTracingLevels::TTablet::TopLevel,
+ TabletBasic = TComponentTracingLevels::TTablet::Basic,
+ TabletDetailed = TComponentTracingLevels::TTablet::Detailed,
+ TabletFull = TComponentTracingLevels::TTablet::Full,
};
};
struct TWilsonGrpc {
enum {
- RequestProxy = 9,
- RequestActor = 9,
+ RequestProxy = TComponentTracingLevels::TGrpcProxy::TopLevel,
+ RequestActor = TComponentTracingLevels::TGrpcProxy::TopLevel,
};
};