Welcome to mirror list, hosted at ThFree Co, Russian Federation.

gitlab.com/gitlab-org/gitaly.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJohn Cai <jcai@gitlab.com>2023-10-18 22:52:04 +0300
committerJohn Cai <jcai@gitlab.com>2023-10-18 22:58:45 +0300
commiteec91e1adecfdc063adadde5c2e7065025a4028e (patch)
treefa8f3921d3d6c41fdd6ee6f1a1d7f03de3ee6362
parentd78bfacca7ca4130c30de43ce09de9e04c193b0f (diff)
praefect: Remove RenameRepository callsjc/remove-rename-repository
-rw-r--r--internal/praefect/coordinator.go18
-rw-r--r--internal/praefect/rename_repository.go78
-rw-r--r--internal/praefect/replicator.go39
-rw-r--r--internal/praefect/server.go1
-rw-r--r--internal/praefect/server_test.go113
5 files changed, 1 insertions, 248 deletions
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index ac1d554f3..0719e68cb 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -95,7 +95,6 @@ var transactionRPCs = map[string]transactionsCondition{
// transactional in the future if the need arises.
"/gitaly.ObjectPoolService/CreateObjectPool": transactionsDisabled,
"/gitaly.ObjectPoolService/DeleteObjectPool": transactionsDisabled,
- "/gitaly.RepositoryService/RenameRepository": transactionsDisabled,
}
// forcePrimaryRoutingRPCs tracks RPCs which need to always get routed to the primary. This should
@@ -162,12 +161,6 @@ func getReplicationDetails(methodName string, m proto.Message) (datastore.Change
"/gitaly.RepositoryService/ReplicateRepository",
"/gitaly.RepositoryService/RestoreRepository":
return datastore.CreateRepo, nil, nil
- case "/gitaly.RepositoryService/RenameRepository":
- req, ok := m.(*gitalypb.RenameRepositoryRequest)
- if !ok {
- return "", nil, fmt.Errorf("protocol changed: for method %q expected message type '%T', got '%T'", methodName, req, m)
- }
- return datastore.RenameRepo, datastore.Params{"RelativePath": req.RelativePath}, nil
default:
return datastore.UpdateRepo, nil, nil
}
@@ -1093,17 +1086,6 @@ func (c *Coordinator) newRequestFinalizer(
if err := c.rs.IncrementGeneration(ctx, repositoryID, primary, updatedSecondaries); err != nil {
return fmt.Errorf("increment generation: %w", err)
}
- case datastore.RenameRepo:
- // Renaming a repository is not idempotent on Gitaly's side. This combined with a failure here results in a problematic state,
- // where the client receives an error but can't retry the call as the repository has already been moved on the primary.
- // Ideally the rename RPC should copy the repository instead of moving so the client can retry if this failed.
- if err := c.rs.RenameRepository(ctx, virtualStorage, targetRepo.GetRelativePath(), primary, params["RelativePath"].(string)); err != nil {
- if !errors.Is(err, datastore.ErrRepositoryNotFound) {
- return fmt.Errorf("rename repository: %w", err)
- }
-
- c.logger.WithError(err).InfoContext(ctx, "renamed repository does not have a store entry")
- }
case datastore.CreateRepo:
repositorySpecificPrimariesEnabled := c.conf.Failover.ElectionStrategy == config.ElectionStrategyPerRepository
variableReplicationFactorEnabled := repositorySpecificPrimariesEnabled &&
diff --git a/internal/praefect/rename_repository.go b/internal/praefect/rename_repository.go
deleted file mode 100644
index 089ab557c..000000000
--- a/internal/praefect/rename_repository.go
+++ /dev/null
@@ -1,78 +0,0 @@
-package praefect
-
-import (
- "errors"
- "fmt"
-
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage"
- "gitlab.com/gitlab-org/gitaly/v16/internal/praefect/datastore"
- "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
- "google.golang.org/grpc"
-)
-
-func validateRenameRepositoryRequest(req *gitalypb.RenameRepositoryRequest, virtualStorages map[string]struct{}) error {
- // These checks are not strictly necessary but they exist to keep retain compatibility with
- // Gitaly's tested behavior.
- repository := req.GetRepository()
- if repository == nil || repository.GetStorageName() == "" && repository.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrRepositoryNotSet)
- } else if repository.GetStorageName() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrStorageNotSet)
- } else if repository.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrRepositoryPathNotSet)
- } else if _, ok := virtualStorages[repository.GetStorageName()]; !ok {
- return storage.NewStorageNotFoundError(repository.GetStorageName())
- } else if req.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("destination relative path is empty")
- } else if _, err := storage.ValidateRelativePath("/fake-root", req.GetRelativePath()); err != nil {
- // Gitaly uses ValidateRelativePath to verify there are no traversals, so we use the same function
- // here. Praefect is not susceptible to path traversals as it generates its own disk paths but we
- // do this to retain API compatibility with Gitaly. ValidateRelativePath checks for traversals by
- // seeing whether the relative path escapes the root directory. It's not possible to traverse up
- // from the /, so the traversals in the path wouldn't be caught. To allow for the check to work,
- // we use the /fake-root directory simply to notice if there were traversals in the path.
- return structerr.NewInvalidArgument("%w", err).WithMetadata("relative_path", req.GetRelativePath())
- }
-
- return nil
-}
-
-// RenameRepositoryHandler handles /gitaly.RepositoryService/RenameRepository calls by renaming
-// the repository in the lookup table stored in the database.
-func RenameRepositoryHandler(virtualStoragesNames []string, rs datastore.RepositoryStore) grpc.StreamHandler {
- virtualStorages := make(map[string]struct{}, len(virtualStoragesNames))
- for _, virtualStorage := range virtualStoragesNames {
- virtualStorages[virtualStorage] = struct{}{}
- }
-
- return func(srv interface{}, stream grpc.ServerStream) error {
- var req gitalypb.RenameRepositoryRequest
- if err := stream.RecvMsg(&req); err != nil {
- return fmt.Errorf("receive request: %w", err)
- }
-
- if err := validateRenameRepositoryRequest(&req, virtualStorages); err != nil {
- return err
- }
-
- if err := rs.RenameRepositoryInPlace(stream.Context(),
- req.GetRepository().GetStorageName(),
- req.GetRepository().GetRelativePath(),
- req.GetRelativePath(),
- ); err != nil {
- if errors.Is(err, datastore.ErrRepositoryNotFound) {
- return storage.NewRepositoryNotFoundError(
- req.GetRepository().GetStorageName(),
- req.GetRepository().GetRelativePath(),
- )
- } else if errors.Is(err, datastore.ErrRepositoryAlreadyExists) {
- return structerr.NewAlreadyExists("target repo exists already")
- }
-
- return structerr.NewInternal("%w", err)
- }
-
- return stream.SendMsg(&gitalypb.RenameRepositoryResponse{})
- }
-}
diff --git a/internal/praefect/replicator.go b/internal/praefect/replicator.go
index 42b3eebee..c6fd4a64f 100644
--- a/internal/praefect/replicator.go
+++ b/internal/praefect/replicator.go
@@ -156,44 +156,7 @@ func (dr defaultReplicator) Destroy(ctx context.Context, event datastore.Replica
}
func (dr defaultReplicator) Rename(ctx context.Context, event datastore.ReplicationEvent, targetCC *grpc.ClientConn) error {
- targetRepo := &gitalypb.Repository{
- StorageName: event.Job.TargetNodeStorage,
- RelativePath: event.Job.RelativePath,
- }
-
- repoSvcClient := gitalypb.NewRepositoryServiceClient(targetCC)
-
- val, found := event.Job.Params["RelativePath"]
- if !found {
- return errors.New("no 'RelativePath' parameter for rename")
- }
-
- relativePath, ok := val.(string)
- if !ok {
- return fmt.Errorf("parameter 'RelativePath' has unexpected type: %T", relativePath)
- }
-
- if _, err := repoSvcClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: targetRepo,
- RelativePath: relativePath,
- }); err != nil {
- return err
- }
-
- // If the repository was moved but this fails, we'll have a stale record on the storage but it is missing from the
- // virtual storage. We can later schedule a deletion to fix the situation. The newly named repository's record
- // will be present once a replication job arrives for it.
- if err := dr.rs.RenameRepository(ctx,
- event.Job.VirtualStorage, event.Job.RelativePath, event.Job.TargetNodeStorage, relativePath); err != nil {
- if !errors.Is(err, datastore.ErrRepositoryNotFound) {
- return err
- }
-
- dr.log.WithField(correlation.FieldName, correlation.ExtractFromContext(ctx)).
- WithError(err).
- Info("replicated repository rename does not have a store entry")
- }
-
+ // RenameRepository has been deprecated
return nil
}
diff --git a/internal/praefect/server.go b/internal/praefect/server.go
index c54c2b502..4792e4655 100644
--- a/internal/praefect/server.go
+++ b/internal/praefect/server.go
@@ -187,7 +187,6 @@ func NewGRPCServer(
proxy.RegisterStreamHandlers(srv, "gitaly.RepositoryService", map[string]grpc.StreamHandler{
"RemoveAll": RemoveAllHandler(deps.RepositoryStore, deps.Conns),
"RemoveRepository": RemoveRepositoryHandler(deps.RepositoryStore, deps.Logger, deps.Conns),
- "RenameRepository": RenameRepositoryHandler(deps.Config.VirtualStorageNames(), deps.RepositoryStore),
"ReplicateRepository": ReplicateRepositoryHandler(deps.Coordinator),
"RepositoryExists": RepositoryExistsHandler(deps.RepositoryStore),
})
diff --git a/internal/praefect/server_test.go b/internal/praefect/server_test.go
index 16b51b084..c0bc070eb 100644
--- a/internal/praefect/server_test.go
+++ b/internal/praefect/server_test.go
@@ -5,7 +5,6 @@ import (
"context"
"errors"
"io"
- "math/rand"
"net"
"sort"
"strings"
@@ -578,118 +577,6 @@ func TestRemoveRepository(t *testing.T) {
verifyReposExistence(t, codes.NotFound)
}
-func TestRenameRepository(t *testing.T) {
- gitalyStorages := []string{"gitaly-1", "gitaly-2", "gitaly-3"}
- praefectCfg := config.Config{
- VirtualStorages: []*config.VirtualStorage{{Name: "praefect"}},
- Failover: config.Failover{Enabled: true, ElectionStrategy: config.ElectionStrategyPerRepository},
- }
-
- for _, storageName := range gitalyStorages {
- cfgBuilder := testcfg.NewGitalyCfgBuilder(testcfg.WithStorages(storageName))
- gitalyCfg := cfgBuilder.Build(t)
- gitalyAddr := testserver.RunGitalyServer(t, gitalyCfg, setup.RegisterAll, testserver.WithDisablePraefect())
-
- praefectCfg.VirtualStorages[0].Nodes = append(praefectCfg.VirtualStorages[0].Nodes, &config.Node{
- Storage: storageName,
- Address: gitalyAddr,
- Token: gitalyCfg.Auth.Token,
- })
- }
-
- evq := datastore.NewReplicationEventQueueInterceptor(datastore.NewPostgresReplicationEventQueue(testdb.New(t)))
-
- db := testdb.New(t)
-
- rs := datastore.NewPostgresRepositoryStore(db, nil)
-
- logger := testhelper.SharedLogger(t)
- txManager := transactions.NewManager(praefectCfg, logger)
- clientHandshaker := backchannel.NewClientHandshaker(
- logger,
- NewBackchannelServerFactory(
- logger,
- transaction.NewServer(txManager),
- nil,
- ),
- backchannel.DefaultConfiguration(),
- )
-
- ctx := testhelper.Context(t)
- nodeSet, err := DialNodes(ctx, praefectCfg.VirtualStorages, nil, nil, clientHandshaker, nil, logger)
- require.NoError(t, err)
- defer nodeSet.Close()
-
- cc, _, cleanup := RunPraefectServer(t, ctx, praefectCfg, BuildOptions{
- WithQueue: evq,
- WithRepoStore: rs,
- WithRouter: NewPerRepositoryRouter(
- nodeSet.Connections(),
- nodes.NewPerRepositoryElector(logger, db),
- StaticHealthChecker(praefectCfg.StorageNames()),
- NewLockedRandom(rand.New(rand.NewSource(0))),
- rs,
- datastore.NewAssignmentStore(db, praefectCfg.StorageNames()),
- rs,
- nil,
- ),
- WithTxMgr: txManager,
- })
- t.Cleanup(cleanup)
-
- virtualRepo1, _ := gittest.CreateRepository(t, ctx, gconfig.Cfg{
- Storages: []gconfig.Storage{{Name: "praefect"}},
- }, gittest.CreateRepositoryConfig{ClientConn: cc})
-
- virtualRepo2, _ := gittest.CreateRepository(t, ctx, gconfig.Cfg{
- Storages: []gconfig.Storage{{Name: "praefect"}},
- }, gittest.CreateRepositoryConfig{ClientConn: cc})
-
- const newRelativePath = "unused-relative-path"
-
- repoServiceClient := gitalypb.NewRepositoryServiceClient(cc)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: &gitalypb.Repository{
- StorageName: virtualRepo1.StorageName,
- RelativePath: "not-found",
- },
- RelativePath: virtualRepo2.RelativePath,
- })
- testhelper.RequireGrpcError(t, testhelper.ToInterceptedMetadata(
- structerr.NewNotFound("%w", storage.NewRepositoryNotFoundError("praefect", "not-found")),
- ), err)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: virtualRepo1,
- RelativePath: virtualRepo2.RelativePath,
- })
-
- expectedErr := structerr.NewAlreadyExists("target repo exists already")
- testhelper.RequireGrpcError(t, expectedErr, err)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: virtualRepo1,
- RelativePath: newRelativePath,
- })
- require.NoError(t, err)
-
- resp, err := repoServiceClient.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: virtualRepo1,
- })
- require.NoError(t, err)
- require.False(t, resp.GetExists(), "repo with old name must gone")
-
- resp, err = repoServiceClient.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: &gitalypb.Repository{
- StorageName: virtualRepo1.StorageName,
- RelativePath: newRelativePath,
- },
- })
- require.NoError(t, err)
- require.True(t, resp.GetExists(), "repo with new name must exist")
-}
-
type mockSmartHTTP struct {
gitalypb.UnimplementedSmartHTTPServiceServer
txMgr *transactions.Manager