mirror of https://github.com/minio/minio.git
prom: Add online and healing drives metrics per erasure set (#18700)
This commit is contained in:
parent
7c948adf88
commit
8432fd5ac2
|
@ -340,7 +340,7 @@ func (a adminAPIHandlers) StorageInfoHandler(w http.ResponseWriter, r *http.Requ
|
|||
return
|
||||
}
|
||||
|
||||
storageInfo := objectAPI.StorageInfo(ctx)
|
||||
storageInfo := objectAPI.StorageInfo(ctx, true)
|
||||
|
||||
// Collect any disk healing.
|
||||
healing, _ := getAggregatedBackgroundHealState(ctx, nil)
|
||||
|
@ -1313,7 +1313,7 @@ func (a adminAPIHandlers) ObjectSpeedTestHandler(w http.ResponseWriter, r *http.
|
|||
duration = time.Second * 10
|
||||
}
|
||||
|
||||
storageInfo := objectAPI.StorageInfo(ctx)
|
||||
storageInfo := objectAPI.StorageInfo(ctx, true)
|
||||
|
||||
sufficientCapacity, canAutotune, capacityErrMsg := validateObjPerfOptions(ctx, storageInfo, concurrent, size, autotune)
|
||||
if !sufficientCapacity {
|
||||
|
@ -2726,7 +2726,7 @@ func getClusterMetaInfo(ctx context.Context) []byte {
|
|||
ci.Info.NoOfServers = len(globalEndpoints.Hostnames())
|
||||
ci.Info.MinioVersion = Version
|
||||
|
||||
si := objectAPI.StorageInfo(ctx)
|
||||
si := objectAPI.StorageInfo(ctx, true)
|
||||
|
||||
ci.Info.NoOfDrives = len(si.Disks)
|
||||
for _, disk := range si.Disks {
|
||||
|
|
|
@ -141,7 +141,7 @@ func getLocalServerProperty(endpointServerPools EndpointServerPools, r *http.Req
|
|||
|
||||
objLayer := newObjectLayerFn()
|
||||
if objLayer != nil {
|
||||
storageInfo := objLayer.LocalStorageInfo(GlobalContext)
|
||||
storageInfo := objLayer.LocalStorageInfo(GlobalContext, true)
|
||||
props.State = string(madmin.ItemOnline)
|
||||
props.Disks = storageInfo.Disks
|
||||
} else {
|
||||
|
|
|
@ -129,7 +129,7 @@ func (z *erasureServerPools) initRebalanceMeta(ctx context.Context, buckets []st
|
|||
}
|
||||
|
||||
// Fetch disk capacity and available space.
|
||||
si := z.StorageInfo(ctx)
|
||||
si := z.StorageInfo(ctx, true)
|
||||
diskStats := make([]struct {
|
||||
AvailableSpace uint64
|
||||
TotalSpace uint64
|
||||
|
|
|
@ -613,7 +613,7 @@ func (z *erasureServerPools) BackendInfo() (b madmin.BackendInfo) {
|
|||
return
|
||||
}
|
||||
|
||||
func (z *erasureServerPools) LocalStorageInfo(ctx context.Context) StorageInfo {
|
||||
func (z *erasureServerPools) LocalStorageInfo(ctx context.Context, metrics bool) StorageInfo {
|
||||
var storageInfo StorageInfo
|
||||
|
||||
storageInfos := make([]StorageInfo, len(z.serverPools))
|
||||
|
@ -621,7 +621,7 @@ func (z *erasureServerPools) LocalStorageInfo(ctx context.Context) StorageInfo {
|
|||
for index := range z.serverPools {
|
||||
index := index
|
||||
g.Go(func() error {
|
||||
storageInfos[index] = z.serverPools[index].LocalStorageInfo(ctx)
|
||||
storageInfos[index] = z.serverPools[index].LocalStorageInfo(ctx, metrics)
|
||||
return nil
|
||||
}, index)
|
||||
}
|
||||
|
@ -637,8 +637,8 @@ func (z *erasureServerPools) LocalStorageInfo(ctx context.Context) StorageInfo {
|
|||
return storageInfo
|
||||
}
|
||||
|
||||
func (z *erasureServerPools) StorageInfo(ctx context.Context) StorageInfo {
|
||||
return globalNotificationSys.StorageInfo(z)
|
||||
func (z *erasureServerPools) StorageInfo(ctx context.Context, metrics bool) StorageInfo {
|
||||
return globalNotificationSys.StorageInfo(z, metrics)
|
||||
}
|
||||
|
||||
func (z *erasureServerPools) NSScanner(ctx context.Context, updates chan<- DataUsageInfo, wantCycle uint32, healScanMode madmin.HealScanMode) error {
|
||||
|
@ -2285,6 +2285,7 @@ type HealthResult struct {
|
|||
Maintenance bool
|
||||
PoolID, SetID int
|
||||
HealthyDrives int
|
||||
HealingDrives int
|
||||
WriteQuorum int
|
||||
}
|
||||
WriteQuorum int
|
||||
|
@ -2331,29 +2332,36 @@ func (z *erasureServerPools) ReadHealth(ctx context.Context) bool {
|
|||
// can be used to query scenarios if health may be lost
|
||||
// if this node is taken down by an external orchestrator.
|
||||
func (z *erasureServerPools) Health(ctx context.Context, opts HealthOptions) HealthResult {
|
||||
erasureSetUpCount := make([][]int, len(z.serverPools))
|
||||
for i := range z.serverPools {
|
||||
erasureSetUpCount[i] = make([]int, len(z.serverPools[i].sets))
|
||||
}
|
||||
|
||||
diskIDs := globalNotificationSys.GetLocalDiskIDs(ctx)
|
||||
if !opts.Maintenance {
|
||||
diskIDs = append(diskIDs, getLocalDiskIDs(z))
|
||||
}
|
||||
|
||||
for _, localDiskIDs := range diskIDs {
|
||||
for _, id := range localDiskIDs {
|
||||
poolIdx, setIdx, _, err := z.getPoolAndSet(id)
|
||||
if err != nil {
|
||||
logger.LogIf(ctx, err)
|
||||
continue
|
||||
}
|
||||
erasureSetUpCount[poolIdx][setIdx]++
|
||||
}
|
||||
}
|
||||
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("maintenance", strconv.FormatBool(opts.Maintenance))
|
||||
|
||||
type setInfo struct {
|
||||
online int
|
||||
healing int
|
||||
}
|
||||
|
||||
var drivesHealing int
|
||||
|
||||
erasureSetUpCount := make([][]setInfo, len(z.serverPools))
|
||||
for i := range z.serverPools {
|
||||
erasureSetUpCount[i] = make([]setInfo, len(z.serverPools[i].sets))
|
||||
}
|
||||
|
||||
storageInfo := z.StorageInfo(ctx, false)
|
||||
|
||||
for _, disk := range storageInfo.Disks {
|
||||
if disk.PoolIndex > -1 && disk.SetIndex > -1 {
|
||||
if disk.State == madmin.DriveStateOk {
|
||||
si := erasureSetUpCount[disk.PoolIndex][disk.SetIndex]
|
||||
si.online++
|
||||
if disk.Healing {
|
||||
si.healing++
|
||||
drivesHealing++
|
||||
}
|
||||
erasureSetUpCount[disk.PoolIndex][disk.SetIndex] = si
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
b := z.BackendInfo()
|
||||
poolWriteQuorums := make([]int, len(b.StandardSCData))
|
||||
for i, data := range b.StandardSCData {
|
||||
|
@ -2363,23 +2371,10 @@ func (z *erasureServerPools) Health(ctx context.Context, opts HealthOptions) Hea
|
|||
}
|
||||
}
|
||||
|
||||
var aggHealStateResult madmin.BgHealState
|
||||
// Check if disks are healing on in-case of VMware vsphere deployments.
|
||||
if opts.Maintenance && opts.DeploymentType == vmware {
|
||||
// check if local disks are being healed, if they are being healed
|
||||
// we need to tell healthy status as 'false' so that this server
|
||||
// is not taken down for maintenance
|
||||
var err error
|
||||
aggHealStateResult, err = getAggregatedBackgroundHealState(ctx, nil)
|
||||
if err != nil {
|
||||
logger.LogIf(logger.SetReqInfo(ctx, reqInfo), fmt.Errorf("Unable to verify global heal status: %w", err))
|
||||
return HealthResult{
|
||||
Healthy: false,
|
||||
}
|
||||
}
|
||||
|
||||
if len(aggHealStateResult.HealDisks) > 0 {
|
||||
logger.LogIf(logger.SetReqInfo(ctx, reqInfo), fmt.Errorf("Total drives to be healed %d", len(aggHealStateResult.HealDisks)))
|
||||
if drivesHealing > 0 {
|
||||
logger.LogIf(logger.SetReqInfo(ctx, reqInfo), fmt.Errorf("Total drives to be healed %d", drivesHealing))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2409,16 +2404,17 @@ func (z *erasureServerPools) Health(ctx context.Context, opts HealthOptions) Hea
|
|||
result.ESHealth = append(result.ESHealth, struct {
|
||||
Maintenance bool
|
||||
PoolID, SetID int
|
||||
HealthyDrives, WriteQuorum int
|
||||
HealthyDrives, HealingDrives, WriteQuorum int
|
||||
}{
|
||||
Maintenance: opts.Maintenance,
|
||||
SetID: setIdx,
|
||||
PoolID: poolIdx,
|
||||
HealthyDrives: erasureSetUpCount[poolIdx][setIdx],
|
||||
HealthyDrives: erasureSetUpCount[poolIdx][setIdx].online,
|
||||
HealingDrives: erasureSetUpCount[poolIdx][setIdx].healing,
|
||||
WriteQuorum: poolWriteQuorums[poolIdx],
|
||||
})
|
||||
|
||||
if erasureSetUpCount[poolIdx][setIdx] < poolWriteQuorums[poolIdx] {
|
||||
if erasureSetUpCount[poolIdx][setIdx].online < poolWriteQuorums[poolIdx] {
|
||||
logger.LogIf(logger.SetReqInfo(ctx, reqInfo),
|
||||
fmt.Errorf("Write quorum may be lost on pool: %d, set: %d, expected write quorum: %d",
|
||||
poolIdx, setIdx, poolWriteQuorums[poolIdx]))
|
||||
|
@ -2428,8 +2424,8 @@ func (z *erasureServerPools) Health(ctx context.Context, opts HealthOptions) Hea
|
|||
}
|
||||
|
||||
if opts.Maintenance {
|
||||
result.Healthy = result.Healthy && len(aggHealStateResult.HealDisks) == 0
|
||||
result.HealingDrives = len(aggHealStateResult.HealDisks)
|
||||
result.Healthy = result.Healthy && drivesHealing == 0
|
||||
result.HealingDrives = drivesHealing
|
||||
}
|
||||
|
||||
return result
|
||||
|
|
|
@ -597,7 +597,7 @@ func (s *erasureSets) StorageInfo(ctx context.Context) StorageInfo {
|
|||
}
|
||||
|
||||
// StorageInfo - combines output of StorageInfo across all erasure coded object sets.
|
||||
func (s *erasureSets) LocalStorageInfo(ctx context.Context) StorageInfo {
|
||||
func (s *erasureSets) LocalStorageInfo(ctx context.Context, metrics bool) StorageInfo {
|
||||
var storageInfo StorageInfo
|
||||
|
||||
storageInfos := make([]StorageInfo, len(s.sets))
|
||||
|
@ -606,7 +606,7 @@ func (s *erasureSets) LocalStorageInfo(ctx context.Context) StorageInfo {
|
|||
for index := range s.sets {
|
||||
index := index
|
||||
g.Go(func() error {
|
||||
storageInfos[index] = s.sets[index].LocalStorageInfo(ctx)
|
||||
storageInfos[index] = s.sets[index].LocalStorageInfo(ctx, metrics)
|
||||
return nil
|
||||
}, index)
|
||||
}
|
||||
|
|
|
@ -160,7 +160,7 @@ func getOnlineOfflineDisksStats(disksInfo []madmin.Disk) (onlineDisks, offlineDi
|
|||
}
|
||||
|
||||
// getDisksInfo - fetch disks info across all other storage API.
|
||||
func getDisksInfo(disks []StorageAPI, endpoints []Endpoint) (disksInfo []madmin.Disk) {
|
||||
func getDisksInfo(disks []StorageAPI, endpoints []Endpoint, metrics bool) (disksInfo []madmin.Disk) {
|
||||
disksInfo = make([]madmin.Disk, len(disks))
|
||||
|
||||
g := errgroup.WithNErrs(len(disks))
|
||||
|
@ -178,7 +178,7 @@ func getDisksInfo(disks []StorageAPI, endpoints []Endpoint) (disksInfo []madmin.
|
|||
disksInfo[index] = di
|
||||
return nil
|
||||
}
|
||||
info, err := disks[index].DiskInfo(context.TODO(), true)
|
||||
info, err := disks[index].DiskInfo(context.TODO(), metrics)
|
||||
di.DrivePath = info.MountPath
|
||||
di.TotalSpace = info.Total
|
||||
di.UsedSpace = info.Used
|
||||
|
@ -225,8 +225,8 @@ func getDisksInfo(disks []StorageAPI, endpoints []Endpoint) (disksInfo []madmin.
|
|||
}
|
||||
|
||||
// Get an aggregated storage info across all disks.
|
||||
func getStorageInfo(disks []StorageAPI, endpoints []Endpoint) StorageInfo {
|
||||
disksInfo := getDisksInfo(disks, endpoints)
|
||||
func getStorageInfo(disks []StorageAPI, endpoints []Endpoint, metrics bool) StorageInfo {
|
||||
disksInfo := getDisksInfo(disks, endpoints, metrics)
|
||||
|
||||
// Sort so that the first element is the smallest.
|
||||
sort.Slice(disksInfo, func(i, j int) bool {
|
||||
|
@ -245,11 +245,11 @@ func getStorageInfo(disks []StorageAPI, endpoints []Endpoint) StorageInfo {
|
|||
func (er erasureObjects) StorageInfo(ctx context.Context) StorageInfo {
|
||||
disks := er.getDisks()
|
||||
endpoints := er.getEndpoints()
|
||||
return getStorageInfo(disks, endpoints)
|
||||
return getStorageInfo(disks, endpoints, true)
|
||||
}
|
||||
|
||||
// LocalStorageInfo - returns underlying local storage statistics.
|
||||
func (er erasureObjects) LocalStorageInfo(ctx context.Context) StorageInfo {
|
||||
func (er erasureObjects) LocalStorageInfo(ctx context.Context, metrics bool) StorageInfo {
|
||||
disks := er.getDisks()
|
||||
endpoints := er.getEndpoints()
|
||||
|
||||
|
@ -263,7 +263,7 @@ func (er erasureObjects) LocalStorageInfo(ctx context.Context) StorageInfo {
|
|||
}
|
||||
}
|
||||
|
||||
return getStorageInfo(localDisks, localEndpoints)
|
||||
return getStorageInfo(localDisks, localEndpoints, metrics)
|
||||
}
|
||||
|
||||
// getOnlineDisksWithHealing - returns online disks and overall healing status.
|
||||
|
|
|
@ -97,7 +97,7 @@ func getLocalBackgroundHealStatus(ctx context.Context, o ObjectLayer) (madmin.Bg
|
|||
return status, true
|
||||
}
|
||||
|
||||
si := o.LocalStorageInfo(ctx)
|
||||
si := o.LocalStorageInfo(ctx, true)
|
||||
|
||||
indexed := make(map[string][]madmin.Disk)
|
||||
for _, disk := range si.Disks {
|
||||
|
|
|
@ -136,7 +136,7 @@ func collectLocalDisksMetrics(disks map[string]struct{}) map[string]madmin.DiskM
|
|||
return metrics
|
||||
}
|
||||
|
||||
storageInfo := objLayer.LocalStorageInfo(GlobalContext)
|
||||
storageInfo := objLayer.LocalStorageInfo(GlobalContext, true)
|
||||
for _, d := range storageInfo.Disks {
|
||||
if len(disks) != 0 {
|
||||
_, ok := disks[d.Endpoint]
|
||||
|
|
|
@ -3133,7 +3133,7 @@ func getLocalStorageMetrics() *MetricsGroup {
|
|||
}
|
||||
|
||||
metrics = make([]Metric, 0, 50)
|
||||
storageInfo := objLayer.LocalStorageInfo(ctx)
|
||||
storageInfo := objLayer.LocalStorageInfo(ctx, true)
|
||||
onlineDrives, offlineDrives := getOnlineOfflineDisksStats(storageInfo.Disks)
|
||||
totalDrives := onlineDrives.Merge(offlineDrives)
|
||||
|
||||
|
@ -3235,12 +3235,32 @@ func getClusterHealthStatusMD() MetricDescription {
|
|||
}
|
||||
}
|
||||
|
||||
func getClusterErasureSetToleranceMD() MetricDescription {
|
||||
func getClusterErasureSetWriteQuorumMD() MetricDescription {
|
||||
return MetricDescription{
|
||||
Namespace: clusterMetricNamespace,
|
||||
Subsystem: "health",
|
||||
Name: "erasure_set_tolerance",
|
||||
Help: "Get erasure set tolerance status",
|
||||
Name: "erasure_set_write_quorum",
|
||||
Help: "Get the write quorum for this erasure set",
|
||||
Type: gaugeMetric,
|
||||
}
|
||||
}
|
||||
|
||||
func getClusterErasureSetOnlineDrivesMD() MetricDescription {
|
||||
return MetricDescription{
|
||||
Namespace: clusterMetricNamespace,
|
||||
Subsystem: "health",
|
||||
Name: "erasure_set_online_drives",
|
||||
Help: "Get the count of the online drives in this erasure set",
|
||||
Type: gaugeMetric,
|
||||
}
|
||||
}
|
||||
|
||||
func getClusterErasureSetHealingDrivesMD() MetricDescription {
|
||||
return MetricDescription{
|
||||
Namespace: clusterMetricNamespace,
|
||||
Subsystem: "health",
|
||||
Name: "erasure_set_healing_drives",
|
||||
Help: "Get the count of healing drives of this erasure set",
|
||||
Type: gaugeMetric,
|
||||
}
|
||||
}
|
||||
|
@ -3256,11 +3276,11 @@ func getClusterHealthMetrics() *MetricsGroup {
|
|||
return
|
||||
}
|
||||
|
||||
metrics = make([]Metric, 0, 2)
|
||||
|
||||
opts := HealthOptions{}
|
||||
result := objLayer.Health(ctx, opts)
|
||||
|
||||
metrics = make([]Metric, 0, 2+3*len(result.ESHealth))
|
||||
|
||||
metrics = append(metrics, Metric{
|
||||
Description: getClusterWriteQuorumMD(),
|
||||
Value: float64(result.WriteQuorum),
|
||||
|
@ -3282,9 +3302,19 @@ func getClusterHealthMetrics() *MetricsGroup {
|
|||
"set": strconv.Itoa(h.SetID),
|
||||
}
|
||||
metrics = append(metrics, Metric{
|
||||
Description: getClusterErasureSetToleranceMD(),
|
||||
Description: getClusterErasureSetWriteQuorumMD(),
|
||||
VariableLabels: labels,
|
||||
Value: float64(h.HealthyDrives - h.WriteQuorum),
|
||||
Value: float64(h.WriteQuorum),
|
||||
})
|
||||
metrics = append(metrics, Metric{
|
||||
Description: getClusterErasureSetOnlineDrivesMD(),
|
||||
VariableLabels: labels,
|
||||
Value: float64(h.HealthyDrives),
|
||||
})
|
||||
metrics = append(metrics, Metric{
|
||||
Description: getClusterErasureSetHealingDrivesMD(),
|
||||
VariableLabels: labels,
|
||||
Value: float64(h.HealingDrives),
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -3378,7 +3408,7 @@ func getClusterStorageMetrics() *MetricsGroup {
|
|||
|
||||
// Fetch disk space info, ignore errors
|
||||
metrics = make([]Metric, 0, 10)
|
||||
storageInfo := objLayer.StorageInfo(ctx)
|
||||
storageInfo := objLayer.StorageInfo(ctx, true)
|
||||
onlineDrives, offlineDrives := getOnlineOfflineDisksStats(storageInfo.Disks)
|
||||
totalDrives := onlineDrives.Merge(offlineDrives)
|
||||
|
||||
|
|
|
@ -406,7 +406,7 @@ func storageMetricsPrometheus(ch chan<- prometheus.Metric) {
|
|||
float64(GetTotalCapacityFree(server.Disks)),
|
||||
)
|
||||
|
||||
sinfo := objLayer.StorageInfo(GlobalContext)
|
||||
sinfo := objLayer.StorageInfo(GlobalContext, true)
|
||||
|
||||
// Report total usable capacity
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
|
|
|
@ -988,7 +988,7 @@ func getOfflineDisks(offlineHost string, endpoints EndpointServerPools) []madmin
|
|||
}
|
||||
|
||||
// StorageInfo returns disk information across all peers
|
||||
func (sys *NotificationSys) StorageInfo(objLayer ObjectLayer) StorageInfo {
|
||||
func (sys *NotificationSys) StorageInfo(objLayer ObjectLayer, metrics bool) StorageInfo {
|
||||
var storageInfo StorageInfo
|
||||
replies := make([]StorageInfo, len(sys.peerClients))
|
||||
|
||||
|
@ -1000,7 +1000,7 @@ func (sys *NotificationSys) StorageInfo(objLayer ObjectLayer) StorageInfo {
|
|||
wg.Add(1)
|
||||
go func(client *peerRESTClient, idx int) {
|
||||
defer wg.Done()
|
||||
info, err := client.LocalStorageInfo()
|
||||
info, err := client.LocalStorageInfo(metrics)
|
||||
if err != nil {
|
||||
info.Disks = getOfflineDisks(client.host.String(), globalEndpoints)
|
||||
}
|
||||
|
@ -1010,7 +1010,7 @@ func (sys *NotificationSys) StorageInfo(objLayer ObjectLayer) StorageInfo {
|
|||
wg.Wait()
|
||||
|
||||
// Add local to this server.
|
||||
replies = append(replies, objLayer.LocalStorageInfo(GlobalContext))
|
||||
replies = append(replies, objLayer.LocalStorageInfo(GlobalContext, metrics))
|
||||
|
||||
storageInfo.Backend = objLayer.BackendInfo()
|
||||
for _, sinfo := range replies {
|
||||
|
|
|
@ -228,8 +228,8 @@ type ObjectLayer interface {
|
|||
Shutdown(context.Context) error
|
||||
NSScanner(ctx context.Context, updates chan<- DataUsageInfo, wantCycle uint32, scanMode madmin.HealScanMode) error
|
||||
BackendInfo() madmin.BackendInfo
|
||||
StorageInfo(ctx context.Context) StorageInfo
|
||||
LocalStorageInfo(ctx context.Context) StorageInfo
|
||||
StorageInfo(ctx context.Context, metrics bool) StorageInfo
|
||||
LocalStorageInfo(ctx context.Context, metrics bool) StorageInfo
|
||||
|
||||
// Bucket operations.
|
||||
MakeBucket(ctx context.Context, bucket string, opts MakeBucketOptions) error
|
||||
|
|
|
@ -102,8 +102,10 @@ func (client *peerRESTClient) GetLocks() (lockMap map[string][]lockRequesterInfo
|
|||
}
|
||||
|
||||
// LocalStorageInfo - fetch server information for a remote node.
|
||||
func (client *peerRESTClient) LocalStorageInfo() (info StorageInfo, err error) {
|
||||
respBody, err := client.call(peerRESTMethodLocalStorageInfo, nil, nil, -1)
|
||||
func (client *peerRESTClient) LocalStorageInfo(metrics bool) (info StorageInfo, err error) {
|
||||
values := make(url.Values)
|
||||
values.Set(peerRESTMetrics, strconv.FormatBool(metrics))
|
||||
respBody, err := client.call(peerRESTMethodLocalStorageInfo, values, nil, -1)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package cmd
|
||||
|
||||
const (
|
||||
peerRESTVersion = "v33" // Add SRMetrics
|
||||
peerRESTVersion = "v34" // Add metrics flag to LocalStorageInfo call
|
||||
|
||||
peerRESTVersionPrefix = SlashSeparator + peerRESTVersion
|
||||
peerRESTPrefix = minioReservedBucketPath + "/peer"
|
||||
|
@ -106,6 +106,7 @@ const (
|
|||
peerRESTJobID = "job-id"
|
||||
peerRESTDepID = "depID"
|
||||
peerRESTStartRebalance = "start-rebalance"
|
||||
peerRESTMetrics = "metrics"
|
||||
|
||||
peerRESTListenBucket = "bucket"
|
||||
peerRESTListenPrefix = "prefix"
|
||||
|
|
|
@ -345,7 +345,14 @@ func (s *peerRESTServer) LocalStorageInfoHandler(w http.ResponseWriter, r *http.
|
|||
return
|
||||
}
|
||||
|
||||
logger.LogIf(ctx, gob.NewEncoder(w).Encode(objLayer.LocalStorageInfo(r.Context())))
|
||||
metrics, err := strconv.ParseBool(r.Form.Get(peerRESTMetrics))
|
||||
if err != nil {
|
||||
s.writeErrorResponse(w, err)
|
||||
return
|
||||
|
||||
}
|
||||
|
||||
logger.LogIf(ctx, gob.NewEncoder(w).Encode(objLayer.LocalStorageInfo(r.Context(), metrics)))
|
||||
}
|
||||
|
||||
// ServerInfoHandler - returns Server Info
|
||||
|
|
|
@ -55,7 +55,7 @@ func rebalanceStatus(ctx context.Context, z *erasureServerPools) (r rebalanceAdm
|
|||
}
|
||||
|
||||
// Compute disk usage percentage
|
||||
si := z.StorageInfo(ctx)
|
||||
si := z.StorageInfo(ctx, true)
|
||||
diskStats := make([]struct {
|
||||
AvailableSpace uint64
|
||||
TotalSpace uint64
|
||||
|
|
|
@ -55,7 +55,7 @@ func printStartupMessage(apiEndpoints []string, err error) {
|
|||
// Object layer is initialized then print StorageInfo.
|
||||
objAPI := newObjectLayerFn()
|
||||
if objAPI != nil {
|
||||
printStorageInfo(objAPI.StorageInfo(GlobalContext))
|
||||
printStorageInfo(objAPI.StorageInfo(GlobalContext, true))
|
||||
}
|
||||
|
||||
// Prints credential, region and browser access.
|
||||
|
|
|
@ -163,7 +163,7 @@ func veeamSOSAPIGetObject(ctx context.Context, bucket, object string, rs *HTTPRa
|
|||
}
|
||||
|
||||
if quotaSize == 0 {
|
||||
info := objAPI.StorageInfo(ctx)
|
||||
info := objAPI.StorageInfo(ctx, true)
|
||||
info.Backend = objAPI.BackendInfo()
|
||||
|
||||
ci.Capacity = int64(GetTotalUsableCapacity(info.Disks, info))
|
||||
|
|
Loading…
Reference in New Issue