2021-04-18 15:41:13 -04:00
|
|
|
// Copyright (c) 2015-2021 MinIO, Inc.
|
|
|
|
//
|
|
|
|
// This file is part of MinIO Object Storage stack
|
|
|
|
//
|
|
|
|
// This program is free software: you can redistribute it and/or modify
|
|
|
|
// it under the terms of the GNU Affero General Public License as published by
|
|
|
|
// the Free Software Foundation, either version 3 of the License, or
|
|
|
|
// (at your option) any later version.
|
|
|
|
//
|
|
|
|
// This program is distributed in the hope that it will be useful
|
|
|
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
|
|
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
|
|
|
// GNU Affero General Public License for more details.
|
|
|
|
//
|
|
|
|
// You should have received a copy of the GNU Affero General Public License
|
|
|
|
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
2019-03-14 19:27:31 -04:00
|
|
|
|
|
|
|
package cmd
|
|
|
|
|
|
|
|
import (
|
|
|
|
"bytes"
|
|
|
|
"context"
|
|
|
|
"encoding/gob"
|
2020-06-17 17:49:26 -04:00
|
|
|
"errors"
|
2020-10-28 12:18:35 -04:00
|
|
|
"fmt"
|
2019-03-14 19:27:31 -04:00
|
|
|
"io"
|
2020-03-27 00:07:39 -04:00
|
|
|
"math"
|
2019-03-14 19:27:31 -04:00
|
|
|
"net/url"
|
2019-06-06 20:46:22 -04:00
|
|
|
"strconv"
|
2020-10-09 23:36:00 -04:00
|
|
|
"strings"
|
2020-03-27 00:07:39 -04:00
|
|
|
"sync"
|
2019-10-09 02:11:15 -04:00
|
|
|
"sync/atomic"
|
2019-06-08 18:54:41 -04:00
|
|
|
"time"
|
2019-03-14 19:27:31 -04:00
|
|
|
|
2020-04-12 22:37:09 -04:00
|
|
|
"github.com/dustin/go-humanize"
|
2021-05-06 11:52:02 -04:00
|
|
|
"github.com/minio/madmin-go"
|
2021-06-01 17:59:40 -04:00
|
|
|
"github.com/minio/minio/internal/event"
|
|
|
|
"github.com/minio/minio/internal/http"
|
|
|
|
xhttp "github.com/minio/minio/internal/http"
|
|
|
|
"github.com/minio/minio/internal/logger"
|
|
|
|
"github.com/minio/minio/internal/rest"
|
2021-06-14 17:54:37 -04:00
|
|
|
xnet "github.com/minio/pkg/net"
|
2020-10-28 12:18:35 -04:00
|
|
|
"github.com/tinylib/msgp/msgp"
|
2019-03-14 19:27:31 -04:00
|
|
|
)
|
|
|
|
|
2020-03-27 00:07:39 -04:00
|
|
|
// client to talk to peer Nodes.
|
2019-03-14 19:27:31 -04:00
|
|
|
type peerRESTClient struct {
|
|
|
|
host *xnet.Host
|
|
|
|
restClient *rest.Client
|
|
|
|
}
|
|
|
|
|
|
|
|
// Wrapper to restClient.Call to handle network errors, in case of network error the connection is marked disconnected
|
|
|
|
// permanently. The only way to restore the connection is at the xl-sets layer by xlsets.monitorAndConnectEndpoints()
|
|
|
|
// after verifying format.json
|
|
|
|
func (client *peerRESTClient) call(method string, values url.Values, body io.Reader, length int64) (respBody io.ReadCloser, err error) {
|
2020-04-09 12:30:02 -04:00
|
|
|
return client.callWithContext(GlobalContext, method, values, body, length)
|
2019-06-08 18:54:41 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// Wrapper to restClient.Call to handle network errors, in case of network error the connection is marked disconnected
|
|
|
|
// permanently. The only way to restore the connection is at the xl-sets layer by xlsets.monitorAndConnectEndpoints()
|
|
|
|
// after verifying format.json
|
|
|
|
func (client *peerRESTClient) callWithContext(ctx context.Context, method string, values url.Values, body io.Reader, length int64) (respBody io.ReadCloser, err error) {
|
2019-03-14 19:27:31 -04:00
|
|
|
if values == nil {
|
|
|
|
values = make(url.Values)
|
|
|
|
}
|
|
|
|
|
2020-09-04 12:45:06 -04:00
|
|
|
respBody, err = client.restClient.Call(ctx, method, values, body, length)
|
2019-03-14 19:27:31 -04:00
|
|
|
if err == nil {
|
|
|
|
return respBody, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Stringer provides a canonicalized representation of node.
|
|
|
|
func (client *peerRESTClient) String() string {
|
|
|
|
return client.host.String()
|
|
|
|
}
|
|
|
|
|
2020-12-22 12:16:07 -05:00
|
|
|
// IsOnline returns true if the peer client is online.
|
|
|
|
func (client *peerRESTClient) IsOnline() bool {
|
|
|
|
return client.restClient.IsOnline()
|
|
|
|
}
|
|
|
|
|
2019-03-14 19:27:31 -04:00
|
|
|
// Close - marks the client as closed.
|
|
|
|
func (client *peerRESTClient) Close() error {
|
|
|
|
client.restClient.Close()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetLocks - fetch older locks for a remote node.
|
2020-12-10 10:28:37 -05:00
|
|
|
func (client *peerRESTClient) GetLocks() (lockMap map[string][]lockRequesterInfo, err error) {
|
2019-03-14 19:27:31 -04:00
|
|
|
respBody, err := client.call(peerRESTMethodGetLocks, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
2020-12-10 10:28:37 -05:00
|
|
|
lockMap = map[string][]lockRequesterInfo{}
|
2019-03-14 19:27:31 -04:00
|
|
|
defer http.DrainBody(respBody)
|
2020-12-10 10:28:37 -05:00
|
|
|
err = gob.NewDecoder(respBody).Decode(&lockMap)
|
|
|
|
return lockMap, err
|
2019-03-14 19:27:31 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// ServerInfo - fetch server information for a remote node.
|
2019-12-11 17:27:03 -05:00
|
|
|
func (client *peerRESTClient) ServerInfo() (info madmin.ServerProperties, err error) {
|
2019-03-14 19:27:31 -04:00
|
|
|
respBody, err := client.call(peerRESTMethodServerInfo, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2020-03-27 00:07:39 -04:00
|
|
|
type networkOverloadedErr struct{}
|
|
|
|
|
|
|
|
var networkOverloaded networkOverloadedErr
|
|
|
|
|
|
|
|
func (n networkOverloadedErr) Error() string {
|
|
|
|
return "network overloaded"
|
|
|
|
}
|
|
|
|
|
2020-07-27 17:51:59 -04:00
|
|
|
type nullReader struct{}
|
|
|
|
|
|
|
|
func (r *nullReader) Read(b []byte) (int, error) {
|
|
|
|
return len(b), nil
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
func (client *peerRESTClient) doNetTest(ctx context.Context, dataSize int64, threadCount uint) (info madmin.PeerNetPerfInfo, err error) {
|
2020-10-01 12:50:08 -04:00
|
|
|
var mu sync.Mutex // mutex used to protect these slices in go-routines
|
2020-03-27 00:07:39 -04:00
|
|
|
latencies := []float64{}
|
|
|
|
throughputs := []float64{}
|
|
|
|
|
|
|
|
buflimiter := make(chan struct{}, threadCount)
|
|
|
|
errChan := make(chan error, threadCount)
|
|
|
|
|
2020-10-01 17:34:51 -04:00
|
|
|
var totalTransferred int64
|
2020-03-27 00:07:39 -04:00
|
|
|
|
|
|
|
// ensure enough samples to obtain normal distribution
|
|
|
|
maxSamples := int(10 * threadCount)
|
|
|
|
|
|
|
|
innerCtx, cancel := context.WithCancel(ctx)
|
|
|
|
|
|
|
|
slowSamples := int32(0)
|
|
|
|
maxSlowSamples := int32(maxSamples / 20)
|
|
|
|
slowSample := func() {
|
|
|
|
if slowSamples > maxSlowSamples { // 5% of total
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if atomic.AddInt32(&slowSamples, 1) >= maxSlowSamples {
|
|
|
|
errChan <- networkOverloaded
|
|
|
|
cancel()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-07-27 17:51:59 -04:00
|
|
|
var wg sync.WaitGroup
|
2020-03-27 00:07:39 -04:00
|
|
|
finish := func() {
|
|
|
|
<-buflimiter
|
|
|
|
wg.Done()
|
|
|
|
}
|
|
|
|
|
|
|
|
for i := 0; i < maxSamples; i++ {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return info, ctx.Err()
|
|
|
|
case err = <-errChan:
|
|
|
|
case buflimiter <- struct{}{}:
|
|
|
|
wg.Add(1)
|
|
|
|
|
|
|
|
if innerCtx.Err() != nil {
|
|
|
|
finish()
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
go func(i int) {
|
|
|
|
start := time.Now()
|
|
|
|
before := atomic.LoadInt64(&totalTransferred)
|
|
|
|
|
|
|
|
ctx, cancel := context.WithTimeout(innerCtx, 10*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2020-10-01 17:34:51 -04:00
|
|
|
progress := io.LimitReader(&nullReader{}, dataSize)
|
|
|
|
|
2020-11-20 15:52:53 -05:00
|
|
|
// Turn off healthCheckFn for health tests to cater for higher load on the peers.
|
2020-07-27 17:51:59 -04:00
|
|
|
clnt := newPeerRESTClient(client.host)
|
|
|
|
clnt.restClient.HealthCheckFn = nil
|
|
|
|
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := clnt.callWithContext(ctx, peerRESTMethodNetInfo, nil, progress, dataSize)
|
2020-07-27 17:51:59 -04:00
|
|
|
if err != nil {
|
|
|
|
if errors.Is(err, context.DeadlineExceeded) {
|
|
|
|
slowSample()
|
|
|
|
finish()
|
|
|
|
return
|
2020-03-27 00:07:39 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
errChan <- err
|
|
|
|
finish()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
http.DrainBody(respBody)
|
|
|
|
|
|
|
|
finish()
|
2020-10-01 17:34:51 -04:00
|
|
|
atomic.AddInt64(&totalTransferred, dataSize)
|
|
|
|
after := atomic.LoadInt64(&totalTransferred)
|
2020-03-27 00:07:39 -04:00
|
|
|
end := time.Now()
|
|
|
|
|
2020-08-24 15:11:20 -04:00
|
|
|
latency := end.Sub(start).Seconds()
|
2020-03-27 00:07:39 -04:00
|
|
|
|
|
|
|
if latency > maxLatencyForSizeThreads(dataSize, threadCount) {
|
|
|
|
slowSample()
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Throughput = (total data transferred across all threads / time taken) */
|
2020-08-24 15:11:20 -04:00
|
|
|
throughput := float64((after - before)) / latency
|
2020-03-27 00:07:39 -04:00
|
|
|
|
2020-10-01 12:50:08 -04:00
|
|
|
// Protect updating latencies and throughputs slices from
|
|
|
|
// multiple go-routines.
|
|
|
|
mu.Lock()
|
2020-03-27 00:07:39 -04:00
|
|
|
latencies = append(latencies, latency)
|
|
|
|
throughputs = append(throughputs, throughput)
|
2020-10-01 12:50:08 -04:00
|
|
|
mu.Unlock()
|
2020-03-27 00:07:39 -04:00
|
|
|
}(i)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
wg.Wait()
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2020-11-20 15:52:53 -05:00
|
|
|
latency, throughput, err := xnet.ComputePerfStats(latencies, throughputs)
|
2021-06-01 11:55:49 -04:00
|
|
|
return madmin.PeerNetPerfInfo{
|
|
|
|
Latency: madmin.Latency{
|
|
|
|
Avg: round(latency.Avg, 3),
|
|
|
|
Max: round(latency.Max, 3),
|
|
|
|
Min: round(latency.Min, 3),
|
|
|
|
Percentile50: round(latency.Percentile50, 3),
|
|
|
|
Percentile90: round(latency.Percentile90, 3),
|
|
|
|
Percentile99: round(latency.Percentile99, 3),
|
|
|
|
},
|
|
|
|
Throughput: madmin.Throughput{
|
|
|
|
Avg: uint64(round(throughput.Avg, 0)),
|
|
|
|
Max: uint64(round(throughput.Max, 0)),
|
|
|
|
Min: uint64(round(throughput.Min, 0)),
|
|
|
|
Percentile50: uint64(round(throughput.Percentile50, 0)),
|
|
|
|
Percentile90: uint64(round(throughput.Percentile90, 0)),
|
|
|
|
Percentile99: uint64(round(throughput.Percentile99, 0)),
|
|
|
|
},
|
|
|
|
}, nil
|
2020-03-27 00:07:39 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
func maxLatencyForSizeThreads(size int64, threadCount uint) float64 {
|
2020-04-12 22:37:09 -04:00
|
|
|
Gbit100 := 12.5 * float64(humanize.GiByte)
|
|
|
|
Gbit40 := 5.00 * float64(humanize.GiByte)
|
|
|
|
Gbit25 := 3.25 * float64(humanize.GiByte)
|
|
|
|
Gbit10 := 1.25 * float64(humanize.GiByte)
|
|
|
|
// Gbit1 := 0.25 * float64(humanize.GiByte)
|
2020-03-27 00:07:39 -04:00
|
|
|
|
|
|
|
// Given the current defaults, each combination of size/thread
|
|
|
|
// is supposed to fully saturate the intended pipe when all threads are active
|
|
|
|
// i.e. if the test is performed in a perfectly controlled environment, i.e. without
|
|
|
|
// CPU scheduling latencies and/or network jitters, then all threads working
|
|
|
|
// simultaneously should result in each of them completing in 1s
|
|
|
|
//
|
|
|
|
// In reality, I've assumed a normal distribution of latency with expected mean of 1s and min of 0s
|
|
|
|
// Then, 95% of threads should complete within 2 seconds (2 std. deviations from the mean). The 2s comes
|
|
|
|
// from fitting the normal curve such that the mean is 1.
|
|
|
|
//
|
|
|
|
// i.e. we expect that no more than 5% of threads to take longer than 2s to push the data.
|
|
|
|
//
|
|
|
|
// throughput | max latency
|
|
|
|
// 100 Gbit | 2s
|
|
|
|
// 40 Gbit | 2s
|
|
|
|
// 25 Gbit | 2s
|
|
|
|
// 10 Gbit | 2s
|
|
|
|
// 1 Gbit | inf
|
|
|
|
|
2020-08-24 15:11:20 -04:00
|
|
|
throughput := float64(size * int64(threadCount))
|
2020-03-27 00:07:39 -04:00
|
|
|
if throughput >= Gbit100 {
|
|
|
|
return 2.0
|
|
|
|
} else if throughput >= Gbit40 {
|
|
|
|
return 2.0
|
|
|
|
} else if throughput >= Gbit25 {
|
|
|
|
return 2.0
|
|
|
|
} else if throughput >= Gbit10 {
|
|
|
|
return 2.0
|
|
|
|
}
|
|
|
|
return math.MaxFloat64
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetNetPerfInfo - fetch network information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetNetPerfInfo(ctx context.Context) (info madmin.PeerNetPerfInfo, err error) {
|
2020-03-27 00:07:39 -04:00
|
|
|
|
|
|
|
// 100 Gbit -> 256 MiB * 50 threads
|
|
|
|
// 40 Gbit -> 256 MiB * 20 threads
|
|
|
|
// 25 Gbit -> 128 MiB * 25 threads
|
|
|
|
// 10 Gbit -> 128 MiB * 10 threads
|
|
|
|
// 1 Gbit -> 64 MiB * 2 threads
|
|
|
|
|
|
|
|
type step struct {
|
|
|
|
size int64
|
|
|
|
threads uint
|
|
|
|
}
|
|
|
|
steps := []step{
|
|
|
|
{ // 100 Gbit
|
2020-04-12 22:37:09 -04:00
|
|
|
size: 256 * humanize.MiByte,
|
2020-03-27 00:07:39 -04:00
|
|
|
threads: 50,
|
|
|
|
},
|
|
|
|
{ // 40 Gbit
|
2020-04-12 22:37:09 -04:00
|
|
|
size: 256 * humanize.MiByte,
|
2020-03-27 00:07:39 -04:00
|
|
|
threads: 20,
|
|
|
|
},
|
|
|
|
{ // 25 Gbit
|
2020-04-12 22:37:09 -04:00
|
|
|
size: 128 * humanize.MiByte,
|
2020-03-27 00:07:39 -04:00
|
|
|
threads: 25,
|
|
|
|
},
|
|
|
|
{ // 10 Gbit
|
2020-04-12 22:37:09 -04:00
|
|
|
size: 128 * humanize.MiByte,
|
2020-03-27 00:07:39 -04:00
|
|
|
threads: 10,
|
|
|
|
},
|
|
|
|
{ // 1 Gbit
|
2020-04-12 22:37:09 -04:00
|
|
|
size: 64 * humanize.MiByte,
|
2020-03-27 00:07:39 -04:00
|
|
|
threads: 2,
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
for i := range steps {
|
|
|
|
size := steps[i].size
|
|
|
|
threads := steps[i].threads
|
|
|
|
|
2020-11-20 15:52:53 -05:00
|
|
|
if info, err = client.doNetTest(ctx, size, threads); err != nil {
|
2020-03-27 00:07:39 -04:00
|
|
|
if err == networkOverloaded {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2020-07-27 17:51:59 -04:00
|
|
|
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
|
|
|
|
continue
|
2020-03-27 00:07:39 -04:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2020-11-20 15:52:53 -05:00
|
|
|
// DispatchNetInfo - dispatch other nodes to run Net info.
|
2021-06-01 11:55:49 -04:00
|
|
|
func (client *peerRESTClient) DispatchNetInfo(ctx context.Context) (info madmin.NetPerfInfo, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodDispatchNetInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
2020-04-18 14:06:11 -04:00
|
|
|
waitReader, err := waitForHTTPResponse(respBody)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
err = gob.NewDecoder(waitReader).Decode(&info)
|
2020-03-27 00:07:39 -04:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetDrivePerfInfos - fetch all disk's serial/parallal performance information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetDrivePerfInfos(ctx context.Context) (info madmin.DrivePerfInfos, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodDriveInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetCPUs - fetch CPU information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetCPUs(ctx context.Context) (info madmin.CPUs, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodCPUInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetPartitions - fetch disk partition information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetPartitions(ctx context.Context) (info madmin.Partitions, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodDiskHwInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetOSInfo - fetch OS information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetOSInfo(ctx context.Context) (info madmin.OSInfo, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodOsInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-07-30 02:05:34 -04:00
|
|
|
// GetSysErrors - fetch memory information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetSysErrors(ctx context.Context) (info madmin.SysErrors, err error) {
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodSysErrors, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetMemInfo - fetch memory information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetMemInfo(ctx context.Context) (info madmin.MemInfo, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodMemInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2021-06-01 11:55:49 -04:00
|
|
|
// GetProcInfo - fetch MinIO process information for a remote node.
|
|
|
|
func (client *peerRESTClient) GetProcInfo(ctx context.Context) (info madmin.ProcInfo, err error) {
|
2020-11-20 15:52:53 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodProcInfo, nil, nil, -1)
|
2020-03-27 00:07:39 -04:00
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&info)
|
|
|
|
return info, err
|
|
|
|
}
|
|
|
|
|
2019-03-14 19:27:31 -04:00
|
|
|
// StartProfiling - Issues profiling command on the peer node.
|
|
|
|
func (client *peerRESTClient) StartProfiling(profiler string) error {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTProfiler, profiler)
|
|
|
|
respBody, err := client.call(peerRESTMethodStartProfiling, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// DownloadProfileData - download profiled data from a remote node.
|
2020-01-10 20:19:58 -05:00
|
|
|
func (client *peerRESTClient) DownloadProfileData() (data map[string][]byte, err error) {
|
2019-03-14 19:27:31 -04:00
|
|
|
respBody, err := client.call(peerRESTMethodDownloadProfilingData, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&data)
|
|
|
|
return data, err
|
|
|
|
}
|
|
|
|
|
2021-04-04 18:34:33 -04:00
|
|
|
// GetBucketStats - load bucket statistics
|
|
|
|
func (client *peerRESTClient) GetBucketStats(bucket string) (BucketStats, error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTBucket, bucket)
|
|
|
|
respBody, err := client.call(peerRESTMethodGetBucketStats, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return BucketStats{}, err
|
|
|
|
}
|
|
|
|
|
|
|
|
var bs BucketStats
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return bs, msgp.Decode(respBody, &bs)
|
|
|
|
}
|
|
|
|
|
2020-05-19 16:53:54 -04:00
|
|
|
// LoadBucketMetadata - load bucket metadata
|
|
|
|
func (client *peerRESTClient) LoadBucketMetadata(bucket string) error {
|
2019-03-14 19:27:31 -04:00
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTBucket, bucket)
|
2020-05-19 16:53:54 -04:00
|
|
|
respBody, err := client.call(peerRESTMethodLoadBucketMetadata, values, nil, -1)
|
2019-03-14 19:27:31 -04:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-05-19 16:53:54 -04:00
|
|
|
// DeleteBucketMetadata - Delete bucket metadata
|
|
|
|
func (client *peerRESTClient) DeleteBucketMetadata(bucket string) error {
|
2019-03-14 19:27:31 -04:00
|
|
|
values := make(url.Values)
|
2020-05-19 16:53:54 -04:00
|
|
|
values.Set(peerRESTBucket, bucket)
|
|
|
|
respBody, err := client.call(peerRESTMethodDeleteBucketMetadata, values, nil, -1)
|
2019-03-14 19:27:31 -04:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-04-27 13:06:21 -04:00
|
|
|
// cycleServerBloomFilter will cycle the bloom filter to start recording to index y if not already.
|
|
|
|
// The response will contain a bloom filter starting at index x up to, but not including index y.
|
|
|
|
// If y is 0, the response will not update y, but return the currently recorded information
|
|
|
|
// from the current x to y-1.
|
|
|
|
func (client *peerRESTClient) cycleServerBloomFilter(ctx context.Context, req bloomFilterRequest) (*bloomFilterResponse, error) {
|
|
|
|
var reader bytes.Buffer
|
|
|
|
err := gob.NewEncoder(&reader).Encode(req)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2020-05-18 12:59:45 -04:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodCycleBloom, nil, &reader, -1)
|
2020-04-27 13:06:21 -04:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
var resp bloomFilterResponse
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return &resp, gob.NewDecoder(respBody).Decode(&resp)
|
|
|
|
}
|
|
|
|
|
2019-06-06 20:46:22 -04:00
|
|
|
// DeletePolicy - delete a specific canned policy.
|
|
|
|
func (client *peerRESTClient) DeletePolicy(policyName string) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTPolicy, policyName)
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodDeletePolicy, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// LoadPolicy - reload a specific canned policy.
|
|
|
|
func (client *peerRESTClient) LoadPolicy(policyName string) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTPolicy, policyName)
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodLoadPolicy, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-08-13 16:41:06 -04:00
|
|
|
// LoadPolicyMapping - reload a specific policy mapping
|
|
|
|
func (client *peerRESTClient) LoadPolicyMapping(userOrGroup string, isGroup bool) error {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTUserOrGroup, userOrGroup)
|
|
|
|
if isGroup {
|
|
|
|
values.Set(peerRESTIsGroup, "")
|
|
|
|
}
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodLoadPolicyMapping, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-06-06 20:46:22 -04:00
|
|
|
// DeleteUser - delete a specific user.
|
|
|
|
func (client *peerRESTClient) DeleteUser(accessKey string) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTUser, accessKey)
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodDeleteUser, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-04-24 15:10:09 -04:00
|
|
|
// DeleteServiceAccount - delete a specific service account.
|
|
|
|
func (client *peerRESTClient) DeleteServiceAccount(accessKey string) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTUser, accessKey)
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodDeleteServiceAccount, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-06-06 20:46:22 -04:00
|
|
|
// LoadUser - reload a specific user.
|
|
|
|
func (client *peerRESTClient) LoadUser(accessKey string, temp bool) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTUser, accessKey)
|
|
|
|
values.Set(peerRESTUserTemp, strconv.FormatBool(temp))
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodLoadUser, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-04-24 15:10:09 -04:00
|
|
|
// LoadServiceAccount - reload a specific service account.
|
|
|
|
func (client *peerRESTClient) LoadServiceAccount(accessKey string) (err error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTUser, accessKey)
|
|
|
|
|
|
|
|
respBody, err := client.call(peerRESTMethodLoadServiceAccount, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-08-02 17:25:00 -04:00
|
|
|
// LoadGroup - send load group command to peers.
|
|
|
|
func (client *peerRESTClient) LoadGroup(group string) error {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTGroup, group)
|
|
|
|
respBody, err := client.call(peerRESTMethodLoadGroup, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-07-23 11:03:31 -04:00
|
|
|
type serverUpdateInfo struct {
|
2021-04-08 12:51:11 -04:00
|
|
|
URL *url.URL
|
|
|
|
Sha256Sum []byte
|
|
|
|
Time time.Time
|
|
|
|
ReleaseInfo string
|
2020-07-23 11:03:31 -04:00
|
|
|
}
|
|
|
|
|
2019-08-28 18:04:43 -04:00
|
|
|
// ServerUpdate - sends server update message to remote peers.
|
2021-04-08 12:51:11 -04:00
|
|
|
func (client *peerRESTClient) ServerUpdate(ctx context.Context, u *url.URL, sha256Sum []byte, lrTime time.Time, releaseInfo string) error {
|
2019-08-28 18:04:43 -04:00
|
|
|
values := make(url.Values)
|
2020-07-23 11:03:31 -04:00
|
|
|
var reader bytes.Buffer
|
|
|
|
if err := gob.NewEncoder(&reader).Encode(serverUpdateInfo{
|
2021-04-08 12:51:11 -04:00
|
|
|
URL: u,
|
|
|
|
Sha256Sum: sha256Sum,
|
|
|
|
Time: lrTime,
|
|
|
|
ReleaseInfo: releaseInfo,
|
2020-07-23 11:03:31 -04:00
|
|
|
}); err != nil {
|
|
|
|
return err
|
2019-08-28 18:04:43 -04:00
|
|
|
}
|
2020-07-23 11:03:31 -04:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodServerUpdate, values, &reader, -1)
|
2019-08-28 18:04:43 -04:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-03-14 19:27:31 -04:00
|
|
|
// SignalService - sends signal to peer nodes.
|
|
|
|
func (client *peerRESTClient) SignalService(sig serviceSignal) error {
|
|
|
|
values := make(url.Values)
|
2019-08-27 14:37:47 -04:00
|
|
|
values.Set(peerRESTSignal, strconv.Itoa(int(sig)))
|
2019-03-14 19:27:31 -04:00
|
|
|
respBody, err := client.call(peerRESTMethodSignalService, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-06-25 19:42:24 -04:00
|
|
|
func (client *peerRESTClient) BackgroundHealStatus() (madmin.BgHealState, error) {
|
|
|
|
respBody, err := client.call(peerRESTMethodBackgroundHealStatus, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return madmin.BgHealState{}, err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
|
|
|
|
state := madmin.BgHealState{}
|
|
|
|
err = gob.NewDecoder(respBody).Decode(&state)
|
|
|
|
return state, err
|
|
|
|
}
|
|
|
|
|
2020-05-23 20:38:39 -04:00
|
|
|
// GetLocalDiskIDs - get a peer's local disks' IDs.
|
2020-06-09 22:19:03 -04:00
|
|
|
func (client *peerRESTClient) GetLocalDiskIDs(ctx context.Context) (diskIDs []string) {
|
2020-05-23 20:38:39 -04:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodGetLocalDiskIDs, nil, nil, -1)
|
|
|
|
if err != nil {
|
2020-06-09 22:19:03 -04:00
|
|
|
logger.LogIf(ctx, err)
|
2020-05-23 20:38:39 -04:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
if err = gob.NewDecoder(respBody).Decode(&diskIDs); err != nil {
|
2020-06-09 22:19:03 -04:00
|
|
|
logger.LogIf(ctx, err)
|
2020-05-23 20:38:39 -04:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return diskIDs
|
|
|
|
}
|
|
|
|
|
2020-10-28 12:18:35 -04:00
|
|
|
// GetMetacacheListing - get a new or existing metacache.
|
|
|
|
func (client *peerRESTClient) GetMetacacheListing(ctx context.Context, o listPathOptions) (*metacache, error) {
|
|
|
|
var reader bytes.Buffer
|
|
|
|
err := gob.NewEncoder(&reader).Encode(o)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodGetMetacacheListing, nil, &reader, int64(reader.Len()))
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
var resp metacache
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return &resp, msgp.Decode(respBody, &resp)
|
|
|
|
}
|
|
|
|
|
|
|
|
// UpdateMetacacheListing - update an existing metacache it will unconditionally be updated to the new state.
|
|
|
|
func (client *peerRESTClient) UpdateMetacacheListing(ctx context.Context, m metacache) (metacache, error) {
|
|
|
|
b, err := m.MarshalMsg(nil)
|
|
|
|
if err != nil {
|
|
|
|
return m, err
|
|
|
|
}
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodUpdateMetacacheListing, nil, bytes.NewBuffer(b), int64(len(b)))
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return m, err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
var resp metacache
|
|
|
|
return resp, msgp.Decode(respBody, &resp)
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2021-04-19 13:30:42 -04:00
|
|
|
func (client *peerRESTClient) LoadTransitionTierConfig(ctx context.Context) error {
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodLoadTransitionTierConfig, nil, nil, 0)
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-03-27 02:24:07 -04:00
|
|
|
func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh <-chan struct{}, traceOpts madmin.ServiceTraceOpts) {
|
2019-06-27 01:41:12 -04:00
|
|
|
values := make(url.Values)
|
2021-03-27 02:24:07 -04:00
|
|
|
values.Set(peerRESTTraceErr, strconv.FormatBool(traceOpts.OnlyErrors))
|
|
|
|
values.Set(peerRESTTraceS3, strconv.FormatBool(traceOpts.S3))
|
|
|
|
values.Set(peerRESTTraceStorage, strconv.FormatBool(traceOpts.Storage))
|
|
|
|
values.Set(peerRESTTraceOS, strconv.FormatBool(traceOpts.OS))
|
|
|
|
values.Set(peerRESTTraceInternal, strconv.FormatBool(traceOpts.Internal))
|
|
|
|
values.Set(peerRESTTraceThreshold, traceOpts.Threshold.String())
|
2019-06-27 01:41:12 -04:00
|
|
|
|
|
|
|
// To cancel the REST request in case doneCh gets closed.
|
2020-04-09 12:30:02 -04:00
|
|
|
ctx, cancel := context.WithCancel(GlobalContext)
|
2019-06-27 01:41:12 -04:00
|
|
|
|
|
|
|
cancelCh := make(chan struct{})
|
|
|
|
defer close(cancelCh)
|
2019-06-08 18:54:41 -04:00
|
|
|
go func() {
|
2019-06-27 01:41:12 -04:00
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
case <-cancelCh:
|
|
|
|
// There was an error in the REST request.
|
|
|
|
}
|
|
|
|
cancel()
|
|
|
|
}()
|
|
|
|
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodTrace, values, nil, -1)
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
dec := gob.NewDecoder(respBody)
|
|
|
|
for {
|
2021-05-06 11:52:02 -04:00
|
|
|
var info madmin.TraceInfo
|
2019-06-27 01:41:12 -04:00
|
|
|
if err = dec.Decode(&info); err != nil {
|
|
|
|
return
|
|
|
|
}
|
2019-07-31 14:08:39 -04:00
|
|
|
if len(info.NodeName) > 0 {
|
|
|
|
select {
|
|
|
|
case traceCh <- info:
|
|
|
|
default:
|
|
|
|
// Do not block on slow receivers.
|
|
|
|
}
|
2019-06-08 18:54:41 -04:00
|
|
|
}
|
2019-06-27 01:41:12 -04:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-16 13:56:18 -04:00
|
|
|
func (client *peerRESTClient) doListen(listenCh chan interface{}, doneCh <-chan struct{}, v url.Values) {
|
2019-12-12 13:01:23 -05:00
|
|
|
// To cancel the REST request in case doneCh gets closed.
|
2020-04-09 12:30:02 -04:00
|
|
|
ctx, cancel := context.WithCancel(GlobalContext)
|
2019-12-12 13:01:23 -05:00
|
|
|
|
|
|
|
cancelCh := make(chan struct{})
|
|
|
|
defer close(cancelCh)
|
|
|
|
go func() {
|
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
case <-cancelCh:
|
|
|
|
// There was an error in the REST request.
|
|
|
|
}
|
|
|
|
cancel()
|
|
|
|
}()
|
|
|
|
|
2019-12-16 23:30:57 -05:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodListen, v, nil, -1)
|
2019-12-12 13:01:23 -05:00
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
dec := gob.NewDecoder(respBody)
|
|
|
|
for {
|
|
|
|
var ev event.Event
|
2021-01-18 23:35:38 -05:00
|
|
|
if err := dec.Decode(&ev); err != nil {
|
2019-12-12 13:01:23 -05:00
|
|
|
return
|
|
|
|
}
|
|
|
|
if len(ev.EventVersion) > 0 {
|
|
|
|
select {
|
|
|
|
case listenCh <- ev:
|
|
|
|
default:
|
|
|
|
// Do not block on slow receivers.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Listen - listen on peers.
|
2020-04-16 13:56:18 -04:00
|
|
|
func (client *peerRESTClient) Listen(listenCh chan interface{}, doneCh <-chan struct{}, v url.Values) {
|
2019-12-12 13:01:23 -05:00
|
|
|
go func() {
|
|
|
|
for {
|
2019-12-16 23:30:57 -05:00
|
|
|
client.doListen(listenCh, doneCh, v)
|
2019-12-12 13:01:23 -05:00
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
// There was error in the REST request, retry after sometime as probably the peer is down.
|
|
|
|
time.Sleep(5 * time.Second)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
2019-06-27 01:41:12 -04:00
|
|
|
// Trace - send http trace request to peer nodes
|
2021-03-27 02:24:07 -04:00
|
|
|
func (client *peerRESTClient) Trace(traceCh chan interface{}, doneCh <-chan struct{}, traceOpts madmin.ServiceTraceOpts) {
|
2019-06-27 01:41:12 -04:00
|
|
|
go func() {
|
2019-06-08 18:54:41 -04:00
|
|
|
for {
|
2021-03-27 02:24:07 -04:00
|
|
|
client.doTrace(traceCh, doneCh, traceOpts)
|
2019-06-08 18:54:41 -04:00
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
return
|
|
|
|
default:
|
2019-06-27 01:41:12 -04:00
|
|
|
// There was error in the REST request, retry after sometime as probably the peer is down.
|
|
|
|
time.Sleep(5 * time.Second)
|
2019-06-08 18:54:41 -04:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
2019-09-03 14:10:48 -04:00
|
|
|
// ConsoleLog - sends request to peer nodes to get console logs
|
2020-04-16 13:56:18 -04:00
|
|
|
func (client *peerRESTClient) ConsoleLog(logCh chan interface{}, doneCh <-chan struct{}) {
|
2019-09-03 14:10:48 -04:00
|
|
|
go func() {
|
|
|
|
for {
|
|
|
|
// get cancellation context to properly unsubscribe peers
|
2020-04-09 12:30:02 -04:00
|
|
|
ctx, cancel := context.WithCancel(GlobalContext)
|
2019-09-03 14:10:48 -04:00
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodLog, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
// Retry the failed request.
|
|
|
|
time.Sleep(5 * time.Second)
|
|
|
|
} else {
|
|
|
|
dec := gob.NewDecoder(respBody)
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
<-doneCh
|
|
|
|
cancel()
|
|
|
|
}()
|
|
|
|
|
|
|
|
for {
|
|
|
|
var log madmin.LogInfo
|
|
|
|
if err = dec.Decode(&log); err != nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
select {
|
|
|
|
case logCh <- log:
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
select {
|
|
|
|
case <-doneCh:
|
|
|
|
cancel()
|
|
|
|
http.DrainBody(respBody)
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
// There was error in the REST request, retry.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
2020-10-28 12:18:35 -04:00
|
|
|
// newPeerRestClients creates new peer clients.
|
|
|
|
// The two slices will point to the same clients,
|
|
|
|
// but 'all' will contain nil entry for local client.
|
|
|
|
// The 'all' slice will be in the same order across the cluster.
|
2020-12-01 16:50:33 -05:00
|
|
|
func newPeerRestClients(endpoints EndpointServerPools) (remote, all []*peerRESTClient) {
|
2020-10-29 12:25:43 -04:00
|
|
|
if !globalIsDistErasure {
|
|
|
|
// Only useful in distributed setups
|
|
|
|
return nil, nil
|
|
|
|
}
|
2020-10-28 12:18:35 -04:00
|
|
|
hosts := endpoints.hostsSorted()
|
|
|
|
remote = make([]*peerRESTClient, 0, len(hosts))
|
|
|
|
all = make([]*peerRESTClient, len(hosts))
|
|
|
|
for i, host := range hosts {
|
|
|
|
if host == nil {
|
2019-11-09 12:27:23 -05:00
|
|
|
continue
|
|
|
|
}
|
2020-10-28 12:18:35 -04:00
|
|
|
all[i] = newPeerRESTClient(host)
|
|
|
|
remote = append(remote, all[i])
|
2019-03-14 19:27:31 -04:00
|
|
|
}
|
2020-10-28 12:18:35 -04:00
|
|
|
if len(all) != len(remote)+1 {
|
|
|
|
logger.LogIf(context.Background(), fmt.Errorf("WARNING: Expected number of all hosts (%v) to be remote +1 (%v)", len(all), len(remote)))
|
2019-03-14 19:27:31 -04:00
|
|
|
}
|
2020-10-28 12:18:35 -04:00
|
|
|
return remote, all
|
2019-03-14 19:27:31 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// Returns a peer rest client.
|
2020-07-12 01:19:38 -04:00
|
|
|
func newPeerRESTClient(peer *xnet.Host) *peerRESTClient {
|
2019-03-14 19:27:31 -04:00
|
|
|
scheme := "http"
|
2020-12-22 00:42:38 -05:00
|
|
|
if globalIsTLS {
|
2019-03-14 19:27:31 -04:00
|
|
|
scheme = "https"
|
|
|
|
}
|
|
|
|
|
|
|
|
serverURL := &url.URL{
|
|
|
|
Scheme: scheme,
|
|
|
|
Host: peer.String(),
|
|
|
|
Path: peerRESTPath,
|
|
|
|
}
|
|
|
|
|
2020-11-02 10:43:11 -05:00
|
|
|
restClient := rest.NewClient(serverURL, globalInternodeTransport, newAuthToken)
|
2020-11-10 12:28:23 -05:00
|
|
|
// Use a separate client to avoid recursive calls.
|
|
|
|
healthClient := rest.NewClient(serverURL, globalInternodeTransport, newAuthToken)
|
|
|
|
healthClient.ExpectTimeouts = true
|
2021-06-08 17:09:26 -04:00
|
|
|
healthClient.NoMetrics = true
|
2020-11-10 12:28:23 -05:00
|
|
|
|
2020-06-17 17:49:26 -04:00
|
|
|
// Construct a new health function.
|
|
|
|
restClient.HealthCheckFn = func() bool {
|
2021-01-28 16:38:12 -05:00
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), restClient.HealthCheckTimeout)
|
2020-11-10 12:28:23 -05:00
|
|
|
defer cancel()
|
|
|
|
respBody, err := healthClient.Call(ctx, peerRESTMethodHealth, nil, nil, -1)
|
2020-06-17 17:49:26 -04:00
|
|
|
xhttp.DrainBody(respBody)
|
2020-11-19 16:53:49 -05:00
|
|
|
return !isNetworkError(err)
|
2020-06-17 17:49:26 -04:00
|
|
|
}
|
2019-03-14 19:27:31 -04:00
|
|
|
|
2020-07-12 01:19:38 -04:00
|
|
|
return &peerRESTClient{host: peer, restClient: restClient}
|
2019-03-14 19:27:31 -04:00
|
|
|
}
|
2020-10-09 23:36:00 -04:00
|
|
|
|
|
|
|
// MonitorBandwidth - send http trace request to peer nodes
|
2021-04-18 15:41:13 -04:00
|
|
|
func (client *peerRESTClient) MonitorBandwidth(ctx context.Context, buckets []string) (*madmin.BucketBandwidthReport, error) {
|
2020-10-09 23:36:00 -04:00
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTBuckets, strings.Join(buckets, ","))
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodGetBandwidth, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
|
|
|
|
dec := gob.NewDecoder(respBody)
|
2021-04-18 15:41:13 -04:00
|
|
|
var bandwidthReport madmin.BucketBandwidthReport
|
2020-10-09 23:36:00 -04:00
|
|
|
err = dec.Decode(&bandwidthReport)
|
|
|
|
return &bandwidthReport, err
|
|
|
|
}
|
2021-01-18 23:35:38 -05:00
|
|
|
|
|
|
|
func (client *peerRESTClient) GetPeerMetrics(ctx context.Context) (<-chan Metric, error) {
|
|
|
|
respBody, err := client.callWithContext(ctx, peerRESTMethodGetPeerMetrics, nil, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
dec := gob.NewDecoder(respBody)
|
|
|
|
ch := make(chan Metric)
|
|
|
|
go func(ch chan<- Metric) {
|
|
|
|
for {
|
|
|
|
var metric Metric
|
|
|
|
if err := dec.Decode(&metric); err != nil {
|
|
|
|
http.DrainBody(respBody)
|
|
|
|
close(ch)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
ch <- metric
|
|
|
|
}
|
|
|
|
}(ch)
|
|
|
|
return ch, nil
|
|
|
|
}
|
2021-07-27 15:55:56 -04:00
|
|
|
|
|
|
|
func (client *peerRESTClient) Speedtest(ctx context.Context, size, concurrent int, duration time.Duration) (SpeedtestResult, error) {
|
|
|
|
values := make(url.Values)
|
|
|
|
values.Set(peerRESTSize, strconv.Itoa(size))
|
|
|
|
values.Set(peerRESTConcurrent, strconv.Itoa(concurrent))
|
|
|
|
values.Set(peerRESTDuration, duration.String())
|
|
|
|
|
|
|
|
respBody, err := client.callWithContext(context.Background(), peerRESTMethodSpeedtest, values, nil, -1)
|
|
|
|
if err != nil {
|
|
|
|
return SpeedtestResult{}, err
|
|
|
|
}
|
|
|
|
defer http.DrainBody(respBody)
|
|
|
|
|
|
|
|
dec := gob.NewDecoder(respBody)
|
|
|
|
var result SpeedtestResult
|
|
|
|
err = dec.Decode(&result)
|
|
|
|
return result, err
|
|
|
|
}
|