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:
authorKarthik Nayak <knayak@gitlab.com>2023-12-11 21:47:01 +0300
committerKarthik Nayak <knayak@gitlab.com>2023-12-19 11:53:32 +0300
commitecdd4ab0afaef7ec7f06872a32bd43c5d63c6348 (patch)
treeda52b2a8684e560587b3d1fd0e3af27b431fc7e8
parentf10129dde26656252c6c590102034aace4b14cae (diff)
hook: Add the `ProcReceiveHook` server
With the creation of `ProcReceiveHook` under the `HookManager`, we can now create the `ProcReceiveHook` server. The server simply propagates the data to/from the `HookMananger.ProcReceiveHook` to the client.
-rw-r--r--internal/git/hooks_payload.go2
-rw-r--r--internal/gitaly/hook/disabled_manager.go10
-rw-r--r--internal/gitaly/hook/manager.go10
-rw-r--r--internal/gitaly/hook/manager_mock.go16
-rw-r--r--internal/gitaly/hook/updateref/update_with_hooks_test.go3
-rw-r--r--internal/gitaly/service/dependencies.go6
-rw-r--r--internal/gitaly/service/hook/proc_receive.go55
-rw-r--r--internal/gitaly/service/hook/proc_receive_test.go234
-rw-r--r--internal/gitaly/service/hook/testhelper_test.go11
-rw-r--r--internal/gitaly/service/repository/replicate_test.go2
-rw-r--r--internal/gitaly/service/smarthttp/receive_pack_test.go1
-rw-r--r--internal/testhelper/testserver/gitaly.go16
12 files changed, 362 insertions, 4 deletions
diff --git a/internal/git/hooks_payload.go b/internal/git/hooks_payload.go
index c1528d0f6..ae645fe8d 100644
--- a/internal/git/hooks_payload.go
+++ b/internal/git/hooks_payload.go
@@ -40,6 +40,8 @@ const (
PostReceiveHook
// PackObjectsHook represents the pack-objects git hook.
PackObjectsHook
+ // ProcReceiveHook represents the proc-receive git hook.
+ ProcReceiveHook
// AllHooks is the bitwise set of all hooks supported by Gitaly.
AllHooks = ReferenceTransactionHook | UpdateHook | PreReceiveHook | PostReceiveHook | PackObjectsHook
diff --git a/internal/gitaly/hook/disabled_manager.go b/internal/gitaly/hook/disabled_manager.go
index e1dee4757..b8c34be7a 100644
--- a/internal/gitaly/hook/disabled_manager.go
+++ b/internal/gitaly/hook/disabled_manager.go
@@ -29,3 +29,13 @@ func (DisabledManager) UpdateHook(context.Context, *gitalypb.Repository, string,
func (DisabledManager) ReferenceTransactionHook(context.Context, ReferenceTransactionState, []string, io.Reader) error {
return nil
}
+
+// ProcReceiveHook ignores its parameters and returns a nil error.
+func (DisabledManager) ProcReceiveHook(ctx context.Context, repo *gitalypb.Repository, env []string, stdin io.Reader, stdout, stderr io.Writer) error {
+ return nil
+}
+
+// ProcReceiveRegistry returns nil.
+func (DisabledManager) ProcReceiveRegistry() *ProcReceiveRegistry {
+ return nil
+}
diff --git a/internal/gitaly/hook/manager.go b/internal/gitaly/hook/manager.go
index 1cdd18e33..0c16974f3 100644
--- a/internal/gitaly/hook/manager.go
+++ b/internal/gitaly/hook/manager.go
@@ -47,6 +47,10 @@ type Manager interface {
// ReferenceTransactionHook executes the reference-transaction Git hook. stdin must contain
// all references to be updated and match the format specified in githooks(5).
ReferenceTransactionHook(ctx context.Context, state ReferenceTransactionState, env []string, stdin io.Reader) error
+
+ // ProcReceiveRegistry provides the ProcReceiveRegistry assigned to the Manager. The registry
+ // allows RPCs to hook into the proc-receive handler.
+ ProcReceiveRegistry() *ProcReceiveRegistry
}
// Transaction is the interface of storagemgr.Transaction. It's used for mocking in the tests.
@@ -88,6 +92,12 @@ type GitLabHookManager struct {
procReceiveRegistry *ProcReceiveRegistry
}
+// ProcReceiveRegistry provides the ProcReceiveRegistry assigned to the Manager. The registry
+// allows RPCs to hook into the proc-receive handler.
+func (m *GitLabHookManager) ProcReceiveRegistry() *ProcReceiveRegistry {
+ return m.procReceiveRegistry
+}
+
// NewManager returns a new hook manager
func NewManager(
cfg config.Cfg,
diff --git a/internal/gitaly/hook/manager_mock.go b/internal/gitaly/hook/manager_mock.go
index 51a7715e7..fa68d4afd 100644
--- a/internal/gitaly/hook/manager_mock.go
+++ b/internal/gitaly/hook/manager_mock.go
@@ -16,6 +16,8 @@ type MockManager struct {
postReceive func(t *testing.T, ctx context.Context, repo *gitalypb.Repository, pushOptions, env []string, stdin io.Reader, stdout, stderr io.Writer) error
update func(t *testing.T, ctx context.Context, repo *gitalypb.Repository, ref, oldValue, newValue string, env []string, stdout, stderr io.Writer) error
referenceTransaction func(t *testing.T, ctx context.Context, state ReferenceTransactionState, env []string, stdin io.Reader) error
+ procReceive func(t *testing.T, ctx context.Context, repo *gitalypb.Repository, env []string, stdin io.Reader, stdout, stderr io.Writer) error
+ procReceiveRegistry *ProcReceiveRegistry
}
var (
@@ -47,6 +49,7 @@ func NewMockManager(
postReceive func(t *testing.T, ctx context.Context, repo *gitalypb.Repository, pushOptions, env []string, stdin io.Reader, stdout, stderr io.Writer) error,
update func(t *testing.T, ctx context.Context, repo *gitalypb.Repository, ref, oldValue, newValue string, env []string, stdout, stderr io.Writer) error,
referenceTransaction func(t *testing.T, ctx context.Context, state ReferenceTransactionState, env []string, stdin io.Reader) error,
+ procReceiveRegistry *ProcReceiveRegistry,
) Manager {
return &MockManager{
t: t,
@@ -84,3 +87,16 @@ func (m *MockManager) ReferenceTransactionHook(ctx context.Context, state Refere
return m.referenceTransaction(m.t, ctx, state, env, stdin)
}
+
+// ProcReceiveHook executes the mocked proc-receive hook
+func (m *MockManager) ProcReceiveHook(ctx context.Context, repo *gitalypb.Repository, env []string, stdin io.Reader, stdout, stderr io.Writer) error {
+ require.NotNil(m.t, m.procReceive, "procReceive not implemented")
+
+ return m.procReceive(m.t, ctx, repo, env, stdin, stdout, stderr)
+}
+
+// ProcReceiveRegistry provides the ProcReceiveRegistry.
+func (m *MockManager) ProcReceiveRegistry() *ProcReceiveRegistry {
+ require.NotNil(m.t, m.procReceiveRegistry, "ProcReceiveRegistry not available")
+ return m.procReceiveRegistry
+}
diff --git a/internal/gitaly/hook/updateref/update_with_hooks_test.go b/internal/gitaly/hook/updateref/update_with_hooks_test.go
index 36bfbe658..71dab297a 100644
--- a/internal/gitaly/hook/updateref/update_with_hooks_test.go
+++ b/internal/gitaly/hook/updateref/update_with_hooks_test.go
@@ -275,7 +275,7 @@ func TestUpdaterWithHooks_UpdateReference(t *testing.T) {
for _, tc := range testCases {
referenceTransactionCalls = 0
t.Run(tc.desc, func(t *testing.T) {
- hookManager := hook.NewMockManager(t, tc.preReceive, tc.postReceive, tc.update, tc.referenceTransaction)
+ hookManager := hook.NewMockManager(t, tc.preReceive, tc.postReceive, tc.update, tc.referenceTransaction, hook.NewProcReceiveRegistry())
gitCmdFactory := gittest.NewCommandFactory(t, cfg)
updater := updateref.NewUpdaterWithHooks(cfg, testhelper.NewLogger(t), config.NewLocator(cfg), hookManager, gitCmdFactory, nil)
@@ -384,6 +384,7 @@ func TestUpdaterWithHooks_quarantine(t *testing.T) {
}
return nil
},
+ hook.NewProcReceiveRegistry(),
)
require.NoError(t, updateref.NewUpdaterWithHooks(cfg, testhelper.NewLogger(t), locator, hookManager, gitCmdFactory, nil).UpdateReference(
diff --git a/internal/gitaly/service/dependencies.go b/internal/gitaly/service/dependencies.go
index 6e73950e8..ce08ac4cc 100644
--- a/internal/gitaly/service/dependencies.go
+++ b/internal/gitaly/service/dependencies.go
@@ -45,6 +45,7 @@ type Dependencies struct {
PartitionManager *storagemgr.PartitionManager
BackupSink backup.Sink
BackupLocator backup.Locator
+ ProcReceiveRegistry *gitalyhook.ProcReceiveRegistry
}
// GetLogger returns the logger.
@@ -151,3 +152,8 @@ func (dc *Dependencies) GetBackupSink() backup.Sink {
func (dc *Dependencies) GetBackupLocator() backup.Locator {
return dc.BackupLocator
}
+
+// GetProcReceiveRegistry returns the ProcReceiveRegistry.
+func (dc *Dependencies) GetProcReceiveRegistry() *gitalyhook.ProcReceiveRegistry {
+ return dc.ProcReceiveRegistry
+}
diff --git a/internal/gitaly/service/hook/proc_receive.go b/internal/gitaly/service/hook/proc_receive.go
new file mode 100644
index 000000000..50c25a445
--- /dev/null
+++ b/internal/gitaly/service/hook/proc_receive.go
@@ -0,0 +1,55 @@
+package hook
+
+import (
+ "fmt"
+
+ gitalyhook "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/hook"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
+ "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
+ "gitlab.com/gitlab-org/gitaly/v16/streamio"
+)
+
+func sendProcReceiveHookResponse(stream gitalypb.HookService_ProcReceiveHookServer, code int32, stderr string) error {
+ if err := stream.Send(&gitalypb.ProcReceiveHookResponse{
+ ExitStatus: &gitalypb.ExitStatus{Value: code},
+ Stderr: []byte(stderr),
+ }); err != nil {
+ return structerr.NewInternal("sending response: %w", err)
+ }
+
+ return nil
+}
+
+func (s *server) ProcReceiveHook(stream gitalypb.HookService_ProcReceiveHookServer) error {
+ ctx := stream.Context()
+
+ firstRequest, err := stream.Recv()
+ if err != nil {
+ return structerr.NewInternal("receiving first request: %w", err)
+ }
+
+ stdin := streamio.NewReader(func() ([]byte, error) {
+ req, err := stream.Recv()
+ return req.GetStdin(), err
+ })
+
+ stdout := streamio.NewWriter(func(p []byte) error {
+ return stream.Send(&gitalypb.ProcReceiveHookResponse{Stdout: p})
+ })
+
+ handler, doneCh, err := gitalyhook.NewProcReceiveHandler(firstRequest.GetEnvironmentVariables(), stdin, stdout)
+ if err != nil {
+ return structerr.NewInternal("creating handler: %w", err)
+ }
+
+ registry := s.manager.ProcReceiveRegistry()
+ if err := registry.Transmit(ctx, handler); err != nil {
+ return sendProcReceiveHookResponse(stream, 1, fmt.Sprintf("transmitting handler: %s", err))
+ }
+
+ if err := <-doneCh; err != nil {
+ return sendProcReceiveHookResponse(stream, 1, fmt.Sprintf("handler finished: %s", err.Error()))
+ }
+
+ return sendProcReceiveHookResponse(stream, 0, "")
+}
diff --git a/internal/gitaly/service/hook/proc_receive_test.go b/internal/gitaly/service/hook/proc_receive_test.go
new file mode 100644
index 000000000..8401f70a1
--- /dev/null
+++ b/internal/gitaly/service/hook/proc_receive_test.go
@@ -0,0 +1,234 @@
+package hook
+
+import (
+ "bytes"
+ "errors"
+ "fmt"
+ "sync"
+ "testing"
+
+ "github.com/stretchr/testify/assert"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/featureflag"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/pktline"
+ gitalyhook "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/hook"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testcfg"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testserver"
+ "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
+)
+
+func TestProcReceiveHook(t *testing.T) {
+ procReceiveRegistry := gitalyhook.NewProcReceiveRegistry()
+
+ cfg := testcfg.Build(t)
+ cfg.SocketPath = runHooksServer(t, cfg, nil, testserver.WithProcReceiveRegistry(procReceiveRegistry))
+ ctx := testhelper.Context(t)
+ repo, _ := gittest.CreateRepository(t, ctx, cfg)
+
+ clientConn := func(
+ t *testing.T,
+ transactionID storage.TransactionID,
+ steps []func(bytes.Buffer) *gitalypb.ProcReceiveHookResponse,
+ ) {
+ hooksPayload, err := git.NewHooksPayload(
+ cfg,
+ repo,
+ gittest.DefaultObjectHash,
+ nil,
+ nil,
+ git.ProcReceiveHook,
+ featureflag.FromContext(ctx),
+ transactionID,
+ ).Env()
+ assert.NoError(t, err)
+
+ client, conn := newHooksClient(t, cfg.SocketPath)
+ defer conn.Close()
+
+ stream, err := client.ProcReceiveHook(ctx)
+ assert.NoError(t, err)
+
+ assert.NoError(t, stream.Send(&gitalypb.ProcReceiveHookRequest{
+ EnvironmentVariables: []string{hooksPayload},
+ Repository: repo,
+ }))
+
+ ref := git.ReferenceName(fmt.Sprintf("refs/heads/main_%d", transactionID))
+
+ var stdin bytes.Buffer
+ _, err = pktline.WriteString(&stdin, "version=1\000atomic")
+ assert.NoError(t, err)
+ err = pktline.WriteFlush(&stdin)
+ assert.NoError(t, err)
+ _, err = pktline.WriteString(&stdin, fmt.Sprintf("%s %s %s",
+ gittest.DefaultObjectHash.ZeroOID, gittest.DefaultObjectHash.EmptyTreeOID, ref))
+ assert.NoError(t, err)
+ err = pktline.WriteFlush(&stdin)
+ assert.NoError(t, err)
+
+ assert.NoError(t, stream.Send(&gitalypb.ProcReceiveHookRequest{
+ Stdin: stdin.Bytes(),
+ }))
+
+ var buf bytes.Buffer
+ for _, f := range steps {
+ expectedResp := f(buf)
+
+ resp, err := stream.Recv()
+ assert.NoError(t, err)
+
+ testhelper.ProtoEqualAssert(t, expectedResp, resp)
+
+ buf.Reset()
+ }
+ }
+
+ registerWaiter := func(t *testing.T, transactionID storage.TransactionID) (<-chan gitalyhook.ProcReceiveHandler, func()) {
+ recvCh, cleanup, err := procReceiveRegistry.RegisterWaiter(transactionID)
+ assert.NoError(t, err)
+ return recvCh, cleanup
+ }
+
+ for _, tc := range []struct {
+ desc string
+ number int
+ clientSteps func(t *testing.T, transactionID storage.TransactionID) []func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse
+ mockReceivePackRPC func(t *testing.T, recvCh <-chan gitalyhook.ProcReceiveHandler, transactionID storage.TransactionID)
+ }{
+ {
+ desc: "multiple transactions",
+ number: 10,
+ clientSteps: func(t *testing.T, transactionID storage.TransactionID) []func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ ref := git.ReferenceName(fmt.Sprintf("refs/heads/main_%d", transactionID))
+
+ return []func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse{
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ _, err := pktline.WriteString(&buf, "version=1\000atomic")
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ }
+ },
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ err := pktline.WriteFlush(&buf)
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ }
+ },
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ _, err := pktline.WriteString(&buf, "ok "+ref.String())
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ }
+ },
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ err := pktline.WriteFlush(&buf)
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ }
+ },
+ }
+ },
+ mockReceivePackRPC: func(t *testing.T, recvCh <-chan gitalyhook.ProcReceiveHandler, transactionID storage.TransactionID) {
+ transmitter := <-recvCh
+
+ ref := git.ReferenceName(fmt.Sprintf("refs/heads/main_%d", transactionID))
+
+ assert.True(t, transmitter.Atomic())
+ assert.Equal(t, []gitalyhook.ReferenceUpdate{{
+ Ref: ref,
+ OldOID: gittest.DefaultObjectHash.ZeroOID,
+ NewOID: gittest.DefaultObjectHash.EmptyTreeOID,
+ }}, transmitter.ReferenceUpdates())
+
+ err := transmitter.AcceptUpdate(ref)
+ assert.NoError(t, err)
+
+ err = transmitter.Close(nil)
+ assert.NoError(t, err)
+ },
+ },
+ {
+ desc: "client failure",
+ number: 1,
+ clientSteps: func(t *testing.T, transactionID storage.TransactionID) []func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ return []func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse{
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ _, err := pktline.WriteString(&buf, "version=1\000atomic")
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ }
+ },
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ err := pktline.WriteFlush(&buf)
+ assert.NoError(t, err)
+
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: buf.Bytes(),
+ Stderr: nil,
+ }
+ },
+ func(buf bytes.Buffer) *gitalypb.ProcReceiveHookResponse {
+ return &gitalypb.ProcReceiveHookResponse{
+ Stdout: nil,
+ Stderr: []byte("handler finished: don't need reason"),
+ ExitStatus: &gitalypb.ExitStatus{Value: 1},
+ }
+ },
+ }
+ },
+ mockReceivePackRPC: func(t *testing.T, recvCh <-chan gitalyhook.ProcReceiveHandler, transactionID storage.TransactionID) {
+ transmitter := <-recvCh
+
+ ref := git.ReferenceName(fmt.Sprintf("refs/heads/main_%d", transactionID))
+
+ assert.True(t, transmitter.Atomic())
+ assert.Equal(t, []gitalyhook.ReferenceUpdate{{
+ Ref: ref,
+ OldOID: gittest.DefaultObjectHash.ZeroOID,
+ NewOID: gittest.DefaultObjectHash.EmptyTreeOID,
+ }}, transmitter.ReferenceUpdates())
+
+ err := transmitter.Close(errors.New("don't need reason"))
+ assert.NoError(t, err)
+ },
+ },
+ } {
+ t.Run(tc.desc, func(t *testing.T) {
+ var wg sync.WaitGroup
+
+ for i := 1; i <= tc.number; i++ {
+ id := storage.TransactionID(i)
+
+ recvCh, cleanup := registerWaiter(t, id)
+
+ wg.Add(1)
+ go func() {
+ defer wg.Done()
+ tc.mockReceivePackRPC(t, recvCh, id)
+ cleanup()
+ }()
+
+ wg.Add(1)
+ go func() {
+ defer wg.Done()
+ clientConn(t, id, tc.clientSteps(t, id))
+ }()
+ }
+
+ wg.Wait()
+ })
+ }
+}
diff --git a/internal/gitaly/service/hook/testhelper_test.go b/internal/gitaly/service/hook/testhelper_test.go
index 16b9d1f14..b0c60f9fb 100644
--- a/internal/gitaly/service/hook/testhelper_test.go
+++ b/internal/gitaly/service/hook/testhelper_test.go
@@ -59,7 +59,16 @@ func runHooksServerWithTransactionRegistry(tb testing.TB, cfg config.Cfg, opts [
return testserver.RunGitalyServer(tb, cfg, func(srv *grpc.Server, deps *service.Dependencies) {
if txRegistry != nil {
- deps.GitalyHookManager = gitalyhook.NewManager(deps.GetCfg(), deps.GetLocator(), deps.GetLogger(), deps.GetGitCmdFactory(), deps.GetTxManager(), deps.GetGitlabClient(), txRegistry, gitalyhook.NewProcReceiveRegistry())
+ deps.GitalyHookManager = gitalyhook.NewManager(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetLogger(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetGitlabClient(),
+ txRegistry,
+ deps.ProcReceiveRegistry,
+ )
}
hookServer := NewServer(deps)
diff --git a/internal/gitaly/service/repository/replicate_test.go b/internal/gitaly/service/repository/replicate_test.go
index 26b7fc00a..f8f514a81 100644
--- a/internal/gitaly/service/repository/replicate_test.go
+++ b/internal/gitaly/service/repository/replicate_test.go
@@ -813,7 +813,7 @@ func TestFetchInternalRemote_successful(t *testing.T) {
referenceTransactionHookCalled++
return nil
- }),
+ }, nil),
))
ctx, err := storage.InjectGitalyServers(ctx, remoteRepo.GetStorageName(), remoteAddr, remoteCfg.Auth.Token)
diff --git a/internal/gitaly/service/smarthttp/receive_pack_test.go b/internal/gitaly/service/smarthttp/receive_pack_test.go
index f5ef91820..61bb37ae5 100644
--- a/internal/gitaly/service/smarthttp/receive_pack_test.go
+++ b/internal/gitaly/service/smarthttp/receive_pack_test.go
@@ -850,6 +850,7 @@ func TestPostReceivePack_notAllowed(t *testing.T) {
gitalyhook.NopPostReceive,
gitalyhook.NopUpdate,
gitalyhook.NopReferenceTransaction,
+ gitalyhook.NewProcReceiveRegistry(),
)
server := startSmartHTTPServerWithOptions(t, cfg, nil, []testserver.GitalyServerOpt{
diff --git a/internal/testhelper/testserver/gitaly.go b/internal/testhelper/testserver/gitaly.go
index d7210b6b8..56089bb4b 100644
--- a/internal/testhelper/testserver/gitaly.go
+++ b/internal/testhelper/testserver/gitaly.go
@@ -286,6 +286,7 @@ type gitalyServerDeps struct {
backupLocator backup.Locator
signingKey string
transactionRegistry *storagemgr.TransactionRegistry
+ procReceiveRegistry *hook.ProcReceiveRegistry
}
func (gsd *gitalyServerDeps) createDependencies(tb testing.TB, cfg config.Cfg) *service.Dependencies {
@@ -323,6 +324,10 @@ func (gsd *gitalyServerDeps) createDependencies(tb testing.TB, cfg config.Cfg) *
gsd.transactionRegistry = storagemgr.NewTransactionRegistry()
}
+ if gsd.procReceiveRegistry == nil {
+ gsd.procReceiveRegistry = hook.NewProcReceiveRegistry()
+ }
+
if gsd.hookMgr == nil {
gsd.hookMgr = hook.NewManager(
cfg, gsd.locator,
@@ -331,7 +336,7 @@ func (gsd *gitalyServerDeps) createDependencies(tb testing.TB, cfg config.Cfg) *
gsd.txMgr,
gsd.gitlabClient,
hook.NewTransactionRegistry(gsd.transactionRegistry),
- hook.NewProcReceiveRegistry(),
+ gsd.procReceiveRegistry,
)
}
@@ -418,6 +423,7 @@ func (gsd *gitalyServerDeps) createDependencies(tb testing.TB, cfg config.Cfg) *
PartitionManager: partitionManager,
BackupSink: gsd.backupSink,
BackupLocator: gsd.backupLocator,
+ ProcReceiveRegistry: gsd.procReceiveRegistry,
}
}
@@ -555,3 +561,11 @@ func WithTransactionRegistry(registry *storagemgr.TransactionRegistry) GitalySer
return deps
}
}
+
+// WithProcReceiveRegistry sets the proc receive registry that will be used for Gitaly services.
+func WithProcReceiveRegistry(registry *hook.ProcReceiveRegistry) GitalyServerOpt {
+ return func(deps gitalyServerDeps) gitalyServerDeps {
+ deps.procReceiveRegistry = registry
+ return deps
+ }
+}