Skip to content

Commit 7cd35f3

Browse files
authored
Use FromProto/ToProto method names in TPathId and TRowVersion (#12880)
1 parent 18ed18e commit 7cd35f3

File tree

87 files changed

+221
-198
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

87 files changed

+221
-198
lines changed

ydb/core/base/row_version.cpp

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -6,15 +6,25 @@
66

77
namespace NKikimr {
88

9-
TRowVersion TRowVersion::Parse(const NKikimrProto::TRowVersion& proto) {
9+
TRowVersion TRowVersion::FromProto(const NKikimrProto::TRowVersion& proto) {
1010
return TRowVersion(proto.GetStep(), proto.GetTxId());
1111
}
1212

13-
void TRowVersion::Serialize(NKikimrProto::TRowVersion& proto) const {
13+
void TRowVersion::ToProto(NKikimrProto::TRowVersion& proto) const {
1414
proto.SetStep(Step);
1515
proto.SetTxId(TxId);
1616
}
1717

18+
void TRowVersion::ToProto(NKikimrProto::TRowVersion* proto) const {
19+
ToProto(*proto);
20+
}
21+
22+
NKikimrProto::TRowVersion TRowVersion::ToProto() const {
23+
NKikimrProto::TRowVersion proto;
24+
ToProto(proto);
25+
return proto;
26+
}
27+
1828
} // NKikimr
1929

2030
Y_DECLARE_OUT_SPEC(, NKikimr::TRowVersion, stream, value) {

ydb/core/base/row_version.h

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -75,8 +75,10 @@ namespace NKikimr {
7575
return ++copy;
7676
}
7777

78-
static TRowVersion Parse(const NKikimrProto::TRowVersion& proto);
79-
void Serialize(NKikimrProto::TRowVersion& proto) const;
78+
static TRowVersion FromProto(const NKikimrProto::TRowVersion& proto);
79+
void ToProto(NKikimrProto::TRowVersion& proto) const;
80+
void ToProto(NKikimrProto::TRowVersion* proto) const;
81+
NKikimrProto::TRowVersion ToProto() const;
8082

8183
friend constexpr bool operator==(const TRowVersion& a, const TRowVersion& b) {
8284
return a.Step == b.Step && a.TxId == b.TxId;

ydb/core/grpc_services/rpc_replication.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ class TDescribeReplicationRPC: public TRpcSchemeRequestActor<TDescribeReplicatio
7575

7676
ConvertDirectoryEntry(desc.GetSelf(), Result.mutable_self(), true);
7777
return DescribeReplication(desc.GetReplicationDescription().GetControllerId(),
78-
PathIdFromPathId(desc.GetReplicationDescription().GetPathId()));
78+
TPathId::FromProto(desc.GetReplicationDescription().GetPathId()));
7979

8080
case NKikimrScheme::StatusPathDoesNotExist:
8181
case NKikimrScheme::StatusSchemeError:
@@ -103,7 +103,7 @@ class TDescribeReplicationRPC: public TRpcSchemeRequestActor<TDescribeReplicatio
103103
}
104104

105105
auto ev = std::make_unique<NReplication::TEvController::TEvDescribeReplication>();
106-
PathIdFromPathId(pathId, ev->Record.MutablePathId());
106+
pathId.ToProto(ev->Record.MutablePathId());
107107
ev->Record.SetIncludeStats(GetProtoRequest()->include_stats());
108108

109109
NTabletPipe::SendData(SelfId(), ControllerPipeClient, ev.release());

ydb/core/kqp/gateway/actors/analyze_actor.cpp

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -191,9 +191,8 @@ void TAnalyzeActor::SendStatisticsAggregatorAnalyze(const NSchemeCache::TSchemeC
191191
auto& record = Request.Record;
192192
record.SetOperationId(OperationId);
193193
auto table = record.AddTables();
194-
195-
PathIdFromPathId(PathId, table->MutablePathId());
196194

195+
PathId.ToProto(table->MutablePathId());
197196

198197
THashMap<TString, ui32> tagByColumnName;
199198
for (const auto& [_, tableInfo]: entry.Columns) {

ydb/core/kqp/workload_service/actors/scheme_actors.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,7 @@ class TPoolFetcherActor : public TSchemeActorBase<TPoolFetcherActor> {
223223
}
224224

225225
Issues.AddIssues(std::move(issues));
226-
Send(ReplyActorId, new TEvPrivate::TEvFetchPoolResponse(status, DatabaseId, PoolId, PoolConfig, PathIdFromPathId(PathId), std::move(Issues)));
226+
Send(ReplyActorId, new TEvPrivate::TEvFetchPoolResponse(status, DatabaseId, PoolId, PoolConfig, TPathId::FromProto(PathId), std::move(Issues)));
227227
PassAway();
228228
}
229229

ydb/core/persqueue/offload_actor.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -71,10 +71,10 @@ class TOffloadActor
7171
auto CreateWriterFactory() {
7272
return [=]() -> IActor* {
7373
if (Config.HasIncrementalBackup()) {
74-
return NBackup::NImpl::CreateLocalTableWriter(PathIdFromPathId(Config.GetIncrementalBackup().GetDstPathId()));
74+
return NBackup::NImpl::CreateLocalTableWriter(TPathId::FromProto(Config.GetIncrementalBackup().GetDstPathId()));
7575
} else {
7676
return NBackup::NImpl::CreateLocalTableWriter(
77-
PathIdFromPathId(Config.GetIncrementalRestore().GetDstPathId()),
77+
TPathId::FromProto(Config.GetIncrementalRestore().GetDstPathId()),
7878
NBackup::NImpl::EWriterType::Restore);
7979
}
8080
};

ydb/core/scheme/scheme_pathid.cpp

Lines changed: 14 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -97,13 +97,23 @@ TPathId::operator bool() const {
9797
return OwnerId != InvalidOwnerId && LocalPathId != InvalidLocalPathId;
9898
}
9999

100-
TPathId PathIdFromPathId(const NKikimrProto::TPathID& proto) {
100+
TPathId TPathId::FromProto(const NKikimrProto::TPathID& proto) {
101101
return TPathId(proto.GetOwnerId(), proto.GetLocalId());
102102
}
103103

104-
void PathIdFromPathId(const TPathId& pathId, NKikimrProto::TPathID* proto) {
105-
proto->SetOwnerId(pathId.OwnerId);
106-
proto->SetLocalId(pathId.LocalPathId);
104+
void TPathId::ToProto(NKikimrProto::TPathID& proto) const {
105+
proto.SetOwnerId(OwnerId);
106+
proto.SetLocalId(LocalPathId);
107+
}
108+
109+
void TPathId::ToProto(NKikimrProto::TPathID* proto) const {
110+
ToProto(*proto);
111+
}
112+
113+
NKikimrProto::TPathID TPathId::ToProto() const {
114+
NKikimrProto::TPathID proto;
115+
ToProto(proto);
116+
return proto;
107117
}
108118

109119
} // NKikimr

ydb/core/scheme/scheme_pathid.h

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,10 +44,12 @@ struct TPathId {
4444
TPathId NextId() const;
4545
TPathId PrevId() const;
4646

47-
}; // TPathId
47+
static TPathId FromProto(const NKikimrProto::TPathID& proto);
48+
void ToProto(NKikimrProto::TPathID& proto) const;
49+
void ToProto(NKikimrProto::TPathID* proto) const;
50+
NKikimrProto::TPathID ToProto() const;
4851

49-
TPathId PathIdFromPathId(const NKikimrProto::TPathID& proto);
50-
void PathIdFromPathId(const TPathId& pathId, NKikimrProto::TPathID* proto);
52+
}; // TPathId
5153

5254
} // NKikimr
5355

ydb/core/statistics/aggregator/tx_analyze.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ struct TStatisticsAggregator::TTxAnalyze : public TTxBase {
5858
};
5959

6060
for (const auto& table : Record.GetTables()) {
61-
const TPathId pathId = PathIdFromPathId(table.GetPathId());
61+
const TPathId pathId = TPathId::FromProto(table.GetPathId());
6262
const TString columnTags = JoinVectorIntoString(TVector<ui32>{table.GetColumnTags().begin(),table.GetColumnTags().end()},",");
6363
const auto status = TForceTraversalTable::EStatus::None;
6464

ydb/core/statistics/aggregator/tx_analyze_table_request.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ struct TStatisticsAggregator::TTxAnalyzeTableRequest : public TTxBase {
2121
auto& record = request->Record;
2222
record.SetOperationId(operationId);
2323
auto& table = *record.MutableTable();
24-
PathIdFromPathId(operationTable.PathId, table.MutablePathId());
24+
operationTable.PathId.ToProto(table.MutablePathId());
2525
TVector<ui32> columnTags = Scan<ui32>(SplitString(operationTable.ColumnTags, ","));
2626
table.MutableColumnTags()->Add(columnTags.begin(), columnTags.end());
2727
return request;

ydb/core/statistics/aggregator/tx_analyze_table_response.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ struct TStatisticsAggregator::TTxAnalyzeTableResponse : public TTxBase {
2121
SA_LOG_D("[" << Self->TabletID() << "] TTxAnalyzeTableResponse::Execute");
2222

2323
const TString operationId = Record.GetOperationId();
24-
const TPathId pathId = PathIdFromPathId(Record.GetPathId());
24+
const TPathId pathId = TPathId::FromProto(Record.GetPathId());
2525
auto operationTable = Self->ForceTraversalTable(operationId, pathId);
2626
if (!operationTable) {
2727
SA_LOG_E("[" << Self->TabletID() << "] TTxAnalyzeTableResponse::Execute. Unknown OperationTable. Record: " << Record.ShortDebugString());

ydb/core/statistics/aggregator/tx_response_tablet_distribution.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ struct TStatisticsAggregator::TTxResponseTabletDistribution : public TTxBase {
3737
AggregateStatisticsRequest = std::make_unique<TEvStatistics::TEvAggregateStatistics>();
3838
auto& outRecord = AggregateStatisticsRequest->Record;
3939
outRecord.SetRound(Self->GlobalTraversalRound);
40-
PathIdFromPathId(Self->TraversalPathId, outRecord.MutablePathId());
40+
Self->TraversalPathId.ToProto(outRecord.MutablePathId());
4141

4242
const auto forceTraversalTable = Self->CurrentForceTraversalTable();
4343
if (forceTraversalTable) {

ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
4848
THashMap<TPathId, TOldStats> oldStatsMap;
4949

5050
for (const auto& entry : oldStatRecord.GetEntries()) {
51-
auto& oldEntry = oldStatsMap[PathIdFromPathId(entry.GetPathId())];
51+
auto& oldEntry = oldStatsMap[TPathId::FromProto(entry.GetPathId())];
5252
oldEntry.RowCount = entry.GetRowCount();
5353
oldEntry.BytesSize = entry.GetBytesSize();
5454
}
@@ -64,7 +64,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
6464
newEntry->SetRowCount(entry.GetRowCount());
6565
newEntry->SetBytesSize(entry.GetBytesSize());
6666
} else {
67-
auto oldIter = oldStatsMap.find(PathIdFromPathId(entry.GetPathId()));
67+
auto oldIter = oldStatsMap.find(TPathId::FromProto(entry.GetPathId()));
6868
if (oldIter != oldStatsMap.end()) {
6969
newEntry->SetRowCount(oldIter->second.RowCount);
7070
newEntry->SetBytesSize(oldIter->second.BytesSize);
@@ -91,7 +91,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
9191
std::unordered_set<TPathId> newPathIds;
9292

9393
for (auto& entry : statRecord.GetEntries()) {
94-
auto pathId = PathIdFromPathId(entry.GetPathId());
94+
auto pathId = TPathId::FromProto(entry.GetPathId());
9595
newPathIds.insert(pathId);
9696
if (oldPathIds.find(pathId) == oldPathIds.end()) {
9797
TStatisticsAggregator::TScheduleTraversal traversalTable;

ydb/core/statistics/service/http_request.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -178,7 +178,7 @@ void THttpRequest::DoAnalyze(const TNavigate::TEntry& entry) {
178178
record.SetOperationId(operationId.ToBinary());
179179

180180
const auto& pathId = entry.TableId.PathId;
181-
PathIdFromPathId(pathId, record.AddTables()->MutablePathId());
181+
pathId.ToProto(record.AddTables()->MutablePathId());
182182

183183
Send(MakePipePerNodeCacheID(false), new TEvPipeCache::TEvForward(analyze.release(), statisticsAggregatorId, true));
184184
HttpReply("Analyze sent. OperationId: " + operationId.ToString());

ydb/core/statistics/service/ut/ut_aggregation/ut_aggregate_statistics.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ std::unique_ptr<TEvStatistics::TEvAggregateStatistics> CreateStatisticsRequest(c
5555
auto& record = ev->Record;
5656
record.SetRound(data.Round);
5757

58-
PathIdFromPathId(data.PathId, record.MutablePathId());
58+
data.PathId.ToProto(record.MutablePathId());
5959

6060
auto columnTags = record.MutableColumnTags();
6161
for (auto tag : data.ColumnTags) {

ydb/core/statistics/ut_common/ut_common.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -454,7 +454,7 @@ TAnalyzedTable::TAnalyzedTable(const TPathId& pathId, const std::vector<ui32>& c
454454
{}
455455

456456
void TAnalyzedTable::ToProto(NKikimrStat::TTable& tableProto) const {
457-
PathIdFromPathId(PathId, tableProto.MutablePathId());
457+
PathId.ToProto(tableProto.MutablePathId());
458458
tableProto.MutableColumnTags()->Add(ColumnTags.begin(), ColumnTags.end());
459459
}
460460

ydb/core/tx/datashard/alter_cdc_stream_unit.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,10 @@ class TAlterCdcStreamUnit : public TExecutionUnit {
2929

3030
const auto& params = schemeTx.GetAlterCdcStreamNotice();
3131
const auto& streamDesc = params.GetStreamDescription();
32-
const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId());
32+
const auto streamPathId = TPathId::FromProto(streamDesc.GetPathId());
3333
const auto state = streamDesc.GetState();
3434

35-
const auto pathId = PathIdFromPathId(params.GetPathId());
35+
const auto pathId = TPathId::FromProto(params.GetPathId());
3636
Y_ABORT_UNLESS(pathId.OwnerId == DataShard.GetPathOwnerId());
3737

3838
const auto version = params.GetTableSchemaVersion();

ydb/core/tx/datashard/cdc_stream_scan.cpp

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -425,8 +425,8 @@ class TCdcStreamScan: public IActorCallback, public IScan {
425425
auto response = MakeHolder<TEvDataShard::TEvCdcStreamScanResponse>();
426426

427427
response->Record.SetTabletId(DataShard.TabletId);
428-
PathIdFromPathId(TablePathId, response->Record.MutableTablePathId());
429-
PathIdFromPathId(StreamPathId, response->Record.MutableStreamPathId());
428+
TablePathId.ToProto(response->Record.MutableTablePathId());
429+
StreamPathId.ToProto(response->Record.MutableStreamPathId());
430430
response->Record.SetStatus(status);
431431
response->Record.SetErrorDescription(error);
432432
Stats.Serialize(*response->Record.MutableStats());
@@ -570,7 +570,7 @@ class TDataShard::TTxCdcStreamScanRun: public TTransactionBase<TDataShard> {
570570
LOG_D("Run"
571571
<< ": ev# " << record.ShortDebugString());
572572

573-
const auto tablePathId = PathIdFromPathId(record.GetTablePathId());
573+
const auto tablePathId = TPathId::FromProto(record.GetTablePathId());
574574
if (!Self->GetUserTables().contains(tablePathId.LocalPathId)) {
575575
Response = MakeResponse(ctx, NKikimrTxDataShard::TEvCdcStreamScanResponse::BAD_REQUEST,
576576
TStringBuilder() << "Unknown table"
@@ -588,7 +588,7 @@ class TDataShard::TTxCdcStreamScanRun: public TTransactionBase<TDataShard> {
588588
return true;
589589
}
590590

591-
const auto streamPathId = PathIdFromPathId(record.GetStreamPathId());
591+
const auto streamPathId = TPathId::FromProto(record.GetStreamPathId());
592592
auto it = table->CdcStreams.find(streamPathId);
593593
if (it == table->CdcStreams.end()) {
594594
Response = MakeResponse(ctx, NKikimrTxDataShard::TEvCdcStreamScanResponse::SCHEME_ERROR,

ydb/core/tx/datashard/check_scheme_tx_unit.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -303,7 +303,7 @@ bool TCheckSchemeTxUnit::HasPathId(TActiveTransaction *activeTx, const T &op, co
303303

304304
template <typename T>
305305
TPathId TCheckSchemeTxUnit::GetPathId(const T &op) const {
306-
auto pathId = PathIdFromPathId(op.GetPathId());
306+
auto pathId = TPathId::FromProto(op.GetPathId());
307307
Y_ABORT_UNLESS(DataShard.GetPathOwnerId() == pathId.OwnerId);
308308
return pathId;
309309
}

ydb/core/tx/datashard/create_cdc_stream_unit.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,9 @@ class TCreateCdcStreamUnit : public TExecutionUnit {
3535
schemeTx.GetCreateCdcStreamNotice() :
3636
schemeTx.GetCreateIncrementalBackupSrc().GetCreateCdcStreamNotice();
3737
const auto& streamDesc = params.GetStreamDescription();
38-
const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId());
38+
const auto streamPathId = TPathId::FromProto(streamDesc.GetPathId());
3939

40-
const auto pathId = PathIdFromPathId(params.GetPathId());
40+
const auto pathId = TPathId::FromProto(params.GetPathId());
4141
Y_ABORT_UNLESS(pathId.OwnerId == DataShard.GetPathOwnerId());
4242

4343
const auto version = params.GetTableSchemaVersion();

ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -61,8 +61,8 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit {
6161
const ::NKikimrSchemeOp::TRestoreIncrementalBackup& incrBackup,
6262
ui64 txId)
6363
{
64-
TPathId tablePathId = PathIdFromPathId(incrBackup.GetSrcPathId());
65-
TPathId dstTablePathId = PathIdFromPathId(incrBackup.GetDstPathId());
64+
TPathId tablePathId = TPathId::FromProto(incrBackup.GetSrcPathId());
65+
TPathId dstTablePathId = TPathId::FromProto(incrBackup.GetDstPathId());
6666
const ui64 tableId = incrBackup.GetSrcPathId().GetLocalId();
6767

6868
return CreateIncrementalRestoreScan(

ydb/core/tx/datashard/datashard.cpp

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1886,8 +1886,8 @@ TUserTable::TPtr TDataShard::CreateUserTable(TTransactionContext& txc,
18861886
THashMap<TPathId, TPathId> TDataShard::GetRemapIndexes(const NKikimrTxDataShard::TMoveTable& move) {
18871887
THashMap<TPathId, TPathId> remap;
18881888
for (const auto& item: move.GetReMapIndexes()) {
1889-
const auto prevId = PathIdFromPathId(item.GetSrcPathId());
1890-
const auto newId = PathIdFromPathId(item.GetDstPathId());
1889+
const auto prevId = TPathId::FromProto(item.GetSrcPathId());
1890+
const auto newId = TPathId::FromProto(item.GetDstPathId());
18911891
remap[prevId] = newId;
18921892
}
18931893
return remap;
@@ -1896,8 +1896,8 @@ THashMap<TPathId, TPathId> TDataShard::GetRemapIndexes(const NKikimrTxDataShard:
18961896
TUserTable::TPtr TDataShard::MoveUserTable(TOperation::TPtr op, const NKikimrTxDataShard::TMoveTable& move,
18971897
const TActorContext& ctx, TTransactionContext& txc)
18981898
{
1899-
const auto prevId = PathIdFromPathId(move.GetPathId());
1900-
const auto newId = PathIdFromPathId(move.GetDstPathId());
1899+
const auto prevId = TPathId::FromProto(move.GetPathId());
1900+
const auto newId = TPathId::FromProto(move.GetDstPathId());
19011901

19021902
Y_ABORT_UNLESS(GetPathOwnerId() == prevId.OwnerId);
19031903
Y_ABORT_UNLESS(TableInfos.contains(prevId.LocalPathId));
@@ -1959,7 +1959,7 @@ TUserTable::TPtr TDataShard::MoveUserTable(TOperation::TPtr op, const NKikimrTxD
19591959
TUserTable::TPtr TDataShard::MoveUserIndex(TOperation::TPtr op, const NKikimrTxDataShard::TMoveIndex& move,
19601960
const TActorContext& ctx, TTransactionContext& txc)
19611961
{
1962-
const auto pathId = PathIdFromPathId(move.GetPathId());
1962+
const auto pathId = TPathId::FromProto(move.GetPathId());
19631963

19641964
Y_ABORT_UNLESS(GetPathOwnerId() == pathId.OwnerId);
19651965
Y_ABORT_UNLESS(TableInfos.contains(pathId.LocalPathId));
@@ -1973,7 +1973,7 @@ TUserTable::TPtr TDataShard::MoveUserIndex(TOperation::TPtr op, const NKikimrTxD
19731973
newTableInfo->GetSchema(schema);
19741974

19751975
if (move.GetReMapIndex().HasReplacedPathId()) {
1976-
const auto oldPathId = PathIdFromPathId(move.GetReMapIndex().GetReplacedPathId());
1976+
const auto oldPathId = TPathId::FromProto(move.GetReMapIndex().GetReplacedPathId());
19771977
newTableInfo->Indexes.erase(oldPathId);
19781978

19791979
auto& indexes = *schema.MutableTableIndexes();
@@ -1986,8 +1986,8 @@ TUserTable::TPtr TDataShard::MoveUserIndex(TOperation::TPtr op, const NKikimrTxD
19861986
}
19871987
}
19881988

1989-
const auto remapPrevId = PathIdFromPathId(move.GetReMapIndex().GetSrcPathId());
1990-
const auto remapNewId = PathIdFromPathId(move.GetReMapIndex().GetDstPathId());
1989+
const auto remapPrevId = TPathId::FromProto(move.GetReMapIndex().GetSrcPathId());
1990+
const auto remapNewId = TPathId::FromProto(move.GetReMapIndex().GetDstPathId());
19911991
Y_ABORT_UNLESS(move.GetReMapIndex().HasDstName());
19921992
const auto dstIndexName = move.GetReMapIndex().GetDstName();
19931993

ydb/core/tx/datashard/datashard__compact_borrowed.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@ class TDataShard::TTxCompactBorrowed : public NTabletFlatExecutor::TTransactionB
1515
bool Execute(TTransactionContext& txc, const TActorContext& ctx) override {
1616
const auto& record = Ev->Get()->Record;
1717

18-
const auto pathId = PathIdFromPathId(record.GetPathId());
18+
const auto pathId = TPathId::FromProto(record.GetPathId());
1919
LOG_INFO_S(ctx, NKikimrServices::TX_DATASHARD,
2020
"TEvCompactBorrowed request from " << Ev->Sender
2121
<< " for table " << pathId

ydb/core/tx/datashard/datashard__compaction.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ class TDataShard::TTxCompactTable : public NTabletFlatExecutor::TTransactionBase
3232
return true;
3333
}
3434

35-
const auto pathId = PathIdFromPathId(record.GetPathId());
35+
const auto pathId = TPathId::FromProto(record.GetPathId());
3636

3737
if (Self->GetPathOwnerId() != pathId.OwnerId) {
3838
LOG_WARN_S(ctx, NKikimrServices::TX_DATASHARD,
@@ -293,7 +293,7 @@ void TDataShard::Handle(TEvDataShard::TEvGetCompactTableStats::TPtr& ev, const T
293293
auto &record = ev->Get()->Record;
294294
auto response = MakeHolder<TEvDataShard::TEvGetCompactTableStatsResult>();
295295

296-
const auto pathId = PathIdFromPathId(record.GetPathId());
296+
const auto pathId = TPathId::FromProto(record.GetPathId());
297297

298298
const auto& tableId = pathId.LocalPathId;
299299
auto it = TableInfos.find(tableId);

ydb/core/tx/datashard/datashard__read_iterator.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -712,7 +712,7 @@ class TReader {
712712
record.SetSeqNo(State.SeqNo + 1);
713713

714714
if (!State.IsHeadRead) {
715-
State.ReadVersion.Serialize(*record.MutableSnapshot());
715+
State.ReadVersion.ToProto(record.MutableSnapshot());
716716
}
717717

718718
return useful;

ydb/core/tx/datashard/datashard_split_dst.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ class TDataShard::TTxInitSplitMergeDestination : public NTabletFlatExecutor::TTr
3838
TPathId tableId(Self->GetPathOwnerId(), createTable.GetId_Deprecated());
3939
if (createTable.HasPathId()) {
4040
Y_ABORT_UNLESS(Self->GetPathOwnerId() == createTable.GetPathId().GetOwnerId() || Self->GetPathOwnerId() == INVALID_TABLET_ID);
41-
tableId = PathIdFromPathId(createTable.GetPathId());
41+
tableId = TPathId::FromProto(createTable.GetPathId());
4242
} else if (tableId.OwnerId == INVALID_TABLET_ID) {
4343
// Legacy schemeshard before migrations, shouldn't be possible
4444
tableId.OwnerId = Ev->Get()->Record.GetSchemeshardTabletId();

0 commit comments

Comments
 (0)