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:
-rw-r--r--cmd/gitaly/main.go29
-rw-r--r--internal/cgroups/cgroups.go31
-rw-r--r--internal/cgroups/cgroups_linux_test.go146
-rw-r--r--internal/cgroups/v1.go2
-rw-r--r--internal/cgroups/v1_linux.go11
-rw-r--r--internal/cgroups/v1_linux_test.go29
-rw-r--r--internal/git/command_factory.go2
-rw-r--r--internal/gitaly/config/config.go69
-rw-r--r--internal/gitaly/config/config_test.go87
-rw-r--r--internal/gitaly/config/temp_dir.go87
-rw-r--r--internal/gitaly/config/temp_dir_test.go113
11 files changed, 420 insertions, 186 deletions
diff --git a/cmd/gitaly/main.go b/cmd/gitaly/main.go
index 765d729ca..7306b09e8 100644
--- a/cmd/gitaly/main.go
+++ b/cmd/gitaly/main.go
@@ -122,10 +122,6 @@ func configure(configPath string) (config.Cfg, error) {
glog.Configure(glog.Loggers, cfg.Logging.Format, cfg.Logging.Level)
- if err := cgroups.NewManager(cfg.Cgroups).Setup(); err != nil {
- return config.Cfg{}, fmt.Errorf("failed setting up cgroups: %w", err)
- }
-
sentry.ConfigureSentry(version.GetVersion(), sentry.Config(cfg.Logging.Sentry))
cfg.Prometheus.Configure()
tracing.Initialize(tracing.WithServiceName("gitaly"))
@@ -149,7 +145,7 @@ func run(cfg config.Cfg) error {
defer cancel()
if cfg.RuntimeDir != "" {
- if err := config.PruneRuntimeDirectories(log.StandardLogger(), cfg.RuntimeDir); err != nil {
+ if err := config.PruneOldGitalyProcessDirectories(log.StandardLogger(), cfg.RuntimeDir); err != nil {
return fmt.Errorf("prune runtime directories: %w", err)
}
}
@@ -160,6 +156,23 @@ func run(cfg config.Cfg) error {
}
cfg.RuntimeDir = runtimeDir
+ // When cgroups are configured, we create a directory structure each
+ // time a gitaly process is spawned. Look through the hierarchy root
+ // to find any cgroup directories that belong to old gitaly processes
+ // and remove them.
+ cgroups.PruneOldCgroups(cfg.Cgroups, log.StandardLogger())
+ cgroupMgr := cgroups.NewManager(cfg.Cgroups, os.Getpid())
+
+ if err := cgroupMgr.Setup(); err != nil {
+ return fmt.Errorf("failed setting up cgroups: %w", err)
+ }
+
+ defer func() {
+ if err := cgroupMgr.Cleanup(); err != nil {
+ log.WithError(err).Warn("error cleaning up cgroups")
+ }
+ }()
+
defer func() {
if err := os.RemoveAll(cfg.RuntimeDir); err != nil {
log.Warn("could not clean up runtime dir")
@@ -380,12 +393,6 @@ func run(cfg config.Cfg) error {
}
defer shutdownWorkers()
- defer func() {
- if err := cgroups.NewManager(cfg.Cgroups).Cleanup(); err != nil {
- log.WithError(err).Warn("error cleaning up cgroups")
- }
- }()
-
gracefulStopTicker := helper.NewTimerTicker(cfg.GracefulRestartTimeout.Duration())
defer gracefulStopTicker.Stop()
diff --git a/internal/cgroups/cgroups.go b/internal/cgroups/cgroups.go
index b2209b509..53c2a47d4 100644
--- a/internal/cgroups/cgroups.go
+++ b/internal/cgroups/cgroups.go
@@ -1,9 +1,13 @@
package cgroups
import (
+ "path/filepath"
+
"github.com/prometheus/client_golang/prometheus"
+ log "github.com/sirupsen/logrus"
"gitlab.com/gitlab-org/gitaly/v15/internal/command"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config/cgroups"
)
@@ -24,10 +28,33 @@ type Manager interface {
}
// NewManager returns the appropriate Cgroups manager
-func NewManager(cfg cgroups.Config) Manager {
+func NewManager(cfg cgroups.Config, pid int) Manager {
if cfg.Repositories.Count > 0 {
- return newV1Manager(cfg)
+ return newV1Manager(cfg, pid)
}
return &NoopManager{}
}
+
+// PruneOldCgroups prunes old cgroups for both the memory and cpu subsystems
+func PruneOldCgroups(cfg cgroups.Config, logger log.FieldLogger) {
+ if cfg.HierarchyRoot == "" {
+ return
+ }
+
+ if err := config.PruneOldGitalyProcessDirectories(
+ logger,
+ filepath.Join(cfg.Mountpoint, "memory",
+ cfg.HierarchyRoot),
+ ); err != nil {
+ logger.WithError(err).Error("failed to clean up memory cgroups")
+ }
+
+ if err := config.PruneOldGitalyProcessDirectories(
+ logger,
+ filepath.Join(cfg.Mountpoint, "cpu",
+ cfg.HierarchyRoot),
+ ); err != nil {
+ logger.WithError(err).Error("failed to clean up cpu cgroups")
+ }
+}
diff --git a/internal/cgroups/cgroups_linux_test.go b/internal/cgroups/cgroups_linux_test.go
index 98d469731..a921bc778 100644
--- a/internal/cgroups/cgroups_linux_test.go
+++ b/internal/cgroups/cgroups_linux_test.go
@@ -3,8 +3,13 @@
package cgroups
import (
+ "fmt"
+ "os"
+ "os/exec"
+ "path/filepath"
"testing"
+ "github.com/sirupsen/logrus/hooks/test"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config/cgroups"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
@@ -18,5 +23,144 @@ func TestNewManager(t *testing.T) {
cfg := cgroups.Config{Repositories: cgroups.Repositories{Count: 10}}
require.IsType(t, &CGroupV1Manager{}, &CGroupV1Manager{cfg: cfg})
- require.IsType(t, &NoopManager{}, NewManager(cgroups.Config{}))
+ require.IsType(t, &NoopManager{}, NewManager(cgroups.Config{}, 1))
+}
+
+func TestPruneOldCgroups(t *testing.T) {
+ t.Parallel()
+
+ testCases := []struct {
+ desc string
+ cfg cgroups.Config
+ expectedPruned bool
+ // setup returns a pid
+ setup func(*testing.T, cgroups.Config) int
+ }{
+ {
+ desc: "process belongs to another user",
+ cfg: cgroups.Config{
+ Mountpoint: testhelper.TempDir(t),
+ HierarchyRoot: "gitaly",
+ Repositories: cgroups.Repositories{
+ Count: 10,
+ MemoryBytes: 10 * 1024 * 1024,
+ CPUShares: 1024,
+ },
+ },
+ setup: func(t *testing.T, cfg cgroups.Config) int {
+ pid := 1
+ cgroupManager := NewManager(cfg, pid)
+ require.NoError(t, cgroupManager.Setup())
+
+ return pid
+ },
+ expectedPruned: true,
+ },
+ {
+ desc: "no hierarchy root",
+ cfg: cgroups.Config{
+ Mountpoint: testhelper.TempDir(t),
+ HierarchyRoot: "",
+ Repositories: cgroups.Repositories{
+ Count: 10,
+ MemoryBytes: 10 * 1024 * 1024,
+ CPUShares: 1024,
+ },
+ },
+ setup: func(t *testing.T, cfg cgroups.Config) int {
+ pid := 1
+ cgroupManager := NewManager(cfg, pid)
+ require.NoError(t, cgroupManager.Setup())
+
+ return 1
+ },
+ expectedPruned: false,
+ },
+ {
+ desc: "pid of finished process",
+ cfg: cgroups.Config{
+ Mountpoint: testhelper.TempDir(t),
+ HierarchyRoot: "gitaly",
+ Repositories: cgroups.Repositories{
+ Count: 10,
+ MemoryBytes: 10 * 1024 * 1024,
+ CPUShares: 1024,
+ },
+ },
+ setup: func(t *testing.T, cfg cgroups.Config) int {
+ cmd := exec.Command("ls")
+ require.NoError(t, cmd.Run())
+ pid := cmd.Process.Pid
+
+ cgroupManager := NewManager(cfg, pid)
+ require.NoError(t, cgroupManager.Setup())
+
+ return pid
+ },
+ expectedPruned: true,
+ },
+ {
+ desc: "pid of running process",
+ cfg: cgroups.Config{
+ Mountpoint: testhelper.TempDir(t),
+ HierarchyRoot: "gitaly",
+ Repositories: cgroups.Repositories{
+ Count: 10,
+ MemoryBytes: 10 * 1024 * 1024,
+ CPUShares: 1024,
+ },
+ },
+ setup: func(t *testing.T, cfg cgroups.Config) int {
+ pid := os.Getpid()
+
+ cgroupManager := NewManager(cfg, pid)
+ require.NoError(t, cgroupManager.Setup())
+
+ return pid
+ },
+ expectedPruned: false,
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.desc, func(t *testing.T) {
+ memoryRoot := filepath.Join(
+ tc.cfg.Mountpoint,
+ "memory",
+ tc.cfg.HierarchyRoot,
+ )
+ cpuRoot := filepath.Join(
+ tc.cfg.Mountpoint,
+ "cpu",
+ tc.cfg.HierarchyRoot,
+ )
+
+ require.NoError(t, os.MkdirAll(cpuRoot, os.ModePerm))
+ require.NoError(t, os.MkdirAll(memoryRoot, os.ModePerm))
+
+ pid := tc.setup(t, tc.cfg)
+
+ logger, hook := test.NewNullLogger()
+ PruneOldCgroups(tc.cfg, logger)
+
+ // create cgroups directories with a different pid
+ oldGitalyProcessMemoryDir := filepath.Join(
+ memoryRoot,
+ fmt.Sprintf("gitaly-%d", pid),
+ )
+ oldGitalyProcesssCPUDir := filepath.Join(
+ cpuRoot,
+ fmt.Sprintf("gitaly-%d", pid),
+ )
+
+ if tc.expectedPruned {
+ require.NoDirExists(t, oldGitalyProcessMemoryDir)
+ require.NoDirExists(t, oldGitalyProcesssCPUDir)
+ } else {
+ require.DirExists(t, oldGitalyProcessMemoryDir)
+ require.DirExists(t, oldGitalyProcesssCPUDir)
+ require.Len(t, hook.Entries, 0)
+ }
+ })
+ }
}
diff --git a/internal/cgroups/v1.go b/internal/cgroups/v1.go
index 545f27bed..eb19db799 100644
--- a/internal/cgroups/v1.go
+++ b/internal/cgroups/v1.go
@@ -7,6 +7,6 @@ import (
)
// For systems other than Linux, we return a noop manager if cgroups was enabled.
-func newV1Manager(cfg cgroups.Config) *NoopManager {
+func newV1Manager(cfg cgroups.Config, pid int) *NoopManager {
return &NoopManager{}
}
diff --git a/internal/cgroups/v1_linux.go b/internal/cgroups/v1_linux.go
index 9b284dd1f..819abbbe8 100644
--- a/internal/cgroups/v1_linux.go
+++ b/internal/cgroups/v1_linux.go
@@ -3,7 +3,7 @@ package cgroups
import (
"fmt"
"hash/crc32"
- "os"
+ "path/filepath"
"strings"
"github.com/containerd/cgroups"
@@ -11,6 +11,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/v15/internal/command"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
cgroupscfg "gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config/cgroups"
"gitlab.com/gitlab-org/gitaly/v15/internal/log"
)
@@ -21,11 +22,13 @@ type CGroupV1Manager struct {
hierarchy func() ([]cgroups.Subsystem, error)
memoryReclaimAttemptsTotal, cpuUsage *prometheus.GaugeVec
procs *prometheus.GaugeVec
+ pid int
}
-func newV1Manager(cfg cgroupscfg.Config) *CGroupV1Manager {
+func newV1Manager(cfg cgroupscfg.Config, pid int) *CGroupV1Manager {
return &CGroupV1Manager{
cfg: cfg,
+ pid: pid,
hierarchy: func() ([]cgroups.Subsystem, error) {
return defaultSubsystems(cfg.Mountpoint)
},
@@ -214,11 +217,11 @@ func (cg *CGroupV1Manager) Cleanup() error {
}
func (cg *CGroupV1Manager) repoPath(groupID int) string {
- return fmt.Sprintf("%s/repos-%d", cg.currentProcessCgroup(), groupID)
+ return filepath.Join(cg.currentProcessCgroup(), fmt.Sprintf("repos-%d", groupID))
}
func (cg *CGroupV1Manager) currentProcessCgroup() string {
- return fmt.Sprintf("/%s/gitaly-%d", cg.cfg.HierarchyRoot, os.Getpid())
+ return config.GetGitalyProcessTempDir(cg.cfg.HierarchyRoot, cg.pid)
}
func defaultSubsystems(root string) ([]cgroups.Subsystem, error) {
diff --git a/internal/cgroups/v1_linux_test.go b/internal/cgroups/v1_linux_test.go
index efa15b0bf..ad19884e7 100644
--- a/internal/cgroups/v1_linux_test.go
+++ b/internal/cgroups/v1_linux_test.go
@@ -38,22 +38,24 @@ func defaultCgroupsConfig() cgroups.Config {
func TestSetup(t *testing.T) {
mock := newMock(t)
+ pid := 1
v1Manager := &CGroupV1Manager{
cfg: defaultCgroupsConfig(),
hierarchy: mock.hierarchy,
+ pid: pid,
}
require.NoError(t, v1Manager.Setup())
for i := 0; i < 3; i++ {
memoryPath := filepath.Join(
- mock.root, "memory", "gitaly", fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", i), "memory.limit_in_bytes",
+ mock.root, "memory", "gitaly", fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", i), "memory.limit_in_bytes",
)
memoryContent := readCgroupFile(t, memoryPath)
require.Equal(t, string(memoryContent), "1024000")
cpuPath := filepath.Join(
- mock.root, "cpu", "gitaly", fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", i), "cpu.shares",
+ mock.root, "cpu", "gitaly", fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", i), "cpu.shares",
)
cpuContent := readCgroupFile(t, cpuPath)
@@ -74,9 +76,11 @@ func TestAddCommand(t *testing.T) {
config.Repositories.MemoryBytes = 1024
config.Repositories.CPUShares = 16
+ pid := 1
v1Manager1 := &CGroupV1Manager{
cfg: config,
hierarchy: mock.hierarchy,
+ pid: pid,
}
require.NoError(t, v1Manager1.Setup())
ctx := testhelper.Context(t)
@@ -88,6 +92,7 @@ func TestAddCommand(t *testing.T) {
v1Manager2 := &CGroupV1Manager{
cfg: config,
hierarchy: mock.hierarchy,
+ pid: pid,
}
t.Run("without a repository", func(t *testing.T) {
@@ -99,13 +104,13 @@ func TestAddCommand(t *testing.T) {
for _, s := range mock.subsystems {
path := filepath.Join(mock.root, string(s.Name()), "gitaly",
- fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", groupID), "cgroup.procs")
+ fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", groupID), "cgroup.procs")
content := readCgroupFile(t, path)
- pid, err := strconv.Atoi(string(content))
+ cmdPid, err := strconv.Atoi(string(content))
require.NoError(t, err)
- require.Equal(t, cmd2.Pid(), pid)
+ require.Equal(t, cmd2.Pid(), cmdPid)
}
})
@@ -121,13 +126,13 @@ func TestAddCommand(t *testing.T) {
for _, s := range mock.subsystems {
path := filepath.Join(mock.root, string(s.Name()), "gitaly",
- fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", groupID), "cgroup.procs")
+ fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", groupID), "cgroup.procs")
content := readCgroupFile(t, path)
- pid, err := strconv.Atoi(string(content))
+ cmdPid, err := strconv.Atoi(string(content))
require.NoError(t, err)
- require.Equal(t, cmd2.Pid(), pid)
+ require.Equal(t, cmd2.Pid(), cmdPid)
}
})
}
@@ -135,16 +140,18 @@ func TestAddCommand(t *testing.T) {
func TestCleanup(t *testing.T) {
mock := newMock(t)
+ pid := 1
v1Manager := &CGroupV1Manager{
cfg: defaultCgroupsConfig(),
hierarchy: mock.hierarchy,
+ pid: pid,
}
require.NoError(t, v1Manager.Setup())
require.NoError(t, v1Manager.Cleanup())
for i := 0; i < 3; i++ {
- memoryPath := filepath.Join(mock.root, "memory", "gitaly", fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", i))
- cpuPath := filepath.Join(mock.root, "cpu", "gitaly", fmt.Sprintf("gitaly-%d", os.Getpid()), fmt.Sprintf("repos-%d", i))
+ memoryPath := filepath.Join(mock.root, "memory", "gitaly", fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", i))
+ cpuPath := filepath.Join(mock.root, "cpu", "gitaly", fmt.Sprintf("gitaly-%d", pid), fmt.Sprintf("repos-%d", i))
require.NoDirExists(t, memoryPath)
require.NoDirExists(t, cpuPath)
@@ -165,7 +172,7 @@ func TestMetrics(t *testing.T) {
config.Repositories.MemoryBytes = 1048576
config.Repositories.CPUShares = 16
- v1Manager1 := newV1Manager(config)
+ v1Manager1 := newV1Manager(config, 1)
v1Manager1.hierarchy = mock.hierarchy
mock.setupMockCgroupFiles(t, v1Manager1, 2)
diff --git a/internal/git/command_factory.go b/internal/git/command_factory.go
index 77ff1976c..060082ced 100644
--- a/internal/git/command_factory.go
+++ b/internal/git/command_factory.go
@@ -128,7 +128,7 @@ func NewExecCommandFactory(cfg config.Cfg, opts ...ExecCommandFactoryOption) (_
cfg: cfg,
execEnvs: execEnvs,
locator: config.NewLocator(cfg),
- cgroupsManager: cgroups.NewManager(cfg.Cgroups),
+ cgroupsManager: cgroups.NewManager(cfg.Cgroups, os.Getpid()),
invalidCommandsMetric: prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "gitaly_invalid_commands_total",
diff --git a/internal/gitaly/config/config.go b/internal/gitaly/config/config.go
index 43fae6f40..4b816b752 100644
--- a/internal/gitaly/config/config.go
+++ b/internal/gitaly/config/config.go
@@ -8,7 +8,6 @@ import (
"os"
"path/filepath"
"reflect"
- "strconv"
"strings"
"syscall"
"time"
@@ -543,72 +542,6 @@ func (cfg *Cfg) configurePackObjectsCache() error {
return nil
}
-// PruneRuntimeDirectories removes leftover runtime directories that belonged to processes that
-// no longer exist. The removals are logged prior to being executed. Unexpected directory entries
-// are logged but not removed
-func PruneRuntimeDirectories(log log.FieldLogger, runtimeDir string) error {
- entries, err := os.ReadDir(runtimeDir)
- if err != nil {
- return fmt.Errorf("list runtime directory: %w", err)
- }
-
- for _, entry := range entries {
- if err := func() error {
- log := log.WithField("path", filepath.Join(runtimeDir, entry.Name()))
- if !entry.IsDir() {
- // There should be no files, only the runtime directories.
- log.Error("runtime directory contains an unexpected file")
- return nil
- }
-
- components := strings.Split(entry.Name(), "-")
- if len(components) != 2 || components[0] != "gitaly" {
- // This directory does not match the runtime directory naming format
- // of `gitaly-<process id>.
- log.Error("runtime directory contains an unexpected directory")
- return nil
- }
-
- processID, err := strconv.ParseInt(components[1], 10, 64)
- if err != nil {
- // This is not a runtime directory as the section after the hyphen is not a process id.
- log.Error("runtime directory contains an unexpected directory")
- return nil
- }
-
- process, err := os.FindProcess(int(processID))
- if err != nil {
- return fmt.Errorf("find process: %w", err)
- }
- defer func() {
- if err := process.Release(); err != nil {
- log.WithError(err).Error("failed releasing process")
- }
- }()
-
- if err := process.Signal(syscall.Signal(0)); err != nil {
- // Either the process does not exist, or the pid has been re-used by for a
- // process owned by another user and is not a Gitaly process.
- if !errors.Is(err, os.ErrProcessDone) && !errors.Is(err, syscall.EPERM) {
- return fmt.Errorf("signal: %w", err)
- }
-
- log.Info("removing leftover runtime directory")
-
- if err := os.RemoveAll(filepath.Join(runtimeDir, entry.Name())); err != nil {
- return fmt.Errorf("remove leftover runtime directory: %w", err)
- }
- }
-
- return nil
- }(); err != nil {
- return err
- }
- }
-
- return nil
-}
-
// SetupRuntimeDirectory creates a new runtime directory. Runtime directory contains internal
// runtime data generated by Gitaly such as the internal sockets. If cfg.RuntimeDir is set,
// it's used as the parent directory for the runtime directory. Runtime directory owner process
@@ -638,7 +571,7 @@ func SetupRuntimeDirectory(cfg Cfg, processID int) (string, error) {
// PID exists. Furthermore, it allows easier debugging in case one wants to inspect
// the runtime directory of a running Gitaly node.
- runtimeDir = filepath.Join(cfg.RuntimeDir, fmt.Sprintf("gitaly-%d", processID))
+ runtimeDir = GetGitalyProcessTempDir(cfg.RuntimeDir, processID)
if _, err := os.Stat(runtimeDir); err != nil && !os.IsNotExist(err) {
return "", fmt.Errorf("statting runtime directory: %w", err)
diff --git a/internal/gitaly/config/config_test.go b/internal/gitaly/config/config_test.go
index fb93676f8..ed56ae236 100644
--- a/internal/gitaly/config/config_test.go
+++ b/internal/gitaly/config/config_test.go
@@ -6,13 +6,11 @@ import (
"errors"
"fmt"
"os"
- "os/exec"
"path/filepath"
"strings"
"testing"
"time"
- "github.com/sirupsen/logrus/hooks/test"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config/auth"
@@ -1301,88 +1299,3 @@ func TestSetupRuntimeDirectory(t *testing.T) {
}
})
}
-
-func TestPruneRuntimeDirectories(t *testing.T) {
- t.Run("no runtime directories", func(t *testing.T) {
- require.NoError(t, PruneRuntimeDirectories(testhelper.NewDiscardingLogEntry(t), testhelper.TempDir(t)))
- })
-
- t.Run("unset runtime directory", func(t *testing.T) {
- require.EqualError(t, PruneRuntimeDirectories(testhelper.NewDiscardingLogEntry(t), ""), "list runtime directory: open : no such file or directory")
- })
-
- t.Run("non-existent runtime directory", func(t *testing.T) {
- require.EqualError(t, PruneRuntimeDirectories(testhelper.NewDiscardingLogEntry(t), "/path/does/not/exist"), "list runtime directory: open /path/does/not/exist: no such file or directory")
- })
-
- t.Run("invalid, stale and active runtime directories", func(t *testing.T) {
- baseDir := testhelper.TempDir(t)
- cfg := Cfg{RuntimeDir: baseDir}
-
- // Setup a runtime directory for our process, it can't be stale as long as
- // we are running.
- ownRuntimeDir, err := SetupRuntimeDirectory(cfg, os.Getpid())
- require.NoError(t, err)
-
- expectedLogs := map[string]string{}
-
- // Setup runtime directories for processes that have finished.
- var prunableDirs []string
- for i := 0; i < 2; i++ {
- cmd := exec.Command("cat")
- require.NoError(t, cmd.Run())
-
- staleRuntimeDir, err := SetupRuntimeDirectory(cfg, cmd.Process.Pid)
- require.NoError(t, err)
-
- prunableDirs = append(prunableDirs, staleRuntimeDir)
- expectedLogs[staleRuntimeDir] = "removing leftover runtime directory"
- }
-
- // Setup runtime directory with pid of process not owned by git user
- rootRuntimeDir, err := SetupRuntimeDirectory(cfg, 1)
- require.NoError(t, err)
- expectedLogs[rootRuntimeDir] = "removing leftover runtime directory"
- prunableDirs = append(prunableDirs, rootRuntimeDir)
-
- // Create an unexpected file in the runtime directory
- unexpectedFilePath := filepath.Join(baseDir, "unexpected-file")
- require.NoError(t, os.WriteFile(unexpectedFilePath, []byte(""), os.ModePerm))
- expectedLogs[unexpectedFilePath] = "runtime directory contains an unexpected file"
-
- nonPrunableDirs := []string{ownRuntimeDir}
-
- // Setup some unexpected directories in the runtime directory
- for _, dirName := range []string{
- "nohyphen",
- "too-many-hyphens",
- "invalidprefix-3",
- "gitaly-invalidpid",
- } {
- dirPath := filepath.Join(baseDir, dirName)
- require.NoError(t, os.Mkdir(dirPath, os.ModePerm))
- expectedLogs[dirPath] = "runtime directory contains an unexpected directory"
- nonPrunableDirs = append(nonPrunableDirs, dirPath)
- }
-
- logger, hook := test.NewNullLogger()
- require.NoError(t, PruneRuntimeDirectories(logger, cfg.RuntimeDir))
-
- actualLogs := map[string]string{}
- for _, entry := range hook.Entries {
- actualLogs[entry.Data["path"].(string)] = entry.Message
- }
-
- require.Equal(t, expectedLogs, actualLogs)
-
- require.FileExists(t, unexpectedFilePath)
-
- for _, nonPrunableEntry := range nonPrunableDirs {
- require.DirExists(t, nonPrunableEntry, nonPrunableEntry)
- }
-
- for _, prunableEntry := range prunableDirs {
- require.NoDirExists(t, prunableEntry, prunableEntry)
- }
- })
-}
diff --git a/internal/gitaly/config/temp_dir.go b/internal/gitaly/config/temp_dir.go
new file mode 100644
index 000000000..d27100edf
--- /dev/null
+++ b/internal/gitaly/config/temp_dir.go
@@ -0,0 +1,87 @@
+package config
+
+import (
+ "errors"
+ "fmt"
+ "os"
+ "path/filepath"
+ "strconv"
+ "strings"
+ "syscall"
+
+ log "github.com/sirupsen/logrus"
+)
+
+// PruneOldGitalyProcessDirectories removes leftover temporary directories that belonged to processes that
+// no longer exist. Directories are expected to be in the form gitaly-<pid>.
+// The removals are logged prior to being executed. Unexpected directory entries are logged
+// but not removed.
+func PruneOldGitalyProcessDirectories(log log.FieldLogger, directory string) error {
+ entries, err := os.ReadDir(directory)
+ if err != nil {
+ return fmt.Errorf("list gitaly process directory: %w", err)
+ }
+
+ for _, entry := range entries {
+ log := log.WithField("path", filepath.Join(directory, entry.Name()))
+ if err := func() error {
+ if !entry.IsDir() {
+ // There should be no files, only the gitaly process directories.
+ return errors.New("gitaly process directory contains an unexpected file")
+ }
+
+ components := strings.Split(entry.Name(), "-")
+ if len(components) != 2 || components[0] != "gitaly" {
+ // This directory does not match the gitaly process directory naming format
+ // of `gitaly-<process id>.
+ return errors.New("gitaly process directory contains an unexpected directory")
+ }
+
+ processID, err := strconv.ParseInt(components[1], 10, 64)
+ if err != nil {
+ // This is not a temporary gitaly process directory as the section
+ // after the hyphen is not a process id.
+ return errors.New("gitaly process directory contains an unexpected directory")
+ }
+
+ process, err := os.FindProcess(int(processID))
+ if err != nil {
+ return fmt.Errorf("could not find process: %w", err)
+ }
+
+ defer func() {
+ if err := process.Release(); err != nil {
+ log.WithError(err).Error("failed releasing process")
+ }
+ }()
+
+ if err := process.Signal(syscall.Signal(0)); err != nil {
+ // Either the process does not exist, or the pid has been re-used by for a
+ // process owned by another user and is not a Gitaly process.
+ if !errors.Is(err, os.ErrProcessDone) && !errors.Is(err, syscall.EPERM) {
+ return fmt.Errorf("sending signal 0 to process: %w", err)
+ }
+
+ if err := os.RemoveAll(filepath.Join(directory, entry.Name())); err != nil {
+ return fmt.Errorf("removing leftover gitaly process directory: %w", err)
+ }
+
+ log.Info("removed leftover gitaly process directory")
+ }
+
+ return nil
+ }(); err != nil {
+ log.WithError(err).Error("could not prune entry")
+ continue
+ }
+ }
+
+ return nil
+}
+
+// GetGitalyProcessTempDir constructs a temporary directory name for the current gitaly
+// process. This way, we can clean up old temporary directories by inspecting the pid attached
+// to the folder.
+func GetGitalyProcessTempDir(parentDir string, processID int) string {
+ return filepath.Join(parentDir, fmt.Sprintf("gitaly-%d", processID))
+}
diff --git a/internal/gitaly/config/temp_dir_test.go b/internal/gitaly/config/temp_dir_test.go
new file mode 100644
index 000000000..9961278ce
--- /dev/null
+++ b/internal/gitaly/config/temp_dir_test.go
@@ -0,0 +1,113 @@
+//go:build !gitaly_test_sha256
+
+package config
+
+import (
+ "errors"
+ "os"
+ "os/exec"
+ "path/filepath"
+ "testing"
+
+ "github.com/sirupsen/logrus/hooks/test"
+ "github.com/stretchr/testify/require"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
+)
+
+func TestPruneOldGitalyProcessDirectories(t *testing.T) {
+ t.Run("no runtime directories", func(t *testing.T) {
+ require.NoError(t, PruneOldGitalyProcessDirectories(testhelper.NewDiscardingLogEntry(t), testhelper.TempDir(t)))
+ })
+
+ t.Run("unset runtime directory", func(t *testing.T) {
+ require.EqualError(t,
+ PruneOldGitalyProcessDirectories(testhelper.NewDiscardingLogEntry(t), ""), "list gitaly process directory: open : no such file or directory")
+ })
+
+ t.Run("non-existent runtime directory", func(t *testing.T) {
+ require.EqualError(t,
+ PruneOldGitalyProcessDirectories(testhelper.NewDiscardingLogEntry(t),
+ "/path/does/not/exist"), "list gitaly process directory: open /path/does/not/exist: no such file or directory")
+ })
+
+ t.Run("invalid, stale and active runtime directories", func(t *testing.T) {
+ baseDir := testhelper.TempDir(t)
+ cfg := Cfg{RuntimeDir: baseDir}
+
+ // Setup a runtime directory for our process, it can't be stale as long as
+ // we are running.
+ ownRuntimeDir, err := SetupRuntimeDirectory(cfg, os.Getpid())
+ require.NoError(t, err)
+
+ expectedLogs := map[string]string{}
+ expectedErrs := map[string]error{}
+
+ // Setup runtime directories for processes that have finished.
+ var prunableDirs []string
+ for i := 0; i < 2; i++ {
+ cmd := exec.Command("cat")
+ require.NoError(t, cmd.Run())
+
+ staleRuntimeDir, err := SetupRuntimeDirectory(cfg, cmd.Process.Pid)
+ require.NoError(t, err)
+
+ prunableDirs = append(prunableDirs, staleRuntimeDir)
+ expectedLogs[staleRuntimeDir] = "removed leftover gitaly process directory"
+ }
+
+ // Setup runtime directory with pid of process not owned by git user
+ rootRuntimeDir, err := SetupRuntimeDirectory(cfg, 1)
+ require.NoError(t, err)
+ expectedLogs[rootRuntimeDir] = "removed leftover gitaly process directory"
+ prunableDirs = append(prunableDirs, rootRuntimeDir)
+
+ // Create an unexpected file in the runtime directory
+ unexpectedFilePath := filepath.Join(baseDir, "unexpected-file")
+ require.NoError(t, os.WriteFile(unexpectedFilePath, []byte(""), os.ModePerm))
+ expectedLogs[unexpectedFilePath] = "could not prune entry"
+ expectedErrs[unexpectedFilePath] = errors.New("gitaly process directory contains an unexpected file")
+
+ nonPrunableDirs := []string{ownRuntimeDir}
+
+ // Setup some unexpected directories in the runtime directory
+ for _, dirName := range []string{
+ "nohyphen",
+ "too-many-hyphens",
+ "invalidprefix-3",
+ "gitaly-invalidpid",
+ } {
+ dirPath := filepath.Join(baseDir, dirName)
+ require.NoError(t, os.Mkdir(dirPath, os.ModePerm))
+ expectedLogs[dirPath] = "could not prune entry"
+ expectedErrs[dirPath] = errors.New("gitaly process directory contains an unexpected directory")
+ nonPrunableDirs = append(nonPrunableDirs, dirPath)
+ }
+
+ logger, hook := test.NewNullLogger()
+ require.NoError(t, PruneOldGitalyProcessDirectories(logger, cfg.RuntimeDir))
+
+ actualLogs := map[string]string{}
+ actualErrs := map[string]error{}
+ for _, entry := range hook.Entries {
+ actualLogs[entry.Data["path"].(string)] = entry.Message
+ if entry.Data["error"] != nil {
+ err, ok := entry.Data["error"].(error)
+ require.True(t, ok)
+ actualErrs[entry.Data["path"].(string)] = err
+ }
+ }
+
+ require.Equal(t, expectedLogs, actualLogs)
+ require.Equal(t, expectedErrs, actualErrs)
+
+ require.FileExists(t, unexpectedFilePath)
+
+ for _, nonPrunableEntry := range nonPrunableDirs {
+ require.DirExists(t, nonPrunableEntry, nonPrunableEntry)
+ }
+
+ for _, prunableEntry := range prunableDirs {
+ require.NoDirExists(t, prunableEntry, prunableEntry)
+ }
+ })
+}