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:
authorWill Chandler <wchandler@gitlab.com>2023-11-09 23:05:49 +0300
committerWill Chandler <wchandler@gitlab.com>2023-11-11 00:30:55 +0300
commit2bb704a1deb70925ad1d825a9a75e03d1a218e30 (patch)
treef45a0672d7a1dbe4277dd96d82aff137e62c443a
parent02b3a2d7513a17327789af186210dc17528f1702 (diff)
cgroups: Remove v1 and v2 handlerswc/cgroup-generic
Remove the now unused v1 and v2 handler implementations, copying still-used freestanding functions into `handler_linux.go`.
-rw-r--r--internal/cgroups/handler_linux.go40
-rw-r--r--internal/cgroups/v1_linux.go237
-rw-r--r--internal/cgroups/v2_linux.go220
3 files changed, 40 insertions, 457 deletions
diff --git a/internal/cgroups/handler_linux.go b/internal/cgroups/handler_linux.go
index b9869e543..2b92253fb 100644
--- a/internal/cgroups/handler_linux.go
+++ b/internal/cgroups/handler_linux.go
@@ -5,6 +5,7 @@ package cgroups
import (
"errors"
"fmt"
+ "io/fs"
"path/filepath"
"strings"
"time"
@@ -350,3 +351,42 @@ func v2Collect(control *cgroup2.Manager, m *cgroupsMetrics, repoPath string, log
}
}
}
+
+func defaultSubsystems(root string) ([]cgroup1.Subsystem, error) {
+ subsystems := []cgroup1.Subsystem{
+ cgroup1.NewMemory(root, cgroup1.OptionalSwap()),
+ cgroup1.NewCpu(root),
+ }
+
+ return subsystems, nil
+}
+
+func pruneOldCgroupsV1(cfg cgroupscfg.Config, logger log.Logger) {
+ 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")
+ }
+}
+
+func pruneOldCgroupsV2(cfg cgroupscfg.Config, logger log.Logger) {
+ if err := config.PruneOldGitalyProcessDirectories(
+ logger,
+ filepath.Join(cfg.Mountpoint, cfg.HierarchyRoot),
+ ); err != nil {
+ var pathError *fs.PathError
+ if !errors.As(err, &pathError) {
+ logger.WithError(err).Error("failed to clean up cpu cgroups")
+ }
+ }
+}
diff --git a/internal/cgroups/v1_linux.go b/internal/cgroups/v1_linux.go
deleted file mode 100644
index 53f6cb5a4..000000000
--- a/internal/cgroups/v1_linux.go
+++ /dev/null
@@ -1,237 +0,0 @@
-//go:build linux
-
-package cgroups
-
-import (
- "fmt"
- "path/filepath"
- "strings"
- "time"
-
- "github.com/containerd/cgroups/v3/cgroup1"
- specs "github.com/opencontainers/runtime-spec/specs-go"
- "github.com/prometheus/client_golang/prometheus"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
- cgroupscfg "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config/cgroups"
- "gitlab.com/gitlab-org/gitaly/v16/internal/log"
-)
-
-type cgroupV1Handler struct {
- cfg cgroupscfg.Config
- logger log.Logger
- hierarchy func() ([]cgroup1.Subsystem, error)
-
- *cgroupsMetrics
- pid int
-}
-
-func newV1Handler(cfg cgroupscfg.Config, logger log.Logger, pid int) *cgroupV1Handler {
- return &cgroupV1Handler{
- cfg: cfg,
- logger: logger,
- pid: pid,
- hierarchy: func() ([]cgroup1.Subsystem, error) {
- return defaultSubsystems(cfg.Mountpoint)
- },
- cgroupsMetrics: newV1CgroupsMetrics(),
- }
-}
-
-func (cvh *cgroupV1Handler) setupParent(parentResources *specs.LinuxResources) error {
- if _, err := cgroup1.New(
- cgroup1.StaticPath(cvh.currentProcessCgroup()),
- parentResources,
- cgroup1.WithHiearchy(cvh.hierarchy),
- ); err != nil {
- return fmt.Errorf("failed creating parent cgroup: %w", err)
- }
- return nil
-}
-
-func (cvh *cgroupV1Handler) createCgroup(reposResources *specs.LinuxResources, cgroupPath string) error {
- _, err := cgroup1.New(
- cgroup1.StaticPath(cgroupPath),
- reposResources,
- cgroup1.WithHiearchy(cvh.hierarchy),
- )
-
- return err
-}
-
-func (cvh *cgroupV1Handler) addToCgroup(pid int, cgroupPath string) error {
- control, err := cvh.loadCgroup(cgroupPath)
- if err != nil {
- return err
- }
-
- if err := control.Add(cgroup1.Process{Pid: pid}); err != nil {
- // Command could finish so quickly before we can add it to a cgroup, so
- // we don't consider it an error.
- if strings.Contains(err.Error(), "no such process") {
- return nil
- }
- return fmt.Errorf("failed adding process to cgroup: %w", err)
- }
-
- return nil
-}
-
-func (cvh *cgroupV1Handler) loadCgroup(cgroupPath string) (cgroup1.Cgroup, error) {
- control, err := cgroup1.Load(
- cgroup1.StaticPath(cgroupPath),
- cgroup1.WithHiearchy(cvh.hierarchy),
- )
- if err != nil {
- return nil, fmt.Errorf("failed loading %s cgroup: %w", cgroupPath, err)
- }
- return control, nil
-}
-
-func (cvh *cgroupV1Handler) collect(repoPath string, ch chan<- prometheus.Metric) {
- logger := cvh.logger.WithField("cgroup_path", repoPath)
- control, err := cvh.loadCgroup(repoPath)
- if err != nil {
- logger.WithError(err).Warn("unable to load cgroup controller")
- return
- }
-
- if metrics, err := control.Stat(); err != nil {
- logger.WithError(err).Warn("unable to get cgroup stats")
- } else {
- memoryMetric := cvh.memoryReclaimAttemptsTotal.WithLabelValues(repoPath)
- memoryMetric.Set(float64(metrics.Memory.Usage.Failcnt))
- ch <- memoryMetric
-
- cpuUserMetric := cvh.cpuUsage.WithLabelValues(repoPath, "user")
- cpuUserMetric.Set(float64(metrics.CPU.Usage.User))
- ch <- cpuUserMetric
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSPeriods,
- prometheus.CounterValue,
- float64(metrics.CPU.Throttling.Periods),
- repoPath,
- )
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSThrottledPeriods,
- prometheus.CounterValue,
- float64(metrics.CPU.Throttling.ThrottledPeriods),
- repoPath,
- )
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSThrottledTime,
- prometheus.CounterValue,
- float64(metrics.CPU.Throttling.ThrottledTime)/float64(time.Second),
- repoPath,
- )
-
- cpuKernelMetric := cvh.cpuUsage.WithLabelValues(repoPath, "kernel")
- cpuKernelMetric.Set(float64(metrics.CPU.Usage.Kernel))
- ch <- cpuKernelMetric
- }
-
- if subsystems, err := cvh.hierarchy(); err != nil {
- logger.WithError(err).Warn("unable to get cgroup hierarchy")
- } else {
- for _, subsystem := range subsystems {
- processes, err := control.Processes(subsystem.Name(), true)
- if err != nil {
- logger.WithField("subsystem", subsystem.Name()).
- WithError(err).
- Warn("unable to get process list")
- continue
- }
-
- procsMetric := cvh.procs.WithLabelValues(repoPath, string(subsystem.Name()))
- procsMetric.Set(float64(len(processes)))
- ch <- procsMetric
- }
- }
-}
-
-func (cvh *cgroupV1Handler) cleanup() error {
- processCgroupPath := cvh.currentProcessCgroup()
-
- control, err := cvh.loadCgroup(processCgroupPath)
- if err != nil {
- return err
- }
-
- if err := control.Delete(); err != nil {
- return fmt.Errorf("failed cleaning up cgroup %s: %w", processCgroupPath, err)
- }
-
- return nil
-}
-
-func (cvh *cgroupV1Handler) repoPath(groupID int) string {
- return filepath.Join(cvh.currentProcessCgroup(), fmt.Sprintf("repos-%d", groupID))
-}
-
-func (cvh *cgroupV1Handler) currentProcessCgroup() string {
- return config.GetGitalyProcessTempDir(cvh.cfg.HierarchyRoot, cvh.pid)
-}
-
-func (cvh *cgroupV1Handler) stats() (Stats, error) {
- processCgroupPath := cvh.currentProcessCgroup()
-
- control, err := cvh.loadCgroup(processCgroupPath)
- if err != nil {
- return Stats{}, err
- }
-
- metrics, err := control.Stat()
- if err != nil {
- return Stats{}, fmt.Errorf("failed to fetch metrics %s: %w", processCgroupPath, err)
- }
-
- return Stats{
- ParentStats: CgroupStats{
- CPUThrottledCount: metrics.CPU.Throttling.ThrottledPeriods,
- CPUThrottledDuration: float64(metrics.CPU.Throttling.ThrottledTime) / float64(time.Second),
- MemoryUsage: metrics.Memory.Usage.Usage,
- MemoryLimit: metrics.Memory.Usage.Limit,
- OOMKills: metrics.MemoryOomControl.OomKill,
- UnderOOM: metrics.MemoryOomControl.UnderOom != 0,
- Anon: metrics.Memory.RSS,
- ActiveAnon: metrics.Memory.ActiveAnon,
- InactiveAnon: metrics.Memory.InactiveAnon,
- File: metrics.Memory.Cache,
- ActiveFile: metrics.Memory.ActiveFile,
- InactiveFile: metrics.Memory.InactiveFile,
- },
- }, nil
-}
-
-func (cvh *cgroupV1Handler) supportsCloneIntoCgroup() bool {
- return false
-}
-
-func defaultSubsystems(root string) ([]cgroup1.Subsystem, error) {
- subsystems := []cgroup1.Subsystem{
- cgroup1.NewMemory(root, cgroup1.OptionalSwap()),
- cgroup1.NewCpu(root),
- }
-
- return subsystems, nil
-}
-
-func pruneOldCgroupsV1(cfg cgroupscfg.Config, logger log.Logger) {
- 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/v2_linux.go b/internal/cgroups/v2_linux.go
deleted file mode 100644
index 396cf104f..000000000
--- a/internal/cgroups/v2_linux.go
+++ /dev/null
@@ -1,220 +0,0 @@
-//go:build linux
-
-package cgroups
-
-import (
- "errors"
- "fmt"
- "io/fs"
- "path/filepath"
- "strings"
- "time"
-
- "github.com/containerd/cgroups/v3/cgroup2"
- "github.com/opencontainers/runtime-spec/specs-go"
- "github.com/prometheus/client_golang/prometheus"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
- cgroupscfg "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config/cgroups"
- "gitlab.com/gitlab-org/gitaly/v16/internal/kernel"
- "gitlab.com/gitlab-org/gitaly/v16/internal/log"
-)
-
-type cgroupV2Handler struct {
- cfg cgroupscfg.Config
- logger log.Logger
-
- *cgroupsMetrics
- pid int
- cloneIntoCgroup bool
-}
-
-func newV2Handler(cfg cgroupscfg.Config, logger log.Logger, pid int) *cgroupV2Handler {
- cloneIntoCgroup, err := kernel.IsAtLeast(kernel.Version{Major: 5, Minor: 7})
- if err != nil {
- // Log the error for now as we're only rolling out functionality behind feature flag.
- logger.WithError(err).Error("failed detecting kernel version, CLONE_INTO_CGROUP support disabled")
- }
-
- return &cgroupV2Handler{
- cfg: cfg,
- logger: logger,
- pid: pid,
- cgroupsMetrics: newV2CgroupsMetrics(),
- cloneIntoCgroup: cloneIntoCgroup,
- }
-}
-
-func (cvh *cgroupV2Handler) setupParent(parentResources *specs.LinuxResources) error {
- if _, err := cgroup2.NewManager(cvh.cfg.Mountpoint, "/"+cvh.currentProcessCgroup(), cgroup2.ToResources(parentResources)); err != nil {
- return fmt.Errorf("failed creating parent cgroup: %w", err)
- }
-
- return nil
-}
-
-func (cvh *cgroupV2Handler) createCgroup(reposResources *specs.LinuxResources, cgroupPath string) error {
- _, err := cgroup2.NewManager(
- cvh.cfg.Mountpoint,
- "/"+cgroupPath,
- cgroup2.ToResources(reposResources),
- )
-
- return err
-}
-
-func (cvh *cgroupV2Handler) addToCgroup(pid int, cgroupPath string) error {
- control, err := cvh.loadCgroup(cgroupPath)
- if err != nil {
- return err
- }
-
- if err := control.AddProc(uint64(pid)); err != nil {
- // Command could finish so quickly before we can add it to a cgroup, so
- // we don't consider it an error.
- if strings.Contains(err.Error(), "no such process") {
- return nil
- }
- return fmt.Errorf("failed adding process to cgroup: %w", err)
- }
-
- return nil
-}
-
-func (cvh *cgroupV2Handler) loadCgroup(cgroupPath string) (*cgroup2.Manager, error) {
- control, err := cgroup2.Load("/"+cgroupPath, cgroup2.WithMountpoint(cvh.cfg.Mountpoint))
- if err != nil {
- return nil, fmt.Errorf("failed loading %s cgroup: %w", cgroupPath, err)
- }
- return control, nil
-}
-
-func (cvh *cgroupV2Handler) collect(repoPath string, ch chan<- prometheus.Metric) {
- logger := cvh.logger.WithField("cgroup_path", repoPath)
- control, err := cvh.loadCgroup(repoPath)
- if err != nil {
- logger.WithError(err).Warn("unable to load cgroup controller")
- return
- }
-
- if metrics, err := control.Stat(); err != nil {
- logger.WithError(err).Warn("unable to get cgroup stats")
- } else {
- cpuUserMetric := cvh.cpuUsage.WithLabelValues(repoPath, "user")
- cpuUserMetric.Set(float64(metrics.CPU.UserUsec))
- ch <- cpuUserMetric
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSPeriods,
- prometheus.CounterValue,
- float64(metrics.CPU.NrPeriods),
- repoPath,
- )
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSThrottledPeriods,
- prometheus.CounterValue,
- float64(metrics.CPU.NrThrottled),
- repoPath,
- )
-
- ch <- prometheus.MustNewConstMetric(
- cvh.cpuCFSThrottledTime,
- prometheus.CounterValue,
- float64(metrics.CPU.ThrottledUsec)/float64(time.Second),
- repoPath,
- )
-
- cpuKernelMetric := cvh.cpuUsage.WithLabelValues(repoPath, "kernel")
- cpuKernelMetric.Set(float64(metrics.CPU.SystemUsec))
- ch <- cpuKernelMetric
- }
-
- if subsystems, err := control.Controllers(); err != nil {
- logger.WithError(err).Warn("unable to get cgroup hierarchy")
- } else {
- processes, err := control.Procs(true)
- if err != nil {
- logger.WithError(err).
- Warn("unable to get process list")
- return
- }
-
- for _, subsystem := range subsystems {
- procsMetric := cvh.procs.WithLabelValues(repoPath, subsystem)
- procsMetric.Set(float64(len(processes)))
- ch <- procsMetric
- }
- }
-}
-
-func (cvh *cgroupV2Handler) cleanup() error {
- processCgroupPath := cvh.currentProcessCgroup()
-
- control, err := cvh.loadCgroup(processCgroupPath)
- if err != nil {
- return err
- }
-
- if err := control.Delete(); err != nil {
- return fmt.Errorf("failed cleaning up cgroup %s: %w", processCgroupPath, err)
- }
-
- return nil
-}
-
-func (cvh *cgroupV2Handler) repoPath(groupID int) string {
- return filepath.Join(cvh.currentProcessCgroup(), fmt.Sprintf("repos-%d", groupID))
-}
-
-func (cvh *cgroupV2Handler) currentProcessCgroup() string {
- return config.GetGitalyProcessTempDir(cvh.cfg.HierarchyRoot, cvh.pid)
-}
-
-func (cvh *cgroupV2Handler) stats() (Stats, error) {
- processCgroupPath := cvh.currentProcessCgroup()
-
- control, err := cvh.loadCgroup(processCgroupPath)
- if err != nil {
- return Stats{}, err
- }
-
- metrics, err := control.Stat()
- if err != nil {
- return Stats{}, fmt.Errorf("failed to fetch metrics %s: %w", processCgroupPath, err)
- }
-
- stats := Stats{
- ParentStats: CgroupStats{
- CPUThrottledCount: metrics.CPU.NrThrottled,
- CPUThrottledDuration: float64(metrics.CPU.ThrottledUsec) / float64(time.Second),
- MemoryUsage: metrics.Memory.Usage,
- MemoryLimit: metrics.Memory.UsageLimit,
- Anon: metrics.Memory.Anon,
- ActiveAnon: metrics.Memory.ActiveAnon,
- InactiveAnon: metrics.Memory.InactiveAnon,
- File: metrics.Memory.File,
- ActiveFile: metrics.Memory.ActiveFile,
- InactiveFile: metrics.Memory.InactiveFile,
- },
- }
- if metrics.MemoryEvents != nil {
- stats.ParentStats.OOMKills = metrics.MemoryEvents.OomKill
- }
- return stats, nil
-}
-
-func (cvh *cgroupV2Handler) supportsCloneIntoCgroup() bool {
- return cvh.cloneIntoCgroup
-}
-
-func pruneOldCgroupsV2(cfg cgroupscfg.Config, logger log.Logger) {
- if err := config.PruneOldGitalyProcessDirectories(
- logger,
- filepath.Join(cfg.Mountpoint, cfg.HierarchyRoot),
- ); err != nil {
- var pathError *fs.PathError
- if !errors.As(err, &pathError) {
- logger.WithError(err).Error("failed to clean up cpu cgroups")
- }
- }
-}