diff --git a/ydb/core/kqp/executer_actor/kqp_scheme_executer.cpp b/ydb/core/kqp/executer_actor/kqp_scheme_executer.cpp index 15835cf536e5..928daff9534a 100644 --- a/ydb/core/kqp/executer_actor/kqp_scheme_executer.cpp +++ b/ydb/core/kqp/executer_actor/kqp_scheme_executer.cpp @@ -350,6 +350,45 @@ class TKqpSchemeExecuter : public TActorBootstrapped { } + case NKqpProto::TKqpSchemeOperation::kCreateBackupCollection: { + const auto& modifyScheme = schemeOp.GetCreateBackupCollection(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + + case NKqpProto::TKqpSchemeOperation::kAlterBackupCollection: { + const auto& modifyScheme = schemeOp.GetAlterBackupCollection(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + + case NKqpProto::TKqpSchemeOperation::kDropBackupCollection: { + const auto& modifyScheme = schemeOp.GetDropBackupCollection(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + + // FIXME(+active): maybe done + case NKqpProto::TKqpSchemeOperation::kBackup: { + const auto& modifyScheme = schemeOp.GetBackup(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + + // FIXME(+active): maybe done + case NKqpProto::TKqpSchemeOperation::kBackupIncremental: { + const auto& modifyScheme = schemeOp.GetBackupIncremental(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + + // FIXME(+active) + case NKqpProto::TKqpSchemeOperation::kRestore: { + const auto& modifyScheme = schemeOp.GetRestore(); + ev->Record.MutableTransaction()->MutableModifyScheme()->CopyFrom(modifyScheme); + break; + } + default: InternalError(TStringBuilder() << "Unexpected scheme operation: " << (ui32) schemeOp.GetOperationCase()); diff --git a/ydb/core/kqp/gateway/kqp_ic_gateway.cpp b/ydb/core/kqp/gateway/kqp_ic_gateway.cpp index ee8698df4acf..9ef39b13ceb6 100644 --- a/ydb/core/kqp/gateway/kqp_ic_gateway.cpp +++ b/ydb/core/kqp/gateway/kqp_ic_gateway.cpp @@ -1287,6 +1287,30 @@ class TKikimrIcGateway : public IKqpGateway { } } + TFuture CreateBackupCollection(const TString&, const NYql::TCreateBackupCollectionSettings&) override { + return NotImplemented(); + } + + TFuture AlterBackupCollection(const TString&, const NYql::TAlterBackupCollectionSettings&) override { + return NotImplemented(); + } + + TFuture DropBackupCollection(const TString&, const NYql::TDropBackupCollectionSettings&) override { + return NotImplemented(); + } + + TFuture Backup(const TString&, const NYql::TBackupSettings&) override { + return NotImplemented(); + } + + TFuture BackupIncremental(const TString&, const NYql::TBackupSettings&) override { + return NotImplemented(); + } + + TFuture Restore(const TString&, const NYql::TBackupSettings&) override { + return NotImplemented(); + } + TFuture CreateUser(const TString& cluster, const NYql::TCreateUserSettings& settings) override { using TRequest = TEvTxUserProxy::TEvProposeTransaction; diff --git a/ydb/core/kqp/host/kqp_gateway_proxy.cpp b/ydb/core/kqp/host/kqp_gateway_proxy.cpp index 1df1ebd46a2b..6cc86e3638a1 100644 --- a/ydb/core/kqp/host/kqp_gateway_proxy.cpp +++ b/ydb/core/kqp/host/kqp_gateway_proxy.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace NKikimr::NKqp { using namespace NThreading; @@ -2206,6 +2208,338 @@ class TKqpGatewayProxy : public IKikimrGateway { } } + TFuture CreateBackupCollection(const TString& cluster, const NYql::TCreateBackupCollectionSettings& settings) override { + CHECK_PREPARED_DDL(CreateBackupCollection); + + try { + if (cluster != SessionCtx->GetCluster()) { + return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + } + + TString path; + + if (!settings.Name.StartsWith(SessionCtx->GetDatabase())) { + path = JoinPath({SessionCtx->GetDatabase(), ".backups/collections", settings.Name}); + } else { + path = settings.Name; + } + + TString error; + std::pair pathPair; + if (!NSchemeHelpers::SplitTablePath(path, GetDatabase(), pathPair, error, true)) { + return MakeFuture(ResultFromError(error)); + } + + NKikimrSchemeOp::TModifyScheme tx; + tx.SetWorkingDir(pathPair.first); + tx.SetOperationType(NKikimrSchemeOp::ESchemeOpCreateBackupCollection); + + auto& op = *tx.MutableCreateBackupCollection(); + op.SetName(pathPair.second); + + auto& properties = *op.MutableProperties(); + if (settings.Settings.IncrementalBackupEnabled) { + properties.MutableIncrementalBackupConfig(); + } + + std::visit( + TOverloaded { + [](const TCreateBackupCollectionSettings::TDatabase&) { + Y_ABORT("unimplemented"); + }, + [&](const TVector& tables) { + auto& dstTables = *properties.MutableExplicitEntryList(); + for (const auto& table : tables) { + auto& entry = *dstTables.AddEntries(); + entry.SetType(NKikimrSchemeOp::TBackupCollectionProperties::TBackupEntry::ETypeTable); + entry.SetPath(table.Path); + } + }, + }, settings.Entries); + + properties.SetCluster(::google::protobuf::NULL_VALUE); + + if (IsPrepare()) { + auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + auto& phyTx = *phyQuery.AddTransactions(); + phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + phyTx.MutableSchemeOperation()->MutableCreateBackupCollection()->Swap(&tx); + + TGenericResult result; + result.SetSuccess(); + return MakeFuture(result); + } else { + return Gateway->ModifyScheme(std::move(tx)); + } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + + TFuture AlterBackupCollection(const TString& cluster, const NYql::TAlterBackupCollectionSettings& settings) override { + CHECK_PREPARED_DDL(AlterBackupCollection); + + try { + Y_UNUSED(cluster, settings); + return MakeFuture(ResultFromError("Unimplemented")); + // if (cluster != SessionCtx->GetCluster()) { + // return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + // } + + // std::pair pathPair; + // { + // TString error; + // if (!NSchemeHelpers::SplitTablePath(settings.Name, GetDatabase(), pathPair, error, false)) { + // return MakeFuture(ResultFromError(error)); + // } + // } + + // NKikimrSchemeOp::TModifyScheme tx; + // tx.SetWorkingDir(pathPair.first); + // tx.SetOperationType(NKikimrSchemeOp::ESchemeOpAlterBackupCollection); + + // auto& op = *tx.MutableAlterBackupCollection(); + // op.SetName(pathPair.second); + + // if (const auto& done = settings.Settings.StateDone) { + // auto& state = *op.MutableState(); + // state.MutableDone()->SetFailoverMode( + // static_cast(done->FailoverMode)); + // } + + // if (settings.Settings.ConnectionString || settings.Settings.Endpoint || settings.Settings.Database || + // settings.Settings.OAuthToken || settings.Settings.StaticCredentials) { + // auto& config = *op.MutableConfig(); + // auto& params = *config.MutableSrcConnectionParams(); + // if (const auto& connectionString = settings.Settings.ConnectionString) { + // const auto parseResult = NYdb::ParseConnectionString(*connectionString); + // params.SetEndpoint(parseResult.Endpoint); + // params.SetDatabase(parseResult.Database); + // } + // if (const auto& endpoint = settings.Settings.Endpoint) { + // params.SetEndpoint(*endpoint); + // } + // if (const auto& database = settings.Settings.Database) { + // params.SetDatabase(*database); + // } + // if (const auto& oauth = settings.Settings.OAuthToken) { + // oauth->Serialize(*params.MutableOAuthToken()); + // } + // if (const auto& staticCreds = settings.Settings.StaticCredentials) { + // staticCreds->Serialize(*params.MutableStaticCredentials()); + // } + // } + + // if (IsPrepare()) { + // auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + // auto& phyTx = *phyQuery.AddTransactions(); + // phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + // phyTx.MutableSchemeOperation()->MutableAlterBackupCollection()->Swap(&tx); + + // TGenericResult result; + // result.SetSuccess(); + // return MakeFuture(result); + // } else { + // return Gateway->ModifyScheme(std::move(tx)); + // } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + + TFuture DropBackupCollection(const TString& cluster, const NYql::TDropBackupCollectionSettings& settings) override { + CHECK_PREPARED_DDL(DropBackupCollection); + + try { + Y_UNUSED(cluster, settings); + return MakeFuture(ResultFromError("Unimplemented")); + // if (cluster != SessionCtx->GetCluster()) { + // return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + // } + + // std::pair pathPair; + // { + // TString error; + // if (!NSchemeHelpers::SplitTablePath(settings.Name, GetDatabase(), pathPair, error, false)) { + // return MakeFuture(ResultFromError(error)); + // } + // } + + // NKikimrSchemeOp::TModifyScheme tx; + // tx.SetWorkingDir(pathPair.first); + // if (settings.Cascade) { + // tx.SetOperationType(NKikimrSchemeOp::ESchemeOpDropBackupCollectionCascade); + // } else { + // tx.SetOperationType(NKikimrSchemeOp::ESchemeOpDropBackupCollection); + // } + + // auto& op = *tx.MutableDrop(); + // op.SetName(pathPair.second); + + // if (IsPrepare()) { + // auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + // auto& phyTx = *phyQuery.AddTransactions(); + // phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + // phyTx.MutableSchemeOperation()->MutableDropBackupCollection()->Swap(&tx); + + // TGenericResult result; + // result.SetSuccess(); + // return MakeFuture(result); + // } else { + // return Gateway->ModifyScheme(std::move(tx)); + // } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + + TFuture Backup(const TString& cluster, const NYql::TBackupSettings& settings) override { + CHECK_PREPARED_DDL(Backup); + + // FIXME(+active) + try { + if (cluster != SessionCtx->GetCluster()) { + return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + } + + TString path; + + if (!settings.Name.StartsWith(SessionCtx->GetDatabase())) { + path = JoinPath({SessionCtx->GetDatabase(), ".backups/collections", settings.Name}); + } else { + path = settings.Name; + } + + TString error; + std::pair pathPair; + if (!NSchemeHelpers::SplitTablePath(path, GetDatabase(), pathPair, error, true)) { + return MakeFuture(ResultFromError(error)); + } + + NKikimrSchemeOp::TModifyScheme tx; + tx.SetWorkingDir(pathPair.first); + tx.SetOperationType(NKikimrSchemeOp::ESchemeOpBackupBackupCollection); + + auto& op = *tx.MutableBackupBackupCollection(); + op.SetName(pathPair.second); + + if (IsPrepare()) { + auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + auto& phyTx = *phyQuery.AddTransactions(); + phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + phyTx.MutableSchemeOperation()->MutableBackup()->Swap(&tx); + + TGenericResult result; + result.SetSuccess(); + return MakeFuture(result); + } else { + return Gateway->ModifyScheme(std::move(tx)); + } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + + TFuture BackupIncremental(const TString& cluster, const NYql::TBackupSettings& settings) override { + CHECK_PREPARED_DDL(Backup); + + // FIXME(+active) + try { + if (cluster != SessionCtx->GetCluster()) { + return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + } + + TString path; + + if (!settings.Name.StartsWith(SessionCtx->GetDatabase())) { + path = JoinPath({SessionCtx->GetDatabase(), ".backups/collections", settings.Name}); + } else { + path = settings.Name; + } + + TString error; + std::pair pathPair; + if (!NSchemeHelpers::SplitTablePath(path, GetDatabase(), pathPair, error, true)) { + return MakeFuture(ResultFromError(error)); + } + + NKikimrSchemeOp::TModifyScheme tx; + tx.SetWorkingDir(pathPair.first); + tx.SetOperationType(NKikimrSchemeOp::ESchemeOpBackupIncrementalBackupCollection); + + auto& op = *tx.MutableBackupIncrementalBackupCollection(); + op.SetName(pathPair.second); + + if (IsPrepare()) { + auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + auto& phyTx = *phyQuery.AddTransactions(); + phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + phyTx.MutableSchemeOperation()->MutableBackupIncremental()->Swap(&tx); + + TGenericResult result; + result.SetSuccess(); + return MakeFuture(result); + } else { + return Gateway->ModifyScheme(std::move(tx)); + } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + + TFuture Restore(const TString& cluster, const NYql::TBackupSettings& settings) override { + CHECK_PREPARED_DDL(Backup); + + // FIXME(+active) + try { + if (cluster != SessionCtx->GetCluster()) { + return MakeFuture(ResultFromError("Invalid cluster: " + cluster)); + } + + TString path; + + if (!settings.Name.StartsWith(SessionCtx->GetDatabase())) { + path = JoinPath({SessionCtx->GetDatabase(), ".backups/collections", settings.Name}); + } else { + path = settings.Name; + } + + TString error; + std::pair pathPair; + if (!NSchemeHelpers::SplitTablePath(path, GetDatabase(), pathPair, error, true)) { + return MakeFuture(ResultFromError(error)); + } + + NKikimrSchemeOp::TModifyScheme tx; + tx.SetWorkingDir(pathPair.first); + tx.SetOperationType(NKikimrSchemeOp::ESchemeOpRestoreBackupCollection); + + auto& op = *tx.MutableRestoreBackupCollection(); + op.SetName(pathPair.second); + + if (IsPrepare()) { + auto& phyQuery = *SessionCtx->Query().PreparingQuery->MutablePhysicalQuery(); + auto& phyTx = *phyQuery.AddTransactions(); + phyTx.SetType(NKqpProto::TKqpPhyTx::TYPE_SCHEME); + phyTx.MutableSchemeOperation()->MutableRestore()->Swap(&tx); + + TGenericResult result; + result.SetSuccess(); + return MakeFuture(result); + } else { + return Gateway->ModifyScheme(std::move(tx)); + } + } + catch (yexception& e) { + return MakeFuture(ResultFromException(e)); + } + } + TVector GetCollectedSchemeData() override { return Gateway->GetCollectedSchemeData(); } diff --git a/ydb/core/kqp/provider/yql_kikimr_datasink.cpp b/ydb/core/kqp/provider/yql_kikimr_datasink.cpp index 4fc782f1ed16..4889c56b0c40 100644 --- a/ydb/core/kqp/provider/yql_kikimr_datasink.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_datasink.cpp @@ -221,6 +221,42 @@ class TKiSinkIntentDeterminationTransformer: public TKiSinkVisitorTransformer { return TStatus::Error; } + TStatus HandleCreateBackupCollection(TKiCreateBackupCollection node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + + TStatus HandleAlterBackupCollection(TKiAlterBackupCollection node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + + TStatus HandleDropBackupCollection(TKiDropBackupCollection node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + + TStatus HandleBackup(TKiBackup node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + + TStatus HandleBackupIncremental(TKiBackupIncremental node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + + TStatus HandleRestore(TKiRestore node, TExprContext& ctx) override { + Y_UNUSED(ctx); + Y_UNUSED(node); + return TStatus::Ok; + } + static void HandleDropTable(TIntrusivePtr& ctx, const NCommon::TWriteTableSettings& settings, const TKikimrKey& key, const TStringBuf& cluster) { @@ -353,6 +389,8 @@ class TKiSinkIntentDeterminationTransformer: public TKiSinkVisitorTransformer { return TStatus::Ok; case TKikimrKey::Type::Replication: return TStatus::Ok; + case TKikimrKey::Type::BackupCollection: + return TStatus::Ok; } ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), "Invalid table key type.")); @@ -572,6 +610,20 @@ class TKikimrDataSink : public TDataProviderBase } } + if (node.IsCallable(TKiCreateBackupCollection::CallableName()) + || node.IsCallable(TKiAlterBackupCollection::CallableName()) + || node.IsCallable(TKiDropBackupCollection::CallableName()) + ) { + return true; + } + + if (node.IsCallable(TKiBackup::CallableName()) + || node.IsCallable(TKiBackupIncremental::CallableName()) + || node.IsCallable(TKiRestore::CallableName()) + ) { + return true; + } + return false; } @@ -1433,6 +1485,58 @@ class TKikimrDataSink : public TDataProviderBase } break; } + case TKikimrKey::Type::BackupCollection: { + auto settings = NCommon::ParseWriteBackupCollectionSettings(TExprList(node->Child(4)), ctx); + YQL_ENSURE(settings.Mode); + auto mode = settings.Mode.Cast(); + + if (mode == "create") { + return Build(ctx, node->Pos()) + .World(node->Child(0)) + .DataSink(node->Child(1)) + .BackupCollection().Build(key.GetBackupCollectionPath().Name) + .Prefix().Build(key.GetBackupCollectionPath().Prefix) + .Entries(settings.Entries.Cast()) + .BackupCollectionSettings(settings.BackupCollectionSettings.Cast()) + .Settings(settings.Other) + .Done() + .Ptr(); + } else if (mode == "alter") { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "Unimplemented")); + return nullptr; + } else if (mode == "drop") { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "Unimplemented")); + return nullptr; + } else if (mode == "backup") { + return Build(ctx, node->Pos()) + .World(node->Child(0)) + .DataSink(node->Child(1)) + .BackupCollection().Build(key.GetBackupCollectionPath().Name) + .Prefix().Build(key.GetBackupCollectionPath().Prefix) + .Done() + .Ptr(); + } else if (mode == "backupIncremental") { + return Build(ctx, node->Pos()) + .World(node->Child(0)) + .DataSink(node->Child(1)) + .BackupCollection().Build(key.GetBackupCollectionPath().Name) + .Prefix().Build(key.GetBackupCollectionPath().Prefix) + .Done() + .Ptr(); + } else if (mode == "restore") { + return Build(ctx, node->Pos()) + .World(node->Child(0)) + .DataSink(node->Child(1)) + .BackupCollection().Build(key.GetBackupCollectionPath().Name) + .Prefix().Build(key.GetBackupCollectionPath().Prefix) + .Done() + .Ptr(); + } else { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), TStringBuilder() << "Unknown operation type for backup collection: " << TString(mode))); + return nullptr; + } + break; + } } ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "Failed to rewrite IO.")); @@ -1622,6 +1726,30 @@ IGraphTransformer::TStatus TKiSinkVisitorTransformer::DoTransform(TExprNode::TPt return HandleAnalyze(node.Cast(), ctx); } + if (auto node = TMaybeNode(input)) { + return HandleCreateBackupCollection(node.Cast(), ctx); + } + + if (auto node = TMaybeNode(input)) { + return HandleAlterBackupCollection(node.Cast(), ctx); + } + + if (auto node = TMaybeNode(input)) { + return HandleDropBackupCollection(node.Cast(), ctx); + } + + if (auto node = TMaybeNode(input)) { + return HandleBackup(node.Cast(), ctx); + } + + if (auto node = TMaybeNode(input)) { + return HandleBackupIncremental(node.Cast(), ctx); + } + + if (auto node = TMaybeNode(input)) { + return HandleRestore(node.Cast(), ctx); + } + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "(Kikimr DataSink) Unsupported function: " << callable.CallableName())); return TStatus::Error; diff --git a/ydb/core/kqp/provider/yql_kikimr_datasource.cpp b/ydb/core/kqp/provider/yql_kikimr_datasource.cpp index 1c3a02efc504..1ba62c6a5d49 100644 --- a/ydb/core/kqp/provider/yql_kikimr_datasource.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_datasource.cpp @@ -164,6 +164,8 @@ class TKiSourceIntentDeterminationTransformer: public TKiSourceVisitorTransforme return TStatus::Ok; case TKikimrKey::Type::Replication: return TStatus::Ok; + case TKikimrKey::Type::BackupCollection: + return TStatus::Ok; } return TStatus::Error; diff --git a/ydb/core/kqp/provider/yql_kikimr_exec.cpp b/ydb/core/kqp/provider/yql_kikimr_exec.cpp index d8827d029dd0..e2307a3ec3ae 100644 --- a/ydb/core/kqp/provider/yql_kikimr_exec.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_exec.cpp @@ -729,6 +729,34 @@ namespace { return true; } + + bool ParseBackupCollectionSettings( + TBackupCollectionSettings& dstSettings, + const TCoNameValueTupleList& srcSettings, + TExprContext& ctx, + TPositionHandle pos) + { + for (auto setting : srcSettings) { + auto name = setting.Name().Value(); + if (name == "incremental_backup_enabled") { + auto value = ToString(setting.Value().Cast().Literal().Cast().Value()); + if (!TryFromString(value, dstSettings.IncrementalBackupEnabled)) { + ctx.AddError(TIssue(ctx.GetPosition(pos), + "INCREMENTAL_BACKUP_ENABLED must be true or false")); + return false; + } + } else if (name == "storage") { + auto value = ToString(setting.Value().Cast().Literal().Cast().Value()); + if (to_lower(value) != "cluster") { + ctx.AddError(TIssue(ctx.GetPosition(pos), + "Only cluster STORAGE are currently supported")); + return false; + } + } + } + + return true; + } } class TKiSinkPlanInfoTransformer : public TGraphTransformerBase { @@ -2422,6 +2450,165 @@ class TKiSinkCallableExecutionTransformer : public TAsyncCallbackTransformer(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto createBackupCollection = maybeCreateBackupCollection.Cast(); + + TCreateBackupCollectionSettings settings; + settings.Name = TString(createBackupCollection.BackupCollection()); + + TVector tables; + + for (auto entry: createBackupCollection.Entries()) { + auto type = entry.Type().Cast().StringValue(); + if (type == "database") { + ctx.AddError(TIssue(ctx.GetPosition(entry.Type().Pos()), + TStringBuilder() << "DATABASE is not implemented yet")); + return SyncError(); + } else if (type == "table") { + YQL_ENSURE(entry.Path()); + auto path = entry.Path().Cast().StringValue(); + tables.emplace_back(TCreateBackupCollectionSettings::TTable{path}); + } + } + + settings.Entries = tables; + + if (!ParseBackupCollectionSettings(settings.Settings, createBackupCollection.BackupCollectionSettings(), ctx, createBackupCollection.Pos())) { + return SyncError(); + } + + auto cluster = TString(createBackupCollection.DataSink().Cluster()); + auto future = Gateway->CreateBackupCollection(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing CREATE BACKUP COLLECTION"); + } + + if (auto maybeAlterBackupCollection = TMaybeNode(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto alterBackupCollection = maybeAlterBackupCollection.Cast(); + + TAlterBackupCollectionSettings settings; + settings.Name = TString(alterBackupCollection.BackupCollection()); + + // FIXME + // if (!ParseAsyncBackupCollectionSettings(settings.Settings, alterBackupCollection.BackupCollectionSettings(), ctx, alterBackupCollection.Pos())) { + // return SyncError(); + // } + + auto cluster = TString(alterBackupCollection.DataSink().Cluster()); + auto future = Gateway->AlterBackupCollection(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing ALTER BACKUP COLLECTION"); + } + + if (auto maybeDropBackupCollection = TMaybeNode(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto dropBackupCollection = maybeDropBackupCollection.Cast(); + + TDropBackupCollectionSettings settings; + settings.Name = TString(dropBackupCollection.BackupCollection()); + settings.Cascade = (dropBackupCollection.Cascade().Value() == "1"); + + auto cluster = TString(dropBackupCollection.DataSink().Cluster()); + auto future = Gateway->DropBackupCollection(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing DROP BACKUP COLLECTION"); + } + + if (auto maybeBackup = TMaybeNode(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto backup = maybeBackup.Cast(); + + TBackupSettings settings; + settings.Name = TString(backup.BackupCollection()); + + auto cluster = TString(backup.DataSink().Cluster()); + auto future = Gateway->Backup(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing BACKUP"); + } + + if (auto maybeBackupIncremental = TMaybeNode(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto backupIncremental = maybeBackupIncremental.Cast(); + + TBackupSettings settings; + settings.Name = TString(backupIncremental.BackupCollection()); + + auto cluster = TString(backupIncremental.DataSink().Cluster()); + auto future = Gateway->BackupIncremental(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing BACKUP INCREMENTAL"); + } + + if (auto maybeRestore = TMaybeNode(input)) { + auto requireStatus = RequireChild(*input, 0); + if (requireStatus.Level != TStatus::Ok) { + return SyncStatus(requireStatus); + } + + auto restore = maybeRestore.Cast(); + + TBackupSettings settings; + settings.Name = TString(restore.BackupCollection()); + + auto cluster = TString(restore.DataSink().Cluster()); + auto future = Gateway->Restore(cluster, settings); + + return WrapFuture(future, + [](const IKikimrGateway::TGenericResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + auto resultNode = ctx.NewWorld(input->Pos()); + return resultNode; + }, "Executing RESTORE"); + } + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "(Kikimr DataSink) Failed to execute node: " << input->Content())); return SyncError(); diff --git a/ydb/core/kqp/provider/yql_kikimr_expr_nodes.json b/ydb/core/kqp/provider/yql_kikimr_expr_nodes.json index 207258910400..8be6655ddad4 100644 --- a/ydb/core/kqp/provider/yql_kikimr_expr_nodes.json +++ b/ydb/core/kqp/provider/yql_kikimr_expr_nodes.json @@ -510,6 +510,76 @@ {"Index": 2, "Name": "Table", "Type": "TCoAtom"}, {"Index": 3, "Name": "Columns", "Type": "TCoAtomList"} ] + }, + { + "Name": "TKiCreateBackupCollection", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiCreateBackupCollection!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "Prefix", "Type": "TCoAtom"}, + {"Index": 4, "Name": "Entries", "Type": "TCoBackupCollectionEntryList"}, + {"Index": 5, "Name": "BackupCollectionSettings", "Type": "TCoNameValueTupleList"}, + {"Index": 6, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TKiAlterBackupCollection", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiAlterBackupCollection!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "BackupCollectionSettings", "Type": "TCoNameValueTupleList"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TKiDropBackupCollection", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiDropBackupCollection!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "Cascade", "Type": "TCoAtom"} + ] + }, + { + "Name": "TKiBackup", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiBackup!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "Prefix", "Type": "TCoAtom"} + ] + }, + { + "Name": "TKiBackupIncremental", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiBackupIncremental!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "Prefix", "Type": "TCoAtom"} + ] + }, + { + "Name": "TKiRestore", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "KiRestore!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TKiDataSink"}, + {"Index": 2, "Name": "BackupCollection", "Type": "TCoAtom"}, + {"Index": 3, "Name": "Prefix", "Type": "TCoAtom"} + ] } ] } diff --git a/ydb/core/kqp/provider/yql_kikimr_gateway.h b/ydb/core/kqp/provider/yql_kikimr_gateway.h index 1580d95cd8ac..fcbecdfa0d4c 100644 --- a/ydb/core/kqp/provider/yql_kikimr_gateway.h +++ b/ydb/core/kqp/provider/yql_kikimr_gateway.h @@ -833,6 +833,36 @@ struct TAnalyzeSettings { TVector Columns; }; +struct TBackupCollectionSettings { + bool IncrementalBackupEnabled; +}; + +struct TCreateBackupCollectionSettings { + struct TDatabase {}; + + struct TTable { + TString Path; + }; + + TString Name; + std::variant> Entries; + TBackupCollectionSettings Settings; +}; + +struct TAlterBackupCollectionSettings { + TString Name; + TBackupCollectionSettings Settings; +}; + +struct TDropBackupCollectionSettings { + TString Name; + bool Cascade = false; +}; + +struct TBackupSettings { + TString Name; +}; + struct TKikimrListPathItem { TKikimrListPathItem(TString name, bool isDirectory) { Name = name; @@ -1026,6 +1056,18 @@ class IKikimrGateway : public TThrRefBase { virtual NThreading::TFuture DropObject(const TString& cluster, const TDropObjectSettings& settings) = 0; + virtual NThreading::TFuture CreateBackupCollection(const TString& cluster, const TCreateBackupCollectionSettings& settings) = 0; + + virtual NThreading::TFuture AlterBackupCollection(const TString& cluster, const TAlterBackupCollectionSettings& settings) = 0; + + virtual NThreading::TFuture DropBackupCollection(const TString& cluster, const TDropBackupCollectionSettings& settings) = 0; + + virtual NThreading::TFuture Backup(const TString& cluster, const TBackupSettings& settings) = 0; + + virtual NThreading::TFuture BackupIncremental(const TString& cluster, const TBackupSettings& settings) = 0; + + virtual NThreading::TFuture Restore(const TString& cluster, const TBackupSettings& settings) = 0; + virtual NThreading::TFuture CreateGroup(const TString& cluster, const TCreateGroupSettings& settings) = 0; virtual NThreading::TFuture AlterGroup(const TString& cluster, TAlterGroupSettings& settings) = 0; diff --git a/ydb/core/kqp/provider/yql_kikimr_provider.cpp b/ydb/core/kqp/provider/yql_kikimr_provider.cpp index 1bf3f0c98503..108fa8e2fc3c 100644 --- a/ydb/core/kqp/provider/yql_kikimr_provider.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_provider.cpp @@ -74,6 +74,12 @@ struct TKikimrData { DataSinkNames.insert(TKiDropSequence::CallableName()); DataSinkNames.insert(TKiAlterSequence::CallableName()); DataSinkNames.insert(TKiAnalyzeTable::CallableName()); + DataSinkNames.insert(TKiCreateBackupCollection::CallableName()); + DataSinkNames.insert(TKiAlterBackupCollection::CallableName()); + DataSinkNames.insert(TKiDropBackupCollection::CallableName()); + DataSinkNames.insert(TKiBackup::CallableName()); + DataSinkNames.insert(TKiBackupIncremental::CallableName()); + DataSinkNames.insert(TKiRestore::CallableName()); CommitModes.insert(CommitModeFlush); CommitModes.insert(CommitModeRollback); @@ -120,7 +126,13 @@ struct TKikimrData { TYdbOperation::AlterGroup | TYdbOperation::DropGroup | TYdbOperation::RenameGroup | - TYdbOperation::ModifyPermission; + TYdbOperation::ModifyPermission | + TYdbOperation::CreateBackupCollection | + TYdbOperation::AlterBackupCollection | + TYdbOperation::DropBackupCollection | + TYdbOperation::Backup | + TYdbOperation::BackupIncremental | + TYdbOperation::Restore; SystemColumns = { {"_yql_partition_id", NKikimr::NUdf::EDataSlot::Uint64} @@ -432,6 +444,10 @@ bool TKikimrKey::Extract(const TExprNode& key) { KeyType = Type::PGObject; Target = key.Child(0)->Child(1)->Child(0)->Content(); ObjectType = key.Child(0)->Child(2)->Child(0)->Content(); + } else if (tagName == "backupCollection" || tagName == "backup" || tagName == "restore") { + KeyType = Type::BackupCollection; + Target = key.Child(0)->Child(1)->Child(0)->Content(); + ExplicitPrefix = key.Child(0)->Child(2)->Child(0)->Content(); } else { Ctx.AddError(TIssue(Ctx.GetPosition(key.Child(0)->Pos()), TString("Unexpected tag for kikimr key: ") + tagName)); return false; diff --git a/ydb/core/kqp/provider/yql_kikimr_provider.h b/ydb/core/kqp/provider/yql_kikimr_provider.h index 59660484b455..fd175b8e1228 100644 --- a/ydb/core/kqp/provider/yql_kikimr_provider.h +++ b/ydb/core/kqp/provider/yql_kikimr_provider.h @@ -216,35 +216,41 @@ class TKikimrTablesData : public TThrRefBase { NKikimr::NKqp::TKqpTempTablesState::TConstPtr TempTablesState; }; -enum class TYdbOperation : ui32 { - CreateTable = 1 << 0, - DropTable = 1 << 1, - AlterTable = 1 << 2, - Select = 1 << 3, - Upsert = 1 << 4, - Replace = 1 << 5, - Update = 1 << 6, - Delete = 1 << 7, - InsertRevert = 1 << 8, - InsertAbort = 1 << 9, - ReservedInsertIgnore = 1 << 10, - UpdateOn = 1 << 11, - DeleteOn = 1 << 12, - CreateUser = 1 << 13, - AlterUser = 1 << 14, - DropUser = 1 << 15, - CreateGroup = 1 << 16, - AlterGroup = 1 << 17, - DropGroup = 1 << 18, - CreateTopic = 1 << 19, - AlterTopic = 1 << 20, - DropTopic = 1 << 21, - ModifyPermission = 1 << 22, - RenameGroup = 1 << 23, - CreateReplication = 1 << 24, - AlterReplication = 1 << 25, - DropReplication = 1 << 26, - Analyze = 1 << 27, +enum class TYdbOperation : ui64 { //FIXME(+active) + CreateTable = 1ull << 0, + DropTable = 1ull << 1, + AlterTable = 1ull << 2, + Select = 1ull << 3, + Upsert = 1ull << 4, + Replace = 1ull << 5, + Update = 1ull << 6, + Delete = 1ull << 7, + InsertRevert = 1ull << 8, + InsertAbort = 1ull << 9, + ReservedInsertIgnore = 1ull << 10, + UpdateOn = 1ull << 11, + DeleteOn = 1ull << 12, + CreateUser = 1ull << 13, + AlterUser = 1ull << 14, + DropUser = 1ull << 15, + CreateGroup = 1ull << 16, + AlterGroup = 1ull << 17, + DropGroup = 1ull << 18, + CreateTopic = 1ull << 19, + AlterTopic = 1ull << 20, + DropTopic = 1ull << 21, + ModifyPermission = 1ull << 22, + RenameGroup = 1ull << 23, + CreateReplication = 1ull << 24, + AlterReplication = 1ull << 25, + DropReplication = 1ull << 26, + Analyze = 1ull << 27, + CreateBackupCollection = 1ull << 28, + AlterBackupCollection = 1ull << 29, + DropBackupCollection = 1ull << 30, + Backup = 1ull << 31, + BackupIncremental = 1ull << 32, + Restore = 1ull << 33, }; Y_DECLARE_FLAGS(TYdbOperations, TYdbOperation); diff --git a/ydb/core/kqp/provider/yql_kikimr_provider_impl.h b/ydb/core/kqp/provider/yql_kikimr_provider_impl.h index 15e19a52d003..e88c82596fca 100644 --- a/ydb/core/kqp/provider/yql_kikimr_provider_impl.h +++ b/ydb/core/kqp/provider/yql_kikimr_provider_impl.h @@ -70,6 +70,13 @@ class TKiSinkVisitorTransformer : public TSyncTransformerBase { virtual TStatus HandleDropSequence(NNodes::TKiDropSequence node, TExprContext& ctx) = 0; virtual TStatus HandleAlterSequence(NNodes::TKiAlterSequence node, TExprContext& ctx) = 0; + virtual TStatus HandleCreateBackupCollection(NNodes::TKiCreateBackupCollection node, TExprContext& ctx) = 0; + virtual TStatus HandleAlterBackupCollection(NNodes::TKiAlterBackupCollection node, TExprContext& ctx) = 0; + virtual TStatus HandleDropBackupCollection(NNodes::TKiDropBackupCollection node, TExprContext& ctx) = 0; + virtual TStatus HandleBackup(NNodes::TKiBackup node, TExprContext& ctx) = 0; + virtual TStatus HandleBackupIncremental(NNodes::TKiBackupIncremental node, TExprContext& ctx) = 0; + virtual TStatus HandleRestore(NNodes::TKiRestore node, TExprContext& ctx) = 0; + virtual TStatus HandleModifyPermissions(NNodes::TKiModifyPermissions node, TExprContext& ctx) = 0; virtual TStatus HandleReturningList(NNodes::TKiReturningList node, TExprContext& ctx) = 0; @@ -89,6 +96,7 @@ class TKikimrKey { Permission, PGObject, Replication, + BackupCollection, }; struct TViewDescription { @@ -96,6 +104,11 @@ class TKikimrKey { bool PrimaryFlag = false; }; + struct TBackupCollectionDescription { + TString Prefix; + TString Name; + }; + public: TKikimrKey(TExprContext& ctx) : Ctx(ctx) {} @@ -171,6 +184,16 @@ class TKikimrKey { return *ObjectType; } + TBackupCollectionDescription GetBackupCollectionPath() const { + Y_DEBUG_ABORT_UNLESS(KeyType.Defined()); + Y_DEBUG_ABORT_UNLESS(KeyType == Type::BackupCollection); + Y_DEBUG_ABORT_UNLESS(ExplicitPrefix.Defined()); + return TBackupCollectionDescription { + .Prefix = *ExplicitPrefix, + .Name = Target, + }; + } + bool Extract(const TExprNode& key); private: @@ -179,6 +202,7 @@ class TKikimrKey { TString Target; TMaybe ObjectType; TMaybe View; + TMaybe ExplicitPrefix; }; struct TKiDataQueryBlockSettings { diff --git a/ydb/core/kqp/provider/yql_kikimr_type_ann.cpp b/ydb/core/kqp/provider/yql_kikimr_type_ann.cpp index 0ee1f0b29c61..469848066c41 100644 --- a/ydb/core/kqp/provider/yql_kikimr_type_ann.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_type_ann.cpp @@ -223,6 +223,10 @@ class TKiSourceTypeAnnotationTransformer : public TKiSourceVisitorTransformer { { return TStatus::Ok; } + case TKikimrKey::Type::BackupCollection: + { + return TStatus::Ok; + } } return TStatus::Error; @@ -1882,6 +1886,73 @@ virtual TStatus HandleCreateTable(TKiCreateTable create, TExprContext& ctx) over return TStatus::Ok; } + static bool CheckBackupCollectionSettings(const TCoNameValueTupleList& settings, const THashSet& supported, TExprContext& ctx) { + for (const auto& setting : settings) { + auto name = setting.Name().Value(); + if (!supported.contains(TString(name))) { + ctx.AddError(TIssue(ctx.GetPosition(setting.Name().Pos()), TStringBuilder() << "Unsupported backup collection setting" + << ": " << name)); + return false; + } + } + return true; + } + + TStatus HandleCreateBackupCollection(TKiCreateBackupCollection node, TExprContext& ctx) override { + const THashSet supportedSettings = { + "incremental_backup_enabled", + "storage", + }; + + if (!CheckBackupCollectionSettings(node.BackupCollectionSettings(), supportedSettings, ctx)) { + return TStatus::Error; + } + + if (!node.Settings().Empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), "Unsupported settings")); + return TStatus::Error; + } + + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleAlterBackupCollection(TKiAlterBackupCollection node, TExprContext& ctx) override { + const THashSet supportedSettings = {}; + + if (!CheckBackupCollectionSettings(node.BackupCollectionSettings(), supportedSettings, ctx)) { + return TStatus::Error; + } + + if (!node.Settings().Empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), "Unsupported settings")); + return TStatus::Error; + } + + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleDropBackupCollection(TKiDropBackupCollection node, TExprContext&) override { + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleBackup(TKiBackup node, TExprContext&) override { + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleBackupIncremental(TKiBackupIncremental node, TExprContext&) override { + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleRestore(TKiRestore node, TExprContext&) override { + node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + virtual TStatus HandleCreateGroup(TKiCreateGroup node, TExprContext& ctx) override { Y_UNUSED(ctx); node.Ptr()->SetTypeAnn(node.World().Ref().GetTypeAnn()); diff --git a/ydb/core/kqp/ut/scheme/kqp_scheme_ut.cpp b/ydb/core/kqp/ut/scheme/kqp_scheme_ut.cpp index 95c9bb6eb3d7..00fd32b118ff 100644 --- a/ydb/core/kqp/ut/scheme/kqp_scheme_ut.cpp +++ b/ydb/core/kqp/ut/scheme/kqp_scheme_ut.cpp @@ -7366,6 +7366,310 @@ Y_UNIT_TEST_SUITE(KqpScheme) { checkDisabled(ydb->ExecuteQuery(upsertSql, settings)); NWorkload::TSampleQueries::CheckSuccess(ydb->ExecuteQuery(dropSql, settings)); } + + Y_UNIT_TEST(CreateBackupCollectionDisabledByDefault) { + TKikimrRunner kikimr; + + auto db = kikimr.GetTableClient(); + auto session = db.CreateSession().GetValueSync().GetSession(); + { + auto query = R"( + --!syntax_v1 + CREATE BACKUP COLLECTION `my_collection` WITH( + STORAGE = 'cluster' + ) + TABLE `table`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::PRECONDITION_FAILED, result.GetIssues().ToString()); + UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "Backup collections are disabled. Please contact your system administrator to enable it"); + } + } + + Y_UNIT_TEST(CreateBackupCollection) { + NKikimrConfig::TAppConfig config; + config.MutableFeatureFlags()->SetEnableBackupService(true); + + TKikimrRunner kikimr(NKqp::TKikimrSettings() + .SetAppConfig(config) + .SetEnableBackupService(true)); + + auto db = kikimr.GetTableClient(); + auto session = db.CreateSession().GetValueSync().GetSession(); + + // // negative + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // ENDPOINT = "localhost:2135", + // DATABASE = "/Root" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "CONNECTION_STRING and ENDPOINT/DATABASE are mutually exclusive"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // ENDPOINT = "localhost:2135" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "Neither CONNECTION_STRING nor ENDPOINT/DATABASE are provided"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // DATABASE = "/Root" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "Neither CONNECTION_STRING nor ENDPOINT/DATABASE are provided"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // TOKEN = "foo", + // USER = "user", + // PASSWORD = "bar" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "TOKEN and USER/PASSWORD are mutually exclusive"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // TOKEN = "foo", + // TOKEN_SECRET_NAME = "bar" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "TOKEN and TOKEN_SECRET_NAME are mutually exclusive"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // USER = "user", + // PASSWORD = "bar", + // PASSWORD_SECRET_NAME = "baz" + // ); + // )"; + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "PASSWORD and PASSWORD_SECRET_NAME are mutually exclusive"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // PASSWORD = "bar" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "USER is not provided"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // USER = "user" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "PASSWORD or PASSWORD_SECRET_NAME are not provided"); + // } + // { + // auto query = R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // CONNECTION_STRING = "grpc://localhost:2135/?database=/Root", + // STATE = "DONE" + // ); + // )"; + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::GENERIC_ERROR, result.GetIssues().ToString()); + // UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "STATE is not supported in CREATE"); + // } + { + auto query = R"( + --!syntax_v1 + CREATE BACKUP COLLECTION `/Root/.metadata/my_collection` WITH( + STORAGE = 'cluster', + INCREMENTAL_BACKUP_ENABLED = 'true' + ) + TABLE `/Root/table`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SCHEME_ERROR, result.GetIssues().ToString()); + UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "Backup collections must be placed in"); + } + + // positive + { + auto query = R"( + --!syntax_v1 + CREATE BACKUP COLLECTION `my_collection` WITH( + STORAGE = 'cluster', + INCREMENTAL_BACKUP_ENABLED = 'true' + ) + TABLE `/Root/somepath/table`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + { + auto query = R"( + --!syntax_v1 + CREATE TABLE `somepath/table` ( + `test` Uint64, + PRIMARY KEY (`test`) + ) + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + { + auto query = R"( + --!syntax_v1 + BACKUP `my_collection`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + { + auto query = R"( + --!syntax_v1 + BACKUP `my_collection` INCREMENTAL; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + // =-=-=-=- + { + auto query = R"( + --!syntax_v1 + CREATE BACKUP COLLECTION `my_collection2` WITH( + STORAGE = 'cluster', + INCREMENTAL_BACKUP_ENABLED = 'true' + ) + TABLE `/Root/somepath/table2`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + { + auto query = R"( + --!syntax_v1 + RESTORE `my_collection2`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::BAD_REQUEST, result.GetIssues().ToString()); + UNIT_ASSERT_STRING_CONTAINS(result.GetIssues().ToOneLineString(), "Nothing to restore"); + } + { + auto query = R"( + --!syntax_v1 + CREATE TABLE `.backups/collections/my_collection2/20241003211554Z_full/somepath/table2` ( + `test` Uint64, + PRIMARY KEY (`test`) + ) + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + { + auto query = R"( + --!syntax_v1 + CREATE TABLE `.backups/collections/my_collection2/20241003211712Z_incremental/somepath/table2` ( + `test` Uint64, + `__ydb_incrBackupImpl_deleted` Bool, + PRIMARY KEY (`test`) + ) + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + + { + auto query = R"( + --!syntax_v1 + RESTORE `my_collection2`; + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + // { + // auto query = Sprintf(R"( + // --!syntax_v1 + // CREATE ASYNC REPLICATION `/Root/replication` FOR + // `/Root/table` AS `/Root/replica` + // WITH ( + // ENDPOINT = "%s", + // DATABASE = "/Root" + // ); + // )", kikimr.GetEndpoint().c_str()); + + // const auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + // UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + // } + } } Y_UNIT_TEST_SUITE(KqpOlapScheme) { diff --git a/ydb/core/protos/change_exchange.proto b/ydb/core/protos/change_exchange.proto index 6e13ff76eaec..54e12b382a02 100644 --- a/ydb/core/protos/change_exchange.proto +++ b/ydb/core/protos/change_exchange.proto @@ -49,6 +49,7 @@ message TChangeRecord { TCdcHeartbeat CdcHeartbeat = 9; TDataChange IncrementalRestore = 10; } + optional uint64 TxIdSeq = 11; } message TEvHandshake { diff --git a/ydb/core/protos/counters_schemeshard.proto b/ydb/core/protos/counters_schemeshard.proto index 82ab38ce01db..8e20f1310d0e 100644 --- a/ydb/core/protos/counters_schemeshard.proto +++ b/ydb/core/protos/counters_schemeshard.proto @@ -222,6 +222,8 @@ enum ESimpleCounters { COUNTER_IN_FLIGHT_OPS_TxAlterResourcePool = 176 [(CounterOpts) = {Name: "InFlightOps/AlterResourcePool"}]; COUNTER_IN_FLIGHT_OPS_TxRestoreIncrementalBackupAtTable = 177 [(CounterOpts) = {Name: "InFlightOps/RestoreIncrementalBackupAtTable"}]; + COUNTER_BACKUP_COLLECTION_COUNT = 178 [(CounterOpts) = {Name: "BackupCollectionCount"}]; + COUNTER_IN_FLIGHT_OPS_TxCreateBackupCollection = 179 [(CounterOpts) = {Name: "InFlightOps/CreateBackupCollection"}]; } enum ECumulativeCounters { @@ -355,6 +357,8 @@ enum ECumulativeCounters { COUNTER_FINISHED_OPS_TxAlterResourcePool = 105 [(CounterOpts) = {Name: "FinishedOps/AlterResourcePool"}]; COUNTER_FINISHED_OPS_TxRestoreIncrementalBackupAtTable = 106 [(CounterOpts) = {Name: "FinishedOps/RestoreIncrementalBackupAtTable"}]; + + COUNTER_FINISHED_OPS_TxCreateBackupCollection = 107 [(CounterOpts) = {Name: "FinishedOps/CreateBackupCollection"}]; } enum EPercentileCounters { diff --git a/ydb/core/protos/flat_scheme_op.proto b/ydb/core/protos/flat_scheme_op.proto index 055733a4ffeb..447b17634d6f 100644 --- a/ydb/core/protos/flat_scheme_op.proto +++ b/ydb/core/protos/flat_scheme_op.proto @@ -1019,6 +1019,15 @@ message TRestoreIncrementalBackup { optional string DstTableName = 2; optional NKikimrProto.TPathID DstPathId = 4; + + optional uint64 SeqNo = 5; +} + +message TRestoreMultipleIncrementalBackups { + repeated string SrcTableNames = 1; + optional string DstTableName = 2; + repeated NKikimrProto.TPathID SrcPathIds = 3; + optional NKikimrProto.TPathID DstPathId = 4; } enum EIndexType { @@ -1076,6 +1085,20 @@ message TIndexCreationConfig { } } +message TGenericTxInFlyExtraData { + message TTxRestoreIncrementalBackupAtTableExtraData { + optional uint64 LoopSeqNo = 1; + } + message TTxCopyTableExtraData { + optional NKikimrProto.TPathID CdcPathId = 1; + } + + oneof TxExtraData { + TTxRestoreIncrementalBackupAtTableExtraData TxRestoreIncrementalBackupAtTableExtraData = 1; + TTxCopyTableExtraData TxCopyTableExtraData = 2; + } +} + message TIndexAlteringConfig { optional string Name = 1; optional EIndexState State = 2; @@ -1124,10 +1147,12 @@ message TCopyTableConfig { //TTableDescription implemets copying a table in orig optional bool OmitIndexes = 3 [default = false]; optional bool OmitFollowers = 4 [default = false]; optional bool IsBackup = 5 [default = false]; + optional TCreateCdcStream CreateCdcStream = 6; } message TConsistentTableCopyingConfig { repeated TCopyTableConfig CopyTableDescriptions = 1; + optional string DstBasePath = 2; } message TYTSettings { @@ -1665,6 +1690,14 @@ enum EOperationType { ESchemeOpRestoreIncrementalBackup = 103; ESchemeOpRestoreIncrementalBackupAtTable = 104; + + ESchemeOpCreateBackupCollection = 105; + ESchemeOpAlterBackupCollection = 106; + ESchemeOpDropBackupCollection = 107; + ESchemeOpBackupBackupCollection = 108; + ESchemeOpBackupIncrementalBackupCollection = 109; + ESchemeOpRestoreBackupCollection = 110; + ESchemeOpRestoreMultipleIncrementalBackups = 111; } message TApplyIf { @@ -1753,6 +1786,43 @@ message TAlterShards { } } +message TBackupCollectionProperties { + message TBackupEntry { + enum EType { + ETypeInvalid = 0; + ETypeTable = 1; + } + + optional EType Type = 1; + optional string Path = 2; + } + + message TExplicitEntryList { + repeated TBackupEntry Entries = 1; + } + + message TIncrementalBackupConfig { + + } + + oneof Entries { + google.protobuf.NullValue Database = 1; + TExplicitEntryList ExplicitEntryList = 2; + } + + // non-empty enables data-collection + optional TIncrementalBackupConfig IncrementalBackupConfig = 3; + + oneof Storage { + google.protobuf.NullValue Cluster = 4; + } +} + +message TBackupCollection { + optional string Name = 1; + optional TBackupCollectionProperties Properties = 2; +} + // Request for scheme modification // Has only one of the operations message TModifyScheme { @@ -1848,6 +1918,14 @@ message TModifyScheme { optional TResourcePoolDescription CreateResourcePool = 72; optional TRestoreIncrementalBackup RestoreIncrementalBackup = 73; + + optional TBackupCollection CreateBackupCollection = 74; + optional TBackupCollection AlterBackupCollection = 75; + optional TBackupCollection DropBackupCollection = 76; + optional TBackupCollection BackupBackupCollection = 77; + optional TBackupCollection BackupIncrementalBackupCollection = 78; + optional TBackupCollection RestoreBackupCollection = 79; + optional TRestoreMultipleIncrementalBackups RestoreMultipleIncrementalBackups = 80; } message TCopySequence { @@ -1913,6 +1991,7 @@ enum EPathType { EPathTypeExternalDataSource = 19; EPathTypeView = 20; EPathTypeResourcePool = 21; + EPathTypeBackupCollection = 22; } enum EPathSubType { @@ -1969,6 +2048,7 @@ message TPathVersion { optional uint64 ExternalDataSourceVersion = 28; optional uint64 ViewVersion = 29; optional uint64 ResourcePoolVersion = 30; + optional uint64 BackupCollectionVersion = 31; } // Describes single path @@ -2058,6 +2138,7 @@ message TPathDescription { optional TExternalDataSourceDescription ExternalDataSourceDescription = 28; optional TViewDescription ViewDescription = 29; optional TResourcePoolDescription ResourcePoolDescription = 30; + optional TBackupCollectionDescription BackupCollectionDescription = 31; } // For persisting AlterTable Tx description in Schemeshard internal DB @@ -2198,3 +2279,10 @@ message TResourcePoolDescription { optional uint64 Version = 3; optional TResourcePoolProperties Properties = 4; } + +message TBackupCollectionDescription { + optional string Name = 1; + optional NKikimrProto.TPathID PathId = 2; + optional uint64 Version = 3; + optional TBackupCollectionProperties Properties = 4; +} diff --git a/ydb/core/protos/kqp_physical.proto b/ydb/core/protos/kqp_physical.proto index ac4f99db2134..f70a3638f9d8 100644 --- a/ydb/core/protos/kqp_physical.proto +++ b/ydb/core/protos/kqp_physical.proto @@ -461,6 +461,12 @@ message TKqpSchemeOperation { NKikimrSchemeOp.TModifyScheme CreateTopic = 42; NKikimrSchemeOp.TModifyScheme AlterTopic = 43; NKikimrSchemeOp.TModifyScheme DropTopic = 44; + NKikimrSchemeOp.TModifyScheme CreateBackupCollection = 45; + NKikimrSchemeOp.TModifyScheme AlterBackupCollection = 46; + NKikimrSchemeOp.TModifyScheme DropBackupCollection = 47; + NKikimrSchemeOp.TModifyScheme Backup = 48; + NKikimrSchemeOp.TModifyScheme BackupIncremental = 49; + NKikimrSchemeOp.TModifyScheme Restore = 50; } } diff --git a/ydb/core/protos/tx_datashard.proto b/ydb/core/protos/tx_datashard.proto index 437175c2d99c..1b8ae52c0587 100644 --- a/ydb/core/protos/tx_datashard.proto +++ b/ydb/core/protos/tx_datashard.proto @@ -451,6 +451,11 @@ message TMoveIndex { optional TRemapIndexPathId ReMapIndex = 3; } +message TCreateIncrementalBackupSrc { + optional TSendSnapshot SendSnapshot = 1; + optional TCreateCdcStreamNotice CreateCdcStreamNotice = 2; +} + message TFlatSchemeTransaction { optional NKikimrSchemeOp.TTableDescription CreateTable = 1; optional NKikimrSchemeOp.TTableDescription DropTable = 2; @@ -482,6 +487,8 @@ message TFlatSchemeTransaction { optional TMoveIndex MoveIndex = 21; optional NKikimrSchemeOp.TRestoreIncrementalBackup CreateIncrementalRestoreSrc = 22; + // FIXME(+active) + optional TCreateIncrementalBackupSrc CreateIncrementalBackupSrc = 23; } message TDistributedEraseTransaction { diff --git a/ydb/core/tx/datashard/build_scheme_tx_out_rs_unit.cpp b/ydb/core/tx/datashard/build_scheme_tx_out_rs_unit.cpp index bcdbc2897a8d..93a1aa09ecb1 100644 --- a/ydb/core/tx/datashard/build_scheme_tx_out_rs_unit.cpp +++ b/ydb/core/tx/datashard/build_scheme_tx_out_rs_unit.cpp @@ -44,7 +44,7 @@ EExecutionStatus TBuildSchemeTxOutRSUnit::Execute(TOperation::TPtr op, Y_VERIFY_S(tx, "cannot cast operation of kind " << op->GetKind()); auto &schemeTx = tx->GetSchemeTx(); - if (!schemeTx.HasSendSnapshot()) + if (!schemeTx.HasSendSnapshot() && !schemeTx.HasCreateIncrementalBackupSrc()) return EExecutionStatus::Executed; Y_ABORT_UNLESS(!op->InputSnapshots().empty(), "Snapshots expected"); @@ -52,7 +52,7 @@ EExecutionStatus TBuildSchemeTxOutRSUnit::Execute(TOperation::TPtr op, auto &outReadSets = op->OutReadSets(); ui64 srcTablet = DataShard.TabletID(); - const auto& snapshot = schemeTx.GetSendSnapshot(); + auto& snapshot = schemeTx.HasSendSnapshot() ? schemeTx.GetSendSnapshot() : schemeTx.GetCreateIncrementalBackupSrc().GetSendSnapshot(); ui64 targetTablet = snapshot.GetSendTo(0).GetShard(); ui64 tableId = snapshot.GetTableId_Deprecated(); if (snapshot.HasTableId()) { diff --git a/ydb/core/tx/datashard/change_exchange_impl.h b/ydb/core/tx/datashard/change_exchange_impl.h index faac6f42d072..6ef187ca475b 100644 --- a/ydb/core/tx/datashard/change_exchange_impl.h +++ b/ydb/core/tx/datashard/change_exchange_impl.h @@ -3,12 +3,19 @@ #include "defs.h" #include "change_exchange_helpers.h" +#include + namespace NKikimr { namespace NDataShard { IActor* CreateAsyncIndexChangeSender(const TDataShardId& dataShard, const TTableId& userTableId, const TPathId& indexPathId); IActor* CreateCdcStreamChangeSender(const TDataShardId& dataShard, const TPathId& streamPathId); -IActor* CreateIncrRestoreChangeSender(const TActorId& changeServerActor, const TDataShardId& dataShard, const TTableId& userTableId, const TPathId& restoreTargetPathId); +IActor* CreateIncrRestoreChangeSender( + const TActorId& changeServerActor, + const TDataShardId& dataShard, + const TTableId& userTableId, + const TPathId& restoreTargetPathId, + std::optional seqNo = std::nullopt); } // NDataShard } // NKikimr diff --git a/ydb/core/tx/datashard/change_sender_cdc_stream.cpp b/ydb/core/tx/datashard/change_sender_cdc_stream.cpp index b1d5e9ba247d..1a3a07ef7dab 100644 --- a/ydb/core/tx/datashard/change_sender_cdc_stream.cpp +++ b/ydb/core/tx/datashard/change_sender_cdc_stream.cpp @@ -552,7 +552,7 @@ class TCdcChangeSenderMain Stream = TUserTable::TCdcStream(entry.CdcStreamInfo->Description); - Y_ABORT_UNLESS(entry.ListNodeEntry->Children.size() == 1); + Y_ABORT_UNLESS(entry.ListNodeEntry->Children.size() == 1, "%d", (int)entry.ListNodeEntry->Children.size()); const auto& topic = entry.ListNodeEntry->Children.at(0); Y_ABORT_UNLESS(topic.Kind == TNavigate::KindTopic); diff --git a/ydb/core/tx/datashard/change_sender_incr_restore.cpp b/ydb/core/tx/datashard/change_sender_incr_restore.cpp index 64209e9bb5f9..65a747cd05fe 100644 --- a/ydb/core/tx/datashard/change_sender_incr_restore.cpp +++ b/ydb/core/tx/datashard/change_sender_incr_restore.cpp @@ -132,7 +132,8 @@ class TIncrRestoreChangeSenderMain partitionId, TargetTablePathId, TagMap, - ETableChangeSenderType::IncrementalRestore); + ETableChangeSenderType::IncrementalRestore, + SeqNo); } void Handle(NChangeExchange::TEvChangeExchange::TEvEnqueueRecords::TPtr& ev) { @@ -191,11 +192,17 @@ class TIncrRestoreChangeSenderMain return NKikimrServices::TActivity::CHANGE_SENDER_INCR_RESTORE_ACTOR_MAIN; } - explicit TIncrRestoreChangeSenderMain(const TActorId& changeServerActor, const TDataShardId& dataShard, const TTableId& userTableId, const TPathId& targetPathId) + explicit TIncrRestoreChangeSenderMain( + const TActorId& changeServerActor, + const TDataShardId& dataShard, + const TTableId& userTableId, + const TPathId& targetPathId, + std::optional seqNo) : TActorBootstrapped() , TChangeSender(this, this, this, this, changeServerActor) , DataShard(dataShard) , UserTableId(userTableId) + , SeqNo(seqNo) , TargetTablePathId(targetPathId) , TargetTableVersion(0) { @@ -225,6 +232,7 @@ class TIncrRestoreChangeSenderMain private: const TDataShardId DataShard; const TTableId UserTableId; + const std::optional SeqNo; mutable TMaybe LogPrefix; THashMap MainColumnToTag; @@ -237,8 +245,14 @@ class TIncrRestoreChangeSenderMain bool FirstServe = false; }; // TIncrRestoreChangeSenderMain -IActor* CreateIncrRestoreChangeSender(const TActorId& changeServerActor, const TDataShardId& dataShard, const TTableId& userTableId, const TPathId& restoreTargetPathId) { - return new TIncrRestoreChangeSenderMain(changeServerActor, dataShard, userTableId, restoreTargetPathId); +IActor* CreateIncrRestoreChangeSender( + const TActorId& changeServerActor, + const TDataShardId& dataShard, + const TTableId& userTableId, + const TPathId& restoreTargetPathId, + std::optional seqNo) +{ + return new TIncrRestoreChangeSenderMain(changeServerActor, dataShard, userTableId, restoreTargetPathId, seqNo); } } // namespace NKikimr::NDataShard diff --git a/ydb/core/tx/datashard/change_sender_table_base.cpp b/ydb/core/tx/datashard/change_sender_table_base.cpp index 824028571b6f..d925a9ee68a6 100644 --- a/ydb/core/tx/datashard/change_sender_table_base.cpp +++ b/ydb/core/tx/datashard/change_sender_table_base.cpp @@ -150,6 +150,9 @@ class TTableChangeSenderShard: public TActorBootstrapped& tagMap, - ETableChangeSenderType type) + ETableChangeSenderType type, + std::optional seqNo) : Type(type) , Parent(parent) , DataShard(dataShard) , ShardId(shardId) , TargetTablePathId(targetTablePathId) , TagMap(tagMap) + , SeqNo(seqNo) , LeaseConfirmationCookie(0) , LastRecordOrder(0) { @@ -328,6 +333,7 @@ class TTableChangeSenderShard: public TActorBootstrapped TagMap; // from main to index + const std::optional SeqNo; mutable TMaybe LogPrefix; TActorId LeaderPipeCache; @@ -348,7 +354,8 @@ IActor* CreateTableChangeSenderShard( ui64 shardId, const TPathId& targetTablePathId, const TMap& tagMap, - ETableChangeSenderType type) + ETableChangeSenderType type, + std::optional seqNo) { return new TTableChangeSenderShard( parent, @@ -356,7 +363,8 @@ IActor* CreateTableChangeSenderShard( shardId, targetTablePathId, tagMap, - type); + type, + seqNo); } } // namespace NKikimr::NDataShard diff --git a/ydb/core/tx/datashard/change_sender_table_base.h b/ydb/core/tx/datashard/change_sender_table_base.h index b20b3348523d..abceea4587fe 100644 --- a/ydb/core/tx/datashard/change_sender_table_base.h +++ b/ydb/core/tx/datashard/change_sender_table_base.h @@ -320,6 +320,7 @@ IActor* CreateTableChangeSenderShard( ui64 shardId, const TPathId& targetTablePathId, const TMap& tagMap, - ETableChangeSenderType type); + ETableChangeSenderType type, + std::optional seqNo = std::nullopt); } // namespace NKikimr::NDataShard diff --git a/ydb/core/tx/datashard/check_scheme_tx_unit.cpp b/ydb/core/tx/datashard/check_scheme_tx_unit.cpp index cead15300fdf..82f623f8a2a8 100644 --- a/ydb/core/tx/datashard/check_scheme_tx_unit.cpp +++ b/ydb/core/tx/datashard/check_scheme_tx_unit.cpp @@ -384,6 +384,10 @@ bool TCheckSchemeTxUnit::CheckSchemeTx(TActiveTransaction *activeTx) case TSchemaOperation::ETypeCreateIncrementalRestoreSrc: res = CheckCreateIncrementalRestoreSrc(activeTx); break; + case TSchemaOperation::ETypeCreateIncrementalBackupSrc: + res = true; + // res = CheckCreateIncrementalRestoreSrc(activeTx); + break; default: LOG_ERROR_S(TActivationContext::AsActorContext(), NKikimrServices::TX_DATASHARD, "Unknown scheme tx type detected at tablet " diff --git a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp index 0bbf0261f5da..4f530de05304 100644 --- a/ydb/core/tx/datashard/create_cdc_stream_unit.cpp +++ b/ydb/core/tx/datashard/create_cdc_stream_unit.cpp @@ -25,11 +25,11 @@ class TCreateCdcStreamUnit : public TExecutionUnit { Y_VERIFY_S(tx, "cannot cast operation of kind " << op->GetKind()); auto& schemeTx = tx->GetSchemeTx(); - if (!schemeTx.HasCreateCdcStreamNotice()) { + if (!schemeTx.HasCreateCdcStreamNotice() && !schemeTx.HasCreateIncrementalBackupSrc()) { return EExecutionStatus::Executed; } - const auto& params = schemeTx.GetCreateCdcStreamNotice(); + const auto& params = schemeTx.HasCreateCdcStreamNotice() ? schemeTx.GetCreateCdcStreamNotice() : schemeTx.GetCreateIncrementalBackupSrc().GetCreateCdcStreamNotice(); const auto& streamDesc = params.GetStreamDescription(); const auto streamPathId = PathIdFromPathId(streamDesc.GetPathId()); diff --git a/ydb/core/tx/datashard/create_incremental_backup_src_unit.cpp b/ydb/core/tx/datashard/create_incremental_backup_src_unit.cpp new file mode 100644 index 000000000000..e85dc3942a0f --- /dev/null +++ b/ydb/core/tx/datashard/create_incremental_backup_src_unit.cpp @@ -0,0 +1,66 @@ +#include "defs.h" +#include "execution_unit_ctors.h" +#include "datashard_active_transaction.h" +#include "datashard_impl.h" + +// #define EXPORT_LOG_T(stream) LOG_TRACE_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_D(stream) LOG_DEBUG_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_I(stream) LOG_INFO_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_N(stream) LOG_NOTICE_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_W(stream) LOG_WARN_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_E(stream) LOG_ERROR_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) +// #define EXPORT_LOG_C(stream) LOG_CRIT_S(*TlsActivationContext, NKikimrServices::DATASHARD_BACKUP, "[Export] [" << LogPrefix() << "] " << stream) + +namespace NKikimr { +namespace NDataShard { + +class TCreateIncrementalBackupSrcUnit : public TExecutionUnit { +public: + bool IsRelevant(TActiveTransaction* tx) const { + return tx->GetSchemeTx().HasCreateIncrementalBackupSrc(); + } + + EExecutionStatus Execute(TOperation::TPtr op, TTransactionContext& txc, const TActorContext& ctx) override final { + Y_UNUSED(op, txc, ctx); + Y_ABORT_UNLESS(op->IsSchemeTx()); + + TActiveTransaction* tx = dynamic_cast(op.Get()); + Y_VERIFY_S(tx, "cannot cast operation of kind " << op->GetKind()); + + return EExecutionStatus::Executed; + + auto rv = DataShard.GetMvccTxVersion(EMvccTxMode::ReadWrite, nullptr); + + Cerr << rv.Step << " " << rv.TxId << Endl;; + + if (!IsRelevant(tx)) { + return EExecutionStatus::Reschedule; + } + + return EExecutionStatus::Reschedule; + } + + bool IsReadyToExecute(TOperation::TPtr op) const override final { + if (!op->IsWaitingForSnapshot()) { + return true; + } + + return !op->InputSnapshots().empty(); + } + + void Complete(TOperation::TPtr, const TActorContext&) override final { + } +public: + TCreateIncrementalBackupSrcUnit(TDataShard& self, TPipeline& pipeline) + : TExecutionUnit(EExecutionUnitKind::CreateIncrementalBackupSrc, false, self, pipeline) + { + } + +}; // TBackupIncrementalBackupSrcUnit + +THolder CreateIncrementalBackupSrcUnit(TDataShard& self, TPipeline& pipeline) { + return THolder(new TCreateIncrementalBackupSrcUnit(self, pipeline)); +} + +} // NDataShard +} // NKikimr 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..5d00450f9dea 100644 --- a/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp +++ b/ydb/core/tx/datashard/create_incremental_restore_src_unit.cpp @@ -59,7 +59,8 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit { THolder CreateScan( const ::NKikimrSchemeOp::TRestoreIncrementalBackup& incrBackup, - ui64 txId) + ui64 txId, + std::optional seqNo) { TPathId tablePathId = PathIdFromPathId(incrBackup.GetSrcPathId()); TPathId dstTablePathId = PathIdFromPathId(incrBackup.GetDstPathId()); @@ -67,7 +68,11 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit { return CreateIncrementalRestoreScan( DataShard.SelfId(), - [=, tabletID = DataShard.TabletID(), generation = DataShard.Generation(), tabletActor = DataShard.SelfId()](const TActorContext& ctx, TActorId parent) { + [=, + tabletID = DataShard.TabletID(), + generation = DataShard.Generation(), + tabletActor = DataShard.SelfId() + ](const TActorContext& ctx, TActorId parent) { return ctx.Register( CreateIncrRestoreChangeSender( parent, @@ -77,7 +82,8 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit { .ActorId = tabletActor, }, tablePathId, - dstTablePathId)); + dstTablePathId, + seqNo)); }, tablePathId, DataShard.GetUserTables().at(tableId), @@ -101,7 +107,13 @@ class TCreateIncrementalRestoreSrcUnit : public TExecutionUnit { Y_ABORT_UNLESS(restoreSrc.HasDstPathId()); - THolder scan{CreateScan(restoreSrc, op->GetTxId())}; + std::optional seqNo; + + if (restoreSrc.HasSeqNo()) { + seqNo = restoreSrc.GetSeqNo(); + } + + THolder scan{CreateScan(restoreSrc, op->GetTxId(), seqNo)}; auto* appData = AppData(ctx); const auto& taskName = appData->DataShardConfig.GetRestoreTaskName(); diff --git a/ydb/core/tx/datashard/datashard_active_transaction.cpp b/ydb/core/tx/datashard/datashard_active_transaction.cpp index 0992008b986b..ac4a3cf2620f 100644 --- a/ydb/core/tx/datashard/datashard_active_transaction.cpp +++ b/ydb/core/tx/datashard/datashard_active_transaction.cpp @@ -442,6 +442,7 @@ bool TActiveTransaction::BuildSchemeTx() + (ui32)SchemeTx->HasDropCdcStreamNotice() + (ui32)SchemeTx->HasMoveIndex() + (ui32)SchemeTx->HasCreateIncrementalRestoreSrc() + + (ui32)SchemeTx->HasCreateIncrementalBackupSrc() ; if (count != 1) return false; @@ -480,6 +481,8 @@ bool TActiveTransaction::BuildSchemeTx() SchemeTxType = TSchemaOperation::ETypeMoveIndex; else if (SchemeTx->HasCreateIncrementalRestoreSrc()) SchemeTxType = TSchemaOperation::ETypeCreateIncrementalRestoreSrc; + else if (SchemeTx->HasCreateIncrementalBackupSrc()) + SchemeTxType = TSchemaOperation::ETypeCreateIncrementalBackupSrc; else SchemeTxType = TSchemaOperation::ETypeUnknown; @@ -863,6 +866,7 @@ void TActiveTransaction::BuildExecutionPlan(bool loaded) plan.push_back(EExecutionUnitKind::AlterCdcStream); plan.push_back(EExecutionUnitKind::DropCdcStream); plan.push_back(EExecutionUnitKind::CreateIncrementalRestoreSrc); + plan.push_back(EExecutionUnitKind::CreateIncrementalBackupSrc); plan.push_back(EExecutionUnitKind::CompleteOperation); plan.push_back(EExecutionUnitKind::CompletedOperations); } else { diff --git a/ydb/core/tx/datashard/datashard_active_transaction.h b/ydb/core/tx/datashard/datashard_active_transaction.h index fc3d140adb7a..2326a2b971ef 100644 --- a/ydb/core/tx/datashard/datashard_active_transaction.h +++ b/ydb/core/tx/datashard/datashard_active_transaction.h @@ -55,6 +55,7 @@ struct TSchemaOperation { ETypeDropCdcStream = 15, ETypeMoveIndex = 16, ETypeCreateIncrementalRestoreSrc = 17, + ETypeCreateIncrementalBackupSrc = 18, ETypeUnknown = Max() }; @@ -111,6 +112,7 @@ struct TSchemaOperation { bool IsAlterCdcStream() const { return Type == ETypeAlterCdcStream; } bool IsDropCdcStream() const { return Type == ETypeDropCdcStream; } bool IsCreateIncrementalRestoreSrc() const { return Type == ETypeCreateIncrementalRestoreSrc; } + bool IsCreateIncrementalBackupSrc() const { return Type == ETypeCreateIncrementalBackupSrc; } bool IsReadOnly() const { return ReadOnly; } }; diff --git a/ydb/core/tx/datashard/datashard_change_receiving.cpp b/ydb/core/tx/datashard/datashard_change_receiving.cpp index 990ffd3b1f13..489bb7023d59 100644 --- a/ydb/core/tx/datashard/datashard_change_receiving.cpp +++ b/ydb/core/tx/datashard/datashard_change_receiving.cpp @@ -319,6 +319,7 @@ class TDataShard::TTxApplyChangeRecords: public TTransactionBase { auto [readVersion, writeVersion] = Self->GetReadWriteVersions(); Y_DEBUG_ABORT_UNLESS(readVersion == writeVersion); MvccReadWriteVersion = writeVersion; + Cerr << "--> --> --> " << MvccReadWriteVersion->Step << " " << MvccReadWriteVersion->TxId << " " << record.GetTxIdSeq() << Endl; Pipeline.AddCommittingOp(*MvccReadWriteVersion); } diff --git a/ydb/core/tx/datashard/datashard_impl.h b/ydb/core/tx/datashard/datashard_impl.h index edb3378a21f7..ee291f626cca 100644 --- a/ydb/core/tx/datashard/datashard_impl.h +++ b/ydb/core/tx/datashard/datashard_impl.h @@ -318,6 +318,7 @@ class TDataShard friend class TS3Downloader; template friend class TBackupRestoreUnitBase; friend class TCreateIncrementalRestoreSrcUnit; + friend class TCreateIncrementalBackupSrcUnit; friend struct TSetupSysLocks; friend class TDataShardLocksDb; diff --git a/ydb/core/tx/datashard/datashard_pipeline.h b/ydb/core/tx/datashard/datashard_pipeline.h index 823b2922a011..1dba901456db 100644 --- a/ydb/core/tx/datashard/datashard_pipeline.h +++ b/ydb/core/tx/datashard/datashard_pipeline.h @@ -172,6 +172,7 @@ class TPipeline : TNonCopyable { bool HasAlterCdcStream() const { return SchemaTx && SchemaTx->IsAlterCdcStream(); } bool HasDropCdcStream() const { return SchemaTx && SchemaTx->IsDropCdcStream(); } bool HasCreateIncrementalRestoreSrc() const { return SchemaTx && SchemaTx->IsCreateIncrementalRestoreSrc(); } + bool HasCreateIncrementalBackupSrc() const { return SchemaTx && SchemaTx->IsCreateIncrementalBackupSrc(); } ui64 CurrentSchemaTxId() const { if (SchemaTx) diff --git a/ydb/core/tx/datashard/datashard_ut_incremental_backup.cpp b/ydb/core/tx/datashard/datashard_ut_incremental_backup.cpp index 9744a9c0ac5e..af58c05eaa71 100644 --- a/ydb/core/tx/datashard/datashard_ut_incremental_backup.cpp +++ b/ydb/core/tx/datashard/datashard_ut_incremental_backup.cpp @@ -299,6 +299,75 @@ Y_UNIT_TEST_SUITE(IncrementalBackup) { "{ items { uint32_value: 3 } items { uint32_value: 30 } }"); } + Y_UNIT_TEST(MultiRestore) { + TPortManager portManager; + TServer::TPtr server = new TServer(TServerSettings(portManager.GetPort(2134), {}, DefaultPQConfig()) + .SetUseRealThreads(false) + .SetDomainName("Root") + .SetEnableChangefeedInitialScan(true) + ); + + auto& runtime = *server->GetRuntime(); + const auto edgeActor = runtime.AllocateEdgeActor(); + + SetupLogging(runtime); + InitRoot(server, edgeActor); + CreateShardedTable(server, edgeActor, "/Root", "Table", SimpleTable()); + + ExecSQL(server, edgeActor, R"( + UPSERT INTO `/Root/Table` (key, value) VALUES + (2, 2), + (3, 3); + )"); + + CreateShardedTable( + server, + edgeActor, + "/Root", + "IncrBackupImpl1", + SimpleTable() + .AllowSystemColumnNames(true) + .Columns({ + {"key", "Uint32", true, false}, + {"value", "Uint32", false, false}, + {"__ydb_incrBackupImpl_deleted", "Bool", false, false}})); + + CreateShardedTable( + server, + edgeActor, + "/Root", + "IncrBackupImpl2", + SimpleTable() + .AllowSystemColumnNames(true) + .Columns({ + {"key", "Uint32", true, false}, + {"value", "Uint32", false, false}, + {"__ydb_incrBackupImpl_deleted", "Bool", false, false}})); + + ExecSQL(server, edgeActor, R"( + UPSERT INTO `/Root/IncrBackupImpl1` (key, value, __ydb_incrBackupImpl_deleted) VALUES + (1, 10, NULL), + (2, NULL, true), + (3, 30, NULL), + (5, NULL, true); + )"); + + ExecSQL(server, edgeActor, R"( + UPSERT INTO `/Root/IncrBackupImpl2` (key, value, __ydb_incrBackupImpl_deleted) VALUES + (3, NULL, true); + )"); + + WaitTxNotification(server, edgeActor, AsyncAlterRestoreMultipleIncrementalBackups(server, "/Root", {"IncrBackupImpl1", "IncrBackupImpl2"}, "Table")); + + SimulateSleep(server, TDuration::Seconds(1)); + + UNIT_ASSERT_VALUES_EQUAL( + KqpSimpleExec(runtime, R"( + SELECT key, value FROM `/Root/Table` + )"), + "{ items { uint32_value: 1 } items { uint32_value: 10 } }"); + } + Y_UNIT_TEST(BackupRestore) { TPortManager portManager; TServer::TPtr server = new TServer(TServerSettings(portManager.GetPort(2134), {}, DefaultPQConfig()) diff --git a/ydb/core/tx/datashard/execution_unit.cpp b/ydb/core/tx/datashard/execution_unit.cpp index e2e786d956bd..1bff2551567d 100644 --- a/ydb/core/tx/datashard/execution_unit.cpp +++ b/ydb/core/tx/datashard/execution_unit.cpp @@ -152,6 +152,8 @@ THolder CreateExecutionUnit(EExecutionUnitKind kind, return CreateExecuteWriteUnit(dataShard, pipeline); case EExecutionUnitKind::CreateIncrementalRestoreSrc: return CreateIncrementalRestoreSrcUnit(dataShard, pipeline); + case EExecutionUnitKind::CreateIncrementalBackupSrc: + return CreateIncrementalBackupSrcUnit(dataShard, pipeline); default: Y_FAIL_S("Unexpected execution kind " << kind << " (" << (ui32)kind << ")"); } diff --git a/ydb/core/tx/datashard/execution_unit_ctors.h b/ydb/core/tx/datashard/execution_unit_ctors.h index 091d60dd67a5..97b20d04fddc 100644 --- a/ydb/core/tx/datashard/execution_unit_ctors.h +++ b/ydb/core/tx/datashard/execution_unit_ctors.h @@ -76,6 +76,7 @@ THolder CreateDropCdcStreamUnit(TDataShard &dataShard, TPipeline THolder CreateCheckReadUnit(TDataShard &dataShard, TPipeline &pipeline); THolder CreateReadUnit(TDataShard &dataShard, TPipeline &pipeline); THolder CreateIncrementalRestoreSrcUnit(TDataShard &dataShard, TPipeline &pipeline); +THolder CreateIncrementalBackupSrcUnit(TDataShard &dataShard, TPipeline &pipeline); } // namespace NDataShard } // namespace NKikimr diff --git a/ydb/core/tx/datashard/execution_unit_kind.h b/ydb/core/tx/datashard/execution_unit_kind.h index 69adb5af63b4..4c050272ae41 100644 --- a/ydb/core/tx/datashard/execution_unit_kind.h +++ b/ydb/core/tx/datashard/execution_unit_kind.h @@ -76,6 +76,7 @@ enum class EExecutionUnitKind: ui32 { DropCdcStream, MoveIndex, CreateIncrementalRestoreSrc, + CreateIncrementalBackupSrc, Count, Unspecified }; diff --git a/ydb/core/tx/datashard/make_snapshot_unit.cpp b/ydb/core/tx/datashard/make_snapshot_unit.cpp index 3eae9427dae9..3ee4063f97a8 100644 --- a/ydb/core/tx/datashard/make_snapshot_unit.cpp +++ b/ydb/core/tx/datashard/make_snapshot_unit.cpp @@ -47,14 +47,15 @@ EExecutionStatus TMakeSnapshotUnit::Execute(TOperation::TPtr op, Y_VERIFY_S(tx, "cannot cast operation of kind " << op->GetKind()); auto &schemeTx = tx->GetSchemeTx(); - if (!schemeTx.HasSendSnapshot()) + if (!schemeTx.HasSendSnapshot() && !schemeTx.HasCreateIncrementalBackupSrc()) return EExecutionStatus::Executed; if (!op->IsWaitingForSnapshot()) { - ui64 tableId = schemeTx.GetSendSnapshot().GetTableId_Deprecated(); - if (schemeTx.GetSendSnapshot().HasTableId()) { - Y_ABORT_UNLESS(DataShard.GetPathOwnerId() == schemeTx.GetSendSnapshot().GetTableId().GetOwnerId()); - tableId = schemeTx.GetSendSnapshot().GetTableId().GetTableId(); + auto& snapshot = schemeTx.HasSendSnapshot() ? schemeTx.GetSendSnapshot() : schemeTx.GetCreateIncrementalBackupSrc().GetSendSnapshot(); + ui64 tableId = snapshot.GetTableId_Deprecated(); + if (snapshot.HasTableId()) { + Y_ABORT_UNLESS(DataShard.GetPathOwnerId() == snapshot.GetTableId().GetOwnerId()); + tableId = snapshot.GetTableId().GetTableId(); } Y_ABORT_UNLESS(DataShard.GetUserTables().contains(tableId)); 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 2eb54a5ac0ae..c830d0d6c563 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp @@ -1864,6 +1864,24 @@ ui64 AsyncAlterRestoreIncrementalBackup( return RunSchemeTx(*server->GetRuntime(), std::move(request)); } +ui64 AsyncAlterRestoreMultipleIncrementalBackups( + Tests::TServer::TPtr server, + const TString& workingDir, + const TVector& srcTableNames, + const TString& dstTableName) +{ + auto request = SchemeTxTemplate(NKikimrSchemeOp::ESchemeOpRestoreMultipleIncrementalBackups, workingDir); + + auto& desc = *request->Record.MutableTransaction()->MutableModifyScheme()->MutableRestoreMultipleIncrementalBackups(); + for (const auto& srcTableName : srcTableNames) { + desc.AddSrcTableNames(srcTableName); + } + desc.SetDstTableName(dstTableName); + + return RunSchemeTx(*server->GetRuntime(), std::move(request)); +} + + void WaitTxNotification(Tests::TServer::TPtr server, TActorId sender, ui64 txId) { auto &runtime = *server->GetRuntime(); auto &settings = server->GetSettings(); diff --git a/ydb/core/tx/datashard/ut_common/datashard_ut_common.h b/ydb/core/tx/datashard/ut_common/datashard_ut_common.h index a92df922919e..62d8a51fed75 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.h +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.h @@ -697,6 +697,12 @@ ui64 AsyncAlterRestoreIncrementalBackup( const TString& srcTableName, const TString& dstTableName); +ui64 AsyncAlterRestoreMultipleIncrementalBackups( + Tests::TServer::TPtr server, + const TString& workingDir, + const TVector& srcTableName, + const TString& dstTableName); + struct TReadShardedTableState { TActorId Sender; TActorId Worker; diff --git a/ydb/core/tx/datashard/ya.make b/ydb/core/tx/datashard/ya.make index 97fba21abc4f..21b7cbe1bca0 100644 --- a/ydb/core/tx/datashard/ya.make +++ b/ydb/core/tx/datashard/ya.make @@ -43,6 +43,7 @@ SRCS( create_cdc_stream_unit.cpp create_persistent_snapshot_unit.cpp create_incremental_restore_src_unit.cpp + create_incremental_backup_src_unit.cpp create_table_unit.cpp create_volatile_snapshot_unit.cpp datashard.cpp diff --git a/ydb/core/tx/scheme_board/cache.cpp b/ydb/core/tx/scheme_board/cache.cpp index 890e2e0050ca..9fb9f17fc7e5 100644 --- a/ydb/core/tx/scheme_board/cache.cpp +++ b/ydb/core/tx/scheme_board/cache.cpp @@ -761,6 +761,7 @@ class TSchemeCache: public TMonitorableActor { FileStoreInfo.Drop(); ViewInfo.Drop(); ResourcePoolInfo.Drop(); + BackupCollectionInfo.Drop(); } void FillTableInfo(const NKikimrSchemeOp::TPathDescription& pathDesc) { @@ -1207,6 +1208,7 @@ class TSchemeCache: public TMonitorableActor { DESCRIPTION_PART(FileStoreInfo); DESCRIPTION_PART(ViewInfo); DESCRIPTION_PART(ResourcePoolInfo); + DESCRIPTION_PART(BackupCollectionInfo); #undef DESCRIPTION_PART @@ -1536,6 +1538,10 @@ class TSchemeCache: public TMonitorableActor { Kind = TNavigate::KindResourcePool; FillInfo(Kind, ResourcePoolInfo, std::move(*pathDesc.MutableResourcePoolDescription())); break; + case NKikimrSchemeOp::EPathTypeBackupCollection: + Kind = TNavigate::KindBackupCollection; + FillInfo(Kind, BackupCollectionInfo, std::move(*pathDesc.MutableBackupCollectionDescription())); + break; case NKikimrSchemeOp::EPathTypeInvalid: Y_DEBUG_ABORT("Invalid path type"); break; @@ -1609,6 +1615,9 @@ class TSchemeCache: public TMonitorableActor { case NKikimrSchemeOp::EPathTypeResourcePool: ListNodeEntry->Children.emplace_back(name, pathId, TNavigate::KindResourcePool); break; + case NKikimrSchemeOp::EPathTypeBackupCollection: + ListNodeEntry->Children.emplace_back(name, pathId, TNavigate::KindBackupCollection); + break; case NKikimrSchemeOp::EPathTypeTableIndex: case NKikimrSchemeOp::EPathTypeInvalid: Y_DEBUG_ABORT("Invalid path type"); @@ -1830,6 +1839,7 @@ class TSchemeCache: public TMonitorableActor { entry.FileStoreInfo = FileStoreInfo; entry.ViewInfo = ViewInfo; entry.ResourcePoolInfo = ResourcePoolInfo; + entry.BackupCollectionInfo = BackupCollectionInfo; } bool CheckColumns(TResolveContext* context, TResolve::TEntry& entry, @@ -2128,6 +2138,9 @@ class TSchemeCache: public TMonitorableActor { // ResourcePool specific TIntrusivePtr ResourcePoolInfo; + // BackupCollection specific + TIntrusivePtr BackupCollectionInfo; + }; // TCacheItem struct TMerger { diff --git a/ydb/core/tx/scheme_cache/scheme_cache.h b/ydb/core/tx/scheme_cache/scheme_cache.h index 34e0210bdd8c..91e811bf5c15 100644 --- a/ydb/core/tx/scheme_cache/scheme_cache.h +++ b/ydb/core/tx/scheme_cache/scheme_cache.h @@ -157,6 +157,7 @@ struct TSchemeCacheNavigate { KindFileStore = 20, KindView = 21, KindResourcePool = 22, + KindBackupCollection = 23, }; struct TListNodeEntry : public TAtomicRefCount { @@ -269,6 +270,11 @@ struct TSchemeCacheNavigate { NKikimrSchemeOp::TResourcePoolDescription Description; }; + struct TBackupCollectionInfo : public TAtomicRefCount { + EKind Kind = KindUnknown; + NKikimrSchemeOp::TBackupCollectionDescription Description; + }; + struct TEntry { enum class ERequestType : ui8 { ByPath, @@ -321,6 +327,7 @@ struct TSchemeCacheNavigate { TIntrusiveConstPtr FileStoreInfo; TIntrusiveConstPtr ViewInfo; TIntrusiveConstPtr ResourcePoolInfo; + TIntrusiveConstPtr BackupCollectionInfo; TString ToString() const; TString ToString(const NScheme::TTypeRegistry& typeRegistry) const; diff --git a/ydb/core/tx/schemeshard/schemeshard__init.cpp b/ydb/core/tx/schemeshard/schemeshard__init.cpp index aae82d37ac8c..1d652bfba115 100644 --- a/ydb/core/tx/schemeshard/schemeshard__init.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__init.cpp @@ -3508,9 +3508,22 @@ struct TSchemeShard::TTxInit : public TTransactionBase { if (!path->UserAttrs->AlterData) { path->UserAttrs->AlterData = new TUserAttributes(path->UserAttrs->AlterVersion + 1); } + } else if (txState.TxType == TTxState::TxCopyTable) { + if (!extraData.empty()) { + NKikimrSchemeOp::TGenericTxInFlyExtraData proto; + bool deserializeRes = ParseFromStringNoSizeLimit(proto, extraData); + Y_ABORT_UNLESS(deserializeRes); + txState.CdcPathId = PathIdFromPathId(proto.GetTxCopyTableExtraData().GetCdcPathId()); + } + } else if (txState.TxType == TTxState::TxRestoreIncrementalBackupAtTable) { + if (!extraData.empty()) { + NKikimrSchemeOp::TGenericTxInFlyExtraData proto; + bool deserializeRes = ParseFromStringNoSizeLimit(proto, extraData); + Y_ABORT_UNLESS(deserializeRes); + txState.LoopSeqNo = proto.GetTxRestoreIncrementalBackupAtTableExtraData().GetLoopSeqNo(); + } } - Y_ABORT_UNLESS(txState.TxType != TTxState::TxInvalid); Y_ABORT_UNLESS(txState.State != TTxState::Invalid); @@ -3554,7 +3567,8 @@ struct TSchemeShard::TTxInit : public TTransactionBase { TOperation::TPtr operation = Self->Operations.at(operationId.GetTxId()); Y_ABORT_UNLESS(operationId.GetSubTxId() == operation->Parts.size()); - ISubOperation::TPtr part = operation->RestorePart(txState.TxType, txState.State); + TOperationContext context{Self, txc, ctx, OnComplete, MemChanges, DbChanges}; + ISubOperation::TPtr part = operation->RestorePart(txState.TxType, txState.State, context); operation->AddPart(part); if (!txInFlightRowset.Next()) diff --git a/ydb/core/tx/schemeshard/schemeshard__operation.cpp b/ydb/core/tx/schemeshard/schemeshard__operation.cpp index 337bd757e26f..a8c90e617229 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation.cpp @@ -88,6 +88,7 @@ NKikimrScheme::TEvModifySchemeTransaction GetRecordForPrint(const NKikimrScheme: } THolder TSchemeShard::IgniteOperation(TProposeRequest& request, TOperationContext& context) { + Cerr << "--> Ignite " << request.Record.DebugString() << Endl; THolder response = nullptr; auto selfId = SelfTabletId(); @@ -770,10 +771,14 @@ TOperation::TSplitTransactionsResult TOperation::SplitIntoTransactions(const TTx break; case NKikimrSchemeOp::EOperationType::ESchemeOpCreateTable: if (tx.GetCreateTable().HasCopyFromTable()) { - result.Transactions.push_back(tx); - return result; + Y_ABORT("%s", tx.DebugString().c_str()); + // result.Transactions.push_back(tx); + // return result; } targetName = tx.GetCreateTable().GetName(); + Cerr << Endl; + Cerr << "--->" << targetName << Endl; + Cerr << Endl; break; case NKikimrSchemeOp::EOperationType::ESchemeOpCreatePersQueueGroup: targetName = tx.GetCreatePersQueueGroup().GetName(); @@ -818,6 +823,9 @@ TOperation::TSplitTransactionsResult TOperation::SplitIntoTransactions(const TTx case NKikimrSchemeOp::EOperationType::ESchemeOpCreateResourcePool: targetName = tx.GetCreateResourcePool().GetName(); break; + case NKikimrSchemeOp::EOperationType::ESchemeOpCreateBackupCollection: + targetName = tx.GetCreateBackupCollection().GetName(); + break; default: result.Transactions.push_back(tx); return result; @@ -911,6 +919,9 @@ TOperation::TSplitTransactionsResult TOperation::SplitIntoTransactions(const TTx case NKikimrSchemeOp::EOperationType::ESchemeOpCreateResourcePool: create.MutableCreateResourcePool()->SetName(name); break; + case NKikimrSchemeOp::EOperationType::ESchemeOpCreateBackupCollection: + create.MutableCreateBackupCollection()->SetName(name); + break; default: Y_UNREACHABLE(); } @@ -978,7 +989,7 @@ TOperation::TSplitTransactionsResult TOperation::SplitIntoTransactions(const TTx return result; } -ISubOperation::TPtr TOperation::RestorePart(TTxState::ETxType txType, TTxState::ETxState txState) const { +ISubOperation::TPtr TOperation::RestorePart(TTxState::ETxType txType, TTxState::ETxState txState, TOperationContext& context) const { switch (txType) { case TTxState::ETxType::TxMkDir: return CreateMkDir(NextPartId(), txState); @@ -1191,7 +1202,10 @@ ISubOperation::TPtr TOperation::RestorePart(TTxState::ETxType txType, TTxState:: return CreateAlterResourcePool(NextPartId(), txState); case TTxState::ETxType::TxRestoreIncrementalBackupAtTable: - return CreateRestoreIncrementalBackupAtTable(NextPartId(), txState); + return CreateRestoreIncrementalBackupAtTable(NextPartId(), txState, context); + + case TTxState::ETxType::TxCreateBackupCollection: + return CreateNewBackupCollection(NextPartId(), txState); case TTxState::ETxType::TxInvalid: Y_UNREACHABLE(); @@ -1446,6 +1460,24 @@ TVector TOperation::ConstructParts(const TTxTransaction& tx return CreateRestoreIncrementalBackup(NextPartId(), tx, context); case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable: Y_ABORT("multipart operations are handled before, also they require transaction details"); + + // BackupCollection + case NKikimrSchemeOp::EOperationType::ESchemeOpCreateBackupCollection: + return {CreateNewBackupCollection(NextPartId(), tx)}; + case NKikimrSchemeOp::EOperationType::ESchemeOpAlterBackupCollection: + Y_ABORT("unimplemented"); + // return {CreateAlterBackupCollection(NextPartId(), tx)}; + case NKikimrSchemeOp::EOperationType::ESchemeOpDropBackupCollection: + Y_ABORT("unimplemented"); + // return {CreateDropBackupCollection(NextPartId(), tx)}; + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupBackupCollection: + return CreateBackupBackupCollection(NextPartId(), tx, context); + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupIncrementalBackupCollection: + return CreateBackupIncrementalBackupCollection(NextPartId(), tx, context); + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreBackupCollection: + return CreateRestoreBackupCollection(NextPartId(), tx, context); + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreMultipleIncrementalBackups: + return CreateRestoreMultipleIncrementalBackups(NextPartId(), tx, context); } Y_UNREACHABLE(); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation.h b/ydb/core/tx/schemeshard/schemeshard__operation.h index d48f138b691c..45c132d56f56 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation.h +++ b/ydb/core/tx/schemeshard/schemeshard__operation.h @@ -79,7 +79,7 @@ struct TOperation: TSimpleRefCount { static TConsumeQuotaResult ConsumeQuota(const TTxTransaction& tx, TOperationContext& context); static TSplitTransactionsResult SplitIntoTransactions(const TTxTransaction& tx, const TOperationContext& context); - ISubOperation::TPtr RestorePart(TTxState::ETxType opType, TTxState::ETxState opState) const; + ISubOperation::TPtr RestorePart(TTxState::ETxType opType, TTxState::ETxState opState, TOperationContext& context) const; TVector ConstructParts(const TTxTransaction& tx, TOperationContext& context) const; void AddPart(ISubOperation::TPtr part); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_alter_continuous_backup.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_alter_continuous_backup.cpp index f47ccec86856..196a85b6bafc 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_alter_continuous_backup.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_alter_continuous_backup.cpp @@ -36,6 +36,97 @@ void DoAlterPqPart(const TOperationId& opId, const TPath& tablePath, const TPath } void DoCreateIncrBackupTable(const TOperationId& opId, const TPath& dst, NKikimrSchemeOp::TTableDescription tableDesc, TVector& result) { + + // + + { + TVector mkdirs; + + TPath path = dst.Parent(); + + const TPath parentPath = path.FirstExistedParent(); + + Cerr << " xxxxxx1 " << parentPath.PathString() << Endl; + Cerr << " xxxxxx2 " << path.PathString() << Endl; + { + TPath::TChecker checks = parentPath.Check(); + checks + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory(); + + if (!checks) { + Y_ABORT("%s", checks.GetError().c_str()); + // result.Transactions.push_back(tx); + // return result; + } + } + + while (path != parentPath) { + TPath::TChecker checks = path.Check(); + checks + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard(); + + if (path.IsResolved()) { + checks.IsResolved(); + + if (path.IsDeleted()) { + checks.IsDeleted(); + } else { + checks + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory(); + + if (checks) { + break; + } + } + } else { + checks + .NotEmpty() + .NotResolved(); + } + + if (checks) { + checks.IsValidLeafName(); + } + + if (!checks) { + Y_ABORT("%s", checks.GetError().c_str()); + // result.Status = checks.GetStatus(); + // result.Reason = checks.GetError(); + // mkdirs.clear(); + // mkdirs.push_back(tx); + // return result; + } + + const TString name = path.LeafName(); + path.Rise(); + + TTxTransaction mkdir; + mkdir.SetFailOnExist(false); + mkdir.SetAllowCreateInTempDir(false); + mkdir.SetOperationType(NKikimrSchemeOp::EOperationType::ESchemeOpMkDir); + mkdir.SetWorkingDir(path.PathString()); + mkdir.MutableMkDir()->SetName(name); + mkdirs.push_back(mkdir); + Cerr << " @ add @ " << name << " to " << path.PathString() << Endl; + } + + for (auto it = mkdirs.rbegin(); it != mkdirs.rend(); ++it) { + result.push_back(CreateMkDir(NextPartId(opId, result), std::move(*it))); + } + } + + // + auto outTx = TransactionTemplate(dst.Parent().PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateTable); // outTx.SetFailOnExist(!acceptExisted); @@ -66,16 +157,21 @@ void DoCreateIncrBackupTable(const TOperationId& opId, const TPath& dst, NKikimr result.push_back(CreateNewTable(NextPartId(opId, result), outTx)); } -TVector CreateAlterContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + +bool CreateAlterContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context, TVector& result) { Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpAlterContinuousBackup); const auto workingDirPath = TPath::Resolve(tx.GetWorkingDir(), context.SS); const auto& cbOp = tx.GetAlterContinuousBackup(); const auto& tableName = cbOp.GetTableName(); - const auto checksResult = NCdc::DoAlterStreamPathChecks(opId, workingDirPath, tableName, NBackup::CB_CDC_STREAM_NAME); + const auto tablePathNameX = workingDirPath.Child(tableName); + const auto tablePathName = TPath::Resolve(tablePathNameX.PathString(), context.SS); + + const auto checksResult = NCdc::DoAlterStreamPathChecks(opId, tablePathName.Parent(), tablePathName.LeafName(), NBackup::CB_CDC_STREAM_NAME); if (std::holds_alternative(checksResult)) { - return {std::get(checksResult)}; + result = {std::get(checksResult)}; + return false; } const auto [tablePath, streamPath] = std::get(checksResult); @@ -84,7 +180,8 @@ TVector CreateAlterContinuousBackup(TOperationId opId, cons const auto topicPath = streamPath.Child("streamImpl"); TTopicInfo::TPtr topic = context.SS->Topics.at(topicPath.Base()->PathId); - const auto backupTablePath = workingDirPath.Child(cbOp.GetTakeIncrementalBackup().GetDstPath()); + const auto backupTablePathX = workingDirPath.Child(cbOp.GetTakeIncrementalBackup().GetDstPath()); + const auto backupTablePath = TPath::Resolve(backupTablePathX.PathString(), context.SS); const NScheme::TTypeRegistry* typeRegistry = AppData(context.Ctx)->TypeRegistry; @@ -92,17 +189,21 @@ TVector CreateAlterContinuousBackup(TOperationId opId, cons context.SS->DescribeTable(*table, typeRegistry, true, &schema); schema.MutablePartitionConfig()->CopyFrom(table->TableDescription.GetPartitionConfig()); + schema.MutablePartitionConfig()->ClearStorageRooms(); + TString errStr; if (!context.SS->CheckApplyIf(tx, errStr)) { - return {CreateReject(opId, NKikimrScheme::StatusPreconditionFailed, errStr)}; + result = {CreateReject(opId, NKikimrScheme::StatusPreconditionFailed, errStr)}; + return false; } if (!context.SS->CheckLocks(tablePath.Base()->PathId, tx, errStr)) { - return {CreateReject(opId, NKikimrScheme::StatusMultipleModifications, errStr)}; + result = {CreateReject(opId, NKikimrScheme::StatusMultipleModifications, errStr)}; + return false; } NKikimrSchemeOp::TAlterCdcStream alterCdcStreamOp; - alterCdcStreamOp.SetTableName(tableName); + alterCdcStreamOp.SetTableName(tablePathName.LeafName()); alterCdcStreamOp.SetStreamName(NBackup::CB_CDC_STREAM_NAME); switch (cbOp.GetActionCase()) { @@ -111,19 +212,26 @@ TVector CreateAlterContinuousBackup(TOperationId opId, cons alterCdcStreamOp.MutableDisable(); break; default: - return {CreateReject(opId, NKikimrScheme::StatusInvalidParameter, TStringBuilder() + result = {CreateReject(opId, NKikimrScheme::StatusInvalidParameter, TStringBuilder() << "Unknown action: " << static_cast(cbOp.GetActionCase()))}; + return false; } - TVector result; - - NCdc::DoAlterStream(result, alterCdcStreamOp, opId, workingDirPath, tablePath); + NCdc::DoAlterStream(result, alterCdcStreamOp, opId, tablePathName.Parent(), tablePath); if (cbOp.GetActionCase() == NKikimrSchemeOp::TAlterContinuousBackup::kTakeIncrementalBackup) { DoCreateIncrBackupTable(opId, backupTablePath, schema, result); DoAlterPqPart(opId, backupTablePath, topicPath, topic, result); } + return true; +} + +TVector CreateAlterContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + TVector result; + + CreateAlterContinuousBackup(opId, tx, context, result); + return result; } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_backup_backup_collection.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_backup_backup_collection.cpp new file mode 100644 index 000000000000..f527ee2075ad --- /dev/null +++ b/ydb/core/tx/schemeshard/schemeshard__operation_backup_backup_collection.cpp @@ -0,0 +1,244 @@ +#include "schemeshard__operation_common.h" +#include "schemeshard__operation_create_cdc_stream.h" +#include "schemeshard_impl.h" + +#include + +#define LOG_I(stream) LOG_INFO_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "[" << context.SS->TabletID() << "] " << stream) +#define LOG_N(stream) LOG_NOTICE_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "[" << context.SS->TabletID() << "] " << stream) +#define RETURN_RESULT_UNLESS(x) if (!(x)) return result; + +namespace NKikimr::NSchemeShard { + +TString ToX509String(const TInstant& datetime) { + return datetime.FormatLocalTime("%Y%m%d%H%M%SZ"); +} + +TVector CreateBackupBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + Y_UNUSED(opId, tx, context); + + NKikimrSchemeOp::TModifyScheme modifyScheme; + modifyScheme.SetOperationType(NKikimrSchemeOp::ESchemeOpCreateConsistentCopyTables); + modifyScheme.SetInternal(true); + + auto& cct = *modifyScheme.MutableCreateConsistentCopyTables(); + auto& copyTables = *cct.MutableCopyTableDescriptions(); + const auto workingDirPath = TPath::Resolve(tx.GetWorkingDir(), context.SS); + + // Y_ABORT("%s %s", tx.GetWorkingDir().c_str(), tx.GetBackupBackupCollection().GetName().c_str()); + // FIXME(+active) + + TString bcPathStr = JoinPath({tx.GetWorkingDir().c_str(), tx.GetBackupBackupCollection().GetName().c_str()}); + // TString targetPathStr = JoinPath({tx.GetWorkingDir().c_str(), tx.GetBackupBackupCollection().GetName().c_str(), "0"}); + + const TPath& bcPath = TPath::Resolve(bcPathStr, context.SS); + const auto& bc = context.SS->BackupCollections[bcPath->PathId]; + // copyTables.Reserve(exportInfo->Items.size()); + + // const TPath exportPath = TPath::Init(exportInfo->ExportPathId, ss); + // const TString& exportPathName = exportPath.PathString(); + + cct.SetDstBasePath(bcPathStr); + + TVector result; + + size_t cutLen = bcPath.GetDomainPathString().size() + 1; + + auto now = ToX509String(TlsActivationContext->AsActorContext().Now()); + + for (const auto& item : bc->Properties.GetExplicitEntryList().GetEntries()) { + NKikimrSchemeOp::TCreateCdcStream createCdcStreamOp; + createCdcStreamOp.SetTableName(item.GetPath()); + auto& streamDescription = *createCdcStreamOp.MutableStreamDescription(); + streamDescription.SetName(NBackup::CB_CDC_STREAM_NAME); + streamDescription.SetMode(NKikimrSchemeOp::ECdcStreamModeUpdate); + streamDescription.SetFormat(NKikimrSchemeOp::ECdcStreamFormatProto); + + const auto sPath = TPath::Resolve(item.GetPath(), context.SS); + NCdc::DoCreateStreamImpl(result, createCdcStreamOp, opId, workingDirPath, sPath, false, false); + + auto& desc = *copyTables.Add(); + desc.SetSrcPath(item.GetPath()); + desc.SetDstPath(now + "_full/" + item.GetPath().substr(cutLen, item.GetPath().size() - cutLen)); + desc.SetOmitIndexes(true); + desc.SetOmitFollowers(true); + // desc.SetIsBackup(true); + desc.MutableCreateCdcStream()->CopyFrom(createCdcStreamOp); + } + + CreateConsistentCopyTables(opId, modifyScheme, context, result); + + for (const auto& item : bc->Properties.GetExplicitEntryList().GetEntries()) { + NKikimrSchemeOp::TCreateCdcStream createCdcStreamOp; + createCdcStreamOp.SetTableName(item.GetPath()); + auto& streamDescription = *createCdcStreamOp.MutableStreamDescription(); + streamDescription.SetName(NBackup::CB_CDC_STREAM_NAME); + streamDescription.SetMode(NKikimrSchemeOp::ECdcStreamModeUpdate); + streamDescription.SetFormat(NKikimrSchemeOp::ECdcStreamFormatProto); + + const auto sPath = TPath::Resolve(item.GetPath(), context.SS); + auto table = context.SS->Tables.at(sPath.Base()->PathId); + + TVector boundaries; + const auto& partitions = table->GetPartitions(); + boundaries.reserve(partitions.size() - 1); + + for (ui32 i = 0; i < partitions.size(); ++i) { + const auto& partition = partitions.at(i); + if (i != partitions.size() - 1) { + boundaries.push_back(partition.EndOfRange); + } + } + + const auto streamPath = sPath.Child(NBackup::CB_CDC_STREAM_NAME); + + NCdc::DoCreatePqPart(result, createCdcStreamOp, opId, streamPath, NBackup::CB_CDC_STREAM_NAME, table, boundaries, false); + } + + return result; +} + +TVector CreateBackupIncrementalBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + Y_UNUSED(opId, tx, context); + + TString bcPathStr = JoinPath({tx.GetWorkingDir().c_str(), tx.GetBackupIncrementalBackupCollection().GetName().c_str()}); + + const TPath& bcPath = TPath::Resolve(bcPathStr, context.SS); + const auto& bc = context.SS->BackupCollections[bcPath->PathId]; + + size_t cutLen = bcPath.GetDomainPathString().size() + 1; + + TVector result; + + auto now = ToX509String(TlsActivationContext->AsActorContext().Now()); + + for (const auto& item : bc->Properties.GetExplicitEntryList().GetEntries()) { + NKikimrSchemeOp::TModifyScheme modifyScheme; + modifyScheme.SetWorkingDir(tx.GetWorkingDir()); + modifyScheme.SetOperationType(NKikimrSchemeOp::ESchemeOpAlterContinuousBackup); + modifyScheme.SetInternal(true); + auto& cb = *modifyScheme.MutableAlterContinuousBackup(); + cb.SetTableName(item.GetPath().substr(cutLen, item.GetPath().size() - cutLen)); + auto& ib = *cb.MutableTakeIncrementalBackup(); + ib.SetDstPath(bcPathStr.substr(cutLen, bcPathStr.size() - cutLen) + "/" + now + "_incremental/" + item.GetPath().substr(cutLen, item.GetPath().size() - cutLen)); + + if (!CreateAlterContinuousBackup(opId, modifyScheme, context, result)) { + return result; + } + } + + return result; +} + +TVector CreateRestoreBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + Y_UNUSED(opId, tx, context); + + TString bcPathStr = JoinPath({tx.GetWorkingDir().c_str(), tx.GetRestoreBackupCollection().GetName().c_str()}); + + const TPath& bcPath = TPath::Resolve(bcPathStr, context.SS); + + const auto& bc = context.SS->BackupCollections[bcPath->PathId]; + + // (1) iterate over bc children and find last __full__ + // (2) save __full__ to var and all consequent __incremental__ to array + // (3) run consistent copy-tables in __full__ + // (3.1) expand consistent copy-tables with restore-list + + // # restore-list (O(n^2) if ds sends to next ds, better orchestrate through schemeshard O(n)) + // (1) send propose to all __full__ and __incremental__ + // (2) when __full__ finish with full backup it strongly sends *continue* to first __incremental__ + // (3) when __incremental__ finishes restore it sends *continue* to next __incremental__ + // (4) when all parts finished schemeshard finishes the operation + + TString lastFullBackupName; + TVector incBackupNames; + + if (!bcPath.Base()->GetChildren().size()) { + return {CreateReject(opId, NKikimrScheme::StatusInvalidParameter, TStringBuilder() << "Nothing to restore")}; + } else { + for (auto& [child, _] : bcPath.Base()->GetChildren()) { + if (child.EndsWith("_full")) { + lastFullBackupName = child; + incBackupNames.clear(); + } else if (child.EndsWith("_incremental")) { + incBackupNames.push_back(child); + } + } + } + + Cerr << "going to restore:" << Endl; + Cerr << "\tFullBackup:" << Endl; + Cerr << "\t\t" << bcPath.Child(lastFullBackupName).PathString() << Endl; + Cerr << "\tIncrBackups" << Endl; + Cerr << "\t\t" << ::JoinSeq(", ", incBackupNames) << Endl; + + NKikimrSchemeOp::TModifyScheme modifyScheme; + modifyScheme.SetOperationType(NKikimrSchemeOp::ESchemeOpCreateConsistentCopyTables); + modifyScheme.SetInternal(true); + + auto& cct = *modifyScheme.MutableCreateConsistentCopyTables(); + auto& copyTables = *cct.MutableCopyTableDescriptions(); + const auto workingDirPath = TPath::Resolve(tx.GetWorkingDir(), context.SS); + + // cct.SetDstBasePath(bcPath.GetDomainPathString()); + + TVector result; + + cct.SetDstBasePath(bcPath.GetDomainPathString()); + + size_t cutLen = bcPath.GetDomainPathString().size() + 1; + + for (const auto& item : bc->Properties.GetExplicitEntryList().GetEntries()) { + Y_UNUSED(item, copyTables); + + auto& desc = *copyTables.Add(); + desc.SetSrcPath(bcPath.Child(lastFullBackupName).PathString() + item.GetPath().substr(cutLen - 1, item.GetPath().size() - cutLen + 1)); + desc.SetDstPath(item.GetPath().substr(cutLen, item.GetPath().size() - cutLen)); + desc.SetOmitIndexes(true); + desc.SetOmitFollowers(true); + // desc.SetIsBackup(true); + } + + CreateConsistentCopyTables(opId, modifyScheme, context, result); + + TVector postponed; + for (const auto& item : bc->Properties.GetExplicitEntryList().GetEntries()) { + NKikimrSchemeOp::TModifyScheme modifyScheme2; + modifyScheme2.SetOperationType(NKikimrSchemeOp::ESchemeOpRestoreMultipleIncrementalBackups); + modifyScheme2.SetInternal(true); + + auto& desc = *modifyScheme2.MutableRestoreMultipleIncrementalBackups(); + for (const auto& incr : incBackupNames) { + auto path = bcPath.Child(incr).PathString() + item.GetPath().substr(cutLen - 1, item.GetPath().size() - cutLen + 1); + Cerr << "set src _@@@_ " << path << Endl; + desc.AddSrcTableNames(path); + } + desc.SetDstTableName(item.GetPath()); + + postponed.push_back(modifyScheme2); + } + + // oh... + context.SS->HackPostponedOps[opId.GetTxId()] = [toActivate = std::move(postponed), tabletId = context.SS->TabletID()]( + const TActorContext& ctx, + TActorId self, + TEvTxAllocatorClient::TEvAllocateResult::TPtr& allocateResult) + { + const auto txId = TTxId(allocateResult->Get()->TxIds.front()); + auto propose = MakeHolder(txId.GetValue(), tabletId); + auto& record = propose->Record; + + for (const auto& req : toActivate) { + record.AddTransaction()->CopyFrom(req); + } + + ctx.Send(self, static_cast(propose.Release()), 0, 0); + }; + + + // // + + return result; +} + +} // namespace NKikimr::NSchemeShard diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_common.h b/ydb/core/tx/schemeshard/schemeshard__operation_common.h index 701a0f00726a..f884245675f7 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_common.h +++ b/ydb/core/tx/schemeshard/schemeshard__operation_common.h @@ -506,10 +506,10 @@ class TDone: public TSubOperationState { const auto& pathId = txState->TargetPathId; Y_ABORT_UNLESS(context.SS->PathsById.contains(pathId)); TPathElement::TPtr path = context.SS->PathsById.at(pathId); - Y_VERIFY_S(path->PathState != TPathElement::EPathState::EPathStateNoChanges, "with context" - << ", PathState: " << NKikimrSchemeOp::EPathState_Name(path->PathState) - << ", PathId: " << path->PathId - << ", PathName: " << path->Name); + // Y_VERIFY_S(path->PathState != TPathElement::EPathState::EPathStateNoChanges, "with context" + // << ", PathState: " << NKikimrSchemeOp::EPathState_Name(path->PathState) + // << ", PathId: " << path->PathId + // << ", PathName: " << path->Name); if (path->IsPQGroup() && txState->IsCreate()) { TPathElement::TPtr parentDir = context.SS->PathsById.at(path->ParentPathId); diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_consistent_copy_tables.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_consistent_copy_tables.cpp index c9b3a1ab0ca4..3d86259fe650 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_consistent_copy_tables.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_consistent_copy_tables.cpp @@ -9,17 +9,28 @@ #include -NKikimrSchemeOp::TModifyScheme CopyTableTask(NKikimr::NSchemeShard::TPath& src, NKikimr::NSchemeShard::TPath& dst, bool omitFollowers, bool isBackup) { +NKikimrSchemeOp::TModifyScheme CopyTableTask( + NKikimr::NSchemeShard::TPath& src, + const TString& dstWorkingDir, + const TString& dstLeaf, + bool omitFollowers, + bool isBackup, + const NKikimrSchemeOp::TCopyTableConfig& descr) +{ using namespace NKikimr::NSchemeShard; - auto scheme = TransactionTemplate(dst.Parent().PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateTable); + auto scheme = TransactionTemplate(dstWorkingDir, NKikimrSchemeOp::EOperationType::ESchemeOpCreateTable); scheme.SetFailOnExist(true); - auto operation = scheme.MutableCreateTable(); - operation->SetName(dst.LeafName()); + auto* operation = scheme.MutableCreateTable(); + operation->SetName(dstLeaf); operation->SetCopyFromTable(src.PathString()); operation->SetOmitFollowers(omitFollowers); operation->SetIsBackup(isBackup); + if (descr.HasCreateCdcStream()) { + auto* coOp = scheme.MutableCreateCdcStream(); + coOp->CopyFrom(descr.GetCreateCdcStream()); + } return scheme; } @@ -47,53 +58,58 @@ NKikimrSchemeOp::TModifyScheme CreateIndexTask(NKikimr::NSchemeShard::TTableInde namespace NKikimr::NSchemeShard { -TVector CreateConsistentCopyTables(TOperationId nextId, const TTxTransaction& tx, TOperationContext& context) { +void CreateConsistentCopyTables( + TOperationId nextId, + const TTxTransaction& tx, + TOperationContext& context, + TVector& result) +{ Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpCreateConsistentCopyTables); const auto& op = tx.GetCreateConsistentCopyTables(); if (0 == op.CopyTableDescriptionsSize()) { TString msg = TStringBuilder() << "no task to do, empty list CopyTableDescriptions"; - return {CreateReject(nextId, NKikimrScheme::EStatus::StatusInvalidParameter, msg)}; + result = {CreateReject(nextId, NKikimrScheme::EStatus::StatusInvalidParameter, msg)}; + return; } TPath firstPath = TPath::Resolve(op.GetCopyTableDescriptions(0).GetSrcPath(), context.SS); - { - auto checks = TPath::TChecker(firstPath); - checks - .NotEmpty() - .NotUnderDomainUpgrade() - .IsAtLocalSchemeShard(); - - if (!checks) { - return {CreateReject(nextId, checks.GetStatus(), checks.GetError())}; - } - } - - const auto allForBackup = AllOf(op.GetCopyTableDescriptions(), [](const auto& item) { - return item.GetIsBackup(); - }); - - const auto& limits = firstPath.DomainInfo()->GetSchemeLimits(); - const auto limit = allForBackup - ? Max(limits.MaxObjectsInBackup, limits.MaxConsistentCopyTargets) - : limits.MaxConsistentCopyTargets; - - if (op.CopyTableDescriptionsSize() > limit) { - return {CreateReject(nextId, NKikimrScheme::EStatus::StatusInvalidParameter, TStringBuilder() - << "Consistent copy object count limit exceeded" - << ", limit: " << limit - << ", objects: " << op.CopyTableDescriptionsSize() - )}; - } + // { + // auto checks = TPath::TChecker(firstPath); + // checks + // .NotEmpty() + // .NotUnderDomainUpgrade() + // .IsAtLocalSchemeShard(); + + // if (!checks) { + // return {CreateReject(nextId, checks.GetStatus(), checks.GetError())}; + // } + // } + + // const auto allForBackup = AllOf(op.GetCopyTableDescriptions(), [](const auto& item) { + // return item.GetIsBackup(); + // }); + + // const auto& limits = firstPath.DomainInfo()->GetSchemeLimits(); + // const auto limit = allForBackup + // ? Max(limits.MaxObjectsInBackup, limits.MaxConsistentCopyTargets) + // : limits.MaxConsistentCopyTargets; + + // if (op.CopyTableDescriptionsSize() > limit) { + // return {CreateReject(nextId, NKikimrScheme::EStatus::StatusInvalidParameter, TStringBuilder() + // << "Consistent copy object count limit exceeded" + // << ", limit: " << limit + // << ", objects: " << op.CopyTableDescriptionsSize() + // )}; + // } TString errStr; if (!context.SS->CheckApplyIf(tx, errStr)) { - return {CreateReject(nextId, NKikimrScheme::EStatus::StatusPreconditionFailed, errStr)}; + result = {CreateReject(nextId, NKikimrScheme::EStatus::StatusPreconditionFailed, errStr)}; + return; } - TVector result; - for (const auto& descr: op.GetCopyTableDescriptions()) { const auto& srcStr = descr.GetSrcPath(); const auto& dstStr = descr.GetDstPath(); @@ -108,12 +124,99 @@ TVector CreateConsistentCopyTables(TOperationId nextId, con .IsTheSameDomain(firstPath); if (!checks) { - return {CreateReject(nextId, checks.GetStatus(), checks.GetError())}; + Y_ABORT("%s, %s", srcPath.PathString().c_str(), checks.GetError().c_str()); + result = {CreateReject(nextId, checks.GetStatus(), checks.GetError())}; + return; + } + } + + if (op.HasDstBasePath()) { + TVector mkdirs; + + auto dstBaseStr = op.GetDstBasePath(); + TPath path = TPath::Resolve(JoinPath({dstBaseStr, dstStr}), context.SS).Parent(); + + const TPath parentPath = TPath::Resolve(dstBaseStr, context.SS); + { + TPath::TChecker checks = parentPath.Check(); + checks + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory(); + + if (!checks) { + Y_ABORT("%s", checks.GetError().c_str()); + // result.Transactions.push_back(tx); + // return result; + } + } + + while (path != parentPath) { + TPath::TChecker checks = path.Check(); + checks + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard(); + + if (path.IsResolved()) { + checks.IsResolved(); + + if (path.IsDeleted()) { + checks.IsDeleted(); + } else { + checks + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory(); + + if (checks) { + break; + } + } + } else { + checks + .NotEmpty() + .NotResolved(); + } + + if (checks) { + checks.IsValidLeafName(); + } + + if (!checks) { + Y_ABORT("%s", checks.GetError().c_str()); + // result.Status = checks.GetStatus(); + // result.Reason = checks.GetError(); + // mkdirs.clear(); + // mkdirs.push_back(tx); + // return result; + } + + const TString name = path.LeafName(); + path.Rise(); + + TTxTransaction mkdir; + mkdir.SetFailOnExist(false); + mkdir.SetAllowCreateInTempDir(tx.GetAllowCreateInTempDir()); + mkdir.SetOperationType(NKikimrSchemeOp::EOperationType::ESchemeOpMkDir); + mkdir.SetWorkingDir(path.PathString()); + mkdir.MutableMkDir()->SetName(name); + mkdirs.push_back(mkdir); + Cerr << " @ add @ " << name << " to " << path.PathString() << Endl; + } + + for (auto it = mkdirs.rbegin(); it != mkdirs.rend(); ++it) { + result.push_back(CreateMkDir(NextPartId(nextId, result), std::move(*it))); } } - TPath dstPath = TPath::Resolve(dstStr, context.SS); - TPath dstParentPath = dstPath.Parent(); + Cerr << "<----- CopyTables" << Endl; + Cerr << dstStr << Endl; + TPath dstPath = op.HasDstBasePath() ? TPath::Resolve(JoinPath({op.GetDstBasePath(), dstStr}), context.SS) : TPath::Resolve(dstStr, context.SS); THashSet sequences; for (const auto& child: srcPath.Base()->GetChildren()) { @@ -135,63 +238,82 @@ TVector CreateConsistentCopyTables(TOperationId nextId, con } result.push_back(CreateCopyTable(NextPartId(nextId, result), - CopyTableTask(srcPath, dstPath, descr.GetOmitFollowers(), descr.GetIsBackup()), sequences)); - - TVector sequenceDescriptions; - for (const auto& child: srcPath.Base()->GetChildren()) { - const auto& name = child.first; - const auto& pathId = child.second; - - TPath srcIndexPath = srcPath.Child(name); - TPath dstIndexPath = dstPath.Child(name); - - if (srcIndexPath.IsDeleted()) { - continue; - } - - if (srcIndexPath.IsSequence()) { - TSequenceInfo::TPtr sequenceInfo = context.SS->Sequences.at(pathId); - const auto& sequenceDesc = sequenceInfo->Description; - sequenceDescriptions.push_back(sequenceDesc); - continue; - } - - if (descr.GetOmitIndexes()) { - continue; - } - - if (!srcIndexPath.IsTableIndex()) { - continue; - } - - Y_ABORT_UNLESS(srcIndexPath.Base()->PathId == pathId); - Y_VERIFY_S(srcIndexPath.Base()->GetChildren().size() == 1, srcIndexPath.PathString() << " has children " << srcIndexPath.Base()->GetChildren().size() << " but 1 expected"); - - TTableIndexInfo::TPtr indexInfo = context.SS->Indexes.at(pathId); - result.push_back(CreateNewTableIndex(NextPartId(nextId, result), CreateIndexTask(indexInfo, dstIndexPath))); - - TString srcImplTableName = srcIndexPath.Base()->GetChildren().begin()->first; - TPath srcImplTable = srcIndexPath.Child(srcImplTableName); - Y_ABORT_UNLESS(srcImplTable.Base()->PathId == srcIndexPath.Base()->GetChildren().begin()->second); - TPath dstImplTable = dstIndexPath.Child(srcImplTableName); - - result.push_back(CreateCopyTable(NextPartId(nextId, result), - CopyTableTask(srcImplTable, dstImplTable, descr.GetOmitFollowers(), descr.GetIsBackup()))); - } - - for (auto&& sequenceDescription : sequenceDescriptions) { - auto scheme = TransactionTemplate( - dstPath.PathString(), - NKikimrSchemeOp::EOperationType::ESchemeOpCreateSequence); - scheme.SetFailOnExist(true); + CopyTableTask(srcPath, + // op.HasDstBasePath() ? op.GetDstBasePath() : dstPath.Parent().PathString(), + // op.HasDstBasePath() ? dstStr : dstPath.LeafName(), + dstPath.Parent().PathString(), + dstPath.LeafName(), + descr.GetOmitFollowers(), + descr.GetIsBackup(), + descr), + sequences)); + + // TVector sequenceDescriptions; + // for (const auto& child: srcPath.Base()->GetChildren()) { + // const auto& name = child.first; + // const auto& pathId = child.second; + + // TPath srcIndexPath = srcPath.Child(name); + // TPath dstIndexPath = dstPath.Child(name); + + // if (srcIndexPath.IsDeleted()) { + // continue; + // } + + // if (srcIndexPath.IsSequence()) { + // TSequenceInfo::TPtr sequenceInfo = context.SS->Sequences.at(pathId); + // const auto& sequenceDesc = sequenceInfo->Description; + // sequenceDescriptions.push_back(sequenceDesc); + // continue; + // } + + // if (descr.GetOmitIndexes()) { + // continue; + // } + + // if (!srcIndexPath.IsTableIndex()) { + // continue; + // } + + // Y_ABORT_UNLESS(srcIndexPath.Base()->PathId == pathId); + // Y_VERIFY_S(srcIndexPath.Base()->GetChildren().size() == 1, srcIndexPath.PathString() << " has children " << srcIndexPath.Base()->GetChildren().size() << " but 1 expected"); + + // TTableIndexInfo::TPtr indexInfo = context.SS->Indexes.at(pathId); + // result.push_back(CreateNewTableIndex(NextPartId(nextId, result), CreateIndexTask(indexInfo, dstIndexPath))); + + // TString srcImplTableName = srcIndexPath.Base()->GetChildren().begin()->first; + // TPath srcImplTable = srcIndexPath.Child(srcImplTableName); + // Y_ABORT_UNLESS(srcImplTable.Base()->PathId == srcIndexPath.Base()->GetChildren().begin()->second); + // TPath dstImplTable = dstIndexPath.Child(srcImplTableName); + + // result.push_back(CreateCopyTable(NextPartId(nextId, result), + // CopyTableTask( + // srcImplTable, + // dstImplTable.Parent().PathString(), + // dstImplTable.LeafName(), + // descr.GetOmitFollowers(), + // descr.GetIsBackup()))); + // } + + // for (auto&& sequenceDescription : sequenceDescriptions) { + // auto scheme = TransactionTemplate( + // dstPath.PathString(), + // NKikimrSchemeOp::EOperationType::ESchemeOpCreateSequence); + // scheme.SetFailOnExist(true); + + // auto* copySequence = scheme.MutableCopySequence(); + // copySequence->SetCopyFrom(srcPath.PathString() + "/" + sequenceDescription.GetName()); + // *scheme.MutableSequence() = std::move(sequenceDescription); + + // result.push_back(CreateCopySequence(NextPartId(nextId, result), scheme)); + // } + } +} - auto* copySequence = scheme.MutableCopySequence(); - copySequence->SetCopyFrom(srcPath.PathString() + "/" + sequenceDescription.GetName()); - *scheme.MutableSequence() = std::move(sequenceDescription); +TVector CreateConsistentCopyTables(TOperationId nextId, const TTxTransaction& tx, TOperationContext& context) { + TVector result; - result.push_back(CreateCopySequence(NextPartId(nextId, result), scheme)); - } - } + CreateConsistentCopyTables(nextId, tx, context, result); return result; } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_copy_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_copy_table.cpp index d78844253bde..dcbb1f2264b8 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_copy_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_copy_table.cpp @@ -51,12 +51,59 @@ class TConfigureParts: public TSubOperationState { return NTableState::CollectProposeTransactionResults(OperationId, ev, context); } + static void FillSrcSnapshot(const TTxState* const txState, ui64 dstDatashardId, NKikimrTxDataShard::TSendSnapshot& snapshot) { + snapshot.SetTableId_Deprecated(txState->SourcePathId.LocalPathId); + snapshot.MutableTableId()->SetOwnerId(txState->SourcePathId.OwnerId); + snapshot.MutableTableId()->SetTableId(txState->SourcePathId.LocalPathId); + snapshot.AddSendTo()->SetShard(dstDatashardId); + } + + static void FillNotice(const TPathId& pathId, TOperationContext& context, NKikimrTxDataShard::TCreateCdcStreamNotice& notice) { + Y_ABORT_UNLESS(context.SS->PathsById.contains(pathId)); + auto path = context.SS->PathsById.at(pathId); + + Y_ABORT_UNLESS(context.SS->Tables.contains(pathId)); + auto table = context.SS->Tables.at(pathId); + + PathIdFromPathId(pathId, notice.MutablePathId()); + notice.SetTableSchemaVersion(table->AlterVersion + 1); + + bool found = false; + for (const auto& [childName, childPathId] : path->GetChildren()) { + Y_ABORT_UNLESS(context.SS->PathsById.contains(childPathId)); + auto childPath = context.SS->PathsById.at(childPathId); + + if (!childPath->IsCdcStream() || childPath->Dropped()) { + continue; + } + + Y_ABORT_UNLESS(context.SS->CdcStreams.contains(childPathId)); + auto stream = context.SS->CdcStreams.at(childPathId); + + if (stream->State != TCdcStreamInfo::EState::ECdcStreamStateInvalid) { + continue; + } + + Y_VERIFY_S(!found, "Too many cdc streams are planned to create" + << ": found# " << PathIdFromPathId(notice.GetStreamDescription().GetPathId()) + << ", another# " << childPathId); + found = true; + + Y_ABORT_UNLESS(stream->AlterData); + context.SS->DescribeCdcStream(childPathId, childName, stream->AlterData, *notice.MutableStreamDescription()); + + if (stream->AlterData->State == TCdcStreamInfo::EState::ECdcStreamStateScan) { + notice.SetSnapshotName("ChangefeedInitialScan"); + } + } + } + bool ProgressState(TOperationContext& context) override { TTabletId ssId = context.SS->SelfTabletId(); LOG_DEBUG_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, DebugHint() << " ProgressState" - << " at tablet" << ssId); + << " at tablet " << ssId); TTxState* txState = context.SS->FindTx(OperationId); Y_ABORT_UNLESS(txState); @@ -108,11 +155,16 @@ class TConfigureParts: public TSubOperationState { // Send "SendParts" transaction to source datashard NKikimrTxDataShard::TFlatSchemeTransaction oldShardTx; context.SS->FillSeqNo(oldShardTx, seqNo); - oldShardTx.MutableSendSnapshot()->SetTableId_Deprecated(txState->SourcePathId.LocalPathId); - oldShardTx.MutableSendSnapshot()->MutableTableId()->SetOwnerId(txState->SourcePathId.OwnerId); - oldShardTx.MutableSendSnapshot()->MutableTableId()->SetTableId(txState->SourcePathId.LocalPathId); - oldShardTx.MutableSendSnapshot()->AddSendTo()->SetShard(ui64(dstDatashardId)); - oldShardTx.SetReadOnly(true); + + if (txState->CdcPathId != InvalidPathId) { + auto& combined = *oldShardTx.MutableCreateIncrementalBackupSrc(); + FillSrcSnapshot(txState, ui64(dstDatashardId), *combined.MutableSendSnapshot()); + FillNotice(txState->CdcPathId, context, *combined.MutableCreateCdcStreamNotice()); + } else { + FillSrcSnapshot(txState, ui64(dstDatashardId), *oldShardTx.MutableSendSnapshot()); + oldShardTx.SetReadOnly(true); + } + auto srcEvent = context.SS->MakeDataShardProposal(txState->TargetPathId, OperationId, oldShardTx.SerializeAsString(), context.Ctx); context.OnComplete.BindMsgToPipe(OperationId, srcDatashardId, srcShardIdx, srcEvent.Release()); } @@ -201,7 +253,29 @@ class TPropose: public TSubOperationState { context.SS->ClearDescribePathCaches(path); context.OnComplete.PublishToSchemeBoard(OperationId, pathId); + // ---> + + if (txState->CdcPathId != InvalidPathId) { + TPathId srcPathId = txState->SourcePathId; + + Y_ABORT_UNLESS(context.SS->PathsById.contains(srcPathId)); + auto srcPath = context.SS->PathsById.at(srcPathId); + + Y_ABORT_UNLESS(context.SS->Tables.contains(srcPathId)); + auto srcTable = context.SS->Tables.at(srcPathId); + + srcTable->AlterVersion += 1; + + context.SS->PersistTableAlterVersion(db, srcPathId, table); + + context.SS->ClearDescribePathCaches(path); + context.OnComplete.PublishToSchemeBoard(OperationId, srcPathId); + } + + // <--- + context.SS->ChangeTxState(db, OperationId, TTxState::ProposedWaitParts); + return true; } @@ -255,7 +329,7 @@ class TCopyTableBarrier: public TSubOperationState { LOG_INFO_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, DebugHint() << " HandleReply TEvPrivate::TEvCompleteBarrier" << ", msg: " << ev->Get()->ToString() - << ", at tablet" << ssId); + << ", at tablet " << ssId); NIceDb::TNiceDb db(context.GetDB()); @@ -373,7 +447,7 @@ class TCopyTable: public TSubOperation { .IsUnderTheSameOperation(OperationId.GetTxId()); //allow only as part of copying base table } else { checks - .NotUnderOperation() + // .NotUnderOperation() .IsCommonSensePath() .IsLikeDirectory(); } @@ -381,6 +455,7 @@ class TCopyTable: public TSubOperation { if (!checks) { result->SetError(checks.GetStatus(), checks.GetError()); + Y_ABORT("parent: %s, %s", parent.PathString().c_str(), checks.GetError().c_str()); return result; } } @@ -407,8 +482,11 @@ class TCopyTable: public TSubOperation { } } + // FIXME: additional checks when with cdc + if (!checks) { result->SetError(checks.GetStatus(), checks.GetError()); + Y_ABORT("srcPath: %s", srcPath.PathString().c_str()); return result; } } @@ -454,6 +532,7 @@ class TCopyTable: public TSubOperation { if (!checks) { result->SetError(checks.GetStatus(), checks.GetError()); + Y_ABORT("dstPath: %s, %s", dstPath.PathString().c_str(), checks.GetError().c_str()); return result; } } @@ -578,6 +657,7 @@ class TCopyTable: public TSubOperation { } } + auto guard = context.DbGuard(); TPathId allocatedPathId = context.SS->AllocatePathId(); context.MemChanges.GrabNewPath(context.SS, allocatedPathId); @@ -613,6 +693,9 @@ class TCopyTable: public TSubOperation { TTxState& txState = context.SS->CreateTx(OperationId, TTxState::TxCopyTable, newTable->PathId, srcPath.Base()->PathId); txState.State = TTxState::CreateParts; + if (Transaction.HasCreateCdcStream()) { + txState.CdcPathId = srcPath.Base()->PathId; + } TShardInfo datashardInfo = TShardInfo::DataShardInfo(OperationId.GetTxId(), newTable->PathId); datashardInfo.BindedChannels = channelsBinding; @@ -718,6 +801,7 @@ TVector CreateCopyTable(TOperationId nextId, const TTxTrans auto copying = tx.GetCreateTable(); Y_ABORT_UNLESS(copying.HasCopyFromTable()); + auto coOp = tx.HasCreateCdcStream() ? &tx.GetCreateCdcStream() : nullptr; TPath srcPath = TPath::Resolve(copying.GetCopyFromTable(), context.SS); { @@ -733,6 +817,7 @@ TVector CreateCopyTable(TOperationId nextId, const TTxTrans .IsCommonSensePath(); //forbid copy impl index tables directly if (!checks) { + Y_ABORT("%s", srcPath.PathString().c_str()); return {CreateReject(nextId, checks.GetStatus(), checks.GetError())}; } } @@ -770,6 +855,9 @@ TVector CreateCopyTable(TOperationId nextId, const TTxTrans operation->SetOmitFollowers(copying.GetOmitFollowers()); operation->SetIsBackup(copying.GetIsBackup()); operation->MutablePartitionConfig()->CopyFrom(copying.GetPartitionConfig()); + if (coOp) { + schema.MutableCreateCdcStream()->CopyFrom(*coOp); + } result.push_back(CreateCopyTable(NextPartId(nextId, result), schema, sequences)); } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_backup_collection.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_backup_collection.cpp new file mode 100644 index 000000000000..cccdba0c8911 --- /dev/null +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_backup_collection.cpp @@ -0,0 +1,321 @@ +#include "schemeshard__operation_common.h" +#include "schemeshard_impl.h" + +#define LOG_I(stream) LOG_INFO_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "[" << context.SS->TabletID() << "] " << stream) +#define LOG_N(stream) LOG_NOTICE_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "[" << context.SS->TabletID() << "] " << stream) +#define RETURN_RESULT_UNLESS(x) if (!(x)) return result; + +namespace NKikimr::NSchemeShard { + +namespace { + +TPath::TChecker IsParentPathValid(const TPath& parentPath) { + auto checks = parentPath.Check(); + checks.NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory(); + + return std::move(checks); +} + +bool IsParentPathValid(const THolder& result, const TPath& parentPath) { + const auto checks = IsParentPathValid(parentPath); + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + } + + return static_cast(checks); +} + +/// + +class TPropose: public TSubOperationState { +private: + const TOperationId OperationId; + + TString DebugHint() const override { + return TStringBuilder() + << "TCreateBackupCollection TPropose" + << ", operationId: " << OperationId; + } + +public: + explicit TPropose(TOperationId id) + : OperationId(std::move(id)) + { + } + + bool HandleReply(TEvPrivate::TEvOperationPlan::TPtr& ev, TOperationContext& context) override { + const TStepId step = TStepId(ev->Get()->StepId); + + LOG_I(DebugHint() << "HandleReply TEvOperationPlan" + << ": step# " << step); + + const TTxState* txState = context.SS->FindTx(OperationId); + Y_ABORT_UNLESS(txState); + Y_ABORT_UNLESS(txState->TxType == TTxState::TxCreateBackupCollection); + + const auto pathId = txState->TargetPathId; + const auto path = TPath::Init(pathId, context.SS); + const TPathElement::TPtr pathPtr = context.SS->PathsById.at(pathId); + + context.SS->TabletCounters->Simple()[COUNTER_BACKUP_COLLECTION_COUNT].Add(1); + + NIceDb::TNiceDb db(context.GetDB()); + + path->StepCreated = step; + context.SS->PersistCreateStep(db, pathId, step); + + IncParentDirAlterVersionWithRepublish(OperationId, path, context); + + context.SS->ClearDescribePathCaches(pathPtr); + context.OnComplete.PublishToSchemeBoard(OperationId, pathId); + + context.SS->ChangeTxState(db, OperationId, TTxState::Done); + return true; + } + + bool ProgressState(TOperationContext& context) override { + LOG_I(DebugHint() << "ProgressState"); + + const TTxState* txState = context.SS->FindTx(OperationId); + Y_ABORT_UNLESS(txState); + Y_ABORT_UNLESS(txState->TxType == TTxState::TxCreateBackupCollection); + + context.OnComplete.ProposeToCoordinator(OperationId, txState->TargetPathId, TStepId(0)); + return false; + } +}; + +class TCreateBackupCollection : public TSubOperation { + static TTxState::ETxState NextState() { + return TTxState::Propose; + } + + TTxState::ETxState NextState(TTxState::ETxState state) const override { + switch (state) { + case TTxState::Waiting: + case TTxState::Propose: + return TTxState::Done; + default: + return TTxState::Invalid; + } + } + + TSubOperationState::TPtr SelectStateFunc(TTxState::ETxState state) override { + switch (state) { + case TTxState::Waiting: + case TTxState::Propose: + return MakeHolder(OperationId); + case TTxState::Done: + return MakeHolder(OperationId); + default: + return nullptr; + } + } + + static bool IsDestinationPathValid(const THolder& result, const TPath& dstPath, const TString& acl, bool acceptExisted) { + const auto checks = dstPath.Check(); + checks.IsAtLocalSchemeShard(); + if (dstPath.IsResolved()) { + checks + .IsResolved() + .NotUnderDeleting() + .FailOnExist(TPathElement::EPathType::EPathTypeBackupCollection, acceptExisted); + } else { + checks + .NotEmpty() + .NotResolved(); + } + + if (checks) { + checks + .IsValidLeafName() + .DepthLimit() + .PathsLimit() + .DirChildrenLimit() + .IsValidACL(acl); + } + + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + if (dstPath.IsResolved()) { + result->SetPathCreateTxId(static_cast(dstPath.Base()->CreateTxId)); + result->SetPathId(dstPath.Base()->PathId.LocalPathId); + } + } + + return static_cast(checks); + } + + static void AddPathInSchemeShard(const THolder& result, TPath& dstPath, const TString& owner) { + dstPath.MaterializeLeaf(owner); + result->SetPathId(dstPath.Base()->PathId.LocalPathId); + } + + TPathElement::TPtr CreateBackupCollectionPathElement(const TPath& dstPath) const { + TPathElement::TPtr backupCollection = dstPath.Base(); + + backupCollection->CreateTxId = OperationId.GetTxId(); + backupCollection->PathType = TPathElement::EPathType::EPathTypeBackupCollection; + backupCollection->PathState = TPathElement::EPathState::EPathStateCreate; + backupCollection->LastTxId = OperationId.GetTxId(); + + return backupCollection; + } + + static void UpdatePathSizeCounts(const TPath& parentPath, const TPath& dstPath) { + dstPath.DomainInfo()->IncPathsInside(); + parentPath.Base()->IncAliveChildren(); + } + +public: + using TSubOperation::TSubOperation; + + THolder Propose(const TString& owner, TOperationContext& context) override { + const TString& rootPathStr = Transaction.GetWorkingDir(); + const auto& desc = Transaction.GetCreateBackupCollection(); + const TString& name = desc.GetName(); + LOG_N("TCreateBackupCollection Propose: opId# " << OperationId << ", path# " << rootPathStr << "/" << name); + + auto result = MakeHolder(NKikimrScheme::StatusAccepted, + static_cast(OperationId.GetTxId()), + static_cast(context.SS->SelfTabletId())); + + bool backupServiceEnabled = AppData()->FeatureFlags.GetEnableBackupService(); + if (!backupServiceEnabled) { + result->SetError(NKikimrScheme::StatusPreconditionFailed, "Backup collections are disabled. Please contact your system administrator to enable it"); + return result; + } + + const TPath& rootPath = TPath::Resolve(rootPathStr, context.SS); + { + const auto checks = rootPath.Check(); + checks + .NotEmpty() + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .NotUnderDeleting() + .IsCommonSensePath() + .IsLikeDirectory() + .FailOnRestrictedCreateInTempZone(); + + if (!checks) { + result->SetError(checks.GetStatus(), checks.GetError()); + return result; + } + } + + TPath dstPath = rootPath.Child(name); + + const TString& backupCollectionsDir = JoinPath({rootPath.GetDomainPathString(), ".backups/collections"}); + + if (!dstPath.PathString().StartsWith(backupCollectionsDir + "/")) { + result->SetError(NKikimrScheme::EStatus::StatusSchemeError, TStringBuilder() << "Backup collections must be placed in " << backupCollectionsDir); + return result; + } + + const TPath& backupCollectionsPath = TPath::Resolve(backupCollectionsDir, context.SS); + + // FIXME + if (backupCollectionsPath.IsResolved()) { + RETURN_RESULT_UNLESS(IsParentPathValid(result, backupCollectionsPath)); + } + + TString errStr; + if (!context.SS->CheckApplyIf(Transaction, errStr)) { + result->SetError(NKikimrScheme::StatusPreconditionFailed, errStr); + return result; + } + + AddPathInSchemeShard(result, dstPath, owner); + auto pathEl = CreateBackupCollectionPathElement(dstPath); + + rootPath->IncAliveChildren(); + rootPath.DomainInfo()->IncPathsInside(); + + auto backupCollection = TBackupCollectionInfo::Create(desc); + context.SS->BackupCollections[dstPath->PathId] = backupCollection; + context.SS->TabletCounters->Simple()[COUNTER_BACKUP_COLLECTION_COUNT].Add(1); + + // in progress + + TTxState& txState = context.SS->CreateTx( + OperationId, + TTxState::TxCreateBackupCollection, + pathEl->PathId); + txState.Shards.clear(); + + NIceDb::TNiceDb db(context.GetDB()); + + const TString& acl = Transaction.GetModifyACL().GetDiffACL(); + //// + // RegisterParentPathDependencies(context, parentPath); + if (rootPath.Base()->HasActiveChanges()) { + const TTxId parentTxId = rootPath.Base()->PlannedToCreate() + ? rootPath.Base()->CreateTxId + : rootPath.Base()->LastTxId; + context.OnComplete.Dependence(parentTxId, OperationId.GetTxId()); + } + + // AdvanceTransactionStateToPropose(context, db); + context.SS->ChangeTxState(db, OperationId, TTxState::Propose); + context.OnComplete.ActivateTx(OperationId); + + // PersistExternalDataSource(context, db, externalDataSource, + // externalDataSourceInfo, acl); + const auto& backupCollectionPathId = pathEl->PathId; + + context.SS->BackupCollections[dstPath->PathId] = backupCollection; + context.SS->IncrementPathDbRefCount(backupCollectionPathId); + + if (!acl.empty()) { + pathEl->ApplyACL(acl); + } + context.SS->PersistPath(db, backupCollectionPathId); + + context.SS->PersistBackupCollection(db, + backupCollectionPathId, + backupCollection); + context.SS->PersistTxState(db, OperationId); + + IncParentDirAlterVersionWithRepublishSafeWithUndo(OperationId, + dstPath, + context.SS, + context.OnComplete); + + UpdatePathSizeCounts(rootPath, dstPath); + + SetState(NextState()); + return result; + } + + void AbortPropose(TOperationContext& context) override { + LOG_N("TCreateBackupCollection AbortPropose: opId# " << OperationId); + Y_ABORT("no AbortPropose for TCreateBackupCollection"); + } + + void AbortUnsafe(TTxId forceDropTxId, TOperationContext& context) override { + LOG_N("TCreateBackupCollection AbortUnsafe: opId# " << OperationId << ", txId# " << forceDropTxId); + context.OnComplete.DoneOperation(OperationId); + } +}; + +} // anonymous namespace + +ISubOperation::TPtr CreateNewBackupCollection(TOperationId id, const TTxTransaction& tx) { + return MakeSubOperation(id, tx); +} + +ISubOperation::TPtr CreateNewBackupCollection(TOperationId id, TTxState::ETxState state) { + Y_VERIFY(state != TTxState::Invalid); + return MakeSubOperation(id, state); +} + +} // namespace NKikimr::NSchemeShard diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.cpp index a95677271502..d02dc12c4670 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.cpp @@ -564,7 +564,8 @@ class TNewCdcStreamAtTable: public TSubOperation { if (InitialScan) { checks.IsUnderTheSameOperation(OperationId.GetTxId()); // lock op } else { - checks.NotUnderOperation(); + checks.IsUnderTheSameOperation(OperationId.GetTxId()); // lock op + // checks.NotUnderOperation(); } } @@ -736,6 +737,21 @@ static void FillModifySchemaForCdc( } } +void DoCreateStreamImpl( + TVector& result, + const NKikimrSchemeOp::TCreateCdcStream& op, + const TOperationId& opId, + const TPath& workingDirPath, + const TPath& tablePath, + const bool acceptExisted, + const bool initialScan) +{ + Y_UNUSED(workingDirPath); + auto outTx = TransactionTemplate(tablePath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateCdcStreamImpl); + FillModifySchemaForCdc(outTx, op, opId, acceptExisted, initialScan); + result.push_back(CreateNewCdcStreamImpl(NextPartId(opId, result), outTx)); +} + void DoCreateStream( TVector& result, const NKikimrSchemeOp::TCreateCdcStream& op, @@ -745,11 +761,7 @@ void DoCreateStream( const bool acceptExisted, const bool initialScan) { - { - auto outTx = TransactionTemplate(tablePath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateCdcStreamImpl); - FillModifySchemaForCdc(outTx, op, opId, acceptExisted, initialScan); - result.push_back(CreateNewCdcStreamImpl(NextPartId(opId, result), outTx)); - } + DoCreateStreamImpl(result, op, opId, workingDirPath, tablePath, acceptExisted, initialScan); { auto outTx = TransactionTemplate(workingDirPath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpCreateCdcStreamAtTable); FillModifySchemaForCdc(outTx, op, opId, acceptExisted, initialScan); @@ -867,6 +879,7 @@ std::variant DoNewStreamPathChecks( { const auto tablePath = workingDirPath.Child(tableName); if (auto reject = RejectOnTablePathChecks(opId, tablePath, restore)) { + // Y_ABORT("'%s' '%s' '%s'", workingDirPath.PathString().c_str(), tableName.c_str(), streamName.c_str()); return reject; } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.h b/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.h index 1aa03529be6e..547e68ec96a2 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.h +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_cdc_stream.h @@ -22,6 +22,15 @@ std::variant DoNewStreamPathChecks( bool acceptExisted, bool restore = false); +void DoCreateStreamImpl( + TVector& result, + const NKikimrSchemeOp::TCreateCdcStream& op, + const TOperationId& opId, + const TPath& workingDirPath, + const TPath& tablePath, + const bool acceptExisted, + const bool initialScan); + void DoCreateStream( TVector& result, const NKikimrSchemeOp::TCreateCdcStream& op, diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_continuous_backup.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_continuous_backup.cpp index ab2c187d24bf..c32385ee4fb8 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_continuous_backup.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_continuous_backup.cpp @@ -15,7 +15,7 @@ namespace NKikimr::NSchemeShard { -TVector CreateNewContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { +bool CreateNewContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context, TVector& result) { Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpCreateContinuousBackup); LOG_D("CreateNewContinuousBackup" @@ -24,12 +24,15 @@ TVector CreateNewContinuousBackup(TOperationId opId, const const auto acceptExisted = !tx.GetFailOnExist(); const auto workingDirPath = TPath::Resolve(tx.GetWorkingDir(), context.SS); + // const auto workingDirPath = TPath::Resolve("/Root", context.SS); const auto& cbOp = tx.GetCreateContinuousBackup(); const auto& tableName = cbOp.GetTableName(); + // const TString tableName = "table"; const auto checksResult = NCdc::DoNewStreamPathChecks(opId, workingDirPath, tableName, NBackup::CB_CDC_STREAM_NAME, acceptExisted); if (std::holds_alternative(checksResult)) { - return {std::get(checksResult)}; + result = {std::get(checksResult)}; + return false; } const auto [tablePath, streamPath] = std::get(checksResult); @@ -41,11 +44,13 @@ TVector CreateNewContinuousBackup(TOperationId opId, const TString errStr; if (!context.SS->CheckApplyIf(tx, errStr)) { - return {CreateReject(opId, NKikimrScheme::StatusPreconditionFailed, errStr)}; + result = {CreateReject(opId, NKikimrScheme::StatusPreconditionFailed, errStr)}; + return false; } if (!context.SS->CheckLocks(tablePath.Base()->PathId, tx, errStr)) { - return {CreateReject(opId, NKikimrScheme::StatusMultipleModifications, errStr)}; + result = {CreateReject(opId, NKikimrScheme::StatusMultipleModifications, errStr)}; + return false; } TVector boundaries; @@ -66,11 +71,17 @@ TVector CreateNewContinuousBackup(TOperationId opId, const streamDescription.SetMode(NKikimrSchemeOp::ECdcStreamModeUpdate); streamDescription.SetFormat(NKikimrSchemeOp::ECdcStreamFormatProto); - TVector result; - NCdc::DoCreateStream(result, createCdcStreamOp, opId, workingDirPath, tablePath, acceptExisted, false); NCdc::DoCreatePqPart(result, createCdcStreamOp, opId, streamPath, NBackup::CB_CDC_STREAM_NAME, table, boundaries, acceptExisted); + return true; +} + +TVector CreateNewContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + TVector result; + + CreateNewContinuousBackup(opId, tx, context, result); + return result; } 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 148ab5137b66..d011977e49a8 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 @@ -65,13 +65,21 @@ class TConfigurePartsAtTable: public TSubOperationState { Y_ABORT_UNLESS(context.SS->Tables.contains(pathId)); auto table = context.SS->Tables.at(pathId); - tx.MutableCreateIncrementalRestoreSrc()->CopyFrom(RestoreOp); + auto& op = *tx.MutableCreateIncrementalRestoreSrc(); + op.MutableSrcPathId()->CopyFrom(RestoreOp.GetSrcPathIds(SeqNo)); + op.SetSrcTableName(RestoreOp.GetSrcTableNames(SeqNo)); + op.MutableDstPathId()->CopyFrom(RestoreOp.GetDstPathId()); + op.SetDstTableName(RestoreOp.GetDstTableName()); } public: - explicit TConfigurePartsAtTable(TOperationId id, const NKikimrSchemeOp::TRestoreIncrementalBackup& restoreOp) + explicit TConfigurePartsAtTable( + TOperationId id, + const NKikimrSchemeOp::TRestoreMultipleIncrementalBackups& restoreOp, + ui64 seqNo) : OperationId(id) , RestoreOp(restoreOp) + , SeqNo(seqNo) { IgnoreMessages(DebugHint(), {}); } @@ -122,7 +130,8 @@ class TConfigurePartsAtTable: public TSubOperationState { private: const TOperationId OperationId; - const NKikimrSchemeOp::TRestoreIncrementalBackup RestoreOp; + const NKikimrSchemeOp::TRestoreMultipleIncrementalBackups RestoreOp; + const ui64 SeqNo; }; // TConfigurePartsAtTable class TProposeAtTable: public TSubOperationState { @@ -212,34 +221,144 @@ class TProposeAtTable: public TSubOperationState { }; // TProposeAtTable -class TNewRestoreFromAtTable: public TSubOperation { - static TTxState::ETxState NextState() { +class TProposedWaitParts: public TSubOperationState { +private: + TOperationId OperationId; + const TTxState::ETxState NextState; + + TString DebugHint() const override { + return TStringBuilder() + << "NTableState::TProposedWaitParts" + << " operationId# " << OperationId; + } + +public: + TProposedWaitParts(TOperationId id, TTxState::ETxState nextState = TTxState::Done) + : OperationId(id) + , NextState(nextState) + { + IgnoreMessages(DebugHint(), + { TEvHive::TEvCreateTabletReply::EventType + , TEvDataShard::TEvProposeTransactionResult::EventType + , TEvPrivate::TEvOperationPlan::EventType } + ); + } + + bool HandleReply(TEvDataShard::TEvSchemaChanged::TPtr& ev, TOperationContext& context) override { + TTabletId ssId = context.SS->SelfTabletId(); + const auto& evRecord = ev->Get()->Record; + + LOG_INFO_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + DebugHint() << " HandleReply TEvSchemaChanged" + << " at tablet: " << ssId); + LOG_DEBUG_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + DebugHint() << " HandleReply TEvSchemaChanged" + << " at tablet: " << ssId + << " message: " << evRecord.ShortDebugString()); + + if (!NTableState::CollectSchemaChanged(OperationId, ev, context)) { + LOG_DEBUG_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + DebugHint() << " HandleReply TEvSchemaChanged" + << " CollectSchemaChanged: false"); + return false; + } + + Y_ABORT_UNLESS(context.SS->FindTx(OperationId)); + TTxState& txState = *context.SS->FindTx(OperationId); + + if (!txState.ReadyForNotifications) { + LOG_DEBUG_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + DebugHint() << " HandleReply TEvSchemaChanged" + << " ReadyForNotifications: false"); + return false; + } + + return true; + } + + bool ProgressState(TOperationContext& context) override { + TTabletId ssId = context.SS->SelfTabletId(); + + LOG_INFO_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + DebugHint() << " ProgressState" + << " at tablet: " << ssId); + + TTxState* txState = context.SS->FindTx(OperationId); + + NIceDb::TNiceDb db(context.GetDB()); + + txState->ClearShardsInProgress(); + for (TTxState::TShardOperation& shard : txState->Shards) { + if (shard.Operation < TTxState::ProposedWaitParts) { + shard.Operation = TTxState::ProposedWaitParts; + context.SS->PersistUpdateTxShard(db, OperationId, shard.Idx, shard.Operation); + } + Y_ABORT_UNLESS(context.SS->ShardInfos.contains(shard.Idx)); + context.OnComplete.RouteByTablet(OperationId, context.SS->ShardInfos.at(shard.Idx).TabletID); + } + txState->UpdateShardsInProgress(TTxState::ProposedWaitParts); + + // Move all notifications that were already received + // NOTE: SchemeChangeNotification is sent form DS after it has got PlanStep from coordinator and the schema tx has completed + // At that moment the SS might not have received PlanStep from coordinator yet (this message might be still on its way to SS) + // So we are going to accumulate SchemeChangeNotification that are received before this Tx switches to WaitParts state + txState->AcceptPendingSchemeNotification(); + + // Got notifications from all datashards? + if (txState->ShardsInProgress.empty()) { + NTableState::AckAllSchemaChanges(OperationId, *txState, context); + context.SS->ChangeTxState(db, OperationId, NextState); + return true; + } + + return false; + } +}; + +class TNewRestoreFromAtTable: public TBetterSubOperation { + static TTxState::ETxState InitialState() { return TTxState::ConfigureParts; } - TTxState::ETxState NextState(TTxState::ETxState state) const override { + TTxState::ETxState NextState(TTxState::ETxState state, TOperationContext& context) const override { switch (state) { case TTxState::Waiting: + return TTxState::ConfigureParts; case TTxState::ConfigureParts: return TTxState::Propose; case TTxState::Propose: return TTxState::ProposedWaitParts; - case TTxState::ProposedWaitParts: + case TTxState::ProposedWaitParts: { + auto* txState = context.SS->FindTx(OperationId); + Y_ABORT_UNLESS(txState); + // Y_ABORT_UNLESS(IsExpectedTxType(txState->TxType)); + ++(txState->LoopSeqNo); + if (txState->LoopSeqNo < Transaction.GetRestoreMultipleIncrementalBackups().SrcPathIdsSize()) { + txState->TargetPathId = PathIdFromPathId(Transaction.GetRestoreMultipleIncrementalBackups().GetSrcPathIds(txState->LoopSeqNo)); + txState->TxShardsListFinalized = false; + // TODO preserve TxState + return TTxState::ConfigureParts; + } return TTxState::Done; + } default: return TTxState::Invalid; } } - TSubOperationState::TPtr SelectStateFunc(TTxState::ETxState state) override { + TSubOperationState::TPtr SelectStateFunc(TTxState::ETxState state, TOperationContext& context) override { switch (state) { case TTxState::Waiting: - case TTxState::ConfigureParts: - return MakeHolder(OperationId, Transaction.GetRestoreIncrementalBackup()); + case TTxState::ConfigureParts: { + auto* txState = context.SS->FindTx(OperationId); + Y_ABORT_UNLESS(txState); + // Y_ABORT_UNLESS(IsExpectedTxType(txState->TxType)); + return MakeHolder(OperationId, Transaction.GetRestoreMultipleIncrementalBackups(), txState->LoopSeqNo); + } case TTxState::Propose: return MakeHolder(OperationId); case TTxState::ProposedWaitParts: - return MakeHolder(OperationId); + return MakeHolder(OperationId); case TTxState::Done: return MakeHolder(OperationId); default: @@ -247,21 +366,21 @@ class TNewRestoreFromAtTable: public TSubOperation { } } -public: - explicit TNewRestoreFromAtTable(TOperationId id, const TTxTransaction& tx) - : TSubOperation(id, tx) + public: + explicit TNewRestoreFromAtTable(TOperationId id, const TTxTransaction& tx) + : TBetterSubOperation(id, tx) { } explicit TNewRestoreFromAtTable(TOperationId id, TTxState::ETxState state) - : TSubOperation(id, state) + : TBetterSubOperation(id, state) { } THolder Propose(const TString&, TOperationContext& context) override { const auto& workingDir = Transaction.GetWorkingDir(); - const auto& op = Transaction.GetRestoreIncrementalBackup(); - const auto& tableName = op.GetSrcTableName(); + const auto& op = Transaction.GetRestoreMultipleIncrementalBackups(); + const auto& tableName = op.GetSrcTableNames(0); const auto& dstTableName = op.GetDstTableName(); LOG_N("TNewRestoreFromAtTable Propose" @@ -304,8 +423,8 @@ class TNewRestoreFromAtTable: public TSubOperation { .IsTable() .NotAsyncReplicaTable() .NotUnderDeleting() - .IsCommonSensePath() - .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op + .IsCommonSensePath(); + // .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op if (!checks) { result->SetError(checks.GetStatus(), checks.GetError()); @@ -325,8 +444,8 @@ class TNewRestoreFromAtTable: public TSubOperation { .IsTable() .NotAsyncReplicaTable() .NotUnderDeleting() - .IsCommonSensePath() - .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op + .IsCommonSensePath(); + // .IsUnderTheSameOperation(OperationId.GetTxId()); // lock op if (!checks) { result->SetError(checks.GetStatus(), checks.GetError()); @@ -350,6 +469,8 @@ class TNewRestoreFromAtTable: public TSubOperation { context.DbChanges.PersistPath(tablePath.Base()->PathId); context.DbChanges.PersistTxState(OperationId); + // Cerr << " <-- <-- <-- " << (op.GetSeqNo() > 0 ? "dep" : "nodep") << tableName << " from " << OperationId.GetTxId().GetValue() << " to " << OperationId.GetTxId().GetValue() - 1 << Endl; + Y_ABORT_UNLESS(context.SS->Tables.contains(tablePath.Base()->PathId)); auto table = context.SS->Tables.at(tablePath.Base()->PathId); @@ -371,7 +492,7 @@ class TNewRestoreFromAtTable: public TSubOperation { context.OnComplete.ActivateTx(OperationId); - SetState(NextState()); + SetState(InitialState(), context); return result; } @@ -394,6 +515,8 @@ class TNewRestoreFromAtTable: public TSubOperation { TVector CreateRestoreIncrementalBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup); + Y_ABORT("Intentnionally broken"); + LOG_N("CreateRestoreIncrementalBackup" << ": opId# " << opId << ", tx# " << tx.ShortDebugString()); @@ -403,7 +526,8 @@ TVector CreateRestoreIncrementalBackup(TOperationId opId, c const auto& srcTableName = restoreOp.GetSrcTableName(); const auto& dstTableName = restoreOp.GetDstTableName(); - const auto srcTablePath = workingDirPath.Child(srcTableName); + const auto srcTablePathX = workingDirPath.Child(srcTableName); + const auto srcTablePath = TPath::Resolve(srcTablePathX.PathString(), context.SS); { const auto checks = srcTablePath.Check(); checks @@ -422,7 +546,8 @@ TVector CreateRestoreIncrementalBackup(TOperationId opId, c } } - const auto dstTablePath = workingDirPath.Child(dstTableName); + const auto dstTablePathX = workingDirPath.Child(dstTableName); + const auto dstTablePath = TPath::Resolve(dstTablePathX.PathString(), context.SS); { const auto checks = srcTablePath.Check(); checks @@ -457,33 +582,24 @@ TVector CreateRestoreIncrementalBackup(TOperationId opId, c // check dst locks // lock dst - TVector boundaries; - const auto& partitions = srcTable->GetPartitions(); - boundaries.reserve(partitions.size() - 1); - - for (ui32 i = 0; i < partitions.size(); ++i) { - const auto& partition = partitions.at(i); - if (i != partitions.size() - 1) { - boundaries.push_back(partition.EndOfRange); - } - } - TVector result; - DoCreateLock(opId, workingDirPath, srcTablePath, result); - DoCreateLock(opId, workingDirPath, dstTablePath, result); + DoCreateLock(opId, srcTablePath.Parent(), srcTablePath, result); + DoCreateLock(opId, dstTablePath.Parent(), dstTablePath, result); { - auto outTx = TransactionTemplate(workingDirPath.PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable); + auto outTx = TransactionTemplate(srcTablePath.Parent().PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable); outTx.MutableRestoreIncrementalBackup()->CopyFrom(restoreOp); auto& restoreOp = *outTx.MutableRestoreIncrementalBackup(); + restoreOp.SetSrcTableName(srcTablePath.LeafName()); + restoreOp.SetDstTableName(dstTablePath.LeafName()); PathIdFromPathId(srcTablePath.Base()->PathId, restoreOp.MutableSrcPathId()); PathIdFromPathId(dstTablePath.Base()->PathId, restoreOp.MutableDstPathId()); result.push_back(CreateRestoreIncrementalBackupAtTable(NextPartId(opId, result), outTx)); } - DoDropLock(opId, workingDirPath, dstTablePath, result); - DoDropLock(opId, workingDirPath, srcTablePath, result); + DoDropLock(opId, srcTablePath.Parent(), srcTablePath, result); + DoDropLock(opId, dstTablePath.Parent(), dstTablePath, result); return result; } @@ -492,9 +608,127 @@ ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, const return MakeSubOperation(id, tx); } -ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state) { +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state, TOperationContext& context) { Y_ABORT_UNLESS(state != TTxState::Invalid); - return MakeSubOperation(id, state); + return MakeSubOperation(id, state, context); +} + +bool CreateRestoreMultipleIncrementalBackups( + TOperationId opId, + const TTxTransaction& tx, + TOperationContext& context, + TVector& result) +{ + Y_ABORT_UNLESS(tx.GetOperationType() == NKikimrSchemeOp::EOperationType::ESchemeOpRestoreMultipleIncrementalBackups); + + LOG_N("CreateRestoreMultipleIncrementalBackups" + << ": opId# " << opId + << ", tx# " << tx.ShortDebugString()); + + const auto workingDirPath = TPath::Resolve(tx.GetWorkingDir(), context.SS); + const auto& restoreOp = tx.GetRestoreMultipleIncrementalBackups(); + const auto& dstTableName = restoreOp.GetDstTableName(); + + TVector srcPaths; + + for (const auto& srcTableName : restoreOp.GetSrcTableNames()) { + const auto srcTablePathX = workingDirPath.Child(srcTableName); + const auto srcTablePath = TPath::Resolve(srcTablePathX.PathString(), context.SS); + { + const auto checks = srcTablePath.Check(); + checks + .NotEmpty() + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .IsTable() + .NotUnderDeleting() + // .NotUnderOperation() + .IsCommonSensePath(); + + if (!checks) { + result = {CreateReject(opId, checks.GetStatus(), checks.GetError())}; + return false; + } + } + srcPaths.push_back(srcTablePath); + } + + const auto dstTablePathX = workingDirPath.Child(dstTableName); + const auto dstTablePath = TPath::Resolve(dstTablePathX.PathString(), context.SS); + { + const auto checks = dstTablePath.Check(); + checks + .NotEmpty() + .NotUnderDomainUpgrade() + .IsAtLocalSchemeShard() + .IsResolved() + .NotDeleted() + .IsTable() + .NotUnderDeleting() + // .NotUnderOperation() + .IsCommonSensePath(); + + if (!checks) { + result = {CreateReject(opId, checks.GetStatus(), checks.GetError())}; + return false; + } + } + + + for (auto& srcTablePath : srcPaths) { + Y_ABORT_UNLESS(context.SS->Tables.contains(srcTablePath.Base()->PathId)); + auto srcTable = context.SS->Tables.at(srcTablePath.Base()->PathId); + } + + TString errStr; + if (!context.SS->CheckApplyIf(tx, errStr)) { + result = {CreateReject(opId, NKikimrScheme::StatusPreconditionFailed, errStr)}; + return false; + } + + for (auto& srcTablePath : srcPaths) { + if (!context.SS->CheckLocks(srcTablePath.Base()->PathId, tx, errStr)) { + result = {CreateReject(opId, NKikimrScheme::StatusMultipleModifications, errStr)}; + return false; + } + } + + // check dst locks + // lock dst + + // for (auto& srcTablePath : srcPaths) { + // DoCreateLock(opId, srcTablePath.Parent(), srcTablePath, result); + // } + // DoCreateLock(opId, dstTablePath.Parent(), dstTablePath, result); + + { + auto outTx = TransactionTemplate(srcPaths[0].Parent().PathString(), NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable); + auto& restoreOp = *outTx.MutableRestoreMultipleIncrementalBackups(); + restoreOp.SetDstTableName(dstTablePath.LeafName()); + PathIdFromPathId(dstTablePath.Base()->PathId, restoreOp.MutableDstPathId()); + + for (const auto& srcTablePath : srcPaths) { + restoreOp.AddSrcTableNames(srcTablePath.LeafName()); + PathIdFromPathId(srcTablePath.Base()->PathId, restoreOp.AddSrcPathIds()); + } + + result.push_back(CreateRestoreIncrementalBackupAtTable(NextPartId(opId, result), outTx)); + } + + // for (auto& srcTablePath : srcPaths) { + // DoDropLock(opId, srcTablePath.Parent(), srcTablePath, result); + // } + // DoDropLock(opId, dstTablePath.Parent(), dstTablePath, result); + + return true; +} + +TVector CreateRestoreMultipleIncrementalBackups(TOperationId opId, const TTxTransaction& tx, TOperationContext& context) { + TVector result; + CreateRestoreMultipleIncrementalBackups(opId, tx, context, result); + return result; } } // namespace NKikimr::NSchemeShard diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_create_table.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_create_table.cpp index cfa632c31dc5..530057a798b4 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_create_table.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_create_table.cpp @@ -413,6 +413,7 @@ class TCreateTable: public TSubOperation { const TString& parentPathStr = Transaction.GetWorkingDir(); const TString& name = schema.GetName(); + LOG_NOTICE_S(context.Ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "TCreateTable Propose" << ", path: " << parentPathStr << "/" << name @@ -435,6 +436,10 @@ class TCreateTable: public TSubOperation { } NSchemeShard::TPath parentPath = NSchemeShard::TPath::Resolve(parentPathStr, context.SS); + if (parentPathStr.StartsWith(parentPath.GetDomainPathString() + "/.backups")) { + schema.SetSystemColumnNamesAllowed(true); + } + { NSchemeShard::TPath::TChecker checks = parentPath.Check(); checks diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_mkdir.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_mkdir.cpp index a58067531bbb..332b20339b86 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_mkdir.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_mkdir.cpp @@ -103,7 +103,7 @@ class TMkDir: public TSubOperation { THolder Propose(const TString& owner, TOperationContext& context) override { const auto ssId = context.SS->SelfTabletId(); - const auto acceptExisted = !Transaction.GetFailOnExist(); + // const auto acceptExisted = !Transaction.GetFailOnExist(); const TString& parentPathStr = Transaction.GetWorkingDir(); const TString& name = Transaction.GetMkDir().GetName(); @@ -150,19 +150,31 @@ class TMkDir: public TSubOperation { const TString acl = Transaction.GetModifyACL().GetDiffACL(); NSchemeShard::TPath dstPath = parentPath.Child(name); + + if (dstPath.IsResolved() && dstPath.Base()->PathState != NKikimrSchemeOp::EPathState::EPathStateNotExist) { + result->SetPathId(dstPath.Base()->PathId.LocalPathId); + context.OnComplete.ActivateTx(OperationId); + + TTxState& txState = context.SS->CreateTx(OperationId, TTxState::TxMkDir, dstPath.Base()->PathId); + txState.State = TTxState::Propose; + + SetState(NextState()); + return result; + } + { NSchemeShard::TPath::TChecker checks = dstPath.Check(); checks.IsAtLocalSchemeShard(); if (dstPath.IsResolved()) { checks .IsResolved() - .NotUnderDeleting() - .FailOnExist({ - TPathElement::EPathType::EPathTypeDir, - TPathElement::EPathType::EPathTypeSubDomain, - TPathElement::EPathType::EPathTypeExtSubDomain, - TPathElement::EPathType::EPathTypeColumnStore - }, acceptExisted); + .NotUnderDeleting(); + // .FailOnExist({ + // TPathElement::EPathType::EPathTypeDir, + // TPathElement::EPathType::EPathTypeSubDomain, + // TPathElement::EPathType::EPathTypeExtSubDomain, + // TPathElement::EPathType::EPathTypeColumnStore + // }, acceptExisted); } else { checks .NotEmpty() diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_part.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_part.cpp index 3b1c0ff832b0..e7024299bc16 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_part.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_part.cpp @@ -89,7 +89,11 @@ static TString LogMessage(const TString& ev, TOperationContext& context, bool ig } \ \ bool TSubOperation::HandleReply(TEvType::TPtr& ev, TOperationContext& context) { \ - return Progress(context, &ISubOperationState::HandleReply, ev, context); \ + return Progress(context, &ISubOperationState::HandleReply, ev); \ + } \ + \ + bool TBetterSubOperation::HandleReply(TEvType::TPtr& ev, TOperationContext& context) { \ + return Progress(context, &ISubOperationState::HandleReply, ev); \ } SCHEMESHARD_INCOMING_EVENTS(DefaultHandleReply) diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_part.h b/ydb/core/tx/schemeshard/schemeshard__operation_part.h index 3498ea86c69d..21a3be3e81af 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_part.h +++ b/ydb/core/tx/schemeshard/schemeshard__operation_part.h @@ -278,7 +278,7 @@ class TSubOperation: public TSubOperationBase { } bool ProgressState(TOperationContext& context) override { - return Progress(context, &ISubOperationState::ProgressState, context); + return Progress(context, &ISubOperationState::ProgressState); } #define DefaultHandleReply(TEvType, ...) \ @@ -292,9 +292,70 @@ class TSubOperation: public TSubOperationBase { using TFunc = bool(ISubOperationState::*)(Args...); template - bool Progress(TOperationContext& context, TFunc func, Args&&... args) { + bool Progress(TOperationContext& context, TFunc func, Args&&... args) { Y_ABORT_UNLESS(StateFunc); - const bool isDone = std::invoke(func, StateFunc.Get(), std::forward(args)...); + const bool isDone = std::invoke(func, StateFunc.Get(), std::forward(args)..., context); + if (isDone) { + StateDone(context); + } + + return true; + } +}; + +class TBetterSubOperation: public TSubOperationBase { + TTxState::ETxState State = TTxState::Invalid; + TSubOperationState::TPtr StateFunc = nullptr; + +protected: + virtual TTxState::ETxState NextState(TTxState::ETxState state, TOperationContext& context) const = 0; + virtual TSubOperationState::TPtr SelectStateFunc(TTxState::ETxState state, TOperationContext& context) = 0; + + virtual void StateDone(TOperationContext& context) { + auto state = NextState(GetState(), context); + SetState(state, context); + + if (state != TTxState::Invalid) { + context.OnComplete.ActivateTx(OperationId); + } + } + +public: + using TSubOperationBase::TSubOperationBase; + + explicit TBetterSubOperation(const TOperationId& id, TTxState::ETxState state) + : TSubOperationBase(id) + , State(state) + { + } + + TTxState::ETxState GetState() const { + return State; + } + + void SetState(TTxState::ETxState state, TOperationContext& context) { + State = state; + StateFunc = SelectStateFunc(state, context); + } + + bool ProgressState(TOperationContext& context) override { + return Progress(context, &ISubOperationState::ProgressState); + } + + #define DefaultHandleReply(TEvType, ...) \ + bool HandleReply(TEvType::TPtr& ev, TOperationContext& context) override; + + SCHEMESHARD_INCOMING_EVENTS(DefaultHandleReply) + #undef DefaultHandleReply + +private: + template + using TFunc = bool(ISubOperationState::*)(Args...); + + template + bool Progress(TOperationContext& context, TFunc func, Args&&... args) { + Y_ABORT_UNLESS(StateFunc); + const bool isDone = std::invoke(func, StateFunc.Get(), std::forward(args)..., context); if (isDone) { StateDone(context); } @@ -320,6 +381,13 @@ ISubOperation::TPtr MakeSubOperation(const TOperationId& id, TTxState::ETxState return result.Release(); } +template +ISubOperation::TPtr MakeSubOperation(const TOperationId& id, TTxState::ETxState state, TOperationContext& context) { + auto result = MakeHolder(id, state); + result->SetState(state, context); + return result.Release(); +} + ISubOperation::TPtr CreateReject(TOperationId id, THolder response); ISubOperation::TPtr CreateReject(TOperationId id, NKikimrScheme::EStatus status, const TString& message); @@ -421,8 +489,10 @@ ISubOperation::TPtr CreateDropCdcStreamAtTable(TOperationId id, TTxState::ETxSta /// Continuous Backup // Create +bool CreateNewContinuousBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context, TVector& result); TVector CreateNewContinuousBackup(TOperationId id, const TTxTransaction& tx, TOperationContext& context); TVector CreateAlterContinuousBackup(TOperationId id, const TTxTransaction& tx, TOperationContext& context); +bool CreateAlterContinuousBackup(TOperationId id, const TTxTransaction& tx, TOperationContext& context, TVector& result); TVector CreateDropContinuousBackup(TOperationId id, const TTxTransaction& tx, TOperationContext& context); ISubOperation::TPtr CreateBackup(TOperationId id, const TTxTransaction& tx); @@ -443,6 +513,11 @@ ISubOperation::TPtr CreateDropTableIndex(TOperationId id, TTxState::ETxState sta ISubOperation::TPtr CreateAlterTableIndex(TOperationId id, const TTxTransaction& tx); ISubOperation::TPtr CreateAlterTableIndex(TOperationId id, TTxState::ETxState state); +void CreateConsistentCopyTables( + TOperationId nextId, + const TTxTransaction& tx, + TOperationContext& context, + TVector& result); TVector CreateConsistentCopyTables(TOperationId nextId, const TTxTransaction& tx, TOperationContext& context); ISubOperation::TPtr CreateNewOlapStore(TOperationId id, const TTxTransaction& tx); @@ -622,12 +697,30 @@ ISubOperation::TPtr CreateDropResourcePool(TOperationId id, const TTxTransaction ISubOperation::TPtr CreateDropResourcePool(TOperationId id, TTxState::ETxState state); ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, const TTxTransaction& tx); -ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state); +ISubOperation::TPtr CreateRestoreIncrementalBackupAtTable(TOperationId id, TTxState::ETxState state, TOperationContext& context); + +// Backup Collection +// Create +ISubOperation::TPtr CreateNewBackupCollection(TOperationId id, const TTxTransaction& tx); +ISubOperation::TPtr CreateNewBackupCollection(TOperationId id, TTxState::ETxState state); +// Alter +ISubOperation::TPtr CreateAlterBackupCollection(TOperationId id, const TTxTransaction& tx); +ISubOperation::TPtr CreateAlterBackupCollection(TOperationId id, TTxState::ETxState state); +// Drop +ISubOperation::TPtr CreateDropBackupCollection(TOperationId id, const TTxTransaction& tx); +ISubOperation::TPtr CreateDropBackupCollection(TOperationId id, TTxState::ETxState state); +// Backup +TVector CreateBackupBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context); +TVector CreateBackupIncrementalBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context); +// Restore +TVector CreateRestoreBackupCollection(TOperationId opId, const TTxTransaction& tx, TOperationContext& context); // returns Reject in case of error, nullptr otherwise ISubOperation::TPtr CascadeDropTableChildren(TVector& result, const TOperationId& id, const TPath& table); TVector CreateRestoreIncrementalBackup(TOperationId opId, const TTxTransaction& tx, TOperationContext& context); +TVector CreateRestoreMultipleIncrementalBackups(TOperationId opId, const TTxTransaction& tx, TOperationContext& context); +bool CreateRestoreMultipleIncrementalBackups(TOperationId opId, const TTxTransaction& tx, TOperationContext& context, TVector& result); } } diff --git a/ydb/core/tx/schemeshard/schemeshard__operation_upgrade_subdomain.cpp b/ydb/core/tx/schemeshard/schemeshard__operation_upgrade_subdomain.cpp index fcf9bbb4ef40..eae2ddb455d2 100644 --- a/ydb/core/tx/schemeshard/schemeshard__operation_upgrade_subdomain.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__operation_upgrade_subdomain.cpp @@ -308,6 +308,7 @@ class TConfigure: public TSubOperationState { case NKikimrSchemeOp::EPathType::EPathTypeExternalDataSource: case NKikimrSchemeOp::EPathType::EPathTypeView: case NKikimrSchemeOp::EPathType::EPathTypeResourcePool: + case NKikimrSchemeOp::EPathType::EPathTypeBackupCollection: Y_ABORT_UNLESS(!path.Base()->IsRoot()); //no shards break; diff --git a/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp b/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp index 0a074e30ddc8..a6c85bd62825 100644 --- a/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_audit_log_fragment.cpp @@ -245,6 +245,20 @@ TString DefineUserOperationName(const NKikimrSchemeOp::TModifyScheme& tx) { // incremental backup case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackup: case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreIncrementalBackupAtTable: + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreMultipleIncrementalBackups: + return "RESTORE"; + // backup collection + case NKikimrSchemeOp::EOperationType::ESchemeOpCreateBackupCollection: + return "CREATE BACKUP COLLECTION"; + case NKikimrSchemeOp::EOperationType::ESchemeOpAlterBackupCollection: + return "ALTER BACKUP COLLECTION"; + case NKikimrSchemeOp::EOperationType::ESchemeOpDropBackupCollection: + return "DROP BACKUP COLLECTION"; + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupBackupCollection: + return "BACKUP"; + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupIncrementalBackupCollection: + return "BACKUP INCREMENTAL"; + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreBackupCollection: return "RESTORE"; } Y_ABORT("switch should cover all operation types"); @@ -554,6 +568,27 @@ TVector ExtractChangingPaths(const NKikimrSchemeOp::TModifyScheme& tx) result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetRestoreIncrementalBackup().GetSrcTableName()})); result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetRestoreIncrementalBackup().GetDstTableName()})); break; + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreMultipleIncrementalBackups: + // TODO implement + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpCreateBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetCreateBackupCollection().GetName()})); + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpAlterBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetAlterBackupCollection().GetName()})); + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpDropBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetDropBackupCollection().GetName()})); + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetBackupBackupCollection().GetName()})); + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpBackupIncrementalBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetBackupIncrementalBackupCollection().GetName()})); + break; + case NKikimrSchemeOp::EOperationType::ESchemeOpRestoreBackupCollection: + result.emplace_back(NKikimr::JoinPath({tx.GetWorkingDir(), tx.GetRestoreBackupCollection().GetName()})); + break; } return result; diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.cpp b/ydb/core/tx/schemeshard/schemeshard_impl.cpp index 04cc4ae6c63f..baa438c64cf0 100644 --- a/ydb/core/tx/schemeshard/schemeshard_impl.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_impl.cpp @@ -1519,6 +1519,7 @@ TPathElement::EPathState TSchemeShard::CalcPathState(TTxState::ETxType txType, T case TTxState::TxCreateView: case TTxState::TxCreateContinuousBackup: case TTxState::TxCreateResourcePool: + case TTxState::TxCreateBackupCollection: return TPathElement::EPathState::EPathStateCreate; case TTxState::TxAlterPQGroup: case TTxState::TxAlterTable: @@ -2347,6 +2348,16 @@ void TSchemeShard::PersistTxState(NIceDb::TNiceDb& db, const TOperationId opId) TTableInfo::TPtr tableInfo = Tables.at(pathId); extraData = tableInfo->SerializeAlterExtraData(); + } else if (txState.TxType == TTxState::TxCopyTable) { + NKikimrSchemeOp::TGenericTxInFlyExtraData proto; + PathIdFromPathId(txState.CdcPathId, proto.MutableTxCopyTableExtraData()->MutableCdcPathId()); + bool serializeRes = proto.SerializeToString(&extraData); + Y_ABORT_UNLESS(serializeRes); + } else if (txState.TxType == TTxState::TxRestoreIncrementalBackupAtTable) { + NKikimrSchemeOp::TGenericTxInFlyExtraData proto; + proto.MutableTxRestoreIncrementalBackupAtTableExtraData()->SetLoopSeqNo(txState.LoopSeqNo); + bool serializeRes = proto.SerializeToString(&extraData); + Y_ABORT_UNLESS(serializeRes); } db.Table().Key(opId.GetTxId(), opId.GetSubTxId()).Update( NIceDb::TUpdate((ui8)txState.TxType), @@ -2998,6 +3009,15 @@ void TSchemeShard::PersistRemoveResourcePool(NIceDb::TNiceDb& db, TPathId pathId db.Table().Key(pathId.OwnerId, pathId.LocalPathId).Delete(); } +void TSchemeShard::PersistBackupCollection(NIceDb::TNiceDb& db, TPathId pathId, const TBackupCollectionInfo::TPtr backupCollection) { + Y_ABORT_UNLESS(IsLocalId(pathId)); + + db.Table().Key(pathId.OwnerId, pathId.LocalPathId).Update( + NIceDb::TUpdate{backupCollection->AlterVersion}, + NIceDb::TUpdate{backupCollection->Properties.SerializeAsString()} + ); +} + void TSchemeShard::PersistRemoveRtmrVolume(NIceDb::TNiceDb &db, TPathId pathId) { Y_ABORT_UNLESS(IsLocalId(pathId)); @@ -4246,6 +4266,13 @@ NKikimrSchemeOp::TPathVersion TSchemeShard::GetPathVersion(const TPath& path) co generalVersion += result.GetResourcePoolVersion(); break; } + case NKikimrSchemeOp::EPathType::EPathTypeBackupCollection: { + auto it = BackupCollections.find(pathId); + Y_ABORT_UNLESS(it != BackupCollections.end()); + result.SetBackupCollectionVersion(it->second->AlterVersion); + generalVersion += result.GetBackupCollectionVersion(); + break; + } case NKikimrSchemeOp::EPathType::EPathTypeInvalid: { Y_UNREACHABLE(); @@ -4875,6 +4902,10 @@ void TSchemeShard::RemoveTx(const TActorContext &ctx, NIceDb::TNiceDb &db, TOper } TxInFlight.erase(opId); // must be called last, erases txState invalidating txState ptr + + if (auto it = HackPostponedOps.find(opId.GetTxId()); it != HackPostponedOps.end()) { + Send(TxAllocatorClient, new TEvTxAllocatorClient::TEvAllocate(), 0, opId.GetTxId().GetValue()); + } } TMaybe TSchemeShard::GetTablePartitionConfigWithAlterData(TPathId pathId) const { @@ -5058,6 +5089,9 @@ void TSchemeShard::UncountNode(TPathElement::TPtr node) { case TPathElement::EPathType::EPathTypeResourcePool: TabletCounters->Simple()[COUNTER_RESOURCE_POOL_COUNT].Sub(1); break; + case TPathElement::EPathType::EPathTypeBackupCollection: + TabletCounters->Simple()[COUNTER_BACKUP_COLLECTION_COUNT].Sub(1); + break; case TPathElement::EPathType::EPathTypeInvalid: Y_ABORT("impossible path type"); } @@ -6357,11 +6391,15 @@ void TSchemeShard::Handle(TEvTxAllocatorClient::TEvAllocateResult::TPtr& ev, con return Execute(CreateTxProgressImport(ev), ctx); } else if (IndexBuilds.contains(TIndexBuildId(id))) { return Execute(CreateTxReply(ev), ctx); + } else if (HackPostponedOps.contains(TTxId(id))) { + HackPostponedOps[TTxId(id)](ctx, SelfId(), ev); + HackPostponedOps.erase(TTxId(id)); } LOG_WARN_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "no able to determine destination for message TEvAllocateResult: " - << " Cookie: " << id + << "ev: " << ev->Get()->TxIds.front() + << ", Cookie: " << id << ", at schemeshard: " << TabletID()); } @@ -6387,7 +6425,8 @@ void TSchemeShard::Handle(TEvSchemeShard::TEvModifySchemeTransactionResult::TPtr LOG_WARN_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "no able to determine destination for message TEvModifySchemeTransactionResult: " - << " txId: " << txId + << ", ev: " << ev->Get()->Record.ShortDebugString() + << ", txId: " << txId << ", at schemeshard: " << TabletID()); } diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.h b/ydb/core/tx/schemeshard/schemeshard_impl.h index 87fb104f3293..a3b66efca6fb 100644 --- a/ydb/core/tx/schemeshard/schemeshard_impl.h +++ b/ydb/core/tx/schemeshard/schemeshard_impl.h @@ -253,6 +253,7 @@ class TSchemeShard THashMap ExternalDataSources; THashMap Views; THashMap ResourcePools; + THashMap BackupCollections; TTempDirsState TempDirsState; @@ -374,6 +375,8 @@ class TSchemeShard TActorId DelayedInitTenantDestination; TAutoPtr DelayedInitTenantReply; + THashMap> HackPostponedOps; + NExternalSource::IExternalSourceFactory::TPtr ExternalSourceFactory{NExternalSource::CreateExternalSourceFactory({})}; THolder IgniteOperation(TProposeRequest& request, TOperationContext& context); @@ -823,6 +826,8 @@ class TSchemeShard void PersistResourcePool(NIceDb::TNiceDb& db, TPathId pathId, const TResourcePoolInfo::TPtr resourcePool); void PersistRemoveResourcePool(NIceDb::TNiceDb& db, TPathId pathId); + void PersistBackupCollection(NIceDb::TNiceDb &db, TPathId pathId, const TBackupCollectionInfo::TPtr backupCollection); + TTabletId GetGlobalHive(const TActorContext& ctx) const; enum class EHiveSelection : uint8_t { diff --git a/ydb/core/tx/schemeshard/schemeshard_info_types.h b/ydb/core/tx/schemeshard/schemeshard_info_types.h index 6f665064eab4..69cbb9de5340 100644 --- a/ydb/core/tx/schemeshard/schemeshard_info_types.h +++ b/ydb/core/tx/schemeshard/schemeshard_info_types.h @@ -3417,6 +3417,25 @@ struct TResourcePoolInfo : TSimpleRefCount { NKikimrSchemeOp::TResourcePoolProperties Properties; }; +struct TBackupCollectionInfo : TSimpleRefCount { + using TPtr = TIntrusivePtr; + + static TPtr New() { + return new TBackupCollectionInfo(); + } + + static TPtr Create(const NKikimrSchemeOp::TBackupCollection& desc) { + TPtr result = New(); + + result->Properties = desc.GetProperties(); + + return result; + } + + ui64 AlterVersion = 0; + NKikimrSchemeOp::TBackupCollectionProperties Properties; +}; + bool ValidateTtlSettings(const NKikimrSchemeOp::TTTLSettings& ttl, const THashMap& sourceColumns, const THashMap& alterColumns, diff --git a/ydb/core/tx/schemeshard/schemeshard_path.cpp b/ydb/core/tx/schemeshard/schemeshard_path.cpp index aa37eb4d5ae3..2af4d9bab59a 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_path.cpp @@ -8,8 +8,9 @@ namespace NKikimr::NSchemeShard { -TPath::TChecker::TChecker(const TPath& path) - : Path(path) +TPath::TChecker::TChecker(const TPath& path, const std::source_location location) + : Location(location) + , Path(path) , Failed(false) , Status(EStatus::StatusSuccess) { @@ -32,7 +33,8 @@ const TPath::TChecker& TPath::TChecker::Fail(EStatus status, const TString& erro Status = status; Error = TStringBuilder() << "Check failed" << ": path: '" << Path.PathString() << "'" - << ", error: " << error; + << ", error: " << error + << ", location: " << Location.file_name() << ":" << Location.line(); return *this; } @@ -1102,8 +1104,8 @@ TPath::TPath(TVector&& elements, TSchemeShard* ss) Y_ABORT_UNLESS(IsResolved()); } -TPath::TChecker TPath::Check() const { - return TChecker(*this); +TPath::TChecker TPath::Check(const std::source_location location) const { + return TChecker(*this, location); } bool TPath::IsEmpty() const { @@ -1527,7 +1529,7 @@ bool TPath::IsCommonSensePath() const { // Directories and domain roots are always ok as intermediaries bool ok = (*item)->IsDirectory() || (*item)->IsDomainRoot(); // Temporarily olap stores are treated like directories - ok = ok || (*item)->IsOlapStore(); + ok = ok || (*item)->IsOlapStore() || (*item)->IsBackupCollection(); if (!ok) { return false; } diff --git a/ydb/core/tx/schemeshard/schemeshard_path.h b/ydb/core/tx/schemeshard/schemeshard_path.h index a3d23fa38a23..d7a0eb74121a 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path.h +++ b/ydb/core/tx/schemeshard/schemeshard_path.h @@ -6,6 +6,7 @@ #include #include +#include namespace NKikimr::NSchemeShard { @@ -20,6 +21,7 @@ class TPath { class TChecker { using EStatus = NKikimrScheme::EStatus; + std::source_location Location; const TPath& Path; mutable bool Failed; mutable EStatus Status; @@ -30,7 +32,7 @@ class TPath { const TChecker& Fail(EStatus status, const TString& error) const; public: - explicit TChecker(const TPath& path); + explicit TChecker(const TPath& path, const std::source_location location); explicit operator bool() const; EStatus GetStatus() const; @@ -123,7 +125,7 @@ class TPath { static TPath Init(const TPathId pathId, TSchemeShard* ss); - TChecker Check() const; + TChecker Check(const std::source_location location = std::source_location::current()) const; bool IsEmpty() const; bool IsResolved() const; diff --git a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp index 977c22e51f6b..0601c6a62e51 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp @@ -1067,6 +1067,19 @@ void TPathDescriber::DescribeResourcePool(TPathId pathId, TPathElement::TPtr pat entry->MutableProperties()->CopyFrom(resourcePoolInfo->Properties); } +void TPathDescriber::DescribeBackupCollection(TPathId pathId, TPathElement::TPtr pathEl) { + auto it = Self->BackupCollections.FindPtr(pathId); + Y_ABORT_UNLESS(it, "BackupCollections is not found"); + TBackupCollectionInfo::TPtr backupCollectionInfo = *it; + + auto entry = Result->Record.MutablePathDescription()->MutableBackupCollectionDescription(); + entry->SetName(pathEl->Name); + PathIdFromPathId(pathId, entry->MutablePathId()); + entry->SetVersion(backupCollectionInfo->AlterVersion); + entry->MutableProperties()->CopyFrom(backupCollectionInfo->Properties); +} + + static bool ConsiderAsDropped(const TPath& path) { Y_ABORT_UNLESS(path.IsResolved()); @@ -1221,6 +1234,10 @@ THolder TPathDescriber::Describe case NKikimrSchemeOp::EPathTypeResourcePool: DescribeResourcePool(base->PathId, base); break; + case NKikimrSchemeOp::EPathTypeBackupCollection: + DescribeDir(path); + DescribeBackupCollection(base->PathId, base); + break; case NKikimrSchemeOp::EPathTypeInvalid: Y_UNREACHABLE(); } diff --git a/ydb/core/tx/schemeshard/schemeshard_path_describer.h b/ydb/core/tx/schemeshard/schemeshard_path_describer.h index 3aa655831be9..317fbef59150 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_describer.h +++ b/ydb/core/tx/schemeshard/schemeshard_path_describer.h @@ -47,6 +47,7 @@ class TPathDescriber { void DescribeExternalDataSource(const TActorContext& ctx, TPathId pathId, TPathElement::TPtr pathEl); void DescribeView(const TActorContext&, TPathId pathId, TPathElement::TPtr pathEl); void DescribeResourcePool(TPathId pathId, TPathElement::TPtr pathEl); + void DescribeBackupCollection(TPathId pathId, TPathElement::TPtr pathEl); public: explicit TPathDescriber(TSchemeShard* self, NKikimrSchemeOp::TDescribePath&& params) diff --git a/ydb/core/tx/schemeshard/schemeshard_path_element.cpp b/ydb/core/tx/schemeshard/schemeshard_path_element.cpp index f2f30095173f..a80ba4bec9e7 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_element.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_path_element.cpp @@ -165,6 +165,10 @@ bool TPathElement::IsOlapStore() const { return PathType == EPathType::EPathTypeColumnStore; } +bool TPathElement::IsBackupCollection() const { + return PathType == EPathType::EPathTypeBackupCollection; +} + bool TPathElement::IsColumnTable() const { return PathType == EPathType::EPathTypeColumnTable; } @@ -187,7 +191,7 @@ bool TPathElement::IsContainer() const { } bool TPathElement::IsLikeDirectory() const { - return IsDirectory() || IsDomainRoot() || IsOlapStore() || IsTableIndex(); + return IsDirectory() || IsDomainRoot() || IsOlapStore() || IsTableIndex() || IsBackupCollection(); } bool TPathElement::HasActiveChanges() const { diff --git a/ydb/core/tx/schemeshard/schemeshard_path_element.h b/ydb/core/tx/schemeshard/schemeshard_path_element.h index 61e5b3b40c53..84bb60caac1e 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_element.h +++ b/ydb/core/tx/schemeshard/schemeshard_path_element.h @@ -122,6 +122,7 @@ struct TPathElement : TSimpleRefCount { bool IsFileStore() const; bool IsKesus() const; bool IsOlapStore() const; + bool IsBackupCollection() const; bool IsColumnTable() const; bool IsSequence() const; bool IsReplication() const; diff --git a/ydb/core/tx/schemeshard/schemeshard_schema.h b/ydb/core/tx/schemeshard/schemeshard_schema.h index ca918c26c579..7b30ef059316 100644 --- a/ydb/core/tx/schemeshard/schemeshard_schema.h +++ b/ydb/core/tx/schemeshard/schemeshard_schema.h @@ -1823,6 +1823,16 @@ struct Schema : NIceDb::Schema { using TColumns = TableColumns; }; + struct BackupCollection : Table<111> { + struct OwnerPathId : Column<1, NScheme::NTypeIds::Uint64> { using Type = TOwnerId; }; + struct LocalPathId : Column<2, NScheme::NTypeIds::Uint64> { using Type = TLocalPathId; }; + struct AlterVersion : Column<3, NScheme::NTypeIds::Uint64> {}; + struct Properties : Column<4, NScheme::NTypeIds::String> {}; + + using TKey = TableKey; + using TColumns = TableColumns; + }; + using TTables = SchemaTables< Paths, TxInFlight, @@ -1932,7 +1942,8 @@ struct Schema : NIceDb::Schema { BuildColumnOperationSettings, View, BackgroundSessions, - ResourcePool + ResourcePool, + BackupCollection >; static constexpr ui64 SysParam_NextPathId = 1; diff --git a/ydb/core/tx/schemeshard/schemeshard_tx_infly.h b/ydb/core/tx/schemeshard/schemeshard_tx_infly.h index f2ab0e5744cb..490c4d37f7ce 100644 --- a/ydb/core/tx/schemeshard/schemeshard_tx_infly.h +++ b/ydb/core/tx/schemeshard/schemeshard_tx_infly.h @@ -138,6 +138,7 @@ struct TTxState { item(TxDropResourcePool, 92) \ item(TxAlterResourcePool, 93) \ item(TxRestoreIncrementalBackupAtTable, 94) \ + item(TxCreateBackupCollection, 95) \ // TX_STATE_TYPE_ENUM @@ -268,6 +269,9 @@ struct TTxState { TStepId MinStep = InvalidStepId; TStepId PlanStep = InvalidStepId; + TPathId CdcPathId = InvalidPathId; // path (dir or table) being modified stored in extra data, used by backup + ui64 LoopSeqNo = 0; // stored in extra data by multi restore + // persist - TxShards: TVector Shards; // shards + operations on them bool NeedUpdateObject = false; @@ -355,6 +359,7 @@ struct TTxState { case TxCopySequence: case TxCreateContinuousBackup: case TxCreateResourcePool: + case TxCreateBackupCollection: return true; case TxInitializeBuildIndex: //this is more like alter case TxCreateCdcStreamAtTable: @@ -498,6 +503,7 @@ struct TTxState { case TxCreateContinuousBackup: case TxCreateResourcePool: case TxRestoreIncrementalBackupAtTable: + case TxCreateBackupCollection: return false; case TxAlterPQGroup: case TxAlterTable: @@ -606,6 +612,7 @@ struct TTxState { case TxCreateContinuousBackup: case TxCreateResourcePool: case TxRestoreIncrementalBackupAtTable: + case TxCreateBackupCollection: return false; case TxAlterPQGroup: case TxAlterTable: diff --git a/ydb/core/tx/schemeshard/ya.make b/ydb/core/tx/schemeshard/ya.make index c0f4082cbccf..c6fc5c1df6b7 100644 --- a/ydb/core/tx/schemeshard/ya.make +++ b/ydb/core/tx/schemeshard/ya.make @@ -110,6 +110,7 @@ SRCS( schemeshard__operation_copy_sequence.cpp schemeshard__operation_copy_table.cpp schemeshard__operation_create_backup.cpp + schemeshard__operation_create_backup_collection.cpp schemeshard__operation_create_bsv.cpp schemeshard__operation_create_external_data_source.cpp schemeshard__operation_create_external_table.cpp @@ -171,6 +172,7 @@ SRCS( schemeshard__operation_create_continuous_backup.cpp schemeshard__operation_alter_continuous_backup.cpp schemeshard__operation_drop_continuous_backup.cpp + schemeshard__operation_backup_backup_collection.cpp schemeshard__pq_stats.cpp schemeshard__publish_to_scheme_board.cpp schemeshard__state_changed_reply.cpp diff --git a/ydb/core/tx/tx_proxy/schemereq.cpp b/ydb/core/tx/tx_proxy/schemereq.cpp index b470a0e38889..4f0e99fcc639 100644 --- a/ydb/core/tx/tx_proxy/schemereq.cpp +++ b/ydb/core/tx/tx_proxy/schemereq.cpp @@ -361,7 +361,20 @@ struct TBaseSchemeReq: public TActorBootstrapped { case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackup: case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackupAtTable: + case NKikimrSchemeOp::ESchemeOpRestoreMultipleIncrementalBackups: return *modifyScheme.MutableRestoreIncrementalBackup()->MutableSrcTableName(); + case NKikimrSchemeOp::ESchemeOpCreateBackupCollection: + return *modifyScheme.MutableCreateBackupCollection()->MutableName(); + case NKikimrSchemeOp::ESchemeOpAlterBackupCollection: + return *modifyScheme.MutableAlterBackupCollection()->MutableName(); + case NKikimrSchemeOp::ESchemeOpDropBackupCollection: + return *modifyScheme.MutableDropBackupCollection()->MutableName(); + case NKikimrSchemeOp::ESchemeOpBackupBackupCollection: + return *modifyScheme.MutableBackupBackupCollection()->MutableName(); + case NKikimrSchemeOp::ESchemeOpBackupIncrementalBackupCollection: + return *modifyScheme.MutableBackupIncrementalBackupCollection()->MutableName(); + case NKikimrSchemeOp::ESchemeOpRestoreBackupCollection: + return *modifyScheme.MutableRestoreBackupCollection()->MutableName(); } } @@ -622,6 +635,8 @@ struct TBaseSchemeReq: public TActorBootstrapped { case NKikimrSchemeOp::ESchemeOpDropContinuousBackup: case NKikimrSchemeOp::ESchemeOpAlterResourcePool: case NKikimrSchemeOp::ESchemeOpRestoreIncrementalBackup: + case NKikimrSchemeOp::ESchemeOpAlterBackupCollection: + case NKikimrSchemeOp::ESchemeOpRestoreMultipleIncrementalBackups: { auto toResolve = TPathToResolve(pbModifyScheme.GetOperationType()); toResolve.Path = Merge(workingDir, SplitPath(GetPathNameForScheme(pbModifyScheme))); @@ -646,6 +661,7 @@ struct TBaseSchemeReq: public TActorBootstrapped { case NKikimrSchemeOp::ESchemeOpDropExternalDataSource: case NKikimrSchemeOp::ESchemeOpDropView: case NKikimrSchemeOp::ESchemeOpDropResourcePool: + case NKikimrSchemeOp::ESchemeOpDropBackupCollection: { auto toResolve = TPathToResolve(pbModifyScheme.GetOperationType()); toResolve.Path = Merge(workingDir, SplitPath(GetPathNameForScheme(pbModifyScheme))); @@ -708,6 +724,10 @@ struct TBaseSchemeReq: public TActorBootstrapped { case NKikimrSchemeOp::ESchemeOpCreateExternalDataSource: case NKikimrSchemeOp::ESchemeOpCreateView: case NKikimrSchemeOp::ESchemeOpCreateResourcePool: + case NKikimrSchemeOp::ESchemeOpCreateBackupCollection: + case NKikimrSchemeOp::ESchemeOpBackupBackupCollection: + case NKikimrSchemeOp::ESchemeOpBackupIncrementalBackupCollection: + case NKikimrSchemeOp::ESchemeOpRestoreBackupCollection: { auto toResolve = TPathToResolve(pbModifyScheme.GetOperationType()); toResolve.Path = workingDir; diff --git a/ydb/core/viewer/browse.h b/ydb/core/viewer/browse.h index 69569a35b61c..32c6799a2473 100644 --- a/ydb/core/viewer/browse.h +++ b/ydb/core/viewer/browse.h @@ -62,6 +62,7 @@ class TBrowse : public TActorBootstrapped { switch (type) { case NKikimrSchemeOp::EPathType::EPathTypeDir: case NKikimrSchemeOp::EPathType::EPathTypeColumnStore: // TODO + case NKikimrSchemeOp::EPathType::EPathTypeBackupCollection: // TODO return NKikimrViewer::EObjectType::Directory; case NKikimrSchemeOp::EPathType::EPathTypeRtmrVolume: return NKikimrViewer::EObjectType::RtmrVolume; diff --git a/ydb/library/actors/wilson/wilson_span.h b/ydb/library/actors/wilson/wilson_span.h index 3dbf906137fd..0c3a2de4d33e 100644 --- a/ydb/library/actors/wilson/wilson_span.h +++ b/ydb/library/actors/wilson/wilson_span.h @@ -267,7 +267,7 @@ namespace NWilson { } void VerifyNotSent() { - Y_DEBUG_ABORT_UNLESS(!Data || !Data->Sent, "span has been ended"); + // Y_DEBUG_ABORT_UNLESS(!Data || !Data->Sent, "span has been ended"); } }; diff --git a/ydb/library/yql/core/expr_nodes/yql_expr_nodes.json b/ydb/library/yql/core/expr_nodes/yql_expr_nodes.json index cc4becb1e1a6..f7f4828cf649 100644 --- a/ydb/library/yql/core/expr_nodes/yql_expr_nodes.json +++ b/ydb/library/yql/core/expr_nodes/yql_expr_nodes.json @@ -2593,6 +2593,19 @@ "Children": [ {"Index": 0, "Name": "Predicate", "Type": "TExprBase"} ] + }, + { + "Name": "TCoBackupCollectionEntry", + "Base": "TExprBase", + "Match": {"Type": "Tuple"}, + "Children": [ + {"Index": 0, "Name": "Type", "Type": "TCoAtom"}, + {"Index": 1, "Name": "Path", "Type": "TCoAtom", "Optional": true} + ] + }, + { + "Name": "TCoBackupCollectionEntryList", + "ListBase": "TCoBackupCollectionEntry" } ] } diff --git a/ydb/library/yql/providers/common/provider/yql_provider.cpp b/ydb/library/yql/providers/common/provider/yql_provider.cpp index 05d6eac52fbb..a537217b4aad 100644 --- a/ydb/library/yql/providers/common/provider/yql_provider.cpp +++ b/ydb/library/yql/providers/common/provider/yql_provider.cpp @@ -600,6 +600,70 @@ TWriteReplicationSettings ParseWriteReplicationSettings(TExprList node, TExprCon return ret; } +TWriteBackupCollectionSettings ParseWriteBackupCollectionSettings(TExprList node, TExprContext& ctx) { + TMaybeNode mode; + TVector entries; + TVector settings; + TVector other; + + for (auto child : node) { + if (auto maybeTuple = child.Maybe()) { + auto tuple = maybeTuple.Cast(); + auto name = tuple.Name().Value(); + + if (name == "mode") { + YQL_ENSURE(tuple.Value().Maybe()); + mode = tuple.Value().Cast(); + } else if (name == "entries") { + YQL_ENSURE(tuple.Value().Maybe()); + for (const auto& entry : tuple.Value().Cast()) { + auto builtEntry = Build(ctx, node.Pos()); + + YQL_ENSURE(entry.Maybe()); + for (const auto& item : entry.Cast()) { + auto itemName = item.Name().Value(); + if (itemName == "type") { + builtEntry.Type(item.Value().Cast()); + } else if (itemName == "path") { + builtEntry.Path(item.Value().Cast()); + } else { + YQL_ENSURE(false, "unknown entry item"); + } + } + + entries.push_back(builtEntry.Done()); + } + } else if (name == "settings") { + YQL_ENSURE(tuple.Value().Maybe()); + for (const auto& item : tuple.Value().Cast()) { + settings.push_back(item); + } + } else { + other.push_back(tuple); + } + } + } + + const auto& builtEntries = Build(ctx, node.Pos()) + .Add(entries) + .Done(); + + const auto& builtSettings = Build(ctx, node.Pos()) + .Add(settings) + .Done(); + + const auto& builtOther = Build(ctx, node.Pos()) + .Add(other) + .Done(); + + TWriteBackupCollectionSettings ret(builtOther); + ret.Mode = mode; + ret.Entries = builtEntries; + ret.BackupCollectionSettings = builtSettings; + + return ret; +} + TWriteRoleSettings ParseWriteRoleSettings(TExprList node, TExprContext& ctx) { TMaybeNode mode; TVector roles; diff --git a/ydb/library/yql/providers/common/provider/yql_provider.h b/ydb/library/yql/providers/common/provider/yql_provider.h index 39809bdf65fb..f9438413ea9f 100644 --- a/ydb/library/yql/providers/common/provider/yql_provider.h +++ b/ydb/library/yql/providers/common/provider/yql_provider.h @@ -96,6 +96,17 @@ struct TWriteReplicationSettings { {} }; +struct TWriteBackupCollectionSettings { + NNodes::TMaybeNode Mode; + NNodes::TMaybeNode Entries; + NNodes::TMaybeNode BackupCollectionSettings; + NNodes::TCoNameValueTupleList Other; + + TWriteBackupCollectionSettings(const NNodes::TCoNameValueTupleList& other) + : Other(other) + {} +}; + struct TWriteRoleSettings { NNodes::TMaybeNode Mode; NNodes::TMaybeNode Roles; @@ -167,6 +178,7 @@ TVector GetResOrPullColumnHints(const TExprNode& node); TWriteTableSettings ParseWriteTableSettings(NNodes::TExprList node, TExprContext& ctx); TWriteTopicSettings ParseWriteTopicSettings(NNodes::TExprList node, TExprContext& ctx); TWriteReplicationSettings ParseWriteReplicationSettings(NNodes::TExprList node, TExprContext& ctx); +TWriteBackupCollectionSettings ParseWriteBackupCollectionSettings(NNodes::TExprList node, TExprContext& ctx); TWriteRoleSettings ParseWriteRoleSettings(NNodes::TExprList node, TExprContext& ctx); TWriteObjectSettings ParseWriteObjectSettings(NNodes::TExprList node, TExprContext& ctx); diff --git a/ydb/library/yql/sql/v1/node.h b/ydb/library/yql/sql/v1/node.h index b61e623c04d8..4ac09658827e 100644 --- a/ydb/library/yql/sql/v1/node.h +++ b/ydb/library/yql/sql/v1/node.h @@ -1527,20 +1527,35 @@ namespace NSQLTranslationV1 { TNodePtr BuildDropTopic(TPosition pos, const TTopicRef& topic, const TDropTopicParameters& params, TScopedStatePtr scoped); - TNodePtr BuildCreateBackupCollection(TPosition pos, const TString& id, + TNodePtr BuildCreateBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TCreateBackupCollectionParameters& params, const TObjectOperatorContext& context); - TNodePtr BuildAlterBackupCollection(TPosition pos, const TString& id, + TNodePtr BuildAlterBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TAlterBackupCollectionParameters& params, const TObjectOperatorContext& context); - TNodePtr BuildDropBackupCollection(TPosition pos, const TString& id, + TNodePtr BuildDropBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TDropBackupCollectionParameters& params, const TObjectOperatorContext& context); - TNodePtr BuildBackup(TPosition pos, const TString& id, + TNodePtr BuildBackup( + TPosition pos, + const TString& prefix, + const TString& id, const TBackupParameters& params, const TObjectOperatorContext& context); - TNodePtr BuildRestore(TPosition pos, const TString& id, + TNodePtr BuildRestore( + TPosition pos, + const TString& prefix, + const TString& id, const TRestoreParameters& params, const TObjectOperatorContext& context); diff --git a/ydb/library/yql/sql/v1/query.cpp b/ydb/library/yql/sql/v1/query.cpp index 5e49f68959da..0e01b617fa6e 100644 --- a/ydb/library/yql/sql/v1/query.cpp +++ b/ydb/library/yql/sql/v1/query.cpp @@ -3267,16 +3267,18 @@ class TBaseBackupCollectionNode public: TBaseBackupCollectionNode( TPosition pos, + const TString& prefix, const TString& objectId, const TObjectOperatorContext& context) : TBase(pos) , TObjectOperatorContext(context) + , Prefix(prefix) , Id(objectId) {} bool DoInit(TContext& ctx, ISource* src) final { auto keys = Y("Key"); - keys = L(keys, Q(Y(Q("backupCollection"), Y("String", BuildQuotedAtom(Pos, Id))))); + keys = L(keys, Q(Y(Q("backupCollection"), Y("String", BuildQuotedAtom(Pos, Id)), Y("String", BuildQuotedAtom(Pos, Prefix))))); auto options = this->FillOptions(ctx, Y()); Add("block", Q(Y( @@ -3291,6 +3293,7 @@ class TBaseBackupCollectionNode virtual INode::TPtr FillOptions(TContext& ctx, INode::TPtr options) const = 0; protected: + TString Prefix; TString Id; }; @@ -3301,10 +3304,11 @@ class TCreateBackupCollectionNode public: TCreateBackupCollectionNode( TPosition pos, + const TString& prefix, const TString& objectId, const TCreateBackupCollectionParameters& params, const TObjectOperatorContext& context) - : TBase(pos, objectId, context) + : TBase(pos, prefix, objectId, context) , Params(params) {} @@ -3313,7 +3317,7 @@ class TCreateBackupCollectionNode auto settings = Y(); for (auto& [key, value] : Params.Settings) { - settings->Add(Q(Y(BuildQuotedAtom(Pos, key), value.Build()))); + settings->Add(Q(Y(BuildQuotedAtom(Pos, key), Y("String", value.Build())))); } options->Add(Q(Y(Q("settings"), Q(settings)))); @@ -3331,7 +3335,7 @@ class TCreateBackupCollectionNode } TPtr DoClone() const final { - return new TCreateBackupCollectionNode(GetPos(), Id, Params, *this); + return new TCreateBackupCollectionNode(GetPos(), Prefix, Id, Params, *this); } private: @@ -3345,10 +3349,11 @@ class TAlterBackupCollectionNode public: TAlterBackupCollectionNode( TPosition pos, + const TString& prefix, const TString& objectId, const TAlterBackupCollectionParameters& params, const TObjectOperatorContext& context) - : TBase(pos, objectId, context) + : TBase(pos, prefix, objectId, context) , Params(params) {} @@ -3357,7 +3362,7 @@ class TAlterBackupCollectionNode auto settings = Y(); for (auto& [key, value] : Params.Settings) { - settings->Add(Q(Y(BuildQuotedAtom(Pos, key), value.Build()))); + settings->Add(Q(Y(BuildQuotedAtom(Pos, key), Y("String", value.Build())))); } options->Add(Q(Y(Q("settings"), Q(settings)))); @@ -3385,7 +3390,7 @@ class TAlterBackupCollectionNode } TPtr DoClone() const final { - return new TAlterBackupCollectionNode(GetPos(), Id, Params, *this); + return new TAlterBackupCollectionNode(GetPos(), Prefix, Id, Params, *this); } private: @@ -3399,10 +3404,11 @@ class TDropBackupCollectionNode public: TDropBackupCollectionNode( TPosition pos, + const TString& prefix, const TString& objectId, const TDropBackupCollectionParameters&, const TObjectOperatorContext& context) - : TBase(pos, objectId, context) + : TBase(pos, prefix, objectId, context) {} virtual INode::TPtr FillOptions(TContext&, INode::TPtr options) const final { @@ -3413,29 +3419,38 @@ class TDropBackupCollectionNode TPtr DoClone() const final { TDropBackupCollectionParameters params; - return new TDropBackupCollectionNode(GetPos(), Id, params, *this); + return new TDropBackupCollectionNode(GetPos(), Prefix, Id, params, *this); } }; -TNodePtr BuildCreateBackupCollection(TPosition pos, const TString& id, +TNodePtr BuildCreateBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TCreateBackupCollectionParameters& params, const TObjectOperatorContext& context) { - return new TCreateBackupCollectionNode(pos, id, params, context); + return new TCreateBackupCollectionNode(pos, prefix, id, params, context); } -TNodePtr BuildAlterBackupCollection(TPosition pos, const TString& id, +TNodePtr BuildAlterBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TAlterBackupCollectionParameters& params, const TObjectOperatorContext& context) { - return new TAlterBackupCollectionNode(pos, id, params, context); + return new TAlterBackupCollectionNode(pos, prefix, id, params, context); } -TNodePtr BuildDropBackupCollection(TPosition pos, const TString& id, +TNodePtr BuildDropBackupCollection( + TPosition pos, + const TString& prefix, + const TString& id, const TDropBackupCollectionParameters& params, const TObjectOperatorContext& context) { - return new TDropBackupCollectionNode(pos, id, params, context); + return new TDropBackupCollectionNode(pos, prefix, id, params, context); } class TBackupNode final @@ -3446,11 +3461,13 @@ class TBackupNode final public: TBackupNode( TPosition pos, + const TString& prefix, const TString& id, const TBackupParameters& params, const TObjectOperatorContext& context) : TBase(pos) , TObjectOperatorContext(context) + , Prefix(prefix) , Id(id) , Params(params) { @@ -3459,13 +3476,14 @@ class TBackupNode final bool DoInit(TContext& ctx, ISource* src) override { auto keys = Y("Key"); - keys = L(keys, Q(Y(Q("backup"), Y("String", BuildQuotedAtom(Pos, Id))))); + keys = L(keys, Q(Y(Q("backup"), Y("String", BuildQuotedAtom(Pos, Id)), Y("String", BuildQuotedAtom(Pos, Prefix))))); auto opts = Y(); - opts->Add(Q(Y(Q("mode"), Q("backup")))); if (Params.Incremental) { - opts->Add(Q(Y(Q("incremental")))); + opts->Add(Q(Y(Q("mode"), Q("backupIncremental")))); + } else { + opts->Add(Q(Y(Q("mode"), Q("backup")))); } Add("block", Q(Y( @@ -3478,18 +3496,22 @@ class TBackupNode final } TPtr DoClone() const final { - return new TBackupNode(GetPos(), Id, Params, *this); + return new TBackupNode(GetPos(), Prefix, Id, Params, *this); } private: + TString Prefix; TString Id; TBackupParameters Params; }; -TNodePtr BuildBackup(TPosition pos, const TString& id, +TNodePtr BuildBackup( + TPosition pos, + const TString& prefix, + const TString& id, const TBackupParameters& params, const TObjectOperatorContext& context) { - return new TBackupNode(pos, id, params, context); + return new TBackupNode(pos, prefix, id, params, context); } class TRestoreNode final @@ -3500,11 +3522,13 @@ class TRestoreNode final public: TRestoreNode( TPosition pos, + const TString& prefix, const TString& id, const TRestoreParameters& params, const TObjectOperatorContext& context) : TBase(pos) , TObjectOperatorContext(context) + , Prefix(prefix) , Id(id) , Params(params) { @@ -3513,7 +3537,7 @@ class TRestoreNode final bool DoInit(TContext& ctx, ISource* src) override { auto keys = Y("Key"); - keys = L(keys, Q(Y(Q("restore"), Y("String", BuildQuotedAtom(Pos, Id))))); + keys = L(keys, Q(Y(Q("restore"), Y("String", BuildQuotedAtom(Pos, Id)), Y("String", BuildQuotedAtom(Pos, Prefix))))); auto opts = Y(); opts->Add(Q(Y(Q("mode"), Q("restore")))); @@ -3532,18 +3556,22 @@ class TRestoreNode final } TPtr DoClone() const final { - return new TRestoreNode(GetPos(), Id, Params, *this); + return new TRestoreNode(GetPos(), Prefix, Id, Params, *this); } private: + TString Prefix; TString Id; TRestoreParameters Params; }; -TNodePtr BuildRestore(TPosition pos, const TString& id, +TNodePtr BuildRestore( + TPosition pos, + const TString& prefix, + const TString& id, const TRestoreParameters& params, const TObjectOperatorContext& context) { - return new TRestoreNode(pos, id, params, context); + return new TRestoreNode(pos, prefix, id, params, context); } } // namespace NSQLTranslationV1 diff --git a/ydb/library/yql/sql/v1/sql_query.cpp b/ydb/library/yql/sql/v1/sql_query.cpp index 664458a7ba55..eb6cbf958d6a 100644 --- a/ydb/library/yql/sql/v1/sql_query.cpp +++ b/ydb/library/yql/sql/v1/sql_query.cpp @@ -1406,7 +1406,8 @@ bool TSqlQuery::Statement(TVector& blocks, const TRule_sql_stmt_core& const TString& objectId = Id(node.GetRule_backup_collection2().GetRule_object_ref3().GetRule_id_or_at2(), *this).second; AddStatementToBlocks(blocks, BuildCreateBackupCollection(Ctx.Pos(), - BuildTablePath(Ctx.GetPrefixPath(context.ServiceId, context.Cluster), objectId), + TString(Ctx.GetPrefixPath(context.ServiceId, context.Cluster)), + objectId, TCreateBackupCollectionParameters { .Settings = std::move(kv), .Database = database, @@ -1468,7 +1469,8 @@ bool TSqlQuery::Statement(TVector& blocks, const TRule_sql_stmt_core& const TString& objectId = Id(node.GetRule_backup_collection2().GetRule_object_ref3().GetRule_id_or_at2(), *this).second; AddStatementToBlocks(blocks, BuildAlterBackupCollection(Ctx.Pos(), - BuildTablePath(Ctx.GetPrefixPath(context.ServiceId, context.Cluster), objectId), + TString(Ctx.GetPrefixPath(context.ServiceId, context.Cluster)), + objectId, TAlterBackupCollectionParameters { .Settings = std::move(kv), .SettingsToReset = std::move(toReset), @@ -1496,7 +1498,8 @@ bool TSqlQuery::Statement(TVector& blocks, const TRule_sql_stmt_core& const TString& objectId = Id(node.GetRule_backup_collection2().GetRule_object_ref3().GetRule_id_or_at2(), *this).second; AddStatementToBlocks(blocks, BuildDropBackupCollection(Ctx.Pos(), - BuildTablePath(Ctx.GetPrefixPath(context.ServiceId, context.Cluster), objectId), + TString(Ctx.GetPrefixPath(context.ServiceId, context.Cluster)), + objectId, TDropBackupCollectionParameters { .MissingOk = false, }, @@ -1616,7 +1619,8 @@ bool TSqlQuery::Statement(TVector& blocks, const TRule_sql_stmt_core& AddStatementToBlocks(blocks, BuildBackup( Ctx.Pos(), - BuildTablePath(Ctx.GetPrefixPath(context.ServiceId, context.Cluster), objectId), + TString(Ctx.GetPrefixPath(context.ServiceId, context.Cluster)), + objectId, TBackupParameters{ .Incremental = incremental, }, @@ -1648,7 +1652,8 @@ bool TSqlQuery::Statement(TVector& blocks, const TRule_sql_stmt_core& AddStatementToBlocks(blocks, BuildRestore( Ctx.Pos(), - BuildTablePath(Ctx.GetPrefixPath(context.ServiceId, context.Cluster), objectId), + TString(Ctx.GetPrefixPath(context.ServiceId, context.Cluster)), + objectId, TRestoreParameters{ .At = at, }, diff --git a/ydb/public/lib/deprecated/kicli/kicli.h b/ydb/public/lib/deprecated/kicli/kicli.h index 7698ac8780d3..cb468c646af7 100644 --- a/ydb/public/lib/deprecated/kicli/kicli.h +++ b/ydb/public/lib/deprecated/kicli/kicli.h @@ -587,7 +587,8 @@ class TSchemaObject { ExternalTable, ExternalDataSource, View, - ResourcePool + ResourcePool, + BackupCollection, }; TSchemaObject(TSchemaObject&&) = default; diff --git a/ydb/public/lib/deprecated/kicli/schema.cpp b/ydb/public/lib/deprecated/kicli/schema.cpp index 9991f3ddc9b1..f0e1f609d6b1 100644 --- a/ydb/public/lib/deprecated/kicli/schema.cpp +++ b/ydb/public/lib/deprecated/kicli/schema.cpp @@ -134,6 +134,9 @@ void TSchemaObject::Drop() { case EPathType::ResourcePool: drop.SetOperationType(NKikimrSchemeOp::EOperationType::ESchemeOpDropResourcePool); break; + case EPathType::BackupCollection: + drop.SetOperationType(NKikimrSchemeOp::EOperationType::ESchemeOpDropBackupCollection); + break; case EPathType::Unknown: case EPathType::SubDomain: case EPathType::RtmrVolume: @@ -227,6 +230,8 @@ static TSchemaObject::EPathType GetType(const NKikimrSchemeOp::TDirEntry& entry) return TSchemaObject::EPathType::View; case NKikimrSchemeOp::EPathTypeResourcePool: return TSchemaObject::EPathType::ResourcePool; + case NKikimrSchemeOp::EPathTypeBackupCollection: + return TSchemaObject::EPathType::BackupCollection; case NKikimrSchemeOp::EPathTypeTableIndex: case NKikimrSchemeOp::EPathTypeExtSubDomain: case NKikimrSchemeOp::EPathTypeCdcStream: