From 7cd35f3faa4922035e4cec57b311936082425e59 Mon Sep 17 00:00:00 2001 From: Aleksei Borzenkov Date: Tue, 24 Dec 2024 11:29:53 +0300 Subject: [PATCH] Use FromProto/ToProto method names in TPathId and TRowVersion (#12880) --- ydb/core/base/row_version.cpp | 14 +++++++-- ydb/core/base/row_version.h | 6 ++-- ydb/core/grpc_services/rpc_replication.cpp | 4 +-- ydb/core/kqp/gateway/actors/analyze_actor.cpp | 3 +- .../workload_service/actors/scheme_actors.cpp | 2 +- ydb/core/persqueue/offload_actor.cpp | 4 +-- ydb/core/scheme/scheme_pathid.cpp | 18 ++++++++--- ydb/core/scheme/scheme_pathid.h | 8 +++-- ydb/core/statistics/aggregator/tx_analyze.cpp | 2 +- .../aggregator/tx_analyze_table_request.cpp | 2 +- .../aggregator/tx_analyze_table_response.cpp | 2 +- .../tx_response_tablet_distribution.cpp | 2 +- .../aggregator/tx_schemeshard_stats.cpp | 6 ++-- ydb/core/statistics/service/http_request.cpp | 2 +- .../ut_aggregate_statistics.cpp | 2 +- ydb/core/statistics/ut_common/ut_common.cpp | 2 +- .../tx/datashard/alter_cdc_stream_unit.cpp | 4 +-- ydb/core/tx/datashard/cdc_stream_scan.cpp | 8 ++--- .../tx/datashard/check_scheme_tx_unit.cpp | 2 +- .../tx/datashard/create_cdc_stream_unit.cpp | 4 +-- .../create_incremental_restore_src_unit.cpp | 4 +-- ydb/core/tx/datashard/datashard.cpp | 16 +++++----- .../datashard/datashard__compact_borrowed.cpp | 2 +- .../tx/datashard/datashard__compaction.cpp | 4 +-- .../tx/datashard/datashard__read_iterator.cpp | 2 +- ydb/core/tx/datashard/datashard_split_dst.cpp | 2 +- .../tx/datashard/datashard_user_table.cpp | 8 ++--- .../datashard_ut_change_collector.cpp | 2 +- .../datashard/datashard_ut_local_kmeans.cpp | 6 ++-- .../datashard/datashard_ut_read_iterator.cpp | 4 +-- .../datashard_ut_reshuffle_kmeans.cpp | 6 ++-- .../tx/datashard/datashard_ut_sample_k.cpp | 6 ++-- .../tx/datashard/datashard_ut_snapshot.cpp | 4 +-- .../tx/datashard/drop_cdc_stream_unit.cpp | 4 +-- .../tx/datashard/drop_index_notice_unit.cpp | 4 +-- .../datashard/finalize_build_index_unit.cpp | 6 ++-- .../datashard/initiate_build_index_unit.cpp | 2 +- ydb/core/tx/datashard/local_kmeans.cpp | 2 +- .../tx/datashard/memory_state_migration.cpp | 30 +++++++++---------- ydb/core/tx/datashard/move_index_unit.cpp | 4 +-- ydb/core/tx/datashard/reshuffle_kmeans.cpp | 2 +- ydb/core/tx/datashard/sample_k.cpp | 2 +- .../ut_common/datashard_ut_common.cpp | 6 ++-- .../controller/assign_tx_id_ut.cpp | 2 +- .../tx/replication/controller/controller.cpp | 2 +- .../tx/replication/controller/dst_alterer.cpp | 2 +- .../tx/replication/controller/event_util.cpp | 2 +- .../controller/tx_alter_replication.cpp | 2 +- .../controller/tx_assign_tx_id.cpp | 2 +- .../controller/tx_create_replication.cpp | 2 +- .../controller/tx_describe_replication.cpp | 2 +- .../controller/tx_drop_replication.cpp | 2 +- .../replication/service/base_table_writer.cpp | 2 +- ydb/core/tx/replication/service/service.cpp | 10 +++---- ydb/core/tx/replication/service/service.h | 4 +-- .../replication/service/table_writer_ut.cpp | 6 ++-- ydb/core/tx/scheme_board/events.cpp | 6 ++-- ydb/core/tx/scheme_board/helpers.cpp | 2 +- ydb/core/tx/scheme_board/replica_ut.cpp | 12 ++++---- ydb/core/tx/scheme_board/subscriber.cpp | 2 +- ydb/core/tx/schemeshard/olap/table/table.h | 4 +-- ydb/core/tx/schemeshard/schemeshard__init.cpp | 2 +- ...chemeshard__operation_alter_cdc_stream.cpp | 4 +-- ...__operation_alter_external_data_source.cpp | 2 +- ...eshard__operation_alter_external_table.cpp | 4 +-- .../schemeshard__operation_alter_pq.cpp | 2 +- ...hemeshard__operation_alter_replication.cpp | 4 +-- .../schemeshard__operation_alter_table.cpp | 4 +-- ...hemeshard__operation_apply_build_index.cpp | 4 +-- ...shard__operation_create_external_table.cpp | 2 +- ...emeshard__operation_create_replication.cpp | 2 +- ...tion_create_restore_incremental_backup.cpp | 10 +++---- ...schemeshard__operation_drop_cdc_stream.cpp | 6 ++-- ...meshard__operation_drop_external_table.cpp | 2 +- .../schemeshard__operation_drop_index.cpp | 6 ++-- ...chemeshard__operation_drop_replication.cpp | 2 +- .../schemeshard__operation_drop_table.cpp | 2 +- ...eshard__operation_finalize_build_index.cpp | 2 +- ...eshard__operation_initiate_build_index.cpp | 2 +- .../schemeshard__operation_move_index.cpp | 8 ++--- .../schemeshard__operation_move_table.cpp | 8 ++--- .../schemeshard_build_index__progress.cpp | 12 ++++---- .../schemeshard_cdc_stream_common.cpp | 4 +-- .../schemeshard_cdc_stream_scan.cpp | 6 ++-- ydb/core/tx/schemeshard/schemeshard_impl.cpp | 8 ++--- .../tx/schemeshard/schemeshard_info_types.cpp | 2 +- .../schemeshard_path_describer.cpp | 18 +++++------ 87 files changed, 221 insertions(+), 198 deletions(-) diff --git a/ydb/core/base/row_version.cpp b/ydb/core/base/row_version.cpp index 19943f9a501d..c20e6cac94f7 100644 --- a/ydb/core/base/row_version.cpp +++ b/ydb/core/base/row_version.cpp @@ -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) { diff --git a/ydb/core/base/row_version.h b/ydb/core/base/row_version.h index 29edad310163..ba0e589ffecf 100644 --- a/ydb/core/base/row_version.h +++ b/ydb/core/base/row_version.h @@ -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; diff --git a/ydb/core/grpc_services/rpc_replication.cpp b/ydb/core/grpc_services/rpc_replication.cpp index 8b75b72a08d0..b07d0a88558d 100644 --- a/ydb/core/grpc_services/rpc_replication.cpp +++ b/ydb/core/grpc_services/rpc_replication.cpp @@ -75,7 +75,7 @@ class TDescribeReplicationRPC: public TRpcSchemeRequestActor(); - PathIdFromPathId(pathId, ev->Record.MutablePathId()); + pathId.ToProto(ev->Record.MutablePathId()); ev->Record.SetIncludeStats(GetProtoRequest()->include_stats()); NTabletPipe::SendData(SelfId(), ControllerPipeClient, ev.release()); diff --git a/ydb/core/kqp/gateway/actors/analyze_actor.cpp b/ydb/core/kqp/gateway/actors/analyze_actor.cpp index 84f18e3047d5..599330b1e55c 100644 --- a/ydb/core/kqp/gateway/actors/analyze_actor.cpp +++ b/ydb/core/kqp/gateway/actors/analyze_actor.cpp @@ -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 tagByColumnName; for (const auto& [_, tableInfo]: entry.Columns) { diff --git a/ydb/core/kqp/workload_service/actors/scheme_actors.cpp b/ydb/core/kqp/workload_service/actors/scheme_actors.cpp index fcec4fa74307..e7e5e7b4c0b8 100644 --- a/ydb/core/kqp/workload_service/actors/scheme_actors.cpp +++ b/ydb/core/kqp/workload_service/actors/scheme_actors.cpp @@ -223,7 +223,7 @@ class TPoolFetcherActor : public TSchemeActorBase { } 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(); } diff --git a/ydb/core/persqueue/offload_actor.cpp b/ydb/core/persqueue/offload_actor.cpp index 38e309b909fe..31abe3908411 100644 --- a/ydb/core/persqueue/offload_actor.cpp +++ b/ydb/core/persqueue/offload_actor.cpp @@ -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); } }; diff --git a/ydb/core/scheme/scheme_pathid.cpp b/ydb/core/scheme/scheme_pathid.cpp index 6093a418424d..bd75747041c2 100644 --- a/ydb/core/scheme/scheme_pathid.cpp +++ b/ydb/core/scheme/scheme_pathid.cpp @@ -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 diff --git a/ydb/core/scheme/scheme_pathid.h b/ydb/core/scheme/scheme_pathid.h index c62e80cca896..9ca3dceb11d2 100644 --- a/ydb/core/scheme/scheme_pathid.h +++ b/ydb/core/scheme/scheme_pathid.h @@ -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 diff --git a/ydb/core/statistics/aggregator/tx_analyze.cpp b/ydb/core/statistics/aggregator/tx_analyze.cpp index 58548d41d47e..43deab306d96 100644 --- a/ydb/core/statistics/aggregator/tx_analyze.cpp +++ b/ydb/core/statistics/aggregator/tx_analyze.cpp @@ -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{table.GetColumnTags().begin(),table.GetColumnTags().end()},","); const auto status = TForceTraversalTable::EStatus::None; diff --git a/ydb/core/statistics/aggregator/tx_analyze_table_request.cpp b/ydb/core/statistics/aggregator/tx_analyze_table_request.cpp index 983a4761c78a..0df134efef6a 100644 --- a/ydb/core/statistics/aggregator/tx_analyze_table_request.cpp +++ b/ydb/core/statistics/aggregator/tx_analyze_table_request.cpp @@ -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 columnTags = Scan(SplitString(operationTable.ColumnTags, ",")); table.MutableColumnTags()->Add(columnTags.begin(), columnTags.end()); return request; diff --git a/ydb/core/statistics/aggregator/tx_analyze_table_response.cpp b/ydb/core/statistics/aggregator/tx_analyze_table_response.cpp index 2bbade5ed146..9e5d1aaaabae 100644 --- a/ydb/core/statistics/aggregator/tx_analyze_table_response.cpp +++ b/ydb/core/statistics/aggregator/tx_analyze_table_response.cpp @@ -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()); diff --git a/ydb/core/statistics/aggregator/tx_response_tablet_distribution.cpp b/ydb/core/statistics/aggregator/tx_response_tablet_distribution.cpp index 323a8dec6afe..cf3ea094bec4 100644 --- a/ydb/core/statistics/aggregator/tx_response_tablet_distribution.cpp +++ b/ydb/core/statistics/aggregator/tx_response_tablet_distribution.cpp @@ -37,7 +37,7 @@ struct TStatisticsAggregator::TTxResponseTabletDistribution : public TTxBase { AggregateStatisticsRequest = std::make_unique(); 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) { diff --git a/ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp b/ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp index 51c59e451e33..337ddf00f944 100644 --- a/ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp +++ b/ydb/core/statistics/aggregator/tx_schemeshard_stats.cpp @@ -48,7 +48,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase { THashMap 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(); } @@ -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); @@ -91,7 +91,7 @@ struct TStatisticsAggregator::TTxSchemeShardStats : public TTxBase { std::unordered_set 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; diff --git a/ydb/core/statistics/service/http_request.cpp b/ydb/core/statistics/service/http_request.cpp index 8b44cc61d866..83044c12a155 100644 --- a/ydb/core/statistics/service/http_request.cpp +++ b/ydb/core/statistics/service/http_request.cpp @@ -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()); diff --git a/ydb/core/statistics/service/ut/ut_aggregation/ut_aggregate_statistics.cpp b/ydb/core/statistics/service/ut/ut_aggregation/ut_aggregate_statistics.cpp index dc0315d89bce..23b688a0247f 100644 --- a/ydb/core/statistics/service/ut/ut_aggregation/ut_aggregate_statistics.cpp +++ b/ydb/core/statistics/service/ut/ut_aggregation/ut_aggregate_statistics.cpp @@ -55,7 +55,7 @@ std::unique_ptr 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) { diff --git a/ydb/core/statistics/ut_common/ut_common.cpp b/ydb/core/statistics/ut_common/ut_common.cpp index 6b97cbc11f67..01a29bd1627b 100644 --- a/ydb/core/statistics/ut_common/ut_common.cpp +++ b/ydb/core/statistics/ut_common/ut_common.cpp @@ -454,7 +454,7 @@ TAnalyzedTable::TAnalyzedTable(const TPathId& pathId, const std::vector& c {} void TAnalyzedTable::ToProto(NKikimrStat::TTable& tableProto) const { - PathIdFromPathId(PathId, tableProto.MutablePathId()); + PathId.ToProto(tableProto.MutablePathId()); tableProto.MutableColumnTags()->Add(ColumnTags.begin(), ColumnTags.end()); } diff --git a/ydb/core/tx/datashard/alter_cdc_stream_unit.cpp b/ydb/core/tx/datashard/alter_cdc_stream_unit.cpp index 221297980180..fce561a62164 100644 --- a/ydb/core/tx/datashard/alter_cdc_stream_unit.cpp +++ b/ydb/core/tx/datashard/alter_cdc_stream_unit.cpp @@ -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(); diff --git a/ydb/core/tx/datashard/cdc_stream_scan.cpp b/ydb/core/tx/datashard/cdc_stream_scan.cpp index f4d300ad6cea..96b536bf7eb3 100644 --- a/ydb/core/tx/datashard/cdc_stream_scan.cpp +++ b/ydb/core/tx/datashard/cdc_stream_scan.cpp @@ -425,8 +425,8 @@ class TCdcStreamScan: public IActorCallback, public IScan { auto response = MakeHolder(); 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()); @@ -570,7 +570,7 @@ class TDataShard::TTxCdcStreamScanRun: public TTransactionBase { 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" @@ -588,7 +588,7 @@ class TDataShard::TTxCdcStreamScanRun: public TTransactionBase { 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, diff --git a/ydb/core/tx/datashard/check_scheme_tx_unit.cpp b/ydb/core/tx/datashard/check_scheme_tx_unit.cpp index 7ca8241a5a04..bbeb026be9ed 100644 --- a/ydb/core/tx/datashard/check_scheme_tx_unit.cpp +++ b/ydb/core/tx/datashard/check_scheme_tx_unit.cpp @@ -303,7 +303,7 @@ bool TCheckSchemeTxUnit::HasPathId(TActiveTransaction *activeTx, const T &op, co template 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; } diff --git a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp index 67b31fbdc6ba..b48ba444b283 100644 --- a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp +++ b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp @@ -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(); diff --git a/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp b/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp index be0177532ebc..c032d4bba19b 100644 --- a/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp +++ b/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp @@ -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( diff --git a/ydb/core/tx/datashard/datashard.cpp b/ydb/core/tx/datashard/datashard.cpp index 5cc344c85ce9..8c9845346bce 100644 --- a/ydb/core/tx/datashard/datashard.cpp +++ b/ydb/core/tx/datashard/datashard.cpp @@ -1886,8 +1886,8 @@ TUserTable::TPtr TDataShard::CreateUserTable(TTransactionContext& txc, THashMap TDataShard::GetRemapIndexes(const NKikimrTxDataShard::TMoveTable& move) { THashMap 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; @@ -1896,8 +1896,8 @@ THashMap 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)); @@ -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)); @@ -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(); @@ -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(); diff --git a/ydb/core/tx/datashard/datashard__compact_borrowed.cpp b/ydb/core/tx/datashard/datashard__compact_borrowed.cpp index 3dae5cc8ad6d..3dc6cc7adeef 100644 --- a/ydb/core/tx/datashard/datashard__compact_borrowed.cpp +++ b/ydb/core/tx/datashard/datashard__compact_borrowed.cpp @@ -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 diff --git a/ydb/core/tx/datashard/datashard__compaction.cpp b/ydb/core/tx/datashard/datashard__compaction.cpp index be8842c6373d..2a75161f5421 100644 --- a/ydb/core/tx/datashard/datashard__compaction.cpp +++ b/ydb/core/tx/datashard/datashard__compaction.cpp @@ -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, @@ -293,7 +293,7 @@ void TDataShard::Handle(TEvDataShard::TEvGetCompactTableStats::TPtr& ev, const T auto &record = ev->Get()->Record; auto response = MakeHolder(); - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); const auto& tableId = pathId.LocalPathId; auto it = TableInfos.find(tableId); diff --git a/ydb/core/tx/datashard/datashard__read_iterator.cpp b/ydb/core/tx/datashard/datashard__read_iterator.cpp index 8ae864aab400..6354d5017e4c 100644 --- a/ydb/core/tx/datashard/datashard__read_iterator.cpp +++ b/ydb/core/tx/datashard/datashard__read_iterator.cpp @@ -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; diff --git a/ydb/core/tx/datashard/datashard_split_dst.cpp b/ydb/core/tx/datashard/datashard_split_dst.cpp index d4f8cf034908..41c45727c21b 100644 --- a/ydb/core/tx/datashard/datashard_split_dst.cpp +++ b/ydb/core/tx/datashard/datashard_split_dst.cpp @@ -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(); diff --git a/ydb/core/tx/datashard/datashard_user_table.cpp b/ydb/core/tx/datashard/datashard_user_table.cpp index 0be5e9959be1..42f8ce5e1a9f 100644 --- a/ydb/core/tx/datashard/datashard_user_table.cpp +++ b/ydb/core/tx/datashard/datashard_user_table.cpp @@ -147,7 +147,7 @@ static bool IsJsonCdcStream(TUserTable::TCdcStream::EFormat format) { void TUserTable::AddCdcStream(const NKikimrSchemeOp::TCdcStreamDescription& streamDesc) { Y_ABORT_UNLESS(streamDesc.HasPathId()); - const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId()); + const auto streamPathId = TPathId::FromProto(streamDesc.GetPathId()); if (CdcStreams.contains(streamPathId)) { return; @@ -175,7 +175,7 @@ void TUserTable::SwitchCdcStreamState(const TPathId& streamPathId, TCdcStream::E GetSchema(schema); for (auto it = schema.MutableCdcStreams()->begin(); it != schema.MutableCdcStreams()->end(); ++it) { - if (streamPathId != PathIdFromPathId(it->GetPathId())) { + if (streamPathId != TPathId::FromProto(it->GetPathId())) { continue; } @@ -201,7 +201,7 @@ void TUserTable::DropCdcStream(const TPathId& streamPathId) { GetSchema(schema); for (auto it = schema.GetCdcStreams().begin(); it != schema.GetCdcStreams().end(); ++it) { - if (streamPathId != PathIdFromPathId(it->GetPathId())) { + if (streamPathId != TPathId::FromProto(it->GetPathId())) { continue; } @@ -327,7 +327,7 @@ void TUserTable::ParseProto(const NKikimrSchemeOp::TTableDescription& descr) for (const auto& streamDesc : descr.GetCdcStreams()) { Y_ABORT_UNLESS(streamDesc.HasPathId()); - CdcStreams.emplace(PathIdFromPathId(streamDesc.GetPathId()), TCdcStream(streamDesc)); + CdcStreams.emplace(TPathId::FromProto(streamDesc.GetPathId()), TCdcStream(streamDesc)); JsonCdcStreamCount += ui32(IsJsonCdcStream(streamDesc.GetFormat())); } } diff --git a/ydb/core/tx/datashard/datashard_ut_change_collector.cpp b/ydb/core/tx/datashard/datashard_ut_change_collector.cpp index 4b1c6cdcef5b..231ed6a2f92a 100644 --- a/ydb/core/tx/datashard/datashard_ut_change_collector.cpp +++ b/ydb/core/tx/datashard/datashard_ut_change_collector.cpp @@ -744,7 +744,7 @@ Y_UNIT_TEST_SUITE(CdcStreamChangeCollector) { THashMap streamPathIdToName; for (const auto& stream : entry.CdcStreams) { const auto& name = stream.GetName(); - const auto pathId = PathIdFromPathId(stream.GetPathId()); + const auto pathId = TPathId::FromProto(stream.GetPathId()); streamPathIdToName.emplace(pathId, name); } diff --git a/ydb/core/tx/datashard/datashard_ut_local_kmeans.cpp b/ydb/core/tx/datashard/datashard_ut_local_kmeans.cpp index ee7a32ab2668..5fb3c5297982 100644 --- a/ydb/core/tx/datashard/datashard_ut_local_kmeans.cpp +++ b/ydb/core/tx/datashard/datashard_ut_local_kmeans.cpp @@ -46,7 +46,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardLocalKMeansScan) { rec.SetTabletId(tid); } if (!rec.HasPathId()) { - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); } rec.SetSnapshotTxId(snapshot.TxId); @@ -113,7 +113,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardLocalKMeansScan) { rec.SetSeqNoRound(1); rec.SetTabletId(tid); - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); rec.SetSnapshotTxId(snapshot.TxId); rec.SetSnapshotStep(snapshot.Step); @@ -261,7 +261,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardLocalKMeansScan) { auto ev = std::make_unique(); auto& rec = ev->Record; - PathIdFromPathId({0, 0}, rec.MutablePathId()); + TPathId(0, 0).ToProto(rec.MutablePathId()); DoBadRequest(server, sender, ev); } { diff --git a/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp b/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp index ec390294d9ef..de1609bb9e11 100644 --- a/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp +++ b/ydb/core/tx/datashard/datashard_ut_read_iterator.cpp @@ -3143,7 +3143,7 @@ Y_UNIT_TEST_SUITE(DataShardReadIterator) { // Read in a transaction. auto readRequest1 = helper.GetBaseReadRequest(tableName, 1); readRequest1->Record.SetLockTxId(lockTxId); - snapshot.Serialize(*readRequest1->Record.MutableSnapshot()); + snapshot.ToProto(readRequest1->Record.MutableSnapshot()); AddKeyQuery(*readRequest1, {1, 1, 1}); auto readResult1 = helper.SendRead(tableName, readRequest1.release()); @@ -4594,7 +4594,7 @@ Y_UNIT_TEST_SUITE(DataShardReadIteratorConsistency) { auto* msg = ev->Get(); if (!msg->Record.HasSnapshot()) { Cerr << "... forcing read snapshot " << txVersion << Endl; - txVersion.Serialize(*msg->Record.MutableSnapshot()); + txVersion.ToProto(msg->Record.MutableSnapshot()); } } }); diff --git a/ydb/core/tx/datashard/datashard_ut_reshuffle_kmeans.cpp b/ydb/core/tx/datashard/datashard_ut_reshuffle_kmeans.cpp index f358b393d9fa..63f16142b687 100644 --- a/ydb/core/tx/datashard/datashard_ut_reshuffle_kmeans.cpp +++ b/ydb/core/tx/datashard/datashard_ut_reshuffle_kmeans.cpp @@ -45,7 +45,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardReshuffleKMeansScan) { rec.SetTabletId(tid); } if (!rec.HasPathId()) { - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); } rec.SetSnapshotTxId(snapshot.TxId); @@ -108,7 +108,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardReshuffleKMeansScan) { rec.SetSeqNoRound(1); rec.SetTabletId(tid); - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); rec.SetSnapshotTxId(snapshot.TxId); rec.SetSnapshotStep(snapshot.Step); @@ -231,7 +231,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardReshuffleKMeansScan) { auto ev = std::make_unique(); auto& rec = ev->Record; - PathIdFromPathId({0, 0}, rec.MutablePathId()); + TPathId(0, 0).ToProto(rec.MutablePathId()); DoBadRequest(server, sender, ev); } { diff --git a/ydb/core/tx/datashard/datashard_ut_sample_k.cpp b/ydb/core/tx/datashard/datashard_ut_sample_k.cpp index b765e916d5a7..6b36a2bc146b 100644 --- a/ydb/core/tx/datashard/datashard_ut_sample_k.cpp +++ b/ydb/core/tx/datashard/datashard_ut_sample_k.cpp @@ -44,7 +44,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardSampleKScan) { } if (!rec.HasPathId()) { - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); } if (rec.ColumnsSize() == 0) { @@ -92,7 +92,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardSampleKScan) { rec.SetSeqNoRound(1); rec.SetTabletId(tid); - PathIdFromPathId(tableId.PathId, rec.MutablePathId()); + tableId.PathId.ToProto(rec.MutablePathId()); rec.AddColumns("value"); rec.AddColumns("key"); @@ -250,7 +250,7 @@ Y_UNIT_TEST_SUITE (TTxDataShardSampleKScan) { auto ev = std::make_unique(); auto& rec = ev->Record; - PathIdFromPathId({0, 0}, rec.MutablePathId()); + TPathId(0, 0).ToProto(rec.MutablePathId()); DoSampleKBad(server, sender, "/Root/table-1", snapshot, ev); } { diff --git a/ydb/core/tx/datashard/datashard_ut_snapshot.cpp b/ydb/core/tx/datashard/datashard_ut_snapshot.cpp index e0d6934bf7da..6353c7b0c3ee 100644 --- a/ydb/core/tx/datashard/datashard_ut_snapshot.cpp +++ b/ydb/core/tx/datashard/datashard_ut_snapshot.cpp @@ -1187,7 +1187,7 @@ Y_UNIT_TEST_SUITE(DataShardSnapshots) { Last.MvccSnapshot.Step = record.GetSnapshot().GetStep(); Last.MvccSnapshot.TxId = record.GetSnapshot().GetTxId(); } else if (Inject.MvccSnapshot) { - Inject.MvccSnapshot.Serialize(*record.MutableSnapshot()); + Inject.MvccSnapshot.ToProto(record.MutableSnapshot()); Cerr << "TEvRead: injected MvccSnapshot" << Endl; } break; @@ -1787,7 +1787,7 @@ Y_UNIT_TEST_SUITE(DataShardSnapshots) { record.MutableTableId()->SetOwnerId(tableId.PathId.OwnerId); record.MutableTableId()->SetTableId(tableId.PathId.LocalPathId); record.MutableTableId()->SetSchemaVersion(tableId.SchemaVersion); - snapshot.Serialize(*record.MutableSnapshot()); + snapshot.ToProto(record.MutableSnapshot()); for (ui32 columnId : columns) { record.AddColumns(columnId); } diff --git a/ydb/core/tx/datashard/drop_cdc_stream_unit.cpp b/ydb/core/tx/datashard/drop_cdc_stream_unit.cpp index 64fcf4863460..0f484c4928f4 100644 --- a/ydb/core/tx/datashard/drop_cdc_stream_unit.cpp +++ b/ydb/core/tx/datashard/drop_cdc_stream_unit.cpp @@ -32,10 +32,10 @@ class TDropCdcStreamUnit : public TExecutionUnit { const auto& params = schemeTx.GetDropCdcStreamNotice(); - const auto pathId = PathIdFromPathId(params.GetPathId()); + const auto pathId = TPathId::FromProto(params.GetPathId()); Y_ABORT_UNLESS(pathId.OwnerId == DataShard.GetPathOwnerId()); - const auto streamPathId = PathIdFromPathId(params.GetStreamPathId()); + const auto streamPathId = TPathId::FromProto(params.GetStreamPathId()); const auto version = params.GetTableSchemaVersion(); Y_ABORT_UNLESS(version); diff --git a/ydb/core/tx/datashard/drop_index_notice_unit.cpp b/ydb/core/tx/datashard/drop_index_notice_unit.cpp index 6d45bd13a2af..62c8a950e796 100644 --- a/ydb/core/tx/datashard/drop_index_notice_unit.cpp +++ b/ydb/core/tx/datashard/drop_index_notice_unit.cpp @@ -31,7 +31,7 @@ class TDropIndexNoticeUnit : public TExecutionUnit { const auto& params = schemeTx.GetDropIndexNotice(); - 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(); @@ -39,7 +39,7 @@ class TDropIndexNoticeUnit : public TExecutionUnit { TUserTable::TPtr tableInfo; if (params.HasIndexPathId()) { - const auto indexPathId = PathIdFromPathId(params.GetIndexPathId()); + const auto indexPathId = TPathId::FromProto(params.GetIndexPathId()); const auto& userTables = DataShard.GetUserTables(); Y_ABORT_UNLESS(userTables.contains(pathId.LocalPathId)); diff --git a/ydb/core/tx/datashard/finalize_build_index_unit.cpp b/ydb/core/tx/datashard/finalize_build_index_unit.cpp index c9cd644393bb..dc82705e117d 100644 --- a/ydb/core/tx/datashard/finalize_build_index_unit.cpp +++ b/ydb/core/tx/datashard/finalize_build_index_unit.cpp @@ -30,7 +30,7 @@ class TFinalizeBuildIndexUnit : public TExecutionUnit { const auto& params = schemeTx.GetFinalizeBuildIndex(); - 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(); @@ -38,11 +38,11 @@ class TFinalizeBuildIndexUnit : public TExecutionUnit { TUserTable::TPtr tableInfo; if (params.HasOutcome() && params.GetOutcome().HasApply()) { - const auto indexPathId = PathIdFromPathId(params.GetOutcome().GetApply().GetIndexPathId()); + const auto indexPathId = TPathId::FromProto(params.GetOutcome().GetApply().GetIndexPathId()); tableInfo = DataShard.AlterTableSwitchIndexState(ctx, txc, pathId, version, indexPathId, NKikimrSchemeOp::EIndexStateReady); } else if (params.HasOutcome() && params.GetOutcome().HasCancel()) { - const auto indexPathId = PathIdFromPathId(params.GetOutcome().GetCancel().GetIndexPathId()); + const auto indexPathId = TPathId::FromProto(params.GetOutcome().GetCancel().GetIndexPathId()); const auto& userTables = DataShard.GetUserTables(); Y_ABORT_UNLESS(userTables.contains(pathId.LocalPathId)); diff --git a/ydb/core/tx/datashard/initiate_build_index_unit.cpp b/ydb/core/tx/datashard/initiate_build_index_unit.cpp index d0bc48a8aa4b..7257c09a9fb4 100644 --- a/ydb/core/tx/datashard/initiate_build_index_unit.cpp +++ b/ydb/core/tx/datashard/initiate_build_index_unit.cpp @@ -31,7 +31,7 @@ class TInitiateBuildIndexUnit : public TExecutionUnit { const auto& params = schemeTx.GetInitiateBuildIndex(); - 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(); diff --git a/ydb/core/tx/datashard/local_kmeans.cpp b/ydb/core/tx/datashard/local_kmeans.cpp index 2962f2d050f3..b6d6ec756889 100644 --- a/ydb/core/tx/datashard/local_kmeans.cpp +++ b/ydb/core/tx/datashard/local_kmeans.cpp @@ -648,7 +648,7 @@ void TDataShard::HandleSafe(TEvDataShard::TEvLocalKMeansRequest::TPtr& ev, const return; } - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); const auto* userTableIt = GetUserTables().FindPtr(pathId.LocalPathId); if (!userTableIt) { badRequest(TStringBuilder() << "Unknown table id: " << pathId.LocalPathId); diff --git a/ydb/core/tx/datashard/memory_state_migration.cpp b/ydb/core/tx/datashard/memory_state_migration.cpp index 00fb64d07ec0..826031acc31e 100644 --- a/ydb/core/tx/datashard/memory_state_migration.cpp +++ b/ydb/core/tx/datashard/memory_state_migration.cpp @@ -144,13 +144,13 @@ class TDataShardInMemoryRestoreActor Vars.emplace(); } if (protoVars.HasImmediateWriteEdge()) { - Vars->ImmediateWriteEdge = TRowVersion::Parse(protoVars.GetImmediateWriteEdge()); + Vars->ImmediateWriteEdge = TRowVersion::FromProto(protoVars.GetImmediateWriteEdge()); } if (protoVars.HasImmediateWriteEdgeReplied()) { - Vars->ImmediateWriteEdgeReplied = TRowVersion::Parse(protoVars.GetImmediateWriteEdgeReplied()); + Vars->ImmediateWriteEdgeReplied = TRowVersion::FromProto(protoVars.GetImmediateWriteEdgeReplied()); } if (protoVars.HasUnprotectedReadEdge()) { - Vars->UnprotectedReadEdge = TRowVersion::Parse(protoVars.GetUnprotectedReadEdge()); + Vars->UnprotectedReadEdge = TRowVersion::FromProto(protoVars.GetUnprotectedReadEdge()); } } for (const auto& protoLock : state->GetLocks()) { @@ -162,13 +162,13 @@ class TDataShardInMemoryRestoreActor row.CreateTs = protoLock.GetCreateTs(); row.Flags = protoLock.GetFlags(); if (protoLock.HasBreakVersion()) { - row.BreakVersion = TRowVersion::Parse(protoLock.GetBreakVersion()); + row.BreakVersion = TRowVersion::FromProto(protoLock.GetBreakVersion()); } for (const auto& protoPathId : protoLock.GetReadTables()) { - row.ReadTables.push_back(PathIdFromPathId(protoPathId)); + row.ReadTables.push_back(TPathId::FromProto(protoPathId)); } for (const auto& protoPathId : protoLock.GetWriteTables()) { - row.WriteTables.push_back(PathIdFromPathId(protoPathId)); + row.WriteTables.push_back(TPathId::FromProto(protoPathId)); } } for (const auto& protoRange : state->GetLockRanges()) { @@ -180,7 +180,7 @@ class TDataShardInMemoryRestoreActor return; } auto& range = row->Ranges.emplace_back(); - range.TableId = PathIdFromPathId(protoRange.GetTableId()); + range.TableId = TPathId::FromProto(protoRange.GetTableId()); range.Flags = protoRange.GetFlags(); range.Data = protoRange.GetData(); } @@ -562,9 +562,9 @@ TDataShard::TPreservedInMemoryState TDataShard::PreserveInMemoryState() { // Serialize important in-memory vars { auto* vars = state->MutableVars(); - SnapshotManager.GetImmediateWriteEdge().Serialize(*vars->MutableImmediateWriteEdge()); - SnapshotManager.GetImmediateWriteEdgeReplied().Serialize(*vars->MutableImmediateWriteEdgeReplied()); - SnapshotManager.GetUnprotectedReadEdge().Serialize(*vars->MutableUnprotectedReadEdge()); + SnapshotManager.GetImmediateWriteEdge().ToProto(vars->MutableImmediateWriteEdge()); + SnapshotManager.GetImmediateWriteEdgeReplied().ToProto(vars->MutableImmediateWriteEdgeReplied()); + SnapshotManager.GetUnprotectedReadEdge().ToProto(vars->MutableUnprotectedReadEdge()); addedMessage(vars->ByteSizeLong()); maybeCheckpoint(); @@ -588,20 +588,20 @@ TDataShard::TPreservedInMemoryState TDataShard::PreserveInMemoryState() { protoLockInfo->SetCreateTs(lockInfo.GetCreationTime().MicroSeconds()); protoLockInfo->SetFlags((ui64)lockInfo.GetFlags()); if (const auto& version = lockInfo.GetBreakVersion()) { - version->Serialize(*protoLockInfo->MutableBreakVersion()); + version->ToProto(protoLockInfo->MutableBreakVersion()); } for (const auto& pathId : lockInfo.GetReadTables()) { - PathIdFromPathId(pathId, protoLockInfo->AddReadTables()); + pathId.ToProto(protoLockInfo->AddReadTables()); } for (const auto& pathId : lockInfo.GetWriteTables()) { - PathIdFromPathId(pathId, protoLockInfo->AddWriteTables()); + pathId.ToProto(protoLockInfo->AddWriteTables()); } addedMessage(protoLockInfo->ByteSizeLong()); for (const auto& point : lockInfo.GetPoints()) { auto serialized = point.ToSerializedLockRange(); auto* protoRange = state->AddLockRanges(); protoRange->SetLockId(lockInfo.GetLockId()); - PathIdFromPathId(serialized.TableId, protoRange->MutableTableId()); + serialized.TableId.ToProto(protoRange->MutableTableId()); protoRange->SetFlags(serialized.Flags); protoRange->SetData(std::move(serialized.Data)); addedMessage(protoRange->ByteSizeLong()); @@ -610,7 +610,7 @@ TDataShard::TPreservedInMemoryState TDataShard::PreserveInMemoryState() { auto serialized = range.ToSerializedLockRange(); auto* protoRange = state->AddLockRanges(); protoRange->SetLockId(lockInfo.GetLockId()); - PathIdFromPathId(serialized.TableId, protoRange->MutableTableId()); + serialized.TableId.ToProto(protoRange->MutableTableId()); protoRange->SetFlags(serialized.Flags); protoRange->SetData(std::move(serialized.Data)); addedMessage(protoRange->ByteSizeLong()); diff --git a/ydb/core/tx/datashard/move_index_unit.cpp b/ydb/core/tx/datashard/move_index_unit.cpp index 73fa338d35e7..683a369ad9d5 100644 --- a/ydb/core/tx/datashard/move_index_unit.cpp +++ b/ydb/core/tx/datashard/move_index_unit.cpp @@ -18,8 +18,8 @@ class TMoveIndexUnit : public TExecutionUnit { } void MoveChangeRecords(NIceDb::TNiceDb& db, const NKikimrTxDataShard::TMoveIndex& move, TVector& changeRecords) { - 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()); for (auto& record: changeRecords) { if (record.PathId == remapPrevId) { diff --git a/ydb/core/tx/datashard/reshuffle_kmeans.cpp b/ydb/core/tx/datashard/reshuffle_kmeans.cpp index ee55885bdf93..b13f91ac8298 100644 --- a/ydb/core/tx/datashard/reshuffle_kmeans.cpp +++ b/ydb/core/tx/datashard/reshuffle_kmeans.cpp @@ -413,7 +413,7 @@ void TDataShard::HandleSafe(TEvDataShard::TEvReshuffleKMeansRequest::TPtr& ev, c return; } - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); const auto* userTableIt = GetUserTables().FindPtr(pathId.LocalPathId); if (!userTableIt) { badRequest(TStringBuilder() << "Unknown table id: " << pathId.LocalPathId); diff --git a/ydb/core/tx/datashard/sample_k.cpp b/ydb/core/tx/datashard/sample_k.cpp index 3b858b88b230..f2bb84bc03f5 100644 --- a/ydb/core/tx/datashard/sample_k.cpp +++ b/ydb/core/tx/datashard/sample_k.cpp @@ -272,7 +272,7 @@ void TDataShard::HandleSafe(TEvDataShard::TEvSampleKRequest::TPtr& ev, const TAc return; } - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); const auto* userTableIt = GetUserTables().FindPtr(pathId.LocalPathId); if (!userTableIt) { badRequest(TStringBuilder() << "Unknown table id: " << pathId.LocalPathId); diff --git a/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp b/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp index 7b715cc6d0ed..76f47d0c0df0 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp @@ -249,7 +249,7 @@ void TTester::RegisterTableInResolver(const TString& schemeText) table.Table.TableName = tdesc.GetName(); table.TableId.Reset(new TTableId(FAKE_SCHEMESHARD_TABLET_ID, tdesc.GetId_Deprecated())); if (tdesc.HasPathId()) { - table.TableId.Reset(new TTableId(PathIdFromPathId(tdesc.GetPathId()))); + table.TableId.Reset(new TTableId(TPathId::FromProto(tdesc.GetPathId()))); } table.KeyColumnCount = tdesc.KeyColumnIdsSize(); for (size_t i = 0; i < tdesc.ColumnsSize(); i++) { @@ -1376,7 +1376,7 @@ std::pair GetTablesByPathId( TAutoPtr handle; auto response = GetEvGetInfo(server, tabletId, handle); for (auto& table: response->Record.GetUserTables()) { - result[PathIdFromPathId(table.GetDescription().GetPathId())] = table; + result[TPathId::FromProto(table.GetDescription().GetPathId())] = table; } auto ownerId = response->Record.GetTabletInfo().GetSchemeShard(); @@ -2743,7 +2743,7 @@ std::unique_ptr GetBaseReadRequest( record.MutableTableId()->SetSchemaVersion(description.GetTableSchemaVersion()); if (readVersion) { - readVersion.Serialize(*record.MutableSnapshot()); + readVersion.ToProto(record.MutableSnapshot()); } record.SetResultFormat(format); diff --git a/ydb/core/tx/replication/controller/assign_tx_id_ut.cpp b/ydb/core/tx/replication/controller/assign_tx_id_ut.cpp index dcf738ce9e7b..4e8168216597 100644 --- a/ydb/core/tx/replication/controller/assign_tx_id_ut.cpp +++ b/ydb/core/tx/replication/controller/assign_tx_id_ut.cpp @@ -21,7 +21,7 @@ Y_UNIT_TEST_SUITE(AssignTxId) { int i = 0; for (const auto& [version, txId] : expected) { const auto& actual = result.GetVersionTxIds(i++); - UNIT_ASSERT_VALUES_EQUAL(TRowVersion::Parse(actual.GetVersion()), version); + UNIT_ASSERT_VALUES_EQUAL(TRowVersion::FromProto(actual.GetVersion()), version); if (txId) { UNIT_ASSERT_VALUES_EQUAL(actual.GetTxId(), txId); } diff --git a/ydb/core/tx/replication/controller/controller.cpp b/ydb/core/tx/replication/controller/controller.cpp index bdc7d5a97490..bcd818333451 100644 --- a/ydb/core/tx/replication/controller/controller.cpp +++ b/ydb/core/tx/replication/controller/controller.cpp @@ -791,7 +791,7 @@ void TController::Handle(TEvService::TEvHeartbeat::TPtr& ev, const TActorContext const auto& record = ev->Get()->Record; const auto id = TWorkerId::Parse(record.GetWorker()); - const auto version = TRowVersion::Parse(record.GetVersion()); + const auto version = TRowVersion::FromProto(record.GetVersion()); PendingHeartbeats[id] = version; RunTxHeartbeat(ctx); diff --git a/ydb/core/tx/replication/controller/dst_alterer.cpp b/ydb/core/tx/replication/controller/dst_alterer.cpp index 91febca0efff..f26da1fcf9b8 100644 --- a/ydb/core/tx/replication/controller/dst_alterer.cpp +++ b/ydb/core/tx/replication/controller/dst_alterer.cpp @@ -44,7 +44,7 @@ class TDstAlterer: public TActorBootstrapped { case TReplication::ETargetKind::Table: case TReplication::ETargetKind::IndexTable: tx.SetOperationType(NKikimrSchemeOp::ESchemeOpAlterTable); - PathIdFromPathId(DstPathId, tx.MutableAlterTable()->MutablePathId()); + DstPathId.ToProto(tx.MutableAlterTable()->MutablePathId()); tx.MutableAlterTable()->MutableReplicationConfig()->SetMode( NKikimrSchemeOp::TTableReplicationConfig::REPLICATION_MODE_NONE); break; diff --git a/ydb/core/tx/replication/controller/event_util.cpp b/ydb/core/tx/replication/controller/event_util.cpp index c65e064c79d6..e41546e737de 100644 --- a/ydb/core/tx/replication/controller/event_util.cpp +++ b/ydb/core/tx/replication/controller/event_util.cpp @@ -42,7 +42,7 @@ THolder MakeRunWorkerEv( readerSettings.SetConsumerName(ReplicationConsumerName); auto& writerSettings = *record.MutableCommand()->MutableLocalTableWriter(); - PathIdFromPathId(dstPathId, writerSettings.MutablePathId()); + dstPathId.ToProto(writerSettings.MutablePathId()); record.MutableCommand()->MutableConsistencySettings()->CopyFrom(consistencySettings); diff --git a/ydb/core/tx/replication/controller/tx_alter_replication.cpp b/ydb/core/tx/replication/controller/tx_alter_replication.cpp index 70efdb9b8c61..bf50c9548e95 100644 --- a/ydb/core/tx/replication/controller/tx_alter_replication.cpp +++ b/ydb/core/tx/replication/controller/tx_alter_replication.cpp @@ -26,7 +26,7 @@ class TController::TTxAlterReplication: public TTxBase { Result->Record.MutableOperationId()->CopyFrom(record.GetOperationId()); Result->Record.SetOrigin(Self->TabletID()); - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); Replication = Self->Find(pathId); if (!Replication) { diff --git a/ydb/core/tx/replication/controller/tx_assign_tx_id.cpp b/ydb/core/tx/replication/controller/tx_assign_tx_id.cpp index 1b4c3d1915e2..60727ec47460 100644 --- a/ydb/core/tx/replication/controller/tx_assign_tx_id.cpp +++ b/ydb/core/tx/replication/controller/tx_assign_tx_id.cpp @@ -49,7 +49,7 @@ class TController::TTxAssignTxId: public TTxBase { } auto& item = *ev->Record.AddVersionTxIds(); - it->first.Serialize(*item.MutableVersion()); + it->first.ToProto(item.MutableVersion()); item.SetTxId(txId); } diff --git a/ydb/core/tx/replication/controller/tx_create_replication.cpp b/ydb/core/tx/replication/controller/tx_create_replication.cpp index 0899c55c5e63..a477fc49bde6 100644 --- a/ydb/core/tx/replication/controller/tx_create_replication.cpp +++ b/ydb/core/tx/replication/controller/tx_create_replication.cpp @@ -26,7 +26,7 @@ class TController::TTxCreateReplication: public TTxBase { Result->Record.MutableOperationId()->CopyFrom(record.GetOperationId()); Result->Record.SetOrigin(Self->TabletID()); - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); if (Self->Find(pathId)) { CLOG_W(ctx, "Replication already exists" << ": pathId# " << pathId); diff --git a/ydb/core/tx/replication/controller/tx_describe_replication.cpp b/ydb/core/tx/replication/controller/tx_describe_replication.cpp index f78f1516445f..4ae68c709985 100644 --- a/ydb/core/tx/replication/controller/tx_describe_replication.cpp +++ b/ydb/core/tx/replication/controller/tx_describe_replication.cpp @@ -141,7 +141,7 @@ class TController::TTxDescribeReplication: public TTxBase { CLOG_D(ctx, "Execute: " << PubEv->Get()->ToString()); const auto& record = PubEv->Get()->Record; - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); Replication = Self->Find(pathId); if (!Replication) { diff --git a/ydb/core/tx/replication/controller/tx_drop_replication.cpp b/ydb/core/tx/replication/controller/tx_drop_replication.cpp index 0110680d8f01..31ffc2e34634 100644 --- a/ydb/core/tx/replication/controller/tx_drop_replication.cpp +++ b/ydb/core/tx/replication/controller/tx_drop_replication.cpp @@ -39,7 +39,7 @@ class TController::TTxDropReplication: public TTxBase { CLOG_D(ctx, "Execute: " << PubEv->Get()->ToString()); const auto& record = PubEv->Get()->Record; - const auto pathId = PathIdFromPathId(record.GetPathId()); + const auto pathId = TPathId::FromProto(record.GetPathId()); const auto& opId = record.GetOperationId(); Replication = Self->Find(pathId); diff --git a/ydb/core/tx/replication/service/base_table_writer.cpp b/ydb/core/tx/replication/service/base_table_writer.cpp index 159b40584842..09f66c2619a7 100644 --- a/ydb/core/tx/replication/service/base_table_writer.cpp +++ b/ydb/core/tx/replication/service/base_table_writer.cpp @@ -474,7 +474,7 @@ class TLocalTableWriter TVector records; for (const auto& kv : ev->Get()->Record.GetVersionTxIds()) { - const auto version = TRowVersion::Parse(kv.GetVersion()); + const auto version = TRowVersion::FromProto(kv.GetVersion()); TxIds.emplace(version, kv.GetTxId()); for (auto it = PendingTxId.begin(); it != PendingTxId.end();) { diff --git a/ydb/core/tx/replication/service/service.cpp b/ydb/core/tx/replication/service/service.cpp index a8ff4190941e..27b1b9056843 100644 --- a/ydb/core/tx/replication/service/service.cpp +++ b/ydb/core/tx/replication/service/service.cpp @@ -281,7 +281,7 @@ class TReplicationService: public TActorBootstrapped { const auto mode = consistencySettings.HasGlobal() ? EWriteMode::Consistent : EWriteMode::Simple; - return [tablePathId = PathIdFromPathId(writerSettings.GetPathId()), mode]() { + return [tablePathId = TPathId::FromProto(writerSettings.GetPathId()), mode]() { return CreateLocalTableWriter(tablePathId, mode); }; } @@ -370,7 +370,7 @@ class TReplicationService: public TActorBootstrapped { for (const auto& [version, txId] : result) { auto& item = *ev->Record.AddVersionTxIds(); - version.Serialize(*item.MutableVersion()); + version.ToProto(item.MutableVersion()); item.SetTxId(txId); } @@ -395,7 +395,7 @@ class TReplicationService: public TActorBootstrapped { TVector versionsWithoutTxId; for (const auto& v : ev->Get()->Record.GetVersions()) { - const auto version = TRowVersion::Parse(v); + const auto version = TRowVersion::FromProto(v); if (auto it = TxIds.upper_bound(version); it != TxIds.end()) { result[it->first] = it->second; } else { @@ -439,7 +439,7 @@ class TReplicationService: public TActorBootstrapped { THashMap> results; for (const auto& kv : record.GetVersionTxIds()) { - const auto version = TRowVersion::Parse(kv.GetVersion()); + const auto version = TRowVersion::FromProto(kv.GetVersion()); TxIds.emplace(version, kv.GetTxId()); for (auto it = PendingTxId.begin(); it != PendingTxId.end();) { @@ -478,7 +478,7 @@ class TReplicationService: public TActorBootstrapped { LOG_I("Heartbeat" << ": worker# " << ev->Sender - << ", version# " << TRowVersion::Parse(record.GetVersion())); + << ", version# " << TRowVersion::FromProto(record.GetVersion())); session->GetWorkerId(ev->Sender).Serialize(*record.MutableWorker()); Send(ev->Forward(*session)); diff --git a/ydb/core/tx/replication/service/service.h b/ydb/core/tx/replication/service/service.h index 136f2dc00e0e..c8d764fbc6b1 100644 --- a/ydb/core/tx/replication/service/service.h +++ b/ydb/core/tx/replication/service/service.h @@ -88,7 +88,7 @@ struct TEvService { explicit TEvGetTxId(const TContainer& container) { Record.MutableVersions()->Reserve(container.size()); for (const auto& v : container) { - v.Serialize(*Record.AddVersions()); + v.ToProto(Record.AddVersions()); } } }; @@ -106,7 +106,7 @@ struct TEvService { TEvHeartbeat() = default; explicit TEvHeartbeat(const TRowVersion& version) { - version.Serialize(*Record.MutableVersion()); + version.ToProto(Record.MutableVersion()); } }; }; diff --git a/ydb/core/tx/replication/service/table_writer_ut.cpp b/ydb/core/tx/replication/service/table_writer_ut.cpp index e0a97f052735..236387d655e8 100644 --- a/ydb/core/tx/replication/service/table_writer_ut.cpp +++ b/ydb/core/tx/replication/service/table_writer_ut.cpp @@ -129,7 +129,7 @@ Y_UNIT_TEST_SUITE(LocalTableWriter) { for (const auto& [version, txId] : result) { auto& item = *ev->Record.AddVersionTxIds(); - version.Serialize(*item.MutableVersion()); + version.ToProto(item.MutableVersion()); item.SetTxId(txId); } @@ -168,7 +168,7 @@ Y_UNIT_TEST_SUITE(LocalTableWriter) { UNIT_ASSERT_VALUES_EQUAL(versions.size(), 3); for (int i = 0; i < versions.size(); ++i) { - UNIT_ASSERT_VALUES_EQUAL(TRowVersion::Parse(versions[i]), TRowVersion(i + 1, 0)); + UNIT_ASSERT_VALUES_EQUAL(TRowVersion::FromProto(versions[i]), TRowVersion(i + 1, 0)); } } { @@ -180,7 +180,7 @@ Y_UNIT_TEST_SUITE(LocalTableWriter) { auto ev = env.Send(writer, new TEvWorker::TEvData("TestSource", { TRecord(order++, R"({"resolved":[10,0]})"), })); - UNIT_ASSERT_VALUES_EQUAL(TRowVersion::Parse(ev->Get()->Record.GetVersion()), TRowVersion(10, 0)); + UNIT_ASSERT_VALUES_EQUAL(TRowVersion::FromProto(ev->Get()->Record.GetVersion()), TRowVersion(10, 0)); } env.Send(writer, new TEvWorker::TEvData("TestSource", { diff --git a/ydb/core/tx/scheme_board/events.cpp b/ydb/core/tx/scheme_board/events.cpp index 7e753393b895..452d1ab90157 100644 --- a/ydb/core/tx/scheme_board/events.cpp +++ b/ydb/core/tx/scheme_board/events.cpp @@ -38,7 +38,7 @@ TOpaquePathDescription MakeOpaquePathDescription(::NKikimrSchemeBoard::TEvUpdate .PathId = TPathId(update.GetPathOwnerId(), update.GetLocalPathId()), .Path = update.GetPath(), .PathVersion = update.GetPathDirEntryPathVersion(), - .SubdomainPathId = PathIdFromPathId(update.GetPathSubdomainPathId()), + .SubdomainPathId = TPathId::FromProto(update.GetPathSubdomainPathId()), .PathAbandonedTenantsSchemeShards = TSet( update.GetPathAbandonedTenantsSchemeShards().begin(), update.GetPathAbandonedTenantsSchemeShards().end() @@ -102,7 +102,7 @@ NInternalEvents::TEvUpdateBuilder::TEvUpdateBuilder( Record.SetLocalPathId(pathDescription.PathId.LocalPathId); Record.SetPathDirEntryPathVersion(pathDescription.PathVersion); - PathIdFromPathId(pathDescription.SubdomainPathId, Record.MutablePathSubdomainPathId()); + pathDescription.SubdomainPathId.ToProto(Record.MutablePathSubdomainPathId()); Record.MutablePathAbandonedTenantsSchemeShards()->Assign( pathDescription.PathAbandonedTenantsSchemeShards.begin(), @@ -141,7 +141,7 @@ NInternalEvents::TEvNotifyBuilder::TEvNotifyBuilder(const TString& path, const T void NInternalEvents::TEvNotifyBuilder::SetPathDescription(const TOpaquePathDescription& pathDescription) { Record.SetDescribeSchemeResultSerialized(pathDescription.DescribeSchemeResultSerialized); - PathIdFromPathId(pathDescription.SubdomainPathId, Record.MutablePathSubdomainPathId()); + pathDescription.SubdomainPathId.ToProto(Record.MutablePathSubdomainPathId()); Record.MutablePathAbandonedTenantsSchemeShards()->Assign( pathDescription.PathAbandonedTenantsSchemeShards.begin(), pathDescription.PathAbandonedTenantsSchemeShards.end() diff --git a/ydb/core/tx/scheme_board/helpers.cpp b/ydb/core/tx/scheme_board/helpers.cpp index 6e80348b8a7a..c4e5326a453f 100644 --- a/ydb/core/tx/scheme_board/helpers.cpp +++ b/ydb/core/tx/scheme_board/helpers.cpp @@ -109,7 +109,7 @@ ui64 GetPathVersion(const NKikimrSchemeBoard::TEvNotify& record) { } NSchemeBoard::TDomainId GetDomainId(const NKikimrSchemeBoard::TEvNotify& record) { - return PathIdFromPathId(record.GetPathSubdomainPathId()); + return TPathId::FromProto(record.GetPathSubdomainPathId()); } TSet GetAbandonedSchemeShardIds(const NKikimrSchemeBoard::TEvNotify& record) { diff --git a/ydb/core/tx/scheme_board/replica_ut.cpp b/ydb/core/tx/scheme_board/replica_ut.cpp index db910e6b5761..bc6e03e695e3 100644 --- a/ydb/core/tx/scheme_board/replica_ut.cpp +++ b/ydb/core/tx/scheme_board/replica_ut.cpp @@ -680,7 +680,7 @@ void TReplicaCombinationTest::UpdatesCombinationsDomainRoot() { UNIT_ASSERT(ev->Get()->GetRecord().HasDescribeSchemeResultSerialized()); UNIT_ASSERT(ev->Get()->GetRecord().HasPathSubdomainPathId()); - UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId())); + UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId())); } } } @@ -740,7 +740,7 @@ void TReplicaCombinationTest::MigratedPathRecreation() { UNIT_ASSERT(ev->Get()->GetRecord().HasDescribeSchemeResultSerialized()); UNIT_ASSERT(ev->Get()->GetRecord().HasPathSubdomainPathId()); - UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId())); + UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId())); } void TReplicaCombinationTest::UpdatesCombinationsMigratedPath() { @@ -786,7 +786,7 @@ void TReplicaCombinationTest::UpdatesCombinationsMigratedPath() { << " PathID: " << TPathId(ev->Get()->GetRecord().GetPathOwnerId(), ev->Get()->GetRecord().GetLocalPathId()) << " deleted: " << ev->Get()->GetRecord().GetIsDeletion() << " version: " << ev->Get()->GetRecord().GetVersion() - << " domainId: " << PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId()) + << " domainId: " << TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId()) << Endl; if (argsLeft.PathId == TPathId(gssID, gssLocalId) && !argsLeft.IsDeletion && argsLeft.DomainId == TPathId(gssID, 2) @@ -810,7 +810,7 @@ void TReplicaCombinationTest::UpdatesCombinationsMigratedPath() { UNIT_ASSERT(ev->Get()->GetRecord().HasDescribeSchemeResultSerialized()); UNIT_ASSERT(ev->Get()->GetRecord().HasPathSubdomainPathId()); - UNIT_ASSERT_VALUES_EQUAL(std::get<0>(argsRight.GetSuperId()), PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId())); + UNIT_ASSERT_VALUES_EQUAL(std::get<0>(argsRight.GetSuperId()), TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId())); continue; } @@ -829,7 +829,7 @@ void TReplicaCombinationTest::UpdatesCombinationsMigratedPath() { UNIT_ASSERT(ev->Get()->GetRecord().HasDescribeSchemeResultSerialized()); UNIT_ASSERT(ev->Get()->GetRecord().HasPathSubdomainPathId()); - UNIT_ASSERT_VALUES_EQUAL(std::get<0>(argsLeft.GetSuperId()), PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId())); + UNIT_ASSERT_VALUES_EQUAL(std::get<0>(argsLeft.GetSuperId()), TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId())); continue; } @@ -879,7 +879,7 @@ void TReplicaCombinationTest::UpdatesCombinationsMigratedPath() { UNIT_ASSERT(ev->Get()->GetRecord().HasDescribeSchemeResultSerialized()); UNIT_ASSERT(ev->Get()->GetRecord().HasPathSubdomainPathId()); - UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), PathIdFromPathId(ev->Get()->GetRecord().GetPathSubdomainPathId())); + UNIT_ASSERT_VALUES_EQUAL(std::get<0>(winId), TPathId::FromProto(ev->Get()->GetRecord().GetPathSubdomainPathId())); } } } diff --git a/ydb/core/tx/scheme_board/subscriber.cpp b/ydb/core/tx/scheme_board/subscriber.cpp index 01dcb640e4fb..58b97e1a74fe 100644 --- a/ydb/core/tx/scheme_board/subscriber.cpp +++ b/ydb/core/tx/scheme_board/subscriber.cpp @@ -147,7 +147,7 @@ namespace { // Sender implementation is as recent as ours. // Just copy two fields from the notify message (this branch is practically a stub) - auto subdomainPathId = PathIdFromPathId(record.GetPathSubdomainPathId()); + auto subdomainPathId = TPathId::FromProto(record.GetPathSubdomainPathId()); auto pathAbandonedTenantsSchemeShards = TSet( record.GetPathAbandonedTenantsSchemeShards().begin(), record.GetPathAbandonedTenantsSchemeShards().end() diff --git a/ydb/core/tx/schemeshard/olap/table/table.h b/ydb/core/tx/schemeshard/olap/table/table.h index 8a4d665d6fc3..8bb0549be91d 100644 --- a/ydb/core/tx/schemeshard/olap/table/table.h +++ b/ydb/core/tx/schemeshard/olap/table/table.h @@ -23,7 +23,7 @@ struct TColumnTableInfo { TPathId GetOlapStorePathIdVerified() const { AFL_VERIFY(!IsStandalone()); - return PathIdFromPathId(Description.GetColumnStorePathId()); + return TPathId::FromProto(Description.GetColumnStorePathId()); } std::shared_ptr GetShardingVerified(const TOlapSchema& olapSchema) const { @@ -119,4 +119,4 @@ struct TColumnTableInfo { TConclusion> BuildEvolution(const TPathId& pathId, const NOlap::NAlter::TEvolutionInitializationContext& iContext) const; }; -} \ No newline at end of file +} diff --git a/ydb/core/tx/schemeshard/schemeshard__init.cpp b/ydb/core/tx/schemeshard/schemeshard__init.cpp index 019d6e570c9f..10db6d9c49f6 100644 --- a/ydb/core/tx/schemeshard/schemeshard__init.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__init.cpp @@ -3548,7 +3548,7 @@ struct TSchemeShard::TTxInit : public TTransactionBase { NKikimrSchemeOp::TGenericTxInFlyExtraData proto; bool deserializeRes = ParseFromStringNoSizeLimit(proto, extraData); Y_ABORT_UNLESS(deserializeRes); - txState.CdcPathId = PathIdFromPathId(proto.GetTxCopyTableExtraData().GetCdcPathId()); + txState.CdcPathId = TPathId::FromProto(proto.GetTxCopyTableExtraData().GetCdcPathId()); } } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_cdc_stream.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_cdc_stream.cpp index d1bf336a26c1..7d3d9ccbf76b 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_cdc_stream.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_cdc_stream.cpp @@ -240,7 +240,7 @@ class TConfigurePartsAtTable: public NCdcStreamState::TConfigurePartsAtTable { auto table = context.SS->Tables.at(pathId); auto& notice = *tx.MutableAlterCdcStreamNotice(); - PathIdFromPathId(pathId, notice.MutablePathId()); + pathId.ToProto(notice.MutablePathId()); notice.SetTableSchemaVersion(table->AlterVersion + 1); bool found = false; @@ -256,7 +256,7 @@ class TConfigurePartsAtTable: public NCdcStreamState::TConfigurePartsAtTable { auto stream = context.SS->CdcStreams.at(childPathId); Y_VERIFY_S(!found, "Too many cdc streams are planned to alter" - << ": found# " << PathIdFromPathId(notice.GetStreamDescription().GetPathId()) + << ": found# " << TPathId::FromProto(notice.GetStreamDescription().GetPathId()) << ", another# " << childPathId); found = true; diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_data_source.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_data_source.cpp index e80fe117bdd7..4c5358c7abf9 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_data_source.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_data_source.cpp @@ -244,7 +244,7 @@ class TAlterExternalDataSource : public TSubOperation { { bool isTieredStorage = false; for (const auto& referrer : externalDataSourceInfo->ExternalTableReferences.GetReferences()) { - if (TPath::Init(PathIdFromPathId(referrer.GetPathId()), context.SS)->PathType == + if (TPath::Init(TPathId::FromProto(referrer.GetPathId()), context.SS)->PathType == NKikimrSchemeOp::EPathType::EPathTypeColumnTable) { isTieredStorage = true; break; diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_table.cpp index 54f63ca0db57..941b279a6a8c 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_external_table.cpp @@ -252,12 +252,12 @@ class TAlterExternalTable: public TSubOperation { if (!isSameDataSource) { auto& reference = *externalDataSource->ExternalTableReferences.AddReferences(); reference.SetPath(dstPath.PathString()); - PathIdFromPathId(externalTable->PathId, reference.MutablePathId()); + externalTable->PathId.ToProto(reference.MutablePathId()); EraseIf(*oldDataSource->ExternalTableReferences.MutableReferences(), [pathId = externalTable->PathId]( const NKikimrSchemeOp::TExternalTableReferences::TReference& reference) { - return PathIdFromPathId(reference.GetPathId()) == pathId; + return TPathId::FromProto(reference.GetPathId()) == pathId; }); } } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_pq.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_pq.cpp index 8483dcf4b4d8..a05fcadf3096 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_pq.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_pq.cpp @@ -214,7 +214,7 @@ class TAlterPQ: public TSubOperation { if (alterConfig.HasOffloadConfig()) { // TODO: check validity auto* pathId = alterConfig.MutableOffloadConfig()->MutableIncrementalBackup()->MutableDstPathId(); - PathIdFromPathId(TPath::Resolve(alterConfig.GetOffloadConfig().GetIncrementalBackup().GetDstPath(), context.SS).Base()->PathId, pathId); + TPath::Resolve(alterConfig.GetOffloadConfig().GetIncrementalBackup().GetDstPath(), context.SS).Base()->PathId.ToProto(pathId); } alterConfig.MutablePartitionKeySchema()->Swap(tabletConfig->MutablePartitionKeySchema()); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_replication.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_replication.cpp index a9cfccc09d2f..f9b28f18e144 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_replication.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_replication.cpp @@ -75,7 +75,7 @@ class TConfigureParts: public TSubOperationState { context.OnComplete.WaitShardCreated(shard.Idx, OperationId); } else { auto ev = MakeHolder(); - PathIdFromPathId(pathId, ev->Record.MutablePathId()); + pathId.ToProto(ev->Record.MutablePathId()); ev->Record.MutableOperationId()->SetTxId(ui64(OperationId.GetTxId())); ev->Record.MutableOperationId()->SetPartId(ui32(OperationId.GetSubTxId())); ev->Record.MutableConfig()->CopyFrom(alterData->Description.GetConfig()); @@ -314,7 +314,7 @@ class TAlterReplication: public TSubOperation { const auto& op = Transaction.GetAlterReplication(); const auto& name = op.GetName(); const auto pathId = op.HasPathId() - ? PathIdFromPathId(op.GetPathId()) + ? TPathId::FromProto(op.GetPathId()) : InvalidPathId; LOG_N("TAlterReplication Propose" diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_table.cpp index 23f0df7d6296..06c6703ede4e 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_table.cpp @@ -500,7 +500,7 @@ class TAlterTable: public TSubOperation { TPathId pathId; if (alter.HasId_Deprecated() || alter.HasPathId()) { pathId = alter.HasPathId() - ? PathIdFromPathId(alter.GetPathId()) + ? TPathId::FromProto(alter.GetPathId()) : context.SS->MakeLocalId(alter.GetId_Deprecated()); } @@ -712,7 +712,7 @@ TVector CreateConsistentAlterTable(TOperationId id, const T const TString& parentPathStr = tx.GetWorkingDir(); const TString& name = alter.GetName(); - TPathId pathId = alter.HasPathId() ? PathIdFromPathId(alter.GetPathId()) : InvalidPathId; + TPathId pathId = alter.HasPathId() ? TPathId::FromProto(alter.GetPathId()) : InvalidPathId; if (!alter.HasName() && !pathId) { return {CreateAlterTable(id, tx)}; diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_apply_build_index.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_apply_build_index.cpp index 436cb33a2978..06a5c1fa3e66 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_apply_build_index.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_apply_build_index.cpp @@ -73,7 +73,7 @@ TVector ApplyBuildIndex(TOperationId nextId, const TTxTrans op->SetBuildIndexId(config.GetBuildIndexId()); if (!indexName.empty()) { TPath index = table.Child(indexName); - PathIdFromPathId(index.Base()->PathId, op->MutableOutcome()->MutableApply()->MutableIndexPathId()); + index.Base()->PathId.ToProto(op->MutableOutcome()->MutableApply()->MutableIndexPathId()); } result.push_back(CreateFinalizeBuildIndexMainTable(NextPartId(nextId, result), finalize)); @@ -132,7 +132,7 @@ TVector CancelBuildIndex(TOperationId nextId, const TTxTran if (!indexName.empty()) { TPath index = table.Child(indexName); - PathIdFromPathId(index.Base()->PathId, op->MutableOutcome()->MutableCancel()->MutableIndexPathId()); + index.Base()->PathId.ToProto(op->MutableOutcome()->MutableCancel()->MutableIndexPathId()); } result.push_back(CreateFinalizeBuildIndexMainTable(NextPartId(nextId, result), finalize)); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_external_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_external_table.cpp index f44b3fde254f..1663505e9e6b 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_external_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_external_table.cpp @@ -260,7 +260,7 @@ class TCreateExternalTable: public TSubOperation { const TPath& dstPath) { auto& reference = *externalDataSource->ExternalTableReferences.AddReferences(); reference.SetPath(dstPath.PathString()); - PathIdFromPathId(externalTable->PathId, reference.MutablePathId()); + externalTable->PathId.ToProto(reference.MutablePathId()); } void PersistExternalTable( diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_replication.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_replication.cpp index 7fb6fde919d1..0b7c9b289424 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_replication.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_replication.cpp @@ -102,7 +102,7 @@ class TConfigureParts: public TSubOperationState { context.OnComplete.WaitShardCreated(shard.Idx, OperationId); } else { auto ev = MakeHolder(); - PathIdFromPathId(pathId, ev->Record.MutablePathId()); + pathId.ToProto(ev->Record.MutablePathId()); ev->Record.MutableOperationId()->SetTxId(ui64(OperationId.GetTxId())); ev->Record.MutableOperationId()->SetPartId(ui32(OperationId.GetSubTxId())); ev->Record.MutableConfig()->CopyFrom(alterData->Description.GetConfig()); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_restore_incremental_backup.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_restore_incremental_backup.cpp index a5865c42abba..0be64ae573ec 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_restore_incremental_backup.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_restore_incremental_backup.cpp @@ -70,7 +70,7 @@ class TConfigurePartsAtTable : public TSubOperationState { auto& op = *tx.MutableCreateIncrementalRestoreSrc(); op.MutableSrcPathId()->CopyFrom(RestoreOp.GetSrcPathIds(LoopStep)); op.SetSrcTablePath(RestoreOp.GetSrcTablePaths(LoopStep)); - PathIdFromPathId(pathId, op.MutableDstPathId()); + pathId.ToProto(op.MutableDstPathId()); op.SetDstTablePath(RestoreOp.GetDstTablePath()); } @@ -263,10 +263,10 @@ class TDone: public TSubOperationState { Y_ABORT_UNLESS(txState); Y_ABORT_UNLESS(IsExpectedTxType(txState->TxType)); Y_ABORT_UNLESS(txState->LoopStep == RestoreOp.SrcPathIdsSize()); - Y_ABORT_UNLESS(txState->TargetPathId == PathIdFromPathId(RestoreOp.GetSrcPathIds(RestoreOp.SrcPathIdsSize() - 1))); + Y_ABORT_UNLESS(txState->TargetPathId == TPathId::FromProto(RestoreOp.GetSrcPathIds(RestoreOp.SrcPathIdsSize() - 1))); for (const auto& pathId : RestoreOp.GetSrcPathIds()) { - context.OnComplete.ReleasePathState(OperationId, PathIdFromPathId(pathId), TPathElement::EPathState::EPathStateNoChanges); + context.OnComplete.ReleasePathState(OperationId, TPathId::FromProto(pathId), TPathElement::EPathState::EPathStateNoChanges); } context.OnComplete.DoneOperation(OperationId); @@ -355,7 +355,7 @@ class TNewRestoreFromAtTable : public TSubOperation { Y_ABORT_UNLESS(txState); ++(txState->LoopStep); if (txState->LoopStep < Transaction.GetRestoreMultipleIncrementalBackups().SrcPathIdsSize()) { - txState->TargetPathId = PathIdFromPathId(Transaction.GetRestoreMultipleIncrementalBackups().GetSrcPathIds(txState->LoopStep)); + txState->TargetPathId = TPathId::FromProto(Transaction.GetRestoreMultipleIncrementalBackups().GetSrcPathIds(txState->LoopStep)); txState->TxShardsListFinalized = false; // TODO preserve TxState return TTxState::ConfigureParts; @@ -655,7 +655,7 @@ bool CreateRestoreMultipleIncrementalBackups( for (const auto& srcTablePath : srcPaths) { restoreOp.AddSrcTablePaths(srcTablePath.PathString()); - PathIdFromPathId(srcTablePath.Base()->PathId, restoreOp.AddSrcPathIds()); + srcTablePath.Base()->PathId.ToProto(restoreOp.AddSrcPathIds()); } result.push_back(CreateRestoreIncrementalBackupAtTable(NextPartId(opId, result), outTx)); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_drop_cdc_stream.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_drop_cdc_stream.cpp index 9e83fd17836d..98e26d7e0098 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_drop_cdc_stream.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_drop_cdc_stream.cpp @@ -210,7 +210,7 @@ class TConfigurePartsAtTable: public NCdcStreamState::TConfigurePartsAtTable { auto table = context.SS->Tables.at(pathId); auto& notice = *tx.MutableDropCdcStreamNotice(); - PathIdFromPathId(pathId, notice.MutablePathId()); + pathId.ToProto(notice.MutablePathId()); notice.SetTableSchemaVersion(table->AlterVersion + 1); bool found = false; @@ -223,11 +223,11 @@ class TConfigurePartsAtTable: public NCdcStreamState::TConfigurePartsAtTable { } Y_VERIFY_S(!found, "Too many cdc streams are planned to drop" - << ": found# " << PathIdFromPathId(notice.GetStreamPathId()) + << ": found# " << TPathId::FromProto(notice.GetStreamPathId()) << ", another# " << childPathId); found = true; - PathIdFromPathId(childPathId, notice.MutableStreamPathId()); + childPathId.ToProto(notice.MutableStreamPathId()); } } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_drop_external_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_drop_external_table.cpp index a6b152b06181..df6e6917438a 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_drop_external_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_drop_external_table.cpp @@ -58,7 +58,7 @@ class TPropose: public TSubOperationState { TExternalDataSourceInfo::TPtr externalDataSourceInfo = context.SS->ExternalDataSources.Value(dataSourcePathId, nullptr); Y_ABORT_UNLESS(externalDataSourceInfo); - EraseIf(*externalDataSourceInfo->ExternalTableReferences.MutableReferences(), [pathId](const NKikimrSchemeOp::TExternalTableReferences::TReference& reference) { return PathIdFromPathId(reference.GetPathId()) == pathId; }); + EraseIf(*externalDataSourceInfo->ExternalTableReferences.MutableReferences(), [pathId](const NKikimrSchemeOp::TExternalTableReferences::TReference& reference) { return TPathId::FromProto(reference.GetPathId()) == pathId; }); context.SS->TabletCounters->Simple()[COUNTER_EXTERNAL_TABLE_COUNT].Sub(1); context.SS->PersistExternalDataSource(db, dataSourcePathId, externalDataSourceInfo); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_drop_index.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_drop_index.cpp index 6a7e41df53c0..ba790fecbfc5 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_drop_index.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_drop_index.cpp @@ -85,7 +85,7 @@ class TConfigureParts: public TSubOperationState { context.SS->FillSeqNo(tx, seqNo); auto notice = tx.MutableDropIndexNotice(); - PathIdFromPathId(pathId, notice->MutablePathId()); + pathId.ToProto(notice->MutablePathId()); notice->SetTableSchemaVersion(table->AlterVersion + 1); bool found = false; @@ -98,11 +98,11 @@ class TConfigureParts: public TSubOperationState { } Y_VERIFY_S(!found, "Too many indexes are planned to drop" - << ": found# " << PathIdFromPathId(notice->GetIndexPathId()) + << ": found# " << TPathId::FromProto(notice->GetIndexPathId()) << ", another# " << childPathId); found = true; - PathIdFromPathId(childPathId, notice->MutableIndexPathId()); + childPathId.ToProto(notice->MutableIndexPathId()); } Y_PROTOBUF_SUPPRESS_NODISCARD tx.SerializeToString(&txBody); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_drop_replication.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_drop_replication.cpp index da123edc9e2e..41e48e6cf8e0 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_drop_replication.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_drop_replication.cpp @@ -73,7 +73,7 @@ class TDropParts: public TSubOperationState { const auto tabletId = context.SS->ShardInfos.at(shard.Idx).TabletID; auto ev = MakeHolder(); - PathIdFromPathId(pathId, ev->Record.MutablePathId()); + pathId.ToProto(ev->Record.MutablePathId()); ev->Record.MutableOperationId()->SetTxId(ui64(OperationId.GetTxId())); ev->Record.MutableOperationId()->SetPartId(ui32(OperationId.GetSubTxId())); ev->Record.SetCascade(txState->TxType == TTxState::TxDropReplicationCascade); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_drop_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_drop_table.cpp index 8408bdeb4c65..f37689e528c6 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_drop_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_drop_table.cpp @@ -125,7 +125,7 @@ class TDropParts: public TSubOperationState { NKikimrTxDataShard::TFlatSchemeTransaction tx; context.SS->FillSeqNo(tx, seqNo); tx.MutableDropTable()->SetId_Deprecated(pathId.LocalPathId); - PathIdFromPathId(pathId, tx.MutableDropTable()->MutablePathId()); + pathId.ToProto(tx.MutableDropTable()->MutablePathId()); tx.MutableDropTable()->SetName(path->Name); Y_PROTOBUF_SUPPRESS_NODISCARD tx.SerializeToString(&txBody); } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_finalize_build_index.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_finalize_build_index.cpp index ee480d43d30e..2fcfc98c9b87 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_finalize_build_index.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_finalize_build_index.cpp @@ -69,7 +69,7 @@ class TConfigureParts: public TSubOperationState { NKikimrTxDataShard::TFlatSchemeTransaction tx; auto* op = tx.MutableFinalizeBuildIndex(); - PathIdFromPathId(pathId, op->MutablePathId()); + pathId.ToProto(op->MutablePathId()); op->SetSnapshotTxId(ui64(snapshotTxId)); op->SetSnapshotStep(ui64(snapshotStepId)); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_initiate_build_index.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_initiate_build_index.cpp index a00da4df77dd..51adf5c7fbd5 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_initiate_build_index.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_initiate_build_index.cpp @@ -61,7 +61,7 @@ class TConfigureParts: public TSubOperationState { NKikimrTxDataShard::TFlatSchemeTransaction txTemplate; auto initiate = txTemplate.MutableInitiateBuildIndex(); - PathIdFromPathId(pathId, initiate->MutablePathId()); + pathId.ToProto(initiate->MutablePathId()); initiate->SetSnapshotName("Snapshot0"); initiate->SetTableSchemaVersion(tableInfo->AlterVersion + 1); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_move_index.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_move_index.cpp index 41a9264eff5d..32d07c98dbd7 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_move_index.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_move_index.cpp @@ -87,7 +87,7 @@ class TConfigureParts: public TSubOperationState { context.SS->FillSeqNo(tx, seqNo); auto notice = tx.MutableMoveIndex(); - PathIdFromPathId(pathId, notice->MutablePathId()); + pathId.ToProto(notice->MutablePathId()); notice->SetTableSchemaVersion(table->AlterVersion + 1); auto remap = notice->MutableReMapIndex(); @@ -116,8 +116,8 @@ class TConfigureParts: public TSubOperationState { << ", type: " << (int)txState->TxType << ", parent pathId: " << pathId); if (pathId == parent.Base()->PathId) { - PathIdFromPathId(txState->SourcePathId, remap->MutableSrcPathId()); - PathIdFromPathId(txState->TargetPathId, remap->MutableDstPathId()); + txState->SourcePathId.ToProto(remap->MutableSrcPathId()); + txState->TargetPathId.ToProto(remap->MutableDstPathId()); auto targetIndexName = context.SS->PathsById.at(txState->TargetPathId); for (const auto& [_, childPathId] : path->GetChildren()) { @@ -125,7 +125,7 @@ class TConfigureParts: public TSubOperationState { auto childPath = context.SS->PathsById.at(childPathId); if (childPath->Name == targetIndexName->Name) { - PathIdFromPathId(childPathId, remap->MutableReplacedPathId()); + childPathId.ToProto(remap->MutableReplacedPathId()); remap->SetDstName(childPath->Name); } } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_move_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_move_table.cpp index 654d6e5b6868..f51bdcc97667 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_move_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_move_table.cpp @@ -91,10 +91,10 @@ class TConfigureParts: public TSubOperationState { NKikimrTxDataShard::TFlatSchemeTransaction tx; context.SS->FillSeqNo(tx, seqNo); auto move = tx.MutableMoveTable(); - PathIdFromPathId(srcPath->PathId, move->MutablePathId()); + srcPath->PathId.ToProto(move->MutablePathId()); move->SetTableSchemaVersion(srcTable->AlterVersion+1); - PathIdFromPathId(dstPath->PathId, move->MutableDstPathId()); + dstPath->PathId.ToProto(move->MutableDstPathId()); move->SetDstPath(TPath::Init(dstPath->PathId, context.SS).PathString()); for (const auto& child: srcPath->GetChildren()) { @@ -114,8 +114,8 @@ class TConfigureParts: public TSubOperationState { Y_ABORT_UNLESS(dstIndexPath.IsResolved()); auto remap = move->AddReMapIndexes(); - PathIdFromPathId(srcChildPath->PathId, remap->MutableSrcPathId()); - PathIdFromPathId(dstIndexPath->PathId, remap->MutableDstPathId()); + srcChildPath->PathId.ToProto(remap->MutableSrcPathId()); + dstIndexPath->PathId.ToProto(remap->MutableDstPathId()); } Y_PROTOBUF_SUPPRESS_NODISCARD tx.SerializeToString(&txBody); diff --git a/ydb/core/tx/schemeshard/schemeshard_build_index__progress.cpp b/ydb/core/tx/schemeshard/schemeshard_build_index__progress.cpp index 3619f41dbd42..37e55b99b84d 100644 --- a/ydb/core/tx/schemeshard/schemeshard_build_index__progress.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_build_index__progress.cpp @@ -524,10 +524,10 @@ struct TSchemeShard::TIndexBuilder::TTxProgress: public TSchemeShard::TIndexBuil ev->Record.SetId(ui64(BuildId)); if (buildInfo.KMeans.Parent == 0) { - PathIdFromPathId(buildInfo.TablePathId, ev->Record.MutablePathId()); + buildInfo.TablePathId.ToProto(ev->Record.MutablePathId()); } else { auto path = TPath::Init(buildInfo.TablePathId, Self).Dive(buildInfo.IndexName); - PathIdFromPathId(path.Dive(buildInfo.KMeans.ReadFrom())->PathId, ev->Record.MutablePathId()); + path.Dive(buildInfo.KMeans.ReadFrom())->PathId.ToProto(ev->Record.MutablePathId()); } ev->Record.SetK(buildInfo.KMeans.K); @@ -555,9 +555,9 @@ struct TSchemeShard::TIndexBuilder::TTxProgress: public TSchemeShard::TIndexBuil auto path = TPath::Init(buildInfo.TablePathId, Self).Dive(buildInfo.IndexName); if (buildInfo.KMeans.Parent == 0) { - PathIdFromPathId(buildInfo.TablePathId, ev->Record.MutablePathId()); + buildInfo.TablePathId.ToProto(ev->Record.MutablePathId()); } else { - PathIdFromPathId(path.Dive(buildInfo.KMeans.ReadFrom())->PathId, ev->Record.MutablePathId()); + path.Dive(buildInfo.KMeans.ReadFrom())->PathId.ToProto(ev->Record.MutablePathId()); path.Rise(); } @@ -593,9 +593,9 @@ struct TSchemeShard::TIndexBuilder::TTxProgress: public TSchemeShard::TIndexBuil auto path = TPath::Init(buildInfo.TablePathId, Self).Dive(buildInfo.IndexName); if (buildInfo.KMeans.Parent == 0) { - PathIdFromPathId(buildInfo.TablePathId, ev->Record.MutablePathId()); + buildInfo.TablePathId.ToProto(ev->Record.MutablePathId()); } else { - PathIdFromPathId(path.Dive(buildInfo.KMeans.ReadFrom())->PathId, ev->Record.MutablePathId()); + path.Dive(buildInfo.KMeans.ReadFrom())->PathId.ToProto(ev->Record.MutablePathId()); path.Rise(); } *ev->Record.MutableSettings() = std::get( diff --git a/ydb/core/tx/schemeshard/schemeshard_cdc_stream_common.cpp b/ydb/core/tx/schemeshard/schemeshard_cdc_stream_common.cpp index ca655bb69fd3..a51457651d25 100644 --- a/ydb/core/tx/schemeshard/schemeshard_cdc_stream_common.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_cdc_stream_common.cpp @@ -14,7 +14,7 @@ void FillNotice(const TPathId& pathId, TOperationContext& context, NKikimrTxData Y_ABORT_UNLESS(context.SS->Tables.contains(pathId)); auto table = context.SS->Tables.at(pathId); - PathIdFromPathId(pathId, notice.MutablePathId()); + pathId.ToProto(notice.MutablePathId()); notice.SetTableSchemaVersion(table->AlterVersion + 1); bool found = false; @@ -34,7 +34,7 @@ void FillNotice(const TPathId& pathId, TOperationContext& context, NKikimrTxData } Y_VERIFY_S(!found, "Too many cdc streams are planned to create" - << ": found# " << PathIdFromPathId(notice.GetStreamDescription().GetPathId()) + << ": found# " << TPathId::FromProto(notice.GetStreamDescription().GetPathId()) << ", another# " << childPathId); found = true; diff --git a/ydb/core/tx/schemeshard/schemeshard_cdc_stream_scan.cpp b/ydb/core/tx/schemeshard/schemeshard_cdc_stream_scan.cpp index 78e75c112f96..c0476e4e3c13 100644 --- a/ydb/core/tx/schemeshard/schemeshard_cdc_stream_scan.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_cdc_stream_scan.cpp @@ -186,9 +186,9 @@ struct TSchemeShard::TCdcStreamScan::TTxProgress: public TTransactionBasePendingShards.erase(it); auto ev = MakeHolder(); - PathIdFromPathId(tablePathId, ev->Record.MutableTablePathId()); + tablePathId.ToProto(ev->Record.MutableTablePathId()); ev->Record.SetTableSchemaVersion(table->AlterVersion); - PathIdFromPathId(streamPathId, ev->Record.MutableStreamPathId()); + streamPathId.ToProto(ev->Record.MutableStreamPathId()); ev->Record.SetSnapshotStep(ui64(streamPath->StepCreated)); ev->Record.SetSnapshotTxId(ui64(streamPath->CreateTxId)); ScanRequests.emplace_back(streamPathId, tabletId, std::move(ev)); @@ -219,7 +219,7 @@ struct TSchemeShard::TCdcStreamScan::TTxProgress: public TTransactionBaseCdcStreams.contains(streamPathId)) { LOG_W("Cannot process response" << ": streamPathId# " << streamPathId diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.cpp b/ydb/core/tx/schemeshard/schemeshard_impl.cpp index da81487be941..79f8987d3f2d 100644 --- a/ydb/core/tx/schemeshard/schemeshard_impl.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_impl.cpp @@ -2386,7 +2386,7 @@ void TSchemeShard::PersistTxState(NIceDb::TNiceDb& db, const TOperationId opId) extraData = tableInfo->SerializeAlterExtraData(); } else if (txState.TxType == TTxState::TxCopyTable) { NKikimrSchemeOp::TGenericTxInFlyExtraData proto; - PathIdFromPathId(txState.CdcPathId, proto.MutableTxCopyTableExtraData()->MutableCdcPathId()); + txState.CdcPathId.ToProto(proto.MutableTxCopyTableExtraData()->MutableCdcPathId()); bool serializeRes = proto.SerializeToString(&extraData); Y_ABORT_UNLESS(serializeRes); } @@ -2999,7 +2999,7 @@ void TSchemeShard::PersistExternalDataSourceReference(NIceDb::TNiceDb& db, TPath Y_ABORT_UNLESS(findSource); auto* ref = (*findSource)->ExternalTableReferences.AddReferences(); ref->SetPath(referrer.PathString()); - PathIdFromPathId(referrer->PathId, ref->MutablePathId()); + referrer->PathId.ToProto(ref->MutablePathId()); db.Table() .Key(pathId.OwnerId, pathId.LocalPathId) .Update( @@ -3011,7 +3011,7 @@ void TSchemeShard::PersistRemoveExternalDataSourceReference(NIceDb::TNiceDb& db, Y_ABORT_UNLESS(findSource); EraseIf(*(*findSource)->ExternalTableReferences.MutableReferences(), [referrer](const NKikimrSchemeOp::TExternalTableReferences::TReference& reference) { - return PathIdFromPathId(reference.GetPathId()) == referrer; + return TPathId::FromProto(reference.GetPathId()) == referrer; }); db.Table() .Key(pathId.OwnerId, pathId.LocalPathId) @@ -6678,7 +6678,7 @@ TString TSchemeShard::FillAlterTableTxBody(TPathId pathId, TShardIdx shardIdx, T proto->SetName(path->Name); proto->SetId_Deprecated(pathId.LocalPathId); - PathIdFromPathId(pathId, proto->MutablePathId()); + pathId.ToProto(proto->MutablePathId()); for (const auto& col : alterData->Columns) { const TTableInfo::TColumn& colInfo = col.second; diff --git a/ydb/core/tx/schemeshard/schemeshard_info_types.cpp b/ydb/core/tx/schemeshard/schemeshard_info_types.cpp index d688835b71a3..55aa4dac5aca 100644 --- a/ydb/core/tx/schemeshard/schemeshard_info_types.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_info_types.cpp @@ -731,7 +731,7 @@ TVector TTableInfo::FillDescriptionCache(TPathElement::TPtr pathInfo) { if (!TableDescription.HasPathId()) { TableDescription.SetName(pathInfo->Name); TableDescription.SetId_Deprecated(pathInfo->PathId.LocalPathId); - PathIdFromPathId(pathInfo->PathId, TableDescription.MutablePathId()); + pathInfo->PathId.ToProto(TableDescription.MutablePathId()); for (auto& c : Columns) { const TColumn& column = c.second; diff --git a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp index bad67d7a8392..cd40e51a8a99 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp @@ -1025,7 +1025,7 @@ void TPathDescriber::DescribeExternalTable(const TActorContext& ctx, TPathId pat auto entry = Result->Record.MutablePathDescription()->MutableExternalTableDescription(); entry->SetName(pathEl->Name); - PathIdFromPathId(pathId, entry->MutablePathId()); + pathId.ToProto(entry->MutablePathId()); entry->SetSourceType(externalTableInfo->SourceType); entry->SetDataSourcePath(externalTableInfo->DataSourcePath); entry->SetLocation(externalTableInfo->Location); @@ -1058,7 +1058,7 @@ void TPathDescriber::DescribeExternalDataSource(const TActorContext&, TPathId pa auto entry = Result->Record.MutablePathDescription()->MutableExternalDataSourceDescription(); entry->SetName(pathEl->Name); - PathIdFromPathId(pathId, entry->MutablePathId()); + pathId.ToProto(entry->MutablePathId()); externalDataSourceInfo->FillProto(*entry); } @@ -1069,7 +1069,7 @@ void TPathDescriber::DescribeView(const TActorContext&, TPathId pathId, TPathEle auto entry = Result->Record.MutablePathDescription()->MutableViewDescription(); entry->SetName(pathEl->Name); - PathIdFromPathId(pathId, entry->MutablePathId()); + pathId.ToProto(entry->MutablePathId()); entry->SetVersion(viewInfo->AlterVersion); entry->SetQueryText(viewInfo->QueryText); *entry->MutableCapturedContext() = viewInfo->CapturedContext; @@ -1082,7 +1082,7 @@ void TPathDescriber::DescribeResourcePool(TPathId pathId, TPathElement::TPtr pat auto entry = Result->Record.MutablePathDescription()->MutableResourcePoolDescription(); entry->SetName(pathEl->Name); - PathIdFromPathId(pathId, entry->MutablePathId()); + pathId.ToProto(entry->MutablePathId()); entry->SetVersion(resourcePoolInfo->AlterVersion); entry->MutableProperties()->CopyFrom(resourcePoolInfo->Properties); } @@ -1094,7 +1094,7 @@ void TPathDescriber::DescribeBackupCollection(TPathId pathId, TPathElement::TPtr auto entry = Result->Record.MutablePathDescription()->MutableBackupCollectionDescription(); entry->SetName(pathEl->Name); - PathIdFromPathId(pathId, entry->MutablePathId()); + pathId.ToProto(entry->MutablePathId()); entry->SetVersion(backupCollectionInfo->AlterVersion); entry->CopyFrom(backupCollectionInfo->Description); } @@ -1427,7 +1427,7 @@ void TSchemeShard::DescribeCdcStream(const TPathId& pathId, const TString& name, desc.SetVirtualTimestamps(info->VirtualTimestamps); desc.SetResolvedTimestampsIntervalMs(info->ResolvedTimestamps.MilliSeconds()); desc.SetAwsRegion(info->AwsRegion); - PathIdFromPathId(pathId, desc.MutablePathId()); + pathId.ToProto(desc.MutablePathId()); desc.SetState(info->State); desc.SetSchemaVersion(info->AlterVersion); @@ -1471,7 +1471,7 @@ void TSchemeShard::DescribeSequence(const TPathId& pathId, const TString& name, } desc.SetName(name); - PathIdFromPathId(pathId, desc.MutablePathId()); + pathId.ToProto(desc.MutablePathId()); desc.SetVersion(info->AlterVersion); if (info->Sharding.SequenceShardsSize() > 0) { @@ -1525,7 +1525,7 @@ void TSchemeShard::DescribeReplication(const TPathId& pathId, const TString& nam ClearSensitiveFields(&desc); desc.SetName(name); - PathIdFromPathId(pathId, desc.MutablePathId()); + pathId.ToProto(desc.MutablePathId()); desc.SetVersion(info->AlterVersion); if (const auto& shardIdx = info->ControllerShardIdx; shardIdx != InvalidShardIdx) { @@ -1543,7 +1543,7 @@ void TSchemeShard::DescribeBlobDepot(const TPathId& pathId, const TString& name, Y_ABORT_UNLESS(it != BlobDepots.end()); desc = it->second->Description; desc.SetName(name); - PathIdFromPathId(pathId, desc.MutablePathId()); + pathId.ToProto(desc.MutablePathId()); desc.SetVersion(it->second->AlterVersion); desc.SetTabletId(static_cast(it->second->BlobDepotTabletId)); }