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:
authorSami Hiltunen <shiltunen@gitlab.com>2022-08-18 10:10:49 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2022-08-18 10:37:23 +0300
commitb5882821760d8d5c4ee2876c05e4eaf81edfbaef (patch)
treed19746987e65f56107ec820dd8c63b6f9426dfc4
parentbb4376f98acdf012abd74ebb3e9b77b93aaf3b57 (diff)
Remove custom duration type
Back in the day, a custom duration type was added to Gitaly's config package as the TOML parser used didn't support parsing duration types natively. The TOML parser used these days supports the native time.Duration type of Go. Remove the custom type and update all sites to use the native duration type.
-rw-r--r--cmd/gitaly/main.go2
-rw-r--r--cmd/praefect/main.go12
-rw-r--r--internal/gitaly/config/config.go30
-rw-r--r--internal/gitaly/config/config_test.go14
-rw-r--r--internal/gitaly/config/ruby.go47
-rw-r--r--internal/gitaly/maintenance/daily.go6
-rw-r--r--internal/gitaly/maintenance/daily_test.go2
-rw-r--r--internal/gitaly/rubyserver/rubyserver.go4
-rw-r--r--internal/middleware/limithandler/concurrency_limiter.go2
-rw-r--r--internal/praefect/config/config.go38
-rw-r--r--internal/praefect/config/config_test.go38
-rw-r--r--internal/praefect/nodes/tracker/errors.go2
-rw-r--r--internal/streamcache/cache.go4
-rw-r--r--internal/streamcache/cache_test.go2
14 files changed, 89 insertions, 114 deletions
diff --git a/cmd/gitaly/main.go b/cmd/gitaly/main.go
index 0f04a2b5d..7e22bc27f 100644
--- a/cmd/gitaly/main.go
+++ b/cmd/gitaly/main.go
@@ -381,7 +381,7 @@ func run(cfg config.Cfg) error {
}
}()
- gracefulStopTicker := helper.NewTimerTicker(cfg.GracefulRestartTimeout.Duration())
+ gracefulStopTicker := helper.NewTimerTicker(cfg.GracefulRestartTimeout)
defer gracefulStopTicker.Stop()
return b.Wait(gracefulStopTicker, gitalyServerFactory.GracefulStop)
diff --git a/cmd/praefect/main.go b/cmd/praefect/main.go
index b9edba3d1..1a434436c 100644
--- a/cmd/praefect/main.go
+++ b/cmd/praefect/main.go
@@ -394,7 +394,7 @@ func run(
primaryGetter = nodeMgr
nodeManager = nodeMgr
- nodeMgr.Start(conf.Failover.BootstrapInterval.Duration(), conf.Failover.MonitorInterval.Duration())
+ nodeMgr.Start(conf.Failover.BootstrapInterval, conf.Failover.MonitorInterval)
defer nodeMgr.Stop()
}
@@ -520,7 +520,7 @@ func run(
repl.ProcessStale(ctx, staleTicker, time.Minute)
logger.Info("background started: processing of the stale replication events")
- if interval := conf.Reconciliation.SchedulingInterval.Duration(); interval > 0 {
+ if interval := conf.Reconciliation.SchedulingInterval; interval > 0 {
if conf.MemoryQueueEnabled {
logger.Warn("Disabled automatic reconciliation as it is only implemented using SQL queue and in-memory queue is configured.")
} else {
@@ -540,17 +540,17 @@ func run(
}
}
- if interval := conf.RepositoriesCleanup.RunInterval.Duration(); interval > 0 {
+ if interval := conf.RepositoriesCleanup.RunInterval; interval > 0 {
if db != nil {
go func() {
storageSync := datastore.NewStorageCleanup(db)
cfg := repocleaner.Cfg{
- RunInterval: conf.RepositoriesCleanup.RunInterval.Duration(),
+ RunInterval: conf.RepositoriesCleanup.RunInterval,
LivenessInterval: 30 * time.Second,
RepositoriesInBatch: conf.RepositoriesCleanup.RepositoriesInBatch,
}
repoCleaner := repocleaner.NewRunner(cfg, logger, healthChecker, nodeSet.Connections(), storageSync, storageSync, repocleaner.NewLogWarnAction(logger))
- if err := repoCleaner.Run(ctx, helper.NewTimerTicker(conf.RepositoriesCleanup.CheckInterval.Duration())); err != nil && !errors.Is(context.Canceled, err) {
+ if err := repoCleaner.Run(ctx, helper.NewTimerTicker(conf.RepositoriesCleanup.CheckInterval)); err != nil && !errors.Is(context.Canceled, err) {
logger.WithError(err).Error("repository cleaner finished execution")
} else {
logger.Info("repository cleaner finished execution")
@@ -563,7 +563,7 @@ func run(
logger.Warn(`Repository cleanup background task disabled as "repositories_cleanup.run_interval" is not set or 0.`)
}
- gracefulStopTicker := helper.NewTimerTicker(conf.GracefulStopTimeout.Duration())
+ gracefulStopTicker := helper.NewTimerTicker(conf.GracefulStopTimeout)
defer gracefulStopTicker.Stop()
return b.Wait(gracefulStopTicker, srvFactory.GracefulStop)
diff --git a/internal/gitaly/config/config.go b/internal/gitaly/config/config.go
index 8ca0d0d24..c8988f77e 100644
--- a/internal/gitaly/config/config.go
+++ b/internal/gitaly/config/config.go
@@ -32,10 +32,10 @@ const (
// DailyJob enables a daily task to be scheduled for specific storages
type DailyJob struct {
- Hour uint `toml:"start_hour"`
- Minute uint `toml:"start_minute"`
- Duration Duration `toml:"duration"`
- Storages []string `toml:"storages"`
+ Hour uint `toml:"start_hour"`
+ Minute uint `toml:"start_minute"`
+ Duration time.Duration `toml:"duration"`
+ Storages []string `toml:"storages"`
// Disabled will completely disable a daily job, even in cases where a
// default schedule is implied
@@ -62,7 +62,7 @@ type Cfg struct {
Hooks Hooks `toml:"hooks"`
Concurrency []Concurrency `toml:"concurrency"`
RateLimiting []RateLimiting `toml:"rate_limiting"`
- GracefulRestartTimeout Duration `toml:"graceful_restart_timeout"`
+ GracefulRestartTimeout time.Duration `toml:"graceful_restart_timeout"`
DailyMaintenance DailyJob `toml:"daily_maintenance"`
Cgroups cgroups.Config `toml:"cgroups"`
PackObjectsCache StreamCacheConfig `toml:"pack_objects_cache"`
@@ -147,7 +147,7 @@ type Concurrency struct {
MaxQueueSize int `toml:"max_queue_size"`
// MaxQueueWait is the maximum time a request can remain in the concurrency queue
// waiting to be picked up by Gitaly
- MaxQueueWait Duration `toml:"max_queue_wait"`
+ MaxQueueWait time.Duration `toml:"max_queue_wait"`
}
// RateLimiting allows endpoints to be limited to a maximum request rate per
@@ -168,9 +168,9 @@ type RateLimiting struct {
// StreamCacheConfig contains settings for a streamcache instance.
type StreamCacheConfig struct {
- Enabled bool `toml:"enabled"` // Default: false
- Dir string `toml:"dir"` // Default: <FIRST STORAGE PATH>/+gitaly/PackObjectsCache
- MaxAge Duration `toml:"max_age"` // Default: 5m
+ Enabled bool `toml:"enabled"` // Default: false
+ Dir string `toml:"dir"` // Default: <FIRST STORAGE PATH>/+gitaly/PackObjectsCache
+ MaxAge time.Duration `toml:"max_age"` // Default: 5m
}
// Load initializes the Config variable from file and the environment.
@@ -219,8 +219,8 @@ func (cfg *Cfg) Validate() error {
}
func (cfg *Cfg) setDefaults() error {
- if cfg.GracefulRestartTimeout.Duration() == 0 {
- cfg.GracefulRestartTimeout = Duration(time.Minute)
+ if cfg.GracefulRestartTimeout == 0 {
+ cfg.GracefulRestartTimeout = time.Minute
}
if cfg.Gitlab.SecretFile == "" {
@@ -462,7 +462,7 @@ func defaultMaintenanceWindow(storages []Storage) DailyJob {
return DailyJob{
Hour: 12,
Minute: 0,
- Duration: Duration(10 * time.Minute),
+ Duration: 10 * time.Minute,
Storages: storageNames,
}
}
@@ -486,8 +486,8 @@ func (cfg *Cfg) validateMaintenance() error {
if dm.Minute > 59 {
return fmt.Errorf("daily maintenance specified minute '%d' outside range (0-59)", dm.Minute)
}
- if dm.Duration.Duration() > 24*time.Hour {
- return fmt.Errorf("daily maintenance specified duration %s must be less than 24 hours", dm.Duration.Duration())
+ if dm.Duration > 24*time.Hour {
+ return fmt.Errorf("daily maintenance specified duration %s must be less than 24 hours", dm.Duration)
}
return nil
@@ -524,7 +524,7 @@ func (cfg *Cfg) configurePackObjectsCache() error {
}
if poc.MaxAge == 0 {
- poc.MaxAge = Duration(5 * time.Minute)
+ poc.MaxAge = 5 * time.Minute
}
if poc.Dir == "" {
diff --git a/internal/gitaly/config/config_test.go b/internal/gitaly/config/config_test.go
index 47c1542e2..0bc386636 100644
--- a/internal/gitaly/config/config_test.go
+++ b/internal/gitaly/config/config_test.go
@@ -595,7 +595,7 @@ func TestLoadGracefulRestartTimeout(t *testing.T) {
cfg, err := Load(tmpFile)
assert.NoError(t, err)
- assert.Equal(t, test.expected, cfg.GracefulRestartTimeout.Duration())
+ assert.Equal(t, test.expected, cfg.GracefulRestartTimeout)
})
}
}
@@ -715,7 +715,7 @@ func TestLoadDailyMaintenance(t *testing.T) {
expect: DailyJob{
Hour: 11,
Minute: 23,
- Duration: Duration(45 * time.Minute),
+ Duration: 45 * time.Minute,
Storages: []string{"default"},
},
},
@@ -753,7 +753,7 @@ func TestLoadDailyMaintenance(t *testing.T) {
expect: DailyJob{
Hour: 0,
Minute: 59,
- Duration: Duration(24*time.Hour + time.Second),
+ Duration: 24*time.Hour + time.Second,
},
validateErr: errors.New("daily maintenance specified duration 24h0m1s must be less than 24 hours"),
},
@@ -761,7 +761,7 @@ func TestLoadDailyMaintenance(t *testing.T) {
rawCfg: `[daily_maintenance]
duration = "meow"`,
expect: DailyJob{},
- loadErr: errors.New("load toml: (2, 4): unmarshal text: time: invalid duration"),
+ loadErr: errors.New(`load toml: (2, 4): Can't convert meow(string) to time.Duration. time: invalid duration "meow"`),
},
{
rawCfg: `[daily_maintenance]
@@ -780,7 +780,7 @@ func TestLoadDailyMaintenance(t *testing.T) {
expect: DailyJob{
Hour: 12,
Minute: 0,
- Duration: Duration(10 * time.Minute),
+ Duration: 10 * time.Minute,
Storages: []string{"default"},
},
},
@@ -1127,7 +1127,7 @@ path="/foobar"
in: storageConfig + `[pack_objects_cache]
enabled = true
`,
- out: StreamCacheConfig{Enabled: true, MaxAge: Duration(5 * time.Minute), Dir: "/foobar/+gitaly/PackObjectsCache"},
+ out: StreamCacheConfig{Enabled: true, MaxAge: 5 * time.Minute, Dir: "/foobar/+gitaly/PackObjectsCache"},
},
{
desc: "enabled with custom values",
@@ -1136,7 +1136,7 @@ enabled = true
dir = "/bazqux"
max_age = "10m"
`,
- out: StreamCacheConfig{Enabled: true, MaxAge: Duration(10 * time.Minute), Dir: "/bazqux"},
+ out: StreamCacheConfig{Enabled: true, MaxAge: 10 * time.Minute, Dir: "/bazqux"},
},
{
desc: "enabled with 0 storages",
diff --git a/internal/gitaly/config/ruby.go b/internal/gitaly/config/ruby.go
index 032c28456..0392a31d7 100644
--- a/internal/gitaly/config/ruby.go
+++ b/internal/gitaly/config/ruby.go
@@ -8,52 +8,27 @@ import (
// Ruby contains setting for Ruby worker processes
type Ruby struct {
- Dir string `toml:"dir"`
- MaxRSS int `toml:"max_rss"`
- GracefulRestartTimeout Duration `toml:"graceful_restart_timeout"`
- RestartDelay Duration `toml:"restart_delay"`
- NumWorkers int `toml:"num_workers"`
- LinguistLanguagesPath string `toml:"linguist_languages_path"`
- RuggedGitConfigSearchPath string `toml:"rugged_git_config_search_path"`
-}
-
-// Duration is a trick to let our TOML library parse durations from strings.
-type Duration time.Duration
-
-//nolint: stylecheck // This is unintentionally missing documentation.
-func (d *Duration) Duration() time.Duration {
- if d != nil {
- return time.Duration(*d)
- }
- return 0
-}
-
-//nolint: stylecheck // This is unintentionally missing documentation.
-func (d *Duration) UnmarshalText(text []byte) error {
- td, err := time.ParseDuration(string(text))
- if err == nil {
- *d = Duration(td)
- }
- return err
-}
-
-//nolint: stylecheck // This is unintentionally missing documentation.
-func (d Duration) MarshalText() ([]byte, error) {
- return []byte(time.Duration(d).String()), nil
+ Dir string `toml:"dir"`
+ MaxRSS int `toml:"max_rss"`
+ GracefulRestartTimeout time.Duration `toml:"graceful_restart_timeout"`
+ RestartDelay time.Duration `toml:"restart_delay"`
+ NumWorkers int `toml:"num_workers"`
+ LinguistLanguagesPath string `toml:"linguist_languages_path"`
+ RuggedGitConfigSearchPath string `toml:"rugged_git_config_search_path"`
}
// ConfigureRuby validates the gitaly-ruby configuration and sets default values.
func (cfg *Cfg) ConfigureRuby() error {
- if cfg.Ruby.GracefulRestartTimeout.Duration() == 0 {
- cfg.Ruby.GracefulRestartTimeout = Duration(10 * time.Minute)
+ if cfg.Ruby.GracefulRestartTimeout == 0 {
+ cfg.Ruby.GracefulRestartTimeout = 10 * time.Minute
}
if cfg.Ruby.MaxRSS == 0 {
cfg.Ruby.MaxRSS = 200 * 1024 * 1024
}
- if cfg.Ruby.RestartDelay.Duration() == 0 {
- cfg.Ruby.RestartDelay = Duration(5 * time.Minute)
+ if cfg.Ruby.RestartDelay == 0 {
+ cfg.Ruby.RestartDelay = 5 * time.Minute
}
if len(cfg.Ruby.Dir) == 0 {
diff --git a/internal/gitaly/maintenance/daily.go b/internal/gitaly/maintenance/daily.go
index 1707857d5..93f00df2f 100644
--- a/internal/gitaly/maintenance/daily.go
+++ b/internal/gitaly/maintenance/daily.go
@@ -55,20 +55,20 @@ func (dw DailyWorker) StartDaily(ctx context.Context, l logrus.FieldLogger, sche
case <-ctx.Done():
return ctx.Err()
case start = <-dw.timer(nt.Sub(dw.clock())):
- l.WithField("max_duration", schedule.Duration.Duration()).
+ l.WithField("max_duration", schedule.Duration).
Info("maintenance: daily starting")
}
var jobErr error
dontpanic.Try(func() {
- ctx, cancel := context.WithTimeout(ctx, schedule.Duration.Duration())
+ ctx, cancel := context.WithTimeout(ctx, schedule.Duration)
defer cancel()
jobErr = job(ctx, l, schedule.Storages)
})
l.WithError(jobErr).
- WithField("max_duration", schedule.Duration.Duration()).
+ WithField("max_duration", schedule.Duration).
WithField("actual_duration", time.Since(start)).
Info("maintenance: daily completed")
}
diff --git a/internal/gitaly/maintenance/daily_test.go b/internal/gitaly/maintenance/daily_test.go
index 98283176a..ef43fbd54 100644
--- a/internal/gitaly/maintenance/daily_test.go
+++ b/internal/gitaly/maintenance/daily_test.go
@@ -37,7 +37,7 @@ func TestStartDaily(t *testing.T) {
errQ := make(chan error)
s := config.DailyJob{
Hour: 1,
- Duration: config.Duration(time.Hour),
+ Duration: time.Hour,
Storages: []string{"meow"},
}
ctx, cancel := context.WithCancel(testhelper.Context(t))
diff --git a/internal/gitaly/rubyserver/rubyserver.go b/internal/gitaly/rubyserver/rubyserver.go
index 76c901a84..6bda8bdae 100644
--- a/internal/gitaly/rubyserver/rubyserver.go
+++ b/internal/gitaly/rubyserver/rubyserver.go
@@ -200,8 +200,8 @@ func (s *Server) start() error {
return err
}
- restartDelay := cfg.Ruby.RestartDelay.Duration()
- gracefulRestartTimeout := cfg.Ruby.GracefulRestartTimeout.Duration()
+ restartDelay := cfg.Ruby.RestartDelay
+ gracefulRestartTimeout := cfg.Ruby.GracefulRestartTimeout
s.workers = append(s.workers, newWorker(p, socketPath, restartDelay, gracefulRestartTimeout, events, false))
}
diff --git a/internal/middleware/limithandler/concurrency_limiter.go b/internal/middleware/limithandler/concurrency_limiter.go
index e2595571b..cf552917d 100644
--- a/internal/middleware/limithandler/concurrency_limiter.go
+++ b/internal/middleware/limithandler/concurrency_limiter.go
@@ -276,7 +276,7 @@ func WithConcurrencyLimiters(cfg config.Cfg, middleware *LimiterMiddleware) {
if limit.MaxQueueWait > 0 {
limit := limit
newTickerFunc = func() helper.Ticker {
- return helper.NewTimerTicker(limit.MaxQueueWait.Duration())
+ return helper.NewTimerTicker(limit.MaxQueueWait)
}
}
diff --git a/internal/praefect/config/config.go b/internal/praefect/config/config.go
index 5ea334bc9..59a0f2be2 100644
--- a/internal/praefect/config/config.go
+++ b/internal/praefect/config/config.go
@@ -45,15 +45,15 @@ type Failover struct {
Enabled bool `toml:"enabled,omitempty"`
// ElectionStrategy is the strategy to use for electing primaries nodes.
ElectionStrategy ElectionStrategy `toml:"election_strategy,omitempty"`
- ErrorThresholdWindow config.Duration `toml:"error_threshold_window,omitempty"`
+ ErrorThresholdWindow time.Duration `toml:"error_threshold_window,omitempty"`
WriteErrorThresholdCount uint32 `toml:"write_error_threshold_count,omitempty"`
ReadErrorThresholdCount uint32 `toml:"read_error_threshold_count,omitempty"`
// BootstrapInterval allows set a time duration that would be used on startup to make initial health check.
// The default value is 1s.
- BootstrapInterval config.Duration `toml:"bootstrap_interval,omitempty"`
+ BootstrapInterval time.Duration `toml:"bootstrap_interval,omitempty"`
// MonitorInterval allows set a time duration that would be used after bootstrap is completed to execute health checks.
// The default value is 3s.
- MonitorInterval config.Duration `toml:"monitor_interval,omitempty"`
+ MonitorInterval time.Duration `toml:"monitor_interval,omitempty"`
}
// ErrorThresholdsConfigured checks whether returns whether the errors thresholds are configured. If they
@@ -97,7 +97,7 @@ func DefaultBackgroundVerificationConfig() BackgroundVerification {
type Reconciliation struct {
// SchedulingInterval the interval between each automatic reconciliation run. If set to 0,
// automatic reconciliation is disabled.
- SchedulingInterval config.Duration `toml:"scheduling_interval,omitempty"`
+ SchedulingInterval time.Duration `toml:"scheduling_interval,omitempty"`
// HistogramBuckets configures the reconciliation scheduling duration histogram's buckets.
HistogramBuckets []float64 `toml:"histogram_buckets,omitempty"`
}
@@ -105,7 +105,7 @@ type Reconciliation struct {
// DefaultReconciliationConfig returns the default values for reconciliation configuration.
func DefaultReconciliationConfig() Reconciliation {
return Reconciliation{
- SchedulingInterval: 5 * config.Duration(time.Minute),
+ SchedulingInterval: 5 * time.Minute,
HistogramBuckets: promclient.DefBuckets,
}
}
@@ -152,7 +152,7 @@ type Config struct {
// Keep for legacy reasons: remove after Omnibus has switched
FailoverEnabled bool `toml:"failover_enabled,omitempty"`
MemoryQueueEnabled bool `toml:"memory_queue_enabled,omitempty"`
- GracefulStopTimeout config.Duration `toml:"graceful_stop_timeout,omitempty"`
+ GracefulStopTimeout time.Duration `toml:"graceful_stop_timeout,omitempty"`
RepositoriesCleanup RepositoriesCleanup `toml:"repositories_cleanup,omitempty"`
}
@@ -276,11 +276,11 @@ func (c *Config) Validate() error {
}
}
- if c.RepositoriesCleanup.RunInterval.Duration() > 0 {
- if c.RepositoriesCleanup.CheckInterval.Duration() < minimalSyncCheckInterval {
+ if c.RepositoriesCleanup.RunInterval > 0 {
+ if c.RepositoriesCleanup.CheckInterval < minimalSyncCheckInterval {
return fmt.Errorf("repositories_cleanup.check_interval is less then %s, which could lead to a database performance problem", minimalSyncCheckInterval.String())
}
- if c.RepositoriesCleanup.RunInterval.Duration() < minimalSyncRunInterval {
+ if c.RepositoriesCleanup.RunInterval < minimalSyncRunInterval {
return fmt.Errorf("repositories_cleanup.run_interval is less then %s, which could lead to a database performance problem", minimalSyncRunInterval.String())
}
}
@@ -294,17 +294,17 @@ func (c *Config) NeedsSQL() bool {
}
func (c *Config) setDefaults() {
- if c.GracefulStopTimeout.Duration() == 0 {
- c.GracefulStopTimeout = config.Duration(time.Minute)
+ if c.GracefulStopTimeout == 0 {
+ c.GracefulStopTimeout = time.Minute
}
if c.Failover.Enabled {
- if c.Failover.BootstrapInterval.Duration() == 0 {
- c.Failover.BootstrapInterval = config.Duration(time.Second)
+ if c.Failover.BootstrapInterval == 0 {
+ c.Failover.BootstrapInterval = time.Second
}
- if c.Failover.MonitorInterval.Duration() == 0 {
- c.Failover.MonitorInterval = config.Duration(3 * time.Second)
+ if c.Failover.MonitorInterval == 0 {
+ c.Failover.MonitorInterval = 3 * time.Second
}
}
}
@@ -383,9 +383,9 @@ type RepositoriesCleanup struct {
// CheckInterval is a time period used to check if operation should be executed.
// It is recommended to keep it less than run_interval configuration as some
// nodes may be out of service, so they can be stale for too long.
- CheckInterval config.Duration `toml:"check_interval,omitempty"`
+ CheckInterval time.Duration `toml:"check_interval,omitempty"`
// RunInterval: the check runs if the previous operation was done at least RunInterval before.
- RunInterval config.Duration `toml:"run_interval,omitempty"`
+ RunInterval time.Duration `toml:"run_interval,omitempty"`
// RepositoriesInBatch is the number of repositories to pass as a batch for processing.
RepositoriesInBatch int `toml:"repositories_in_batch,omitempty"`
}
@@ -393,8 +393,8 @@ type RepositoriesCleanup struct {
// DefaultRepositoriesCleanup contains default configuration values for the RepositoriesCleanup.
func DefaultRepositoriesCleanup() RepositoriesCleanup {
return RepositoriesCleanup{
- CheckInterval: config.Duration(30 * time.Minute),
- RunInterval: config.Duration(24 * time.Hour),
+ CheckInterval: 30 * time.Minute,
+ RunInterval: 24 * time.Hour,
RepositoriesInBatch: 16,
}
}
diff --git a/internal/praefect/config/config_test.go b/internal/praefect/config/config_test.go
index 8e6585f94..410599d33 100644
--- a/internal/praefect/config/config_test.go
+++ b/internal/praefect/config/config_test.go
@@ -205,14 +205,14 @@ func TestConfigValidation(t *testing.T) {
{
desc: "repositories_cleanup minimal duration is too low",
changeConfig: func(cfg *Config) {
- cfg.RepositoriesCleanup.CheckInterval = config.Duration(minimalSyncCheckInterval - time.Nanosecond)
+ cfg.RepositoriesCleanup.CheckInterval = minimalSyncCheckInterval - time.Nanosecond
},
errMsg: `repositories_cleanup.check_interval is less then 1m0s, which could lead to a database performance problem`,
},
{
desc: "repositories_cleanup minimal duration is too low",
changeConfig: func(cfg *Config) {
- cfg.RepositoriesCleanup.RunInterval = config.Duration(minimalSyncRunInterval - time.Nanosecond)
+ cfg.RepositoriesCleanup.RunInterval = minimalSyncRunInterval - time.Nanosecond
},
errMsg: `repositories_cleanup.run_interval is less then 1m0s, which could lead to a database performance problem`,
},
@@ -317,24 +317,24 @@ func TestConfigParsing(t *testing.T) {
},
},
MemoryQueueEnabled: true,
- GracefulStopTimeout: config.Duration(30 * time.Second),
+ GracefulStopTimeout: 30 * time.Second,
Reconciliation: Reconciliation{
- SchedulingInterval: config.Duration(time.Minute),
+ SchedulingInterval: time.Minute,
HistogramBuckets: []float64{1, 2, 3, 4, 5},
},
Replication: Replication{BatchSize: 1, ParallelStorageProcessingWorkers: 2},
Failover: Failover{
Enabled: true,
ElectionStrategy: ElectionStrategyPerRepository,
- ErrorThresholdWindow: config.Duration(20 * time.Second),
+ ErrorThresholdWindow: 20 * time.Second,
WriteErrorThresholdCount: 1500,
ReadErrorThresholdCount: 100,
- BootstrapInterval: config.Duration(1 * time.Second),
- MonitorInterval: config.Duration(3 * time.Second),
+ BootstrapInterval: 1 * time.Second,
+ MonitorInterval: 3 * time.Second,
},
RepositoriesCleanup: RepositoriesCleanup{
- CheckInterval: config.Duration(time.Second),
- RunInterval: config.Duration(3 * time.Second),
+ CheckInterval: time.Second,
+ RunInterval: 3 * time.Second,
RepositoriesInBatch: 10,
},
BackgroundVerification: BackgroundVerification{
@@ -347,7 +347,7 @@ func TestConfigParsing(t *testing.T) {
desc: "overwriting default values in the config",
filePath: "testdata/config.overwritedefaults.toml",
expected: Config{
- GracefulStopTimeout: config.Duration(time.Minute),
+ GracefulStopTimeout: time.Minute,
Reconciliation: Reconciliation{
SchedulingInterval: 0,
HistogramBuckets: []float64{1, 2, 3, 4, 5},
@@ -358,12 +358,12 @@ func TestConfigParsing(t *testing.T) {
Failover: Failover{
Enabled: false,
ElectionStrategy: "local",
- BootstrapInterval: config.Duration(5 * time.Second),
- MonitorInterval: config.Duration(10 * time.Second),
+ BootstrapInterval: 5 * time.Second,
+ MonitorInterval: 10 * time.Second,
},
RepositoriesCleanup: RepositoriesCleanup{
- CheckInterval: config.Duration(time.Second),
- RunInterval: config.Duration(4 * time.Second),
+ CheckInterval: time.Second,
+ RunInterval: 4 * time.Second,
RepositoriesInBatch: 11,
},
BackgroundVerification: DefaultBackgroundVerificationConfig(),
@@ -373,7 +373,7 @@ func TestConfigParsing(t *testing.T) {
desc: "empty config yields default values",
filePath: "testdata/config.empty.toml",
expected: Config{
- GracefulStopTimeout: config.Duration(time.Minute),
+ GracefulStopTimeout: time.Minute,
Prometheus: prometheus.DefaultConfig(),
PrometheusExcludeDatabaseFromDefaultMetrics: true,
Reconciliation: DefaultReconciliationConfig(),
@@ -381,12 +381,12 @@ func TestConfigParsing(t *testing.T) {
Failover: Failover{
Enabled: true,
ElectionStrategy: ElectionStrategyPerRepository,
- BootstrapInterval: config.Duration(time.Second),
- MonitorInterval: config.Duration(3 * time.Second),
+ BootstrapInterval: time.Second,
+ MonitorInterval: 3 * time.Second,
},
RepositoriesCleanup: RepositoriesCleanup{
- CheckInterval: config.Duration(30 * time.Minute),
- RunInterval: config.Duration(24 * time.Hour),
+ CheckInterval: 30 * time.Minute,
+ RunInterval: 24 * time.Hour,
RepositoriesInBatch: 16,
},
BackgroundVerification: DefaultBackgroundVerificationConfig(),
diff --git a/internal/praefect/nodes/tracker/errors.go b/internal/praefect/nodes/tracker/errors.go
index 8a87d30cc..1fce839bf 100644
--- a/internal/praefect/nodes/tracker/errors.go
+++ b/internal/praefect/nodes/tracker/errors.go
@@ -32,7 +32,7 @@ func NewErrorWindowFunction(cfg config.Failover) (ErrorWindowFunction, error) {
return nil, errors.New("errorWindow must be non zero")
}
- errorWindow := cfg.ErrorThresholdWindow.Duration()
+ errorWindow := cfg.ErrorThresholdWindow
return func(now time.Time, errorTime time.Time) bool {
return errorTime.After(now.Add(-errorWindow))
}, nil
diff --git a/internal/streamcache/cache.go b/internal/streamcache/cache.go
index 333be63c7..53c3c6c44 100644
--- a/internal/streamcache/cache.go
+++ b/internal/streamcache/cache.go
@@ -145,10 +145,10 @@ func New(cfg config.StreamCacheConfig, logger logrus.FieldLogger) Cache {
if cfg.Enabled {
packObjectsCacheEnabled.WithLabelValues(
cfg.Dir,
- strconv.Itoa(int(cfg.MaxAge.Duration().Seconds())),
+ strconv.Itoa(int(cfg.MaxAge.Seconds())),
).Set(1)
- return newCacheWithSleep(cfg.Dir, cfg.MaxAge.Duration(), time.After, time.After, logger)
+ return newCacheWithSleep(cfg.Dir, cfg.MaxAge, time.After, time.After, logger)
}
return NullCache{}
diff --git a/internal/streamcache/cache_test.go b/internal/streamcache/cache_test.go
index 650b87fa2..01c71f6a6 100644
--- a/internal/streamcache/cache_test.go
+++ b/internal/streamcache/cache_test.go
@@ -26,7 +26,7 @@ func newCache(dir string) Cache {
return New(config.StreamCacheConfig{
Enabled: true,
Dir: dir,
- MaxAge: config.Duration(time.Hour),
+ MaxAge: time.Hour,
}, log.Default())
}