mirror of
https://github.com/minio/minio.git
synced 2025-01-25 21:53:16 -05:00
fix: misc fixes for bandwidth reporting amd monitoring (#10683)
* Set peer for fetch bandwidth * Fix the limit for bandwidth that is reported. * Reduce CPU burn from bandwidth management.
This commit is contained in:
parent
ad726b49b4
commit
8a16a1a1a9
@ -241,7 +241,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
|
|||||||
for k, v := range putOpts.Header() {
|
for k, v := range putOpts.Header() {
|
||||||
headerSize += len(k) + len(v)
|
headerSize += len(k) + len(v)
|
||||||
}
|
}
|
||||||
r := bandwidth.NewMonitoredReader(ctx, globalBucketMonitor, objInfo.Bucket, objInfo.Name, gr, headerSize, b)
|
r := bandwidth.NewMonitoredReader(ctx, globalBucketMonitor, objInfo.Bucket, objInfo.Name, gr, headerSize, b, target.BandwidthLimit)
|
||||||
|
|
||||||
_, err = tgt.PutObject(ctx, dest.Bucket, object, r, size, "", "", putOpts)
|
_, err = tgt.PutObject(ctx, dest.Bucket, object, r, size, "", "", putOpts)
|
||||||
r.Close()
|
r.Close()
|
||||||
|
@ -1290,14 +1290,14 @@ func sendEvent(args eventArgs) {
|
|||||||
func (sys *NotificationSys) GetBandwidthReports(ctx context.Context, buckets ...string) bandwidth.Report {
|
func (sys *NotificationSys) GetBandwidthReports(ctx context.Context, buckets ...string) bandwidth.Report {
|
||||||
reports := make([]*bandwidth.Report, len(sys.peerClients))
|
reports := make([]*bandwidth.Report, len(sys.peerClients))
|
||||||
g := errgroup.WithNErrs(len(sys.peerClients))
|
g := errgroup.WithNErrs(len(sys.peerClients))
|
||||||
for index, peer := range sys.peerClients {
|
for index := range sys.peerClients {
|
||||||
if peer == nil {
|
if sys.peerClients[index] == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
index := index
|
index := index
|
||||||
g.Go(func() error {
|
g.Go(func() error {
|
||||||
var err error
|
var err error
|
||||||
reports[index], err = peer.MonitorBandwidth(ctx, buckets)
|
reports[index], err = sys.peerClients[index].MonitorBandwidth(ctx, buckets)
|
||||||
return err
|
return err
|
||||||
}, index)
|
}, index)
|
||||||
}
|
}
|
||||||
@ -1323,6 +1323,9 @@ func (sys *NotificationSys) GetBandwidthReports(ctx context.Context, buckets ...
|
|||||||
d = consolidatedReport.BucketStats[bucket]
|
d = consolidatedReport.BucketStats[bucket]
|
||||||
d.LimitInBytesPerSecond = report.BucketStats[bucket].LimitInBytesPerSecond
|
d.LimitInBytesPerSecond = report.BucketStats[bucket].LimitInBytesPerSecond
|
||||||
}
|
}
|
||||||
|
if d.LimitInBytesPerSecond < report.BucketStats[bucket].LimitInBytesPerSecond {
|
||||||
|
d.LimitInBytesPerSecond = report.BucketStats[bucket].LimitInBytesPerSecond
|
||||||
|
}
|
||||||
d.CurrentBandwidthInBytesPerSecond += report.BucketStats[bucket].CurrentBandwidthInBytesPerSecond
|
d.CurrentBandwidthInBytesPerSecond += report.BucketStats[bucket].CurrentBandwidthInBytesPerSecond
|
||||||
consolidatedReport.BucketStats[bucket] = d
|
consolidatedReport.BucketStats[bucket] = d
|
||||||
}
|
}
|
||||||
|
@ -26,16 +26,16 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
// throttleBandwidth gets the throttle for bucket with the configured value
|
// throttleBandwidth gets the throttle for bucket with the configured value
|
||||||
func (m *Monitor) throttleBandwidth(ctx context.Context, bucket string, bandwidthBytesPerSecond int64) *throttle {
|
func (m *Monitor) throttleBandwidth(ctx context.Context, bucket string, bandwidthBytesPerSecond int64, clusterBandwidth int64) *throttle {
|
||||||
m.lock.Lock()
|
m.lock.Lock()
|
||||||
defer m.lock.Unlock()
|
defer m.lock.Unlock()
|
||||||
throttle, ok := m.bucketThrottle[bucket]
|
throttle, ok := m.bucketThrottle[bucket]
|
||||||
if !ok {
|
if !ok {
|
||||||
throttle = newThrottle(ctx, bandwidthBytesPerSecond)
|
throttle = newThrottle(ctx, bandwidthBytesPerSecond, clusterBandwidth)
|
||||||
m.bucketThrottle[bucket] = throttle
|
m.bucketThrottle[bucket] = throttle
|
||||||
return throttle
|
return throttle
|
||||||
}
|
}
|
||||||
throttle.SetBandwidth(bandwidthBytesPerSecond)
|
throttle.SetBandwidth(bandwidthBytesPerSecond, clusterBandwidth)
|
||||||
return throttle
|
return throttle
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -80,7 +80,7 @@ type Monitor struct {
|
|||||||
func NewMonitor(doneCh <-chan struct{}) *Monitor {
|
func NewMonitor(doneCh <-chan struct{}) *Monitor {
|
||||||
m := &Monitor{
|
m := &Monitor{
|
||||||
activeBuckets: make(map[string]*bucketMeasurement),
|
activeBuckets: make(map[string]*bucketMeasurement),
|
||||||
bucketMovingAvgTicker: time.NewTicker(1 * time.Second),
|
bucketMovingAvgTicker: time.NewTicker(2 * time.Second),
|
||||||
pubsub: pubsub.New(),
|
pubsub: pubsub.New(),
|
||||||
bucketThrottle: make(map[string]*throttle),
|
bucketThrottle: make(map[string]*throttle),
|
||||||
doneCh: doneCh,
|
doneCh: doneCh,
|
||||||
@ -124,7 +124,7 @@ func (m *Monitor) getReport(selectBucket SelectionFunction) *bandwidth.Report {
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
report.BucketStats[bucket] = bandwidth.Details{
|
report.BucketStats[bucket] = bandwidth.Details{
|
||||||
LimitInBytesPerSecond: m.bucketThrottle[bucket].bytesPerSecond,
|
LimitInBytesPerSecond: m.bucketThrottle[bucket].clusterBandwidth,
|
||||||
CurrentBandwidthInBytesPerSecond: bucketMeasurement.getExpMovingAvgBytesPerSecond(),
|
CurrentBandwidthInBytesPerSecond: bucketMeasurement.getExpMovingAvgBytesPerSecond(),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -138,7 +138,6 @@ func (m *Monitor) process(doneCh <-chan struct{}) {
|
|||||||
m.processAvg()
|
m.processAvg()
|
||||||
case <-doneCh:
|
case <-doneCh:
|
||||||
return
|
return
|
||||||
default:
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -35,8 +35,8 @@ func TestMonitor_GetThrottle(t *testing.T) {
|
|||||||
bucket string
|
bucket string
|
||||||
bpi int64
|
bpi int64
|
||||||
}
|
}
|
||||||
t1 := newThrottle(context.Background(), 100)
|
t1 := newThrottle(context.Background(), 100, 1024*1024)
|
||||||
t2 := newThrottle(context.Background(), 200)
|
t2 := newThrottle(context.Background(), 200, 1024*1024)
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
name string
|
name string
|
||||||
fields fields
|
fields fields
|
||||||
@ -68,7 +68,7 @@ func TestMonitor_GetThrottle(t *testing.T) {
|
|||||||
m := &Monitor{
|
m := &Monitor{
|
||||||
bucketThrottle: tt.fields.bucketThrottles,
|
bucketThrottle: tt.fields.bucketThrottles,
|
||||||
}
|
}
|
||||||
if got := m.throttleBandwidth(context.Background(), tt.fields.bucket, tt.fields.bpi); got.bytesPerInterval != tt.want.bytesPerInterval {
|
if got := m.throttleBandwidth(context.Background(), tt.fields.bucket, tt.fields.bpi, 1024*1024); got.bytesPerInterval != tt.want.bytesPerInterval {
|
||||||
t.Errorf("throttleBandwidth() = %v, want %v", got, tt.want)
|
t.Errorf("throttleBandwidth() = %v, want %v", got, tt.want)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
@ -135,7 +135,8 @@ func TestMonitor_GetReport(t *testing.T) {
|
|||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
t.Parallel()
|
t.Parallel()
|
||||||
thr := throttle{
|
thr := throttle{
|
||||||
bytesPerSecond: 1024 * 1024,
|
bytesPerSecond: 1024 * 1024,
|
||||||
|
clusterBandwidth: 1024 * 1024,
|
||||||
}
|
}
|
||||||
m := &Monitor{
|
m := &Monitor{
|
||||||
activeBuckets: tt.fields.activeBuckets,
|
activeBuckets: tt.fields.activeBuckets,
|
||||||
|
@ -37,7 +37,7 @@ type MonitoredReader struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// NewMonitoredReader returns a io.ReadCloser that reports bandwidth details
|
// NewMonitoredReader returns a io.ReadCloser that reports bandwidth details
|
||||||
func NewMonitoredReader(ctx context.Context, monitor *Monitor, bucket string, object string, reader io.Reader, headerSize int, bandwidthBytesPerSecond int64) *MonitoredReader {
|
func NewMonitoredReader(ctx context.Context, monitor *Monitor, bucket string, object string, reader io.Reader, headerSize int, bandwidthBytesPerSecond int64, clusterBandwidth int64) *MonitoredReader {
|
||||||
timeNow := time.Now()
|
timeNow := time.Now()
|
||||||
b := monitor.track(bucket, object, timeNow)
|
b := monitor.track(bucket, object, timeNow)
|
||||||
return &MonitoredReader{
|
return &MonitoredReader{
|
||||||
@ -47,7 +47,7 @@ func NewMonitoredReader(ctx context.Context, monitor *Monitor, bucket string, ob
|
|||||||
reader: reader,
|
reader: reader,
|
||||||
lastStop: timeNow,
|
lastStop: timeNow,
|
||||||
headerSize: headerSize,
|
headerSize: headerSize,
|
||||||
throttle: monitor.throttleBandwidth(ctx, bucket, bandwidthBytesPerSecond),
|
throttle: monitor.throttleBandwidth(ctx, bucket, bandwidthBytesPerSecond, clusterBandwidth),
|
||||||
monitor: monitor,
|
monitor: monitor,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -29,27 +29,31 @@ const (
|
|||||||
|
|
||||||
// throttle implements the throttling for bandwidth
|
// throttle implements the throttling for bandwidth
|
||||||
type throttle struct {
|
type throttle struct {
|
||||||
generateTicker *time.Ticker // Ticker to generate available bandwidth
|
generateTicker *time.Ticker // Ticker to generate available bandwidth
|
||||||
freeBytes int64 // unused bytes in the interval
|
freeBytes int64 // unused bytes in the interval
|
||||||
bytesPerSecond int64 // max limit for bandwidth
|
bytesPerSecond int64 // max limit for bandwidth
|
||||||
bytesPerInterval int64 // bytes allocated for the interval
|
bytesPerInterval int64 // bytes allocated for the interval
|
||||||
cond *sync.Cond // Used to notify waiting threads for bandwidth availability
|
clusterBandwidth int64 // Cluster wide bandwidth needed for reporting
|
||||||
|
cond *sync.Cond // Used to notify waiting threads for bandwidth availability
|
||||||
|
goGenerate int64 // Flag to track if generate routine should be running. 0 == stopped
|
||||||
|
ctx context.Context // Context for generate
|
||||||
}
|
}
|
||||||
|
|
||||||
// newThrottle returns a new bandwidth throttle. Set bytesPerSecond to 0 for no limit
|
// newThrottle returns a new bandwidth throttle. Set bytesPerSecond to 0 for no limit
|
||||||
func newThrottle(ctx context.Context, bytesPerSecond int64) *throttle {
|
func newThrottle(ctx context.Context, bytesPerSecond int64, clusterBandwidth int64) *throttle {
|
||||||
if bytesPerSecond == 0 {
|
if bytesPerSecond == 0 {
|
||||||
return &throttle{}
|
return &throttle{}
|
||||||
}
|
}
|
||||||
t := &throttle{
|
t := &throttle{
|
||||||
bytesPerSecond: bytesPerSecond,
|
bytesPerSecond: bytesPerSecond,
|
||||||
generateTicker: time.NewTicker(throttleInternal),
|
generateTicker: time.NewTicker(throttleInternal),
|
||||||
|
clusterBandwidth: clusterBandwidth,
|
||||||
|
ctx: ctx,
|
||||||
}
|
}
|
||||||
|
|
||||||
t.cond = sync.NewCond(&sync.Mutex{})
|
t.cond = sync.NewCond(&sync.Mutex{})
|
||||||
t.SetBandwidth(bytesPerSecond)
|
t.SetBandwidth(bytesPerSecond, clusterBandwidth)
|
||||||
t.freeBytes = t.bytesPerInterval
|
t.freeBytes = t.bytesPerInterval
|
||||||
go t.generateBandwidth(ctx)
|
|
||||||
return t
|
return t
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -74,12 +78,17 @@ func (t *throttle) GetLimitForBytes(want int64) int64 {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
atomic.AddInt64(&t.freeBytes, -send)
|
atomic.AddInt64(&t.freeBytes, -send)
|
||||||
|
|
||||||
|
// Bandwidth was consumed, start generate routine to allocate bandwidth
|
||||||
|
if atomic.CompareAndSwapInt64(&t.goGenerate, 0, 1) {
|
||||||
|
go t.generateBandwidth(t.ctx)
|
||||||
|
}
|
||||||
return send
|
return send
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// SetBandwidth sets a new bandwidth limit in bytes per second.
|
// SetBandwidth sets a new bandwidth limit in bytes per second.
|
||||||
func (t *throttle) SetBandwidth(bandwidthBiPS int64) {
|
func (t *throttle) SetBandwidth(bandwidthBiPS int64, clusterBandwidth int64) {
|
||||||
bpi := int64(throttleInternal) * bandwidthBiPS / int64(time.Second)
|
bpi := int64(throttleInternal) * bandwidthBiPS / int64(time.Second)
|
||||||
atomic.StoreInt64(&t.bytesPerInterval, bpi)
|
atomic.StoreInt64(&t.bytesPerInterval, bpi)
|
||||||
}
|
}
|
||||||
@ -94,6 +103,11 @@ func (t *throttle) generateBandwidth(ctx context.Context) {
|
|||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-t.generateTicker.C:
|
case <-t.generateTicker.C:
|
||||||
|
if atomic.LoadInt64(&t.freeBytes) == atomic.LoadInt64(&t.bytesPerInterval) {
|
||||||
|
// No bandwidth consumption stop the routine.
|
||||||
|
atomic.StoreInt64(&t.goGenerate, 0)
|
||||||
|
return
|
||||||
|
}
|
||||||
// A new window is available
|
// A new window is available
|
||||||
t.cond.L.Lock()
|
t.cond.L.Lock()
|
||||||
atomic.StoreInt64(&t.freeBytes, atomic.LoadInt64(&t.bytesPerInterval))
|
atomic.StoreInt64(&t.freeBytes, atomic.LoadInt64(&t.bytesPerInterval))
|
||||||
@ -101,7 +115,6 @@ func (t *throttle) generateBandwidth(ctx context.Context) {
|
|||||||
t.cond.L.Unlock()
|
t.cond.L.Unlock()
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
return
|
return
|
||||||
default:
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user