From ab423f8449379f7b10889a20a2334e072a26796c Mon Sep 17 00:00:00 2001 From: Innokentii Mokin Date: Tue, 24 Sep 2024 15:35:49 +0000 Subject: [PATCH] schemeshard thing --- ydb/core/protos/counters_schemeshard.proto | 4 + ydb/core/protos/flat_scheme_op.proto | 1 + .../tx/schemeshard/schemeshard__operation.cpp | 5 + .../schemeshard__operation_common.cpp | 2 + ...tion_create_restore_incremental_backup.cpp | 252 ++++++------------ .../schemeshard/schemeshard__operation_part.h | 3 + .../schemeshard_audit_log_fragment.cpp | 2 + ydb/core/tx/schemeshard/schemeshard_impl.cpp | 1 + .../tx/schemeshard/schemeshard_tx_infly.h | 4 + ydb/core/tx/tx_proxy/schemereq.cpp | 2 + 10 files changed, 106 insertions(+), 170 deletions(-) diff --git a/ydb/core/protos/counters_schemeshard.proto b/ydb/core/protos/counters_schemeshard.proto index 15f8bb5d7f2b..82ab38ce01db 100644 --- a/ydb/core/protos/counters_schemeshard.proto +++ b/ydb/core/protos/counters_schemeshard.proto @@ -220,6 +220,8 @@ enum ESimpleCounters { COUNTER_IN_FLIGHT_OPS_TxCreateResourcePool = 174 [(CounterOpts) = {Name: "InFlightOps/CreateResourcePool"}]; COUNTER_IN_FLIGHT_OPS_TxDropResourcePool = 175 [(CounterOpts) = {Name: "InFlightOps/DropResourcePool"}]; COUNTER_IN_FLIGHT_OPS_TxAlterResourcePool = 176 [(CounterOpts) = {Name: "InFlightOps/AlterResourcePool"}]; + + COUNTER_IN_FLIGHT_OPS_TxRestoreIncrementalBackupAtTable = 177 [(CounterOpts) = {Name: "InFlightOps/RestoreIncrementalBackupAtTable"}]; } enum ECumulativeCounters { @@ -351,6 +353,8 @@ enum ECumulativeCounters { COUNTER_FINISHED_OPS_TxCreateResourcePool = 103 [(CounterOpts) = {Name: "FinishedOps/CreateResourcePool"}]; COUNTER_FINISHED_OPS_TxDropResourcePool = 104 [(CounterOpts) = {Name: "FinishedOps/DropResourcePool"}]; COUNTER_FINISHED_OPS_TxAlterResourcePool = 105 [(CounterOpts) = {Name: "FinishedOps/AlterResourcePool"}]; + + COUNTER_FINISHED_OPS_TxRestoreIncrementalBackupAtTable = 106 [(CounterOpts) = {Name: "FinishedOps/RestoreIncrementalBackupAtTable"}]; } enum EPercentileCounters { diff --git a/ydb/core/protos/flat_scheme_op.proto b/ydb/core/protos/flat_scheme_op.proto index 38d9f0d589d8..b18f8f8f76cc 100644 --- a/ydb/core/protos/flat_scheme_op.proto +++ b/ydb/core/protos/flat_scheme_op.proto @@ -1664,6 +1664,7 @@ enum EOperationType { ESchemeOpAlterResourcePool = 102; ESchemeOpRestoreIncrementalBackup = 103; + ESchemeOpRestoreIncrementalBackupAtTable = 104; } message TApplyIf { diff --git a/ydb/core/tx/schemeshard/schemeshard__operation.cpp b/ydb/core/tx/schemeshard/schemeshard__operation.cpp index f51b938cc104..337bd757e26f 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation.cpp @@ -1190,6 +1190,9 @@ ISubOperation::TPtr TOperation::RestorePart(TTxState::ETxType txType, TTxState:: case TTxState::ETxType::TxAlterResourcePool: return CreateAlterResourcePool(NextPartId(), txState); + case TTxState::ETxType::TxRestoreIncrementalBackupAtTable: + return CreateRestoreIncrementalBackupAtTable(NextPartId(), txState); + case TTxState::ETxType::TxInvalid: Y_UNREACHABLE(); } @@ -1441,6 +1444,8 @@ TVector TOperation::ConstructParts(const TTxTransaction& tx // IncrementalBackup case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup: return CreateRestoreIncrementalBackup(NextPartId(), tx, context); + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable: + Y_ABORT("multipart operations are handled before, also they require transaction details"); } Y_UNREACHABLE(); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_common.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_common.cpp index bdfd80bc99a7..19fe974c4b6b 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_common.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_common.cpp @@ -361,6 +361,8 @@ void NTableState::UpdatePartitioningForTableModification(TOperationId operationI commonShardOp = TTxState::ConfigureParts; } else if (txState.TxType == TTxState::TxDropCdcStreamAtTableDropSnapshot) { commonShardOp = TTxState::ConfigureParts; + } else if (txState.TxType == TTxState::TxRestoreIncrementalBackupAtTable) { + commonShardOp = TTxState::ConfigureParts; } else { Y_ABORT("UNREACHABLE"); } 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 49799a6e74a1..8983e9c6ca8a 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 @@ -112,23 +112,18 @@ void DoCreateAlterTable( result.push_back(CreateAlterTable(NextPartId(opId, result), outTx)); } -namespace NIncrBackup { +namespace NIncrRestore { class TConfigurePartsAtTable: public TSubOperationState { TString DebugHint() const override { return TStringBuilder() - << "NIncrBackupState::TConfigurePartsAtTable" + << "NIncrRestoreState::TConfigurePartsAtTable" << " operationId: " << OperationId; } static bool IsExpectedTxType(TTxState::ETxType txType) { switch (txType) { - case TTxState::TxCreateCdcStreamAtTable: - case TTxState::TxCreateCdcStreamAtTableWithInitialScan: - case TTxState::TxAlterCdcStreamAtTable: - case TTxState::TxAlterCdcStreamAtTableDropSnapshot: - case TTxState::TxDropCdcStreamAtTable: - case TTxState::TxDropCdcStreamAtTableDropSnapshot: + case TTxState::TxRestoreIncrementalBackupAtTable: return true; default: return false; @@ -136,7 +131,6 @@ class TConfigurePartsAtTable: public TSubOperationState { } protected: - // FIXME void FillNotice( const TPathId& pathId, NKikimrTxDataShard::TFlatSchemeTransaction& tx, @@ -149,8 +143,6 @@ class TConfigurePartsAtTable: public TSubOperationState { auto table = context.SS->Tables.at(pathId); tx.MutableCreateIncrementalRestoreSrc()->CopyFrom(RestoreOp); - - // TODO: copy op to notice } public: @@ -213,18 +205,13 @@ class TConfigurePartsAtTable: public TSubOperationState { class TProposeAtTable: public TSubOperationState { TString DebugHint() const override { return TStringBuilder() - << "NIncrBackupState::TProposeAtTable" + << "NIncrRestoreState::TProposeAtTable" << " operationId: " << OperationId; } static bool IsExpectedTxType(TTxState::ETxType txType) { switch (txType) { - case TTxState::TxCreateCdcStreamAtTable: - case TTxState::TxCreateCdcStreamAtTableWithInitialScan: - case TTxState::TxAlterCdcStreamAtTable: - case TTxState::TxAlterCdcStreamAtTableDropSnapshot: - case TTxState::TxDropCdcStreamAtTable: - case TTxState::TxDropCdcStreamAtTableDropSnapshot: + case TTxState::TxRestoreIncrementalBackupAtTable: return true; default: return false; @@ -294,11 +281,6 @@ class TProposeAtTable: public TSubOperationState { context.SS->ChangeTxState(db, OperationId, TTxState::ProposedWaitParts); - const auto step = TStepId(ev->Get()->StepId); - context.SS->SnapshotsStepIds[OperationId.GetTxId()] = step; - context.SS->PersistSnapshotStepId(db, OperationId.GetTxId(), step); - - context.SS->TabletCounters->Simple()[COUNTER_SNAPSHOTS_COUNT].Add(1); return true; } @@ -307,56 +289,6 @@ class TProposeAtTable: public TSubOperationState { }; // TProposeAtTable -class TDoneWithInitialScan: public TDone { -public: - using TDone::TDone; - - bool ProgressState(TOperationContext& context) override { - if (!TDone::ProgressState(context)) { - return false; - } - - const auto* txState = context.SS->FindTx(OperationId); - Y_ABORT_UNLESS(txState); - Y_ABORT_UNLESS(txState->TxType == TTxState::TxCreateCdcStreamAtTableWithInitialScan); - const auto& pathId = txState->TargetPathId; - - Y_ABORT_UNLESS(context.SS->PathsById.contains(pathId)); - auto path = context.SS->PathsById.at(pathId); - - TMaybe streamPathId; - for (const auto& [_, childPathId] : path->GetChildren()) { - Y_ABORT_UNLESS(context.SS->PathsById.contains(childPathId)); - auto childPath = context.SS->PathsById.at(childPathId); - - if (childPath->CreateTxId != OperationId.GetTxId()) { - continue; - } - - Y_ABORT_UNLESS(childPath->IsCdcStream() && !childPath->Dropped()); - Y_ABORT_UNLESS(context.SS->CdcStreams.contains(childPathId)); - auto stream = context.SS->CdcStreams.at(childPathId); - - Y_ABORT_UNLESS(stream->State == TCdcStreamInfo::EState::ECdcStreamStateScan); - Y_VERIFY_S(!streamPathId, "Too many cdc streams are planned to fill with initial scan" - << ": found# " << *streamPathId - << ", another# " << childPathId); - streamPathId = childPathId; - } - - // if (AppData()->DisableCdcAutoSwitchingToReadyStateForTests) { - // return true; - // } - - // FIXME(+active) - // Y_ABORT_UNLESS(streamPathId); - // context.OnComplete.Send(context.SS->SelfId(), new TEvPrivate::TEvRunCdcStreamScan(*streamPathId)); - - return true; - } - -}; // TDoneWithInitialScan - class TNewRestoreFromAtTable: public TSubOperation { static TTxState::ETxState NextState() { return TTxState::ConfigureParts; @@ -380,13 +312,13 @@ class TNewRestoreFromAtTable: public TSubOperation { switch (state) { case TTxState::Waiting: case TTxState::ConfigureParts: - return MakeHolder(OperationId, Transaction.GetRestoreIncrementalBackup()); + return MakeHolder(OperationId, Transaction.GetRestoreIncrementalBackup()); case TTxState::Propose: - return MakeHolder(OperationId); + return MakeHolder(OperationId); case TTxState::ProposedWaitParts: return MakeHolder(OperationId); case TTxState::Done: - return MakeHolder(OperationId); + return MakeHolder(OperationId); default: return nullptr; } @@ -409,9 +341,10 @@ class TNewRestoreFromAtTable: public TSubOperation { const auto& tableName = op.GetSrcTableName(); const auto& dstTableName = op.GetDstTableName(); - // LOG_N("TNewRestoreFromAtTable Propose" - // << ": opId# " << OperationId - // << ", stream# " << workingDir << "/" << tableName << "/" << streamName); + LOG_N("TNewRestoreFromAtTable Propose" + << ": opId# " << OperationId + << ", src# " << workingDir << "/" << tableName + << ", dst# " << workingDir << "/" << dstTableName); auto result = MakeHolder( NKikimrScheme::StatusAccepted, @@ -419,52 +352,64 @@ class TNewRestoreFromAtTable: public TSubOperation { context.SS->TabletID()); const auto workingDirPath = TPath::Resolve(workingDir, context.SS); - // { - // const auto checks = workingDirPath.Check(); - // checks - // .NotUnderDomainUpgrade() - // .IsAtLocalSchemeShard() - // .IsResolved() - // .NotDeleted() - // .IsLikeDirectory() - // .NotUnderDeleting(); - - // if (checks && !workingDirPath.IsTableIndex()) { - // checks.IsCommonSensePath(); - // } - - // if (!checks) { - // result->SetError(checks.GetStatus(), checks.GetError()); - // return result; - // } - // } + { + const auto checks = workingDirPath.Check(); + checks + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .IsLikeDirectory() + .NotUnderDeleting() + .IsCommonSensePath(); + + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + return result; + } + } const auto tablePath = workingDirPath.Child(tableName); - // { - // const auto checks = tablePath.Check(); - // checks - // .NotEmpty() - // .NotUnderDomainUpgrade() - // .IsAtLocalSchemeShard() - // .IsResolved() - // .NotDeleted() - // .IsTable() - // .NotAsyncReplicaTable() - // .NotUnderDeleting(); - - // if (checks) { - // if (!tablePath.IsInsideTableIndexPath()) { - // checks.IsCommonSensePath(); - // } - // checks.IsUnderTheSameOperation(OperationId.GetTxId()); // lock op - // } - - // if (!checks) { - // result->SetError(checks.GetStatus(), checks.GetError()); - // return result; - // } - // } + { + const auto checks = tablePath.Check(); + checks + .NotEmpty() + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .IsTable() + .NotAsyncReplicaTable() + .NotUnderDeleting() + .IsCommonSensePath() + .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op + + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + return result; + } + } + const auto dstTablePath = workingDirPath.Child(dstTableName); + { + const auto checks = tablePath.Check(); + checks + .NotEmpty() + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .IsTable() + .NotAsyncReplicaTable() + .NotUnderDeleting() + .IsCommonSensePath() + .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op + + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + return result; + } + } TString errStr; if (!context.SS->CheckApplyIf(Transaction, errStr)) { @@ -472,11 +417,8 @@ class TNewRestoreFromAtTable: public TSubOperation { return result; } - // NKikimrScheme::EStatus status; - // if (!context.SS->CanCreateSnapshot(tablePath.Base()->PathId, OperationId.GetTxId(), status, errStr)) { - // result->SetError(status, errStr); - // return result; - // } + // we do not need snapshot as far as source table is under operation + // and guaranteed to be unchanged auto guard = context.DbGuard(); context.MemChanges.GrabPath(context.SS, tablePath.Base()->PathId); @@ -485,19 +427,13 @@ class TNewRestoreFromAtTable: public TSubOperation { context.DbChanges.PersistPath(tablePath.Base()->PathId); context.DbChanges.PersistTxState(OperationId); - // context.MemChanges.GrabNewTableSnapshot(context.SS, tablePath.Base()->PathId, OperationId.GetTxId()); - // context.DbChanges.PersistTableSnapshot(tablePath.Base()->PathId, OperationId.GetTxId()); - - // context.SS->TablesWithSnapshots.emplace(tablePath.Base()->PathId, OperationId.GetTxId()); - // context.SS->SnapshotTables[OperationId.GetTxId()].insert(tablePath.Base()->PathId); - Y_ABORT_UNLESS(context.SS->Tables.contains(tablePath.Base()->PathId)); auto table = context.SS->Tables.at(tablePath.Base()->PathId); Y_ABORT_UNLESS(table->AlterVersion != 0); Y_ABORT_UNLESS(!table->AlterData); - const auto txType = TTxState::TxCreateCdcStreamAtTableWithInitialScan; + const auto txType = TTxState::TxRestoreIncrementalBackupAtTable; Y_ABORT_UNLESS(!context.SS->FindTx(OperationId)); auto& txState = context.SS->CreateTx(OperationId, txType, tablePath.Base()->PathId); @@ -530,10 +466,7 @@ class TNewRestoreFromAtTable: public TSubOperation { }; // TNewRestoreFromAtTable -} // namespace NIncrBackup - -/////// -/////// +} // namespace NIncrRestore TVector CreateRestoreIncrementalBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup); @@ -570,7 +503,6 @@ TVector CreateRestoreIncrementalBackup(TOperationId opId, c // check dst locks // lock dst - // start replication read on dst TVector boundaries; const auto& partitions = srcTable->GetPartitions(); @@ -583,49 +515,29 @@ TVector CreateRestoreIncrementalBackup(TOperationId opId, c } } - // NKikimrSchemeOp::TCreateCdcStream createCdcStreamOp; - // createCdcStreamOp.SetTableName(srcTableName); - // auto& streamDescription = *createCdcStreamOp.MutableStreamDescription(); - // streamDescription.SetName(IB_RESTORE_CDC_STREAM_NAME); - // streamDescription.SetMode(NKikimrSchemeOp::ECdcStreamModeRestoreIncrBackup); - // streamDescription.SetFormat(NKikimrSchemeOp::ECdcStreamFormatProto); - // streamDescription.SetState(NKikimrSchemeOp::ECdcStreamStateScan); - TVector result; DoCreateLock(opId, workingDirPath, srcTablePath, false, result); { - auto outTx = TransactionTemplate(workingDirPath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateCdcStreamAtTable); + auto outTx = TransactionTemplate(workingDirPath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable); outTx.MutableRestoreIncrementalBackup()->CopyFrom(restoreOp); auto& restoreOp = *outTx.MutableRestoreIncrementalBackup(); PathIdFromPathId(srcTablePath.Base()->PathId, restoreOp.MutableSrcPathId()); PathIdFromPathId(dstTablePath.Base()->PathId, restoreOp.MutableDstPathId()); - result.push_back(MakeSubOperation(NextPartId(opId, result), outTx)); + result.push_back(MakeSubOperation(NextPartId(opId, result), outTx)); } - // DoCreateAlterTable(opId, dstTablePath, result); - - // NCdc::DoCreateStream( - // result, - // createCdcStreamOp, - // opId, - // workingDirPath, - // srcTablePath, - // acceptExisted, - // true); - // DoCreatePqPart( - // opId, - // streamPath, - // IB_RESTORE_CDC_STREAM_NAME, - // srcTable, - // dstTablePath.Base()->PathId, - // createCdcStreamOp, - // boundaries, - // acceptExisted, - // result); - return result; } +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, const TTxTransaction& tx) { + return MakeSubOperation(id, tx); +} + +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state) { + Y_ABORT_UNLESS(state != TTxState::Invalid); + return MakeSubOperation(id, state); +} + } // namespace NKikimr::NSchemeShard diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_part.h b/ydb/core/tx/schemeshard/schemeshard__operation_part.h index 9bce94226e70..3498ea86c69d 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_part.h +++ b/ydb/core/tx/schemeshard/schemeshard__operation_part.h @@ -621,6 +621,9 @@ ISubOperation::TPtr CreateAlterResourcePool(TOperationId id, TTxState::ETxState ISubOperation::TPtr CreateDropResourcePool(TOperationId id, const TTxTransaction& tx); ISubOperation::TPtr CreateDropResourcePool(TOperationId id, TTxState::ETxState state); +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, const TTxTransaction& tx); +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state); + // returns Reject in case of error, nullptr otherwise ISubOperation::TPtr CascadeDropTableChildren(TVector& result, const TOperationId& id, const TPath& table); diff --git a/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp b/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp index e7cb5192821e..0a074e30ddc8 100644 --- a/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp @@ -244,6 +244,7 @@ TString DefineUserOperationName(const NKikimrSchemeOp::TModifyScheme& tx) { return "ALTER RESOURCE POOL"; // incremental backup case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup: + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable: return "RESTORE"; } Y_ABORT("switch should cover all operation types"); @@ -549,6 +550,7 @@ TVector ExtractChangingPaths(const NKikimrSchemeOp::TModifyScheme& tx) result.emplace_back(tx.GetCreateResourcePool().GetName()); break; case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup: + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable: result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetRestoreIncrementalBackup().GetSrcTableName()})); result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetRestoreIncrementalBackup().GetDstTableName()})); break; diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.cpp b/ydb/core/tx/schemeshard/schemeshard_impl.cpp index e43c71481d04..ba2f048ee5b2 100644 --- a/ydb/core/tx/schemeshard/schemeshard_impl.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_impl.cpp @@ -1554,6 +1554,7 @@ TPathElement::EPathState TSchemeShard::CalcPathState(TTxState::ETxType txType, T case TTxState::TxAlterView: case TTxState::TxAlterContinuousBackup: case TTxState::TxAlterResourcePool: + case TTxState::TxRestoreIncrementalBackupAtTable: return TPathElement::EPathState::EPathStateAlter; case TTxState::TxDropTable: case TTxState::TxDropPQGroup: diff --git a/ydb/core/tx/schemeshard/schemeshard_tx_infly.h b/ydb/core/tx/schemeshard/schemeshard_tx_infly.h index 1d78a1493abc..f2ab0e5744cb 100644 --- a/ydb/core/tx/schemeshard/schemeshard_tx_infly.h +++ b/ydb/core/tx/schemeshard/schemeshard_tx_infly.h @@ -137,6 +137,7 @@ struct TTxState { item(TxCreateResourcePool, 91) \ item(TxDropResourcePool, 92) \ item(TxAlterResourcePool, 93) \ + item(TxRestoreIncrementalBackupAtTable, 94) \ // TX_STATE_TYPE_ENUM @@ -423,6 +424,7 @@ struct TTxState { case TxAlterView: case TxAlterContinuousBackup: case TxAlterResourcePool: + case TxRestoreIncrementalBackupAtTable: return false; case TxMoveTable: case TxMoveTableIndex: @@ -495,6 +497,7 @@ struct TTxState { case TxCopySequence: case TxCreateContinuousBackup: case TxCreateResourcePool: + case TxRestoreIncrementalBackupAtTable: return false; case TxAlterPQGroup: case TxAlterTable: @@ -602,6 +605,7 @@ struct TTxState { case TxCreateView: case TxCreateContinuousBackup: case TxCreateResourcePool: + case TxRestoreIncrementalBackupAtTable: return false; case TxAlterPQGroup: case TxAlterTable: diff --git a/ydb/core/tx/tx_proxy/schemereq.cpp b/ydb/core/tx/tx_proxy/schemereq.cpp index 851bbd97ac1f..7abe7a9fdc0f 100644 --- a/ydb/core/tx/tx_proxy/schemereq.cpp +++ b/ydb/core/tx/tx_proxy/schemereq.cpp @@ -360,6 +360,7 @@ struct TBaseSchemeReq: public TActorBootstrapped { return *modifyScheme.MutableCreateResourcePool()->MutableName(); case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackup: + case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackupAtTable: return *modifyScheme.MutableRestoreIncrementalBackup()->MutableSrcTableName(); } } @@ -621,6 +622,7 @@ struct TBaseSchemeReq: public TActorBootstrapped { case NKikimrSchemeOp::ESchemeOpDropContinuousBackup: case NKikimrSchemeOp::ESchemeOpAlterResourcePool: case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackup: + case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackupAtTable: { auto toResolve = TPathToResolve(pbModifyScheme.GetOperationType()); toResolve.Path = Merge(workingDir, SplitPath(GetPathNameForScheme(pbModifyScheme)));