Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 0 additions & 6 deletions internal/common/autoscaler/autoscaler.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,6 @@ package autoscaler
type (
AutoScaler interface {
Estimator
// Acquire X ResourceUnit of resource
Acquire(ResourceUnit) error
// Release X ResourceUnit of resource
Release(ResourceUnit)
// GetCurrent ResourceUnit of resource
GetCurrent() ResourceUnit
// Start starts the autoscaler go routine that scales the ResourceUnit according to Estimator
Start()
// Stop stops the autoscaler if started or do nothing if not yet started
Expand Down
31 changes: 6 additions & 25 deletions internal/internal_poller_autoscaler.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@ import (
"sync"
"time"

"github.com/marusama/semaphore/v2"
"go.uber.org/atomic"
"go.uber.org/zap"

"go.uber.org/cadence/internal/common/autoscaler"
"go.uber.org/cadence/internal/worker"
)

// defaultPollerScalerCooldownInSeconds
Expand All @@ -53,7 +53,7 @@ type (
isDryRun bool
cooldownTime time.Duration
logger *zap.Logger
sem semaphore.Semaphore // resizable semaphore to control number of concurrent pollers
permit worker.Permit
ctx context.Context
cancel context.CancelFunc
wg *sync.WaitGroup // graceful stop
Expand Down Expand Up @@ -82,6 +82,7 @@ type (
func newPollerScaler(
options pollerAutoScalerOptions,
logger *zap.Logger,
permit worker.Permit,
hooks ...func()) *pollerAutoScaler {
if !options.Enabled {
return nil
Expand All @@ -91,7 +92,7 @@ func newPollerScaler(
isDryRun: options.DryRun,
cooldownTime: options.Cooldown,
logger: logger,
sem: semaphore.New(options.InitCount),
permit: permit,
wg: &sync.WaitGroup{},
ctx: ctx,
cancel: cancel,
Expand All @@ -107,21 +108,6 @@ func newPollerScaler(
}
}

// Acquire concurrent poll quota
func (p *pollerAutoScaler) Acquire(resource autoscaler.ResourceUnit) error {
return p.sem.Acquire(p.ctx, int(resource))
}

// Release concurrent poll quota
func (p *pollerAutoScaler) Release(resource autoscaler.ResourceUnit) {
p.sem.Release(int(resource))
}

// GetCurrent poll quota
func (p *pollerAutoScaler) GetCurrent() autoscaler.ResourceUnit {
return autoscaler.ResourceUnit(p.sem.GetLimit())
}

// Start an auto-scaler go routine and returns a done to stop it
func (p *pollerAutoScaler) Start() {
logger := p.logger.Sugar()
Expand All @@ -133,7 +119,7 @@ func (p *pollerAutoScaler) Start() {
case <-p.ctx.Done():
return
case <-time.After(p.cooldownTime):
currentResource := autoscaler.ResourceUnit(p.sem.GetLimit())
currentResource := autoscaler.ResourceUnit(p.permit.Quota())
currentUsages, err := p.pollerUsageEstimator.Estimate()
if err != nil {
logger.Warnw("poller autoscaler skip due to estimator error", "error", err)
Expand All @@ -146,14 +132,9 @@ func (p *pollerAutoScaler) Start() {
"recommend", uint64(proposedResource),
"isDryRun", p.isDryRun)
if !p.isDryRun {
p.sem.SetLimit(int(proposedResource))
p.permit.SetQuota(int(proposedResource))
}
p.pollerUsageEstimator.Reset()

// hooks
for i := range p.onAutoScale {
p.onAutoScale[i]()
}
}
}
}()
Expand Down
9 changes: 6 additions & 3 deletions internal/internal_poller_autoscaler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,14 @@
package internal

import (
"context"
"math/rand"
"sync"
"testing"
"time"

"go.uber.org/cadence/internal/common/testlogger"
"go.uber.org/cadence/internal/worker"

"github.com/stretchr/testify/assert"
"go.uber.org/atomic"
Expand Down Expand Up @@ -171,6 +173,7 @@ func Test_pollerAutoscaler(t *testing.T) {
TargetUtilization: float64(tt.args.targetMilliUsage) / 1000,
},
testlogger.NewZap(t),
worker.NewResizablePermit(tt.args.initialPollerCount),
// hook function that collects number of iterations
func() {
autoscalerEpoch.Add(1)
Expand All @@ -190,9 +193,9 @@ func Test_pollerAutoscaler(t *testing.T) {
go func() {
defer wg.Done()
for pollResult := range pollChan {
pollerScaler.Acquire(1)
pollerScaler.permit.Acquire(context.Background())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Acquire returns error which should be handled. otherwise Release might be called without acquire succeeding which leads to panic

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch. I have been searching for the issue for hours.

pollerScaler.CollectUsage(pollResult)
pollerScaler.Release(1)
pollerScaler.permit.Release()
}
}()
}
Expand All @@ -201,7 +204,7 @@ func Test_pollerAutoscaler(t *testing.T) {
return autoscalerEpoch.Load() == uint64(tt.args.autoScalerEpoch)
}, tt.args.cooldownTime+20*time.Millisecond, 10*time.Millisecond)
pollerScaler.Stop()
res := pollerScaler.GetCurrent()
res := pollerScaler.permit.Quota() - pollerScaler.permit.Count()
assert.Equal(t, tt.want, int(res))
})
}
Expand Down
41 changes: 25 additions & 16 deletions internal/internal_worker_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"time"

"go.uber.org/cadence/internal/common/debug"
"go.uber.org/cadence/internal/worker"

"github.com/uber-go/tally"
"go.uber.org/zap"
Expand Down Expand Up @@ -141,7 +142,7 @@ type (
logger *zap.Logger
metricsScope tally.Scope

pollerRequestCh chan struct{}
concurrency *worker.ConcurrencyLimit
pollerAutoScaler *pollerAutoScaler
taskQueueCh chan interface{}
sessionTokenBucket *sessionTokenBucket
Expand All @@ -167,11 +168,17 @@ func createPollRetryPolicy() backoff.RetryPolicy {
func newBaseWorker(options baseWorkerOptions, logger *zap.Logger, metricsScope tally.Scope, sessionTokenBucket *sessionTokenBucket) *baseWorker {
ctx, cancel := context.WithCancel(context.Background())

concurrency := &worker.ConcurrencyLimit{
PollerPermit: worker.NewResizablePermit(options.pollerCount),
TaskPermit: worker.NewResizablePermit(options.maxConcurrentTask),
}

var pollerAS *pollerAutoScaler
if pollerOptions := options.pollerAutoScaler; pollerOptions.Enabled {
pollerAS = newPollerScaler(
pollerOptions,
logger,
concurrency.PollerPermit,
)
}

Expand All @@ -182,7 +189,7 @@ func newBaseWorker(options baseWorkerOptions, logger *zap.Logger, metricsScope t
retrier: backoff.NewConcurrentRetrier(pollOperationRetryPolicy),
logger: logger.With(zapcore.Field{Key: tagWorkerType, Type: zapcore.StringType, String: options.workerType}),
metricsScope: tagScope(metricsScope, tagWorkerType, options.workerType),
pollerRequestCh: make(chan struct{}, options.maxConcurrentTask),
concurrency: concurrency,
pollerAutoScaler: pollerAS,
taskQueueCh: make(chan interface{}), // no buffer, so poller only able to poll new task after previous is dispatched.
limiterContext: ctx,
Expand Down Expand Up @@ -241,14 +248,20 @@ func (bw *baseWorker) runPoller() {
bw.metricsScope.Counter(metrics.PollerStartCounter).Inc(1)

for {
// permitChannel can be blocking without passing context because shutdownCh is used
permitChannel := bw.concurrency.PollerPermit.AcquireChan(context.Background())
select {
case <-bw.shutdownCh:
permitChannel.Close()
return
case <-bw.pollerRequestCh:
bw.metricsScope.Gauge(metrics.ConcurrentTaskQuota).Update(float64(cap(bw.pollerRequestCh)))
// This metric is used to monitor how many poll requests have been allocated
// and can be used to approximate number of concurrent task running (not pinpoint accurate)
bw.metricsScope.Gauge(metrics.PollerRequestBufferUsage).Update(float64(cap(bw.pollerRequestCh) - len(bw.pollerRequestCh)))
case <-permitChannel.C(): // don't poll unless there is a task permit
permitChannel.Close()
// TODO move to a centralized place inside the worker
// emit metrics on concurrent task permit quota and current task permit count
// NOTE task permit doesn't mean there is a task running, it still needs to poll until it gets a task to process
// thus the metrics is only an estimated value of how many tasks are running concurrently
bw.metricsScope.Gauge(metrics.ConcurrentTaskQuota).Update(float64(bw.concurrency.TaskPermit.Quota()))
bw.metricsScope.Gauge(metrics.PollerRequestBufferUsage).Update(float64(bw.concurrency.TaskPermit.Count()))
if bw.sessionTokenBucket != nil {
bw.sessionTokenBucket.waitForAvailableToken()
}
Expand All @@ -260,10 +273,6 @@ func (bw *baseWorker) runPoller() {
func (bw *baseWorker) runTaskDispatcher() {
defer bw.shutdownWG.Done()

for i := 0; i < bw.options.maxConcurrentTask; i++ {
bw.pollerRequestCh <- struct{}{}
}

for {
// wait for new task or shutdown
select {
Expand Down Expand Up @@ -294,10 +303,10 @@ func (bw *baseWorker) pollTask() {
var task interface{}

if bw.pollerAutoScaler != nil {
if pErr := bw.pollerAutoScaler.Acquire(1); pErr == nil {
defer bw.pollerAutoScaler.Release(1)
if pErr := bw.concurrency.PollerPermit.Acquire(bw.limiterContext); pErr == nil {
defer bw.concurrency.PollerPermit.Release()
} else {
bw.logger.Warn("poller auto scaler acquire error", zap.Error(pErr))
bw.logger.Warn("poller permit acquire error", zap.Error(pErr))
}
}

Expand Down Expand Up @@ -333,7 +342,7 @@ func (bw *baseWorker) pollTask() {
case <-bw.shutdownCh:
}
} else {
bw.pollerRequestCh <- struct{}{} // poll failed, trigger a new poll
bw.concurrency.TaskPermit.Release() // poll failed, trigger a new poll by returning a task permit
}
}

Expand Down Expand Up @@ -368,7 +377,7 @@ func (bw *baseWorker) processTask(task interface{}) {
}

if isPolledTask {
bw.pollerRequestCh <- struct{}{}
bw.concurrency.TaskPermit.Release() // task processed, trigger a new poll by returning a task permit
}
}()
err := bw.options.taskWorker.ProcessTask(task)
Expand Down
48 changes: 48 additions & 0 deletions internal/worker/concurrency.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
// Copyright (c) 2017-2021 Uber Technologies Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.

package worker

import "context"

var _ Permit = (*resizablePermit)(nil)

// ConcurrencyLimit contains synchronization primitives for dynamically controlling the concurrencies in workers
type ConcurrencyLimit struct {
PollerPermit Permit // controls concurrency of pollers
TaskPermit Permit // controls concurrency of task processing
}

// Permit is an adaptive permit issuer to control concurrency
type Permit interface {
Acquire(context.Context) error
AcquireChan(context.Context) PermitChannel
Count() int
Quota() int
Release()
SetQuota(int)
}

// PermitChannel is a channel that can be used to wait for a permit to be available
// Remember to call Close() to avoid goroutine leak
type PermitChannel interface {
C() <-chan struct{}
Close()
}
Loading
Loading