Skip to content

Commit

Permalink
Use FromProto/ToProto method names in TPathId and TRowVersion (#12880)
Browse files Browse the repository at this point in the history
  • Loading branch information
snaury authored Dec 24, 2024
1 parent 18ed18e commit 7cd35f3
Show file tree
Hide file tree
Showing 87 changed files with 221 additions and 198 deletions.
14 changes: 12 additions & 2 deletions ydb/core/base/row_version.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -6,15 +6,25 @@

namespace NKikimr {

TRowVersion TRowVersion::Parse(const NKikimrProto::TRowVersion& proto) {
TRowVersion TRowVersion::FromProto(const NKikimrProto::TRowVersion& proto) {
return TRowVersion(proto.GetStep(), proto.GetTxId());
}

void TRowVersion::Serialize(NKikimrProto::TRowVersion& proto) const {
void TRowVersion::ToProto(NKikimrProto::TRowVersion& proto) const {
proto.SetStep(Step);
proto.SetTxId(TxId);
}

void TRowVersion::ToProto(NKikimrProto::TRowVersion* proto) const {
ToProto(*proto);
}

NKikimrProto::TRowVersion TRowVersion::ToProto() const {
NKikimrProto::TRowVersion proto;
ToProto(proto);
return proto;
}

} // NKikimr

Y_DECLARE_OUT_SPEC(, NKikimr::TRowVersion, stream, value) {
Expand Down
6 changes: 4 additions & 2 deletions ydb/core/base/row_version.h
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,10 @@ namespace NKikimr {
return ++copy;
}

static TRowVersion Parse(const NKikimrProto::TRowVersion& proto);
void Serialize(NKikimrProto::TRowVersion& proto) const;
static TRowVersion FromProto(const NKikimrProto::TRowVersion& proto);
void ToProto(NKikimrProto::TRowVersion& proto) const;
void ToProto(NKikimrProto::TRowVersion* proto) const;
NKikimrProto::TRowVersion ToProto() const;

friend constexpr bool operator==(const TRowVersion& a, const TRowVersion& b) {
return a.Step == b.Step && a.TxId == b.TxId;
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/grpc_services/rpc_replication.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ class TDescribeReplicationRPC: public TRpcSchemeRequestActor<TDescribeReplicatio

ConvertDirectoryEntry(desc.GetSelf(), Result.mutable_self(), true);
return DescribeReplication(desc.GetReplicationDescription().GetControllerId(),
PathIdFromPathId(desc.GetReplicationDescription().GetPathId()));
TPathId::FromProto(desc.GetReplicationDescription().GetPathId()));

case NKikimrScheme::StatusPathDoesNotExist:
case NKikimrScheme::StatusSchemeError:
Expand Down Expand Up @@ -103,7 +103,7 @@ class TDescribeReplicationRPC: public TRpcSchemeRequestActor<TDescribeReplicatio
}

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

NTabletPipe::SendData(SelfId(), ControllerPipeClient, ev.release());
Expand Down
3 changes: 1 addition & 2 deletions ydb/core/kqp/gateway/actors/analyze_actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -191,9 +191,8 @@ void TAnalyzeActor::SendStatisticsAggregatorAnalyze(const NSchemeCache::TSchemeC
auto& record = Request.Record;
record.SetOperationId(OperationId);
auto table = record.AddTables();

PathIdFromPathId(PathId, table->MutablePathId());

PathId.ToProto(table->MutablePathId());

THashMap<TString, ui32> tagByColumnName;
for (const auto& [_, tableInfo]: entry.Columns) {
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/kqp/workload_service/actors/scheme_actors.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ class TPoolFetcherActor : public TSchemeActorBase<TPoolFetcherActor> {
}

Issues.AddIssues(std::move(issues));
Send(ReplyActorId, new TEvPrivate::TEvFetchPoolResponse(status, DatabaseId, PoolId, PoolConfig, PathIdFromPathId(PathId), std::move(Issues)));
Send(ReplyActorId, new TEvPrivate::TEvFetchPoolResponse(status, DatabaseId, PoolId, PoolConfig, TPathId::FromProto(PathId), std::move(Issues)));
PassAway();
}

Expand Down
4 changes: 2 additions & 2 deletions ydb/core/persqueue/offload_actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -71,10 +71,10 @@ class TOffloadActor
auto CreateWriterFactory() {
return [=]() -> IActor* {
if (Config.HasIncrementalBackup()) {
return NBackup::NImpl::CreateLocalTableWriter(PathIdFromPathId(Config.GetIncrementalBackup().GetDstPathId()));
return NBackup::NImpl::CreateLocalTableWriter(TPathId::FromProto(Config.GetIncrementalBackup().GetDstPathId()));
} else {
return NBackup::NImpl::CreateLocalTableWriter(
PathIdFromPathId(Config.GetIncrementalRestore().GetDstPathId()),
TPathId::FromProto(Config.GetIncrementalRestore().GetDstPathId()),
NBackup::NImpl::EWriterType::Restore);
}
};
Expand Down
18 changes: 14 additions & 4 deletions ydb/core/scheme/scheme_pathid.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -97,13 +97,23 @@ TPathId::operator bool() const {
return OwnerId != InvalidOwnerId && LocalPathId != InvalidLocalPathId;
}

TPathId PathIdFromPathId(const NKikimrProto::TPathID& proto) {
TPathId TPathId::FromProto(const NKikimrProto::TPathID& proto) {
return TPathId(proto.GetOwnerId(), proto.GetLocalId());
}

void PathIdFromPathId(const TPathId& pathId, NKikimrProto::TPathID* proto) {
proto->SetOwnerId(pathId.OwnerId);
proto->SetLocalId(pathId.LocalPathId);
void TPathId::ToProto(NKikimrProto::TPathID& proto) const {
proto.SetOwnerId(OwnerId);
proto.SetLocalId(LocalPathId);
}

void TPathId::ToProto(NKikimrProto::TPathID* proto) const {
ToProto(*proto);
}

NKikimrProto::TPathID TPathId::ToProto() const {
NKikimrProto::TPathID proto;
ToProto(proto);
return proto;
}

} // NKikimr
8 changes: 5 additions & 3 deletions ydb/core/scheme/scheme_pathid.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,10 +44,12 @@ struct TPathId {
TPathId NextId() const;
TPathId PrevId() const;

}; // TPathId
static TPathId FromProto(const NKikimrProto::TPathID& proto);
void ToProto(NKikimrProto::TPathID& proto) const;
void ToProto(NKikimrProto::TPathID* proto) const;
NKikimrProto::TPathID ToProto() const;

TPathId PathIdFromPathId(const NKikimrProto::TPathID& proto);
void PathIdFromPathId(const TPathId& pathId, NKikimrProto::TPathID* proto);
}; // TPathId

} // NKikimr

Expand Down
2 changes: 1 addition & 1 deletion ydb/core/statistics/aggregator/tx_analyze.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ struct TStatisticsAggregator::TTxAnalyze : public TTxBase {
};

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ struct TStatisticsAggregator::TTxAnalyzeTableRequest : public TTxBase {
auto& record = request->Record;
record.SetOperationId(operationId);
auto& table = *record.MutableTable();
PathIdFromPathId(operationTable.PathId, table.MutablePathId());
operationTable.PathId.ToProto(table.MutablePathId());
TVector<ui32> columnTags = Scan<ui32>(SplitString(operationTable.ColumnTags, ","));
table.MutableColumnTags()->Add(columnTags.begin(), columnTags.end());
return request;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ struct TStatisticsAggregator::TTxAnalyzeTableResponse : public TTxBase {
SA_LOG_D("[" << Self->TabletID() << "] TTxAnalyzeTableResponse::Execute");

const TString operationId = Record.GetOperationId();
const TPathId pathId = PathIdFromPathId(Record.GetPathId());
const TPathId pathId = TPathId::FromProto(Record.GetPathId());
auto operationTable = Self->ForceTraversalTable(operationId, pathId);
if (!operationTable) {
SA_LOG_E("[" << Self->TabletID() << "] TTxAnalyzeTableResponse::Execute. Unknown OperationTable. Record: " << Record.ShortDebugString());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ struct TStatisticsAggregator::TTxResponseTabletDistribution : public TTxBase {
AggregateStatisticsRequest = std::make_unique<TEvStatistics::TEvAggregateStatistics>();
auto& outRecord = AggregateStatisticsRequest->Record;
outRecord.SetRound(Self->GlobalTraversalRound);
PathIdFromPathId(Self->TraversalPathId, outRecord.MutablePathId());
Self->TraversalPathId.ToProto(outRecord.MutablePathId());

const auto forceTraversalTable = Self->CurrentForceTraversalTable();
if (forceTraversalTable) {
Expand Down
6 changes: 3 additions & 3 deletions ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
THashMap<TPathId, TOldStats> oldStatsMap;

for (const auto& entry : oldStatRecord.GetEntries()) {
auto& oldEntry = oldStatsMap[PathIdFromPathId(entry.GetPathId())];
auto& oldEntry = oldStatsMap[TPathId::FromProto(entry.GetPathId())];
oldEntry.RowCount = entry.GetRowCount();
oldEntry.BytesSize = entry.GetBytesSize();
}
Expand All @@ -64,7 +64,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
newEntry->SetRowCount(entry.GetRowCount());
newEntry->SetBytesSize(entry.GetBytesSize());
} else {
auto oldIter = oldStatsMap.find(PathIdFromPathId(entry.GetPathId()));
auto oldIter = oldStatsMap.find(TPathId::FromProto(entry.GetPathId()));
if (oldIter != oldStatsMap.end()) {
newEntry->SetRowCount(oldIter->second.RowCount);
newEntry->SetBytesSize(oldIter->second.BytesSize);
Expand All @@ -91,7 +91,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase {
std::unordered_set<TPathId> newPathIds;

for (auto& entry : statRecord.GetEntries()) {
auto pathId = PathIdFromPathId(entry.GetPathId());
auto pathId = TPathId::FromProto(entry.GetPathId());
newPathIds.insert(pathId);
if (oldPathIds.find(pathId) == oldPathIds.end()) {
TStatisticsAggregator::TScheduleTraversal traversalTable;
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/statistics/service/http_request.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ void THttpRequest::DoAnalyze(const TNavigate::TEntry& entry) {
record.SetOperationId(operationId.ToBinary());

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

Send(MakePipePerNodeCacheID(false), new TEvPipeCache::TEvForward(analyze.release(), statisticsAggregatorId, true));
HttpReply("Analyze sent. OperationId: " + operationId.ToString());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ std::unique_ptr<TEvStatistics::TEvAggregateStatistics> CreateStatisticsRequest(c
auto& record = ev->Record;
record.SetRound(data.Round);

PathIdFromPathId(data.PathId, record.MutablePathId());
data.PathId.ToProto(record.MutablePathId());

auto columnTags = record.MutableColumnTags();
for (auto tag : data.ColumnTags) {
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/statistics/ut_common/ut_common.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -454,7 +454,7 @@ TAnalyzedTable::TAnalyzedTable(const TPathId& pathId, const std::vector<ui32>& c
{}

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

Expand Down
4 changes: 2 additions & 2 deletions ydb/core/tx/datashard/alter_cdc_stream_unit.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,10 @@ class TAlterCdcStreamUnit : public TExecutionUnit {

const auto& params = schemeTx.GetAlterCdcStreamNotice();
const auto& streamDesc = params.GetStreamDescription();
const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId());
const auto streamPathId = TPathId::FromProto(streamDesc.GetPathId());
const auto state = streamDesc.GetState();

const auto pathId = PathIdFromPathId(params.GetPathId());
const auto pathId = TPathId::FromProto(params.GetPathId());
Y_ABORT_UNLESS(pathId.OwnerId == DataShard.GetPathOwnerId());

const auto version = params.GetTableSchemaVersion();
Expand Down
8 changes: 4 additions & 4 deletions ydb/core/tx/datashard/cdc_stream_scan.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -425,8 +425,8 @@ class TCdcStreamScan: public IActorCallback, public IScan {
auto response = MakeHolder<TEvDataShard::TEvCdcStreamScanResponse>();

response->Record.SetTabletId(DataShard.TabletId);
PathIdFromPathId(TablePathId, response->Record.MutableTablePathId());
PathIdFromPathId(StreamPathId, response->Record.MutableStreamPathId());
TablePathId.ToProto(response->Record.MutableTablePathId());
StreamPathId.ToProto(response->Record.MutableStreamPathId());
response->Record.SetStatus(status);
response->Record.SetErrorDescription(error);
Stats.Serialize(*response->Record.MutableStats());
Expand Down Expand Up @@ -570,7 +570,7 @@ class TDataShard::TTxCdcStreamScanRun: public TTransactionBase<TDataShard> {
LOG_D("Run"
<< ": ev# " << record.ShortDebugString());

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

const auto streamPathId = PathIdFromPathId(record.GetStreamPathId());
const auto streamPathId = TPathId::FromProto(record.GetStreamPathId());
auto it = table->CdcStreams.find(streamPathId);
if (it == table->CdcStreams.end()) {
Response = MakeResponse(ctx, NKikimrTxDataShard::TEvCdcStreamScanResponse::SCHEME_ERROR,
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/datashard/check_scheme_tx_unit.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ bool TCheckSchemeTxUnit::HasPathId(TActiveTransaction *activeTx, const T &op, co

template <typename T>
TPathId TCheckSchemeTxUnit::GetPathId(const T &op) const {
auto pathId = PathIdFromPathId(op.GetPathId());
auto pathId = TPathId::FromProto(op.GetPathId());
Y_ABORT_UNLESS(DataShard.GetPathOwnerId() == pathId.OwnerId);
return pathId;
}
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/tx/datashard/create_cdc_stream_unit.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,9 @@ class TCreateCdcStreamUnit : public TExecutionUnit {
schemeTx.GetCreateCdcStreamNotice() :
schemeTx.GetCreateIncrementalBackupSrc().GetCreateCdcStreamNotice();
const auto& streamDesc = params.GetStreamDescription();
const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId());
const auto streamPathId = TPathId::FromProto(streamDesc.GetPathId());

const auto pathId = PathIdFromPathId(params.GetPathId());
const auto pathId = TPathId::FromProto(params.GetPathId());
Y_ABORT_UNLESS(pathId.OwnerId == DataShard.GetPathOwnerId());

const auto version = params.GetTableSchemaVersion();
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,8 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit {
const ::NKikimrSchemeOp::TRestoreIncrementalBackup& incrBackup,
ui64 txId)
{
TPathId tablePathId = PathIdFromPathId(incrBackup.GetSrcPathId());
TPathId dstTablePathId = PathIdFromPathId(incrBackup.GetDstPathId());
TPathId tablePathId = TPathId::FromProto(incrBackup.GetSrcPathId());
TPathId dstTablePathId = TPathId::FromProto(incrBackup.GetDstPathId());
const ui64 tableId = incrBackup.GetSrcPathId().GetLocalId();

return CreateIncrementalRestoreScan(
Expand Down
16 changes: 8 additions & 8 deletions ydb/core/tx/datashard/datashard.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1886,8 +1886,8 @@ TUserTable::TPtr TDataShard::CreateUserTable(TTransactionContext& txc,
THashMap<TPathId, TPathId> TDataShard::GetRemapIndexes(const NKikimrTxDataShard::TMoveTable& move) {
THashMap<TPathId, TPathId> remap;
for (const auto& item: move.GetReMapIndexes()) {
const auto prevId = PathIdFromPathId(item.GetSrcPathId());
const auto newId = PathIdFromPathId(item.GetDstPathId());
const auto prevId = TPathId::FromProto(item.GetSrcPathId());
const auto newId = TPathId::FromProto(item.GetDstPathId());
remap[prevId] = newId;
}
return remap;
Expand All @@ -1896,8 +1896,8 @@ THashMap<TPathId, TPathId> TDataShard::GetRemapIndexes(const NKikimrTxDataShard:
TUserTable::TPtr TDataShard::MoveUserTable(TOperation::TPtr op, const NKikimrTxDataShard::TMoveTable& move,
const TActorContext& ctx, TTransactionContext& txc)
{
const auto prevId = PathIdFromPathId(move.GetPathId());
const auto newId = PathIdFromPathId(move.GetDstPathId());
const auto prevId = TPathId::FromProto(move.GetPathId());
const auto newId = TPathId::FromProto(move.GetDstPathId());

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

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

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

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

const auto remapPrevId = PathIdFromPathId(move.GetReMapIndex().GetSrcPathId());
const auto remapNewId = PathIdFromPathId(move.GetReMapIndex().GetDstPathId());
const auto remapPrevId = TPathId::FromProto(move.GetReMapIndex().GetSrcPathId());
const auto remapNewId = TPathId::FromProto(move.GetReMapIndex().GetDstPathId());
Y_ABORT_UNLESS(move.GetReMapIndex().HasDstName());
const auto dstIndexName = move.GetReMapIndex().GetDstName();

Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/datashard/datashard__compact_borrowed.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ class TDataShard::TTxCompactBorrowed : public NTabletFlatExecutor::TTransactionB
bool Execute(TTransactionContext& txc, const TActorContext& ctx) override {
const auto& record = Ev->Get()->Record;

const auto pathId = PathIdFromPathId(record.GetPathId());
const auto pathId = TPathId::FromProto(record.GetPathId());
LOG_INFO_S(ctx, NKikimrServices::TX_DATASHARD,
"TEvCompactBorrowed request from " << Ev->Sender
<< " for table " << pathId
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/tx/datashard/datashard__compaction.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ class TDataShard::TTxCompactTable : public NTabletFlatExecutor::TTransactionBase
return true;
}

const auto pathId = PathIdFromPathId(record.GetPathId());
const auto pathId = TPathId::FromProto(record.GetPathId());

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

const auto pathId = PathIdFromPathId(record.GetPathId());
const auto pathId = TPathId::FromProto(record.GetPathId());

const auto& tableId = pathId.LocalPathId;
auto it = TableInfos.find(tableId);
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/datashard/datashard__read_iterator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -712,7 +712,7 @@ class TReader {
record.SetSeqNo(State.SeqNo + 1);

if (!State.IsHeadRead) {
State.ReadVersion.Serialize(*record.MutableSnapshot());
State.ReadVersion.ToProto(record.MutableSnapshot());
}

return useful;
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/datashard/datashard_split_dst.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ class TDataShard::TTxInitSplitMergeDestination : public NTabletFlatExecutor::TTr
TPathId tableId(Self->GetPathOwnerId(), createTable.GetId_Deprecated());
if (createTable.HasPathId()) {
Y_ABORT_UNLESS(Self->GetPathOwnerId() == createTable.GetPathId().GetOwnerId() || Self->GetPathOwnerId() == INVALID_TABLET_ID);
tableId = PathIdFromPathId(createTable.GetPathId());
tableId = TPathId::FromProto(createTable.GetPathId());
} else if (tableId.OwnerId == INVALID_TABLET_ID) {
// Legacy schemeshard before migrations, shouldn't be possible
tableId.OwnerId = Ev->Get()->Record.GetSchemeshardTabletId();
Expand Down
Loading

0 comments on commit 7cd35f3

Please sign in to comment.