Add basic bandwidth monitoring for replication. (#10501)
This change tracks bandwidth for a bucket and object - [x] Add Admin API - [x] Add Peer API - [x] Add BW throttling - [x] Admin APIs to set replication limit - [x] Admin APIs for fetch bandwidthmaster
parent
071c004f8b
commit
c2f16ee846
@ -0,0 +1,28 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
// Details for the measured bandwidth
|
||||
type Details struct { |
||||
LimitInBytesPerSecond int64 `json:"limitInBits"` |
||||
CurrentBandwidthInBytesPerSecond float64 `json:"currentBandwidth"` |
||||
} |
||||
|
||||
// Report captures the details for all buckets.
|
||||
type Report struct { |
||||
BucketStats map[string]Details `json:"bucketStats,omitempty"` |
||||
} |
@ -0,0 +1,87 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
import ( |
||||
"sync" |
||||
"sync/atomic" |
||||
"time" |
||||
) |
||||
|
||||
const ( |
||||
// betaBucket is the weight used to calculate exponential moving average
|
||||
betaBucket = 0.1 // Number of averages considered = 1/(1-betaObject)
|
||||
) |
||||
|
||||
// bucketMeasurement captures the bandwidth details for one bucket
|
||||
type bucketMeasurement struct { |
||||
lock sync.Mutex |
||||
bytesSinceLastWindow uint64 // Total bytes since last window was processed
|
||||
startTime time.Time // Start time for window
|
||||
expMovingAvg float64 // Previously calculate sliding window
|
||||
} |
||||
|
||||
// newBucketMeasurement creates a new instance of the measurement with the initial start time.
|
||||
func newBucketMeasurement(initTime time.Time) *bucketMeasurement { |
||||
return &bucketMeasurement{ |
||||
startTime: initTime, |
||||
} |
||||
} |
||||
|
||||
// incrementBytes add bytes reported for a bucket.
|
||||
func (m *bucketMeasurement) incrementBytes(bytes uint64) { |
||||
atomic.AddUint64(&m.bytesSinceLastWindow, bytes) |
||||
} |
||||
|
||||
// updateExponentialMovingAverage processes the measurements captured so far.
|
||||
func (m *bucketMeasurement) updateExponentialMovingAverage(endTime time.Time) { |
||||
// Calculate aggregate avg bandwidth and exp window avg
|
||||
m.lock.Lock() |
||||
defer func() { |
||||
m.startTime = endTime |
||||
m.lock.Unlock() |
||||
}() |
||||
|
||||
if endTime.Before(m.startTime) { |
||||
return |
||||
} |
||||
|
||||
duration := endTime.Sub(m.startTime) |
||||
|
||||
bytesSinceLastWindow := atomic.SwapUint64(&m.bytesSinceLastWindow, 0) |
||||
|
||||
if m.expMovingAvg == 0 { |
||||
// Should address initial calculation and should be fine for resuming from 0
|
||||
m.expMovingAvg = float64(bytesSinceLastWindow) / duration.Seconds() |
||||
return |
||||
} |
||||
|
||||
increment := float64(bytesSinceLastWindow) / duration.Seconds() |
||||
m.expMovingAvg = exponentialMovingAverage(betaBucket, m.expMovingAvg, increment) |
||||
} |
||||
|
||||
// exponentialMovingAverage calculates the exponential moving average
|
||||
func exponentialMovingAverage(beta, previousAvg, incrementAvg float64) float64 { |
||||
return (1-beta)*incrementAvg + beta*previousAvg |
||||
} |
||||
|
||||
// getExpMovingAvgBytesPerSecond returns the exponential moving average for the bucket in bytes
|
||||
func (m *bucketMeasurement) getExpMovingAvgBytesPerSecond() float64 { |
||||
m.lock.Lock() |
||||
defer m.lock.Unlock() |
||||
return m.expMovingAvg |
||||
} |
@ -0,0 +1,175 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
import ( |
||||
"context" |
||||
"sync" |
||||
"time" |
||||
|
||||
"github.com/minio/minio/pkg/bandwidth" |
||||
"github.com/minio/minio/pkg/pubsub" |
||||
) |
||||
|
||||
// throttleBandwidth gets the throttle for bucket with the configured value
|
||||
func (m *Monitor) throttleBandwidth(ctx context.Context, bucket string, bandwidthBytesPerSecond int64) *throttle { |
||||
m.lock.Lock() |
||||
defer m.lock.Unlock() |
||||
throttle, ok := m.bucketThrottle[bucket] |
||||
if !ok { |
||||
throttle = newThrottle(ctx, bandwidthBytesPerSecond) |
||||
m.bucketThrottle[bucket] = throttle |
||||
return throttle |
||||
} |
||||
throttle.SetBandwidth(bandwidthBytesPerSecond) |
||||
return throttle |
||||
} |
||||
|
||||
// SubscribeToBuckets subscribes to buckets. Empty array for monitoring all buckets.
|
||||
func (m *Monitor) SubscribeToBuckets(subCh chan interface{}, doneCh <-chan struct{}, buckets []string) { |
||||
m.pubsub.Subscribe(subCh, doneCh, func(f interface{}) bool { |
||||
if buckets != nil || len(buckets) == 0 { |
||||
return true |
||||
} |
||||
report, ok := f.(*bandwidth.Report) |
||||
if !ok { |
||||
return false |
||||
} |
||||
for _, b := range buckets { |
||||
_, ok := report.BucketStats[b] |
||||
if ok { |
||||
return true |
||||
} |
||||
} |
||||
return false |
||||
}) |
||||
} |
||||
|
||||
// Monitor implements the monitoring for bandwidth measurements.
|
||||
type Monitor struct { |
||||
lock sync.Mutex // lock for all updates
|
||||
|
||||
activeBuckets map[string]*bucketMeasurement // Buckets with objects in flight
|
||||
|
||||
bucketMovingAvgTicker *time.Ticker // Ticker for calculating moving averages
|
||||
|
||||
pubsub *pubsub.PubSub // PubSub for reporting bandwidths.
|
||||
|
||||
bucketThrottle map[string]*throttle |
||||
|
||||
startProcessing sync.Once |
||||
|
||||
doneCh <-chan struct{} |
||||
} |
||||
|
||||
// NewMonitor returns a monitor with defaults.
|
||||
func NewMonitor(doneCh <-chan struct{}) *Monitor { |
||||
m := &Monitor{ |
||||
activeBuckets: make(map[string]*bucketMeasurement), |
||||
bucketMovingAvgTicker: time.NewTicker(1 * time.Second), |
||||
pubsub: pubsub.New(), |
||||
bucketThrottle: make(map[string]*throttle), |
||||
doneCh: doneCh, |
||||
} |
||||
return m |
||||
} |
||||
|
||||
// SelectionFunction for buckets
|
||||
type SelectionFunction func(bucket string) bool |
||||
|
||||
// SelectAllBuckets will select all buckets
|
||||
func SelectAllBuckets() SelectionFunction { |
||||
return func(bucket string) bool { |
||||
return true |
||||
} |
||||
} |
||||
|
||||
// SelectBuckets will select all the buckets passed in.
|
||||
func SelectBuckets(buckets ...string) SelectionFunction { |
||||
return func(bucket string) bool { |
||||
for _, b := range buckets { |
||||
if b != "" && b == bucket { |
||||
return true |
||||
} |
||||
} |
||||
return false |
||||
} |
||||
} |
||||
|
||||
// GetReport gets the report for all bucket bandwidth details.
|
||||
func (m *Monitor) GetReport(selectBucket SelectionFunction) *bandwidth.Report { |
||||
m.lock.Lock() |
||||
defer m.lock.Unlock() |
||||
return m.getReport(selectBucket) |
||||
} |
||||
|
||||
func (m *Monitor) getReport(selectBucket SelectionFunction) *bandwidth.Report { |
||||
report := &bandwidth.Report{ |
||||
BucketStats: make(map[string]bandwidth.Details), |
||||
} |
||||
for bucket, bucketMeasurement := range m.activeBuckets { |
||||
if !selectBucket(bucket) { |
||||
continue |
||||
} |
||||
report.BucketStats[bucket] = bandwidth.Details{ |
||||
LimitInBytesPerSecond: m.bucketThrottle[bucket].bytesPerSecond, |
||||
CurrentBandwidthInBytesPerSecond: bucketMeasurement.getExpMovingAvgBytesPerSecond(), |
||||
} |
||||
} |
||||
return report |
||||
} |
||||
|
||||
func (m *Monitor) process(doneCh <-chan struct{}) { |
||||
for { |
||||
select { |
||||
case <-m.bucketMovingAvgTicker.C: |
||||
m.processAvg() |
||||
case <-doneCh: |
||||
return |
||||
default: |
||||
} |
||||
} |
||||
} |
||||
|
||||
func (m *Monitor) getBucketMeasurement(bucket string, initTime time.Time) *bucketMeasurement { |
||||
bucketTracker, ok := m.activeBuckets[bucket] |
||||
if !ok { |
||||
bucketTracker = newBucketMeasurement(initTime) |
||||
m.activeBuckets[bucket] = bucketTracker |
||||
} |
||||
return bucketTracker |
||||
} |
||||
|
||||
func (m *Monitor) processAvg() { |
||||
m.lock.Lock() |
||||
defer m.lock.Unlock() |
||||
for _, bucketMeasurement := range m.activeBuckets { |
||||
bucketMeasurement.updateExponentialMovingAverage(time.Now()) |
||||
} |
||||
m.pubsub.Publish(m.getReport(SelectAllBuckets())) |
||||
} |
||||
|
||||
// track returns the measurement object for bucket and object
|
||||
func (m *Monitor) track(bucket string, object string, timeNow time.Time) *bucketMeasurement { |
||||
m.lock.Lock() |
||||
defer m.lock.Unlock() |
||||
m.startProcessing.Do(func() { |
||||
go m.process(m.doneCh) |
||||
}) |
||||
b := m.getBucketMeasurement(bucket, timeNow) |
||||
return b |
||||
} |
@ -0,0 +1,157 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
import ( |
||||
"context" |
||||
"reflect" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/minio/minio/pkg/bandwidth" |
||||
) |
||||
|
||||
const ( |
||||
oneMiB uint64 = 1024 * 1024 |
||||
) |
||||
|
||||
func TestMonitor_GetThrottle(t *testing.T) { |
||||
type fields struct { |
||||
bucketThrottles map[string]*throttle |
||||
bucket string |
||||
bpi int64 |
||||
} |
||||
t1 := newThrottle(context.Background(), 100) |
||||
t2 := newThrottle(context.Background(), 200) |
||||
tests := []struct { |
||||
name string |
||||
fields fields |
||||
want *throttle |
||||
}{ |
||||
{ |
||||
name: "Existing", |
||||
fields: fields{ |
||||
bucketThrottles: map[string]*throttle{"bucket": t1}, |
||||
bucket: "bucket", |
||||
bpi: 100, |
||||
}, |
||||
want: t1, |
||||
}, |
||||
{ |
||||
name: "new", |
||||
fields: fields{ |
||||
bucketThrottles: map[string]*throttle{"bucket": t1}, |
||||
bucket: "bucket2", |
||||
bpi: 200, |
||||
}, |
||||
want: t2, |
||||
}, |
||||
} |
||||
for _, tt := range tests { |
||||
tt := tt |
||||
t.Run(tt.name, func(t *testing.T) { |
||||
t.Parallel() |
||||
m := &Monitor{ |
||||
bucketThrottle: tt.fields.bucketThrottles, |
||||
} |
||||
if got := m.throttleBandwidth(context.Background(), tt.fields.bucket, tt.fields.bpi); got.bytesPerInterval != tt.want.bytesPerInterval { |
||||
t.Errorf("throttleBandwidth() = %v, want %v", got, tt.want) |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func TestMonitor_GetReport(t *testing.T) { |
||||
type fields struct { |
||||
activeBuckets map[string]*bucketMeasurement |
||||
endTime time.Time |
||||
update2 uint64 |
||||
endTime2 time.Time |
||||
} |
||||
start := time.Now() |
||||
m0 := newBucketMeasurement(start) |
||||
m0.incrementBytes(0) |
||||
m1MiBPS := newBucketMeasurement(start) |
||||
m1MiBPS.incrementBytes(oneMiB) |
||||
tests := []struct { |
||||
name string |
||||
fields fields |
||||
want *bandwidth.Report |
||||
want2 *bandwidth.Report |
||||
}{ |
||||
{ |
||||
name: "ZeroToOne", |
||||
fields: fields{ |
||||
activeBuckets: map[string]*bucketMeasurement{ |
||||
"bucket": m0, |
||||
}, |
||||
endTime: start.Add(1 * time.Second), |
||||
update2: oneMiB, |
||||
endTime2: start.Add(2 * time.Second), |
||||
}, |
||||
want: &bandwidth.Report{ |
||||
BucketStats: map[string]bandwidth.Details{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: 0}}, |
||||
}, |
||||
want2: &bandwidth.Report{ |
||||
BucketStats: map[string]bandwidth.Details{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: (1024 * 1024) / start.Add(2*time.Second).Sub(start.Add(1*time.Second)).Seconds()}}, |
||||
}, |
||||
}, |
||||
{ |
||||
name: "OneToTwo", |
||||
fields: fields{ |
||||
activeBuckets: map[string]*bucketMeasurement{ |
||||
"bucket": m1MiBPS, |
||||
}, |
||||
endTime: start.Add(1 * time.Second), |
||||
update2: 2 * oneMiB, |
||||
endTime2: start.Add(2 * time.Second), |
||||
}, |
||||
want: &bandwidth.Report{ |
||||
BucketStats: map[string]bandwidth.Details{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: float64(oneMiB)}}, |
||||
}, |
||||
want2: &bandwidth.Report{ |
||||
BucketStats: map[string]bandwidth.Details{"bucket": { |
||||
LimitInBytesPerSecond: 1024 * 1024, |
||||
CurrentBandwidthInBytesPerSecond: exponentialMovingAverage(betaBucket, float64(oneMiB), 2*float64(oneMiB))}}, |
||||
}, |
||||
}, |
||||
} |
||||
for _, tt := range tests { |
||||
tt := tt |
||||
t.Run(tt.name, func(t *testing.T) { |
||||
t.Parallel() |
||||
thr := throttle{ |
||||
bytesPerSecond: 1024 * 1024, |
||||
} |
||||
m := &Monitor{ |
||||
activeBuckets: tt.fields.activeBuckets, |
||||
bucketThrottle: map[string]*throttle{"bucket": &thr}, |
||||
} |
||||
m.activeBuckets["bucket"].updateExponentialMovingAverage(tt.fields.endTime) |
||||
got := m.GetReport(SelectAllBuckets()) |
||||
if !reflect.DeepEqual(got, tt.want) { |
||||
t.Errorf("GetReport() = %v, want %v", got, tt.want) |
||||
} |
||||
m.activeBuckets["bucket"].incrementBytes(tt.fields.update2) |
||||
m.activeBuckets["bucket"].updateExponentialMovingAverage(tt.fields.endTime2) |
||||
got = m.GetReport(SelectAllBuckets()) |
||||
if !reflect.DeepEqual(got, tt.want2) { |
||||
t.Errorf("GetReport() = %v, want %v", got, tt.want2) |
||||
} |
||||
}) |
||||
} |
||||
} |
@ -0,0 +1,86 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
* |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
import ( |
||||
"context" |
||||
"io" |
||||
"time" |
||||
) |
||||
|
||||
// MonitoredReader monitors the bandwidth
|
||||
type MonitoredReader struct { |
||||
bucket string // Token to track bucket
|
||||
bucketMeasurement *bucketMeasurement // bucket measurement object
|
||||
object string // Token to track object
|
||||
reader io.Reader // Reader to wrap
|
||||
lastStop time.Time // Last timestamp for a measurement
|
||||
headerSize int // Size of the header not captured by reader
|
||||
throttle *throttle // throttle the rate at which replication occur
|
||||
monitor *Monitor // Monitor reference
|
||||
closed bool // Reader is closed
|
||||
} |
||||
|
||||
// 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 { |
||||
timeNow := time.Now() |
||||
b := monitor.track(bucket, object, timeNow) |
||||
return &MonitoredReader{ |
||||
bucket: bucket, |
||||
object: object, |
||||
bucketMeasurement: b, |
||||
reader: reader, |
||||
lastStop: timeNow, |
||||
headerSize: headerSize, |
||||
throttle: monitor.throttleBandwidth(ctx, bucket, bandwidthBytesPerSecond), |
||||
monitor: monitor, |
||||
} |
||||
} |
||||
|
||||
// Read wraps the read reader
|
||||
func (m *MonitoredReader) Read(p []byte) (n int, err error) { |
||||
if m.closed { |
||||
err = io.ErrClosedPipe |
||||
return |
||||
} |
||||
p = p[:m.throttle.GetLimitForBytes(int64(len(p)))] |
||||
|
||||
n, err = m.reader.Read(p) |
||||
stop := time.Now() |
||||
update := uint64(n + m.headerSize) |
||||
|
||||
m.bucketMeasurement.incrementBytes(update) |
||||
m.lastStop = stop |
||||
unused := len(p) - (n + m.headerSize) |
||||
m.headerSize = 0 // Set to 0 post first read
|
||||
|
||||
if unused > 0 { |
||||
m.throttle.ReleaseUnusedBandwidth(int64(unused)) |
||||
} |
||||
return |
||||
} |
||||
|
||||
// Close stops tracking the io
|
||||
func (m *MonitoredReader) Close() error { |
||||
rc, ok := m.reader.(io.ReadCloser) |
||||
m.closed = true |
||||
if ok { |
||||
return rc.Close() |
||||
} |
||||
return nil |
||||
} |
@ -0,0 +1,107 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package bandwidth |
||||
|
||||
import ( |
||||
"context" |
||||
"sync" |
||||
"sync/atomic" |
||||
"time" |
||||
) |
||||
|
||||
const ( |
||||
throttleInternal = 250 * time.Millisecond |
||||
) |
||||
|
||||
// throttle implements the throttling for bandwidth
|
||||
type throttle struct { |
||||
generateTicker *time.Ticker // Ticker to generate available bandwidth
|
||||
freeBytes int64 // unused bytes in the interval
|
||||
bytesPerSecond int64 // max limit for bandwidth
|
||||
bytesPerInterval int64 // bytes allocated for the interval
|
||||
cond *sync.Cond // Used to notify waiting threads for bandwidth availability
|
||||
} |
||||
|
||||
// newThrottle returns a new bandwidth throttle. Set bytesPerSecond to 0 for no limit
|
||||
func newThrottle(ctx context.Context, bytesPerSecond int64) *throttle { |
||||
if bytesPerSecond == 0 { |
||||
return &throttle{} |
||||
} |
||||
t := &throttle{ |
||||
bytesPerSecond: bytesPerSecond, |
||||
generateTicker: time.NewTicker(throttleInternal), |
||||
} |
||||
|
||||
t.cond = sync.NewCond(&sync.Mutex{}) |
||||
t.SetBandwidth(bytesPerSecond) |
||||
t.freeBytes = t.bytesPerInterval |
||||
go t.generateBandwidth(ctx) |
||||
return t |
||||
} |
||||
|
||||
// GetLimitForBytes gets the bytes that are possible to send within the limit
|
||||
// if want is <= 0 or no bandwidth limit set, returns want.
|
||||
// Otherwise a value > 0 will always be returned.
|
||||
func (t *throttle) GetLimitForBytes(want int64) int64 { |
||||
if want <= 0 || atomic.LoadInt64(&t.bytesPerInterval) == 0 { |
||||
return want |
||||
} |
||||
t.cond.L.Lock() |
||||
defer t.cond.L.Unlock() |
||||
for { |
||||
var send int64 |
||||
freeBytes := atomic.LoadInt64(&t.freeBytes) |
||||
send = want |
||||
if freeBytes < want { |
||||
send = freeBytes |
||||
if send <= 0 { |
||||
t.cond.Wait() |
||||
continue |
||||
} |
||||
} |
||||
atomic.AddInt64(&t.freeBytes, -send) |
||||
return send |
||||
} |
||||
} |
||||
|
||||
// SetBandwidth sets a new bandwidth limit in bytes per second.
|
||||
func (t *throttle) SetBandwidth(bandwidthBiPS int64) { |
||||
bpi := int64(throttleInternal) * bandwidthBiPS / int64(time.Second) |
||||
atomic.StoreInt64(&t.bytesPerInterval, bpi) |
||||
} |
||||
|
||||
// ReleaseUnusedBandwidth releases bandwidth that was allocated for a user
|
||||
func (t *throttle) ReleaseUnusedBandwidth(bytes int64) { |
||||
atomic.AddInt64(&t.freeBytes, bytes) |
||||
} |
||||
|
||||
// generateBandwidth periodically allocates new bandwidth to use
|
||||
func (t *throttle) generateBandwidth(ctx context.Context) { |
||||
for { |
||||
select { |
||||
case <-t.generateTicker.C: |
||||
// A new window is available
|
||||
t.cond.L.Lock() |
||||
atomic.StoreInt64(&t.freeBytes, atomic.LoadInt64(&t.bytesPerInterval)) |
||||
t.cond.Broadcast() |
||||
t.cond.L.Unlock() |
||||
case <-ctx.Done(): |
||||
return |
||||
default: |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,61 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
* |
||||
*/ |
||||
|
||||
package madmin |
||||
|
||||
import ( |
||||
"context" |
||||
"encoding/json" |
||||
"io" |
||||
"net/http" |
||||
"net/url" |
||||
"strings" |
||||
|
||||
"github.com/minio/minio/pkg/bandwidth" |
||||
) |
||||
|
||||
// GetBucketBandwidth - Get a snapshot of the bandwidth measurements for replication buckets. If no buckets
|
||||
// generate replication traffic an empty map is returned.
|
||||
func (adm *AdminClient) GetBucketBandwidth(ctx context.Context, buckets ...string) (bandwidth.Report, error) { |
||||
queryValues := url.Values{} |
||||
if len(buckets) > 0 { |
||||
queryValues.Set("buckets", strings.Join(buckets, ",")) |
||||
} |
||||
|
||||
reqData := requestData{ |
||||
relPath: adminAPIPrefix + "/bandwidth", |
||||
queryValues: queryValues, |
||||
} |
||||
|
||||
resp, err := adm.executeMethod(ctx, http.MethodGet, reqData) |
||||
if err != nil { |
||||
closeResponse(resp) |
||||
return bandwidth.Report{}, err |
||||
} |
||||
if resp.StatusCode != http.StatusOK { |
||||
return bandwidth.Report{}, httpRespToErrorResponse(resp) |
||||
} |
||||
dec := json.NewDecoder(resp.Body) |
||||
for { |
||||
var report bandwidth.Report |
||||
err = dec.Decode(&report) |
||||
if err != nil && err != io.EOF { |
||||
return bandwidth.Report{}, err |
||||
} |
||||
return report, nil |
||||
} |
||||
} |
@ -0,0 +1,50 @@ |
||||
/* |
||||
* MinIO Cloud Storage, (C) 2020 MinIO, Inc. |
||||
* |
||||
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||
* you may not use this file except in compliance with the License. |
||||
* You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package main |
||||
|
||||
import ( |
||||
"context" |
||||
"fmt" |
||||
"log" |
||||
|
||||
"github.com/minio/minio/pkg/madmin" |
||||
) |
||||
|
||||
func main() { |
||||
// Note: YOUR-ACCESSKEYID, YOUR-SECRETACCESSKEY and my-bucketname are
|
||||
// dummy values, please replace them with original values.
|
||||
|
||||
// API requests are secure (HTTPS) if secure=true and insecure (HTTP) otherwise.
|
||||
// New returns an MinIO Admin client object.
|
||||
madminClient, err := madmin.New("your-minio.example.com:9000", "YOUR-ACCESSKEYID", "YOUR-SECRETACCESSKEY", true) |
||||
if err != nil { |
||||
log.Fatalln(err) |
||||
} |
||||
ctx := context.Background() |
||||
report, err := madminClient.GetBucketBandwidth(ctx) |
||||
if err != nil { |
||||
log.Fatalln(err) |
||||
return |
||||
} |
||||
fmt.Printf("Report: %+v\n", report) |
||||
report, err = madminClient.GetBucketBandwidth(ctx, "sourceBucket", "sourceBucket2") |
||||
if err != nil { |
||||
log.Fatalln(err) |
||||
return |
||||
} |
||||
fmt.Printf("Report: %+v\n", report) |
||||
} |
Loading…
Reference in new issue