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