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:
authorPaul Okstad <pokstad@gitlab.com>2021-02-19 10:02:36 +0300
committerPaul Okstad <pokstad@gitlab.com>2021-02-19 10:02:36 +0300
commitb3e781b735969594a5aeff6195faba446d81f7b2 (patch)
treebfc0c194a9a98003178005f10448333471870ebb
parent31a14c8d720e3fbe217d6956dc22efce97ecf5cd (diff)
parent119c8a6b43a14d3893df33ee6a1902658d29b821 (diff)
Merge branch 'ps-extract-rubyserver' into 'master'
Gitaly-ruby server should be a provided dependency See merge request gitlab-org/gitaly!3134
-rw-r--r--cmd/gitaly-ssh/auth_test.go4
-rw-r--r--cmd/gitaly/main.go27
-rw-r--r--cmd/praefect/main.go6
-rw-r--r--internal/bootstrap/starter/starter.go22
-rw-r--r--internal/bootstrap/starter/starter_test.go2
-rw-r--r--internal/gitaly/server/auth_test.go12
-rw-r--r--internal/gitaly/server/server.go44
-rw-r--r--internal/gitaly/server/server_factory.go48
-rw-r--r--internal/gitaly/server/server_factory_test.go17
-rw-r--r--internal/gitaly/service/register.go4
-rw-r--r--internal/gitaly/service/repository/fetch_test.go4
-rw-r--r--internal/praefect/server_factory.go5
-rw-r--r--internal/praefect/server_factory_test.go13
-rw-r--r--internal/testhelper/testserver/gitaly.go5
14 files changed, 94 insertions, 119 deletions
diff --git a/cmd/gitaly-ssh/auth_test.go b/cmd/gitaly-ssh/auth_test.go
index cf68a6039..b0c8b69d1 100644
--- a/cmd/gitaly-ssh/auth_test.go
+++ b/cmd/gitaly-ssh/auth_test.go
@@ -17,6 +17,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/server"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -132,8 +133,9 @@ func runServer(t *testing.T, secure bool, cfg config.Cfg, connectionType string,
txManager := transaction.NewManager(cfg)
hookManager := hook.NewManager(locator, txManager, hook.GitlabAPIStub, cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- srv, err := server.New(secure, nil, hookManager, txManager, cfg, conns, locator, gitCmdFactory)
+ srv, err := server.New(secure, cfg, testhelper.DiscardTestEntry(t))
require.NoError(t, err)
+ service.RegisterAll(srv, cfg, nil, hookManager, txManager, locator, conns, gitCmdFactory)
listener, err := net.Listen(connectionType, addr)
require.NoError(t, err)
diff --git a/cmd/gitaly/main.go b/cmd/gitaly/main.go
index 29396dfe7..8cce4178b 100644
--- a/cmd/gitaly/main.go
+++ b/cmd/gitaly/main.go
@@ -17,7 +17,9 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config/sentry"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/server"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
glog "gitlab.com/gitlab-org/gitaly/internal/log"
"gitlab.com/gitlab-org/gitaly/internal/storage"
@@ -161,10 +163,16 @@ func run(cfg config.Cfg) error {
gitCmdFactory := git.NewExecCommandFactory(cfg)
- servers := server.NewGitalyServerFactory(cfg, hookManager, transactionManager, conns, locator, gitCmdFactory)
- defer servers.Stop()
+ gitalyServerFactory := server.NewGitalyServerFactory(cfg)
+ defer gitalyServerFactory.Stop()
- b.StopAction = servers.GracefulStop
+ b.StopAction = gitalyServerFactory.GracefulStop
+
+ rubySrv := rubyserver.New(cfg)
+ if err := rubySrv.Start(); err != nil {
+ return fmt.Errorf("initialize gitaly-ruby: %v", err)
+ }
+ defer rubySrv.Stop()
for _, c := range []starter.Config{
{starter.Unix, cfg.SocketPath},
@@ -176,7 +184,12 @@ func run(cfg config.Cfg) error {
continue
}
- b.RegisterStarter(starter.New(c, servers))
+ srv, err := gitalyServerFactory.Create(c.IsSecure())
+ if err != nil {
+ return fmt.Errorf("create gRPC server: %w", err)
+ }
+ service.RegisterAll(srv, cfg, rubySrv, hookManager, transactionManager, locator, conns, gitCmdFactory)
+ b.RegisterStarter(starter.New(c, srv))
}
if addr := cfg.PrometheusListenAddr; addr != "" {
@@ -224,11 +237,7 @@ func run(cfg config.Cfg) error {
return fmt.Errorf("unable to start the bootstrap: %v", err)
}
- if err := servers.StartRuby(); err != nil {
- return fmt.Errorf("initialize gitaly-ruby: %v", err)
- }
-
- shutdownWorkers, err := servers.StartWorkers(ctx, glog.Default(), cfg)
+ shutdownWorkers, err := gitalyServerFactory.StartWorkers(ctx, glog.Default(), cfg)
if err != nil {
return fmt.Errorf("initialize auxiliary workers: %v", err)
}
diff --git a/cmd/praefect/main.go b/cmd/praefect/main.go
index 6ce7b2ed1..85719caa6 100644
--- a/cmd/praefect/main.go
+++ b/cmd/praefect/main.go
@@ -407,7 +407,11 @@ func run(cfgs []starter.Config, conf config.Config) error {
b.StopAction = srvFactory.GracefulStop
for _, cfg := range cfgs {
- b.RegisterStarter(starter.New(cfg, srvFactory))
+ srv, err := srvFactory.Create(cfg.IsSecure())
+ if err != nil {
+ return fmt.Errorf("create gRPC server: %w", err)
+ }
+ b.RegisterStarter(starter.New(cfg, srv))
}
if conf.PrometheusListenAddr != "" {
diff --git a/internal/bootstrap/starter/starter.go b/internal/bootstrap/starter/starter.go
index a8b7c018c..7a0e0cd4d 100644
--- a/internal/bootstrap/starter/starter.go
+++ b/internal/bootstrap/starter/starter.go
@@ -84,20 +84,27 @@ func (c *Config) Endpoint() (string, error) {
return ComposeEndpoint(c.Name, c.Addr)
}
-func (c *Config) isSecure() bool {
+// IsSecure returns true if network is secured.
+func (c *Config) IsSecure() bool {
return c.Name == TLS
}
func (c *Config) family() string {
- if c.isSecure() {
+ if c.IsSecure() {
return TCP
}
return c.Name
}
+// Server able to serve requests.
+type Server interface {
+ // Serve accepts requests from the listener and handles them properly.
+ Serve(lis net.Listener) error
+}
+
// New creates a new bootstrap.Starter from a config and a GracefulStoppableServer
-func New(cfg Config, servers GracefulStoppableServer) bootstrap.Starter {
+func New(cfg Config, server Server) bootstrap.Starter {
return func(listen bootstrap.ListenFunc, errCh chan<- error) error {
l, err := listen(cfg.family(), cfg.Addr)
if err != nil {
@@ -108,16 +115,9 @@ func New(cfg Config, servers GracefulStoppableServer) bootstrap.Starter {
l = connectioncounter.New(cfg.Name, l)
go func() {
- errCh <- servers.Serve(l, cfg.isSecure())
+ errCh <- server.Serve(l)
}()
return nil
}
}
-
-// GracefulStoppableServer allows to serve contents on a net.Listener, Stop serving and performing a GracefulStop
-type GracefulStoppableServer interface {
- GracefulStop()
- Stop()
- Serve(l net.Listener, secure bool) error
-}
diff --git a/internal/bootstrap/starter/starter_test.go b/internal/bootstrap/starter/starter_test.go
index c5d84b1f8..9e9198eac 100644
--- a/internal/bootstrap/starter/starter_test.go
+++ b/internal/bootstrap/starter/starter_test.go
@@ -19,7 +19,7 @@ func TestIsSecure(t *testing.T) {
} {
t.Run(test.name, func(t *testing.T) {
conf := Config{Name: test.name}
- require.Equal(t, test.secure, conf.isSecure())
+ require.Equal(t, test.secure, conf.IsSecure())
})
}
}
diff --git a/internal/gitaly/server/auth_test.go b/internal/gitaly/server/auth_test.go
index 72baecd56..76b9dd74c 100644
--- a/internal/gitaly/server/auth_test.go
+++ b/internal/gitaly/server/auth_test.go
@@ -21,6 +21,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config/auth"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -28,6 +29,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
+ "google.golang.org/grpc/health"
healthpb "google.golang.org/grpc/health/grpc_health_v1"
)
@@ -207,14 +209,18 @@ func newOperationClient(t *testing.T, serverSocketPath string) (gitalypb.Operati
}
func runServer(t *testing.T, cfg config.Cfg) (string, func()) {
+ t.Helper()
+
conns := client.NewPool()
locator := config.NewLocator(cfg)
txManager := transaction.NewManager(cfg)
hookManager := hook.NewManager(locator, txManager, hook.GitlabAPIStub, cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- srv, err := New(false, rubyServer, hookManager, txManager, cfg, conns, locator, gitCmdFactory)
+
+ srv, err := New(false, cfg, testhelper.DiscardTestEntry(t))
require.NoError(t, err)
+ service.RegisterAll(srv, cfg, rubyServer, hookManager, txManager, locator, conns, gitCmdFactory)
serverSocketPath := testhelper.GetTemporaryGitalySocketFileName(t)
listener, err := net.Listen("unix", serverSocketPath)
@@ -237,9 +243,11 @@ func runSecureServer(t *testing.T) (string, func()) {
}
conns := client.NewPool()
- srv, err := New(true, nil, nil, nil, config.Config, conns, nil, nil)
+ srv, err := New(true, config.Config, testhelper.DiscardTestEntry(t))
require.NoError(t, err)
+ healthpb.RegisterHealthServer(srv, health.NewServer())
+
listener, hostPort := testhelper.GetLocalhostListener(t)
go srv.Serve(listener)
diff --git a/internal/gitaly/server/server.go b/internal/gitaly/server/server.go
index b8abe0416..12d8d045f 100644
--- a/internal/gitaly/server/server.go
+++ b/internal/gitaly/server/server.go
@@ -11,15 +11,9 @@ import (
grpc_ctxtags "github.com/grpc-ecosystem/go-grpc-middleware/tags"
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
log "github.com/sirupsen/logrus"
- "gitlab.com/gitlab-org/gitaly/client"
diskcache "gitlab.com/gitlab-org/gitaly/internal/cache"
- "gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/server/auth"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/internal/helper/fieldextractors"
gitalylog "gitlab.com/gitlab-org/gitaly/internal/log"
"gitlab.com/gitlab-org/gitaly/internal/logsanitizer"
@@ -31,13 +25,11 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/middleware/panichandler"
"gitlab.com/gitlab-org/gitaly/internal/middleware/sentryhandler"
"gitlab.com/gitlab-org/gitaly/internal/praefect/protoregistry"
- "gitlab.com/gitlab-org/gitaly/internal/storage"
grpccorrelation "gitlab.com/gitlab-org/labkit/correlation/grpc"
grpctracing "gitlab.com/gitlab-org/labkit/tracing/grpc"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/keepalive"
- "google.golang.org/grpc/reflection"
)
func concurrencyKeyFn(ctx context.Context) string {
@@ -55,8 +47,6 @@ func concurrencyKeyFn(ctx context.Context) string {
return ""
}
-var logrusEntry *log.Entry
-
func init() {
for _, l := range gitalylog.Loggers {
urlSanitizer := logsanitizer.NewURLSanitizerHook()
@@ -68,31 +58,13 @@ func init() {
l.Hooks.Add(urlSanitizer)
}
- // logrusEntry is used by middlewares below
- logrusEntry = gitalylog.Default()
-
// grpc-go gets a custom logger; it is too chatty
grpc_logrus.ReplaceGrpcLogger(gitalylog.GrpcGo())
}
-// registerServices registers all services on the provided gRPC server instance.
-func registerServices(
- server *grpc.Server,
- rubyServer *rubyserver.Server,
- hookManager hook.Manager,
- txManager transaction.Manager,
- cfg config.Cfg,
- conns *client.Pool,
- locator storage.Locator,
- gitCmdFactory git.CommandFactory,
-) {
- service.RegisterAll(server, cfg, rubyServer, hookManager, txManager, locator, conns, gitCmdFactory)
- reflection.Register(server)
- grpc_prometheus.Register(server)
-}
-
-// createNewServer creates a new gRPC server with all required middleware configured.
-func createNewServer(cfg config.Cfg, secure bool) (*grpc.Server, error) {
+// New returns a GRPC server instance with a set of interceptors configured.
+// If logrusEntry is nil the default logger will be used.
+func New(secure bool, cfg config.Cfg, logrusEntry *log.Entry) (*grpc.Server, error) {
ctxTagOpts := []grpc_ctxtags.Option{
grpc_ctxtags.WithFieldExtractorForInitialReq(fieldextractors.FieldExtractor),
}
@@ -161,13 +133,3 @@ func createNewServer(cfg config.Cfg, secure bool) (*grpc.Server, error) {
return grpc.NewServer(opts...), nil
}
-
-// New returns a GRPC server with all Gitaly services and interceptors set.
-func New(secure bool, rubyServer *rubyserver.Server, hookManager hook.Manager, txManager transaction.Manager, cfg config.Cfg, conns *client.Pool, locator storage.Locator, gitCmdFactory git.CommandFactory) (*grpc.Server, error) {
- server, err := createNewServer(cfg, secure)
- if err != nil {
- return nil, err
- }
- registerServices(server, rubyServer, hookManager, txManager, cfg, conns, locator, gitCmdFactory)
- return server, nil
-}
diff --git a/internal/gitaly/server/server_factory.go b/internal/gitaly/server/server_factory.go
index d610478a8..ea902c3fa 100644
--- a/internal/gitaly/server/server_factory.go
+++ b/internal/gitaly/server/server_factory.go
@@ -3,20 +3,15 @@ package server
import (
"context"
"fmt"
- "net"
"sync"
"time"
"github.com/sirupsen/logrus"
gitalyauth "gitlab.com/gitlab-org/gitaly/auth"
"gitlab.com/gitlab-org/gitaly/client"
- "gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/maintenance"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
- "gitlab.com/gitlab-org/gitaly/internal/storage"
+ gitalylog "gitlab.com/gitlab-org/gitaly/internal/log"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc"
)
@@ -25,32 +20,13 @@ import (
type GitalyServerFactory struct {
mtx sync.Mutex
cfg config.Cfg
- ruby *rubyserver.Server
- hookManager hook.Manager
- txManager transaction.Manager
secure, insecure []*grpc.Server
- conns *client.Pool
- locator storage.Locator
- gitCmdFactory git.CommandFactory
}
// NewGitalyServerFactory allows to create and start secure/insecure 'grpc.Server'-s with gitaly-ruby
// server shared in between.
-func NewGitalyServerFactory(cfg config.Cfg, hookManager hook.Manager, txManager transaction.Manager, conns *client.Pool, locator storage.Locator, gitCmdFactory git.CommandFactory) *GitalyServerFactory {
- return &GitalyServerFactory{
- cfg: cfg,
- ruby: rubyserver.New(cfg),
- hookManager: hookManager,
- txManager: txManager,
- conns: conns,
- locator: locator,
- gitCmdFactory: gitCmdFactory,
- }
-}
-
-// StartRuby starts the ruby process
-func (s *GitalyServerFactory) StartRuby() error {
- return s.ruby.Start()
+func NewGitalyServerFactory(cfg config.Cfg) *GitalyServerFactory {
+ return &GitalyServerFactory{cfg: cfg}
}
// StartWorkers will start any auxiliary background workers that are allowed
@@ -109,8 +85,6 @@ func (s *GitalyServerFactory) Stop() {
for _, srv := range s.all() {
srv.Stop()
}
-
- s.ruby.Stop()
}
// GracefulStop stops both the secure and insecure servers gracefully
@@ -129,24 +103,16 @@ func (s *GitalyServerFactory) GracefulStop() {
wg.Wait()
}
-// Serve starts serving on the provided listener with newly created grpc.Server
-func (s *GitalyServerFactory) Serve(l net.Listener, secure bool) error {
- srv, err := s.create(secure)
- if err != nil {
- return err
- }
-
- return srv.Serve(l)
-}
-
-func (s *GitalyServerFactory) create(secure bool) (*grpc.Server, error) {
+// Create returns newly instantiated and initialized with interceptors instance of the gRPC server.
+func (s *GitalyServerFactory) Create(secure bool) (*grpc.Server, error) {
s.mtx.Lock()
defer s.mtx.Unlock()
- server, err := New(secure, s.ruby, s.hookManager, s.txManager, s.cfg, s.conns, s.locator, s.gitCmdFactory)
+ server, err := New(secure, s.cfg, gitalylog.Default())
if err != nil {
return nil, err
}
+
if secure {
s.secure = append(s.secure, server)
return s.secure[len(s.secure)-1], nil
diff --git a/internal/gitaly/server/server_factory_test.go b/internal/gitaly/server/server_factory_test.go
index 14e0a8633..8d19d23ee 100644
--- a/internal/gitaly/server/server_factory_test.go
+++ b/internal/gitaly/server/server_factory_test.go
@@ -16,6 +16,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
+ "google.golang.org/grpc/health"
healthpb "google.golang.org/grpc/health/grpc_health_v1"
"google.golang.org/grpc/status"
)
@@ -35,7 +36,10 @@ func TestGitalyServerFactory(t *testing.T) {
require.NoError(t, err)
cleanups = append(cleanups, func() { listener.Close() })
- go sf.Serve(listener, true)
+ srv, err := sf.Create(true)
+ require.NoError(t, err)
+ healthpb.RegisterHealthServer(srv, health.NewServer())
+ go srv.Serve(listener)
certPool, err := x509.SystemCertPool()
require.NoError(t, err)
@@ -57,7 +61,10 @@ func TestGitalyServerFactory(t *testing.T) {
require.NoError(t, err)
cleanups = append(cleanups, func() { listener.Close() })
- go sf.Serve(listener, false)
+ srv, err := sf.Create(false)
+ require.NoError(t, err)
+ healthpb.RegisterHealthServer(srv, health.NewServer())
+ go srv.Serve(listener)
endpoint, err := starter.ComposeEndpoint(schema, listener.Addr().String())
require.NoError(t, err)
@@ -81,7 +88,7 @@ func TestGitalyServerFactory(t *testing.T) {
}
t.Run("insecure", func(t *testing.T) {
- sf := NewGitalyServerFactory(config.Config, nil, nil, nil, nil, nil)
+ sf := NewGitalyServerFactory(config.Config)
_, cleanup := checkHealth(t, sf, starter.TCP, "localhost:0")
defer cleanup()
@@ -97,7 +104,7 @@ func TestGitalyServerFactory(t *testing.T) {
KeyPath: keyFile,
}
- sf := NewGitalyServerFactory(config.Config, nil, nil, nil, nil, nil)
+ sf := NewGitalyServerFactory(config.Config)
defer sf.Stop()
_, cleanup := checkHealth(t, sf, starter.TLS, "localhost:0")
@@ -105,7 +112,7 @@ func TestGitalyServerFactory(t *testing.T) {
})
t.Run("all services must be stopped", func(t *testing.T) {
- sf := NewGitalyServerFactory(config.Config, nil, nil, nil, nil, nil)
+ sf := NewGitalyServerFactory(config.Config)
defer sf.Stop()
tcpHealthClient, tcpCleanup := checkHealth(t, sf, starter.TCP, "localhost:0")
diff --git a/internal/gitaly/service/register.go b/internal/gitaly/service/register.go
index 92f2ad1a0..bfb673e22 100644
--- a/internal/gitaly/service/register.go
+++ b/internal/gitaly/service/register.go
@@ -1,6 +1,7 @@
package service
import (
+ grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"gitlab.com/gitlab-org/gitaly/client"
@@ -31,6 +32,7 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/health"
healthpb "google.golang.org/grpc/health/grpc_health_v1"
+ "google.golang.org/grpc/reflection"
)
var (
@@ -96,4 +98,6 @@ func RegisterAll(
gitalypb.RegisterInternalGitalyServer(grpcServer, internalgitaly.NewServer(cfg.Storages))
healthpb.RegisterHealthServer(grpcServer, health.NewServer())
+ reflection.Register(grpcServer)
+ grpc_prometheus.Register(grpcServer)
}
diff --git a/internal/gitaly/service/repository/fetch_test.go b/internal/gitaly/service/repository/fetch_test.go
index 731cab886..9b8f2da3e 100644
--- a/internal/gitaly/service/repository/fetch_test.go
+++ b/internal/gitaly/service/repository/fetch_test.go
@@ -11,6 +11,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
serverPkg "gitlab.com/gitlab-org/gitaly/internal/gitaly/server"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
hookservice "gitlab.com/gitlab-org/gitaly/internal/gitaly/service/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/service/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
@@ -345,10 +346,11 @@ func runFullSecureServer(t *testing.T, locator storage.Locator) (*grpc.Server, s
hookManager := hook.NewManager(locator, txManager, hook.GitlabAPIStub, cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- server, err := serverPkg.New(true, repository.RubyServer, hookManager, txManager, cfg, conns, config.NewLocator(cfg), gitCmdFactory)
+ server, err := serverPkg.New(true, cfg, testhelper.DiscardTestEntry(t))
require.NoError(t, err)
listener, addr := testhelper.GetLocalhostListener(t)
+ service.RegisterAll(server, cfg, repository.RubyServer, hookManager, txManager, config.NewLocator(cfg), conns, gitCmdFactory)
errQ := make(chan error)
// This creates a secondary GRPC server which isn't "secure". Reusing
diff --git a/internal/praefect/server_factory.go b/internal/praefect/server_factory.go
index 6567799f3..b8b42cd4f 100644
--- a/internal/praefect/server_factory.go
+++ b/internal/praefect/server_factory.go
@@ -60,7 +60,7 @@ type ServerFactory struct {
// Serve starts serving on the provided listener with newly created grpc.Server
func (s *ServerFactory) Serve(l net.Listener, secure bool) error {
- srv, err := s.create(secure)
+ srv, err := s.Create(secure)
if err != nil {
return err
}
@@ -91,7 +91,8 @@ func (s *ServerFactory) GracefulStop() {
wg.Wait()
}
-func (s *ServerFactory) create(secure bool) (*grpc.Server, error) {
+// Create returns newly instantiated and initialized with interceptors instance of the gRPC server.
+func (s *ServerFactory) Create(secure bool) (*grpc.Server, error) {
s.mtx.Lock()
defer s.mtx.Unlock()
diff --git a/internal/praefect/server_factory_test.go b/internal/praefect/server_factory_test.go
index f12d2a57f..6426bb4b3 100644
--- a/internal/praefect/server_factory_test.go
+++ b/internal/praefect/server_factory_test.go
@@ -16,6 +16,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/git"
gconfig "gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/server"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service/commit"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/praefect/config"
"gitlab.com/gitlab-org/gitaly/internal/praefect/datastore"
@@ -27,6 +28,7 @@ import (
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
+ "google.golang.org/grpc/health"
healthpb "google.golang.org/grpc/health/grpc_health_v1"
)
@@ -34,14 +36,19 @@ func TestServerFactory(t *testing.T) {
cfg := gconfig.Config
locator := gconfig.NewLocator(cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- gitalyServerFactory := server.NewGitalyServerFactory(cfg, nil, nil, nil, locator, gitCmdFactory)
+ gitalyServerFactory := server.NewGitalyServerFactory(cfg)
defer gitalyServerFactory.Stop()
+ gitalySrv, err := gitalyServerFactory.Create(false)
+ require.NoError(t, err)
+ healthpb.RegisterHealthServer(gitalySrv, health.NewServer())
+ gitalypb.RegisterCommitServiceServer(gitalySrv, commit.NewServer(cfg, locator, gitCmdFactory))
+
// start gitaly serving on public endpoint
gitalyListener, err := net.Listen(starter.TCP, "localhost:0")
require.NoError(t, err)
defer func() { require.NoError(t, gitalyListener.Close()) }()
- go gitalyServerFactory.Serve(gitalyListener, false)
+ go gitalySrv.Serve(gitalyListener)
// start gitaly serving on internal endpoint
gitalyInternalSocketPath := cfg.GitalyInternalSocketPath()
@@ -49,7 +56,7 @@ func TestServerFactory(t *testing.T) {
gitalyInternalListener, err := net.Listen(starter.Unix, gitalyInternalSocketPath)
require.NoError(t, err)
defer func() { require.NoError(t, gitalyInternalListener.Close()) }()
- go gitalyServerFactory.Serve(gitalyInternalListener, false)
+ go gitalySrv.Serve(gitalyInternalListener)
gitalyAddr, err := starter.ComposeEndpoint(gitalyListener.Addr().Network(), gitalyListener.Addr().String())
require.NoError(t, err)
diff --git a/internal/testhelper/testserver/gitaly.go b/internal/testhelper/testserver/gitaly.go
index e47377ed6..17ab6240c 100644
--- a/internal/testhelper/testserver/gitaly.go
+++ b/internal/testhelper/testserver/gitaly.go
@@ -11,6 +11,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/server"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/service"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
)
@@ -26,9 +27,11 @@ func RunGitalyServer(t *testing.T, cfg config.Cfg, rubyServer *rubyserver.Server
hookMgr := hook.NewManager(locator, txManager, hook.GitlabAPIStub, cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- srv, err := server.New(cfg.TLS.CertPath != "", rubyServer, hookMgr, txManager, cfg, conns, locator, gitCmdFactory)
+ srv, err := server.New(cfg.TLS.CertPath != "", cfg, testhelper.DiscardTestEntry(t))
require.NoError(t, err)
+ service.RegisterAll(srv, cfg, rubyServer, hookMgr, txManager, locator, conns, gitCmdFactory)
+
serverSocketPath := testhelper.GetTemporaryGitalySocketFileName(t)
listener, err := net.Listen("unix", serverSocketPath)
require.NoError(t, err)