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>2020-02-29 01:26:06 +0300
committerJohn Cai <jcai@gitlab.com>2020-02-29 01:26:06 +0300
commit6da6bc59fcc61e71aa96c4159549f5ca28bdf9e6 (patch)
treedbec32e9e2e8ea02be759c60e0538c6c0cab7c7c
parentfddf14364f47dc2204a710caf5d3020dc95bcce7 (diff)
some updates to transactions
-rw-r--r--internal/git/repository/repository.go112
-rw-r--r--internal/middleware/repositoryhandler/transactions.go19
-rw-r--r--internal/praefect/coordinator.go23
-rw-r--r--internal/server/server.go8
-rw-r--r--internal/service/register.go4
-rw-r--r--internal/service/repository/server.go4
-rw-r--r--internal/service/repository/write_ref_tx.go34
7 files changed, 127 insertions, 77 deletions
diff --git a/internal/git/repository/repository.go b/internal/git/repository/repository.go
index 601cfd361..c7b94184c 100644
--- a/internal/git/repository/repository.go
+++ b/internal/git/repository/repository.go
@@ -2,11 +2,11 @@ package repository
import (
"errors"
+ "os/exec"
"sync"
"time"
"github.com/sirupsen/logrus"
- "gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/log"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
@@ -19,25 +19,26 @@ type GitRepo interface {
GetGitAlternateObjectDirectories() []string
}
-func NewTransactions() *Transactions {
- return &Transactions{
+func NewTransactionManager() *TransactionManager {
+ return &TransactionManager{
transactions: make(map[string]*Transaction),
repositories: make(map[string]*sync.Mutex),
log: log.Default(),
}
}
-type Transactions struct {
+type TransactionManager struct {
txMutex, repoMutex sync.RWMutex
transactions map[string]*Transaction
repositories map[string]*sync.Mutex
log *logrus.Entry
}
-func (t *Transactions) NewTransaction(transactionID string, repo *gitalypb.Repository) {
+func (t *TransactionManager) NewTransaction(transactionID string, repo *gitalypb.Repository) {
logrus.WithField("transaction_id", transactionID).Info("creating new transaction")
tx := Transaction{
- Repo: repo,
+ repo: repo,
+ commitCh: make(chan error),
}
t.txMutex.Lock()
@@ -45,47 +46,47 @@ func (t *Transactions) NewTransaction(transactionID string, repo *gitalypb.Repos
t.transactions[transactionID] = &tx
t.repoMutex.Lock()
-
- _, ok := t.repositories[repo.RelativePath]
+ _, ok := t.repositories[tx.repo.GetRelativePath()]
if !ok {
- t.repositories[repo.RelativePath] = &sync.Mutex{}
+ t.repositories[tx.repo.GetRelativePath()] = &sync.Mutex{}
}
-
- t.repositories[repo.RelativePath].Lock()
-
+ t.repositories[tx.repo.GetRelativePath()].Lock()
t.repoMutex.Unlock()
}
-func (t *Transactions) Start(transactionID string) {
+func (t *TransactionManager) Begin(transactionID string) {
t.txMutex.Lock()
defer t.txMutex.Unlock()
- _, ok := t.transactions[transactionID]
+ tx, ok := t.transactions[transactionID]
if !ok {
return
}
- t.transactions[transactionID].inProgress = true
+ tx.inProgress = true
go func() {
- <-time.NewTimer(1 * time.Second).C
+ <-time.NewTimer(2 * time.Second).C
t.txMutex.Lock()
+
tx, ok := t.transactions[transactionID]
if !ok {
+ t.txMutex.Unlock()
return
}
if tx.prepared {
t.log.WithField("transaction_id", transactionID).Info("transaction has already been prepared")
+ t.txMutex.Unlock()
return
}
- t.txMutex.Unlock()
+ t.txMutex.Unlock()
t.log.WithField("transaction_id", transactionID).Info("transaction has not been prepared and is timing out")
- t.Unlock(transactionID)
+ t.Release(transactionID)
}()
}
-func (t *Transactions) PreCommit(transactionID string, c command.Cmd) {
+func (t *TransactionManager) PreCommit(transactionID string, c *exec.Cmd) {
t.txMutex.Lock()
defer t.txMutex.Unlock()
@@ -94,10 +95,13 @@ func (t *Transactions) PreCommit(transactionID string, c command.Cmd) {
return
}
- go tx.PrepareCommitAndWait(c, 2*time.Second)
+ go func() {
+ tx.PrepareCommitAndWait(c, 2*time.Second)
+ t.Release(transactionID)
+ }()
}
-func (t *Transactions) SetRollback(transactionID string, rollback command.Cmd) {
+func (t *TransactionManager) SetRollback(transactionID string, rollback *exec.Cmd) {
t.txMutex.Lock()
defer t.txMutex.Unlock()
tx, ok := t.transactions[transactionID]
@@ -105,10 +109,10 @@ func (t *Transactions) SetRollback(transactionID string, rollback command.Cmd) {
return
}
- tx.Rollback = rollback
+ tx.rollback = rollback
}
-func (t *Transactions) Commit(transactionID string) error {
+func (t *TransactionManager) Commit(transactionID string) error {
t.txMutex.Lock()
defer t.txMutex.Unlock()
@@ -117,13 +121,12 @@ func (t *Transactions) Commit(transactionID string) error {
return errors.New("request_id not found")
}
- tx.Commit()
-
t.log.WithField("transaction_id", transactionID).Info("commited")
- return nil
+ return tx.Commit()
}
-func (t *Transactions) Unlock(transactionID string) error {
+func (t *TransactionManager) Release(transactionID string) error {
+ t.log.WithField("transaction_id", transactionID).Info("unlocking")
t.txMutex.Lock()
defer t.txMutex.Unlock()
@@ -136,7 +139,7 @@ func (t *Transactions) Unlock(transactionID string) error {
t.repoMutex.Lock()
defer t.repoMutex.Unlock()
- repoLock, ok := t.repositories[tx.Repo.GetRelativePath()]
+ repoLock, ok := t.repositories[tx.repo.GetRelativePath()]
if !ok {
return nil
}
@@ -149,7 +152,7 @@ func (t *Transactions) Unlock(transactionID string) error {
return nil
}
-func (t *Transactions) Rollback(transactionID string) error {
+func (t *TransactionManager) Rollback(transactionID string) error {
t.txMutex.Lock()
defer t.txMutex.Unlock()
@@ -158,14 +161,14 @@ func (t *Transactions) Rollback(transactionID string) error {
return errors.New("request_id not found")
}
- if err := tx.Rollback.Wait(); err != nil {
+ if err := tx.rollback.Run(); err != nil {
return err
}
return nil
}
-func (t *Transactions) TransactionStarted(transactionID string) bool {
+func (t *TransactionManager) TransactionStarted(transactionID string) bool {
t.txMutex.Lock()
defer t.txMutex.Unlock()
@@ -174,26 +177,51 @@ func (t *Transactions) TransactionStarted(transactionID string) bool {
return ok
}
-func (t *Transaction) PrepareCommitAndWait(c command.Cmd, d time.Duration) {
+func (t *Transaction) PrepareCommitAndWait(c *exec.Cmd, d time.Duration) {
+ logrus.WithError(t.commitErr).Info("precommit started")
t.prepared = true
+ t.commit = c
- select {
- case <-time.NewTimer(d).C:
- case <-t.commitCh:
+ <-time.NewTimer(d).C
+ if !t.inProgress {
+ return
}
+ logrus.Info("timer went off...i'm doin it!")
+ t.Commit()
- t.commitErr = c.Wait()
- t.inProgress = false
+ logrus.WithError(t.commitErr).Info("precommit completed")
}
-func (t *Transaction) Commit() {
- close(t.commitCh)
+func (t *Transaction) Commit() error {
+ if t.commitErr != nil {
+ return t.commitErr
+ }
+ if !t.inProgress {
+ return nil
+ }
+
+ defer func() {
+ t.inProgress = false
+ }()
+
+ if t.commitErr = t.commit.Start(); t.commitErr != nil {
+ return t.commitErr
+ }
+
+ if t.commitErr = t.commit.Wait(); t.commitErr != nil {
+ return t.commitErr
+ }
+
+ logrus.WithError(t.commitErr).WithField("args", t.commit.Args).Info("Finished Commit")
+
+ return nil
}
type Transaction struct {
- Repo GitRepo
- Rollback command.Cmd
+ repo GitRepo
+ commit *exec.Cmd
+ rollback *exec.Cmd
commitErr error
- commitCh chan struct{}
+ commitCh chan error
inProgress, prepared bool
}
diff --git a/internal/middleware/repositoryhandler/transactions.go b/internal/middleware/repositoryhandler/transactions.go
index 8cfeb7ae6..fe8e1ba91 100644
--- a/internal/middleware/repositoryhandler/transactions.go
+++ b/internal/middleware/repositoryhandler/transactions.go
@@ -6,6 +6,8 @@ import (
"fmt"
"reflect"
+ "github.com/sirupsen/logrus"
+
"github.com/google/uuid"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/praefect/protoregistry"
@@ -18,7 +20,7 @@ type RepositoryRequest interface {
GetRepository() *gitalypb.Repository
}
-func RepositoryTransactionUnaryInterceptor(transactions *repository.Transactions, registry *protoregistry.Registry) grpc.UnaryServerInterceptor {
+func RepositoryTransactionUnaryInterceptor(transactions *repository.TransactionManager, registry *protoregistry.Registry) grpc.UnaryServerInterceptor {
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
mi, err := registry.LookupMethod(info.FullMethod)
if err != nil {
@@ -41,11 +43,14 @@ func RepositoryTransactionUnaryInterceptor(transactions *repository.Transactions
return nil, errors.New("not a repository request")
}
if repoReq.GetRepository() != nil {
+
+ logrus.WithField("repository", repoReq.GetRepository()).Info("trying to start new transaction")
transactions.NewTransaction(transactionID, repoReq.GetRepository())
+ logrus.WithField("repository", repoReq.GetRepository()).Info("started new transaction")
}
}
- defer transactions.Unlock(transactionID)
+ defer transactions.Release(transactionID)
md.Set("transaction_id", transactionID)
ctx = metadata.NewIncomingContext(ctx, md)
@@ -57,7 +62,7 @@ func RepositoryTransactionUnaryInterceptor(transactions *repository.Transactions
}
}
-func NewTransactionServerStream(ss grpc.ServerStream, methodInfo protoregistry.MethodInfo, transactions *repository.Transactions, transactionID string) TransactionServerStream {
+func NewTransactionServerStream(ss grpc.ServerStream, methodInfo protoregistry.MethodInfo, transactions *repository.TransactionManager, transactionID string) TransactionServerStream {
return TransactionServerStream{
transactionID: transactionID,
ss: ss,
@@ -70,7 +75,7 @@ type TransactionServerStream struct {
transactionID string
ss grpc.ServerStream
mi protoregistry.MethodInfo
- transactions *repository.Transactions
+ transactions *repository.TransactionManager
}
func (t TransactionServerStream) SetHeader(m metadata.MD) error {
@@ -98,7 +103,9 @@ func (t TransactionServerStream) RecvMsg(m interface{}) error {
fmt.Printf("Receiving Message: type=%s\n", reflect.TypeOf(m).String())
repoReq, ok := m.(RepositoryRequest)
if ok && repoReq.GetRepository() != nil {
+ logrus.WithField("repository", repoReq.GetRepository()).Info("trying to start new transaction")
t.transactions.NewTransaction(t.transactionID, repoReq.GetRepository())
+ logrus.WithField("repository", repoReq.GetRepository()).Info("started new transaction")
}
}
@@ -108,7 +115,7 @@ func (t TransactionServerStream) RecvMsg(m interface{}) error {
}
// StreamInterceptor returns a Stream Interceptor
-func RepositoryTransactionServerInterceptor(transactions *repository.Transactions, registry *protoregistry.Registry) grpc.StreamServerInterceptor {
+func RepositoryTransactionStreamInterceptor(transactions *repository.TransactionManager, registry *protoregistry.Registry) grpc.StreamServerInterceptor {
return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
mi, err := registry.LookupMethod(info.FullMethod)
if err != nil {
@@ -125,7 +132,7 @@ func RepositoryTransactionServerInterceptor(transactions *repository.Transaction
transactionID = uuid.New().String()
}
- defer transactions.Unlock(transactionID)
+ defer transactions.Release(transactionID)
return handler(srv, NewTransactionServerStream(stream, mi, transactions, transactionID))
}
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index 99b7d34fa..09c4458c1 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -148,6 +148,7 @@ func (c *Coordinator) streamDirector(ctx context.Context, fullMethodName string,
}
func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerStream) error {
+ c.log.Info("I'm in Handle write rEFFFFFFFFFFFFFFFFFFFFFF!!!!!!!!!!!!!!")
var writeRefReq gitalypb.WriteRefRequest
if err := serverStream.RecvMsg(&writeRefReq); err != nil {
@@ -175,7 +176,8 @@ func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
var errs []error
transactionIDs := make(map[string]nodes.Node)
- // Prepare
+ c.log.Info("about to vote")
+ // vote
for _, node := range append(secondaries, primary) {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
targetRepo := &gitalypb.Repository{
@@ -184,9 +186,7 @@ func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
}
var trailer metadata.MD
- metadata.AppendToOutgoingContext(ctx, "transaction_step", "prepare")
-
- if _, err := client.WriteRefTx(ctx, &gitalypb.WriteRefTxRequest{
+ if _, err := client.WriteRefTx(metadata.AppendToOutgoingContext(ctx, "transaction_step", "vote"), &gitalypb.WriteRefTxRequest{
Repository: targetRepo,
Ref: writeRefReq.Ref,
Revision: writeRefReq.Revision,
@@ -208,16 +208,17 @@ func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
return errors.New("votes failed")
}
+ c.log.Info("about to precommit")
+
// PreCommit
- for _, node := range transactionIDs {
+ for transactionID, node := range transactionIDs {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
targetRepo := &gitalypb.Repository{
StorageName: node.GetStorage(),
RelativePath: writeRefReq.GetRepository().GetRelativePath(),
}
- metadata.AppendToOutgoingContext(ctx, "transaction_step", "precommit")
- if _, err := client.WriteRefTx(ctx, &gitalypb.WriteRefTxRequest{
+ if _, err := client.WriteRefTx(metadata.AppendToOutgoingContext(ctx, "transaction_step", "precommit", "transaction_id", transactionID), &gitalypb.WriteRefTxRequest{
Repository: targetRepo,
Ref: writeRefReq.Ref,
Revision: writeRefReq.Revision,
@@ -228,11 +229,12 @@ func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
}
}
+ c.log.Info("about to commit")
// Commit
if len(errs) == 0 {
- for _, node := range transactionIDs {
+ for transactionID, node := range transactionIDs {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
- if _, err = client.WriteRefTx(metadata.AppendToOutgoingContext(ctx, "transaction_step", "commit"), &gitalypb.WriteRefTxRequest{}); err != nil {
+ if _, err = client.WriteRefTx(metadata.AppendToOutgoingContext(ctx, "transaction_step", "commit", "transaction_id", transactionID), &gitalypb.WriteRefTxRequest{}); err != nil {
return err
}
}
@@ -244,7 +246,8 @@ func (c *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
return nil
}
- // Rollback
+ c.log.Info("about to rollback")
+ // rollback
for _, node := range transactionIDs {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
if _, err = client.WriteRefTx(metadata.AppendToOutgoingContext(ctx, "transaction_step", "rollback"), &gitalypb.WriteRefTxRequest{}); err != nil {
diff --git a/internal/server/server.go b/internal/server/server.go
index e7fd93f83..2e421d418 100644
--- a/internal/server/server.go
+++ b/internal/server/server.go
@@ -79,7 +79,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
lh := limithandler.New(concurrencyKeyFn)
- transactions := repository.NewTransactions()
+ transactionMgr := repository.NewTransactionManager()
registry := protoregistry.New()
registry.RegisterFiles(protoregistry.GitalyProtoFileDescriptors...)
@@ -100,7 +100,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.StreamPanicHandler,
- repositoryhandler.RepositoryTransactionServerInterceptor(transactions, registry),
+ repositoryhandler.RepositoryTransactionStreamInterceptor(transactionMgr, registry),
)),
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
grpc_ctxtags.UnaryServerInterceptor(ctxTagOpts...),
@@ -117,7 +117,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.UnaryPanicHandler,
- repositoryhandler.RepositoryTransactionUnaryInterceptor(transactions, registry),
+ repositoryhandler.RepositoryTransactionUnaryInterceptor(transactionMgr, registry),
)),
}
@@ -133,7 +133,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
server := grpc.NewServer(opts...)
- service.RegisterAll(server, rubyServer, transactions)
+ service.RegisterAll(server, rubyServer, transactionMgr)
reflection.Register(server)
grpc_prometheus.Register(server)
diff --git a/internal/service/register.go b/internal/service/register.go
index 2db53735f..597acf610 100644
--- a/internal/service/register.go
+++ b/internal/service/register.go
@@ -27,7 +27,7 @@ import (
// RegisterAll will register all the known grpc services with
// the specified grpc service instance
-func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server, transactions *repo.Transactions) {
+func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server, transactionMgr *repo.TransactionManager) {
gitalypb.RegisterBlobServiceServer(grpcServer, blob.NewServer(rubyServer))
gitalypb.RegisterCleanupServiceServer(grpcServer, cleanup.NewServer())
gitalypb.RegisterCommitServiceServer(grpcServer, commit.NewServer())
@@ -35,7 +35,7 @@ func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server, transac
gitalypb.RegisterNamespaceServiceServer(grpcServer, namespace.NewServer())
gitalypb.RegisterOperationServiceServer(grpcServer, operations.NewServer(rubyServer))
gitalypb.RegisterRefServiceServer(grpcServer, ref.NewServer())
- gitalypb.RegisterRepositoryServiceServer(grpcServer, repository.NewServer(transactions, rubyServer))
+ gitalypb.RegisterRepositoryServiceServer(grpcServer, repository.NewServer(transactionMgr, rubyServer))
gitalypb.RegisterSSHServiceServer(grpcServer, ssh.NewServer())
gitalypb.RegisterSmartHTTPServiceServer(grpcServer, smarthttp.NewServer())
gitalypb.RegisterWikiServiceServer(grpcServer, wiki.NewServer(rubyServer))
diff --git a/internal/service/repository/server.go b/internal/service/repository/server.go
index 9dc817d67..b833d59cf 100644
--- a/internal/service/repository/server.go
+++ b/internal/service/repository/server.go
@@ -16,11 +16,11 @@ type server struct {
gitalypb.UnimplementedRepositoryServiceServer
connsByAddress map[string]*grpc.ClientConn
connsMtx sync.RWMutex
- transactions *repository.Transactions
+ transactions *repository.TransactionManager
}
// NewServer creates a new instance of a gRPC repo server
-func NewServer(transactions *repository.Transactions, rs *rubyserver.Server) gitalypb.RepositoryServiceServer {
+func NewServer(transactions *repository.TransactionManager, rs *rubyserver.Server) gitalypb.RepositoryServiceServer {
return &server{ruby: rs, connsByAddress: make(map[string]*grpc.ClientConn), transactions: transactions}
}
diff --git a/internal/service/repository/write_ref_tx.go b/internal/service/repository/write_ref_tx.go
index 016adac26..457914e97 100644
--- a/internal/service/repository/write_ref_tx.go
+++ b/internal/service/repository/write_ref_tx.go
@@ -6,7 +6,8 @@ import (
"io/ioutil"
"os/exec"
- "gitlab.com/gitlab-org/gitaly/internal/command"
+ "github.com/sirupsen/logrus"
+
"gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
@@ -28,13 +29,18 @@ func (s *server) WriteRefTx(ctx context.Context, req *gitalypb.WriteRefTxRequest
}
transactionSteps := md.Get("transaction_step")
+
+ logrus.WithField("transaction_steps", transactionSteps).Info("got transaction steps")
if len(transactionSteps) == 0 {
return nil, nil
}
+ logrus.WithField("transaction_steps", transactionSteps).Info("got transaction steps")
+
transactionStep := transactionSteps[len(transactionSteps)-1]
if transactionStep == "precommit" {
+ logrus.WithField("relative_path", req.GetRepository().GetRelativePath()).WithField("ref", string(req.GetRef())).WithField("rev", string(req.GetRevision())).WithField("old_rev", string(req.GetOldRevision())).Info("WriteRefReqTx")
if err := validateWriteRefRequest(req); err != nil {
return nil, helper.ErrInvalidArgument(err)
}
@@ -57,14 +63,16 @@ func (s *server) transactionalWriteRef(ctx context.Context, req *gitalypb.WriteR
transactionID := md.Get("transaction_id")[0]
switch transactionStep {
- case "prepare":
- err := prepare(ctx, req)
+ case "vote":
+ logrus.WithField("transaction_step", "vote").Info("transaction!")
+
+ s.transactions.Begin(transactionID)
+ err := vote(ctx, req)
if err != nil {
return nil, err
}
-
- s.transactions.Start(transactionID)
case "precommit":
+ logrus.WithField("transaction_step", "precommit").Info("transaction!")
rollback, err := rollbackRef(req)
if err != nil {
return nil, err
@@ -77,10 +85,12 @@ func (s *server) transactionalWriteRef(ctx context.Context, req *gitalypb.WriteR
s.transactions.SetRollback(transactionID, rollback)
s.transactions.PreCommit(transactionID, commit)
case "commit":
+ logrus.WithField("transaction_step", "commit").Info("transaction!")
if err := s.transactions.Commit(transactionID); err != nil {
return nil, err
}
case "rollback":
+ logrus.WithField("transaction_step", "rollback").Info("transaction!")
if err := s.transactions.Rollback(transactionID); err != nil {
return nil, err
}
@@ -91,7 +101,7 @@ func (s *server) transactionalWriteRef(ctx context.Context, req *gitalypb.WriteR
return &resp, nil
}
-func prepare(ctx context.Context, req *gitalypb.WriteRefTxRequest) error {
+func vote(ctx context.Context, req *gitalypb.WriteRefTxRequest) error {
if req.GetOldRevision() == nil {
return nil
}
@@ -120,7 +130,7 @@ func prepare(ctx context.Context, req *gitalypb.WriteRefTxRequest) error {
return nil
}
-func rollbackSymbolicRef(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
+func rollbackSymbolicRef(req *gitalypb.WriteRefTxRequest) (*exec.Cmd, error) {
repoPath, err := helper.GetRepoPath(req.GetRepository())
if err != nil {
return nil, err
@@ -129,7 +139,7 @@ func rollbackSymbolicRef(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
return exec.Command("git", "-C", repoPath, "symoblic-ref", string(req.GetRef()), string(req.GetOldRevision())), nil
}
-func rollbackRef(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
+func rollbackRef(req *gitalypb.WriteRefTxRequest) (*exec.Cmd, error) {
if string(req.Ref) == "HEAD" {
return rollbackSymbolicRef(req)
}
@@ -152,7 +162,7 @@ func rollbackRef(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
return c, nil
}
-func preCommit(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
+func preCommit(req *gitalypb.WriteRefTxRequest) (*exec.Cmd, error) {
if string(req.Ref) == "HEAD" {
return preCommitSymbolicRef(req)
}
@@ -162,7 +172,7 @@ func preCommit(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
return nil, err
}
- args := []string{"-C", repoPath, "update-ref", string(req.GetRevision())}
+ args := []string{"-C", repoPath, "update-ref", string(req.GetRef()), string(req.GetRevision())}
if req.GetOldRevision() != nil {
args = append(args, string(req.GetOldRevision()))
@@ -170,10 +180,12 @@ func preCommit(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
c := exec.Command("git", args...)
+ logrus.WithField("DA COMMAND ARGS!!!", args).Info("1234567")
+
return c, nil
}
-func preCommitSymbolicRef(req *gitalypb.WriteRefTxRequest) (command.Cmd, error) {
+func preCommitSymbolicRef(req *gitalypb.WriteRefTxRequest) (*exec.Cmd, error) {
repoPath, err := helper.GetRepoPath(req.GetRepository())
if err != nil {
return nil, err