minio/cmd/admin-heal-ops.go

895 lines
24 KiB
Go
Raw Normal View History

// 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/>.
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
package cmd
import (
"context"
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
"encoding/json"
"errors"
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
"fmt"
"net/http"
"sort"
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
"sync"
"time"
"github.com/minio/madmin-go"
"github.com/minio/minio/internal/logger"
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
)
// healStatusSummary - overall short summary of a healing sequence
type healStatusSummary string
// healStatusSummary constants
const (
healNotStartedStatus healStatusSummary = "not started"
healRunningStatus = "running"
healStoppedStatus = "stopped"
healFinishedStatus = "finished"
)
const (
// a heal sequence with this many un-consumed heal result
// items blocks until heal-status consumption resumes or is
// aborted due to timeout.
maxUnconsumedHealResultItems = 1000
// if no heal-results are consumed (via the heal-status API)
// for this timeout duration, the heal sequence is aborted.
healUnconsumedTimeout = 24 * time.Hour
// time-duration to keep heal sequence state after it
// completes.
keepHealSeqStateDuration = time.Minute * 10
// nopHeal is a no operating healing action to
// wait for the current healing operation to finish
nopHeal = ""
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
)
var (
errHealIdleTimeout = fmt.Errorf("healing results were not consumed for too long")
errHealStopSignalled = fmt.Errorf("heal stop signaled")
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
errFnHealFromAPIErr = func(ctx context.Context, err error) error {
apiErr := toAdminAPIErr(ctx, err)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return fmt.Errorf("Heal internal error: %s: %s",
apiErr.Code, apiErr.Description)
}
)
// healSequenceStatus - accumulated status of the heal sequence
type healSequenceStatus struct {
// summary and detail for failures
Summary healStatusSummary `json:"Summary"`
FailureDetail string `json:"Detail,omitempty"`
StartTime time.Time `json:"StartTime"`
// settings for the heal sequence
HealSettings madmin.HealOpts `json:"Settings"`
// slice of available heal result records
Items []madmin.HealResultItem `json:"Items"`
}
// structure to hold state of all heal sequences in server memory
type allHealState struct {
sync.RWMutex
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// map of heal path to heal sequence
healSeqMap map[string]*healSequence // Indexed by endpoint
healLocalDisks map[Endpoint]struct{}
healStatus map[string]healingTracker // Indexed by disk ID
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// newHealState - initialize global heal state management
func newHealState(cleanup bool) *allHealState {
hstate := &allHealState{
healSeqMap: make(map[string]*healSequence),
healLocalDisks: map[Endpoint]struct{}{},
healStatus: make(map[string]healingTracker),
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
if cleanup {
go hstate.periodicHealSeqsClean(GlobalContext)
}
return hstate
}
func (ahs *allHealState) healDriveCount() int {
ahs.RLock()
defer ahs.RUnlock()
return len(ahs.healLocalDisks)
}
func (ahs *allHealState) popHealLocalDisks(healLocalDisks ...Endpoint) {
ahs.Lock()
defer ahs.Unlock()
for _, ep := range healLocalDisks {
delete(ahs.healLocalDisks, ep)
}
for id, disk := range ahs.healStatus {
for _, ep := range healLocalDisks {
if disk.Endpoint == ep.String() {
delete(ahs.healStatus, id)
}
}
}
}
// updateHealStatus will update the heal status.
func (ahs *allHealState) updateHealStatus(tracker *healingTracker) {
ahs.Lock()
defer ahs.Unlock()
ahs.healStatus[tracker.ID] = *tracker
}
// Sort by zone, set and disk index
func sortDisks(disks []madmin.Disk) {
sort.Slice(disks, func(i, j int) bool {
a, b := &disks[i], &disks[j]
if a.PoolIndex != b.PoolIndex {
return a.PoolIndex < b.PoolIndex
}
if a.SetIndex != b.SetIndex {
return a.SetIndex < b.SetIndex
}
return a.DiskIndex < b.DiskIndex
})
}
// getLocalHealingDisks returns local healing disks indexed by endpoint.
func (ahs *allHealState) getLocalHealingDisks() map[string]madmin.HealingDisk {
ahs.RLock()
defer ahs.RUnlock()
dst := make(map[string]madmin.HealingDisk, len(ahs.healStatus))
for _, v := range ahs.healStatus {
dst[v.Endpoint] = v.toHealingDisk()
}
return dst
}
func (ahs *allHealState) getHealLocalDiskEndpoints() Endpoints {
ahs.RLock()
defer ahs.RUnlock()
var endpoints Endpoints
for ep := range ahs.healLocalDisks {
endpoints = append(endpoints, ep)
}
return endpoints
}
func (ahs *allHealState) pushHealLocalDisks(healLocalDisks ...Endpoint) {
ahs.Lock()
defer ahs.Unlock()
for _, ep := range healLocalDisks {
ahs.healLocalDisks[ep] = struct{}{}
}
}
func (ahs *allHealState) periodicHealSeqsClean(ctx context.Context) {
// Launch clean-up routine to remove this heal sequence (after
// it ends) from the global state after timeout has elapsed.
periodicTimer := time.NewTimer(time.Minute * 5)
defer periodicTimer.Stop()
for {
select {
case <-periodicTimer.C:
periodicTimer.Reset(time.Minute * 5)
now := UTCNow()
ahs.Lock()
for path, h := range ahs.healSeqMap {
if h.hasEnded() && h.endTime.Add(keepHealSeqStateDuration).Before(now) {
delete(ahs.healSeqMap, path)
}
}
ahs.Unlock()
case <-ctx.Done():
// server could be restarting - need
// to exit immediately
return
}
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// getHealSequenceByToken - Retrieve a heal sequence by token. The second
// argument returns if a heal sequence actually exists.
func (ahs *allHealState) getHealSequenceByToken(token string) (h *healSequence, exists bool) {
ahs.Lock()
defer ahs.Unlock()
for _, healSeq := range ahs.healSeqMap {
if healSeq.clientToken == token {
return healSeq, true
}
}
return nil, false
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// getHealSequence - Retrieve a heal sequence by path. The second
// argument returns if a heal sequence actually exists.
func (ahs *allHealState) getHealSequence(path string) (h *healSequence, exists bool) {
ahs.Lock()
defer ahs.Unlock()
h, exists = ahs.healSeqMap[path]
return h, exists
}
func (ahs *allHealState) stopHealSequence(path string) ([]byte, APIError) {
var hsp madmin.HealStopSuccess
he, exists := ahs.getHealSequence(path)
if !exists {
hsp = madmin.HealStopSuccess{
ClientToken: "unknown",
StartTime: UTCNow(),
}
} else {
clientToken := he.clientToken
if globalIsDistErasure {
clientToken = fmt.Sprintf("%s@%d", he.clientToken, GetProxyEndpointLocalIndex(globalProxyEndpoints))
}
hsp = madmin.HealStopSuccess{
ClientToken: clientToken,
ClientAddress: he.clientAddress,
StartTime: he.startTime,
}
he.stop()
for !he.hasEnded() {
time.Sleep(1 * time.Second)
}
ahs.Lock()
defer ahs.Unlock()
// Heal sequence explicitly stopped, remove it.
delete(ahs.healSeqMap, path)
}
b, err := json.Marshal(&hsp)
return b, toAdminAPIErr(GlobalContext, err)
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// LaunchNewHealSequence - launches a background routine that performs
// healing according to the healSequence argument. For each heal
// sequence, state is stored in the `globalAllHealState`, which is a
// map of the heal path to `healSequence` which holds state about the
// heal sequence.
//
// Heal results are persisted in server memory for
// `keepHealSeqStateDuration`. This function also launches a
// background routine to clean up heal results after the
// aforementioned duration.
func (ahs *allHealState) LaunchNewHealSequence(h *healSequence, objAPI ObjectLayer) (
respBytes []byte, apiErr APIError, errMsg string) {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
if h.forceStarted {
_, apiErr = ahs.stopHealSequence(pathJoin(h.bucket, h.object))
if apiErr.Code != "" {
return respBytes, apiErr, ""
}
} else {
oh, exists := ahs.getHealSequence(pathJoin(h.bucket, h.object))
if exists && !oh.hasEnded() {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
errMsg = "Heal is already running on the given path " +
"(use force-start option to stop and start afresh). " +
fmt.Sprintf("The heal was started by IP %s at %s, token is %s",
h.clientAddress, h.startTime.Format(http.TimeFormat), h.clientToken)
return nil, errorCodes.ToAPIErr(ErrHealAlreadyRunning), errMsg
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
}
ahs.Lock()
defer ahs.Unlock()
// Check if new heal sequence to be started overlaps with any
// existing, running sequence
hpath := pathJoin(h.bucket, h.object)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
for k, hSeq := range ahs.healSeqMap {
if !hSeq.hasEnded() && (HasPrefix(k, hpath) || HasPrefix(hpath, k)) {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
errMsg = "The provided heal sequence path overlaps with an existing " +
fmt.Sprintf("heal path: %s", k)
return nil, errorCodes.ToAPIErr(ErrHealOverlappingPaths), errMsg
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
}
// Add heal state and start sequence
ahs.healSeqMap[hpath] = h
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Launch top-level background heal go-routine
go h.healSequenceStart(objAPI)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
clientToken := h.clientToken
if globalIsDistErasure {
clientToken = fmt.Sprintf("%s@%d", h.clientToken, GetProxyEndpointLocalIndex(globalProxyEndpoints))
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
b, err := json.Marshal(madmin.HealStartSuccess{
ClientToken: clientToken,
ClientAddress: h.clientAddress,
StartTime: h.startTime,
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
})
if err != nil {
logger.LogIf(h.ctx, err)
return nil, toAdminAPIErr(h.ctx, err), ""
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
return b, noError, ""
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// PopHealStatusJSON - Called by heal-status API. It fetches the heal
// status results from global state and returns its JSON
// representation. The clientToken helps ensure there aren't
// conflicting clients fetching status.
func (ahs *allHealState) PopHealStatusJSON(hpath string,
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
clientToken string) ([]byte, APIErrorCode) {
// fetch heal state for given path
h, exists := ahs.getHealSequence(hpath)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
if !exists {
// heal sequence doesn't exist, must have finished.
jbytes, err := json.Marshal(healSequenceStatus{
Summary: healFinishedStatus,
})
return jbytes, toAdminAPIErrCode(GlobalContext, err)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// Check if client-token is valid
if clientToken != h.clientToken {
return nil, ErrHealInvalidClientToken
}
// Take lock to access and update the heal-sequence
h.mutex.Lock()
defer h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
numItems := len(h.currentStatus.Items)
// calculate index of most recently available heal result
// record.
lastResultIndex := h.lastSentResultIndex
if numItems > 0 {
lastResultIndex = h.currentStatus.Items[numItems-1].ResultIndex
}
h.lastSentResultIndex = lastResultIndex
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
jbytes, err := json.Marshal(h.currentStatus)
if err != nil {
h.currentStatus.Items = nil
logger.LogIf(h.ctx, err)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return nil, ErrInternalError
}
h.currentStatus.Items = nil
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return jbytes, ErrNone
}
// healSource denotes single entity and heal option.
type healSource struct {
bucket string
object string
versionID string
opts *madmin.HealOpts // optional heal option overrides default setting
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// healSequence - state for each heal sequence initiated on the
// server.
type healSequence struct {
// bucket, and object on which heal seq. was initiated
bucket, object string
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// A channel of entities with heal result
respCh chan healResult
// Report healing progress
reportProgress bool
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// time at which heal sequence was started
startTime time.Time
// time at which heal sequence has ended
endTime time.Time
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Heal client info
clientToken, clientAddress string
// was this heal sequence force started?
forceStarted bool
// heal settings applied to this heal sequence
settings madmin.HealOpts
// current accumulated status of the heal sequence
currentStatus healSequenceStatus
// channel signaled by background routine when traversal has
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// completed
traverseAndHealDoneCh chan error
// canceler to cancel heal sequence.
cancelCtx context.CancelFunc
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// the last result index sent to client
lastSentResultIndex int64
// Number of total items scanned against item type
scannedItemsMap map[madmin.HealItemType]int64
// Number of total items healed against item type
healedItemsMap map[madmin.HealItemType]int64
// Number of total items where healing failed against endpoint and drive state
healFailedItemsMap map[string]int64
// The time of the last scan/heal activity
lastHealActivity time.Time
// Holds the request-info for logging
ctx context.Context
// used to lock this structure as it is concurrently accessed
mutex sync.RWMutex
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// NewHealSequence - creates healSettings, assumes bucket and
// objPrefix are already validated.
func newHealSequence(ctx context.Context, bucket, objPrefix, clientAddr string,
hs madmin.HealOpts, forceStart bool) *healSequence {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
reqInfo := &logger.ReqInfo{RemoteHost: clientAddr, API: "Heal", BucketName: bucket}
reqInfo.AppendTags("prefix", objPrefix)
ctx, cancel := context.WithCancel(logger.SetReqInfo(ctx, reqInfo))
clientToken := mustGetUUID()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return &healSequence{
respCh: make(chan healResult),
bucket: bucket,
object: objPrefix,
reportProgress: true,
startTime: UTCNow(),
clientToken: clientToken,
clientAddress: clientAddr,
forceStarted: forceStart,
settings: hs,
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
currentStatus: healSequenceStatus{
Summary: healNotStartedStatus,
HealSettings: hs,
},
traverseAndHealDoneCh: make(chan error),
cancelCtx: cancel,
ctx: ctx,
scannedItemsMap: make(map[madmin.HealItemType]int64),
healedItemsMap: make(map[madmin.HealItemType]int64),
healFailedItemsMap: make(map[string]int64),
}
}
// getScannedItemsCount - returns a count of all scanned items
func (h *healSequence) getScannedItemsCount() int64 {
var count int64
h.mutex.RLock()
defer h.mutex.RUnlock()
for _, v := range h.scannedItemsMap {
count = count + v
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
return count
}
// getScannedItemsMap - returns map of all scanned items against type
func (h *healSequence) getScannedItemsMap() map[madmin.HealItemType]int64 {
h.mutex.RLock()
defer h.mutex.RUnlock()
// Make a copy before returning the value
retMap := make(map[madmin.HealItemType]int64, len(h.scannedItemsMap))
for k, v := range h.scannedItemsMap {
retMap[k] = v
}
return retMap
}
// getHealedItemsMap - returns the map of all healed items against type
func (h *healSequence) getHealedItemsMap() map[madmin.HealItemType]int64 {
h.mutex.RLock()
defer h.mutex.RUnlock()
// Make a copy before returning the value
retMap := make(map[madmin.HealItemType]int64, len(h.healedItemsMap))
for k, v := range h.healedItemsMap {
retMap[k] = v
}
return retMap
}
// gethealFailedItemsMap - returns map of all items where heal failed against
// drive endpoint and status
func (h *healSequence) gethealFailedItemsMap() map[string]int64 {
h.mutex.RLock()
defer h.mutex.RUnlock()
// Make a copy before returning the value
retMap := make(map[string]int64, len(h.healFailedItemsMap))
for k, v := range h.healFailedItemsMap {
retMap[k] = v
}
return retMap
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// isQuitting - determines if the heal sequence is quitting (due to an
// external signal)
func (h *healSequence) isQuitting() bool {
select {
case <-h.ctx.Done():
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return true
default:
return false
}
}
// check if the heal sequence has ended
func (h *healSequence) hasEnded() bool {
h.mutex.RLock()
defer h.mutex.RUnlock()
// background heal never ends
if h.clientToken == bgHealingUUID {
return false
}
return !h.endTime.IsZero()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// stops the heal sequence - safe to call multiple times.
func (h *healSequence) stop() {
h.cancelCtx()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// pushHealResultItem - pushes a heal result item for consumption in
// the heal-status API. It blocks if there are
// maxUnconsumedHealResultItems. When it blocks, the heal sequence
// routine is effectively paused - this happens when the server has
// accumulated the maximum number of heal records per heal
// sequence. When the client consumes further records, the heal
// sequence automatically resumes. The return value indicates if the
// operation succeeded.
func (h *healSequence) pushHealResultItem(r madmin.HealResultItem) error {
// start a timer to keep an upper time limit to find an empty
// slot to add the given heal result - if no slot is found it
// means that the server is holding the maximum amount of
// heal-results in memory and the client has not consumed it
// for too long.
unconsumedTimer := time.NewTimer(healUnconsumedTimeout)
defer func() {
// stop the timeout timer so it is garbage collected.
if !unconsumedTimer.Stop() {
<-unconsumedTimer.C
}
}()
var itemsLen int
for {
h.mutex.Lock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
itemsLen = len(h.currentStatus.Items)
if itemsLen == maxUnconsumedHealResultItems {
// wait for a second, or quit if an external
// stop signal is received or the
// unconsumedTimer fires.
select {
// Check after a second
case <-time.After(time.Second):
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
continue
case <-h.ctx.Done():
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// discard result and return.
return errHealStopSignalled
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Timeout if no results consumed for too long.
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
case <-unconsumedTimer.C:
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return errHealIdleTimeout
}
}
break
}
// Set the correct result index for the new result item
if itemsLen > 0 {
r.ResultIndex = 1 + h.currentStatus.Items[itemsLen-1].ResultIndex
} else {
r.ResultIndex = 1 + h.lastSentResultIndex
}
// append to results
h.currentStatus.Items = append(h.currentStatus.Items, r)
// release lock
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return nil
}
// healSequenceStart - this is the top-level background heal
// routine. It launches another go-routine that actually traverses
// on-disk data, checks and heals according to the selected
// settings. This go-routine itself, (1) monitors the traversal
// routine for completion, and (2) listens for external stop
// signals. When either event happens, it sets the finish status for
// the heal-sequence.
func (h *healSequence) healSequenceStart(objAPI ObjectLayer) {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Set status as running
h.mutex.Lock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
h.currentStatus.Summary = healRunningStatus
h.currentStatus.StartTime = UTCNow()
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
go h.traverseAndHeal(objAPI)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
select {
case err, ok := <-h.traverseAndHealDoneCh:
if !ok {
return
}
h.mutex.Lock()
h.endTime = UTCNow()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Heal traversal is complete.
if err == nil {
// heal traversal succeeded.
h.currentStatus.Summary = healFinishedStatus
} else {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// heal traversal had an error.
h.currentStatus.Summary = healStoppedStatus
h.currentStatus.FailureDetail = err.Error()
}
h.mutex.Unlock()
case <-h.ctx.Done():
h.mutex.Lock()
h.endTime = UTCNow()
h.currentStatus.Summary = healFinishedStatus
h.mutex.Unlock()
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// drain traverse channel so the traversal
// go-routine does not leak.
go func() {
// Eventually the traversal go-routine closes
// the channel and returns, so this go-routine
// itself will not leak.
<-h.traverseAndHealDoneCh
}()
}
}
func (h *healSequence) logHeal(healType madmin.HealItemType) {
h.mutex.Lock()
h.scannedItemsMap[healType]++
h.lastHealActivity = UTCNow()
h.mutex.Unlock()
}
func (h *healSequence) queueHealTask(source healSource, healType madmin.HealItemType) error {
// Send heal request
task := healTask{
bucket: source.bucket,
object: source.object,
versionID: source.versionID,
opts: h.settings,
respCh: h.respCh,
}
if source.opts != nil {
task.opts = *source.opts
}
task.opts.ScanMode = globalHealConfig.ScanMode()
h.mutex.Lock()
h.scannedItemsMap[healType]++
h.lastHealActivity = UTCNow()
h.mutex.Unlock()
select {
case globalBackgroundHealRoutine.tasks <- task:
case <-h.ctx.Done():
return nil
}
select {
case res := <-h.respCh:
if !h.reportProgress {
if errors.Is(res.err, errSkipFile) { // this is only sent usually by nopHeal
return nil
}
h.mutex.Lock()
defer h.mutex.Unlock()
// Progress is not reported in case of background heal processing.
// Instead we increment relevant counter based on the heal result
// for prometheus reporting.
if res.err != nil {
for _, d := range res.result.After.Drives {
// For failed items we report the endpoint and drive state
// This will help users take corrective actions for drives
h.healFailedItemsMap[d.Endpoint+","+d.State]++
}
} else {
// Only object type reported for successful healing
h.healedItemsMap[res.result.Type]++
}
// Report caller of any failure
return res.err
}
res.result.Type = healType
if res.err != nil {
// Only report object error
if healType != madmin.HealItemObject {
return res.err
}
res.result.Detail = res.err.Error()
}
return h.pushHealResultItem(res.result)
case <-h.ctx.Done():
return nil
}
}
func (h *healSequence) healDiskMeta(objAPI ObjectLayer) error {
// Start healing the config prefix.
return h.healMinioSysMeta(objAPI, minioConfigPrefix)()
}
func (h *healSequence) healItems(objAPI ObjectLayer, bucketsOnly bool) error {
if err := h.healDiskMeta(objAPI); err != nil {
return err
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Heal buckets and objects
return h.healBuckets(objAPI, bucketsOnly)
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// traverseAndHeal - traverses on-disk data and performs healing
// according to settings. At each "safe" point it also checks if an
// external quit signal has been received and quits if so. Since the
// healing traversal may be mutating on-disk data when an external
// quit signal is received, this routine cannot quit immediately and
// has to wait until a safe point is reached, such as between scanning
// two objects.
func (h *healSequence) traverseAndHeal(objAPI ObjectLayer) {
bucketsOnly := false // Heals buckets and objects also.
h.traverseAndHealDoneCh <- h.healItems(objAPI, bucketsOnly)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
close(h.traverseAndHealDoneCh)
}
// healMinioSysMeta - heals all files under a given meta prefix, returns a function
// which in-turn heals the respective meta directory path and any files in int.
func (h *healSequence) healMinioSysMeta(objAPI ObjectLayer, metaPrefix string) func() error {
return func() error {
// NOTE: Healing on meta is run regardless
// of any bucket being selected, this is to ensure that
// meta are always upto date and correct.
return objAPI.HealObjects(h.ctx, minioMetaBucket, metaPrefix, h.settings, func(bucket, object, versionID string) error {
if h.isQuitting() {
return errHealStopSignalled
}
err := h.queueHealTask(healSource{
bucket: bucket,
object: object,
versionID: versionID,
}, madmin.HealItemBucketMetadata)
return err
})
}
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// healDiskFormat - heals format.json, return value indicates if a
// failure error occurred.
func (h *healSequence) healDiskFormat() error {
if h.isQuitting() {
return errHealStopSignalled
}
return h.queueHealTask(healSource{bucket: SlashSeparator}, madmin.HealItemMetadata)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// healBuckets - check for all buckets heal or just particular bucket.
func (h *healSequence) healBuckets(objAPI ObjectLayer, bucketsOnly bool) error {
if h.isQuitting() {
return errHealStopSignalled
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// 1. If a bucket was specified, heal only the bucket.
if h.bucket != "" {
return h.healBucket(objAPI, h.bucket, bucketsOnly)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
buckets, err := objAPI.ListBuckets(h.ctx)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
if err != nil {
return errFnHealFromAPIErr(h.ctx, err)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
// Heal latest buckets first.
sort.Slice(buckets, func(i, j int) bool {
return buckets[i].Created.After(buckets[j].Created)
})
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
for _, bucket := range buckets {
if err = h.healBucket(objAPI, bucket.Name, bucketsOnly); err != nil {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return err
}
}
return nil
}
// healBucket - traverses and heals given bucket
func (h *healSequence) healBucket(objAPI ObjectLayer, bucket string, bucketsOnly bool) error {
if err := h.queueHealTask(healSource{bucket: bucket}, madmin.HealItemBucket); err != nil {
return err
}
if bucketsOnly {
return nil
}
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
if !h.settings.Recursive {
if h.object != "" {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
// Check if an object named as the objPrefix exists,
// and if so heal it.
oi, err := objAPI.GetObjectInfo(h.ctx, bucket, h.object, ObjectOptions{})
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
if err == nil {
if err = h.healObject(bucket, h.object, oi.VersionID); err != nil {
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
return err
}
}
}
return nil
}
if err := objAPI.HealObjects(h.ctx, bucket, h.object, h.settings, h.healObject); err != nil {
return errFnHealFromAPIErr(h.ctx, err)
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}
return nil
}
// healObject - heal the given object and record result
func (h *healSequence) healObject(bucket, object, versionID string) error {
if h.isQuitting() {
return errHealStopSignalled
}
err := h.queueHealTask(healSource{
bucket: bucket,
object: object,
versionID: versionID,
}, madmin.HealItemObject)
// Wait and proceed if there are active requests
waitForLowHTTPReq()
return err
Move admin APIs to new path and add redesigned heal APIs (#5351) - Changes related to moving admin APIs - admin APIs now have an endpoint under /minio/admin - admin APIs are now versioned - a new API to server the version is added at "GET /minio/admin/version" and all API operations have the path prefix /minio/admin/v1/<operation> - new service stop API added - credentials change API is moved to /minio/admin/v1/config/credential - credentials change API and configuration get/set API now require TLS so that credentials are protected - all API requests now receive JSON - heal APIs are disabled as they will be changed substantially - Heal API changes Heal API is now provided at a single endpoint with the ability for a client to start a heal sequence on all the data in the server, a single bucket, or under a prefix within a bucket. When a heal sequence is started, the server returns a unique token that needs to be used for subsequent 'status' requests to fetch heal results. On each status request from the client, the server returns heal result records that it has accumulated since the previous status request. The server accumulates upto 1000 records and pauses healing further objects until the client requests for status. If the client does not request any further records for a long time, the server aborts the heal sequence automatically. A heal result record is returned for each entity healed on the server, such as system metadata, object metadata, buckets and objects, and has information about the before and after states on each disk. A client may request to force restart a heal sequence - this causes the running heal sequence to be aborted at the next safe spot and starts a new heal sequence.
2018-01-22 23:54:55 +01:00
}