Implement oboard diagnostics admin API (#9024)

- Implement a graph algorithm to test network bandwidth from every 
  node to every other node
- Saturate any network bandwidth adaptively, accounting for slow 
  and fast network capacity
- Implement parallel drive OBD tests
- Implement a paging mechanism for OBD test to provide periodic updates to client
- Implement Sys, Process, Host, Mem OBD Infos
This commit is contained in:
Sidhartha Mani 2020-03-26 21:07:39 -07:00 committed by GitHub
parent 2777956581
commit 0c80bf45d0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 2153 additions and 7 deletions

View File

@ -1372,6 +1372,135 @@ func (a adminAPIHandlers) ServerHardwareInfoHandler(w http.ResponseWriter, r *ht
}
}
// OBDInfoHandler - GET /minio/admin/v2/obdinfo
// ----------
// Get server on-board diagnostics
func (a adminAPIHandlers) OBDInfoHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "OBDInfo")
objectAPI, _ := validateAdminReq(ctx, w, r, iampolicy.OBDInfoAdminAction)
if objectAPI == nil {
return
}
deadlinedCtx, cancel := context.WithDeadline(ctx, time.Now().Add(10*time.Minute))
obdInfo := madmin.OBDInfo{}
setCommonHeaders(w)
w.Header().Set(xhttp.ContentType, string(mimeJSON))
w.WriteHeader(http.StatusOK)
partialWrite := func() {
jsonBytes, _ := json.Marshal(obdInfo)
_, err := w.Write(jsonBytes)
logger.LogIf(ctx, err)
}
finish := func() {
partialWrite()
w.(http.Flusher).Flush()
cancel()
}
errResp := func(err error) {
errorResponse := getAPIErrorResponse(ctx, toAdminAPIErr(ctx, err), r.URL.String(),
w.Header().Get(xhttp.AmzRequestID), globalDeploymentID)
encodedErrorResponse := encodeResponse(errorResponse)
obdInfo.Error = string(encodedErrorResponse)
finish()
}
nsLock := objectAPI.NewNSLock(deadlinedCtx, minioMetaBucket, "obd-in-progress")
if err := nsLock.GetLock(newDynamicTimeout(10*time.Minute, 600*time.Second)); err != nil { // returns a locked lock
errResp(err)
return
}
defer nsLock.Unlock()
vars := mux.Vars(r)
if cpu, ok := vars["syscpu"]; ok && cpu == "true" {
cpuInfo := getLocalCPUOBDInfo(deadlinedCtx)
obdInfo.Sys.CPUInfo = append(obdInfo.Sys.CPUInfo, cpuInfo)
obdInfo.Sys.CPUInfo = append(obdInfo.Sys.CPUInfo, globalNotificationSys.CPUOBDInfo(deadlinedCtx)...)
partialWrite()
}
if diskHw, ok := vars["sysdiskhw"]; ok && diskHw == "true" {
diskHwInfo := getLocalDiskHwOBD(deadlinedCtx)
obdInfo.Sys.DiskHwInfo = append(obdInfo.Sys.DiskHwInfo, diskHwInfo)
obdInfo.Sys.DiskHwInfo = append(obdInfo.Sys.DiskHwInfo, globalNotificationSys.DiskHwOBDInfo(deadlinedCtx)...)
partialWrite()
}
if osInfo, ok := vars["sysosinfo"]; ok && osInfo == "true" {
osInfo := getLocalOsInfoOBD(deadlinedCtx)
obdInfo.Sys.OsInfo = append(obdInfo.Sys.OsInfo, osInfo)
obdInfo.Sys.OsInfo = append(obdInfo.Sys.OsInfo, globalNotificationSys.OsOBDInfo(deadlinedCtx)...)
partialWrite()
}
if mem, ok := vars["sysmem"]; ok && mem == "true" {
memInfo := getLocalMemOBD(deadlinedCtx)
obdInfo.Sys.MemInfo = append(obdInfo.Sys.MemInfo, memInfo)
obdInfo.Sys.MemInfo = append(obdInfo.Sys.MemInfo, globalNotificationSys.MemOBDInfo(deadlinedCtx)...)
partialWrite()
}
if proc, ok := vars["sysprocess"]; ok && proc == "true" {
procInfo := getLocalProcOBD(deadlinedCtx)
obdInfo.Sys.ProcInfo = append(obdInfo.Sys.ProcInfo, procInfo)
obdInfo.Sys.ProcInfo = append(obdInfo.Sys.ProcInfo, globalNotificationSys.ProcOBDInfo(deadlinedCtx)...)
partialWrite()
}
if config, ok := vars["minioconfig"]; ok && config == "true" {
cfg, err := readServerConfig(ctx, objectAPI)
logger.LogIf(ctx, err)
obdInfo.Minio.Config = cfg
partialWrite()
}
if drive, ok := vars["perfdrive"]; ok && drive == "true" {
// Get drive obd details from local server's drive(s)
driveOBDSerial := getLocalDrivesOBD(deadlinedCtx, false, globalEndpoints, r)
driveOBDParallel := getLocalDrivesOBD(deadlinedCtx, true, globalEndpoints, r)
errStr := ""
if driveOBDSerial.Error != "" {
errStr = "serial: " + driveOBDSerial.Error
}
if driveOBDParallel.Error != "" {
errStr = errStr + " parallel: " + driveOBDParallel.Error
}
driveOBD := madmin.ServerDrivesOBDInfo{
Addr: driveOBDSerial.Addr,
Serial: driveOBDSerial.Serial,
Parallel: driveOBDParallel.Parallel,
Error: errStr,
}
obdInfo.Perf.DriveInfo = append(obdInfo.Perf.DriveInfo, driveOBD)
// Notify all other MinIO peers to report drive obd numbers
driveOBDs := globalNotificationSys.DriveOBDInfo(deadlinedCtx)
obdInfo.Perf.DriveInfo = append(obdInfo.Perf.DriveInfo, driveOBDs...)
partialWrite()
}
if net, ok := vars["perfnet"]; ok && net == "true" && globalIsDistXL {
obdInfo.Perf.Net = append(obdInfo.Perf.Net, globalNotificationSys.NetOBDInfo(deadlinedCtx))
obdInfo.Perf.Net = append(obdInfo.Perf.Net, globalNotificationSys.DispatchNetOBDInfo(deadlinedCtx)...)
partialWrite()
}
finish()
}
// ServerInfoHandler - GET /minio/admin/v2/info
// ----------
// Get server information

View File

@ -166,6 +166,12 @@ func registerAdminRouter(router *mux.Router, enableConfigOps, enableIAMOps bool)
//
adminRouter.Methods(http.MethodGet).Path(adminAPIVersionPrefix + "/kms/key/status").HandlerFunc(httpTraceAll(adminAPI.KMSKeyStatusHandler))
if !globalIsGateway {
// -- OBD API --
adminRouter.Methods(http.MethodGet).Path(adminAPIVersionPrefix+"/obdinfo").HandlerFunc(httpTraceHdrs(adminAPI.OBDInfoHandler)).Queries("perfdrive", "{perfdrive:true|false}", "perfnet", "{perfnet:true|false}", "minioinfo", "{minioinfo:true|false}", "minioconfig", "{minioconfig:true|false}", "syscpu", "{syscpu:true|false}", "sysdiskhw", "{sysdiskhw:true|false}", "sysosinfo", "{sysosinfo:true|false}", "sysmem", "{sysmem:true|false}", "sysprocess", "{sysprocess:true|false}")
}
// If none of the routes match add default error handler routes
adminRouter.NotFoundHandler = http.HandlerFunc(httpTraceAll(errorResponseHandler))
adminRouter.MethodNotAllowedHandler = http.HandlerFunc(httpTraceAll(errorResponseHandler))

View File

@ -25,6 +25,7 @@ import (
"net"
"net/url"
"path"
"sort"
"strings"
"sync"
"time"
@ -38,6 +39,7 @@ import (
objectlock "github.com/minio/minio/pkg/bucket/object/lock"
"github.com/minio/minio/pkg/bucket/policy"
"github.com/minio/minio-go/v6/pkg/set"
"github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/madmin"
xnet "github.com/minio/minio/pkg/net"
@ -890,6 +892,290 @@ func (sys *NotificationSys) CollectNetPerfInfo(size int64) map[string][]ServerNe
return reply
}
// NetOBDInfo - Net OBD information
func (sys *NotificationSys) NetOBDInfo(ctx context.Context) madmin.ServerNetOBDInfo {
var sortedGlobalEndpoints []string
/*
Ensure that only untraversed links are visited by this server
i.e. if netOBD tests have been performed between a -> b, then do
not run it between b -> a
The graph of tests looks like this
a b c d
a | o | x | x | x |
b | o | o | x | x |
c | o | o | o | x |
d | o | o | o | o |
'x's should be tested, and 'o's should be skipped
*/
stripPath := func(hostPath string) string {
return strings.Split(hostPath, slashSeparator)[0]
}
hostSet := set.NewStringSet()
for _, ez := range globalEndpoints {
for _, e := range ez.Endpoints {
host := stripPath(e.Host)
if hostSet.Contains(host) {
sortedGlobalEndpoints = append(sortedGlobalEndpoints, host)
hostSet.Add(host)
}
}
}
sort.Strings(sortedGlobalEndpoints)
var remoteTargets []*peerRESTClient
search := func(host string) *peerRESTClient {
for index, client := range sys.peerClients {
if client == nil {
continue
}
if sys.peerClients[index].host.String() == host {
return client
}
}
return nil
}
for i := 0; i < len(sortedGlobalEndpoints); i++ {
if sortedGlobalEndpoints[i] != GetLocalPeer(globalEndpoints) {
continue
}
for j := 0; j < len(sortedGlobalEndpoints); j++ {
if j > i {
remoteTarget := search(sortedGlobalEndpoints[j])
if remoteTarget != nil {
remoteTargets = append(remoteTargets, remoteTarget)
}
}
}
}
netOBDs := make([]madmin.NetOBDInfo, len(remoteTargets))
for index, client := range remoteTargets {
if client == nil {
continue
}
var err error
netOBDs[index], err = client.NetOBDInfo(ctx)
addr := client.host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
netOBDs[index].Addr = addr
if err != nil {
netOBDs[index].Error = err.Error()
}
}
return madmin.ServerNetOBDInfo{
Net: netOBDs,
Addr: GetLocalPeer(globalEndpoints),
}
}
// DispatchNetOBDInfo - Net OBD information from other nodes
func (sys *NotificationSys) DispatchNetOBDInfo(ctx context.Context) []madmin.ServerNetOBDInfo {
serverNetOBDs := []madmin.ServerNetOBDInfo{}
for index, client := range sys.peerClients {
if client == nil {
continue
}
serverNetOBD, err := sys.peerClients[index].DispatchNetOBDInfo(ctx)
if err != nil {
serverNetOBD.Addr = client.host.String()
serverNetOBD.Error = err.Error()
}
serverNetOBDs = append(serverNetOBDs, serverNetOBD)
}
return serverNetOBDs
}
// DriveOBDInfo - Drive OBD information
func (sys *NotificationSys) DriveOBDInfo(ctx context.Context) []madmin.ServerDrivesOBDInfo {
reply := make([]madmin.ServerDrivesOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].DriveOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// CPUOBDInfo - CPU OBD information
func (sys *NotificationSys) CPUOBDInfo(ctx context.Context) []madmin.ServerCPUOBDInfo {
reply := make([]madmin.ServerCPUOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].CPUOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// DiskHwOBDInfo - Disk HW OBD information
func (sys *NotificationSys) DiskHwOBDInfo(ctx context.Context) []madmin.ServerDiskHwOBDInfo {
reply := make([]madmin.ServerDiskHwOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].DiskHwOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// OsOBDInfo - Os OBD information
func (sys *NotificationSys) OsOBDInfo(ctx context.Context) []madmin.ServerOsOBDInfo {
reply := make([]madmin.ServerOsOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].OsOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// MemOBDInfo - Mem OBD information
func (sys *NotificationSys) MemOBDInfo(ctx context.Context) []madmin.ServerMemOBDInfo {
reply := make([]madmin.ServerMemOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].MemOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// ProcOBDInfo - Process OBD information
func (sys *NotificationSys) ProcOBDInfo(ctx context.Context) []madmin.ServerProcOBDInfo {
reply := make([]madmin.ServerProcOBDInfo, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
var err error
reply[index], err = sys.peerClients[index].ProcOBDInfo(ctx)
return err
}, index)
}
for index, err := range g.Wait() {
if err != nil {
addr := sys.peerClients[index].host.String()
reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, err)
reply[index].Addr = addr
reply[index].Error = err.Error()
}
}
return reply
}
// DrivePerfInfo - Drive speed (read and write) information
func (sys *NotificationSys) DrivePerfInfo(size int64) []madmin.ServerDrivesPerfInfo {
reply := make([]madmin.ServerDrivesPerfInfo, len(sys.peerClients))

422
cmd/obdinfo.go Normal file
View File

@ -0,0 +1,422 @@
/*
* 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 cmd
import (
"context"
"net/http"
"os"
"sync"
"syscall"
"github.com/minio/minio/pkg/disk"
"github.com/minio/minio/pkg/madmin"
cpuhw "github.com/shirou/gopsutil/cpu"
"github.com/shirou/gopsutil/host"
memhw "github.com/shirou/gopsutil/mem"
"github.com/shirou/gopsutil/process"
)
func getLocalCPUOBDInfo(ctx context.Context) madmin.ServerCPUOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
info, err := cpuhw.InfoWithContext(ctx)
if err != nil {
return madmin.ServerCPUOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
time, err := cpuhw.TimesWithContext(ctx, false)
if err != nil {
return madmin.ServerCPUOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
return madmin.ServerCPUOBDInfo{
Addr: addr,
CPUStat: info,
TimeStat: time,
Error: "",
}
}
func getLocalDrivesOBD(ctx context.Context, parallel bool, endpointZones EndpointZones, r *http.Request) madmin.ServerDrivesOBDInfo {
var drivesOBDInfo []madmin.DriveOBDInfo
wg := sync.WaitGroup{}
for _, ep := range endpointZones {
for i, endpoint := range ep.Endpoints {
// Only proceed for local endpoints
if endpoint.IsLocal {
if _, err := os.Stat(endpoint.Path); err != nil {
// Since this drive is not available, add relevant details and proceed
drivesOBDInfo = append(drivesOBDInfo, madmin.DriveOBDInfo{
Path: endpoint.Path,
Error: err.Error(),
})
continue
}
measure := func(index int) {
latency, throughput, err := disk.GetOBDInfo(ctx, pathJoin(endpoint.Path, minioMetaTmpBucket, mustGetUUID()))
driveOBDInfo := madmin.DriveOBDInfo{
Path: endpoint.Path,
Latency: latency,
Throughput: throughput,
}
if err != nil {
driveOBDInfo.Error = err.Error()
}
drivesOBDInfo = append(drivesOBDInfo, driveOBDInfo)
wg.Done()
}
wg.Add(1)
if parallel {
go measure(i)
} else {
measure(i)
}
}
}
}
wg.Wait()
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(endpointZones)
} else {
addr = "minio"
}
if parallel {
return madmin.ServerDrivesOBDInfo{
Addr: addr,
Parallel: drivesOBDInfo,
}
}
return madmin.ServerDrivesOBDInfo{
Addr: addr,
Serial: drivesOBDInfo,
}
}
func getLocalMemOBD(ctx context.Context) madmin.ServerMemOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
swap, err := memhw.SwapMemoryWithContext(ctx)
if err != nil {
return madmin.ServerMemOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
vm, err := memhw.VirtualMemoryWithContext(ctx)
if err != nil {
return madmin.ServerMemOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
return madmin.ServerMemOBDInfo{
Addr: addr,
SwapMem: swap,
VirtualMem: vm,
Error: "",
}
}
func getLocalProcOBD(ctx context.Context) madmin.ServerProcOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
errProcInfo := func(err error) madmin.ServerProcOBDInfo {
return madmin.ServerProcOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
selfPid := int32(syscall.Getpid())
self, err := process.NewProcess(selfPid)
if err != nil {
return errProcInfo(err)
}
processes := []*process.Process{self}
if err != nil {
return errProcInfo(err)
}
sysProcs := []madmin.SysOBDProcess{}
for _, proc := range processes {
sysProc := madmin.SysOBDProcess{}
sysProc.Pid = proc.Pid
bg, err := proc.BackgroundWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Background = bg
cpuPercent, err := proc.CPUPercentWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.CPUPercent = cpuPercent
children, _ := proc.ChildrenWithContext(ctx)
for _, c := range children {
sysProc.Children = append(sysProc.Children, c.Pid)
}
cmdLine, err := proc.CmdlineWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.CmdLine = cmdLine
conns, err := proc.ConnectionsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Connections = conns
createTime, err := proc.CreateTimeWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.CreateTime = createTime
cwd, err := proc.CwdWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Cwd = cwd
exe, err := proc.ExeWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Exe = exe
gids, err := proc.GidsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Gids = gids
ioCounters, err := proc.IOCountersWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.IOCounters = ioCounters
isRunning, err := proc.IsRunningWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.IsRunning = isRunning
memInfo, err := proc.MemoryInfoWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.MemInfo = memInfo
memMaps, err := proc.MemoryMapsWithContext(ctx, true)
if err != nil {
return errProcInfo(err)
}
sysProc.MemMaps = memMaps
memPercent, err := proc.MemoryPercentWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.MemPercent = memPercent
name, err := proc.NameWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Name = name
netIOCounters, err := proc.NetIOCountersWithContext(ctx, false)
if err != nil {
return errProcInfo(err)
}
sysProc.NetIOCounters = netIOCounters
nice, err := proc.NiceWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Nice = nice
numCtxSwitches, err := proc.NumCtxSwitchesWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.NumCtxSwitches = numCtxSwitches
numFds, err := proc.NumFDsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.NumFds = numFds
numThreads, err := proc.NumThreadsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.NumThreads = numThreads
openFiles, err := proc.OpenFilesWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.OpenFiles = openFiles
pageFaults, err := proc.PageFaultsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.PageFaults = pageFaults
parent, err := proc.ParentWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Parent = parent.Pid
ppid, err := proc.PpidWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Ppid = ppid
rlimit, err := proc.RlimitWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Rlimit = rlimit
status, err := proc.StatusWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Status = status
tgid, err := proc.Tgid()
if err != nil {
return errProcInfo(err)
}
sysProc.Tgid = tgid
threads, err := proc.ThreadsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Threads = threads
times, err := proc.TimesWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Times = times
uids, err := proc.UidsWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Uids = uids
username, err := proc.UsernameWithContext(ctx)
if err != nil {
return errProcInfo(err)
}
sysProc.Username = username
sysProcs = append(sysProcs, sysProc)
}
return madmin.ServerProcOBDInfo{
Addr: addr,
Processes: sysProcs,
Error: "",
}
}
func getLocalOsInfoOBD(ctx context.Context) madmin.ServerOsOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
info, err := host.InfoWithContext(ctx)
if err != nil {
return madmin.ServerOsOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
sensors, err := host.SensorsTemperaturesWithContext(ctx)
if err != nil {
return madmin.ServerOsOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
users, err := host.UsersWithContext(ctx)
if err != nil {
return madmin.ServerOsOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
return madmin.ServerOsOBDInfo{
Addr: addr,
Info: info,
Sensors: sensors,
Users: users,
Error: "",
}
}

38
cmd/obdinfo_freebsd.go Normal file
View File

@ -0,0 +1,38 @@
/*
* 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 cmd
import (
"context"
"github.com/minio/minio/pkg/madmin"
)
func getLocalDiskHwOBD(ctx context.Context) madmin.ServerDiskHwOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
return madmin.ServerDiskHwOBDInfo{
Addr: addr,
Error: "unsupported platform",
}
}

86
cmd/obdinfo_other.go Normal file
View File

@ -0,0 +1,86 @@
// +build !freebsd
/*
* 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 cmd
import (
"context"
"strings"
"github.com/minio/minio/pkg/madmin"
diskhw "github.com/shirou/gopsutil/disk"
)
func getLocalDiskHwOBD(ctx context.Context) madmin.ServerDiskHwOBDInfo {
addr := ""
if globalIsDistXL {
addr = GetLocalPeer(globalEndpoints)
} else {
addr = "minio"
}
partitions, err := diskhw.PartitionsWithContext(ctx, true)
if err != nil {
return madmin.ServerDiskHwOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
drives := []string{}
paths := []string{}
for _, partition := range partitions {
device := partition.Device
path := partition.Mountpoint
if strings.Index(device, "/dev/") == 0 {
if strings.Contains(device, "loop") {
continue
}
drives = append(drives, device)
paths = append(paths, path)
}
}
ioCounters, err := diskhw.IOCountersWithContext(ctx, drives...)
if err != nil {
return madmin.ServerDiskHwOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
usages := []*diskhw.UsageStat{}
for _, path := range paths {
usage, err := diskhw.UsageWithContext(ctx, path)
if err != nil {
return madmin.ServerDiskHwOBDInfo{
Addr: addr,
Error: err.Error(),
}
}
usages = append(usages, usage)
}
return madmin.ServerDiskHwOBDInfo{
Addr: addr,
Usage: usages,
Partitions: partitions,
Counters: ioCounters,
Error: "",
}
}

View File

@ -22,9 +22,12 @@ import (
"crypto/tls"
"encoding/gob"
"io"
"io/ioutil"
"math"
"math/rand"
"net/url"
"strconv"
"sync"
"sync/atomic"
"time"
@ -41,7 +44,13 @@ import (
trace "github.com/minio/minio/pkg/trace"
)
// client to talk to peer NEndpoints.
const (
kiB int64 = 1 << 10
miB int64 = kiB << 10
giB int64 = miB << 10
)
// client to talk to peer Nodes.
type peerRESTClient struct {
host *xnet.Host
restClient *rest.Client
@ -190,6 +199,321 @@ func (client *peerRESTClient) NetworkInfo() (info madmin.ServerNetworkHardwareIn
return info, err
}
type networkOverloadedErr struct{}
var networkOverloaded networkOverloadedErr
func (n networkOverloadedErr) Error() string {
return "network overloaded"
}
type progressReader struct {
r io.Reader
progressChan chan int64
}
func (p *progressReader) Read(b []byte) (int, error) {
n, err := p.r.Read(b)
if err != nil && err != io.EOF {
return n, err
}
p.progressChan <- int64(n)
return n, err
}
func (client *peerRESTClient) doNetOBDTest(ctx context.Context, dataSize int64, threadCount uint) (info madmin.NetOBDInfo, err error) {
latencies := []float64{}
throughputs := []float64{}
buf := make([]byte, dataSize)
buflimiter := make(chan struct{}, threadCount)
errChan := make(chan error, threadCount)
totalTransferred := int64(0)
transferChan := make(chan int64, threadCount)
go func() {
for v := range transferChan {
atomic.AddInt64(&totalTransferred, v)
}
}()
// 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()
}
}
wg := sync.WaitGroup{}
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) {
bufReader := bytes.NewReader(buf)
bufReadCloser := ioutil.NopCloser(&progressReader{
r: bufReader,
progressChan: transferChan,
})
start := time.Now()
before := atomic.LoadInt64(&totalTransferred)
ctx, cancel := context.WithTimeout(innerCtx, 10*time.Second)
defer cancel()
respBody, err := client.callWithContext(ctx, peerRESTMethodNetOBDInfo, nil, bufReadCloser, dataSize)
if err != nil {
if netErr, ok := err.(*rest.NetworkError); ok {
if urlErr, ok := netErr.Err.(*url.Error); ok {
if urlErr.Err.Error() == context.DeadlineExceeded.Error() {
slowSample()
finish()
return
}
}
}
errChan <- err
finish()
return
}
http.DrainBody(respBody)
after := atomic.LoadInt64(&totalTransferred)
finish()
end := time.Now()
latency := float64(end.Sub(start).Seconds())
if latency > maxLatencyForSizeThreads(dataSize, threadCount) {
slowSample()
}
/* Throughput = (total data transferred across all threads / time taken) */
throughput := float64(float64((after - before)) / latency)
latencies = append(latencies, latency)
throughputs = append(throughputs, throughput)
}(i)
}
}
wg.Wait()
if err != nil {
return info, err
}
latency, throughput, err := xnet.ComputeOBDStats(latencies, throughputs)
info = madmin.NetOBDInfo{
Latency: latency,
Throughput: throughput,
}
return info, err
}
func maxLatencyForSizeThreads(size int64, threadCount uint) float64 {
Gbit100 := 12.5 * float64(giB)
Gbit40 := 5.00 * float64(giB)
Gbit25 := 3.25 * float64(giB)
Gbit10 := 1.25 * float64(giB)
// Gbit1 := 0.25 * float64(giB)
// 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
throughput := float64(int64(size) * int64(threadCount))
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
}
// NetOBDInfo - fetch Net OBD information for a remote node.
func (client *peerRESTClient) NetOBDInfo(ctx context.Context) (info madmin.NetOBDInfo, err error) {
// 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
size: 256 * miB,
threads: 50,
},
{ // 40 Gbit
size: 256 * miB,
threads: 20,
},
{ // 25 Gbit
size: 128 * miB,
threads: 25,
},
{ // 10 Gbit
size: 128 * miB,
threads: 10,
},
{ // 1 Gbit
size: 64 * miB,
threads: 2,
},
}
for i := range steps {
size := steps[i].size
threads := steps[i].threads
if info, err = client.doNetOBDTest(ctx, size, threads); err != nil {
if err == networkOverloaded {
continue
}
if netErr, ok := err.(*rest.NetworkError); ok {
if urlErr, ok := netErr.Err.(*url.Error); ok {
if urlErr.Err.Error() == context.Canceled.Error() {
continue
}
if urlErr.Err.Error() == context.DeadlineExceeded.Error() {
continue
}
}
}
}
return info, err
}
return info, err
}
// DispatchNetOBDInfo - dispatch other nodes to run Net OBD.
func (client *peerRESTClient) DispatchNetOBDInfo(ctx context.Context) (info madmin.ServerNetOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodDispatchNetOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return
}
// DriveOBDInfo - fetch Drive OBD information for a remote node.
func (client *peerRESTClient) DriveOBDInfo(ctx context.Context) (info madmin.ServerDrivesOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodDriveOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// CPUOBDInfo - fetch CPU OBD information for a remote node.
func (client *peerRESTClient) CPUOBDInfo(ctx context.Context) (info madmin.ServerCPUOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodCPUOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// DiskHwOBDInfo - fetch Disk HW OBD information for a remote node.
func (client *peerRESTClient) DiskHwOBDInfo(ctx context.Context) (info madmin.ServerDiskHwOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodDiskHwOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// OsOBDInfo - fetch OsInfo OBD information for a remote node.
func (client *peerRESTClient) OsOBDInfo(ctx context.Context) (info madmin.ServerOsOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodOsInfoOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// MemOBDInfo - fetch MemInfo OBD information for a remote node.
func (client *peerRESTClient) MemOBDInfo(ctx context.Context) (info madmin.ServerMemOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodMemOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// ProcOBDInfo - fetch ProcInfo OBD information for a remote node.
func (client *peerRESTClient) ProcOBDInfo(ctx context.Context) (info madmin.ServerProcOBDInfo, err error) {
respBody, err := client.callWithContext(ctx, peerRESTMethodProcOBDInfo, nil, nil, -1)
if err != nil {
return
}
defer http.DrainBody(respBody)
err = gob.NewDecoder(respBody).Decode(&info)
return info, err
}
// DrivePerfInfo - fetch Drive performance information for a remote node.
func (client *peerRESTClient) DrivePerfInfo(size int64) (info madmin.ServerDrivesPerfInfo, err error) {
params := make(url.Values)

View File

@ -17,7 +17,7 @@
package cmd
const (
peerRESTVersion = "v6"
peerRESTVersion = "v7"
peerRESTVersionPrefix = SlashSeparator + peerRESTVersion
peerRESTPrefix = minioReservedBucketPath + "/peer"
peerRESTPath = peerRESTPrefix + peerRESTVersionPrefix
@ -30,6 +30,14 @@ const (
peerRESTMethodCPULoadInfo = "/cpuloadinfo"
peerRESTMethodMemUsageInfo = "/memusageinfo"
peerRESTMethodDrivePerfInfo = "/driveperfinfo"
peerRESTMethodDriveOBDInfo = "/driveobdinfo"
peerRESTMethodNetOBDInfo = "/netobdinfo"
peerRESTMethodCPUOBDInfo = "/cpuobdinfo"
peerRESTMethodDiskHwOBDInfo = "/diskhwobdinfo"
peerRESTMethodOsInfoOBDInfo = "/osinfoobdinfo"
peerRESTMethodMemOBDInfo = "/memobdinfo"
peerRESTMethodProcOBDInfo = "/procobdinfo"
peerRESTMethodDispatchNetOBDInfo = "/dispatchnetobdinfo"
peerRESTMethodDeleteBucket = "/deletebucket"
peerRESTMethodServerUpdate = "/serverupdate"
peerRESTMethodSignalService = "/signalservice"

View File

@ -36,6 +36,7 @@ import (
objectlock "github.com/minio/minio/pkg/bucket/object/lock"
"github.com/minio/minio/pkg/bucket/policy"
"github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/madmin"
trace "github.com/minio/minio/pkg/trace"
)
@ -509,6 +510,157 @@ func (s *peerRESTServer) ServerInfoHandler(w http.ResponseWriter, r *http.Reques
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
func (s *peerRESTServer) NetOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "NetOBDInfo")
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
// Use this trailer to send additional headers after sending body
w.Header().Set("Trailer", "FinalStatus")
w.Header().Set("Content-Type", "application/octet-stream")
w.WriteHeader(http.StatusOK)
n, err := io.Copy(ioutil.Discard, r.Body)
if err == io.ErrUnexpectedEOF {
w.Header().Set("FinalStatus", err.Error())
return
}
if err != nil && err != io.EOF {
logger.LogIf(ctx, err)
w.Header().Set("FinalStatus", err.Error())
return
}
if n != r.ContentLength {
err := fmt.Errorf("OBD: short read: expected %d found %d", r.ContentLength, n)
logger.LogIf(ctx, err)
w.Header().Set("FinalStatus", err.Error())
return
}
w.Header().Set("FinalStatus", "Success")
w.(http.Flusher).Flush()
}
func (s *peerRESTServer) DispatchNetOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx := newContext(r, w, "DispatchNetOBDInfo")
info := globalNotificationSys.NetOBDInfo(ctx)
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
w.(http.Flusher).Flush()
}
// DriveOBDInfoHandler - returns Drive OBD info.
func (s *peerRESTServer) DriveOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "DriveOBDInfo"))
defer cancel()
infoSerial := getLocalDrivesOBD(ctx, false, globalEndpoints, r)
infoParallel := getLocalDrivesOBD(ctx, true, globalEndpoints, r)
errStr := ""
if infoSerial.Error != "" {
errStr = "serial: " + infoSerial.Error
}
if infoParallel.Error != "" {
errStr = errStr + " parallel: " + infoParallel.Error
}
info := madmin.ServerDrivesOBDInfo{
Addr: infoSerial.Addr,
Serial: infoSerial.Serial,
Parallel: infoParallel.Parallel,
Error: errStr,
}
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// CPUOBDInfoHandler - returns CPU OBD info.
func (s *peerRESTServer) CPUOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "CpuOBDInfo"))
defer cancel()
info := getLocalCPUOBDInfo(ctx)
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// DiskHwOBDInfoHandler - returns Disk HW OBD info.
func (s *peerRESTServer) DiskHwOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "DiskHwOBDInfo"))
defer cancel()
info := getLocalDiskHwOBD(ctx)
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// OsOBDInfoHandler - returns Os OBD info.
func (s *peerRESTServer) OsOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "OsOBDInfo"))
defer cancel()
info := getLocalOsInfoOBD(ctx)
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// ProcOBDInfoHandler - returns Proc OBD info.
func (s *peerRESTServer) ProcOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "ProcOBDInfo"))
defer cancel()
info := getLocalProcOBD(ctx)
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// MemOBDInfoHandler - returns Mem OBD info.
func (s *peerRESTServer) MemOBDInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
ctx, cancel := context.WithCancel(newContext(r, w, "MemOBDInfo"))
defer cancel()
info := getLocalMemOBD(ctx)
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(info))
}
// DrivePerfInfoHandler - returns Drive Performance info.
func (s *peerRESTServer) DrivePerfInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
@ -1184,6 +1336,14 @@ func registerPeerRESTHandlers(router *mux.Router) {
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodServerInfo).HandlerFunc(httpTraceHdrs(server.ServerInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodCPULoadInfo).HandlerFunc(httpTraceHdrs(server.CPULoadInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodMemUsageInfo).HandlerFunc(httpTraceHdrs(server.MemUsageInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodProcOBDInfo).HandlerFunc(httpTraceHdrs(server.ProcOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodMemOBDInfo).HandlerFunc(httpTraceHdrs(server.MemOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodOsInfoOBDInfo).HandlerFunc(httpTraceHdrs(server.OsOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodDiskHwOBDInfo).HandlerFunc(httpTraceHdrs(server.DiskHwOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodCPUOBDInfo).HandlerFunc(httpTraceHdrs(server.CPUOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodDriveOBDInfo).HandlerFunc(httpTraceHdrs(server.DriveOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodNetOBDInfo).HandlerFunc(httpTraceHdrs(server.NetOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodDispatchNetOBDInfo).HandlerFunc(httpTraceHdrs(server.DispatchNetOBDInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodDrivePerfInfo).HandlerFunc(httpTraceHdrs(server.DrivePerfInfoHandler)).Queries(restQueries(peerRESTDrivePerfSize)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodHardwareCPUInfo).HandlerFunc(httpTraceHdrs(server.CPUInfoHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodHardwareNetworkInfo).HandlerFunc(httpTraceHdrs(server.NetworkInfoHandler))

View File

@ -73,6 +73,12 @@ func (c *Client) CallWithContext(ctx context.Context, method string, values url.
return nil, &NetworkError{err}
}
final := resp.Trailer.Get("FinalStatus")
if final != "" && final != "Success" {
defer xhttp.DrainBody(resp.Body)
return nil, errors.New(final)
}
if resp.StatusCode != http.StatusOK {
defer xhttp.DrainBody(resp.Body)
// Limit the ReadAll(), just in case, because of a bug, the server responds with large data.

View File

@ -78,13 +78,14 @@ EXAMPLES:
2. Start distributed minio server on an 32 node setup with 32 drives each, run following command on all the nodes
{{.Prompt}} {{.EnvVarSetCommand}} MINIO_ACCESS_KEY{{.AssignmentOperator}}minio
{{.Prompt}} {{.EnvVarSetCommand}} MINIO_SECRET_KEY{{.AssignmentOperator}}miniostorage
{{.Prompt}} {{.HelpName}} http://node{1...32}.example.com/mnt/export/{1...32}
{{.Prompt}} {{.HelpName}} http://node{1...32}.example.com/mnt/export{1...32}
3. Start distributed minio server in an expanded setup, run the following command on all the nodes
{{.Prompt}} {{.EnvVarSetCommand}} MINIO_ACCESS_KEY{{.AssignmentOperator}}minio
{{.Prompt}} {{.EnvVarSetCommand}} MINIO_SECRET_KEY{{.AssignmentOperator}}miniostorage
{{.Prompt}} {{.HelpName}} http://node{1...16}.example.com/mnt/export/{1...32} \
http://node{17...64}.example.com/mnt/export/{1...64}
{{.Prompt}} {{.HelpName}} http://node{1...16}.example.com/mnt/export{1...32} \
http://node{17...64}.example.com/mnt/export{1...64}
`,
}

View File

@ -49,6 +49,10 @@ import (
"github.com/gorilla/mux"
)
const (
slashSeparator = "/"
)
// IsErrIgnored returns whether given error is ignored or not.
func IsErrIgnored(err error, ignoredErrs ...error) bool {
return IsErr(err, ignoredErrs...)

View File

@ -23,3 +23,37 @@ To trace entire HTTP request and also internode communication
mc admin trace --all --verbose myminio
```
### On-board Diagnostics
On-board diagnostics help ensure that the underlying infrastructure that runs MinIO is configured correctly, and is functioning properly. This test is one-shot long running one, that is recommended to be run as soon as the cluster is first provisioned, and each time a failure scenrio is encountered. Note that the test incurs majority of the available resources on the system. Care must be taken when using this to debug failure scenario, so as to prevent larger outages. OBD tests can be triggered using [`mc admin obd`](https://github.com/minio/mc/blob/master/docs/minio-admin-complete-guide.md#command-obd---display-minio-server-obd) command.
Example:
```sh
minio server /data
```
The command takes no flags
```sh
mc admin obd myminio
```
The output printed will be of the form
```sh
● Admin Info ... ✔
● CPU ... ✔
● Disk Hardware ... ✔
● Os Info ... ✔
● Mem Info ... ✔
● Process Info ... ✔
● Config ... ✔
● Drive ... ✔
● Net ... ✔
*********************************************************************************
WARNING!!
** THIS FILE MAY CONTAIN SENSITIVE INFORMATION ABOUT YOUR ENVIRONMENT **
** PLEASE INSPECT CONTENTS BEFORE SHARING IT ON ANY PUBLIC FORUM **
*********************************************************************************
OBD data saved to dc-11-obd_20200321053323.json.gz
```
The gzipped output contains debugging information for your system

5
go.mod
View File

@ -77,6 +77,7 @@ require (
github.com/mmcloughlin/avo v0.0.0-20200303042253-6df701fe672f // indirect
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
github.com/modern-go/reflect2 v1.0.1 // indirect
github.com/montanaflynn/stats v0.5.0
github.com/nats-io/gnatsd v1.4.1 // indirect
github.com/nats-io/go-nats v1.7.2 // indirect
github.com/nats-io/go-nats-streaming v0.4.4 // indirect
@ -96,7 +97,7 @@ require (
github.com/rs/cors v1.6.0
github.com/satori/go.uuid v1.2.0 // indirect
github.com/secure-io/sio-go v0.3.0
github.com/shirou/gopsutil v2.18.12+incompatible
github.com/shirou/gopsutil v2.20.3-0.20200314133625-53cec6b37e6a+incompatible
github.com/sirupsen/logrus v1.4.2
github.com/skyrings/skyring-common v0.0.0-20160929130248-d1c0bb1cbd5e
github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3 // indirect
@ -115,7 +116,7 @@ require (
golang.org/x/crypto v0.0.0-20191117063200-497ca9f6d64f
golang.org/x/net v0.0.0-20190923162816-aa69164e4478 // indirect
golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a // indirect
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69
golang.org/x/sys v0.0.0-20200320181252-af34d8274f85
golang.org/x/text v0.3.2 // indirect
google.golang.org/api v0.5.0
google.golang.org/appengine v1.6.0 // indirect

8
go.sum
View File

@ -306,6 +306,8 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ
github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk=
github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc=
github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/nats-io/gnatsd v1.4.1 h1:RconcfDeWpKCD6QIIwiVFcvForlXpWeJP7i5/lDLy44=
github.com/nats-io/gnatsd v1.4.1/go.mod h1:nqco77VO78hLCJpIcVfygDP2rPGfsEHkGTUk94uh5DQ=
@ -389,6 +391,10 @@ github.com/secure-io/sio-go v0.3.0 h1:QKGb6rGJeiExac9wSWxnWPYo8O8OFN7lxXQvHshX6v
github.com/secure-io/sio-go v0.3.0/go.mod h1:D3KmXgKETffyYxBdFRN+Hpd2WzhzqS0EQwT3XWsAcBU=
github.com/shirou/gopsutil v2.18.12+incompatible h1:1eaJvGomDnH74/5cF4CTmTbLHAriGFsTZppLXDX93OM=
github.com/shirou/gopsutil v2.18.12+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shirou/gopsutil v2.20.2+incompatible h1:ucK79BhBpgqQxPASyS2cu9HX8cfDVljBN1WWFvbNvgY=
github.com/shirou/gopsutil v2.20.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shirou/gopsutil v2.20.3-0.20200314133625-53cec6b37e6a+incompatible h1:YiKUe2ZOmfpDBH4OSyxwkx/mjNqHHnNhOtZ2mPyRme8=
github.com/shirou/gopsutil v2.20.3-0.20200314133625-53cec6b37e6a+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4=
github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
@ -502,6 +508,8 @@ golang.org/x/sys v0.0.0-20190523142557-0e01d883c5c5/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69 h1:rOhMmluY6kLMhdnrivzec6lLgaVbMHMn2ISQXJeJ5EM=
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200320181252-af34d8274f85 h1:fD99hd4ciR6T3oPhr2EkmuKe9oHixHx9Hj/hND89j3g=
golang.org/x/sys v0.0.0-20200320181252-af34d8274f85/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.1-0.20181227161524-e6919f6577db/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=

184
pkg/disk/obd.go Normal file
View File

@ -0,0 +1,184 @@
/*
* 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 disk
import (
"context"
"fmt"
"os"
"path/filepath"
"runtime"
"time"
"github.com/montanaflynn/stats"
)
const (
kb = uint64(1 << 10)
mb = uint64(kb << 10)
gb = uint64(mb << 10)
)
var globalLatency = map[string]Latency{}
var globalThroughput = map[string]Throughput{}
// Latency holds latency information for write operations to the drive
type Latency struct {
Avg float64 `json:"avg_secs,omitempty"`
Percentile50 float64 `json:"percentile50_secs,omitempty"`
Percentile90 float64 `json:"percentile90_secs,omitempty"`
Percentile99 float64 `json:"percentile99_secs,omitempty"`
Min float64 `json:"min_secs,omitempty"`
Max float64 `json:"max_secs,omitempty"`
}
// Throughput holds throughput information for write operations to the drive
type Throughput struct {
Avg float64 `json:"avg_bytes_per_sec,omitempty"`
Percentile50 float64 `json:"percentile50_bytes_per_sec,omitempty"`
Percentile90 float64 `json:"percentile90_bytes_per_sec,omitempty"`
Percentile99 float64 `json:"percentile99_bytes_per_sec,omitempty"`
Min float64 `json:"min_bytes_per_sec,omitempty"`
Max float64 `json:"max_bytes_per_sec,omitempty"`
}
// GetOBDInfo about the drive
func GetOBDInfo(ctx context.Context, endpoint string) (Latency, Throughput, error) {
runtime.LockOSThread()
f, err := OpenFileDirectIO(endpoint, os.O_CREATE|os.O_RDWR|os.O_TRUNC, 0755)
if err != nil {
return Latency{}, Throughput{}, err
}
defer func() {
f.Close()
os.Remove(f.Name())
}()
drive := filepath.Dir(endpoint)
// going to leave this here incase we decide to go back to caching again
// if gl, ok := globalLatency[drive]; ok {
// if gt, ok := globalThroughput[drive]; ok {
// return gl, gt, nil
// }
// }
blockSize := 1 * mb
fileSize := 256 * mb
latencies := make([]float64, fileSize/blockSize)
throughputs := make([]float64, fileSize/blockSize)
dioFile := os.NewFile(uintptr(f.Fd()), endpoint)
data := make([]byte, blockSize)
for i := uint64(0); i < (fileSize / blockSize); i++ {
if ctx.Err() != nil {
return Latency{}, Throughput{}, ctx.Err()
}
startTime := time.Now()
if n, err := dioFile.Write(data); err != nil {
return Latency{}, Throughput{}, err
} else if uint64(n) != blockSize {
return Latency{}, Throughput{}, fmt.Errorf("Expected to write %d, but only wrote %d", blockSize, n)
}
latency := time.Since(startTime)
latencies[i] = float64(latency.Seconds())
}
runtime.UnlockOSThread()
for i := range latencies {
throughput := float64(blockSize) / latencies[i]
throughputs[i] = throughput
}
var avgLatency float64
var percentile50Latency float64
var percentile90Latency float64
var percentile99Latency float64
var minLatency float64
var maxLatency float64
var avgThroughput float64
var percentile50Throughput float64
var percentile90Throughput float64
var percentile99Throughput float64
var minThroughput float64
var maxThroughput float64
if avgLatency, err = stats.Mean(latencies); err != nil {
return Latency{}, Throughput{}, err
}
if percentile50Latency, err = stats.Percentile(latencies, 50); err != nil {
return Latency{}, Throughput{}, err
}
if percentile90Latency, err = stats.Percentile(latencies, 90); err != nil {
return Latency{}, Throughput{}, err
}
if percentile99Latency, err = stats.Percentile(latencies, 99); err != nil {
return Latency{}, Throughput{}, err
}
if maxLatency, err = stats.Max(latencies); err != nil {
return Latency{}, Throughput{}, err
}
if minLatency, err = stats.Min(latencies); err != nil {
return Latency{}, Throughput{}, err
}
l := Latency{
Avg: avgLatency,
Percentile50: percentile50Latency,
Percentile90: percentile90Latency,
Percentile99: percentile99Latency,
Min: minLatency,
Max: maxLatency,
}
if avgThroughput, err = stats.Mean(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
if percentile50Throughput, err = stats.Percentile(throughputs, 50); err != nil {
return Latency{}, Throughput{}, err
}
if percentile90Throughput, err = stats.Percentile(throughputs, 90); err != nil {
return Latency{}, Throughput{}, err
}
if percentile99Throughput, err = stats.Percentile(throughputs, 99); err != nil {
return Latency{}, Throughput{}, err
}
if maxThroughput, err = stats.Max(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
if minThroughput, err = stats.Min(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
t := Throughput{
Avg: avgThroughput,
Percentile50: percentile50Throughput,
Percentile90: percentile90Throughput,
Percentile99: percentile99Throughput,
Min: minThroughput,
Max: maxThroughput,
}
globalLatency[drive] = l
globalThroughput[drive] = t
return l, t, nil
}

View File

@ -51,6 +51,8 @@ const (
ServerHardwareInfoAdminAction = "admin:HardwareInfo"
// ServerInfoAdminAction - allow listing server info
ServerInfoAdminAction = "admin:ServerInfo"
// OBDInfoAdminAction - allow obtaining cluster on-board diagnostics
OBDInfoAdminAction = "admin:OBDInfo"
// ServerUpdateAdminAction - allow MinIO binary update
ServerUpdateAdminAction = "admin:ServerUpdate"

291
pkg/madmin/obd.go Normal file
View File

@ -0,0 +1,291 @@
/*
* 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"
"time"
"github.com/minio/minio/pkg/disk"
"github.com/minio/minio/pkg/net"
"github.com/shirou/gopsutil/cpu"
"github.com/shirou/gopsutil/host"
"github.com/shirou/gopsutil/mem"
nethw "github.com/shirou/gopsutil/net"
"github.com/shirou/gopsutil/process"
)
// OBDInfo - MinIO cluster's OBD Info
type OBDInfo struct {
TimeStamp time.Time `json:"timestamp,omitempty"`
Error string `json:"error,omitempty"`
Perf PerfOBDInfo `json:"perf,omitempty"`
Minio MinioOBDInfo `json:"minio,omitempty"`
Sys SysOBDInfo `json:"sys,omitempty"`
}
// SysOBDInfo - Includes hardware and system information of the MinIO cluster
type SysOBDInfo struct {
CPUInfo []ServerCPUOBDInfo `json:"cpus,omitempty"`
DiskHwInfo []ServerDiskHwOBDInfo `json:"disks,omitempty"`
OsInfo []ServerOsOBDInfo `json:"osinfos,omitempty"`
MemInfo []ServerMemOBDInfo `json:"meminfos,omitempty"`
ProcInfo []ServerProcOBDInfo `json:"procinfos,omitempty"`
Error string `json:"error,omitempty"`
}
// ServerProcOBDInfo - Includes host process lvl information
type ServerProcOBDInfo struct {
Addr string `json:"addr"`
Processes []SysOBDProcess `json:"processes,omitempty"`
Error string `json:"error,omitempty"`
}
// SysOBDProcess - Includes process lvl information about a single process
type SysOBDProcess struct {
Pid int32 `json:"pid"`
Background bool `json:"background,omitempty"`
CPUPercent float64 `json:"cpupercent,omitempty"`
Children []int32 `json:"children,omitempty"`
CmdLine string `json:"cmd,omitempty"`
Connections []nethw.ConnectionStat `json:"connections,omitempty"`
CreateTime int64 `json:"createtime,omitempty"`
Cwd string `json:"cwd,omitempty"`
Exe string `json:"exe,omitempty"`
Gids []int32 `json:"gids,omitempty"`
IOCounters *process.IOCountersStat `json:"iocounters,omitempty"`
IsRunning bool `json:"isrunning,omitempty"`
MemInfo *process.MemoryInfoStat `json:"meminfo,omitempty"`
MemMaps *[]process.MemoryMapsStat `json:"memmaps,omitempty"`
MemPercent float32 `json:"mempercent,omitempty"`
Name string `json:"name,omitempty"`
NetIOCounters []nethw.IOCountersStat `json:"netiocounters,omitempty"`
Nice int32 `json:"nice,omitempty"`
NumCtxSwitches *process.NumCtxSwitchesStat `json:"numctxswitches,omitempty"`
NumFds int32 `json:"numfds,omitempty"`
NumThreads int32 `json:"numthreads,omitempty"`
OpenFiles []process.OpenFilesStat `json:"openfiles,omitempty"`
PageFaults *process.PageFaultsStat `json:"pagefaults,omitempty"`
Parent int32 `json:"parent,omitempty"`
Ppid int32 `json:"ppid,omitempty"`
Rlimit []process.RlimitStat `json:"rlimit,omitempty"`
Status string `json:"status,omitempty"`
Tgid int32 `json:"tgid,omitempty"`
Threads map[int32]*cpu.TimesStat `json:"threadstats,omitempty"`
Times *cpu.TimesStat `json:"cputimes,omitempty"`
Uids []int32 `json:"uidsomitempty"`
Username string `json:"username,omitempty"`
}
// ServerMemOBDInfo - Includes host virtual and swap mem information
type ServerMemOBDInfo struct {
Addr string `json:"addr"`
SwapMem *mem.SwapMemoryStat `json:"swap,omitempty"`
VirtualMem *mem.VirtualMemoryStat `json:"virtualmem,omitempty"`
Error string `json:"error,omitempty"`
}
// ServerOsOBDInfo - Includes host os information
type ServerOsOBDInfo struct {
Addr string `json:"addr"`
Info *host.InfoStat `json:"info,omitempty"`
Sensors []host.TemperatureStat `json:"sensors,omitempty"`
Users []host.UserStat `json:"users,omitempty"`
Error string `json:"error,omitempty"`
}
// ServerCPUOBDInfo - Includes cpu and timer stats of each node of the MinIO cluster
type ServerCPUOBDInfo struct {
Addr string `json:"addr"`
CPUStat []cpu.InfoStat `json:"cpu,omitempty"`
TimeStat []cpu.TimesStat `json:"time,omitempty"`
Error string `json:"error,omitempty"`
}
// MinioOBDInfo - Includes MinIO confifuration information
type MinioOBDInfo struct {
Info InfoMessage `json:"info,omitempty"`
Config interface{} `json:"config,omitempty"`
Error string `json:"error,omitempty"`
}
// PerfOBDInfo - Includes Drive and Net perf info for the entire MinIO cluster
type PerfOBDInfo struct {
DriveInfo []ServerDrivesOBDInfo `json:"drives,omitempty"`
Net []ServerNetOBDInfo `json:"net,omitempty"`
Error string `json:"error,omitempty"`
}
// ServerDrivesOBDInfo - Drive OBD info about all drives in a single MinIO node
type ServerDrivesOBDInfo struct {
Addr string `json:"addr"`
Serial []DriveOBDInfo `json:"serial,omitempty"`
Parallel []DriveOBDInfo `json:"parallel,omitempty"`
Error string `json:"error,omitempty"`
}
// DriveOBDInfo - Stats about a single drive in a MinIO node
type DriveOBDInfo struct {
Path string `json:"endpoint"`
Latency disk.Latency `json:"latency,omitempty"`
Throughput disk.Throughput `json:"throughput,omitempty"`
Error string `json:"error,omitempty"`
}
// ServerNetOBDInfo - Network OBD info about a single MinIO node
type ServerNetOBDInfo struct {
Addr string `json:"addr"`
Net []NetOBDInfo `json:"net,omitempty"`
Error string `json:"error,omitempty"`
}
// NetOBDInfo - one-to-one network connectivity Stats between 2 MinIO nodes
type NetOBDInfo struct {
Addr string `json:"remote"`
Latency net.Latency `json:"latency,omitempty"`
Throughput net.Throughput `json:"throughput,omitempty"`
Error string `json:"error,omitempty"`
}
// OBDDataType - Typed OBD data types
type OBDDataType string
// OBDDataTypes
const (
OBDDataTypePerfDrive OBDDataType = "perfdrive"
OBDDataTypePerfNet OBDDataType = "perfnet"
OBDDataTypeMinioInfo OBDDataType = "minioinfo"
OBDDataTypeMinioConfig OBDDataType = "minioconfig"
OBDDataTypeSysCPU OBDDataType = "syscpu"
OBDDataTypeSysDiskHw OBDDataType = "sysdiskhw"
OBDDataTypeSysDocker OBDDataType = "sysdocker" // is this really needed?
OBDDataTypeSysOsInfo OBDDataType = "sysosinfo"
OBDDataTypeSysLoad OBDDataType = "sysload" // provides very little info. Making it TBD
OBDDataTypeSysMem OBDDataType = "sysmem"
OBDDataTypeSysNet OBDDataType = "sysnet"
OBDDataTypeSysProcess OBDDataType = "sysprocess"
)
// OBDDataTypesMap - Map of OBD datatypes
var OBDDataTypesMap = map[string]OBDDataType{
"perfdrive": OBDDataTypePerfDrive,
"perfnet": OBDDataTypePerfNet,
"minioinfo": OBDDataTypeMinioInfo,
"minioconfig": OBDDataTypeMinioConfig,
"syscpu": OBDDataTypeSysCPU,
"sysdiskhw": OBDDataTypeSysDiskHw,
"sysdocker": OBDDataTypeSysDocker,
"sysosinfo": OBDDataTypeSysOsInfo,
"sysload": OBDDataTypeSysLoad,
"sysmem": OBDDataTypeSysMem,
"sysnet": OBDDataTypeSysNet,
"sysprocess": OBDDataTypeSysProcess,
}
// OBDDataTypesList - List of OBD datatypes
var OBDDataTypesList = []OBDDataType{
OBDDataTypePerfDrive,
OBDDataTypePerfNet,
OBDDataTypeMinioInfo,
OBDDataTypeMinioConfig,
OBDDataTypeSysCPU,
OBDDataTypeSysDiskHw,
OBDDataTypeSysDocker,
OBDDataTypeSysOsInfo,
OBDDataTypeSysLoad,
OBDDataTypeSysMem,
OBDDataTypeSysNet,
OBDDataTypeSysProcess,
}
// ServerOBDInfo - Connect to a minio server and call OBD Info Management API
// to fetch server's information represented by OBDInfo structure
func (adm *AdminClient) ServerOBDInfo(ctx context.Context, obdDataTypes []OBDDataType) <-chan OBDInfo {
respChan := make(chan OBDInfo)
go func() {
v := url.Values{}
// start with all set to false
for _, d := range OBDDataTypesList {
v.Set(string(d), "false")
}
// only 'trueify' user provided values
for _, d := range obdDataTypes {
v.Set(string(d), "true")
}
var OBDInfoMessage OBDInfo
if v.Get(string(OBDDataTypeMinioInfo)) == "true" {
info, err := adm.ServerInfo(ctx)
if err != nil {
respChan <- OBDInfo{
Error: err.Error(),
}
return
}
OBDInfoMessage.Minio.Info = info
respChan <- OBDInfoMessage
}
resp, err := adm.executeMethod(ctx, "GET", requestData{
relPath: adminAPIPrefix + "/obdinfo",
queryValues: v,
})
defer closeResponse(resp)
if err != nil {
respChan <- OBDInfo{
Error: err.Error(),
}
return
}
// Check response http status code
if resp.StatusCode != http.StatusOK {
respChan <- OBDInfo{
Error: httpRespToErrorResponse(resp).Error(),
}
return
}
// Unmarshal the server's json response
decoder := json.NewDecoder(resp.Body)
for {
err := decoder.Decode(&OBDInfoMessage)
if err == io.EOF {
break
}
if err != nil {
respChan <- OBDInfo{
Error: err.Error(),
}
}
respChan <- OBDInfoMessage
}
OBDInfoMessage.TimeStamp = time.Now()
respChan <- OBDInfoMessage
close(respChan)
}()
return respChan
}

View File

@ -0,0 +1,7 @@
package madmin
// ServerDiskHwOBDInfo - Includes usage counters, disk counters and partitions
type ServerDiskHwOBDInfo struct {
Addr string `json:"addr"`
Error string `json:"error,omitempty"`
}

33
pkg/madmin/obd_other.go Normal file
View File

@ -0,0 +1,33 @@
// +build !freebsd
/*
* 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 (
diskhw "github.com/shirou/gopsutil/disk"
)
// ServerDiskHwOBDInfo - Includes usage counters, disk counters and partitions
type ServerDiskHwOBDInfo struct {
Addr string `json:"addr"`
Usage []*diskhw.UsageStat `json:"usages,omitempty"`
Partitions []diskhw.PartitionStat `json:"partitions,omitempty"`
Counters map[string]diskhw.IOCountersStat `json:"counters,omitempty"`
Error string `json:"error,omitempty"`
}

116
pkg/net/obd.go Normal file
View File

@ -0,0 +1,116 @@
/*
* 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 net
import (
"github.com/montanaflynn/stats"
)
// Latency holds latency information for read/write operations to the drive
type Latency struct {
Avg float64 `json:"avg_secs,omitempty"`
Percentile50 float64 `json:"percentile50_secs,omitempty"`
Percentile90 float64 `json:"percentile90_secs,omitempty"`
Percentile99 float64 `json:"percentile99_secs,omitempty"`
Min float64 `json:"min_secs,omitempty"`
Max float64 `json:"max_secs,omitempty"`
}
// Throughput holds throughput information for read/write operations to the drive
type Throughput struct {
Avg float64 `json:"avg_bytes_per_sec,omitempty"`
Percentile50 float64 `json:"percentile50_bytes_per_sec,omitempty"`
Percentile90 float64 `json:"percentile90_bytes_per_sec,omitempty"`
Percentile99 float64 `json:"percentile99_bytes_per_sec,omitempty"`
Min float64 `json:"min_bytes_per_sec,omitempty"`
Max float64 `json:"max_bytes_per_sec,omitempty"`
}
// ComputeOBDStats takes arrays of Latency & Throughput to compute Statistics
func ComputeOBDStats(latencies, throughputs []float64) (Latency, Throughput, error) {
var avgLatency float64
var percentile50Latency float64
var percentile90Latency float64
var percentile99Latency float64
var minLatency float64
var maxLatency float64
var avgThroughput float64
var percentile50Throughput float64
var percentile90Throughput float64
var percentile99Throughput float64
var minThroughput float64
var maxThroughput float64
var err error
if avgLatency, err = stats.Mean(latencies); err != nil {
return Latency{}, Throughput{}, err
}
if percentile50Latency, err = stats.Percentile(latencies, 50); err != nil {
return Latency{}, Throughput{}, err
}
if percentile90Latency, err = stats.Percentile(latencies, 90); err != nil {
return Latency{}, Throughput{}, err
}
if percentile99Latency, err = stats.Percentile(latencies, 99); err != nil {
return Latency{}, Throughput{}, err
}
if maxLatency, err = stats.Max(latencies); err != nil {
return Latency{}, Throughput{}, err
}
if minLatency, err = stats.Min(latencies); err != nil {
return Latency{}, Throughput{}, err
}
l := Latency{
Avg: avgLatency,
Percentile50: percentile50Latency,
Percentile90: percentile90Latency,
Percentile99: percentile99Latency,
Min: minLatency,
Max: maxLatency,
}
if avgThroughput, err = stats.Mean(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
if percentile50Throughput, err = stats.Percentile(throughputs, 50); err != nil {
return Latency{}, Throughput{}, err
}
if percentile90Throughput, err = stats.Percentile(throughputs, 90); err != nil {
return Latency{}, Throughput{}, err
}
if percentile99Throughput, err = stats.Percentile(throughputs, 99); err != nil {
return Latency{}, Throughput{}, err
}
if maxThroughput, err = stats.Max(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
if minThroughput, err = stats.Min(throughputs); err != nil {
return Latency{}, Throughput{}, err
}
t := Throughput{
Avg: avgThroughput,
Percentile50: percentile50Throughput,
Percentile90: percentile90Throughput,
Percentile99: percentile99Throughput,
Min: minThroughput,
Max: maxThroughput,
}
return l, t, nil
}