diff --git a/internal/backup/partition_backup.go b/internal/backup/partition_backup.go index f2db683d6e93e81d24d7325417250256f5862889..b4496556fc16c525c79f72dd9c7efc81fbf016f7 100644 --- a/internal/backup/partition_backup.go +++ b/internal/backup/partition_backup.go @@ -162,6 +162,37 @@ func (pbm *PartitionBackupManager) backupPartition(ctx context.Context, client g return err } +// Restore restores partition from backup. +func (pbm *PartitionBackupManager) Restore(ctx context.Context, serverInfo storage.ServerInfo, storageName, partitionID, relativePath string) error { + client, err := pbm.createClient(ctx, serverInfo) + if err != nil { + return fmt.Errorf("create partition client: %w", err) + } + + err = pbm.restorePartition(ctx, client, storageName, partitionID, relativePath) + if err != nil { + return fmt.Errorf("restore partition: %w", err) + } + + return nil +} + +func (pbm *PartitionBackupManager) restorePartition(ctx context.Context, client gitalypb.PartitionServiceClient, storageName, partitionID, relativePath string) error { + backupCtx, backupCancel := context.WithTimeout(ctx, pbm.backupTimeout) + defer backupCancel() + + _, err := client.RestorePartition(backupCtx, &gitalypb.RestorePartitionRequest{ + StorageName: storageName, + PartitionId: partitionID, + Repository: &gitalypb.Repository{ + StorageName: storageName, + RelativePath: relativePath, + }, + }) + + return err +} + func (pbm *PartitionBackupManager) listAndQueuePartitions(ctx context.Context, client gitalypb.PartitionServiceClient, storageName string, workQueue chan<- *gitalypb.Partition, logger log.Logger) error { var cursor *gitalypb.PaginationCursor for { diff --git a/internal/backup/partition_backup_test.go b/internal/backup/partition_backup_test.go index 917334df6f34eba17e877bc31ad9f6192f53b9fe..2aa7f98139b8cf51b80212a119a622d9a6b49fd6 100644 --- a/internal/backup/partition_backup_test.go +++ b/internal/backup/partition_backup_test.go @@ -291,3 +291,11 @@ func (m *mockPartitionServiceClient) BackupPartition(ctx context.Context, req *g return &gitalypb.BackupPartitionResponse{}, nil } + +func (m *mockPartitionServiceClient) RestorePartition(ctx context.Context, req *gitalypb.RestorePartitionRequest, opts ...grpc.CallOption) (*gitalypb.RestorePartitionResponse, error) { + if ctx.Err() != nil { + return nil, ctx.Err() + } + + return &gitalypb.RestorePartitionResponse{}, nil +} diff --git a/internal/cli/gitalybackup/partition.go b/internal/cli/gitalybackup/partition.go index 1a2c08d687c18bc1044cde0bead1d425130b95a5..88f4cebd25eee7fd29c84b4d4d61759e68b93fb9 100644 --- a/internal/cli/gitalybackup/partition.go +++ b/internal/cli/gitalybackup/partition.go @@ -33,12 +33,43 @@ func partitionCreateFlags() []cli.Flag { } } +type partitionRestoreSubcommand struct { + parallel int + partitionID string + relativePath string +} + +func (cmd *partitionRestoreSubcommand) flags(ctx *cli.Context) { + cmd.parallel = ctx.Int("parallel") + cmd.partitionID = ctx.String("partition") + cmd.relativePath = ctx.String("repository") +} + +func partitionRestoreFlags() []cli.Flag { + return []cli.Flag{ + &cli.IntFlag{ + Name: "parallel", + Usage: "maximum number of parallel restores per storage", + Value: 2, + }, + &cli.StringFlag{ + Name: "partition", + Usage: "partition ID to restore", + }, + &cli.StringFlag{ + Name: "repository", + Usage: "relative-path to the repository", + }, + } +} + func newPartitionCommand() *cli.Command { return &cli.Command{ Name: "partition", Usage: "Commands to create and restore partition backups", Subcommands: []*cli.Command{ newPartitionCreateCommand(), + newPartitionRestoreCommand(), }, } } @@ -52,6 +83,15 @@ func newPartitionCreateCommand() *cli.Command { } } +func newPartitionRestoreCommand() *cli.Command { + return &cli.Command{ + Name: "restore", + Usage: "Restore partition backups", + Action: partitionRestoreAction, + Flags: partitionRestoreFlags(), + } +} + func partitionCreateAction(cctx *cli.Context) error { logger, err := log.Configure(cctx.App.Writer, "json", "info") if err != nil { @@ -122,3 +162,74 @@ func (cmd *partitionCreateSubcommand) run(ctx context.Context, logger log.Logger return nil } + +func partitionRestoreAction(cctx *cli.Context) error { + logger, err := log.Configure(cctx.App.Writer, "json", "info") + if err != nil { + fmt.Printf("configuring logger failed: %v", err) + return err + } + + ctx, cancel := context.WithCancel(cctx.Context) + defer cancel() + + // Set up signal handling + signals := []os.Signal{syscall.SIGTERM, syscall.SIGINT} + shutdown := make(chan os.Signal, len(signals)) + signal.Notify(shutdown, signals...) + defer func() { + signal.Stop(shutdown) + close(shutdown) + }() + + // Start a goroutine to handle signals + go func() { + if sig, ok := <-shutdown; ok { + logger.Info(fmt.Sprintf("Received signal (%s), cancelling backup", sig)) + } + + cancel() + }() + + ctx, err = storage.InjectGitalyServersEnv(ctx) + if err != nil { + logger.Error(err.Error()) + return err + } + + subcmd := partitionRestoreSubcommand{} + subcmd.flags(cctx) + + if err := subcmd.run(ctx, logger); err != nil { + logger.Error(err.Error()) + return err + } + return nil +} + +func (cmd *partitionRestoreSubcommand) run(ctx context.Context, logger log.Logger) (returnErr error) { + pool := client.NewPool(client.WithDialOptions(client.UnaryInterceptor(), client.StreamInterceptor())) + defer func() { + returnErr = errors.Join(returnErr, pool.Close()) + }() + + manager := backup.NewPartitionBackupManager(pool, backup.WithPartitionConcurrencyLimit(cmd.parallel)) + + gitalyServers, err := storage.ExtractGitalyServers(ctx) + if err != nil { + return fmt.Errorf("extract gitaly servers: %w", err) + } + + for storage, serverInfo := range gitalyServers { + logger.Info(fmt.Sprintf("restoring repository with relative path %s from partition %s in storage %s", cmd.relativePath, cmd.partitionID, storage)) + + err := manager.Restore(ctx, serverInfo, storage, cmd.partitionID, cmd.relativePath) + if err != nil { + return fmt.Errorf("restore: %w", err) + } + + logger.Info(fmt.Sprintf("done restoring partition %s for storage: %s", cmd.partitionID, storage)) + } + + return nil +} diff --git a/internal/gitaly/service/partition/restore_partition.go b/internal/gitaly/service/partition/restore_partition.go new file mode 100644 index 0000000000000000000000000000000000000000..04cb73240260a2333ddd67213a3ea9c2799cbb5c --- /dev/null +++ b/internal/gitaly/service/partition/restore_partition.go @@ -0,0 +1,226 @@ +package partition + +import ( + "archive/tar" + "context" + "encoding/json" + "errors" + "fmt" + "io" + "os" + "path/filepath" + "strings" + + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage/mode" + "gitlab.com/gitlab-org/gitaly/v16/internal/structerr" + "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb" +) + +// RestorePartition restores a partition from a backup tar file located in configured backup sink. +func (s *server) RestorePartition(ctx context.Context, in *gitalypb.RestorePartitionRequest) (_ *gitalypb.RestorePartitionResponse, returnErr error) { + if s.backupSink == nil { + return nil, structerr.NewFailedPrecondition("restore partition: server-side backups are not configured") + } + + tx := storage.ExtractTransaction(ctx) + if tx == nil { + return nil, structerr.NewInternal("restore partition: transaction not initialized") + } + + manifestRelativePath := filepath.Join("partition-manifests", in.GetStorageName(), in.GetPartitionId()+".json") + entry, err := s.getLatestBackupEntry(ctx, manifestRelativePath) + if err != nil { + return nil, fmt.Errorf("get backup entry: %w", err) + } + + backupReader, err := s.backupSink.GetReader(ctx, entry.Path) + if err != nil { + return nil, fmt.Errorf("get backup reader: %w", err) + } + defer backupReader.Close() + + // Create a temporary directory for the extraction + tempDir, err := os.MkdirTemp("", "gitaly-restore-*") + if err != nil { + return nil, fmt.Errorf("create temp dir: %w", err) + } + defer func() { + err := os.RemoveAll(tempDir) + if err != nil { + returnErr = errors.Join(returnErr, fmt.Errorf("remove temp dir: %w", err)) + } + }() + + if err := extractBackup(backupReader, tempDir); err != nil { + return nil, fmt.Errorf("extract backup: %w", err) + } + + fsRoot := tx.FS().Root() + repoRelPath := in.GetRepository().GetRelativePath() + repoAbsPath := filepath.Join(fsRoot, repoRelPath) + + // Verify the repository exists in the temp directory + tempRepoPath := filepath.Join(tempDir, repoRelPath) + if _, err := os.Stat(tempRepoPath); os.IsNotExist(err) { + return nil, fmt.Errorf("repository not found in backup at path: %s", repoRelPath) + } + + // Physically remove the target directory if it exists + if err := os.RemoveAll(repoAbsPath); err != nil { + return nil, fmt.Errorf("remove existing repository: %w", err) + } + + // Copy the repository structure from temp to target using transaction-aware operations + if err := filepath.Walk(tempDir, func(path string, info os.FileInfo, err error) error { + if err != nil { + return err + } + + // Get path relative to the temp directory + relPath, err := filepath.Rel(tempDir, path) + if err != nil { + return fmt.Errorf("get relative path: %w", err) + } + + // Skip if this is not within the repository relative path + if !strings.HasPrefix(relPath, repoRelPath) { + return nil + } + + targetPath := filepath.Join(fsRoot, relPath) + + if info.IsDir() { + // Also create the physical directory for subsequent file operations + if err := os.MkdirAll(targetPath, info.Mode()); err != nil { + return fmt.Errorf("create directory %s: %w", relPath, err) + } + + // For directories, record creation in transaction + if err := storage.RecordDirectoryCreation(tx.FS(), relPath); err != nil { + return fmt.Errorf("record directory creation for %s: %w", relPath, err) + } + + return nil + } else if info.Mode()&os.ModeSymlink != 0 { + linkTarget, err := os.Readlink(path) + if err != nil { + return fmt.Errorf("read symlink: %w", err) + } + + // Remove existing file/symlink if it exists + if _, err := os.Lstat(targetPath); err == nil { + if err := os.Remove(targetPath); err != nil { + return fmt.Errorf("remove existing file: %w", err) + } + } + + // Create the physical symlink + if err := os.Symlink(linkTarget, targetPath); err != nil { + return fmt.Errorf("create symlink %s -> %s: %w", relPath, linkTarget, err) + } + + return nil + } else { + srcFile, err := os.Open(path) + if err != nil { + return fmt.Errorf("open source file: %w", err) + } + defer srcFile.Close() + + // Create the physical file with original permissions + dstFile, err := os.OpenFile(targetPath, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, info.Mode()) + if err != nil { + return fmt.Errorf("create target file: %w", err) + } + defer dstFile.Close() + + if _, err := io.Copy(dstFile, srcFile); err != nil { + return fmt.Errorf("copy file content: %w", err) + } + + // Record the file in the transaction + if err := tx.FS().RecordFile(relPath); err != nil { + return fmt.Errorf("record file creation for %s: %w", relPath, err) + } + + return nil + } + }); err != nil { + return nil, fmt.Errorf("copy repository structure: %w", err) + } + + return &gitalypb.RestorePartitionResponse{}, nil +} + +// extractBackup extracts the contents of a backup tar file to the specified directory. +// It returns any error encountered during extraction. +func extractBackup(reader io.Reader, destDir string) error { + tarReader := tar.NewReader(reader) + for { + header, err := tarReader.Next() + if err == io.EOF { + break + } + if err != nil { + return fmt.Errorf("read tar header: %w", err) + } + + // Partition state is stored in the fs folder of the archive, + // therefore we need to skip the entries that are not in the "fs" directory. + if !strings.HasPrefix(header.Name, "fs/") { + continue + } + + // Remove the "fs/" prefix + relativePath := strings.TrimPrefix(header.Name, "fs/") + targetPath := filepath.Join(destDir, relativePath) + + switch header.Typeflag { + case tar.TypeDir: + if err := os.MkdirAll(targetPath, mode.Directory); err != nil { + return fmt.Errorf("create directory %s: %w", relativePath, err) + } + case tar.TypeReg: + if err := os.MkdirAll(filepath.Dir(targetPath), mode.Directory); err != nil { + return fmt.Errorf("create parent directory for %s: %w", relativePath, err) + } + file, err := os.Create(targetPath) + if err != nil { + return fmt.Errorf("create file %s: %w", relativePath, err) + } + if _, err := io.Copy(file, tarReader); err != nil { + file.Close() + return fmt.Errorf("write file %s: %w", relativePath, err) + } + file.Close() + case tar.TypeSymlink: + if err := os.MkdirAll(filepath.Dir(targetPath), mode.Directory); err != nil { + return fmt.Errorf("create parent directory for symlink %s: %w", relativePath, err) + } + if err := os.Symlink(header.Linkname, targetPath); err != nil { + return fmt.Errorf("create symlink %s -> %s: %w", relativePath, header.Linkname, err) + } + default: + return fmt.Errorf("unsupported file type %d for %s", header.Typeflag, relativePath) + } + } + + return nil +} + +func (s *server) getLatestBackupEntry(ctx context.Context, manifestRelativePath string) (*BackupEntry, error) { + manifestReader, err := s.backupSink.GetReader(ctx, manifestRelativePath) + if err != nil { + return nil, fmt.Errorf("get backup manifest reader: %w", err) + } + defer manifestReader.Close() + + // Manifest file is in reverse chronological order, so first entry is latest + var entry BackupEntry + if err := json.NewDecoder(manifestReader).Decode(&entry); err != nil { + return nil, fmt.Errorf("decode backup entry: %w", err) + } + + return &entry, nil +} diff --git a/internal/gitaly/service/partition/restore_partition_test.go b/internal/gitaly/service/partition/restore_partition_test.go new file mode 100644 index 0000000000000000000000000000000000000000..14d40830c86da6cb0a878c813ce8b5200a8c2bd4 --- /dev/null +++ b/internal/gitaly/service/partition/restore_partition_test.go @@ -0,0 +1,205 @@ +package partition_test + +import ( + "context" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/require" + "gitlab.com/gitlab-org/gitaly/v16/internal/backup" + "gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config" + "gitlab.com/gitlab-org/gitaly/v16/internal/structerr" + "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper" + "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testserver" + "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb" +) + +func TestRestorePartition(t *testing.T) { + if testhelper.IsPraefectEnabled() { + t.Skip(`Praefect currently doesn't support routing the PARTITION scoped RPC messages.`) + } + + type setupData struct { + cfg config.Cfg + ptnClient gitalypb.PartitionServiceClient + repoClient gitalypb.RepositoryServiceClient + storageName string + partitionID string + backupRoot string + backupSink *backup.Sink + } + + for _, tc := range []struct { + desc string + setup func(t *testing.T, ctx context.Context) setupData + prepareRepo func(t *testing.T, ctx context.Context, data setupData) *gitalypb.Repository + expectedErr error + }{ + { + desc: "success", + setup: func(t *testing.T, ctx context.Context) setupData { + backupRoot := testhelper.TempDir(t) + backupSink, err := backup.ResolveSink(ctx, backupRoot) + require.NoError(t, err) + + cfg, ptnClient, repoClient := setupServices(t, + testserver.WithBackupSink(backupSink), + ) + + return setupData{ + cfg: cfg, + ptnClient: ptnClient, + repoClient: repoClient, + storageName: "default", + partitionID: "2", + backupRoot: backupRoot, + backupSink: backupSink, + } + }, + prepareRepo: func(t *testing.T, ctx context.Context, data setupData) *gitalypb.Repository { + // Create a repository and back it up + repo, repoPath := gittest.CreateRepository(t, ctx, data.cfg, gittest.CreateRepositoryConfig{}) + gittest.WriteCommit(t, data.cfg, repoPath, gittest.WithBranch("main"), gittest.WithMessage("initial commit")) + + _, err := data.ptnClient.BackupPartition(ctx, &gitalypb.BackupPartitionRequest{ + StorageName: data.storageName, + PartitionId: data.partitionID, + }) + + if !testhelper.IsWALEnabled() { + testhelper.RequireGrpcError(t, structerr.NewInternal("backup partition: transaction not initialized"), err) + return nil + } else { + require.NoError(t, err) + } + + // Delete the repository to simulate a restore scenario + require.NoError(t, os.RemoveAll(repoPath)) + + return repo + }, + }, + { + desc: "invalid storage", + setup: func(t *testing.T, ctx context.Context) setupData { + backupRoot := testhelper.TempDir(t) + backupSink, err := backup.ResolveSink(ctx, backupRoot) + require.NoError(t, err) + + cfg, ptnClient, repoClient := setupServices(t, + testserver.WithBackupSink(backupSink), + ) + + return setupData{ + cfg: cfg, + ptnClient: ptnClient, + repoClient: repoClient, + storageName: "non-existent", + partitionID: "2", + backupRoot: backupRoot, + backupSink: backupSink, + } + }, + prepareRepo: func(t *testing.T, ctx context.Context, data setupData) *gitalypb.Repository { + return &gitalypb.Repository{ + StorageName: "non-existent", + RelativePath: "some/path", + } + }, + expectedErr: testhelper.WithInterceptedMetadata( + structerr.NewInvalidArgument("get storage: storage name not found"), "storage_name", "non-existent", + ), + }, + { + desc: "no backup sink", + setup: func(t *testing.T, ctx context.Context) setupData { + cfg, ptnClient, repoClient := setupServices(t) + + return setupData{ + cfg: cfg, + ptnClient: ptnClient, + repoClient: repoClient, + storageName: "default", + partitionID: "2", + } + }, + prepareRepo: func(t *testing.T, ctx context.Context, data setupData) *gitalypb.Repository { + return &gitalypb.Repository{ + StorageName: "default", + RelativePath: "some/path", + } + }, + expectedErr: structerr.NewFailedPrecondition("restore partition: server-side backups are not configured"), + }, + { + desc: "backup does not exist", + setup: func(t *testing.T, ctx context.Context) setupData { + backupRoot := testhelper.TempDir(t) + backupSink, err := backup.ResolveSink(ctx, backupRoot) + require.NoError(t, err) + + cfg, ptnClient, repoClient := setupServices(t, + testserver.WithBackupSink(backupSink), + ) + + return setupData{ + cfg: cfg, + ptnClient: ptnClient, + repoClient: repoClient, + storageName: "default", + partitionID: "2", + backupRoot: backupRoot, + backupSink: backupSink, + } + }, + prepareRepo: func(t *testing.T, ctx context.Context, data setupData) *gitalypb.Repository { + return &gitalypb.Repository{ + StorageName: "default", + RelativePath: "some/path", + } + }, + expectedErr: structerr.NewInternal(`get backup entry: get backup manifest reader: sink: new reader for "partition-manifests/default/2.json": doesn't exist`), + }, + } { + t.Run(tc.desc, func(t *testing.T) { + ctx := testhelper.Context(t) + data := tc.setup(t, ctx) + repo := tc.prepareRepo(t, ctx, data) + repoPath := filepath.Join(data.cfg.Storages[0].Path, repo.GetRelativePath()) + + // Verify the repository does not exists before restore + if testhelper.IsWALEnabled() { + require.NoDirExists(t, repoPath) + } + + resp, err := data.ptnClient.RestorePartition(ctx, &gitalypb.RestorePartitionRequest{ + StorageName: data.storageName, + PartitionId: data.partitionID, + Repository: repo, + }) + + // The test relies on the interceptor being configured in the test server. If WAL is not enabled, the interceptor won't be configured, + // and as a result the transaction won't be initialized. + if !testhelper.IsWALEnabled() && + (tc.expectedErr == nil || tc.expectedErr.Error() != structerr.NewFailedPrecondition("restore partition: server-side backups are not configured").Error()) { + tc.expectedErr = structerr.NewInternal("restore partition: transaction not initialized") + } + if tc.expectedErr != nil { + testhelper.RequireGrpcError(t, tc.expectedErr, err) + return + } + + require.NoError(t, err) + testhelper.ProtoEqual(t, &gitalypb.RestorePartitionResponse{}, resp) + + // Verify the repository exists after restore + require.DirExists(t, repoPath) + + // Verify the repository is functional by checking for the main branch + mainExists := gittest.ResolveRevision(t, data.cfg, repoPath, "refs/heads/main") != "" + require.True(t, mainExists, "main branch should exist after restore") + }) + } +} diff --git a/internal/gitaly/storage/storagemgr/middleware.go b/internal/gitaly/storage/storagemgr/middleware.go index b352ec5d77513492d996e5b009f7c1bd2f985804..1ff1ce4c8c670273ae74689cf63e8d9257b2c814 100644 --- a/internal/gitaly/storage/storagemgr/middleware.go +++ b/internal/gitaly/storage/storagemgr/middleware.go @@ -290,9 +290,16 @@ func beginTransactionForPartition(ctx context.Context, logger log.Logger, txRegi } }() - tx, err := partition.Begin(ctx, storage.BeginOptions{ + beginOptions := storage.BeginOptions{ Write: !isReadOnly(methodInfo), - }) + } + + targetRepository, err := methodInfo.TargetRepo(req) + if err == nil && targetRepository != nil { + beginOptions.RelativePaths = []string{targetRepository.GetRelativePath()} + } + + tx, err := partition.Begin(ctx, beginOptions) if err != nil { return transactionalizedRequest{}, fmt.Errorf("begin: %w", err) } diff --git a/proto/go/gitalypb/partition.pb.go b/proto/go/gitalypb/partition.pb.go index 3d0dce5d698e0f99c573a5b6a4d1a477ef262353..b6b6e0b3e5a315ffc46d29492ee2c52ee523c760 100644 --- a/proto/go/gitalypb/partition.pb.go +++ b/proto/go/gitalypb/partition.pb.go @@ -114,6 +114,109 @@ func (*BackupPartitionResponse) Descriptor() ([]byte, []int) { return file_partition_proto_rawDescGZIP(), []int{1} } +// RestorePartitionRequest is a request for the RestorePartition RPC. +type RestorePartitionRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + // storage_name is the name of the storage containing the partition to be restored. + StorageName string `protobuf:"bytes,1,opt,name=storage_name,json=storageName,proto3" json:"storage_name,omitempty"` + // partition_id is the identifier of the specific partition within the storage to + // be restored, which may contain one or more repositories and associated data. + PartitionId string `protobuf:"bytes,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + // repository is the repository to be restored. The storage_name + // and relative_path attributes must be provided. + Repository *Repository `protobuf:"bytes,3,opt,name=repository,proto3" json:"repository,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RestorePartitionRequest) Reset() { + *x = RestorePartitionRequest{} + mi := &file_partition_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RestorePartitionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RestorePartitionRequest) ProtoMessage() {} + +func (x *RestorePartitionRequest) ProtoReflect() protoreflect.Message { + mi := &file_partition_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RestorePartitionRequest.ProtoReflect.Descriptor instead. +func (*RestorePartitionRequest) Descriptor() ([]byte, []int) { + return file_partition_proto_rawDescGZIP(), []int{2} +} + +func (x *RestorePartitionRequest) GetStorageName() string { + if x != nil { + return x.StorageName + } + return "" +} + +func (x *RestorePartitionRequest) GetPartitionId() string { + if x != nil { + return x.PartitionId + } + return "" +} + +func (x *RestorePartitionRequest) GetRepository() *Repository { + if x != nil { + return x.Repository + } + return nil +} + +// RestorePartitionResponse is a response for the RestorePartition RPC. +type RestorePartitionResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RestorePartitionResponse) Reset() { + *x = RestorePartitionResponse{} + mi := &file_partition_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RestorePartitionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RestorePartitionResponse) ProtoMessage() {} + +func (x *RestorePartitionResponse) ProtoReflect() protoreflect.Message { + mi := &file_partition_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RestorePartitionResponse.ProtoReflect.Descriptor instead. +func (*RestorePartitionResponse) Descriptor() ([]byte, []int) { + return file_partition_proto_rawDescGZIP(), []int{3} +} + // ListPartitionsRequest is a request for the ListPartitions RPC. type ListPartitionsRequest struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -128,7 +231,7 @@ type ListPartitionsRequest struct { func (x *ListPartitionsRequest) Reset() { *x = ListPartitionsRequest{} - mi := &file_partition_proto_msgTypes[2] + mi := &file_partition_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -140,7 +243,7 @@ func (x *ListPartitionsRequest) String() string { func (*ListPartitionsRequest) ProtoMessage() {} func (x *ListPartitionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_partition_proto_msgTypes[2] + mi := &file_partition_proto_msgTypes[4] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -153,7 +256,7 @@ func (x *ListPartitionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListPartitionsRequest.ProtoReflect.Descriptor instead. func (*ListPartitionsRequest) Descriptor() ([]byte, []int) { - return file_partition_proto_rawDescGZIP(), []int{2} + return file_partition_proto_rawDescGZIP(), []int{4} } func (x *ListPartitionsRequest) GetStorageName() string { @@ -184,7 +287,7 @@ type ListPartitionsResponse struct { func (x *ListPartitionsResponse) Reset() { *x = ListPartitionsResponse{} - mi := &file_partition_proto_msgTypes[3] + mi := &file_partition_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -196,7 +299,7 @@ func (x *ListPartitionsResponse) String() string { func (*ListPartitionsResponse) ProtoMessage() {} func (x *ListPartitionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_partition_proto_msgTypes[3] + mi := &file_partition_proto_msgTypes[5] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -209,7 +312,7 @@ func (x *ListPartitionsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListPartitionsResponse.ProtoReflect.Descriptor instead. func (*ListPartitionsResponse) Descriptor() ([]byte, []int) { - return file_partition_proto_rawDescGZIP(), []int{3} + return file_partition_proto_rawDescGZIP(), []int{5} } func (x *ListPartitionsResponse) GetPartitions() []*Partition { @@ -239,7 +342,7 @@ type KVPair struct { func (x *KVPair) Reset() { *x = KVPair{} - mi := &file_partition_proto_msgTypes[4] + mi := &file_partition_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -251,7 +354,7 @@ func (x *KVPair) String() string { func (*KVPair) ProtoMessage() {} func (x *KVPair) ProtoReflect() protoreflect.Message { - mi := &file_partition_proto_msgTypes[4] + mi := &file_partition_proto_msgTypes[6] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -264,7 +367,7 @@ func (x *KVPair) ProtoReflect() protoreflect.Message { // Deprecated: Use KVPair.ProtoReflect.Descriptor instead. func (*KVPair) Descriptor() ([]byte, []int) { - return file_partition_proto_rawDescGZIP(), []int{4} + return file_partition_proto_rawDescGZIP(), []int{6} } func (x *KVPair) GetKey() []byte { @@ -295,45 +398,63 @@ var file_partition_proto_rawDesc = string([]byte{ 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0xa8, 0xc6, 0x2c, 0x01, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x19, 0x0a, 0x17, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x15, 0x4c, - 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0x88, 0xc6, 0x2c, 0x01, - 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x48, 0x0a, - 0x11, 0x70, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, - 0x6d, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, - 0x79, 0x2e, 0x50, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, - 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, 0x10, 0x70, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x31, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x45, 0x0a, 0x11, 0x70, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x75, 0x72, 0x73, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x61, 0x67, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x75, 0x72, 0x73, 0x6f, 0x72, 0x52, 0x10, 0x70, 0x61, 0x67, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x75, 0x72, 0x73, 0x6f, 0x72, 0x22, 0x30, 0x0a, 0x06, - 0x4b, 0x56, 0x50, 0x61, 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x32, 0xcb, - 0x01, 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, - 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, - 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa5, 0x01, 0x0a, 0x17, 0x52, + 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0x88, 0xc6, + 0x2c, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x27, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0xa8, 0xc6, 0x2c, 0x01, 0x52, 0x0b, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, + 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, + 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, + 0x72, 0x79, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8a, + 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, + 0x88, 0xc6, 0x2c, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x48, 0x0a, 0x11, 0x70, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x67, + 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, 0x10, 0x70, 0x61, 0x67, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x16, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x67, 0x69, 0x74, + 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x45, 0x0a, 0x11, 0x70, 0x61, 0x67, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x75, 0x72, 0x73, 0x6f, 0x72, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x61, + 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x75, 0x72, 0x73, 0x6f, 0x72, 0x52, 0x10, + 0x70, 0x61, 0x67, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x75, 0x72, 0x73, 0x6f, 0x72, + 0x22, 0x30, 0x0a, 0x06, 0x4b, 0x56, 0x50, 0x61, 0x69, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x32, 0xac, 0x02, 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5c, 0x0a, 0x0f, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x2e, 0x67, 0x69, 0x74, + 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x67, 0x69, 0x74, + 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xfa, 0x97, 0x28, + 0x04, 0x08, 0x02, 0x10, 0x03, 0x12, 0x5f, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, + 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, + 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xfa, 0x97, + 0x28, 0x04, 0x08, 0x01, 0x10, 0x03, 0x12, 0x59, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, + 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xfa, 0x97, 0x28, 0x04, 0x08, 0x02, 0x10, - 0x03, 0x12, 0x59, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x08, 0xfa, 0x97, 0x28, 0x04, 0x08, 0x02, 0x10, 0x02, 0x42, 0x34, 0x5a, 0x32, - 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x69, 0x74, 0x6c, 0x61, - 0x62, 0x2d, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2f, 0x76, 0x31, 0x36, - 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, - 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x02, 0x42, 0x34, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, + 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2d, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c, + 0x79, 0x2f, 0x76, 0x31, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x2f, 0x67, + 0x69, 0x74, 0x61, 0x6c, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -348,30 +469,36 @@ func file_partition_proto_rawDescGZIP() []byte { return file_partition_proto_rawDescData } -var file_partition_proto_msgTypes = make([]protoimpl.MessageInfo, 5) +var file_partition_proto_msgTypes = make([]protoimpl.MessageInfo, 7) var file_partition_proto_goTypes = []any{ - (*BackupPartitionRequest)(nil), // 0: gitaly.BackupPartitionRequest - (*BackupPartitionResponse)(nil), // 1: gitaly.BackupPartitionResponse - (*ListPartitionsRequest)(nil), // 2: gitaly.ListPartitionsRequest - (*ListPartitionsResponse)(nil), // 3: gitaly.ListPartitionsResponse - (*KVPair)(nil), // 4: gitaly.KVPair - (*PaginationParameter)(nil), // 5: gitaly.PaginationParameter - (*Partition)(nil), // 6: gitaly.Partition - (*PaginationCursor)(nil), // 7: gitaly.PaginationCursor + (*BackupPartitionRequest)(nil), // 0: gitaly.BackupPartitionRequest + (*BackupPartitionResponse)(nil), // 1: gitaly.BackupPartitionResponse + (*RestorePartitionRequest)(nil), // 2: gitaly.RestorePartitionRequest + (*RestorePartitionResponse)(nil), // 3: gitaly.RestorePartitionResponse + (*ListPartitionsRequest)(nil), // 4: gitaly.ListPartitionsRequest + (*ListPartitionsResponse)(nil), // 5: gitaly.ListPartitionsResponse + (*KVPair)(nil), // 6: gitaly.KVPair + (*Repository)(nil), // 7: gitaly.Repository + (*PaginationParameter)(nil), // 8: gitaly.PaginationParameter + (*Partition)(nil), // 9: gitaly.Partition + (*PaginationCursor)(nil), // 10: gitaly.PaginationCursor } var file_partition_proto_depIdxs = []int32{ - 5, // 0: gitaly.ListPartitionsRequest.pagination_params:type_name -> gitaly.PaginationParameter - 6, // 1: gitaly.ListPartitionsResponse.partitions:type_name -> gitaly.Partition - 7, // 2: gitaly.ListPartitionsResponse.pagination_cursor:type_name -> gitaly.PaginationCursor - 0, // 3: gitaly.PartitionService.BackupPartition:input_type -> gitaly.BackupPartitionRequest - 2, // 4: gitaly.PartitionService.ListPartitions:input_type -> gitaly.ListPartitionsRequest - 1, // 5: gitaly.PartitionService.BackupPartition:output_type -> gitaly.BackupPartitionResponse - 3, // 6: gitaly.PartitionService.ListPartitions:output_type -> gitaly.ListPartitionsResponse - 5, // [5:7] is the sub-list for method output_type - 3, // [3:5] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 7, // 0: gitaly.RestorePartitionRequest.repository:type_name -> gitaly.Repository + 8, // 1: gitaly.ListPartitionsRequest.pagination_params:type_name -> gitaly.PaginationParameter + 9, // 2: gitaly.ListPartitionsResponse.partitions:type_name -> gitaly.Partition + 10, // 3: gitaly.ListPartitionsResponse.pagination_cursor:type_name -> gitaly.PaginationCursor + 0, // 4: gitaly.PartitionService.BackupPartition:input_type -> gitaly.BackupPartitionRequest + 2, // 5: gitaly.PartitionService.RestorePartition:input_type -> gitaly.RestorePartitionRequest + 4, // 6: gitaly.PartitionService.ListPartitions:input_type -> gitaly.ListPartitionsRequest + 1, // 7: gitaly.PartitionService.BackupPartition:output_type -> gitaly.BackupPartitionResponse + 3, // 8: gitaly.PartitionService.RestorePartition:output_type -> gitaly.RestorePartitionResponse + 5, // 9: gitaly.PartitionService.ListPartitions:output_type -> gitaly.ListPartitionsResponse + 7, // [7:10] is the sub-list for method output_type + 4, // [4:7] is the sub-list for method input_type + 4, // [4:4] is the sub-list for extension type_name + 4, // [4:4] is the sub-list for extension extendee + 0, // [0:4] is the sub-list for field type_name } func init() { file_partition_proto_init() } @@ -387,7 +514,7 @@ func file_partition_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_partition_proto_rawDesc), len(file_partition_proto_rawDesc)), NumEnums: 0, - NumMessages: 5, + NumMessages: 7, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/go/gitalypb/partition_grpc.pb.go b/proto/go/gitalypb/partition_grpc.pb.go index a5d67cc18797a1a1eae059cb844210c04a3860f2..f266f7fdce3e3f078d78f36709a402d5a3c13ba1 100644 --- a/proto/go/gitalypb/partition_grpc.pb.go +++ b/proto/go/gitalypb/partition_grpc.pb.go @@ -19,8 +19,9 @@ import ( const _ = grpc.SupportPackageIsVersion9 const ( - PartitionService_BackupPartition_FullMethodName = "/gitaly.PartitionService/BackupPartition" - PartitionService_ListPartitions_FullMethodName = "/gitaly.PartitionService/ListPartitions" + PartitionService_BackupPartition_FullMethodName = "/gitaly.PartitionService/BackupPartition" + PartitionService_RestorePartition_FullMethodName = "/gitaly.PartitionService/RestorePartition" + PartitionService_ListPartitions_FullMethodName = "/gitaly.PartitionService/ListPartitions" ) // PartitionServiceClient is the client API for PartitionService service. @@ -32,6 +33,8 @@ type PartitionServiceClient interface { // BackupPartition backs up an entire partition and saves it to // the configured backup sink (filesystem or object storage). BackupPartition(ctx context.Context, in *BackupPartitionRequest, opts ...grpc.CallOption) (*BackupPartitionResponse, error) + // RestorePartition restores a partition from the configured backup sink. + RestorePartition(ctx context.Context, in *RestorePartitionRequest, opts ...grpc.CallOption) (*RestorePartitionResponse, error) // ListPartitions lists partitions present in the storage. ListPartitions(ctx context.Context, in *ListPartitionsRequest, opts ...grpc.CallOption) (*ListPartitionsResponse, error) } @@ -54,6 +57,16 @@ func (c *partitionServiceClient) BackupPartition(ctx context.Context, in *Backup return out, nil } +func (c *partitionServiceClient) RestorePartition(ctx context.Context, in *RestorePartitionRequest, opts ...grpc.CallOption) (*RestorePartitionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RestorePartitionResponse) + err := c.cc.Invoke(ctx, PartitionService_RestorePartition_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *partitionServiceClient) ListPartitions(ctx context.Context, in *ListPartitionsRequest, opts ...grpc.CallOption) (*ListPartitionsResponse, error) { cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ListPartitionsResponse) @@ -73,6 +86,8 @@ type PartitionServiceServer interface { // BackupPartition backs up an entire partition and saves it to // the configured backup sink (filesystem or object storage). BackupPartition(context.Context, *BackupPartitionRequest) (*BackupPartitionResponse, error) + // RestorePartition restores a partition from the configured backup sink. + RestorePartition(context.Context, *RestorePartitionRequest) (*RestorePartitionResponse, error) // ListPartitions lists partitions present in the storage. ListPartitions(context.Context, *ListPartitionsRequest) (*ListPartitionsResponse, error) mustEmbedUnimplementedPartitionServiceServer() @@ -88,6 +103,9 @@ type UnimplementedPartitionServiceServer struct{} func (UnimplementedPartitionServiceServer) BackupPartition(context.Context, *BackupPartitionRequest) (*BackupPartitionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method BackupPartition not implemented") } +func (UnimplementedPartitionServiceServer) RestorePartition(context.Context, *RestorePartitionRequest) (*RestorePartitionResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RestorePartition not implemented") +} func (UnimplementedPartitionServiceServer) ListPartitions(context.Context, *ListPartitionsRequest) (*ListPartitionsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ListPartitions not implemented") } @@ -130,6 +148,24 @@ func _PartitionService_BackupPartition_Handler(srv interface{}, ctx context.Cont return interceptor(ctx, in, info, handler) } +func _PartitionService_RestorePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RestorePartitionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PartitionServiceServer).RestorePartition(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: PartitionService_RestorePartition_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PartitionServiceServer).RestorePartition(ctx, req.(*RestorePartitionRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _PartitionService_ListPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(ListPartitionsRequest) if err := dec(in); err != nil { @@ -159,6 +195,10 @@ var PartitionService_ServiceDesc = grpc.ServiceDesc{ MethodName: "BackupPartition", Handler: _PartitionService_BackupPartition_Handler, }, + { + MethodName: "RestorePartition", + Handler: _PartitionService_RestorePartition_Handler, + }, { MethodName: "ListPartitions", Handler: _PartitionService_ListPartitions_Handler, diff --git a/proto/partition.proto b/proto/partition.proto index ae1e40fbcbb25d1acf7566f1d711859ad00ec370..0a5d00f35fea3e451872a483d8102e24c9e96875 100644 --- a/proto/partition.proto +++ b/proto/partition.proto @@ -19,6 +19,14 @@ service PartitionService { }; } + // RestorePartition restores a partition from the configured backup sink. + rpc RestorePartition(RestorePartitionRequest) returns (RestorePartitionResponse) { + option (op_type) = { + op: MUTATOR + scope_level: PARTITION + }; + } + // ListPartitions lists partitions present in the storage. rpc ListPartitions(ListPartitionsRequest) returns (ListPartitionsResponse) { option (op_type) = { @@ -42,6 +50,22 @@ message BackupPartitionRequest { message BackupPartitionResponse { } +// RestorePartitionRequest is a request for the RestorePartition RPC. +message RestorePartitionRequest { + // storage_name is the name of the storage containing the partition to be restored. + string storage_name = 1 [(gitaly.storage)=true]; + // partition_id is the identifier of the specific partition within the storage to + // be restored, which may contain one or more repositories and associated data. + string partition_id = 2 [(gitaly.partition_id)=true]; + // repository is the repository to be restored. The storage_name + // and relative_path attributes must be provided. + Repository repository = 3 [(gitaly.target_repository)=true]; +} + +// RestorePartitionResponse is a response for the RestorePartition RPC. +message RestorePartitionResponse { +} + // ListPartitionsRequest is a request for the ListPartitions RPC. message ListPartitionsRequest { // storage_name is the name of the storage in which partitions will be searched for.