Skip to content

Commit 6d5c9c8

Browse files
committed
KIKIMR-20217 Initial datashard tracing
1 parent 377bc60 commit 6d5c9c8

20 files changed

+186
-94
lines changed

ydb/core/kqp/runtime/kqp_read_actor.cpp

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -895,8 +895,11 @@ class TKqpReadActor : public TActorBootstrapped<TKqpReadActor>, public NYql::NDq
895895

896896
Counters->CreatedIterators->Inc();
897897
ReadIdByTabletId[state->TabletId].push_back(id);
898+
899+
NWilson::TTraceId traceId; // TODO: get traceId from kqp.
900+
898901
Send(PipeCacheId, new TEvPipeCache::TEvForward(ev.Release(), state->TabletId, true),
899-
IEventHandle::FlagTrackDelivery);
902+
IEventHandle::FlagTrackDelivery, 0, std::move(traceId));
900903

901904
if (!FirstShardStarted) {
902905
state->IsFirst = true;

ydb/core/tablet_flat/flat_exec_seat.h

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,10 @@ namespace NTabletFlatExecutor {
2020
TSeat(ui32 uniqId, TAutoPtr<ITransaction> self, NWilson::TTraceId txTraceId)
2121
: UniqID(uniqId)
2222
, Self(self)
23-
, TxSpan(NWilson::TSpan(TWilsonTablet::Tablet, std::move(txTraceId), "Tablet.Transaction"))
2423
{
25-
24+
if (txTraceId) {
25+
SetupTxSpan(std::move(txTraceId));
26+
}
2627
}
2728

2829
void Describe(IOutputStream &out) const noexcept
@@ -36,7 +37,16 @@ namespace NTabletFlatExecutor {
3637

3738
void Terminate(ETerminationReason reason, const TActorContext& ctx) noexcept;
3839

39-
void CreateEnqueuedSpan() noexcept {
40+
void SetupTxSpan(NWilson::TTraceId txTraceId) noexcept {
41+
TxSpan = NWilson::TSpan(TWilsonTablet::Tablet, std::move(txTraceId), "Tablet.Transaction");
42+
TxSpan.Attribute("Type", TypeName(*Self));
43+
}
44+
45+
NWilson::TSpan CreateExecutionSpan() noexcept {
46+
return NWilson::TSpan(TWilsonTablet::Tablet, TxSpan.GetTraceId(), "Tablet.Transaction.Execute");
47+
}
48+
49+
void StartEnqueuedSpan() noexcept {
4050
WaitingSpan = NWilson::TSpan(TWilsonTablet::Tablet, TxSpan.GetTraceId(), "Tablet.Transaction.Enqueued");
4151
}
4252

ydb/core/tablet_flat/flat_executor.cpp

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,7 @@ void TExecutor::RecreatePageCollectionsCache() noexcept
195195
auto &seat = xpair.second->Seat;
196196
xpair.second->WaitingSpan.EndOk();
197197
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
198-
seat->CreateEnqueuedSpan();
198+
seat->StartEnqueuedSpan();
199199
ActivationQueue->Push(seat.Release());
200200
ActivateTransactionWaiting++;
201201
}
@@ -520,7 +520,7 @@ void TExecutor::PlanTransactionActivation() {
520520
TAutoPtr<TSeat> seat = PendingQueue->Pop();
521521
seat->FinishPendingSpan();
522522
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
523-
seat->CreateEnqueuedSpan();
523+
seat->StartEnqueuedSpan();
524524
ActivationQueue->Push(seat.Release());
525525
ActivateTransactionWaiting++;
526526
--Stats->TxPending;
@@ -541,7 +541,7 @@ void TExecutor::ActivateWaitingTransactions(TPrivatePageCache::TPage::TWaitQueue
541541
it->second->WaitingSpan.EndOk();
542542
auto &seat = it->second->Seat;
543543
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
544-
seat->CreateEnqueuedSpan();
544+
seat->StartEnqueuedSpan();
545545
ActivationQueue->Push(seat.Release());
546546
ActivateTransactionWaiting++;
547547
TransactionWaitPads.erase(waitPad);
@@ -1624,7 +1624,7 @@ void TExecutor::DoExecute(TAutoPtr<ITransaction> self, bool allowImmediate, cons
16241624

16251625
if (ActiveTransaction || ActivateTransactionWaiting || !allowImmediate) {
16261626
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
1627-
seat->CreateEnqueuedSpan();
1627+
seat->StartEnqueuedSpan();
16281628
ActivationQueue->Push(seat.Release());
16291629
ActivateTransactionWaiting++;
16301630
PlanTransactionActivation();
@@ -1653,14 +1653,14 @@ void TExecutor::ExecuteTransaction(TAutoPtr<TSeat> seat, const TActorContext &ct
16531653
PrivatePageCache->ResetTouchesAndToLoad(true);
16541654
TPageCollectionTxEnv env(*Database, *PrivatePageCache);
16551655

1656-
TTransactionContext txc(Owner->TabletID(), Generation(), Step(), *Database, env, seat->CurrentTxDataLimit, seat->TaskId);
1656+
TTransactionContext txc(*seat, Owner->TabletID(), Generation(), Step(), *Database, env, seat->CurrentTxDataLimit, seat->TaskId);
16571657
txc.NotEnoughMemory(seat->NotEnoughMemoryCount);
16581658

16591659
Database->Begin(Stamp(), env);
16601660

16611661
LWTRACK(TransactionExecuteBegin, seat->Self->Orbit, seat->UniqID);
1662-
1663-
NWilson::TSpan txExecuteSpan(TWilsonTablet::Tablet, seat->GetTxTraceId(), "Tablet.Transaction.Execute");
1662+
1663+
NWilson::TSpan txExecuteSpan = seat->CreateExecutionSpan();
16641664
const bool done = seat->Self->Execute(txc, ctx.MakeFor(OwnerActorId));
16651665
txExecuteSpan.EndOk();
16661666

@@ -1857,7 +1857,7 @@ void TExecutor::PostponeTransaction(TAutoPtr<TSeat> seat, TPageCollectionTxEnv &
18571857
// then tx may be re-activated.
18581858
if (!PrivatePageCache->GetStats().CurrentCacheMisses) {
18591859
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
1860-
seat->CreateEnqueuedSpan();
1860+
seat->StartEnqueuedSpan();
18611861
ActivationQueue->Push(seat.Release());
18621862
ActivateTransactionWaiting++;
18631863
PlanTransactionActivation();
@@ -2945,7 +2945,7 @@ void TExecutor::StartSeat(ui64 task, TResource *cookie_) noexcept
29452945
PostponedTransactions.erase(it);
29462946
Memory->AcquiredMemory(*seat, task);
29472947
LWTRACK(TransactionEnqueued, seat->Self->Orbit, seat->UniqID);
2948-
seat->CreateEnqueuedSpan();
2948+
seat->StartEnqueuedSpan();
29492949
ActivationQueue->Push(seat.Release());
29502950
ActivateTransactionWaiting++;
29512951
PlanTransactionActivation();

ydb/core/tablet_flat/flat_executor_txloglogic.cpp

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -184,12 +184,26 @@ TLogicRedo::TCommitRWTransactionResult TLogicRedo::CommitRWTransaction(
184184
if (!Batch->Commit) {
185185
Batch->Commit = CommitManager->Begin(false, ECommit::Redo, seat->GetTxTraceId());
186186
} else {
187-
i64 batchSize = Batch->Bodies.size() + 1;
187+
// Batch commit's TraceId will be used for all blobstorage requests of the batch.
188+
if (!Batch->Commit->TraceId && seat->TxSpan) {
189+
// It is possible that the original or consequent transactions didn't have a TraceId,
190+
// but if a new transaction of a batch has TraceId, use it for the whole batch
191+
// (and consequent traced transactions).
192+
Batch->Commit->TraceId = seat->GetTxTraceId();
193+
} else {
194+
seat->TxSpan.Link(Batch->Commit->TraceId, {});
195+
}
188196

189-
Batch->Commit->FirstTx->TxSpan.Attribute("BatchSize", batchSize);
197+
for (TSeat* tx = Batch->Commit->FirstTx; tx != nullptr; tx = tx->NextCommitTx) {
198+
// Update batch size of the transaction, whose TraceId the commit uses (first transaction in batch, that has TraceId).
199+
if (tx->TxSpan) {
200+
i64 batchSize = Batch->Bodies.size() + 1;
201+
tx->TxSpan.Attribute("BatchSize", batchSize);
202+
break;
203+
}
204+
}
190205

191206
seat->TxSpan.Attribute("Batched", true);
192-
seat->TxSpan.Link(Batch->Commit->FirstTx->GetTxTraceId(), {});
193207
}
194208

195209
Batch->Commit->PushTx(seat.Get());

ydb/core/tablet_flat/tablet_flat_executed.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -39,9 +39,9 @@ void TTabletExecutedFlat::Execute(TAutoPtr<ITransaction> transaction, NWilson::T
3939
static_cast<TExecutor*>(Executor())->Execute(transaction, ExecutorCtx(*TlsActivationContext), std::move(traceId));
4040
}
4141

42-
void TTabletExecutedFlat::EnqueueExecute(TAutoPtr<ITransaction> transaction) {
42+
void TTabletExecutedFlat::EnqueueExecute(TAutoPtr<ITransaction> transaction, NWilson::TTraceId traceId) {
4343
if (transaction)
44-
static_cast<TExecutor*>(Executor())->Enqueue(transaction, ExecutorCtx(*TlsActivationContext));
44+
static_cast<TExecutor*>(Executor())->Enqueue(transaction, ExecutorCtx(*TlsActivationContext), std::move(traceId));
4545
}
4646

4747
const NTable::TScheme& TTabletExecutedFlat::Scheme() const noexcept {

ydb/core/tablet_flat/tablet_flat_executed.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ class TTabletExecutedFlat : public NFlatExecutorSetup::ITablet {
2525

2626
void Execute(TAutoPtr<ITransaction> transaction, const TActorContext &ctx, NWilson::TTraceId traceId = {});
2727
void Execute(TAutoPtr<ITransaction> transaction, NWilson::TTraceId traceId = {});
28-
void EnqueueExecute(TAutoPtr<ITransaction> transaction);
28+
void EnqueueExecute(TAutoPtr<ITransaction> transaction, NWilson::TTraceId traceId = {});
2929

3030
const NTable::TScheme& Scheme() const noexcept;
3131

ydb/core/tablet_flat/tablet_flat_executor.h

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ namespace NTabletFlatExecutor {
2525
class TTransactionContext;
2626
class TExecutor;
2727
struct TPageCollectionTxEnv;
28+
struct TSeat;
2829

2930
class TTableSnapshotContext : public TThrRefBase, TNonCopyable {
3031
friend class TExecutor;
@@ -200,9 +201,10 @@ class TTransactionContext : public TTxMemoryProviderBase {
200201
friend class TExecutor;
201202

202203
public:
203-
TTransactionContext(ui64 tablet, ui32 gen, ui32 step, NTable::TDatabase &db, IExecuting &env,
204+
TTransactionContext(TSeat &seat, ui64 tablet, ui32 gen, ui32 step, NTable::TDatabase &db, IExecuting &env,
204205
ui64 memoryLimit, ui64 taskId)
205206
: TTxMemoryProviderBase(memoryLimit, taskId)
207+
, Seat(seat)
206208
, Tablet(tablet)
207209
, Generation(gen)
208210
, Step(step)
@@ -226,6 +228,7 @@ class TTransactionContext : public TTxMemoryProviderBase {
226228
}
227229

228230
public:
231+
TSeat& Seat;
229232
const ui64 Tablet = Max<ui32>();
230233
const ui32 Generation = Max<ui32>();
231234
const ui32 Step = Max<ui32>();

ydb/core/tx/datashard/datashard.cpp

Lines changed: 12 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2818,9 +2818,11 @@ void TDataShard::ProposeTransaction(TEvDataShard::TEvProposeTransaction::TPtr &&
28182818
UpdateProposeQueueSize();
28192819
} else {
28202820
// Prepare planned transactions as soon as possible
2821-
Execute(new TTxProposeTransactionBase(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false), ctx);
2821+
TTxProposeTransactionBase *tx = new TTxProposeTransactionBase(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false);
2822+
Execute(tx, ctx, tx->GetTraceId());
28222823
}
28232824
}
2825+
28242826
void TDataShard::ProposeTransaction(NEvents::TDataEvents::TEvWrite::TPtr&& ev, const TActorContext& ctx) {
28252827
auto* msg = ev->Get();
28262828
const auto& record = msg->Record;
@@ -2835,7 +2837,8 @@ void TDataShard::ProposeTransaction(NEvents::TDataEvents::TEvWrite::TPtr&& ev, c
28352837
UpdateProposeQueueSize();
28362838
} else {
28372839
// Prepare planned transactions as soon as possible
2838-
Execute(new TTxWrite(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false), ctx);
2840+
TTxWrite *tx = new TTxWrite(this, std::move(ev), TAppData::TimeProvider->Now(), NextTieBreakerIndex++, /* delayed */ false);
2841+
Execute(tx, ctx, tx->GetTraceId());
28392842
}
28402843
}
28412844

@@ -2900,12 +2903,14 @@ void TDataShard::Handle(TEvPrivate::TEvDelayedProposeTransaction::TPtr &ev, cons
29002903
switch (item.Event->GetTypeRewrite()) {
29012904
case TEvDataShard::TEvProposeTransaction::EventType: {
29022905
auto event = IEventHandle::Downcast<TEvDataShard::TEvProposeTransaction>(std::move(item.Event));
2903-
Execute(new TTxProposeTransactionBase(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true), ctx);
2906+
TTxProposeTransactionBase *tx = new TTxProposeTransactionBase(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true);
2907+
Execute(tx, ctx, tx->GetTraceId());
29042908
return;
29052909
}
29062910
case NEvents::TDataEvents::TEvWrite::EventType: {
29072911
auto event = IEventHandle::Downcast<NEvents::TDataEvents::TEvWrite>(std::move(item.Event));
2908-
Execute(new TTxWrite(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true), ctx);
2912+
TTxWrite *tx = new TTxWrite(this, std::move(event), item.ReceivedAt, item.TieBreakerIndex, /* delayed */ true);
2913+
Execute(tx, ctx, tx->GetTraceId());
29092914
return;
29102915
}
29112916
default:
@@ -4116,12 +4121,13 @@ bool TDataShard::ReassignChannelsEnabled() const {
41164121
}
41174122

41184123
void TDataShard::ExecuteProgressTx(const TActorContext& ctx) {
4119-
Execute(new TTxProgressTransaction(this), ctx);
4124+
Execute(new TTxProgressTransaction(this, {}), ctx);
41204125
}
41214126

41224127
void TDataShard::ExecuteProgressTx(TOperation::TPtr op, const TActorContext& ctx) {
41234128
Y_ABORT_UNLESS(op->IsInProgress());
4124-
Execute(new TTxProgressTransaction(this, std::move(op)), ctx);
4129+
NWilson::TTraceId traceId = op->GetTraceId();
4130+
Execute(new TTxProgressTransaction(this, std::move(op)), ctx, std::move(traceId));
41254131
}
41264132

41274133
TDuration TDataShard::CleanupTimeout() const {

ydb/core/tx/datashard/datashard.h

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
#include <ydb/core/scheme/scheme_type_registry.h>
1414
#include <ydb/core/protos/tx_datashard.pb.h>
1515
#include <ydb/core/tablet_flat/flat_row_versions.h>
16+
#include <ydb/library/actors/wilson/wilson_span.h>
1617

1718
#include <library/cpp/lwtrace/shuttle.h>
1819
#include <library/cpp/time_provider/time_provider.h>
@@ -952,6 +953,9 @@ struct TEvDataShard {
952953

953954
// Orbit used for tracking request events
954955
NLWTrace::TOrbit Orbit;
956+
957+
// Wilson span for this request.
958+
NWilson::TSpan ReadSpan;
955959
};
956960

957961
struct TEvReadResult : public TEventPB<TEvReadResult,

ydb/core/tx/datashard/datashard__progress_tx.cpp

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,8 @@
11
#include "datashard_txs.h"
22
#include "datashard_failpoints.h"
33

4+
#include <ydb/core/tablet_flat/flat_exec_seat.h>
5+
46
namespace NKikimr {
57
namespace NDataShard {
68

@@ -23,7 +25,7 @@ bool TDataShard::TTxProgressTransaction::Execute(TTransactionContext &txc, const
2325
return true;
2426
}
2527

26-
NIceDb::TNiceDb db(txc.DB);
28+
NWilson::TSpan auxExecuteSpan;
2729

2830
if (!ActiveOp) {
2931
const bool expireSnapshotsAllowed = (
@@ -44,6 +46,7 @@ bool TDataShard::TTxProgressTransaction::Execute(TTransactionContext &txc, const
4446
Self->Pipeline.ActivateWaitingTxOps(ctx);
4547

4648
ActiveOp = Self->Pipeline.GetNextActiveOp(false);
49+
4750
if (!ActiveOp) {
4851
Self->IncCounter(COUNTER_TX_PROGRESS_IDLE);
4952
LOG_INFO_S(ctx, NKikimrServices::TX_DATASHARD,
@@ -56,6 +59,16 @@ bool TDataShard::TTxProgressTransaction::Execute(TTransactionContext &txc, const
5659
<< ActiveOp->GetKind() << " " << *ActiveOp << " (unit "
5760
<< ActiveOp->GetCurrentUnit() << ") at " << Self->TabletID());
5861
ActiveOp->IncrementInProgress();
62+
63+
if (ActiveOp->OperationSpan) {
64+
if (!txc.Seat.TxSpan) {
65+
// If Progress Tx for this operation is being executed the first time,
66+
// it won't have a span, because we choose what operation to run in the transaction itself.
67+
// We create transaction span and transaction execution spans here instead.
68+
txc.Seat.SetupTxSpan(ActiveOp->GetTraceId());
69+
auxExecuteSpan = txc.Seat.CreateExecutionSpan();
70+
}
71+
}
5972
}
6073

6174
Y_ABORT_UNLESS(ActiveOp && ActiveOp->IsInProgress());
@@ -68,6 +81,7 @@ bool TDataShard::TTxProgressTransaction::Execute(TTransactionContext &txc, const
6881
case EExecutionStatus::Restart:
6982
// Restart even if current CompleteList is not empty
7083
// It will be extended in subsequent iterations
84+
auxExecuteSpan.EndOk();
7185
return false;
7286

7387
case EExecutionStatus::Reschedule:
@@ -103,6 +117,7 @@ bool TDataShard::TTxProgressTransaction::Execute(TTransactionContext &txc, const
103117
}
104118

105119
// Commit all side effects
120+
auxExecuteSpan.EndOk();
106121
return true;
107122
} catch (...) {
108123
Y_ABORT("there must be no leaked exceptions");

0 commit comments

Comments
 (0)