mirror of
https://github.com/minio/minio.git
synced 2025-01-11 23:13:23 -05:00
replication: make large workers configurable (#20077)
This PR also improves throttling by reducing tokens requested from rate limiter based on available tokens to avoid exceeding throttle wait deadlines
This commit is contained in:
parent
ef802f2b2c
commit
989c318a28
@ -1803,15 +1803,18 @@ var (
|
|||||||
type ReplicationPool struct {
|
type ReplicationPool struct {
|
||||||
// atomic ops:
|
// atomic ops:
|
||||||
activeWorkers int32
|
activeWorkers int32
|
||||||
|
activeLrgWorkers int32
|
||||||
activeMRFWorkers int32
|
activeMRFWorkers int32
|
||||||
|
|
||||||
objLayer ObjectLayer
|
objLayer ObjectLayer
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
priority string
|
priority string
|
||||||
maxWorkers int
|
maxWorkers int
|
||||||
mu sync.RWMutex
|
maxLWorkers int
|
||||||
mrfMU sync.Mutex
|
|
||||||
resyncer *replicationResyncer
|
mu sync.RWMutex
|
||||||
|
mrfMU sync.Mutex
|
||||||
|
resyncer *replicationResyncer
|
||||||
|
|
||||||
// workers:
|
// workers:
|
||||||
workers []chan ReplicationWorkerOperation
|
workers []chan ReplicationWorkerOperation
|
||||||
@ -1882,9 +1885,13 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
|
|||||||
if maxWorkers > 0 && failedWorkers > maxWorkers {
|
if maxWorkers > 0 && failedWorkers > maxWorkers {
|
||||||
failedWorkers = maxWorkers
|
failedWorkers = maxWorkers
|
||||||
}
|
}
|
||||||
|
maxLWorkers := LargeWorkerCount
|
||||||
|
if opts.MaxLWorkers > 0 {
|
||||||
|
maxLWorkers = opts.MaxLWorkers
|
||||||
|
}
|
||||||
pool := &ReplicationPool{
|
pool := &ReplicationPool{
|
||||||
workers: make([]chan ReplicationWorkerOperation, 0, workers),
|
workers: make([]chan ReplicationWorkerOperation, 0, workers),
|
||||||
lrgworkers: make([]chan ReplicationWorkerOperation, 0, LargeWorkerCount),
|
lrgworkers: make([]chan ReplicationWorkerOperation, 0, maxLWorkers),
|
||||||
mrfReplicaCh: make(chan ReplicationWorkerOperation, 100000),
|
mrfReplicaCh: make(chan ReplicationWorkerOperation, 100000),
|
||||||
mrfWorkerKillCh: make(chan struct{}, failedWorkers),
|
mrfWorkerKillCh: make(chan struct{}, failedWorkers),
|
||||||
resyncer: newresyncer(),
|
resyncer: newresyncer(),
|
||||||
@ -1894,9 +1901,10 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
|
|||||||
objLayer: o,
|
objLayer: o,
|
||||||
priority: priority,
|
priority: priority,
|
||||||
maxWorkers: maxWorkers,
|
maxWorkers: maxWorkers,
|
||||||
|
maxLWorkers: maxLWorkers,
|
||||||
}
|
}
|
||||||
|
|
||||||
pool.AddLargeWorkers()
|
pool.ResizeLrgWorkers(maxLWorkers, 0)
|
||||||
pool.ResizeWorkers(workers, 0)
|
pool.ResizeWorkers(workers, 0)
|
||||||
pool.ResizeFailedWorkers(failedWorkers)
|
pool.ResizeFailedWorkers(failedWorkers)
|
||||||
go pool.resyncer.PersistToDisk(ctx, o)
|
go pool.resyncer.PersistToDisk(ctx, o)
|
||||||
@ -1975,23 +1983,8 @@ func (p *ReplicationPool) AddWorker(input <-chan ReplicationWorkerOperation, opT
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddLargeWorkers adds a static number of workers to handle large uploads
|
|
||||||
func (p *ReplicationPool) AddLargeWorkers() {
|
|
||||||
for i := 0; i < LargeWorkerCount; i++ {
|
|
||||||
p.lrgworkers = append(p.lrgworkers, make(chan ReplicationWorkerOperation, 100000))
|
|
||||||
i := i
|
|
||||||
go p.AddLargeWorker(p.lrgworkers[i])
|
|
||||||
}
|
|
||||||
go func() {
|
|
||||||
<-p.ctx.Done()
|
|
||||||
for i := 0; i < LargeWorkerCount; i++ {
|
|
||||||
xioutil.SafeClose(p.lrgworkers[i])
|
|
||||||
}
|
|
||||||
}()
|
|
||||||
}
|
|
||||||
|
|
||||||
// AddLargeWorker adds a replication worker to the static pool for large uploads.
|
// AddLargeWorker adds a replication worker to the static pool for large uploads.
|
||||||
func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation) {
|
func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation, opTracker *int32) {
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-p.ctx.Done():
|
case <-p.ctx.Done():
|
||||||
@ -2002,11 +1995,23 @@ func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation
|
|||||||
}
|
}
|
||||||
switch v := oi.(type) {
|
switch v := oi.(type) {
|
||||||
case ReplicateObjectInfo:
|
case ReplicateObjectInfo:
|
||||||
|
if opTracker != nil {
|
||||||
|
atomic.AddInt32(opTracker, 1)
|
||||||
|
}
|
||||||
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
|
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
|
||||||
replicateObject(p.ctx, v, p.objLayer)
|
replicateObject(p.ctx, v, p.objLayer)
|
||||||
globalReplicationStats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
|
globalReplicationStats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
|
||||||
|
if opTracker != nil {
|
||||||
|
atomic.AddInt32(opTracker, -1)
|
||||||
|
}
|
||||||
case DeletedObjectReplicationInfo:
|
case DeletedObjectReplicationInfo:
|
||||||
|
if opTracker != nil {
|
||||||
|
atomic.AddInt32(opTracker, 1)
|
||||||
|
}
|
||||||
replicateDelete(p.ctx, v, p.objLayer)
|
replicateDelete(p.ctx, v, p.objLayer)
|
||||||
|
if opTracker != nil {
|
||||||
|
atomic.AddInt32(opTracker, -1)
|
||||||
|
}
|
||||||
default:
|
default:
|
||||||
bugLogIf(p.ctx, fmt.Errorf("unknown replication type: %T", oi), "unknown-replicate-type")
|
bugLogIf(p.ctx, fmt.Errorf("unknown replication type: %T", oi), "unknown-replicate-type")
|
||||||
}
|
}
|
||||||
@ -2014,6 +2019,30 @@ func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ResizeLrgWorkers sets replication workers pool for large transfers(>=128MiB) to new size.
|
||||||
|
// checkOld can be set to an expected value.
|
||||||
|
// If the worker count changed
|
||||||
|
func (p *ReplicationPool) ResizeLrgWorkers(n, checkOld int) {
|
||||||
|
p.mu.Lock()
|
||||||
|
defer p.mu.Unlock()
|
||||||
|
|
||||||
|
if (checkOld > 0 && len(p.lrgworkers) != checkOld) || n == len(p.lrgworkers) || n < 1 {
|
||||||
|
// Either already satisfied or worker count changed while we waited for the lock.
|
||||||
|
return
|
||||||
|
}
|
||||||
|
for len(p.lrgworkers) < n {
|
||||||
|
input := make(chan ReplicationWorkerOperation, 100000)
|
||||||
|
p.lrgworkers = append(p.lrgworkers, input)
|
||||||
|
|
||||||
|
go p.AddLargeWorker(input, &p.activeLrgWorkers)
|
||||||
|
}
|
||||||
|
for len(p.lrgworkers) > n {
|
||||||
|
worker := p.lrgworkers[len(p.lrgworkers)-1]
|
||||||
|
p.lrgworkers = p.lrgworkers[:len(p.lrgworkers)-1]
|
||||||
|
xioutil.SafeClose(worker)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// ActiveWorkers returns the number of active workers handling replication traffic.
|
// ActiveWorkers returns the number of active workers handling replication traffic.
|
||||||
func (p *ReplicationPool) ActiveWorkers() int {
|
func (p *ReplicationPool) ActiveWorkers() int {
|
||||||
return int(atomic.LoadInt32(&p.activeWorkers))
|
return int(atomic.LoadInt32(&p.activeWorkers))
|
||||||
@ -2024,6 +2053,11 @@ func (p *ReplicationPool) ActiveMRFWorkers() int {
|
|||||||
return int(atomic.LoadInt32(&p.activeMRFWorkers))
|
return int(atomic.LoadInt32(&p.activeMRFWorkers))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ActiveLrgWorkers returns the number of active workers handling traffic > 128MiB object size.
|
||||||
|
func (p *ReplicationPool) ActiveLrgWorkers() int {
|
||||||
|
return int(atomic.LoadInt32(&p.activeLrgWorkers))
|
||||||
|
}
|
||||||
|
|
||||||
// ResizeWorkers sets replication workers pool to new size.
|
// ResizeWorkers sets replication workers pool to new size.
|
||||||
// checkOld can be set to an expected value.
|
// checkOld can be set to an expected value.
|
||||||
// If the worker count changed
|
// If the worker count changed
|
||||||
@ -2049,7 +2083,7 @@ func (p *ReplicationPool) ResizeWorkers(n, checkOld int) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// ResizeWorkerPriority sets replication failed workers pool size
|
// ResizeWorkerPriority sets replication failed workers pool size
|
||||||
func (p *ReplicationPool) ResizeWorkerPriority(pri string, maxWorkers int) {
|
func (p *ReplicationPool) ResizeWorkerPriority(pri string, maxWorkers, maxLWorkers int) {
|
||||||
var workers, mrfWorkers int
|
var workers, mrfWorkers int
|
||||||
p.mu.Lock()
|
p.mu.Lock()
|
||||||
switch pri {
|
switch pri {
|
||||||
@ -2076,11 +2110,15 @@ func (p *ReplicationPool) ResizeWorkerPriority(pri string, maxWorkers int) {
|
|||||||
if maxWorkers > 0 && mrfWorkers > maxWorkers {
|
if maxWorkers > 0 && mrfWorkers > maxWorkers {
|
||||||
mrfWorkers = maxWorkers
|
mrfWorkers = maxWorkers
|
||||||
}
|
}
|
||||||
|
if maxLWorkers <= 0 {
|
||||||
|
maxLWorkers = LargeWorkerCount
|
||||||
|
}
|
||||||
p.priority = pri
|
p.priority = pri
|
||||||
p.maxWorkers = maxWorkers
|
p.maxWorkers = maxWorkers
|
||||||
p.mu.Unlock()
|
p.mu.Unlock()
|
||||||
p.ResizeWorkers(workers, 0)
|
p.ResizeWorkers(workers, 0)
|
||||||
p.ResizeFailedWorkers(mrfWorkers)
|
p.ResizeFailedWorkers(mrfWorkers)
|
||||||
|
p.ResizeLrgWorkers(maxLWorkers, 0)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ResizeFailedWorkers sets replication failed workers pool size
|
// ResizeFailedWorkers sets replication failed workers pool size
|
||||||
@ -2127,6 +2165,15 @@ func (p *ReplicationPool) queueReplicaTask(ri ReplicateObjectInfo) {
|
|||||||
case p.lrgworkers[h%LargeWorkerCount] <- ri:
|
case p.lrgworkers[h%LargeWorkerCount] <- ri:
|
||||||
default:
|
default:
|
||||||
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
|
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
|
||||||
|
p.mu.RLock()
|
||||||
|
maxLWorkers := p.maxLWorkers
|
||||||
|
existing := len(p.lrgworkers)
|
||||||
|
p.mu.RUnlock()
|
||||||
|
maxLWorkers = min(maxLWorkers, LargeWorkerCount)
|
||||||
|
if p.ActiveLrgWorkers() < maxLWorkers {
|
||||||
|
workers := min(existing+1, maxLWorkers)
|
||||||
|
p.ResizeLrgWorkers(workers, existing)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -2229,8 +2276,9 @@ func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectReplicationInf
|
|||||||
}
|
}
|
||||||
|
|
||||||
type replicationPoolOpts struct {
|
type replicationPoolOpts struct {
|
||||||
Priority string
|
Priority string
|
||||||
MaxWorkers int
|
MaxWorkers int
|
||||||
|
MaxLWorkers int
|
||||||
}
|
}
|
||||||
|
|
||||||
func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
|
func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
|
||||||
|
@ -39,14 +39,15 @@ import (
|
|||||||
type apiConfig struct {
|
type apiConfig struct {
|
||||||
mu sync.RWMutex
|
mu sync.RWMutex
|
||||||
|
|
||||||
requestsDeadline time.Duration
|
requestsDeadline time.Duration
|
||||||
requestsPool chan struct{}
|
requestsPool chan struct{}
|
||||||
clusterDeadline time.Duration
|
clusterDeadline time.Duration
|
||||||
listQuorum string
|
listQuorum string
|
||||||
corsAllowOrigins []string
|
corsAllowOrigins []string
|
||||||
replicationPriority string
|
replicationPriority string
|
||||||
replicationMaxWorkers int
|
replicationMaxWorkers int
|
||||||
transitionWorkers int
|
replicationMaxLWorkers int
|
||||||
|
transitionWorkers int
|
||||||
|
|
||||||
staleUploadsExpiry time.Duration
|
staleUploadsExpiry time.Duration
|
||||||
staleUploadsCleanupInterval time.Duration
|
staleUploadsCleanupInterval time.Duration
|
||||||
@ -170,11 +171,12 @@ func (t *apiConfig) init(cfg api.Config, setDriveCounts []int, legacy bool) {
|
|||||||
}
|
}
|
||||||
t.listQuorum = listQuorum
|
t.listQuorum = listQuorum
|
||||||
if globalReplicationPool != nil &&
|
if globalReplicationPool != nil &&
|
||||||
(cfg.ReplicationPriority != t.replicationPriority || cfg.ReplicationMaxWorkers != t.replicationMaxWorkers) {
|
(cfg.ReplicationPriority != t.replicationPriority || cfg.ReplicationMaxWorkers != t.replicationMaxWorkers || cfg.ReplicationMaxLWorkers != t.replicationMaxLWorkers) {
|
||||||
globalReplicationPool.ResizeWorkerPriority(cfg.ReplicationPriority, cfg.ReplicationMaxWorkers)
|
globalReplicationPool.ResizeWorkerPriority(cfg.ReplicationPriority, cfg.ReplicationMaxWorkers, cfg.ReplicationMaxLWorkers)
|
||||||
}
|
}
|
||||||
t.replicationPriority = cfg.ReplicationPriority
|
t.replicationPriority = cfg.ReplicationPriority
|
||||||
t.replicationMaxWorkers = cfg.ReplicationMaxWorkers
|
t.replicationMaxWorkers = cfg.ReplicationMaxWorkers
|
||||||
|
t.replicationMaxLWorkers = cfg.ReplicationMaxLWorkers
|
||||||
|
|
||||||
// N B api.transition_workers will be deprecated
|
// N B api.transition_workers will be deprecated
|
||||||
if globalTransitionState != nil {
|
if globalTransitionState != nil {
|
||||||
@ -381,14 +383,16 @@ func (t *apiConfig) getReplicationOpts() replicationPoolOpts {
|
|||||||
|
|
||||||
if t.replicationPriority == "" {
|
if t.replicationPriority == "" {
|
||||||
return replicationPoolOpts{
|
return replicationPoolOpts{
|
||||||
Priority: "auto",
|
Priority: "auto",
|
||||||
MaxWorkers: WorkerMaxLimit,
|
MaxWorkers: WorkerMaxLimit,
|
||||||
|
MaxLWorkers: LargeWorkerCount,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return replicationPoolOpts{
|
return replicationPoolOpts{
|
||||||
Priority: t.replicationPriority,
|
Priority: t.replicationPriority,
|
||||||
MaxWorkers: t.replicationMaxWorkers,
|
MaxWorkers: t.replicationMaxWorkers,
|
||||||
|
MaxLWorkers: t.replicationMaxLWorkers,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -74,12 +74,16 @@ func (r *MonitoredReader) Read(buf []byte) (n int, err error) {
|
|||||||
need = int(math.Min(float64(b), float64(need)))
|
need = int(math.Min(float64(b), float64(need)))
|
||||||
tokens = need
|
tokens = need
|
||||||
}
|
}
|
||||||
|
// reduce tokens requested according to availability
|
||||||
|
av := int(r.throttle.Tokens())
|
||||||
|
if av < tokens && av > 0 {
|
||||||
|
tokens = av
|
||||||
|
need = int(math.Min(float64(tokens), float64(need)))
|
||||||
|
}
|
||||||
err = r.throttle.WaitN(r.ctx, tokens)
|
err = r.throttle.WaitN(r.ctx, tokens)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
n, err = r.r.Read(buf[:need])
|
n, err = r.r.Read(buf[:need])
|
||||||
if err != nil {
|
if err != nil {
|
||||||
r.lastErr = err
|
r.lastErr = err
|
||||||
|
@ -40,6 +40,7 @@ const (
|
|||||||
apiListQuorum = "list_quorum"
|
apiListQuorum = "list_quorum"
|
||||||
apiReplicationPriority = "replication_priority"
|
apiReplicationPriority = "replication_priority"
|
||||||
apiReplicationMaxWorkers = "replication_max_workers"
|
apiReplicationMaxWorkers = "replication_max_workers"
|
||||||
|
apiReplicationMaxLWorkers = "replication_max_lrg_workers"
|
||||||
|
|
||||||
apiTransitionWorkers = "transition_workers"
|
apiTransitionWorkers = "transition_workers"
|
||||||
apiStaleUploadsCleanupInterval = "stale_uploads_cleanup_interval"
|
apiStaleUploadsCleanupInterval = "stale_uploads_cleanup_interval"
|
||||||
@ -52,16 +53,18 @@ const (
|
|||||||
apiSyncEvents = "sync_events"
|
apiSyncEvents = "sync_events"
|
||||||
apiObjectMaxVersions = "object_max_versions"
|
apiObjectMaxVersions = "object_max_versions"
|
||||||
|
|
||||||
EnvAPIRequestsMax = "MINIO_API_REQUESTS_MAX"
|
EnvAPIRequestsMax = "MINIO_API_REQUESTS_MAX"
|
||||||
EnvAPIRequestsDeadline = "MINIO_API_REQUESTS_DEADLINE"
|
EnvAPIRequestsDeadline = "MINIO_API_REQUESTS_DEADLINE"
|
||||||
EnvAPIClusterDeadline = "MINIO_API_CLUSTER_DEADLINE"
|
EnvAPIClusterDeadline = "MINIO_API_CLUSTER_DEADLINE"
|
||||||
EnvAPICorsAllowOrigin = "MINIO_API_CORS_ALLOW_ORIGIN"
|
EnvAPICorsAllowOrigin = "MINIO_API_CORS_ALLOW_ORIGIN"
|
||||||
EnvAPIRemoteTransportDeadline = "MINIO_API_REMOTE_TRANSPORT_DEADLINE"
|
EnvAPIRemoteTransportDeadline = "MINIO_API_REMOTE_TRANSPORT_DEADLINE"
|
||||||
EnvAPITransitionWorkers = "MINIO_API_TRANSITION_WORKERS"
|
EnvAPITransitionWorkers = "MINIO_API_TRANSITION_WORKERS"
|
||||||
EnvAPIListQuorum = "MINIO_API_LIST_QUORUM"
|
EnvAPIListQuorum = "MINIO_API_LIST_QUORUM"
|
||||||
EnvAPISecureCiphers = "MINIO_API_SECURE_CIPHERS" // default config.EnableOn
|
EnvAPISecureCiphers = "MINIO_API_SECURE_CIPHERS" // default config.EnableOn
|
||||||
EnvAPIReplicationPriority = "MINIO_API_REPLICATION_PRIORITY"
|
EnvAPIReplicationPriority = "MINIO_API_REPLICATION_PRIORITY"
|
||||||
EnvAPIReplicationMaxWorkers = "MINIO_API_REPLICATION_MAX_WORKERS"
|
EnvAPIReplicationMaxWorkers = "MINIO_API_REPLICATION_MAX_WORKERS"
|
||||||
|
EnvAPIReplicationMaxLWorkers = "MINIO_API_REPLICATION_MAX_LRG_WORKERS"
|
||||||
|
|
||||||
EnvAPIStaleUploadsCleanupInterval = "MINIO_API_STALE_UPLOADS_CLEANUP_INTERVAL"
|
EnvAPIStaleUploadsCleanupInterval = "MINIO_API_STALE_UPLOADS_CLEANUP_INTERVAL"
|
||||||
EnvAPIStaleUploadsExpiry = "MINIO_API_STALE_UPLOADS_EXPIRY"
|
EnvAPIStaleUploadsExpiry = "MINIO_API_STALE_UPLOADS_EXPIRY"
|
||||||
EnvAPIDeleteCleanupInterval = "MINIO_API_DELETE_CLEANUP_INTERVAL"
|
EnvAPIDeleteCleanupInterval = "MINIO_API_DELETE_CLEANUP_INTERVAL"
|
||||||
@ -117,6 +120,10 @@ var (
|
|||||||
Key: apiReplicationMaxWorkers,
|
Key: apiReplicationMaxWorkers,
|
||||||
Value: "500",
|
Value: "500",
|
||||||
},
|
},
|
||||||
|
config.KV{
|
||||||
|
Key: apiReplicationMaxLWorkers,
|
||||||
|
Value: "10",
|
||||||
|
},
|
||||||
config.KV{
|
config.KV{
|
||||||
Key: apiTransitionWorkers,
|
Key: apiTransitionWorkers,
|
||||||
Value: "100",
|
Value: "100",
|
||||||
@ -171,6 +178,7 @@ type Config struct {
|
|||||||
ListQuorum string `json:"list_quorum"`
|
ListQuorum string `json:"list_quorum"`
|
||||||
ReplicationPriority string `json:"replication_priority"`
|
ReplicationPriority string `json:"replication_priority"`
|
||||||
ReplicationMaxWorkers int `json:"replication_max_workers"`
|
ReplicationMaxWorkers int `json:"replication_max_workers"`
|
||||||
|
ReplicationMaxLWorkers int `json:"replication_max_lrg_workers"`
|
||||||
TransitionWorkers int `json:"transition_workers"`
|
TransitionWorkers int `json:"transition_workers"`
|
||||||
StaleUploadsCleanupInterval time.Duration `json:"stale_uploads_cleanup_interval"`
|
StaleUploadsCleanupInterval time.Duration `json:"stale_uploads_cleanup_interval"`
|
||||||
StaleUploadsExpiry time.Duration `json:"stale_uploads_expiry"`
|
StaleUploadsExpiry time.Duration `json:"stale_uploads_expiry"`
|
||||||
@ -280,11 +288,21 @@ func LookupConfig(kvs config.KVS) (cfg Config, err error) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return cfg, err
|
return cfg, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if replicationMaxWorkers <= 0 || replicationMaxWorkers > 500 {
|
if replicationMaxWorkers <= 0 || replicationMaxWorkers > 500 {
|
||||||
return cfg, config.ErrInvalidReplicationWorkersValue(nil).Msg("Number of replication workers should be between 1 and 500")
|
return cfg, config.ErrInvalidReplicationWorkersValue(nil).Msg("Number of replication workers should be between 1 and 500")
|
||||||
}
|
}
|
||||||
cfg.ReplicationMaxWorkers = replicationMaxWorkers
|
cfg.ReplicationMaxWorkers = replicationMaxWorkers
|
||||||
|
|
||||||
|
replicationMaxLWorkers, err := strconv.Atoi(env.Get(EnvAPIReplicationMaxLWorkers, kvs.GetWithDefault(apiReplicationMaxLWorkers, DefaultKVS)))
|
||||||
|
if err != nil {
|
||||||
|
return cfg, err
|
||||||
|
}
|
||||||
|
if replicationMaxLWorkers <= 0 || replicationMaxLWorkers > 10 {
|
||||||
|
return cfg, config.ErrInvalidReplicationWorkersValue(nil).Msg("Number of replication workers for transfers >=128MiB should be between 1 and 10 per node")
|
||||||
|
}
|
||||||
|
|
||||||
|
cfg.ReplicationMaxLWorkers = replicationMaxLWorkers
|
||||||
|
|
||||||
transitionWorkers, err := strconv.Atoi(env.Get(EnvAPITransitionWorkers, kvs.GetWithDefault(apiTransitionWorkers, DefaultKVS)))
|
transitionWorkers, err := strconv.Atoi(env.Get(EnvAPITransitionWorkers, kvs.GetWithDefault(apiTransitionWorkers, DefaultKVS)))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return cfg, err
|
return cfg, err
|
||||||
|
Loading…
Reference in New Issue
Block a user