mirror of
https://github.com/minio/minio.git
synced 2024-12-24 22:25:54 -05:00
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.
This commit is contained in:
parent
f3f09ed14e
commit
a337ea4d11
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
683
cmd/admin-heal-ops.go
Normal file
683
cmd/admin-heal-ops.go
Normal file
@ -0,0 +1,683 @@
|
|||||||
|
/*
|
||||||
|
* Minio Cloud Storage, (C) 2017 Minio, Inc.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
* you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package cmd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
"fmt"
|
||||||
|
"strings"
|
||||||
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/minio/minio/pkg/madmin"
|
||||||
|
)
|
||||||
|
|
||||||
|
// 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
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
errHealIdleTimeout = fmt.Errorf("healing results were not consumed for too long")
|
||||||
|
errHealPushStopNDiscard = fmt.Errorf("heal push stopped due to heal stop signal")
|
||||||
|
errHealStopSignalled = fmt.Errorf("heal stop signalled")
|
||||||
|
|
||||||
|
errFnHealFromAPIErr = func(err error) error {
|
||||||
|
errCode := toAPIErrorCode(err)
|
||||||
|
apiErr := getAPIError(errCode)
|
||||||
|
return fmt.Errorf("Heal internal error: %s: %s",
|
||||||
|
apiErr.Code, apiErr.Description)
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
// healSequenceStatus - accumulated status of the heal sequence
|
||||||
|
type healSequenceStatus struct {
|
||||||
|
// lock to update this structure as it is concurrently
|
||||||
|
// accessed
|
||||||
|
updateLock *sync.RWMutex
|
||||||
|
|
||||||
|
// summary and detail for failures
|
||||||
|
Summary healStatusSummary `json:"Summary"`
|
||||||
|
FailureDetail string `json:"Detail,omitempty"`
|
||||||
|
StartTime time.Time `json:"StartTime"`
|
||||||
|
|
||||||
|
// disk information
|
||||||
|
NumDisks int `json:"NumDisks"`
|
||||||
|
|
||||||
|
// 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.Mutex
|
||||||
|
|
||||||
|
// map of heal path to heal sequence
|
||||||
|
healSeqMap map[string]*healSequence
|
||||||
|
}
|
||||||
|
|
||||||
|
var (
|
||||||
|
// global server heal state
|
||||||
|
globalAllHealState allHealState
|
||||||
|
)
|
||||||
|
|
||||||
|
// initAllHealState - initialize healing apparatus
|
||||||
|
func initAllHealState(isErasureMode bool) {
|
||||||
|
if !isErasureMode {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
globalAllHealState = allHealState{
|
||||||
|
healSeqMap: make(map[string]*healSequence),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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) (
|
||||||
|
respBytes []byte, errCode APIErrorCode, errMsg string) {
|
||||||
|
|
||||||
|
existsAndLive := false
|
||||||
|
he, exists := ahs.getHealSequence(h.path)
|
||||||
|
if exists {
|
||||||
|
if !he.hasEnded() || len(he.currentStatus.Items) > 0 {
|
||||||
|
existsAndLive = true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if existsAndLive {
|
||||||
|
// A heal sequence exists on the given path.
|
||||||
|
if h.forceStarted {
|
||||||
|
// stop the running heal sequence - wait for
|
||||||
|
// it to finish.
|
||||||
|
he.stop()
|
||||||
|
for !he.hasEnded() {
|
||||||
|
time.Sleep(10 * time.Second)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
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",
|
||||||
|
h.clientAddress, h.startTime)
|
||||||
|
|
||||||
|
return nil, ErrHealAlreadyRunning, errMsg
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ahs.Lock()
|
||||||
|
defer ahs.Unlock()
|
||||||
|
|
||||||
|
// Check if new heal sequence to be started overlaps with any
|
||||||
|
// existing, running sequence
|
||||||
|
for k, hSeq := range ahs.healSeqMap {
|
||||||
|
if !hSeq.hasEnded() && (strings.HasPrefix(k, h.path) ||
|
||||||
|
strings.HasPrefix(h.path, k)) {
|
||||||
|
|
||||||
|
errMsg = "The provided heal sequence path overlaps with an existing " +
|
||||||
|
fmt.Sprintf("heal path: %s", k)
|
||||||
|
return nil, ErrHealOverlappingPaths, errMsg
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add heal state and start sequence
|
||||||
|
ahs.healSeqMap[h.path] = h
|
||||||
|
|
||||||
|
// Launch top-level background heal go-routine
|
||||||
|
go h.healSequenceStart()
|
||||||
|
|
||||||
|
// Launch clean-up routine to remove this heal sequence (after
|
||||||
|
// it ends) from the global state after timeout has elapsed.
|
||||||
|
go func() {
|
||||||
|
var keepStateTimeout <-chan time.Time
|
||||||
|
ticker := time.NewTicker(time.Minute)
|
||||||
|
defer ticker.Stop()
|
||||||
|
everyMinute := ticker.C
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
// Check every minute if heal sequence has ended.
|
||||||
|
case <-everyMinute:
|
||||||
|
if h.hasEnded() {
|
||||||
|
keepStateTimeout = time.After(keepHealSeqStateDuration)
|
||||||
|
everyMinute = nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// This case does not fire until the heal
|
||||||
|
// sequence completes.
|
||||||
|
case <-keepStateTimeout:
|
||||||
|
// Heal sequence has ended, keep
|
||||||
|
// results state duration has elapsed,
|
||||||
|
// so purge state.
|
||||||
|
ahs.Lock()
|
||||||
|
defer ahs.Unlock()
|
||||||
|
delete(ahs.healSeqMap, h.path)
|
||||||
|
return
|
||||||
|
|
||||||
|
case <-globalServiceDoneCh:
|
||||||
|
// server could be restarting - need
|
||||||
|
// to exit immediately
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
b, err := json.Marshal(madmin.HealStartSuccess{
|
||||||
|
h.clientToken,
|
||||||
|
h.clientAddress,
|
||||||
|
h.startTime,
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
errorIf(err, "Failed to marshal heal result into json.")
|
||||||
|
return nil, ErrInternalError, ""
|
||||||
|
}
|
||||||
|
return b, ErrNone, ""
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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(path string,
|
||||||
|
clientToken string) ([]byte, APIErrorCode) {
|
||||||
|
|
||||||
|
// fetch heal state for given path
|
||||||
|
h, exists := ahs.getHealSequence(path)
|
||||||
|
if !exists {
|
||||||
|
// If there is no such heal sequence, return error.
|
||||||
|
return nil, ErrHealNoSuchProcess
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check if client-token is valid
|
||||||
|
if clientToken != h.clientToken {
|
||||||
|
return nil, ErrHealInvalidClientToken
|
||||||
|
}
|
||||||
|
|
||||||
|
// Take lock to access and update the heal-sequence
|
||||||
|
h.currentStatus.updateLock.Lock()
|
||||||
|
defer h.currentStatus.updateLock.Unlock()
|
||||||
|
|
||||||
|
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
|
||||||
|
}
|
||||||
|
|
||||||
|
// After sending status to client, and before relinquishing
|
||||||
|
// the updateLock, reset Item to nil and record the result
|
||||||
|
// index sent to the client.
|
||||||
|
defer func(i int64) {
|
||||||
|
h.lastSentResultIndex = i
|
||||||
|
h.currentStatus.Items = nil
|
||||||
|
}(lastResultIndex)
|
||||||
|
|
||||||
|
jbytes, err := json.Marshal(h.currentStatus)
|
||||||
|
if err != nil {
|
||||||
|
errorIf(err, "Failed to marshal heal result into json.")
|
||||||
|
return nil, ErrInternalError
|
||||||
|
}
|
||||||
|
|
||||||
|
return jbytes, ErrNone
|
||||||
|
}
|
||||||
|
|
||||||
|
// healSequence - state for each heal sequence initiated on the
|
||||||
|
// server.
|
||||||
|
type healSequence struct {
|
||||||
|
// bucket, and prefix on which heal seq. was initiated
|
||||||
|
bucket, objPrefix string
|
||||||
|
|
||||||
|
// path is just bucket + "/" + objPrefix
|
||||||
|
path string
|
||||||
|
|
||||||
|
// time at which heal sequence was started
|
||||||
|
startTime time.Time
|
||||||
|
|
||||||
|
// 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 signalled by background routine when traversal has
|
||||||
|
// completed
|
||||||
|
traverseAndHealDoneCh chan error
|
||||||
|
|
||||||
|
// channel to signal heal sequence to stop (e.g. from the
|
||||||
|
// heal-stop API)
|
||||||
|
stopSignalCh chan struct{}
|
||||||
|
|
||||||
|
// the last result index sent to client
|
||||||
|
lastSentResultIndex int64
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewHealSequence - creates healSettings, assumes bucket and
|
||||||
|
// objPrefix are already validated.
|
||||||
|
func newHealSequence(bucket, objPrefix, clientAddr string,
|
||||||
|
numDisks int, hs madmin.HealOpts, forceStart bool) *healSequence {
|
||||||
|
|
||||||
|
return &healSequence{
|
||||||
|
bucket: bucket,
|
||||||
|
objPrefix: objPrefix,
|
||||||
|
path: bucket + "/" + objPrefix,
|
||||||
|
startTime: UTCNow(),
|
||||||
|
clientToken: mustGetUUID(),
|
||||||
|
clientAddress: clientAddr,
|
||||||
|
forceStarted: forceStart,
|
||||||
|
settings: hs,
|
||||||
|
currentStatus: healSequenceStatus{
|
||||||
|
Summary: healNotStartedStatus,
|
||||||
|
HealSettings: hs,
|
||||||
|
NumDisks: numDisks,
|
||||||
|
updateLock: &sync.RWMutex{},
|
||||||
|
},
|
||||||
|
traverseAndHealDoneCh: make(chan error),
|
||||||
|
stopSignalCh: make(chan struct{}),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// isQuitting - determines if the heal sequence is quitting (due to an
|
||||||
|
// external signal)
|
||||||
|
func (h *healSequence) isQuitting() bool {
|
||||||
|
select {
|
||||||
|
case <-h.stopSignalCh:
|
||||||
|
return true
|
||||||
|
default:
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// check if the heal sequence has ended
|
||||||
|
func (h *healSequence) hasEnded() bool {
|
||||||
|
h.currentStatus.updateLock.RLock()
|
||||||
|
summary := h.currentStatus.Summary
|
||||||
|
h.currentStatus.updateLock.RUnlock()
|
||||||
|
return summary == healStoppedStatus || summary == healFinishedStatus
|
||||||
|
}
|
||||||
|
|
||||||
|
// stops the heal sequence - safe to call multiple times.
|
||||||
|
func (h *healSequence) stop() {
|
||||||
|
select {
|
||||||
|
case <-h.stopSignalCh:
|
||||||
|
default:
|
||||||
|
close(h.stopSignalCh)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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.currentStatus.updateLock.Lock()
|
||||||
|
itemsLen = len(h.currentStatus.Items)
|
||||||
|
if itemsLen == maxUnconsumedHealResultItems {
|
||||||
|
// unlock and wait to check again if we can push
|
||||||
|
h.currentStatus.updateLock.Unlock()
|
||||||
|
|
||||||
|
// 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):
|
||||||
|
continue
|
||||||
|
|
||||||
|
case <-h.stopSignalCh:
|
||||||
|
// discard result and return.
|
||||||
|
return errHealPushStopNDiscard
|
||||||
|
|
||||||
|
// Timeout if no results consumed for too
|
||||||
|
// long.
|
||||||
|
case <-unconsumedTimer.C:
|
||||||
|
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.currentStatus.updateLock.Unlock()
|
||||||
|
|
||||||
|
// This is a "safe" point for the heal sequence to quit if
|
||||||
|
// signalled externally.
|
||||||
|
if h.isQuitting() {
|
||||||
|
return errHealStopSignalled
|
||||||
|
}
|
||||||
|
|
||||||
|
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() {
|
||||||
|
// Set status as running
|
||||||
|
h.currentStatus.updateLock.Lock()
|
||||||
|
h.currentStatus.Summary = healRunningStatus
|
||||||
|
h.currentStatus.StartTime = UTCNow()
|
||||||
|
h.currentStatus.updateLock.Unlock()
|
||||||
|
|
||||||
|
go h.traverseAndHeal()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case err, ok := <-h.traverseAndHealDoneCh:
|
||||||
|
h.currentStatus.updateLock.Lock()
|
||||||
|
defer h.currentStatus.updateLock.Unlock()
|
||||||
|
// Heal traversal is complete.
|
||||||
|
if ok {
|
||||||
|
// heal traversal had an error.
|
||||||
|
h.currentStatus.Summary = healStoppedStatus
|
||||||
|
h.currentStatus.FailureDetail = err.Error()
|
||||||
|
} else {
|
||||||
|
// heal traversal succeeded.
|
||||||
|
h.currentStatus.Summary = healFinishedStatus
|
||||||
|
}
|
||||||
|
|
||||||
|
case <-h.stopSignalCh:
|
||||||
|
h.currentStatus.updateLock.Lock()
|
||||||
|
h.currentStatus.Summary = healStoppedStatus
|
||||||
|
h.currentStatus.FailureDetail = errHealStopSignalled.Error()
|
||||||
|
h.currentStatus.updateLock.Unlock()
|
||||||
|
|
||||||
|
// 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
|
||||||
|
}()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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() {
|
||||||
|
var err error
|
||||||
|
checkErr := func(f func() error) {
|
||||||
|
switch {
|
||||||
|
case err != nil:
|
||||||
|
return
|
||||||
|
case h.isQuitting():
|
||||||
|
err = errHealStopSignalled
|
||||||
|
return
|
||||||
|
}
|
||||||
|
err = f()
|
||||||
|
}
|
||||||
|
|
||||||
|
// Start with format healing
|
||||||
|
checkErr(h.healDiskFormat)
|
||||||
|
|
||||||
|
// Heal buckets and objects
|
||||||
|
checkErr(h.healBuckets)
|
||||||
|
|
||||||
|
if err != nil {
|
||||||
|
h.traverseAndHealDoneCh <- err
|
||||||
|
}
|
||||||
|
|
||||||
|
close(h.traverseAndHealDoneCh)
|
||||||
|
}
|
||||||
|
|
||||||
|
// healDiskFormat - heals format.json, return value indicates if a
|
||||||
|
// failure error occurred.
|
||||||
|
func (h *healSequence) healDiskFormat() error {
|
||||||
|
// Get current object layer instance.
|
||||||
|
objectAPI := newObjectLayerFn()
|
||||||
|
if objectAPI == nil {
|
||||||
|
return errServerNotInitialized
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create a new set of storage instances to heal format.json.
|
||||||
|
bootstrapDisks, err := initStorageDisks(globalEndpoints)
|
||||||
|
if err != nil {
|
||||||
|
return errFnHealFromAPIErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wrap into retrying disks
|
||||||
|
retryingDisks := initRetryableStorageDisks(bootstrapDisks,
|
||||||
|
time.Millisecond, time.Millisecond*5,
|
||||||
|
globalStorageHealthCheckInterval, globalStorageRetryThreshold)
|
||||||
|
|
||||||
|
// Heal format.json on available storage.
|
||||||
|
hres, err := healFormatXL(retryingDisks, h.settings.DryRun)
|
||||||
|
if err != nil {
|
||||||
|
return errFnHealFromAPIErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// reload object layer global only if we healed some disk
|
||||||
|
onlineBefore, onlineAfter := hres.GetOnlineCounts()
|
||||||
|
numHealed := onlineAfter - onlineBefore
|
||||||
|
if numHealed > 0 {
|
||||||
|
// Instantiate new object layer with newly formatted
|
||||||
|
// storage.
|
||||||
|
newObjectAPI, err := newXLObjects(retryingDisks)
|
||||||
|
if err != nil {
|
||||||
|
return errFnHealFromAPIErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Set object layer with newly formatted storage to
|
||||||
|
// globalObjectAPI.
|
||||||
|
globalObjLayerMutex.Lock()
|
||||||
|
globalObjectAPI = newObjectAPI
|
||||||
|
globalObjLayerMutex.Unlock()
|
||||||
|
|
||||||
|
// Shutdown storage belonging to old object layer
|
||||||
|
// instance.
|
||||||
|
objectAPI.Shutdown()
|
||||||
|
|
||||||
|
// Inform peers to reinitialize storage with newly
|
||||||
|
// formatted storage.
|
||||||
|
reInitPeerDisks(globalAdminPeers)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Push format heal result
|
||||||
|
return h.pushHealResultItem(hres)
|
||||||
|
}
|
||||||
|
|
||||||
|
// healBuckets - check for all buckets heal or just particular bucket.
|
||||||
|
func (h *healSequence) healBuckets() error {
|
||||||
|
// 1. If a bucket was specified, heal only the bucket.
|
||||||
|
if h.bucket != "" {
|
||||||
|
return h.healBucket(h.bucket)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get current object layer instance.
|
||||||
|
objectAPI := newObjectLayerFn()
|
||||||
|
if objectAPI == nil {
|
||||||
|
return errServerNotInitialized
|
||||||
|
}
|
||||||
|
|
||||||
|
buckets, err := objectAPI.ListBucketsHeal()
|
||||||
|
if err != nil {
|
||||||
|
return errFnHealFromAPIErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, bucket := range buckets {
|
||||||
|
err = h.healBucket(bucket.Name)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// healBucket - traverses and heals given bucket
|
||||||
|
func (h *healSequence) healBucket(bucket string) error {
|
||||||
|
if h.isQuitting() {
|
||||||
|
return errHealStopSignalled
|
||||||
|
}
|
||||||
|
// Get current object layer instance.
|
||||||
|
objectAPI := newObjectLayerFn()
|
||||||
|
if objectAPI == nil {
|
||||||
|
return errServerNotInitialized
|
||||||
|
}
|
||||||
|
|
||||||
|
results, err := objectAPI.HealBucket(bucket, h.settings.DryRun)
|
||||||
|
// push any available results before checking for error
|
||||||
|
for _, result := range results {
|
||||||
|
if perr := h.pushHealResultItem(result); perr != nil {
|
||||||
|
return perr
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// handle heal-bucket error
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
if !h.settings.Recursive {
|
||||||
|
if h.objPrefix != "" {
|
||||||
|
// Check if an object named as the objPrefix exists,
|
||||||
|
// and if so heal it.
|
||||||
|
_, err = objectAPI.GetObjectInfo(bucket, h.objPrefix)
|
||||||
|
if err == nil {
|
||||||
|
err = h.healObject(bucket, h.objPrefix)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
marker := ""
|
||||||
|
isTruncated := true
|
||||||
|
for isTruncated {
|
||||||
|
objectInfos, err := objectAPI.ListObjectsHeal(bucket,
|
||||||
|
h.objPrefix, marker, "", 1000)
|
||||||
|
if err != nil {
|
||||||
|
return errFnHealFromAPIErr(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, o := range objectInfos.Objects {
|
||||||
|
if err := h.healObject(o.Bucket, o.Name); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
isTruncated = objectInfos.IsTruncated
|
||||||
|
marker = objectInfos.NextMarker
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// healObject - heal the given object and record result
|
||||||
|
func (h *healSequence) healObject(bucket, object string) error {
|
||||||
|
if h.isQuitting() {
|
||||||
|
return errHealStopSignalled
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get current object layer instance.
|
||||||
|
objectAPI := newObjectLayerFn()
|
||||||
|
if objectAPI == nil {
|
||||||
|
return errServerNotInitialized
|
||||||
|
}
|
||||||
|
|
||||||
|
hri, err := objectAPI.HealObject(bucket, object, h.settings.DryRun)
|
||||||
|
if err != nil {
|
||||||
|
hri.Detail = err.Error()
|
||||||
|
}
|
||||||
|
return h.pushHealResultItem(hri)
|
||||||
|
}
|
@ -16,7 +16,15 @@
|
|||||||
|
|
||||||
package cmd
|
package cmd
|
||||||
|
|
||||||
import router "github.com/gorilla/mux"
|
import (
|
||||||
|
"net/http"
|
||||||
|
|
||||||
|
router "github.com/gorilla/mux"
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
adminAPIPathPrefix = "/minio/admin"
|
||||||
|
)
|
||||||
|
|
||||||
// adminAPIHandlers provides HTTP handlers for Minio admin API.
|
// adminAPIHandlers provides HTTP handlers for Minio admin API.
|
||||||
type adminAPIHandlers struct {
|
type adminAPIHandlers struct {
|
||||||
@ -27,46 +35,44 @@ func registerAdminRouter(mux *router.Router) {
|
|||||||
|
|
||||||
adminAPI := adminAPIHandlers{}
|
adminAPI := adminAPIHandlers{}
|
||||||
// Admin router
|
// Admin router
|
||||||
adminRouter := mux.NewRoute().PathPrefix("/").Subrouter()
|
adminRouter := mux.NewRoute().PathPrefix(adminAPIPathPrefix).Subrouter()
|
||||||
|
|
||||||
|
// Version handler
|
||||||
|
adminRouter.Methods(http.MethodGet).Path("/version").HandlerFunc(adminAPI.VersionHandler)
|
||||||
|
|
||||||
|
adminV1Router := adminRouter.PathPrefix("/v1").Subrouter()
|
||||||
|
|
||||||
/// Service operations
|
/// Service operations
|
||||||
|
|
||||||
// Service status
|
// Service status
|
||||||
adminRouter.Methods("GET").Queries("service", "").Headers(minioAdminOpHeader, "status").HandlerFunc(adminAPI.ServiceStatusHandler)
|
adminV1Router.Methods(http.MethodGet).Path("/service").HandlerFunc(adminAPI.ServiceStatusHandler)
|
||||||
|
|
||||||
// Service restart
|
// Service restart and stop - TODO
|
||||||
adminRouter.Methods("POST").Queries("service", "").Headers(minioAdminOpHeader, "restart").HandlerFunc(adminAPI.ServiceRestartHandler)
|
adminV1Router.Methods(http.MethodPost).Path("/service").HandlerFunc(adminAPI.ServiceStopNRestartHandler)
|
||||||
// Service update credentials
|
|
||||||
adminRouter.Methods("POST").Queries("service", "").Headers(minioAdminOpHeader, "set-credentials").HandlerFunc(adminAPI.ServiceCredentialsHandler)
|
|
||||||
|
|
||||||
// Info operations
|
// Info operations
|
||||||
adminRouter.Methods("GET").Queries("info", "").HandlerFunc(adminAPI.ServerInfoHandler)
|
adminV1Router.Methods(http.MethodGet).Path("/info").HandlerFunc(adminAPI.ServerInfoHandler)
|
||||||
|
|
||||||
/// Lock operations
|
/// Lock operations
|
||||||
|
|
||||||
// List Locks
|
// List Locks
|
||||||
adminRouter.Methods("GET").Queries("lock", "").Headers(minioAdminOpHeader, "list").HandlerFunc(adminAPI.ListLocksHandler)
|
adminV1Router.Methods(http.MethodGet).Path("/locks").HandlerFunc(adminAPI.ListLocksHandler)
|
||||||
// Clear locks
|
// Clear locks
|
||||||
adminRouter.Methods("POST").Queries("lock", "").Headers(minioAdminOpHeader, "clear").HandlerFunc(adminAPI.ClearLocksHandler)
|
adminV1Router.Methods(http.MethodDelete).Path("/locks").HandlerFunc(adminAPI.ClearLocksHandler)
|
||||||
|
|
||||||
/// Heal operations
|
/// Heal operations
|
||||||
|
|
||||||
// List Objects needing heal.
|
// Heal processing endpoint.
|
||||||
adminRouter.Methods("GET").Queries("heal", "").Headers(minioAdminOpHeader, "list-objects").HandlerFunc(adminAPI.ListObjectsHealHandler)
|
adminV1Router.Methods(http.MethodPost).Path("/heal/").HandlerFunc(adminAPI.HealHandler)
|
||||||
// List Buckets needing heal.
|
adminV1Router.Methods(http.MethodPost).Path("/heal/{bucket}").HandlerFunc(adminAPI.HealHandler)
|
||||||
adminRouter.Methods("GET").Queries("heal", "").Headers(minioAdminOpHeader, "list-buckets").HandlerFunc(adminAPI.ListBucketsHealHandler)
|
adminV1Router.Methods(http.MethodPost).Path("/heal/{bucket}/{prefix:.*}").HandlerFunc(adminAPI.HealHandler)
|
||||||
|
|
||||||
// Heal Buckets.
|
|
||||||
adminRouter.Methods("POST").Queries("heal", "").Headers(minioAdminOpHeader, "bucket").HandlerFunc(adminAPI.HealBucketHandler)
|
|
||||||
// Heal Objects.
|
|
||||||
adminRouter.Methods("POST").Queries("heal", "").Headers(minioAdminOpHeader, "object").HandlerFunc(adminAPI.HealObjectHandler)
|
|
||||||
// Heal Format.
|
|
||||||
adminRouter.Methods("POST").Queries("heal", "").Headers(minioAdminOpHeader, "format").HandlerFunc(adminAPI.HealFormatHandler)
|
|
||||||
|
|
||||||
/// Config operations
|
/// Config operations
|
||||||
|
|
||||||
|
// Update credentials
|
||||||
|
adminV1Router.Methods(http.MethodPut).Path("/config/credential").HandlerFunc(adminAPI.UpdateCredentialsHandler)
|
||||||
// Get config
|
// Get config
|
||||||
adminRouter.Methods("GET").Queries("config", "").Headers(minioAdminOpHeader, "get").HandlerFunc(adminAPI.GetConfigHandler)
|
adminV1Router.Methods(http.MethodGet).Path("/config").HandlerFunc(adminAPI.GetConfigHandler)
|
||||||
// Set Config
|
// Set config
|
||||||
adminRouter.Methods("PUT").Queries("config", "").Headers(minioAdminOpHeader, "set").HandlerFunc(adminAPI.SetConfigHandler)
|
adminV1Router.Methods(http.MethodPut).Path("/config").HandlerFunc(adminAPI.SetConfigHandler)
|
||||||
}
|
}
|
||||||
|
@ -34,7 +34,7 @@ import (
|
|||||||
|
|
||||||
const (
|
const (
|
||||||
// Admin service names
|
// Admin service names
|
||||||
serviceRestartRPC = "Admin.Restart"
|
signalServiceRPC = "Admin.SignalService"
|
||||||
listLocksRPC = "Admin.ListLocks"
|
listLocksRPC = "Admin.ListLocks"
|
||||||
reInitDisksRPC = "Admin.ReInitDisks"
|
reInitDisksRPC = "Admin.ReInitDisks"
|
||||||
serverInfoDataRPC = "Admin.ServerInfoData"
|
serverInfoDataRPC = "Admin.ServerInfoData"
|
||||||
@ -56,7 +56,7 @@ type remoteAdminClient struct {
|
|||||||
// adminCmdRunner - abstracts local and remote execution of admin
|
// adminCmdRunner - abstracts local and remote execution of admin
|
||||||
// commands like service stop and service restart.
|
// commands like service stop and service restart.
|
||||||
type adminCmdRunner interface {
|
type adminCmdRunner interface {
|
||||||
Restart() error
|
SignalService(s serviceSignal) error
|
||||||
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
||||||
ReInitDisks() error
|
ReInitDisks() error
|
||||||
ServerInfoData() (ServerInfoData, error)
|
ServerInfoData() (ServerInfoData, error)
|
||||||
@ -65,10 +65,16 @@ type adminCmdRunner interface {
|
|||||||
CommitConfig(tmpFileName string) error
|
CommitConfig(tmpFileName string) error
|
||||||
}
|
}
|
||||||
|
|
||||||
// Restart - Sends a message over channel to the go-routine
|
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
|
||||||
// responsible for restarting the process.
|
|
||||||
func (lc localAdminClient) Restart() error {
|
// SignalService - sends a restart or stop signal to the local server
|
||||||
globalServiceSignalCh <- serviceRestart
|
func (lc localAdminClient) SignalService(s serviceSignal) error {
|
||||||
|
switch s {
|
||||||
|
case serviceRestart, serviceStop:
|
||||||
|
globalServiceSignalCh <- s
|
||||||
|
default:
|
||||||
|
return errUnsupportedSignal
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -77,25 +83,31 @@ func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Durati
|
|||||||
return listLocksInfo(bucket, prefix, duration), nil
|
return listLocksInfo(bucket, prefix, duration), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Restart - Sends restart command to remote server via RPC.
|
func (rc remoteAdminClient) SignalService(s serviceSignal) (err error) {
|
||||||
func (rc remoteAdminClient) Restart() error {
|
switch s {
|
||||||
args := AuthRPCArgs{}
|
case serviceRestart, serviceStop:
|
||||||
reply := AuthRPCReply{}
|
reply := AuthRPCReply{}
|
||||||
return rc.Call(serviceRestartRPC, &args, &reply)
|
err = rc.Call(signalServiceRPC, &SignalServiceArgs{Sig: s},
|
||||||
|
&reply)
|
||||||
|
default:
|
||||||
|
err = errUnsupportedSignal
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListLocks - Sends list locks command to remote server via RPC.
|
// ListLocks - Sends list locks command to remote server via RPC.
|
||||||
func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||||
listArgs := ListLocksQuery{
|
listArgs := ListLocksQuery{
|
||||||
bucket: bucket,
|
Bucket: bucket,
|
||||||
prefix: prefix,
|
Prefix: prefix,
|
||||||
duration: duration,
|
Duration: duration,
|
||||||
}
|
}
|
||||||
var reply ListLocksReply
|
var reply ListLocksReply
|
||||||
if err := rc.Call(listLocksRPC, &listArgs, &reply); err != nil {
|
if err := rc.Call(listLocksRPC, &listArgs, &reply); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
return reply.volLocks, nil
|
return reply.VolLocks, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReInitDisks - There is nothing to do here, heal format REST API
|
// ReInitDisks - There is nothing to do here, heal format REST API
|
||||||
@ -225,7 +237,7 @@ func (rc remoteAdminClient) CommitConfig(tmpFileName string) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// adminPeer - represents an entity that implements Restart methods.
|
// adminPeer - represents an entity that implements admin API RPCs.
|
||||||
type adminPeer struct {
|
type adminPeer struct {
|
||||||
addr string
|
addr string
|
||||||
cmdRunner adminCmdRunner
|
cmdRunner adminCmdRunner
|
||||||
@ -274,11 +286,11 @@ func initGlobalAdminPeers(endpoints EndpointList) {
|
|||||||
globalAdminPeers = makeAdminPeers(endpoints)
|
globalAdminPeers = makeAdminPeers(endpoints)
|
||||||
}
|
}
|
||||||
|
|
||||||
// invokeServiceCmd - Invoke Restart command.
|
// invokeServiceCmd - Invoke Restart/Stop command.
|
||||||
func invokeServiceCmd(cp adminPeer, cmd serviceSignal) (err error) {
|
func invokeServiceCmd(cp adminPeer, cmd serviceSignal) (err error) {
|
||||||
switch cmd {
|
switch cmd {
|
||||||
case serviceRestart:
|
case serviceRestart, serviceStop:
|
||||||
err = cp.cmdRunner.Restart()
|
err = cp.cmdRunner.SignalService(cmd)
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -38,18 +38,24 @@ type adminCmd struct {
|
|||||||
AuthRPCServer
|
AuthRPCServer
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// SignalServiceArgs - provides the signal argument to SignalService RPC
|
||||||
|
type SignalServiceArgs struct {
|
||||||
|
AuthRPCArgs
|
||||||
|
Sig serviceSignal
|
||||||
|
}
|
||||||
|
|
||||||
// ListLocksQuery - wraps ListLocks API's query values to send over RPC.
|
// ListLocksQuery - wraps ListLocks API's query values to send over RPC.
|
||||||
type ListLocksQuery struct {
|
type ListLocksQuery struct {
|
||||||
AuthRPCArgs
|
AuthRPCArgs
|
||||||
bucket string
|
Bucket string
|
||||||
prefix string
|
Prefix string
|
||||||
duration time.Duration
|
Duration time.Duration
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListLocksReply - wraps ListLocks response over RPC.
|
// ListLocksReply - wraps ListLocks response over RPC.
|
||||||
type ListLocksReply struct {
|
type ListLocksReply struct {
|
||||||
AuthRPCReply
|
AuthRPCReply
|
||||||
volLocks []VolumeLockInfo
|
VolLocks []VolumeLockInfo
|
||||||
}
|
}
|
||||||
|
|
||||||
// ServerInfoDataReply - wraps the server info response over RPC.
|
// ServerInfoDataReply - wraps the server info response over RPC.
|
||||||
@ -64,13 +70,13 @@ type ConfigReply struct {
|
|||||||
Config []byte // json-marshalled bytes of serverConfigV13
|
Config []byte // json-marshalled bytes of serverConfigV13
|
||||||
}
|
}
|
||||||
|
|
||||||
// Restart - Restart this instance of minio server.
|
// SignalService - Send a restart or stop signal to the service
|
||||||
func (s *adminCmd) Restart(args *AuthRPCArgs, reply *AuthRPCReply) error {
|
func (s *adminCmd) SignalService(args *SignalServiceArgs, reply *AuthRPCReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
if err := args.IsAuthenticated(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
globalServiceSignalCh <- serviceRestart
|
globalServiceSignalCh <- args.Sig
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -79,8 +85,8 @@ func (s *adminCmd) ListLocks(query *ListLocksQuery, reply *ListLocksReply) error
|
|||||||
if err := query.IsAuthenticated(); err != nil {
|
if err := query.IsAuthenticated(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
volLocks := listLocksInfo(query.bucket, query.prefix, query.duration)
|
volLocks := listLocksInfo(query.Bucket, query.Prefix, query.Duration)
|
||||||
*reply = ListLocksReply{volLocks: volLocks}
|
*reply = ListLocksReply{VolLocks: volLocks}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -23,8 +23,8 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
func testAdminCmd(cmd cmdType, t *testing.T) {
|
func testAdminCmd(cmd cmdType, t *testing.T) {
|
||||||
// reset globals.
|
// reset globals. this is to make sure that the tests are not
|
||||||
// this is to make sure that the tests are not affected by modified globals.
|
// affected by modified globals.
|
||||||
resetTestGlobals()
|
resetTestGlobals()
|
||||||
|
|
||||||
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||||
@ -55,12 +55,22 @@ func testAdminCmd(cmd cmdType, t *testing.T) {
|
|||||||
<-globalServiceSignalCh
|
<-globalServiceSignalCh
|
||||||
}()
|
}()
|
||||||
|
|
||||||
ga := AuthRPCArgs{AuthToken: token}
|
sa := SignalServiceArgs{
|
||||||
|
AuthRPCArgs: AuthRPCArgs{AuthToken: token},
|
||||||
|
Sig: cmd.toServiceSignal(),
|
||||||
|
}
|
||||||
genReply := AuthRPCReply{}
|
genReply := AuthRPCReply{}
|
||||||
switch cmd {
|
switch cmd {
|
||||||
case restartCmd:
|
case restartCmd, stopCmd:
|
||||||
if err = adminServer.Restart(&ga, &genReply); err != nil {
|
if err = adminServer.SignalService(&sa, &genReply); err != nil {
|
||||||
t.Errorf("restartCmd: Expected: <nil>, got: %v", err)
|
t.Errorf("restartCmd/stopCmd: Expected: <nil>, got: %v",
|
||||||
|
err)
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
err = adminServer.SignalService(&sa, &genReply)
|
||||||
|
if err != nil && err.Error() != errUnsupportedSignal.Error() {
|
||||||
|
t.Errorf("invalidSignal %s: unexpected error got: %v",
|
||||||
|
cmd, err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -70,6 +80,16 @@ func TestAdminRestart(t *testing.T) {
|
|||||||
testAdminCmd(restartCmd, t)
|
testAdminCmd(restartCmd, t)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TestAdminStop - test for Admin.Stop RPC service.
|
||||||
|
func TestAdminStop(t *testing.T) {
|
||||||
|
testAdminCmd(stopCmd, t)
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestAdminStatus - test for Admin.Status RPC service (error case)
|
||||||
|
func TestAdminStatus(t *testing.T) {
|
||||||
|
testAdminCmd(statusCmd, t)
|
||||||
|
}
|
||||||
|
|
||||||
// TestReInitDisks - test for Admin.ReInitDisks RPC service.
|
// TestReInitDisks - test for Admin.ReInitDisks RPC service.
|
||||||
func TestReInitDisks(t *testing.T) {
|
func TestReInitDisks(t *testing.T) {
|
||||||
// Reset global variables to start afresh.
|
// Reset global variables to start afresh.
|
||||||
|
@ -41,8 +41,8 @@ type APIErrorResponse struct {
|
|||||||
Key string
|
Key string
|
||||||
BucketName string
|
BucketName string
|
||||||
Resource string
|
Resource string
|
||||||
RequestID string `xml:"RequestId"`
|
RequestID string `xml:"RequestId" json:"RequestId"`
|
||||||
HostID string `xml:"HostId"`
|
HostID string `xml:"HostId" json:"HostId"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// APIErrorCode type of error status.
|
// APIErrorCode type of error status.
|
||||||
@ -158,6 +158,7 @@ const (
|
|||||||
ErrReadQuorum
|
ErrReadQuorum
|
||||||
ErrWriteQuorum
|
ErrWriteQuorum
|
||||||
ErrStorageFull
|
ErrStorageFull
|
||||||
|
ErrRequestBodyParse
|
||||||
ErrObjectExistsAsDirectory
|
ErrObjectExistsAsDirectory
|
||||||
ErrPolicyNesting
|
ErrPolicyNesting
|
||||||
ErrInvalidObjectName
|
ErrInvalidObjectName
|
||||||
@ -174,6 +175,7 @@ const (
|
|||||||
// Please open a https://github.com/minio/minio/issues before adding
|
// Please open a https://github.com/minio/minio/issues before adding
|
||||||
// new error codes here.
|
// new error codes here.
|
||||||
|
|
||||||
|
ErrMalformedJSON
|
||||||
ErrAdminInvalidAccessKey
|
ErrAdminInvalidAccessKey
|
||||||
ErrAdminInvalidSecretKey
|
ErrAdminInvalidSecretKey
|
||||||
ErrAdminConfigNoQuorum
|
ErrAdminConfigNoQuorum
|
||||||
@ -183,6 +185,11 @@ const (
|
|||||||
ErrInsecureClientRequest
|
ErrInsecureClientRequest
|
||||||
ErrObjectTampered
|
ErrObjectTampered
|
||||||
ErrHealNotImplemented
|
ErrHealNotImplemented
|
||||||
|
ErrHealNoSuchProcess
|
||||||
|
ErrHealInvalidClientToken
|
||||||
|
ErrHealMissingBucket
|
||||||
|
ErrHealAlreadyRunning
|
||||||
|
ErrHealOverlappingPaths
|
||||||
)
|
)
|
||||||
|
|
||||||
// error code to APIError structure, these fields carry respective
|
// error code to APIError structure, these fields carry respective
|
||||||
@ -673,6 +680,11 @@ var errorCodeResponse = map[APIErrorCode]APIError{
|
|||||||
Description: "Storage backend has reached its minimum free disk threshold. Please delete a few objects to proceed.",
|
Description: "Storage backend has reached its minimum free disk threshold. Please delete a few objects to proceed.",
|
||||||
HTTPStatusCode: http.StatusInternalServerError,
|
HTTPStatusCode: http.StatusInternalServerError,
|
||||||
},
|
},
|
||||||
|
ErrRequestBodyParse: {
|
||||||
|
Code: "XMinioRequestBodyParse",
|
||||||
|
Description: "The request body failed to parse.",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
ErrObjectExistsAsDirectory: {
|
ErrObjectExistsAsDirectory: {
|
||||||
Code: "XMinioObjectExistsAsDirectory",
|
Code: "XMinioObjectExistsAsDirectory",
|
||||||
Description: "Object name already exists as a directory.",
|
Description: "Object name already exists as a directory.",
|
||||||
@ -708,6 +720,11 @@ var errorCodeResponse = map[APIErrorCode]APIError{
|
|||||||
Description: "Server not initialized, please try again.",
|
Description: "Server not initialized, please try again.",
|
||||||
HTTPStatusCode: http.StatusServiceUnavailable,
|
HTTPStatusCode: http.StatusServiceUnavailable,
|
||||||
},
|
},
|
||||||
|
ErrMalformedJSON: {
|
||||||
|
Code: "XMinioMalformedJSON",
|
||||||
|
Description: "The JSON you provided was not well-formed or did not validate against our published format.",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
ErrAdminInvalidAccessKey: {
|
ErrAdminInvalidAccessKey: {
|
||||||
Code: "XMinioAdminInvalidAccessKey",
|
Code: "XMinioAdminInvalidAccessKey",
|
||||||
Description: "The access key is invalid.",
|
Description: "The access key is invalid.",
|
||||||
@ -764,11 +781,6 @@ var errorCodeResponse = map[APIErrorCode]APIError{
|
|||||||
Description: errObjectTampered.Error(),
|
Description: errObjectTampered.Error(),
|
||||||
HTTPStatusCode: http.StatusPartialContent,
|
HTTPStatusCode: http.StatusPartialContent,
|
||||||
},
|
},
|
||||||
ErrHealNotImplemented: {
|
|
||||||
Code: "XMinioHealNotImplemented",
|
|
||||||
Description: "This server does not implement heal functionality.",
|
|
||||||
HTTPStatusCode: http.StatusBadRequest,
|
|
||||||
},
|
|
||||||
ErrMaximumExpires: {
|
ErrMaximumExpires: {
|
||||||
Code: "AuthorizationQueryParametersError",
|
Code: "AuthorizationQueryParametersError",
|
||||||
Description: "X-Amz-Expires must be less than a week (in seconds); that is, the given X-Amz-Expires must be less than 604800 seconds",
|
Description: "X-Amz-Expires must be less than a week (in seconds); that is, the given X-Amz-Expires must be less than 604800 seconds",
|
||||||
@ -782,6 +794,36 @@ var errorCodeResponse = map[APIErrorCode]APIError{
|
|||||||
Description: "Invalid Request",
|
Description: "Invalid Request",
|
||||||
HTTPStatusCode: http.StatusBadRequest,
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
},
|
},
|
||||||
|
ErrHealNotImplemented: {
|
||||||
|
Code: "XMinioHealNotImplemented",
|
||||||
|
Description: "This server does not implement heal functionality.",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
ErrHealNoSuchProcess: {
|
||||||
|
Code: "XMinioHealNoSuchProcess",
|
||||||
|
Description: "No such heal process is running on the server",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
ErrHealInvalidClientToken: {
|
||||||
|
Code: "XMinioHealInvalidClientToken",
|
||||||
|
Description: "Client token mismatch",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
ErrHealMissingBucket: {
|
||||||
|
Code: "XMinioHealMissingBucket",
|
||||||
|
Description: "A heal start request with a non-empty object-prefix parameter requires a bucket to be specified.",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
ErrHealAlreadyRunning: {
|
||||||
|
Code: "XMinioHealAlreadyRunning",
|
||||||
|
Description: "",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
ErrHealOverlappingPaths: {
|
||||||
|
Code: "XMinioHealOverlappingPaths",
|
||||||
|
Description: "",
|
||||||
|
HTTPStatusCode: http.StatusBadRequest,
|
||||||
|
},
|
||||||
|
|
||||||
// Add your error structure here.
|
// Add your error structure here.
|
||||||
}
|
}
|
||||||
|
@ -18,6 +18,7 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"encoding/json"
|
||||||
"encoding/xml"
|
"encoding/xml"
|
||||||
"fmt"
|
"fmt"
|
||||||
"net/http"
|
"net/http"
|
||||||
@ -53,6 +54,14 @@ func encodeResponse(response interface{}) []byte {
|
|||||||
return bytesBuffer.Bytes()
|
return bytesBuffer.Bytes()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Encodes the response headers into JSON format.
|
||||||
|
func encodeResponseJSON(response interface{}) []byte {
|
||||||
|
var bytesBuffer bytes.Buffer
|
||||||
|
e := json.NewEncoder(&bytesBuffer)
|
||||||
|
e.Encode(response)
|
||||||
|
return bytesBuffer.Bytes()
|
||||||
|
}
|
||||||
|
|
||||||
// Write object header
|
// Write object header
|
||||||
func setObjectHeaders(w http.ResponseWriter, objInfo ObjectInfo, contentRange *httpRange) {
|
func setObjectHeaders(w http.ResponseWriter, objInfo ObjectInfo, contentRange *httpRange) {
|
||||||
// set common headers
|
// set common headers
|
||||||
|
@ -166,13 +166,12 @@ type ListBucketsResponse struct {
|
|||||||
|
|
||||||
// Upload container for in progress multipart upload
|
// Upload container for in progress multipart upload
|
||||||
type Upload struct {
|
type Upload struct {
|
||||||
Key string
|
Key string
|
||||||
UploadID string `xml:"UploadId"`
|
UploadID string `xml:"UploadId"`
|
||||||
Initiator Initiator
|
Initiator Initiator
|
||||||
Owner Owner
|
Owner Owner
|
||||||
StorageClass string
|
StorageClass string
|
||||||
Initiated string
|
Initiated string
|
||||||
HealUploadInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// CommonPrefix container for prefix response in ListObjectsResponse
|
// CommonPrefix container for prefix response in ListObjectsResponse
|
||||||
@ -182,9 +181,8 @@ type CommonPrefix struct {
|
|||||||
|
|
||||||
// Bucket container for bucket metadata
|
// Bucket container for bucket metadata
|
||||||
type Bucket struct {
|
type Bucket struct {
|
||||||
Name string
|
Name string
|
||||||
CreationDate string // time string of format "2006-01-02T15:04:05.000Z"
|
CreationDate string // time string of format "2006-01-02T15:04:05.000Z"
|
||||||
HealBucketInfo *HealBucketInfo `xml:"HealBucketInfo,omitempty"`
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Object container for object metadata
|
// Object container for object metadata
|
||||||
@ -198,8 +196,7 @@ type Object struct {
|
|||||||
Owner Owner
|
Owner Owner
|
||||||
|
|
||||||
// The class of storage used to store the object.
|
// The class of storage used to store the object.
|
||||||
StorageClass string
|
StorageClass string
|
||||||
HealObjectInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// CopyObjectResponse container returns ETag and LastModified of the successfully copied object
|
// CopyObjectResponse container returns ETag and LastModified of the successfully copied object
|
||||||
@ -308,7 +305,6 @@ func generateListBucketsResponse(buckets []BucketInfo) ListBucketsResponse {
|
|||||||
var listbucket = Bucket{}
|
var listbucket = Bucket{}
|
||||||
listbucket.Name = bucket.Name
|
listbucket.Name = bucket.Name
|
||||||
listbucket.CreationDate = bucket.Created.UTC().Format(timeFormatAMZLong)
|
listbucket.CreationDate = bucket.Created.UTC().Format(timeFormatAMZLong)
|
||||||
listbucket.HealBucketInfo = bucket.HealBucketInfo
|
|
||||||
listbuckets = append(listbuckets, listbucket)
|
listbuckets = append(listbuckets, listbucket)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -339,8 +335,6 @@ func generateListObjectsV1Response(bucket, prefix, marker, delimiter string, max
|
|||||||
content.Size = object.Size
|
content.Size = object.Size
|
||||||
content.StorageClass = globalMinioDefaultStorageClass
|
content.StorageClass = globalMinioDefaultStorageClass
|
||||||
content.Owner = owner
|
content.Owner = owner
|
||||||
// object.HealObjectInfo is non-empty only when resp is constructed in ListObjectsHeal.
|
|
||||||
content.HealObjectInfo = object.HealObjectInfo
|
|
||||||
contents = append(contents, content)
|
contents = append(contents, content)
|
||||||
}
|
}
|
||||||
// TODO - support EncodingType in xml decoding
|
// TODO - support EncodingType in xml decoding
|
||||||
@ -498,7 +492,6 @@ func generateListMultipartUploadsResponse(bucket string, multipartsInfo ListMult
|
|||||||
newUpload.UploadID = upload.UploadID
|
newUpload.UploadID = upload.UploadID
|
||||||
newUpload.Key = upload.Object
|
newUpload.Key = upload.Object
|
||||||
newUpload.Initiated = upload.Initiated.UTC().Format(timeFormatAMZLong)
|
newUpload.Initiated = upload.Initiated.UTC().Format(timeFormatAMZLong)
|
||||||
newUpload.HealUploadInfo = upload.HealUploadInfo
|
|
||||||
listMultipartUploadsResponse.Uploads[index] = newUpload
|
listMultipartUploadsResponse.Uploads[index] = newUpload
|
||||||
}
|
}
|
||||||
return listMultipartUploadsResponse
|
return listMultipartUploadsResponse
|
||||||
@ -584,3 +577,31 @@ func writeErrorResponseHeadersOnly(w http.ResponseWriter, errorCode APIErrorCode
|
|||||||
apiError := getAPIError(errorCode)
|
apiError := getAPIError(errorCode)
|
||||||
writeResponse(w, apiError.HTTPStatusCode, nil, mimeNone)
|
writeResponse(w, apiError.HTTPStatusCode, nil, mimeNone)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// writeErrorResponseJSON - writes error response in JSON format;
|
||||||
|
// useful for admin APIs.
|
||||||
|
func writeErrorResponseJSON(w http.ResponseWriter, errorCode APIErrorCode, reqURL *url.URL) {
|
||||||
|
apiError := getAPIError(errorCode)
|
||||||
|
// Generate error response.
|
||||||
|
errorResponse := getAPIErrorResponse(apiError, reqURL.Path)
|
||||||
|
encodedErrorResponse := encodeResponseJSON(errorResponse)
|
||||||
|
writeResponse(w, apiError.HTTPStatusCode, encodedErrorResponse, mimeJSON)
|
||||||
|
}
|
||||||
|
|
||||||
|
// writeCustomErrorResponseJSON - similar to writeErrorResponseJSON,
|
||||||
|
// but accepts the error message directly (this allows messages to be
|
||||||
|
// dynamically generated.)
|
||||||
|
func writeCustomErrorResponseJSON(w http.ResponseWriter, errorCode APIErrorCode,
|
||||||
|
errBody string, reqURL *url.URL) {
|
||||||
|
|
||||||
|
apiError := getAPIError(errorCode)
|
||||||
|
errorResponse := APIErrorResponse{
|
||||||
|
Code: apiError.Code,
|
||||||
|
Message: errBody,
|
||||||
|
Resource: reqURL.Path,
|
||||||
|
RequestID: "3L137",
|
||||||
|
HostID: "3L137",
|
||||||
|
}
|
||||||
|
encodedErrorResponse := encodeResponseJSON(errorResponse)
|
||||||
|
writeResponse(w, apiError.HTTPStatusCode, encodedErrorResponse, mimeJSON)
|
||||||
|
}
|
||||||
|
@ -186,6 +186,15 @@ func (endpoints EndpointList) IsHTTPS() bool {
|
|||||||
return endpoints[0].IsHTTPS()
|
return endpoints[0].IsHTTPS()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetString - returns endpoint string of i-th endpoint (0-based),
|
||||||
|
// and empty string for invalid indexes.
|
||||||
|
func (endpoints EndpointList) GetString(i int) string {
|
||||||
|
if i < 0 || i >= len(endpoints) {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
return endpoints[i].String()
|
||||||
|
}
|
||||||
|
|
||||||
// NewEndpointList - returns new endpoint list based on input args.
|
// NewEndpointList - returns new endpoint list based on input args.
|
||||||
func NewEndpointList(args ...string) (endpoints EndpointList, err error) {
|
func NewEndpointList(args ...string) (endpoints EndpointList, err error) {
|
||||||
// isValidDistribution - checks whether given count is a valid distribution for erasure coding.
|
// isValidDistribution - checks whether given count is a valid distribution for erasure coding.
|
||||||
|
@ -143,14 +143,14 @@ func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatXLV1, []error) {
|
|||||||
// Initialize format configs.
|
// Initialize format configs.
|
||||||
var formats = make([]*formatXLV1, len(bootstrapDisks))
|
var formats = make([]*formatXLV1, len(bootstrapDisks))
|
||||||
|
|
||||||
// Make a volume entry on all underlying storage disks.
|
// Load format from each disk in parallel
|
||||||
for index, disk := range bootstrapDisks {
|
for index, disk := range bootstrapDisks {
|
||||||
if disk == nil {
|
if disk == nil {
|
||||||
sErrs[index] = errDiskNotFound
|
sErrs[index] = errDiskNotFound
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
// Make a volume inside a go-routine.
|
// Launch go-routine per disk.
|
||||||
go func(index int, disk StorageAPI) {
|
go func(index int, disk StorageAPI) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
format, lErr := loadFormat(disk)
|
format, lErr := loadFormat(disk)
|
||||||
@ -162,15 +162,9 @@ func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatXLV1, []error) {
|
|||||||
}(index, disk)
|
}(index, disk)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wait for all make vol to finish.
|
// Wait for all go-routines to finish.
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
for _, err := range sErrs {
|
|
||||||
if err != nil {
|
|
||||||
// Return all formats and errors.
|
|
||||||
return formats, sErrs
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Return all formats and nil
|
// Return all formats and nil
|
||||||
return formats, sErrs
|
return formats, sErrs
|
||||||
}
|
}
|
||||||
@ -374,8 +368,10 @@ func loadFormat(disk StorageAPI) (format *formatXLV1, err error) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
if len(vols) > 1 {
|
if len(vols) > 1 || (len(vols) == 1 &&
|
||||||
// 'format.json' not found, but we found user data.
|
vols[0].Name != minioMetaBucket) {
|
||||||
|
// 'format.json' not found, but we
|
||||||
|
// found user data.
|
||||||
return nil, errCorruptedFormat
|
return nil, errCorruptedFormat
|
||||||
}
|
}
|
||||||
// No other data found, its a fresh disk.
|
// No other data found, its a fresh disk.
|
||||||
@ -394,11 +390,10 @@ func loadFormat(disk StorageAPI) (format *formatXLV1, err error) {
|
|||||||
return format, nil
|
return format, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// collectNSaveNewFormatConfigs - creates new format configs based on
|
// collectNSaveNewFormatConfigs - generates new format configs based on
|
||||||
// the reference config and saves it on all disks, this is to be
|
// the given ref. config and saves on each disk
|
||||||
// called from healFormatXL* functions.
|
|
||||||
func collectNSaveNewFormatConfigs(referenceConfig *formatXLV1,
|
func collectNSaveNewFormatConfigs(referenceConfig *formatXLV1,
|
||||||
orderedDisks []StorageAPI) error {
|
orderedDisks []StorageAPI, dryRun bool) error {
|
||||||
|
|
||||||
// Collect new format configs that need to be written.
|
// Collect new format configs that need to be written.
|
||||||
var newFormatConfigs = make([]*formatXLV1, len(orderedDisks))
|
var newFormatConfigs = make([]*formatXLV1, len(orderedDisks))
|
||||||
@ -420,16 +415,19 @@ func collectNSaveNewFormatConfigs(referenceConfig *formatXLV1,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Save new `format.json` across all disks, in JBOD order.
|
// Save new `format.json` across all disks, in JBOD order.
|
||||||
return saveFormatXL(orderedDisks, newFormatConfigs)
|
if !dryRun {
|
||||||
|
return saveFormatXL(orderedDisks, newFormatConfigs)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heals any missing format.json on the drives. Returns error only for
|
// Heals any missing format.json on the drives. Returns error only for
|
||||||
// unexpected errors as regular errors can be ignored since there
|
// unexpected errors as regular errors can be ignored since there
|
||||||
// might be enough quorum to be operational. Heals only fresh disks.
|
// might be enough quorum to be operational. Heals only fresh disks.
|
||||||
func healFormatXLFreshDisks(storageDisks []StorageAPI,
|
func healFormatXLFreshDisks(storageDisks []StorageAPI,
|
||||||
formats []*formatXLV1) error {
|
formats []*formatXLV1, dryRun bool) error {
|
||||||
|
|
||||||
// Reorder the disks based on the JBOD order.
|
// Reorder disks based on JBOD order, and get reference config.
|
||||||
referenceConfig, orderedDisks, err := reorderDisks(storageDisks,
|
referenceConfig, orderedDisks, err := reorderDisks(storageDisks,
|
||||||
formats, true)
|
formats, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -441,21 +439,24 @@ func healFormatXLFreshDisks(storageDisks []StorageAPI,
|
|||||||
// and allowed fresh disks to be arranged anywhere.
|
// and allowed fresh disks to be arranged anywhere.
|
||||||
// Following block facilitates to put fresh disks.
|
// Following block facilitates to put fresh disks.
|
||||||
for index, format := range formats {
|
for index, format := range formats {
|
||||||
|
if format != nil {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
// Format is missing so we go through ordered disks.
|
// Format is missing so we go through ordered disks.
|
||||||
if format == nil {
|
// At this point when disk is missing the fresh disk
|
||||||
// At this point when disk is missing the fresh disk
|
// in the stack get it back from storageDisks.
|
||||||
// in the stack get it back from storageDisks.
|
for oIndex, disk := range orderedDisks {
|
||||||
for oIndex, disk := range orderedDisks {
|
if disk == nil {
|
||||||
if disk == nil {
|
orderedDisks[oIndex] = storageDisks[index]
|
||||||
orderedDisks[oIndex] = storageDisks[index]
|
break
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// apply new format config and save to all disks
|
// apply new format config and save to all disks
|
||||||
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks)
|
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks,
|
||||||
|
dryRun)
|
||||||
}
|
}
|
||||||
|
|
||||||
// collectUnAssignedDisks - collect disks unassigned to orderedDisks
|
// collectUnAssignedDisks - collect disks unassigned to orderedDisks
|
||||||
@ -540,9 +541,9 @@ func reorderDisksByInspection(orderedDisks, storageDisks []StorageAPI,
|
|||||||
|
|
||||||
// Heals corrupted format json in all disks
|
// Heals corrupted format json in all disks
|
||||||
func healFormatXLCorruptedDisks(storageDisks []StorageAPI,
|
func healFormatXLCorruptedDisks(storageDisks []StorageAPI,
|
||||||
formats []*formatXLV1) error {
|
formats []*formatXLV1, dryRun bool) error {
|
||||||
|
|
||||||
// Reorder the disks based on the JBOD order.
|
// Reorder disks based on JBOD order, and update ref. config.
|
||||||
referenceConfig, orderedDisks, err := reorderDisks(storageDisks,
|
referenceConfig, orderedDisks, err := reorderDisks(storageDisks,
|
||||||
formats, true)
|
formats, true)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -570,8 +571,9 @@ func healFormatXLCorruptedDisks(storageDisks []StorageAPI,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// apply new format config and save to all disks
|
// generate and write new configs to all disks
|
||||||
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks)
|
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks,
|
||||||
|
dryRun)
|
||||||
}
|
}
|
||||||
|
|
||||||
// loadFormatXL - loads XL `format.json` and returns back properly
|
// loadFormatXL - loads XL `format.json` and returns back properly
|
||||||
|
@ -278,7 +278,7 @@ func TestFormatXLHealFreshDisks(t *testing.T) {
|
|||||||
formatConfigs, _ := loadAllFormats(storageDisks)
|
formatConfigs, _ := loadAllFormats(storageDisks)
|
||||||
|
|
||||||
// Start healing disks
|
// Start healing disks
|
||||||
err = healFormatXLFreshDisks(storageDisks, formatConfigs)
|
err = healFormatXLFreshDisks(storageDisks, formatConfigs, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal("healing corrupted disk failed: ", err)
|
t.Fatal("healing corrupted disk failed: ", err)
|
||||||
}
|
}
|
||||||
@ -352,7 +352,7 @@ func TestFormatXLHealCorruptedDisks(t *testing.T) {
|
|||||||
formatConfigs, _ := loadAllFormats(permutedStorageDisks)
|
formatConfigs, _ := loadAllFormats(permutedStorageDisks)
|
||||||
|
|
||||||
// Start healing disks
|
// Start healing disks
|
||||||
err = healFormatXLCorruptedDisks(permutedStorageDisks, formatConfigs)
|
err = healFormatXLCorruptedDisks(permutedStorageDisks, formatConfigs, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal("healing corrupted disk failed: ", err)
|
t.Fatal("healing corrupted disk failed: ", err)
|
||||||
}
|
}
|
||||||
@ -761,7 +761,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
|
|||||||
|
|
||||||
xl := obj.(*xlObjects)
|
xl := obj.(*xlObjects)
|
||||||
formatConfigs, _ := loadAllFormats(xl.storageDisks)
|
formatConfigs, _ := loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs); err != nil {
|
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -784,7 +784,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
|
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
|
||||||
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs); err != errFaultyDisk {
|
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs, false); err != errFaultyDisk {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -806,7 +806,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs); err == nil {
|
if err = healFormatXLCorruptedDisks(xl.storageDisks, formatConfigs, false); err == nil {
|
||||||
t.Fatal("Should get a json parsing error, ")
|
t.Fatal("Should get a json parsing error, ")
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -833,7 +833,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl := obj.(*xlObjects)
|
xl := obj.(*xlObjects)
|
||||||
formatConfigs, _ := loadAllFormats(xl.storageDisks)
|
formatConfigs, _ := loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs); err != nil {
|
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -855,7 +855,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
|
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
|
||||||
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs); err != errFaultyDisk {
|
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs, false); err != errFaultyDisk {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -873,7 +873,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
|
|||||||
xl = obj.(*xlObjects)
|
xl = obj.(*xlObjects)
|
||||||
xl.storageDisks[0] = nil
|
xl.storageDisks[0] = nil
|
||||||
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
formatConfigs, _ = loadAllFormats(xl.storageDisks)
|
||||||
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs); err != nil {
|
if err = healFormatXLFreshDisks(xl.storageDisks, formatConfigs, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
|
13
cmd/fs-v1.go
13
cmd/fs-v1.go
@ -31,6 +31,7 @@ import (
|
|||||||
"github.com/minio/minio/pkg/errors"
|
"github.com/minio/minio/pkg/errors"
|
||||||
"github.com/minio/minio/pkg/hash"
|
"github.com/minio/minio/pkg/hash"
|
||||||
"github.com/minio/minio/pkg/lock"
|
"github.com/minio/minio/pkg/lock"
|
||||||
|
"github.com/minio/minio/pkg/madmin"
|
||||||
)
|
)
|
||||||
|
|
||||||
// fsObjects - Implements fs object layer.
|
// fsObjects - Implements fs object layer.
|
||||||
@ -1039,13 +1040,17 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
|
|||||||
}
|
}
|
||||||
|
|
||||||
// HealObject - no-op for fs. Valid only for XL.
|
// HealObject - no-op for fs. Valid only for XL.
|
||||||
func (fs fsObjects) HealObject(bucket, object string) (int, int, error) {
|
func (fs fsObjects) HealObject(bucket, object string, dryRun bool) (
|
||||||
return 0, 0, errors.Trace(NotImplemented{})
|
res madmin.HealResultItem, err error) {
|
||||||
|
|
||||||
|
return res, errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// HealBucket - no-op for fs, Valid only for XL.
|
// HealBucket - no-op for fs, Valid only for XL.
|
||||||
func (fs fsObjects) HealBucket(bucket string) error {
|
func (fs fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem,
|
||||||
return errors.Trace(NotImplemented{})
|
error) {
|
||||||
|
|
||||||
|
return nil, errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListObjectsHeal - list all objects to be healed. Valid only for XL
|
// ListObjectsHeal - list all objects to be healed. Valid only for XL
|
||||||
|
@ -400,7 +400,7 @@ func TestFSHealObject(t *testing.T) {
|
|||||||
defer os.RemoveAll(disk)
|
defer os.RemoveAll(disk)
|
||||||
|
|
||||||
obj := initFSObjects(disk, t)
|
obj := initFSObjects(disk, t)
|
||||||
_, _, err := obj.HealObject("bucket", "object")
|
_, err := obj.HealObject("bucket", "object", false)
|
||||||
if err == nil || !isSameType(errors.Cause(err), NotImplemented{}) {
|
if err == nil || !isSameType(errors.Cause(err), NotImplemented{}) {
|
||||||
t.Fatalf("Heal Object should return NotImplemented error ")
|
t.Fatalf("Heal Object should return NotImplemented error ")
|
||||||
}
|
}
|
||||||
|
@ -23,6 +23,7 @@ import (
|
|||||||
"github.com/minio/minio-go/pkg/policy"
|
"github.com/minio/minio-go/pkg/policy"
|
||||||
"github.com/minio/minio/pkg/errors"
|
"github.com/minio/minio/pkg/errors"
|
||||||
"github.com/minio/minio/pkg/hash"
|
"github.com/minio/minio/pkg/hash"
|
||||||
|
"github.com/minio/minio/pkg/madmin"
|
||||||
)
|
)
|
||||||
|
|
||||||
// GatewayUnsupported list of unsupported call stubs for gateway.
|
// GatewayUnsupported list of unsupported call stubs for gateway.
|
||||||
@ -38,8 +39,8 @@ func (a GatewayUnsupported) NewMultipartUpload(bucket string, object string, met
|
|||||||
return "", errors.Trace(NotImplemented{})
|
return "", errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// CopyObjectPart copy part of object to other bucket and object
|
// CopyObjectPart copy part of object to uploadID for another object
|
||||||
func (a GatewayUnsupported) CopyObjectPart(srcBucket string, srcObject string, destBucket string, destObject string, uploadID string, partID int, startOffset int64, length int64, metadata map[string]string, srcEtag string) (pi PartInfo, err error) {
|
func (a GatewayUnsupported) CopyObjectPart(srcBucket, srcObject, destBucket, destObject, uploadID string, partID int, startOffset, length int64, metadata map[string]string, srcETag string) (pi PartInfo, err error) {
|
||||||
return pi, errors.Trace(NotImplemented{})
|
return pi, errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -79,8 +80,8 @@ func (a GatewayUnsupported) DeleteBucketPolicies(bucket string) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// HealBucket - Not implemented stub
|
// HealBucket - Not implemented stub
|
||||||
func (a GatewayUnsupported) HealBucket(bucket string) error {
|
func (a GatewayUnsupported) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, error) {
|
||||||
return errors.Trace(NotImplemented{})
|
return nil, errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListBucketsHeal - Not implemented stub
|
// ListBucketsHeal - Not implemented stub
|
||||||
@ -89,8 +90,8 @@ func (a GatewayUnsupported) ListBucketsHeal() (buckets []BucketInfo, err error)
|
|||||||
}
|
}
|
||||||
|
|
||||||
// HealObject - Not implemented stub
|
// HealObject - Not implemented stub
|
||||||
func (a GatewayUnsupported) HealObject(bucket, object string) (int, int, error) {
|
func (a GatewayUnsupported) HealObject(bucket, object string, dryRun bool) (h madmin.HealResultItem, e error) {
|
||||||
return 0, 0, errors.Trace(NotImplemented{})
|
return h, errors.Trace(NotImplemented{})
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListObjectsV2 - Not implemented stub
|
// ListObjectsV2 - Not implemented stub
|
||||||
|
@ -246,6 +246,12 @@ func (h cacheControlHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
|||||||
h.handler.ServeHTTP(w, r)
|
h.handler.ServeHTTP(w, r)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Check to allow access to the reserved "bucket" `/minio` for Admin
|
||||||
|
// API requests.
|
||||||
|
func isAdminReq(r *http.Request) bool {
|
||||||
|
return strings.HasPrefix(r.URL.Path, adminAPIPathPrefix+"/")
|
||||||
|
}
|
||||||
|
|
||||||
// Adds verification for incoming paths.
|
// Adds verification for incoming paths.
|
||||||
type minioReservedBucketHandler struct {
|
type minioReservedBucketHandler struct {
|
||||||
handler http.Handler
|
handler http.Handler
|
||||||
@ -256,8 +262,12 @@ func setReservedBucketHandler(h http.Handler) http.Handler {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (h minioReservedBucketHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
func (h minioReservedBucketHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||||
if !guessIsRPCReq(r) && !guessIsBrowserReq(r) {
|
switch {
|
||||||
// For all non browser, non RPC requests, reject access to 'minioReservedBucketPath'.
|
case guessIsRPCReq(r), guessIsBrowserReq(r), isAdminReq(r):
|
||||||
|
// Allow access to reserved buckets
|
||||||
|
default:
|
||||||
|
// For all other requests reject access to reserved
|
||||||
|
// buckets
|
||||||
bucketName, _ := urlPath2BucketObjectName(r.URL)
|
bucketName, _ := urlPath2BucketObjectName(r.URL)
|
||||||
if isMinioReservedBucket(bucketName) || isMinioMetaBucket(bucketName) {
|
if isMinioReservedBucket(bucketName) || isMinioMetaBucket(bucketName) {
|
||||||
writeErrorResponse(w, ErrAllAccessDisabled, r.URL)
|
writeErrorResponse(w, ErrAllAccessDisabled, r.URL)
|
||||||
@ -433,6 +443,11 @@ func (h resourceHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// A put method on path "/" doesn't make sense, ignore it.
|
||||||
|
if r.Method == http.MethodPut && r.URL.Path == "/" {
|
||||||
|
writeErrorResponse(w, ErrNotImplemented, r.URL)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
// Serve HTTP.
|
// Serve HTTP.
|
||||||
h.handler.ServeHTTP(w, r)
|
h.handler.ServeHTTP(w, r)
|
||||||
|
@ -50,21 +50,6 @@ type StorageInfo struct {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type healStatus int
|
|
||||||
|
|
||||||
const (
|
|
||||||
healthy healStatus = iota // Object is healthy
|
|
||||||
canHeal // Object can be healed
|
|
||||||
corrupted // Object can't be healed
|
|
||||||
quorumUnavailable // Object can't be healed until read quorum is available
|
|
||||||
canPartiallyHeal // Object can't be healed completely until outdated disk(s) are online.
|
|
||||||
)
|
|
||||||
|
|
||||||
// HealBucketInfo - represents healing related information of a bucket.
|
|
||||||
type HealBucketInfo struct {
|
|
||||||
Status healStatus
|
|
||||||
}
|
|
||||||
|
|
||||||
// BucketInfo - represents bucket metadata.
|
// BucketInfo - represents bucket metadata.
|
||||||
type BucketInfo struct {
|
type BucketInfo struct {
|
||||||
// Name of the bucket.
|
// Name of the bucket.
|
||||||
@ -72,16 +57,6 @@ type BucketInfo struct {
|
|||||||
|
|
||||||
// Date and time when the bucket was created.
|
// Date and time when the bucket was created.
|
||||||
Created time.Time
|
Created time.Time
|
||||||
|
|
||||||
// Healing information
|
|
||||||
HealBucketInfo *HealBucketInfo `xml:"HealBucketInfo,omitempty"`
|
|
||||||
}
|
|
||||||
|
|
||||||
// HealObjectInfo - represents healing related information of an object.
|
|
||||||
type HealObjectInfo struct {
|
|
||||||
Status healStatus
|
|
||||||
MissingDataCount int
|
|
||||||
MissingParityCount int
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ObjectInfo - represents object metadata.
|
// ObjectInfo - represents object metadata.
|
||||||
@ -113,8 +88,7 @@ type ObjectInfo struct {
|
|||||||
ContentEncoding string
|
ContentEncoding string
|
||||||
|
|
||||||
// User-Defined metadata
|
// User-Defined metadata
|
||||||
UserDefined map[string]string
|
UserDefined map[string]string
|
||||||
HealObjectInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListPartsInfo - represents list of all parts.
|
// ListPartsInfo - represents list of all parts.
|
||||||
@ -273,8 +247,6 @@ type MultipartInfo struct {
|
|||||||
Initiated time.Time
|
Initiated time.Time
|
||||||
|
|
||||||
StorageClass string // Not supported yet.
|
StorageClass string // Not supported yet.
|
||||||
|
|
||||||
HealUploadInfo *HealObjectInfo `xml:"HealUploadInfo,omitempty"`
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// CompletePart - represents the part that was completed, this is sent by the client
|
// CompletePart - represents the part that was completed, this is sent by the client
|
||||||
|
@ -153,9 +153,6 @@ func checkPutObjectArgs(bucket, object string, obj ObjectLayer) error {
|
|||||||
|
|
||||||
// Checks whether bucket exists and returns appropriate error if not.
|
// Checks whether bucket exists and returns appropriate error if not.
|
||||||
func checkBucketExist(bucket string, obj ObjectLayer) error {
|
func checkBucketExist(bucket string, obj ObjectLayer) error {
|
||||||
if !IsValidBucketName(bucket) {
|
|
||||||
return BucketNameInvalid{Bucket: bucket}
|
|
||||||
}
|
|
||||||
_, err := obj.GetBucketInfo(bucket)
|
_, err := obj.GetBucketInfo(bucket)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Cause(err)
|
return errors.Cause(err)
|
||||||
|
@ -21,6 +21,7 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/minio/minio/pkg/hash"
|
"github.com/minio/minio/pkg/hash"
|
||||||
|
"github.com/minio/minio/pkg/madmin"
|
||||||
)
|
)
|
||||||
|
|
||||||
// ObjectLayer implements primitives for object API layer.
|
// ObjectLayer implements primitives for object API layer.
|
||||||
@ -53,9 +54,9 @@ type ObjectLayer interface {
|
|||||||
CompleteMultipartUpload(bucket, object, uploadID string, uploadedParts []CompletePart) (objInfo ObjectInfo, err error)
|
CompleteMultipartUpload(bucket, object, uploadID string, uploadedParts []CompletePart) (objInfo ObjectInfo, err error)
|
||||||
|
|
||||||
// Healing operations.
|
// Healing operations.
|
||||||
HealBucket(bucket string) error
|
HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, error)
|
||||||
|
HealObject(bucket, object string, dryRun bool) (madmin.HealResultItem, error)
|
||||||
ListBucketsHeal() (buckets []BucketInfo, err error)
|
ListBucketsHeal() (buckets []BucketInfo, err error)
|
||||||
HealObject(bucket, object string) (int, int, error)
|
|
||||||
ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (ListObjectsInfo, error)
|
ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (ListObjectsInfo, error)
|
||||||
|
|
||||||
// Locking operations
|
// Locking operations
|
||||||
|
@ -70,6 +70,9 @@ func configureServerHandler(endpoints EndpointList) (http.Handler, error) {
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Add Admin router.
|
||||||
|
registerAdminRouter(mux)
|
||||||
|
|
||||||
// Register web router when its enabled.
|
// Register web router when its enabled.
|
||||||
if globalIsBrowserEnabled {
|
if globalIsBrowserEnabled {
|
||||||
if err := registerWebRouter(mux); err != nil {
|
if err := registerWebRouter(mux); err != nil {
|
||||||
@ -77,9 +80,6 @@ func configureServerHandler(endpoints EndpointList) (http.Handler, error) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add Admin router.
|
|
||||||
registerAdminRouter(mux)
|
|
||||||
|
|
||||||
// Add API router.
|
// Add API router.
|
||||||
registerAPIRouter(mux)
|
registerAPIRouter(mux)
|
||||||
|
|
||||||
|
@ -193,6 +193,9 @@ func serverMain(ctx *cli.Context) {
|
|||||||
// Initialize name space lock.
|
// Initialize name space lock.
|
||||||
initNSLock(globalIsDistXL)
|
initNSLock(globalIsDistXL)
|
||||||
|
|
||||||
|
// Init global heal state
|
||||||
|
initAllHealState(globalIsXL)
|
||||||
|
|
||||||
// Configure server.
|
// Configure server.
|
||||||
var handler http.Handler
|
var handler http.Handler
|
||||||
handler, err = configureServerHandler(globalEndpoints)
|
handler, err = configureServerHandler(globalEndpoints)
|
||||||
|
@ -497,6 +497,17 @@ func resetGlobalStorageEnvs() {
|
|||||||
globalRRStorageClass = storageClass{}
|
globalRRStorageClass = storageClass{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// reset global heal state
|
||||||
|
func resetGlobalHealState() {
|
||||||
|
globalAllHealState.Lock()
|
||||||
|
defer globalAllHealState.Unlock()
|
||||||
|
for _, v := range globalAllHealState.healSeqMap {
|
||||||
|
if !v.hasEnded() {
|
||||||
|
v.stop()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Resets all the globals used modified in tests.
|
// Resets all the globals used modified in tests.
|
||||||
// Resetting ensures that the changes made to globals by one test doesn't affect others.
|
// Resetting ensures that the changes made to globals by one test doesn't affect others.
|
||||||
func resetTestGlobals() {
|
func resetTestGlobals() {
|
||||||
@ -518,6 +529,8 @@ func resetTestGlobals() {
|
|||||||
resetGlobalIsEnvs()
|
resetGlobalIsEnvs()
|
||||||
// Reset global storage class flags
|
// Reset global storage class flags
|
||||||
resetGlobalStorageEnvs()
|
resetGlobalStorageEnvs()
|
||||||
|
// Reset global heal state
|
||||||
|
resetGlobalHealState()
|
||||||
}
|
}
|
||||||
|
|
||||||
// Configure the server for the test run.
|
// Configure the server for the test run.
|
||||||
|
@ -37,11 +37,7 @@ func commonTime(modTimes []time.Time) (modTime time.Time, count int) {
|
|||||||
// Find the common cardinality from previously collected
|
// Find the common cardinality from previously collected
|
||||||
// occurrences of elements.
|
// occurrences of elements.
|
||||||
for time, count := range timeOccurenceMap {
|
for time, count := range timeOccurenceMap {
|
||||||
if count == maxima && time.After(modTime) {
|
if count > maxima || (count == maxima && time.After(modTime)) {
|
||||||
maxima = count
|
|
||||||
modTime = time
|
|
||||||
|
|
||||||
} else if count > maxima {
|
|
||||||
maxima = count
|
maxima = count
|
||||||
modTime = time
|
modTime = time
|
||||||
}
|
}
|
||||||
@ -126,7 +122,7 @@ func getLatestXLMeta(partsMetadata []xlMetaV1, errs []error) (xlMetaV1, int) {
|
|||||||
// List all the file commit ids from parts metadata.
|
// List all the file commit ids from parts metadata.
|
||||||
modTimes := listObjectModtimes(partsMetadata, errs)
|
modTimes := listObjectModtimes(partsMetadata, errs)
|
||||||
|
|
||||||
// Count all lastest updated xlMeta values
|
// Count all latest updated xlMeta values
|
||||||
var count int
|
var count int
|
||||||
var latestXLMeta xlMetaV1
|
var latestXLMeta xlMetaV1
|
||||||
|
|
||||||
@ -144,140 +140,13 @@ func getLatestXLMeta(partsMetadata []xlMetaV1, errs []error) (xlMetaV1, int) {
|
|||||||
return latestXLMeta, count
|
return latestXLMeta, count
|
||||||
}
|
}
|
||||||
|
|
||||||
// outDatedDisks - return disks which don't have the latest object (i.e xl.json).
|
|
||||||
// disks that are offline are not 'marked' outdated.
|
|
||||||
func outDatedDisks(disks, latestDisks []StorageAPI, errs []error, partsMetadata []xlMetaV1,
|
|
||||||
bucket, object string) (outDatedDisks []StorageAPI) {
|
|
||||||
|
|
||||||
outDatedDisks = make([]StorageAPI, len(disks))
|
|
||||||
for index, latestDisk := range latestDisks {
|
|
||||||
if latestDisk != nil {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// disk either has an older xl.json or doesn't have one.
|
|
||||||
switch errors.Cause(errs[index]) {
|
|
||||||
case nil, errFileNotFound:
|
|
||||||
outDatedDisks[index] = disks[index]
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return outDatedDisks
|
|
||||||
}
|
|
||||||
|
|
||||||
// Returns if the object should be healed.
|
|
||||||
func xlShouldHeal(disks []StorageAPI, partsMetadata []xlMetaV1, errs []error, bucket, object string) bool {
|
|
||||||
onlineDisks, _ := listOnlineDisks(disks, partsMetadata,
|
|
||||||
errs)
|
|
||||||
// Return true even if one of the disks have stale data.
|
|
||||||
for _, disk := range onlineDisks {
|
|
||||||
if disk == nil {
|
|
||||||
return true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check if all parts of an object are available and their
|
|
||||||
// checksums are valid.
|
|
||||||
availableDisks, _, err := disksWithAllParts(onlineDisks, partsMetadata,
|
|
||||||
errs, bucket, object)
|
|
||||||
if err != nil {
|
|
||||||
// Note: This error is due to failure of blake2b
|
|
||||||
// checksum computation of a part. It doesn't clearly
|
|
||||||
// indicate if the object needs healing. At this
|
|
||||||
// juncture healing could fail with the same
|
|
||||||
// error. So, we choose to return that there is no
|
|
||||||
// need to heal.
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
|
|
||||||
// Return true even if one disk has xl.json or one or more
|
|
||||||
// parts missing.
|
|
||||||
for _, disk := range availableDisks {
|
|
||||||
if disk == nil {
|
|
||||||
return true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
|
|
||||||
// xlHealStat - returns a structure which describes how many data,
|
|
||||||
// parity erasure blocks are missing and if it is possible to heal
|
|
||||||
// with the blocks present.
|
|
||||||
func xlHealStat(xl xlObjects, partsMetadata []xlMetaV1, errs []error) HealObjectInfo {
|
|
||||||
// Less than quorum erasure coded blocks of the object have the same create time.
|
|
||||||
// This object can't be healed with the information we have.
|
|
||||||
modTime, count := commonTime(listObjectModtimes(partsMetadata, errs))
|
|
||||||
|
|
||||||
// get read quorum for this object
|
|
||||||
readQuorum, _, err := objectQuorumFromMeta(xl, partsMetadata, errs)
|
|
||||||
|
|
||||||
if count < readQuorum || err != nil {
|
|
||||||
return HealObjectInfo{
|
|
||||||
Status: quorumUnavailable,
|
|
||||||
MissingDataCount: 0,
|
|
||||||
MissingParityCount: 0,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If there isn't a valid xlMeta then we can't heal the object.
|
|
||||||
xlMeta, err := pickValidXLMeta(partsMetadata, modTime)
|
|
||||||
if err != nil {
|
|
||||||
return HealObjectInfo{
|
|
||||||
Status: corrupted,
|
|
||||||
MissingDataCount: 0,
|
|
||||||
MissingParityCount: 0,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Compute heal statistics like bytes to be healed, missing
|
|
||||||
// data and missing parity count.
|
|
||||||
missingDataCount := 0
|
|
||||||
missingParityCount := 0
|
|
||||||
|
|
||||||
disksMissing := false
|
|
||||||
for i, err := range errs {
|
|
||||||
// xl.json is not found, which implies the erasure
|
|
||||||
// coded blocks are unavailable in the corresponding disk.
|
|
||||||
// First half of the disks are data and the rest are parity.
|
|
||||||
switch realErr := errors.Cause(err); realErr {
|
|
||||||
case errDiskNotFound:
|
|
||||||
disksMissing = true
|
|
||||||
fallthrough
|
|
||||||
case errFileNotFound:
|
|
||||||
if xlMeta.Erasure.Distribution[i]-1 < xlMeta.Erasure.DataBlocks {
|
|
||||||
missingDataCount++
|
|
||||||
} else {
|
|
||||||
missingParityCount++
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// The object may not be healed completely, since some of the
|
|
||||||
// disks needing healing are unavailable.
|
|
||||||
if disksMissing {
|
|
||||||
return HealObjectInfo{
|
|
||||||
Status: canPartiallyHeal,
|
|
||||||
MissingDataCount: missingDataCount,
|
|
||||||
MissingParityCount: missingParityCount,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// This object can be healed. We have enough object metadata
|
|
||||||
// to reconstruct missing erasure coded blocks.
|
|
||||||
return HealObjectInfo{
|
|
||||||
Status: canHeal,
|
|
||||||
MissingDataCount: missingDataCount,
|
|
||||||
MissingParityCount: missingParityCount,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// disksWithAllParts - This function needs to be called with
|
// disksWithAllParts - This function needs to be called with
|
||||||
// []StorageAPI returned by listOnlineDisks. Returns,
|
// []StorageAPI returned by listOnlineDisks. Returns,
|
||||||
//
|
//
|
||||||
// - disks which have all parts specified in the latest xl.json.
|
// - disks which have all parts specified in the latest xl.json.
|
||||||
//
|
//
|
||||||
// - errs updated to have errFileNotFound in place of disks that had
|
// - slice of errors about the state of data files on disk - can have
|
||||||
// missing or corrupted parts.
|
// a not-found error or a hash-mismatch error.
|
||||||
//
|
//
|
||||||
// - non-nil error if any of the disks failed unexpectedly (i.e. error
|
// - non-nil error if any of the disks failed unexpectedly (i.e. error
|
||||||
// other than file not found and not a checksum error).
|
// other than file not found and not a checksum error).
|
||||||
@ -286,11 +155,13 @@ func disksWithAllParts(onlineDisks []StorageAPI, partsMetadata []xlMetaV1, errs
|
|||||||
|
|
||||||
availableDisks := make([]StorageAPI, len(onlineDisks))
|
availableDisks := make([]StorageAPI, len(onlineDisks))
|
||||||
buffer := []byte{}
|
buffer := []byte{}
|
||||||
|
dataErrs := make([]error, len(onlineDisks))
|
||||||
|
|
||||||
for i, onlineDisk := range onlineDisks {
|
for i, onlineDisk := range onlineDisks {
|
||||||
if onlineDisk == OfflineDisk {
|
if onlineDisk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// disk has a valid xl.json but may not have all the
|
// disk has a valid xl.json but may not have all the
|
||||||
// parts. This is considered an outdated disk, since
|
// parts. This is considered an outdated disk, since
|
||||||
// it needs healing too.
|
// it needs healing too.
|
||||||
@ -302,22 +173,25 @@ func disksWithAllParts(onlineDisks []StorageAPI, partsMetadata []xlMetaV1, errs
|
|||||||
// verification happens even if a 0-length
|
// verification happens even if a 0-length
|
||||||
// buffer is passed
|
// buffer is passed
|
||||||
_, hErr := onlineDisk.ReadFile(bucket, partPath, 0, buffer, verifier)
|
_, hErr := onlineDisk.ReadFile(bucket, partPath, 0, buffer, verifier)
|
||||||
if hErr != nil {
|
|
||||||
_, isCorrupted := hErr.(hashMismatchError)
|
_, isCorrupt := hErr.(hashMismatchError)
|
||||||
if isCorrupted || hErr == errFileNotFound {
|
switch {
|
||||||
errs[i] = errFileNotFound
|
case isCorrupt:
|
||||||
availableDisks[i] = OfflineDisk
|
fallthrough
|
||||||
break
|
case hErr == errFileNotFound, hErr == errVolumeNotFound:
|
||||||
}
|
dataErrs[i] = hErr
|
||||||
|
break
|
||||||
|
case hErr != nil:
|
||||||
|
// abort on unhandled errors
|
||||||
return nil, nil, errors.Trace(hErr)
|
return nil, nil, errors.Trace(hErr)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if errs[i] == nil {
|
if dataErrs[i] == nil {
|
||||||
// All parts verified, mark it as all data available.
|
// All parts verified, mark it as all data available.
|
||||||
availableDisks[i] = onlineDisk
|
availableDisks[i] = onlineDisk
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return availableDisks, errs, nil
|
return availableDisks, dataErrs, nil
|
||||||
}
|
}
|
||||||
|
@ -107,20 +107,6 @@ func partsMetaFromModTimes(modTimes []time.Time, algorithm BitrotAlgorithm, chec
|
|||||||
return partsMetadata
|
return partsMetadata
|
||||||
}
|
}
|
||||||
|
|
||||||
// toPosix - fetches *posix object from StorageAPI.
|
|
||||||
func toPosix(disk StorageAPI) *posix {
|
|
||||||
retryDisk, ok := disk.(*retryStorage)
|
|
||||||
if !ok {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
pDisk, ok := retryDisk.remoteStorage.(*posix)
|
|
||||||
if !ok {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
return pDisk
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
// TestListOnlineDisks - checks if listOnlineDisks and outDatedDisks
|
// TestListOnlineDisks - checks if listOnlineDisks and outDatedDisks
|
||||||
// are consistent with each other.
|
// are consistent with each other.
|
||||||
func TestListOnlineDisks(t *testing.T) {
|
func TestListOnlineDisks(t *testing.T) {
|
||||||
@ -273,62 +259,22 @@ func TestListOnlineDisks(t *testing.T) {
|
|||||||
partsMetadata := partsMetaFromModTimes(test.modTimes, DefaultBitrotAlgorithm, xlMeta.Erasure.Checksums)
|
partsMetadata := partsMetaFromModTimes(test.modTimes, DefaultBitrotAlgorithm, xlMeta.Erasure.Checksums)
|
||||||
|
|
||||||
onlineDisks, modTime := listOnlineDisks(xlDisks, partsMetadata, test.errs)
|
onlineDisks, modTime := listOnlineDisks(xlDisks, partsMetadata, test.errs)
|
||||||
availableDisks, newErrs, _ := disksWithAllParts(onlineDisks, partsMetadata, test.errs, bucket, object)
|
|
||||||
test.errs = newErrs
|
|
||||||
outdatedDisks := outDatedDisks(xlDisks, availableDisks, test.errs, partsMetadata, bucket, object)
|
|
||||||
if modTime.Equal(timeSentinel) {
|
|
||||||
t.Fatalf("Test %d: modTime should never be equal to timeSentinel, but found equal",
|
|
||||||
i+1)
|
|
||||||
}
|
|
||||||
|
|
||||||
if test._tamperBackend != noTamper {
|
|
||||||
if tamperedIndex != -1 && outdatedDisks[tamperedIndex] == nil {
|
|
||||||
t.Fatalf("Test %d: disk (%v) with part.1 missing is an outdated disk, but wasn't listed by outDatedDisks",
|
|
||||||
i+1, xlDisks[tamperedIndex])
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
if !modTime.Equal(test.expectedTime) {
|
if !modTime.Equal(test.expectedTime) {
|
||||||
t.Fatalf("Test %d: Expected modTime to be equal to %v but was found to be %v",
|
t.Fatalf("Test %d: Expected modTime to be equal to %v but was found to be %v",
|
||||||
i+1, test.expectedTime, modTime)
|
i+1, test.expectedTime, modTime)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if a disk is considered both online and outdated,
|
availableDisks, newErrs, _ := disksWithAllParts(onlineDisks, partsMetadata, test.errs, bucket, object)
|
||||||
// which is a contradiction, except if parts are missing.
|
test.errs = newErrs
|
||||||
overlappingDisks := make(map[string]*posix)
|
|
||||||
for _, availableDisk := range availableDisks {
|
if test._tamperBackend != noTamper {
|
||||||
if availableDisk == nil {
|
if tamperedIndex != -1 && availableDisks[tamperedIndex] != nil {
|
||||||
continue
|
t.Fatalf("Test %d: disk (%v) with part.1 missing is not a disk with available data",
|
||||||
|
i+1, xlDisks[tamperedIndex])
|
||||||
}
|
}
|
||||||
pDisk := toPosix(availableDisk)
|
|
||||||
overlappingDisks[pDisk.diskPath] = pDisk
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for index, outdatedDisk := range outdatedDisks {
|
|
||||||
// ignore the intentionally tampered disk,
|
|
||||||
// this is expected to appear as outdated
|
|
||||||
// disk, since it doesn't have all the parts.
|
|
||||||
if index == tamperedIndex {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
if outdatedDisk == nil {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
pDisk := toPosix(outdatedDisk)
|
|
||||||
if _, ok := overlappingDisks[pDisk.diskPath]; ok {
|
|
||||||
t.Errorf("Test %d: Outdated disk %v was also detected as an online disk - %v %v",
|
|
||||||
i+1, pDisk, availableDisks, outdatedDisks)
|
|
||||||
}
|
|
||||||
|
|
||||||
// errors other than errFileNotFound doesn't imply that the disk is outdated.
|
|
||||||
if test.errs[index] != nil && test.errs[index] != errFileNotFound && outdatedDisk != nil {
|
|
||||||
t.Errorf("Test %d: error (%v) other than errFileNotFound doesn't imply that the disk (%v) could be outdated",
|
|
||||||
i+1, test.errs[index], pDisk)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -412,8 +358,8 @@ func TestDisksWithAllParts(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test that all disks are returned without any failures with unmodified
|
// Test that all disks are returned without any failures with
|
||||||
// meta data
|
// unmodified meta data
|
||||||
partsMetadata, errs = readAllXLMetadata(xlDisks, bucket, object)
|
partsMetadata, errs = readAllXLMetadata(xlDisks, bucket, object)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Failed to read xl meta data %v", err)
|
t.Fatalf("Failed to read xl meta data %v", err)
|
||||||
|
@ -23,11 +23,14 @@ import (
|
|||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/minio/minio/pkg/errors"
|
"github.com/minio/minio/pkg/errors"
|
||||||
|
"github.com/minio/minio/pkg/madmin"
|
||||||
)
|
)
|
||||||
|
|
||||||
// healFormatXL - heals missing `format.json` on freshly or corrupted
|
// healFormatXL - heals missing `format.json` on freshly or corrupted
|
||||||
// disks (missing format.json but does have erasure coded data in it).
|
// disks (missing format.json but does have erasure coded data in it).
|
||||||
func healFormatXL(storageDisks []StorageAPI) (err error) {
|
func healFormatXL(storageDisks []StorageAPI, dryRun bool) (res madmin.HealResultItem,
|
||||||
|
err error) {
|
||||||
|
|
||||||
// Attempt to load all `format.json`.
|
// Attempt to load all `format.json`.
|
||||||
formatConfigs, sErrs := loadAllFormats(storageDisks)
|
formatConfigs, sErrs := loadAllFormats(storageDisks)
|
||||||
|
|
||||||
@ -35,7 +38,34 @@ func healFormatXL(storageDisks []StorageAPI) (err error) {
|
|||||||
// - if (no quorum) return error
|
// - if (no quorum) return error
|
||||||
// - if (disks not recognized) // Always error.
|
// - if (disks not recognized) // Always error.
|
||||||
if err = genericFormatCheckXL(formatConfigs, sErrs); err != nil {
|
if err = genericFormatCheckXL(formatConfigs, sErrs); err != nil {
|
||||||
return err
|
return res, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Prepare heal-result
|
||||||
|
res = madmin.HealResultItem{
|
||||||
|
Type: madmin.HealItemMetadata,
|
||||||
|
Detail: "disk-format",
|
||||||
|
DiskCount: len(storageDisks),
|
||||||
|
}
|
||||||
|
res.InitDrives()
|
||||||
|
// Existing formats are available (i.e. ok), so save it in
|
||||||
|
// result, also populate disks to be healed.
|
||||||
|
for i, format := range formatConfigs {
|
||||||
|
drive := globalEndpoints.GetString(i)
|
||||||
|
switch {
|
||||||
|
case format != nil:
|
||||||
|
res.DriveInfo.Before[drive] = madmin.DriveStateOk
|
||||||
|
case sErrs[i] == errCorruptedFormat:
|
||||||
|
res.DriveInfo.Before[drive] = madmin.DriveStateCorrupt
|
||||||
|
case sErrs[i] == errUnformattedDisk:
|
||||||
|
res.DriveInfo.Before[drive] = madmin.DriveStateMissing
|
||||||
|
default:
|
||||||
|
res.DriveInfo.Before[drive] = madmin.DriveStateOffline
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Copy "after" drive state too
|
||||||
|
for k, v := range res.DriveInfo.Before {
|
||||||
|
res.DriveInfo.After[k] = v
|
||||||
}
|
}
|
||||||
|
|
||||||
numDisks := len(storageDisks)
|
numDisks := len(storageDisks)
|
||||||
@ -45,61 +75,95 @@ func healFormatXL(storageDisks []StorageAPI) (err error) {
|
|||||||
switch {
|
switch {
|
||||||
case unformattedDiskCount == numDisks:
|
case unformattedDiskCount == numDisks:
|
||||||
// all unformatted.
|
// all unformatted.
|
||||||
if err = initFormatXL(storageDisks); err != nil {
|
if !dryRun {
|
||||||
return err
|
err = initFormatXL(storageDisks)
|
||||||
|
if err != nil {
|
||||||
|
return res, err
|
||||||
|
}
|
||||||
|
for i := 0; i < len(storageDisks); i++ {
|
||||||
|
drive := globalEndpoints.GetString(i)
|
||||||
|
res.DriveInfo.After[drive] = madmin.DriveStateOk
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
return res, nil
|
||||||
|
|
||||||
case diskNotFoundCount > 0:
|
case diskNotFoundCount > 0:
|
||||||
return fmt.Errorf("cannot proceed with heal as %s",
|
return res, fmt.Errorf("cannot proceed with heal as %s",
|
||||||
errSomeDiskOffline)
|
errSomeDiskOffline)
|
||||||
|
|
||||||
case otherErrCount > 0:
|
case otherErrCount > 0:
|
||||||
return fmt.Errorf("cannot proceed with heal as some disks had unhandled errors")
|
return res, fmt.Errorf("cannot proceed with heal as some disks had unhandled errors")
|
||||||
|
|
||||||
case corruptedFormatCount > 0:
|
case corruptedFormatCount > 0:
|
||||||
if err = healFormatXLCorruptedDisks(storageDisks, formatConfigs); err != nil {
|
// heal corrupted disks
|
||||||
return fmt.Errorf("Unable to repair corrupted format, %s", err)
|
err = healFormatXLCorruptedDisks(storageDisks, formatConfigs,
|
||||||
|
dryRun)
|
||||||
|
if err != nil {
|
||||||
|
return res, err
|
||||||
}
|
}
|
||||||
|
// success
|
||||||
|
if !dryRun {
|
||||||
|
for i := 0; i < len(storageDisks); i++ {
|
||||||
|
drive := globalEndpoints.GetString(i)
|
||||||
|
res.DriveInfo.After[drive] = madmin.DriveStateOk
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return res, nil
|
||||||
|
|
||||||
case unformattedDiskCount > 0:
|
case unformattedDiskCount > 0:
|
||||||
// All drives online but some report missing format.json.
|
// heal unformatted disks
|
||||||
if err = healFormatXLFreshDisks(storageDisks, formatConfigs); err != nil {
|
err = healFormatXLFreshDisks(storageDisks, formatConfigs,
|
||||||
// There was an unexpected unrecoverable error
|
dryRun)
|
||||||
// during healing.
|
if err != nil {
|
||||||
return fmt.Errorf("Unable to heal backend %s", err)
|
return res, err
|
||||||
}
|
}
|
||||||
|
// success
|
||||||
|
if !dryRun {
|
||||||
|
for i := 0; i < len(storageDisks); i++ {
|
||||||
|
drive := globalEndpoints.GetString(i)
|
||||||
|
res.DriveInfo.After[drive] = madmin.DriveStateOk
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return res, nil
|
||||||
}
|
}
|
||||||
return nil
|
|
||||||
|
return res, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heals a bucket if it doesn't exist on one of the disks, additionally
|
// Heals a bucket if it doesn't exist on one of the disks, additionally
|
||||||
// also heals the missing entries for bucket metadata files
|
// also heals the missing entries for bucket metadata files
|
||||||
// `policy.json, notification.xml, listeners.json`.
|
// `policy.json, notification.xml, listeners.json`.
|
||||||
func (xl xlObjects) HealBucket(bucket string) error {
|
func (xl xlObjects) HealBucket(bucket string, dryRun bool) (
|
||||||
if err := checkBucketExist(bucket, xl); err != nil {
|
results []madmin.HealResultItem, err error) {
|
||||||
return err
|
|
||||||
|
if err = checkBucketExist(bucket, xl); err != nil {
|
||||||
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// get write quorum for an object
|
// get write quorum for an object
|
||||||
writeQuorum := len(xl.storageDisks)/2 + 1
|
writeQuorum := len(xl.storageDisks)/2 + 1
|
||||||
bucketLock := xl.nsMutex.NewNSLock(bucket, "")
|
bucketLock := xl.nsMutex.NewNSLock(bucket, "")
|
||||||
if err := bucketLock.GetLock(globalHealingTimeout); err != nil {
|
if err = bucketLock.GetLock(globalHealingTimeout); err != nil {
|
||||||
return err
|
return nil, err
|
||||||
}
|
}
|
||||||
defer bucketLock.Unlock()
|
defer bucketLock.Unlock()
|
||||||
|
|
||||||
// Heal bucket.
|
// Heal bucket.
|
||||||
if err := healBucket(xl.storageDisks, bucket, writeQuorum); err != nil {
|
result, err := healBucket(xl.storageDisks, bucket, writeQuorum, dryRun)
|
||||||
return err
|
if err != nil {
|
||||||
|
return results, err
|
||||||
}
|
}
|
||||||
|
results = append(results, result)
|
||||||
|
|
||||||
// Proceed to heal bucket metadata.
|
// Proceed to heal bucket metadata.
|
||||||
return healBucketMetadata(xl, bucket)
|
metaResults, err := healBucketMetadata(xl, bucket, dryRun)
|
||||||
|
results = append(results, metaResults...)
|
||||||
|
return results, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heal bucket - create buckets on disks where it does not exist.
|
// Heal bucket - create buckets on disks where it does not exist.
|
||||||
func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int) error {
|
func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int,
|
||||||
|
dryRun bool) (res madmin.HealResultItem, err error) {
|
||||||
|
|
||||||
// Initialize sync waitgroup.
|
// Initialize sync waitgroup.
|
||||||
var wg = &sync.WaitGroup{}
|
var wg = &sync.WaitGroup{}
|
||||||
@ -107,24 +171,47 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int) error
|
|||||||
// Initialize list of errors.
|
// Initialize list of errors.
|
||||||
var dErrs = make([]error, len(storageDisks))
|
var dErrs = make([]error, len(storageDisks))
|
||||||
|
|
||||||
|
// Disk states slices
|
||||||
|
beforeState := make([]string, len(storageDisks))
|
||||||
|
afterState := make([]string, len(storageDisks))
|
||||||
|
|
||||||
// Make a volume entry on all underlying storage disks.
|
// Make a volume entry on all underlying storage disks.
|
||||||
for index, disk := range storageDisks {
|
for index, disk := range storageDisks {
|
||||||
if disk == nil {
|
if disk == nil {
|
||||||
dErrs[index] = errors.Trace(errDiskNotFound)
|
dErrs[index] = errors.Trace(errDiskNotFound)
|
||||||
|
beforeState[index] = madmin.DriveStateOffline
|
||||||
|
afterState[index] = madmin.DriveStateOffline
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
|
|
||||||
// Make a volume inside a go-routine.
|
// Make a volume inside a go-routine.
|
||||||
go func(index int, disk StorageAPI) {
|
go func(index int, disk StorageAPI) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
if _, err := disk.StatVol(bucket); err != nil {
|
if _, err := disk.StatVol(bucket); err != nil {
|
||||||
if err != errVolumeNotFound {
|
if errors.Cause(err) != errVolumeNotFound {
|
||||||
|
beforeState[index] = madmin.DriveStateCorrupt
|
||||||
|
afterState[index] = madmin.DriveStateCorrupt
|
||||||
dErrs[index] = errors.Trace(err)
|
dErrs[index] = errors.Trace(err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if err = disk.MakeVol(bucket); err != nil {
|
|
||||||
dErrs[index] = errors.Trace(err)
|
beforeState[index] = madmin.DriveStateMissing
|
||||||
|
afterState[index] = madmin.DriveStateMissing
|
||||||
|
|
||||||
|
// mutate only if not a dry-run
|
||||||
|
if dryRun {
|
||||||
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
makeErr := disk.MakeVol(bucket)
|
||||||
|
dErrs[index] = errors.Trace(makeErr)
|
||||||
|
if makeErr == nil {
|
||||||
|
afterState[index] = madmin.DriveStateOk
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
beforeState[index] = madmin.DriveStateOk
|
||||||
|
afterState[index] = madmin.DriveStateOk
|
||||||
}
|
}
|
||||||
}(index, disk)
|
}(index, disk)
|
||||||
}
|
}
|
||||||
@ -132,44 +219,75 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int) error
|
|||||||
// Wait for all make vol to finish.
|
// Wait for all make vol to finish.
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
|
// Initialize heal result info
|
||||||
|
res = madmin.HealResultItem{
|
||||||
|
Type: madmin.HealItemBucket,
|
||||||
|
Bucket: bucket,
|
||||||
|
DiskCount: len(storageDisks),
|
||||||
|
}
|
||||||
|
res.InitDrives()
|
||||||
|
for i, before := range beforeState {
|
||||||
|
drive := globalEndpoints.GetString(i)
|
||||||
|
res.DriveInfo.Before[drive] = before
|
||||||
|
res.DriveInfo.After[drive] = afterState[i]
|
||||||
|
}
|
||||||
|
|
||||||
reducedErr := reduceWriteQuorumErrs(dErrs, bucketOpIgnoredErrs, writeQuorum)
|
reducedErr := reduceWriteQuorumErrs(dErrs, bucketOpIgnoredErrs, writeQuorum)
|
||||||
if errors.Cause(reducedErr) == errXLWriteQuorum {
|
if errors.Cause(reducedErr) == errXLWriteQuorum {
|
||||||
// Purge successfully created buckets if we don't have writeQuorum.
|
// Purge successfully created buckets if we don't have writeQuorum.
|
||||||
undoMakeBucket(storageDisks, bucket)
|
undoMakeBucket(storageDisks, bucket)
|
||||||
}
|
}
|
||||||
return reducedErr
|
return res, reducedErr
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heals all the metadata associated for a given bucket, this function
|
// Heals all the metadata associated for a given bucket, this function
|
||||||
// heals `policy.json`, `notification.xml` and `listeners.json`.
|
// heals `policy.json`, `notification.xml` and `listeners.json`.
|
||||||
func healBucketMetadata(xlObj xlObjects, bucket string) error {
|
func healBucketMetadata(xl xlObjects, bucket string, dryRun bool) (
|
||||||
|
results []madmin.HealResultItem, err error) {
|
||||||
|
|
||||||
healBucketMetaFn := func(metaPath string) error {
|
healBucketMetaFn := func(metaPath string) error {
|
||||||
if _, _, err := xlObj.HealObject(minioMetaBucket, metaPath); err != nil && !isErrObjectNotFound(err) {
|
result, healErr := xl.HealObject(minioMetaBucket, metaPath, dryRun)
|
||||||
return err
|
// If object is not found, no result to add.
|
||||||
|
if isErrObjectNotFound(healErr) {
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
if healErr != nil {
|
||||||
|
return healErr
|
||||||
|
}
|
||||||
|
result.Type = madmin.HealItemBucketMetadata
|
||||||
|
results = append(results, result)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heal `policy.json` for missing entries, ignores if `policy.json` is not found.
|
// Heal `policy.json` for missing entries, ignores if
|
||||||
|
// `policy.json` is not found.
|
||||||
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
|
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
|
||||||
if err := healBucketMetaFn(policyPath); err != nil {
|
err = healBucketMetaFn(policyPath)
|
||||||
return err
|
if err != nil {
|
||||||
|
return results, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heal `notification.xml` for missing entries, ignores if `notification.xml` is not found.
|
// Heal `notification.xml` for missing entries, ignores if
|
||||||
nConfigPath := path.Join(bucketConfigPrefix, bucket, bucketNotificationConfig)
|
// `notification.xml` is not found.
|
||||||
if err := healBucketMetaFn(nConfigPath); err != nil {
|
nConfigPath := path.Join(bucketConfigPrefix, bucket,
|
||||||
return err
|
bucketNotificationConfig)
|
||||||
|
err = healBucketMetaFn(nConfigPath)
|
||||||
|
if err != nil {
|
||||||
|
return results, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heal `listeners.json` for missing entries, ignores if `listeners.json` is not found.
|
// Heal `listeners.json` for missing entries, ignores if
|
||||||
|
// `listeners.json` is not found.
|
||||||
lConfigPath := path.Join(bucketConfigPrefix, bucket, bucketListenerConfig)
|
lConfigPath := path.Join(bucketConfigPrefix, bucket, bucketListenerConfig)
|
||||||
return healBucketMetaFn(lConfigPath)
|
err = healBucketMetaFn(lConfigPath)
|
||||||
|
return results, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// listAllBuckets lists all buckets from all disks. It also
|
// listAllBuckets lists all buckets from all disks. It also
|
||||||
// returns the occurrence of each buckets in all disks
|
// returns the occurrence of each buckets in all disks
|
||||||
func listAllBuckets(storageDisks []StorageAPI) (buckets map[string]VolInfo, bucketsOcc map[string]int, err error) {
|
func listAllBuckets(storageDisks []StorageAPI) (buckets map[string]VolInfo,
|
||||||
|
bucketsOcc map[string]int, err error) {
|
||||||
|
|
||||||
buckets = make(map[string]VolInfo)
|
buckets = make(map[string]VolInfo)
|
||||||
bucketsOcc = make(map[string]int)
|
bucketsOcc = make(map[string]int)
|
||||||
for _, disk := range storageDisks {
|
for _, disk := range storageDisks {
|
||||||
@ -178,122 +296,42 @@ func listAllBuckets(storageDisks []StorageAPI) (buckets map[string]VolInfo, buck
|
|||||||
}
|
}
|
||||||
var volsInfo []VolInfo
|
var volsInfo []VolInfo
|
||||||
volsInfo, err = disk.ListVols()
|
volsInfo, err = disk.ListVols()
|
||||||
if err == nil {
|
if err != nil {
|
||||||
for _, volInfo := range volsInfo {
|
if errors.IsErrIgnored(err, bucketMetadataOpIgnoredErrs...) {
|
||||||
// StorageAPI can send volume names which are
|
continue
|
||||||
// incompatible with buckets, handle it and skip them.
|
|
||||||
if !IsValidBucketName(volInfo.Name) {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// Skip special volume buckets.
|
|
||||||
if isMinioMetaBucketName(volInfo.Name) {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// Increase counter per bucket name
|
|
||||||
bucketsOcc[volInfo.Name]++
|
|
||||||
// Save volume info under bucket name
|
|
||||||
buckets[volInfo.Name] = volInfo
|
|
||||||
}
|
}
|
||||||
continue
|
break
|
||||||
}
|
}
|
||||||
// Ignore any disks not found.
|
for _, volInfo := range volsInfo {
|
||||||
if errors.IsErrIgnored(err, bucketMetadataOpIgnoredErrs...) {
|
// StorageAPI can send volume names which are
|
||||||
continue
|
// incompatible with buckets - these are
|
||||||
|
// skipped, like the meta-bucket.
|
||||||
|
if !IsValidBucketName(volInfo.Name) ||
|
||||||
|
isMinioMetaBucketName(volInfo.Name) {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
// Increase counter per bucket name
|
||||||
|
bucketsOcc[volInfo.Name]++
|
||||||
|
// Save volume info under bucket name
|
||||||
|
buckets[volInfo.Name] = volInfo
|
||||||
}
|
}
|
||||||
break
|
|
||||||
}
|
}
|
||||||
return buckets, bucketsOcc, err
|
return buckets, bucketsOcc, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// reduceHealStatus - fetches the worst heal status in a provided slice
|
|
||||||
func reduceHealStatus(status []healStatus) healStatus {
|
|
||||||
worstStatus := healthy
|
|
||||||
for _, st := range status {
|
|
||||||
if st > worstStatus {
|
|
||||||
worstStatus = st
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return worstStatus
|
|
||||||
}
|
|
||||||
|
|
||||||
// bucketHealStatus - returns the heal status of the provided bucket. Internally,
|
|
||||||
// this function lists all object heal status of objects inside meta bucket config
|
|
||||||
// directory and returns the worst heal status that can be found
|
|
||||||
func (xl xlObjects) bucketHealStatus(bucketName string) (healStatus, error) {
|
|
||||||
// A list of all the bucket config files
|
|
||||||
configFiles := []string{bucketPolicyConfig, bucketNotificationConfig, bucketListenerConfig}
|
|
||||||
// The status of buckets config files
|
|
||||||
configsHealStatus := make([]healStatus, len(configFiles))
|
|
||||||
// The list of errors found during checking heal status of each config file
|
|
||||||
configsErrs := make([]error, len(configFiles))
|
|
||||||
// The path of meta bucket that contains all config files
|
|
||||||
configBucket := path.Join(minioMetaBucket, bucketConfigPrefix, bucketName)
|
|
||||||
|
|
||||||
// Check of config files heal status in go-routines
|
|
||||||
var wg sync.WaitGroup
|
|
||||||
// Loop over config files
|
|
||||||
for idx, configFile := range configFiles {
|
|
||||||
wg.Add(1)
|
|
||||||
// Compute heal status of current config file
|
|
||||||
go func(bucket, object string, index int) {
|
|
||||||
defer wg.Done()
|
|
||||||
// Check
|
|
||||||
listObjectsHeal, err := xl.listObjectsHeal(bucket, object, "", "", 1)
|
|
||||||
// If any error, save and immediately quit
|
|
||||||
if err != nil {
|
|
||||||
configsErrs[index] = err
|
|
||||||
return
|
|
||||||
}
|
|
||||||
// Check if current bucket contains any not healthy config file and save heal status
|
|
||||||
if len(listObjectsHeal.Objects) > 0 {
|
|
||||||
configsHealStatus[index] = listObjectsHeal.Objects[0].HealObjectInfo.Status
|
|
||||||
}
|
|
||||||
}(configBucket, configFile, idx)
|
|
||||||
}
|
|
||||||
wg.Wait()
|
|
||||||
|
|
||||||
// Return any found error
|
|
||||||
for _, err := range configsErrs {
|
|
||||||
if err != nil {
|
|
||||||
return healthy, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Reduce and return heal status
|
|
||||||
return reduceHealStatus(configsHealStatus), nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// ListBucketsHeal - Find all buckets that need to be healed
|
// ListBucketsHeal - Find all buckets that need to be healed
|
||||||
func (xl xlObjects) ListBucketsHeal() ([]BucketInfo, error) {
|
func (xl xlObjects) ListBucketsHeal() ([]BucketInfo, error) {
|
||||||
listBuckets := []BucketInfo{}
|
listBuckets := []BucketInfo{}
|
||||||
// List all buckets that can be found in all disks
|
// List all buckets that can be found in all disks
|
||||||
buckets, occ, err := listAllBuckets(xl.storageDisks)
|
buckets, _, err := listAllBuckets(xl.storageDisks)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return listBuckets, err
|
return listBuckets, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Iterate over all buckets
|
// Iterate over all buckets
|
||||||
for _, currBucket := range buckets {
|
for _, currBucket := range buckets {
|
||||||
// Check the status of bucket metadata
|
listBuckets = append(listBuckets,
|
||||||
bucketHealStatus, err := xl.bucketHealStatus(currBucket.Name)
|
BucketInfo{currBucket.Name, currBucket.Created})
|
||||||
if err != nil {
|
|
||||||
return []BucketInfo{}, err
|
|
||||||
}
|
|
||||||
// If all metadata are sane, check if the bucket directory is present in all disks
|
|
||||||
if bucketHealStatus == healthy && occ[currBucket.Name] != len(xl.storageDisks) {
|
|
||||||
// Current bucket is missing in some of the storage disks
|
|
||||||
bucketHealStatus = canHeal
|
|
||||||
}
|
|
||||||
// Add current bucket to the returned result if not healthy
|
|
||||||
if bucketHealStatus != healthy {
|
|
||||||
listBuckets = append(listBuckets,
|
|
||||||
BucketInfo{
|
|
||||||
Name: currBucket.Name,
|
|
||||||
Created: currBucket.Created,
|
|
||||||
HealBucketInfo: &HealBucketInfo{Status: bucketHealStatus},
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Sort found buckets
|
// Sort found buckets
|
||||||
@ -323,8 +361,8 @@ func quickHeal(xlObj xlObjects, writeQuorum int, readQuorum int) error {
|
|||||||
defer bucketLock.Unlock()
|
defer bucketLock.Unlock()
|
||||||
|
|
||||||
// Heal bucket and then proceed to heal bucket metadata if any.
|
// Heal bucket and then proceed to heal bucket metadata if any.
|
||||||
if err = healBucket(xlObj.storageDisks, bucketName, writeQuorum); err == nil {
|
if _, err = healBucket(xlObj.storageDisks, bucketName, writeQuorum, false); err == nil {
|
||||||
if err = healBucketMetadata(xlObj, bucketName); err == nil {
|
if _, err = healBucketMetadata(xlObj, bucketName, false); err == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
@ -337,77 +375,108 @@ func quickHeal(xlObj xlObjects, writeQuorum int, readQuorum int) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Heals an object only the corrupted/missing erasure blocks.
|
// Heals an object by re-writing corrupt/missing erasure blocks.
|
||||||
func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (int, int, error) {
|
func healObject(storageDisks []StorageAPI, bucket string, object string,
|
||||||
|
quorum int, dryRun bool) (result madmin.HealResultItem, err error) {
|
||||||
|
|
||||||
partsMetadata, errs := readAllXLMetadata(storageDisks, bucket, object)
|
partsMetadata, errs := readAllXLMetadata(storageDisks, bucket, object)
|
||||||
|
|
||||||
// readQuorum suffices for xl.json since we use monotonic
|
// readQuorum suffices for xl.json since we use monotonic
|
||||||
// system time to break the tie when a split-brain situation
|
// system time to break the tie when a split-brain situation
|
||||||
// arises.
|
// arises.
|
||||||
if rErr := reduceReadQuorumErrs(errs, nil, quorum); rErr != nil {
|
if reducedErr := reduceReadQuorumErrs(errs, nil, quorum); reducedErr != nil {
|
||||||
return 0, 0, toObjectErr(rErr, bucket, object)
|
return result, toObjectErr(reducedErr, bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
// List of disks having latest version of the object.
|
// List of disks having latest version of the object xl.json
|
||||||
|
// (by modtime).
|
||||||
latestDisks, modTime := listOnlineDisks(storageDisks, partsMetadata, errs)
|
latestDisks, modTime := listOnlineDisks(storageDisks, partsMetadata, errs)
|
||||||
|
|
||||||
// List of disks having all parts as per latest xl.json - this
|
// List of disks having all parts as per latest xl.json.
|
||||||
// does a full pass over the data and verifies all part files
|
availableDisks, dataErrs, aErr := disksWithAllParts(latestDisks, partsMetadata, errs, bucket, object)
|
||||||
// on disk
|
|
||||||
availableDisks, errs, aErr := disksWithAllParts(latestDisks, partsMetadata, errs, bucket,
|
|
||||||
object)
|
|
||||||
if aErr != nil {
|
if aErr != nil {
|
||||||
return 0, 0, toObjectErr(aErr, bucket, object)
|
return result, toObjectErr(aErr, bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Number of disks which don't serve data.
|
// Initialize heal result object
|
||||||
numOfflineDisks := 0
|
result = madmin.HealResultItem{
|
||||||
for index, disk := range storageDisks {
|
Type: madmin.HealItemObject,
|
||||||
if disk == nil || errs[index] == errDiskNotFound {
|
Bucket: bucket,
|
||||||
numOfflineDisks++
|
Object: object,
|
||||||
}
|
DiskCount: len(storageDisks),
|
||||||
}
|
|
||||||
|
|
||||||
// Number of disks which have all parts of the given object.
|
// Initialize object size to -1, so we can detect if we are
|
||||||
|
// unable to reliably find the object size.
|
||||||
|
ObjectSize: -1,
|
||||||
|
}
|
||||||
|
result.InitDrives()
|
||||||
|
|
||||||
|
// Loop to find number of disks with valid data, per-drive
|
||||||
|
// data state and a list of outdated disks on which data needs
|
||||||
|
// to be healed.
|
||||||
|
outDatedDisks := make([]StorageAPI, len(storageDisks))
|
||||||
numAvailableDisks := 0
|
numAvailableDisks := 0
|
||||||
for _, disk := range availableDisks {
|
disksToHealCount := 0
|
||||||
if disk != nil {
|
for i, v := range availableDisks {
|
||||||
|
driveState := ""
|
||||||
|
switch {
|
||||||
|
case v != nil:
|
||||||
|
driveState = madmin.DriveStateOk
|
||||||
numAvailableDisks++
|
numAvailableDisks++
|
||||||
|
// If data is sane on any one disk, we can
|
||||||
|
// extract the correct object size.
|
||||||
|
result.ObjectSize = partsMetadata[i].Stat.Size
|
||||||
|
result.ParityBlocks = partsMetadata[i].Erasure.ParityBlocks
|
||||||
|
result.DataBlocks = partsMetadata[i].Erasure.DataBlocks
|
||||||
|
case errors.Cause(errs[i]) == errDiskNotFound:
|
||||||
|
driveState = madmin.DriveStateOffline
|
||||||
|
case errors.Cause(errs[i]) == errFileNotFound, errors.Cause(errs[i]) == errVolumeNotFound:
|
||||||
|
fallthrough
|
||||||
|
case errors.Cause(dataErrs[i]) == errFileNotFound, errors.Cause(dataErrs[i]) == errVolumeNotFound:
|
||||||
|
driveState = madmin.DriveStateMissing
|
||||||
|
default:
|
||||||
|
// all remaining cases imply corrupt data/metadata
|
||||||
|
driveState = madmin.DriveStateCorrupt
|
||||||
}
|
}
|
||||||
}
|
drive := globalEndpoints.GetString(i)
|
||||||
|
result.DriveInfo.Before[drive] = driveState
|
||||||
|
// copy for 'after' state
|
||||||
|
result.DriveInfo.After[drive] = driveState
|
||||||
|
|
||||||
if numAvailableDisks == len(storageDisks) {
|
// an online disk without valid data/metadata is
|
||||||
// nothing to heal in this case
|
// outdated and can be healed.
|
||||||
return 0, 0, nil
|
if errs[i] != errDiskNotFound && v == nil {
|
||||||
|
outDatedDisks[i] = storageDisks[i]
|
||||||
|
disksToHealCount++
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// If less than read quorum number of disks have all the parts
|
// If less than read quorum number of disks have all the parts
|
||||||
// of the data, we can't reconstruct the erasure-coded data.
|
// of the data, we can't reconstruct the erasure-coded data.
|
||||||
if numAvailableDisks < quorum {
|
if numAvailableDisks < quorum {
|
||||||
return 0, 0, toObjectErr(errXLReadQuorum, bucket, object)
|
return result, toObjectErr(errXLReadQuorum, bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
// List of disks having outdated version of the object or missing object.
|
if disksToHealCount == 0 {
|
||||||
outDatedDisks := outDatedDisks(storageDisks, availableDisks, errs, partsMetadata, bucket,
|
// Nothing to heal!
|
||||||
object)
|
return result, nil
|
||||||
|
}
|
||||||
|
|
||||||
// Number of disks that had outdated content of the given
|
// After this point, only have to repair data on disk - so
|
||||||
// object and are online to be healed.
|
// return if it is a dry-run
|
||||||
numHealedDisks := 0
|
if dryRun {
|
||||||
for _, disk := range outDatedDisks {
|
return result, nil
|
||||||
if disk != nil {
|
|
||||||
numHealedDisks++
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Latest xlMetaV1 for reference. If a valid metadata is not
|
// Latest xlMetaV1 for reference. If a valid metadata is not
|
||||||
// present, it is as good as object not found.
|
// present, it is as good as object not found.
|
||||||
latestMeta, pErr := pickValidXLMeta(partsMetadata, modTime)
|
latestMeta, pErr := pickValidXLMeta(partsMetadata, modTime)
|
||||||
if pErr != nil {
|
if pErr != nil {
|
||||||
return 0, 0, toObjectErr(pErr, bucket, object)
|
return result, toObjectErr(pErr, bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
for index, disk := range outDatedDisks {
|
// Clear data files of the object on outdated disks
|
||||||
|
for _, disk := range outDatedDisks {
|
||||||
// Before healing outdated disks, we need to remove
|
// Before healing outdated disks, we need to remove
|
||||||
// xl.json and part files from "bucket/object/" so
|
// xl.json and part files from "bucket/object/" so
|
||||||
// that rename(minioMetaBucket, "tmp/tmpuuid/",
|
// that rename(minioMetaBucket, "tmp/tmpuuid/",
|
||||||
@ -417,18 +486,10 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// errFileNotFound implies that xl.json is missing. We
|
|
||||||
// may have object parts still present in the object
|
|
||||||
// directory. This needs to be deleted for object to
|
|
||||||
// healed successfully.
|
|
||||||
if errs[index] != nil && !errors.IsErr(errs[index], errFileNotFound) {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
// List and delete the object directory, ignoring
|
// List and delete the object directory, ignoring
|
||||||
// errors.
|
// errors.
|
||||||
files, err := disk.ListDir(bucket, object)
|
files, derr := disk.ListDir(bucket, object)
|
||||||
if err == nil {
|
if derr == nil {
|
||||||
for _, entry := range files {
|
for _, entry := range files {
|
||||||
_ = disk.DeleteFile(bucket,
|
_ = disk.DeleteFile(bucket,
|
||||||
pathJoin(object, entry))
|
pathJoin(object, entry))
|
||||||
@ -452,10 +513,10 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
// Heal each part. erasureHealFile() will write the healed
|
// Heal each part. erasureHealFile() will write the healed
|
||||||
// part to .minio/tmp/uuid/ which needs to be renamed later to
|
// part to .minio/tmp/uuid/ which needs to be renamed later to
|
||||||
// the final location.
|
// the final location.
|
||||||
storage, err := NewErasureStorage(latestDisks,
|
storage, err := NewErasureStorage(latestDisks, latestMeta.Erasure.DataBlocks,
|
||||||
latestMeta.Erasure.DataBlocks, latestMeta.Erasure.ParityBlocks, latestMeta.Erasure.BlockSize)
|
latestMeta.Erasure.ParityBlocks, latestMeta.Erasure.BlockSize)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, 0, toObjectErr(err, bucket, object)
|
return result, toObjectErr(err, bucket, object)
|
||||||
}
|
}
|
||||||
checksums := make([][]byte, len(latestDisks))
|
checksums := make([][]byte, len(latestDisks))
|
||||||
for partIndex := 0; partIndex < len(latestMeta.Parts); partIndex++ {
|
for partIndex := 0; partIndex < len(latestMeta.Parts); partIndex++ {
|
||||||
@ -475,7 +536,7 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
erasure.BlockSize, minioMetaTmpBucket, pathJoin(tmpID, partName), partSize,
|
erasure.BlockSize, minioMetaTmpBucket, pathJoin(tmpID, partName), partSize,
|
||||||
algorithm, checksums)
|
algorithm, checksums)
|
||||||
if hErr != nil {
|
if hErr != nil {
|
||||||
return 0, 0, toObjectErr(hErr, bucket, object)
|
return result, toObjectErr(hErr, bucket, object)
|
||||||
}
|
}
|
||||||
// outDatedDisks that had write errors should not be
|
// outDatedDisks that had write errors should not be
|
||||||
// written to for remaining parts, so we nil it out.
|
// written to for remaining parts, so we nil it out.
|
||||||
@ -487,7 +548,7 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
// a healed part checksum had a write error.
|
// a healed part checksum had a write error.
|
||||||
if file.Checksums[i] == nil {
|
if file.Checksums[i] == nil {
|
||||||
outDatedDisks[i] = nil
|
outDatedDisks[i] = nil
|
||||||
numHealedDisks--
|
disksToHealCount--
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
// append part checksums
|
// append part checksums
|
||||||
@ -496,8 +557,8 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
}
|
}
|
||||||
|
|
||||||
// If all disks are having errors, we give up.
|
// If all disks are having errors, we give up.
|
||||||
if numHealedDisks == 0 {
|
if disksToHealCount == 0 {
|
||||||
return 0, 0, fmt.Errorf("all disks without up-to-date data had write errors")
|
return result, fmt.Errorf("all disks without up-to-date data had write errors")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -514,11 +575,11 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
outDatedDisks, aErr = writeUniqueXLMetadata(outDatedDisks, minioMetaTmpBucket, tmpID,
|
outDatedDisks, aErr = writeUniqueXLMetadata(outDatedDisks, minioMetaTmpBucket, tmpID,
|
||||||
partsMetadata, diskCount(outDatedDisks))
|
partsMetadata, diskCount(outDatedDisks))
|
||||||
if aErr != nil {
|
if aErr != nil {
|
||||||
return 0, 0, toObjectErr(aErr, bucket, object)
|
return result, toObjectErr(aErr, bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Rename from tmp location to the actual location.
|
// Rename from tmp location to the actual location.
|
||||||
for _, disk := range outDatedDisks {
|
for diskIndex, disk := range outDatedDisks {
|
||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
@ -527,33 +588,47 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
|
|||||||
aErr = disk.RenameFile(minioMetaTmpBucket, retainSlash(tmpID), bucket,
|
aErr = disk.RenameFile(minioMetaTmpBucket, retainSlash(tmpID), bucket,
|
||||||
retainSlash(object))
|
retainSlash(object))
|
||||||
if aErr != nil {
|
if aErr != nil {
|
||||||
return 0, 0, toObjectErr(errors.Trace(aErr), bucket, object)
|
return result, toObjectErr(errors.Trace(aErr), bucket, object)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
realDiskIdx := unshuffleIndex(diskIndex,
|
||||||
|
latestMeta.Erasure.Distribution)
|
||||||
|
drive := globalEndpoints.GetString(realDiskIdx)
|
||||||
|
result.DriveInfo.After[drive] = madmin.DriveStateOk
|
||||||
}
|
}
|
||||||
return numOfflineDisks, numHealedDisks, nil
|
|
||||||
|
// Set the size of the object in the heal result
|
||||||
|
result.ObjectSize = latestMeta.Stat.Size
|
||||||
|
|
||||||
|
return result, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// HealObject heals a given object for all its missing entries.
|
// HealObject - heal the given object.
|
||||||
|
//
|
||||||
// FIXME: If an object object was deleted and one disk was down,
|
// FIXME: If an object object was deleted and one disk was down,
|
||||||
// and later the disk comes back up again, heal on the object
|
// and later the disk comes back up again, heal on the object
|
||||||
// should delete it.
|
// should delete it.
|
||||||
func (xl xlObjects) HealObject(bucket, object string) (int, int, error) {
|
func (xl xlObjects) HealObject(bucket, object string, dryRun bool) (
|
||||||
|
hr madmin.HealResultItem, err error) {
|
||||||
|
|
||||||
|
// FIXME: Metadata is read again in the healObject() call below.
|
||||||
// Read metadata files from all the disks
|
// Read metadata files from all the disks
|
||||||
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, bucket, object)
|
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, bucket, object)
|
||||||
|
|
||||||
// get read quorum for this object
|
// get read quorum for this object
|
||||||
readQuorum, _, err := objectQuorumFromMeta(xl, partsMetadata, errs)
|
var readQuorum int
|
||||||
|
readQuorum, _, err = objectQuorumFromMeta(xl, partsMetadata, errs)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, 0, err
|
return hr, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Lock the object before healing.
|
// Lock the object before healing.
|
||||||
objectLock := xl.nsMutex.NewNSLock(bucket, object)
|
objectLock := xl.nsMutex.NewNSLock(bucket, object)
|
||||||
if err := objectLock.GetRLock(globalHealingTimeout); err != nil {
|
if lerr := objectLock.GetRLock(globalHealingTimeout); lerr != nil {
|
||||||
return 0, 0, err
|
return hr, lerr
|
||||||
}
|
}
|
||||||
defer objectLock.RUnlock()
|
defer objectLock.RUnlock()
|
||||||
|
|
||||||
// Heal the object.
|
// Heal the object.
|
||||||
return healObject(xl.storageDisks, bucket, object, readQuorum)
|
return healObject(xl.storageDisks, bucket, object, readQuorum, dryRun)
|
||||||
}
|
}
|
||||||
|
@ -23,6 +23,7 @@ import (
|
|||||||
"path/filepath"
|
"path/filepath"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/minio/minio-go/pkg/set"
|
||||||
"github.com/minio/minio/pkg/errors"
|
"github.com/minio/minio/pkg/errors"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -46,7 +47,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
xl := obj.(*xlObjects)
|
xl := obj.(*xlObjects)
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -67,7 +68,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
xl.storageDisks[i] = nil
|
xl.storageDisks[i] = nil
|
||||||
}
|
}
|
||||||
|
|
||||||
if err = healFormatXL(xl.storageDisks); err != errXLReadQuorum {
|
if _, err = healFormatXL(xl.storageDisks, false); err != errXLReadQuorum {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -90,7 +91,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl.storageDisks[i] = newNaughtyDisk(posixDisk, nil, errDiskFull)
|
xl.storageDisks[i] = newNaughtyDisk(posixDisk, nil, errDiskFull)
|
||||||
}
|
}
|
||||||
if err = healFormatXL(xl.storageDisks); err != errXLReadQuorum {
|
if _, err = healFormatXL(xl.storageDisks, false); err != errXLReadQuorum {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -108,7 +109,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl = obj.(*xlObjects)
|
xl = obj.(*xlObjects)
|
||||||
xl.storageDisks[0] = nil
|
xl.storageDisks[0] = nil
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil && err.Error() != "cannot proceed with heal as some disks are offline" {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil && err.Error() != "cannot proceed with heal as some disks are offline" {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -129,7 +130,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -150,7 +151,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if err = healFormatXL(xl.storageDisks); err == nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err == nil {
|
||||||
t.Fatal("Should get a json parsing error, ")
|
t.Fatal("Should get a json parsing error, ")
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -171,7 +172,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -198,7 +199,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskNotFound)
|
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskNotFound)
|
||||||
expectedErr := fmt.Errorf("cannot proceed with heal as %s", errSomeDiskOffline)
|
expectedErr := fmt.Errorf("cannot proceed with heal as %s", errSomeDiskOffline)
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
if err.Error() != expectedErr.Error() {
|
if err.Error() != expectedErr.Error() {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
@ -228,7 +229,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
}
|
}
|
||||||
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskAccessDenied)
|
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskAccessDenied)
|
||||||
expectedErr = fmt.Errorf("cannot proceed with heal as some disks had unhandled errors")
|
expectedErr = fmt.Errorf("cannot proceed with heal as some disks had unhandled errors")
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
if err.Error() != expectedErr.Error() {
|
if err.Error() != expectedErr.Error() {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
@ -254,7 +255,7 @@ func TestHealFormatXL(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if err = healFormatXL(xl.storageDisks); err != nil {
|
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
|
||||||
t.Fatal("Got an unexpected error: ", err)
|
t.Fatal("Got an unexpected error: ", err)
|
||||||
}
|
}
|
||||||
removeRoots(fsDirs)
|
removeRoots(fsDirs)
|
||||||
@ -446,14 +447,18 @@ func TestListBucketsHeal(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bucketSet := set.CreateStringSet(saneBucket, corruptedBucketName)
|
||||||
|
|
||||||
// Check the number of buckets in list buckets heal result
|
// Check the number of buckets in list buckets heal result
|
||||||
if len(buckets) != 1 {
|
if len(buckets) != len(bucketSet) {
|
||||||
t.Fatalf("Length of missing buckets is incorrect, expected: 1, found: %d", len(buckets))
|
t.Fatalf("Length of missing buckets is incorrect, expected: 2, found: %d", len(buckets))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check the name of bucket in list buckets heal result
|
// Check each bucket name is in `bucketSet`v
|
||||||
if buckets[0].Name != corruptedBucketName {
|
for _, b := range buckets {
|
||||||
t.Fatalf("Name of missing bucket is incorrect, expected: %s, found: %s", corruptedBucketName, buckets[0].Name)
|
if !bucketSet.Contains(b.Name) {
|
||||||
|
t.Errorf("Bucket %v is missing from bucket set", b.Name)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -520,7 +525,7 @@ func TestHealObjectXL(t *testing.T) {
|
|||||||
t.Fatalf("Failed to delete a file - %v", err)
|
t.Fatalf("Failed to delete a file - %v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, _, err = obj.HealObject(bucket, object)
|
_, err = obj.HealObject(bucket, object, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Failed to heal object - %v", err)
|
t.Fatalf("Failed to heal object - %v", err)
|
||||||
}
|
}
|
||||||
@ -536,7 +541,7 @@ func TestHealObjectXL(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Try healing now, expect to receive errDiskNotFound.
|
// Try healing now, expect to receive errDiskNotFound.
|
||||||
_, _, err = obj.HealObject(bucket, object)
|
_, err = obj.HealObject(bucket, object, false)
|
||||||
// since majority of xl.jsons are not available, object quorum can't be read properly and error will be errXLReadQuorum
|
// since majority of xl.jsons are not available, object quorum can't be read properly and error will be errXLReadQuorum
|
||||||
if errors.Cause(err) != errXLReadQuorum {
|
if errors.Cause(err) != errXLReadQuorum {
|
||||||
t.Errorf("Expected %v but received %v", errDiskNotFound, err)
|
t.Errorf("Expected %v but received %v", errDiskNotFound, err)
|
||||||
|
@ -37,9 +37,6 @@ func listDirHealFactory(isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// Filter entries that have the prefix prefixEntry.
|
|
||||||
entries = filterMatchingPrefix(entries, prefixEntry)
|
|
||||||
|
|
||||||
// isLeaf() check has to happen here so that
|
// isLeaf() check has to happen here so that
|
||||||
// trailing "/" for objects can be removed.
|
// trailing "/" for objects can be removed.
|
||||||
for i, entry := range entries {
|
for i, entry := range entries {
|
||||||
@ -63,6 +60,9 @@ func listDirHealFactory(isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
|
|||||||
mergedEntries = append(mergedEntries, newEntries...)
|
mergedEntries = append(mergedEntries, newEntries...)
|
||||||
sort.Strings(mergedEntries)
|
sort.Strings(mergedEntries)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Filter entries that have the prefix prefixEntry.
|
||||||
|
mergedEntries = filterMatchingPrefix(mergedEntries, prefixEntry)
|
||||||
}
|
}
|
||||||
return mergedEntries, false, nil
|
return mergedEntries, false, nil
|
||||||
}
|
}
|
||||||
@ -141,23 +141,15 @@ func (xl xlObjects) listObjectsHeal(bucket, prefix, marker, delimiter string, ma
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if the current object needs healing
|
// Add each object seen to the result - objects are
|
||||||
objectLock := xl.nsMutex.NewNSLock(bucket, objInfo.Name)
|
// checked for healing later.
|
||||||
if err := objectLock.GetRLock(globalHealingTimeout); err != nil {
|
result.Objects = append(result.Objects, ObjectInfo{
|
||||||
return loi, err
|
Bucket: bucket,
|
||||||
}
|
Name: objInfo.Name,
|
||||||
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, bucket, objInfo.Name)
|
ModTime: objInfo.ModTime,
|
||||||
if xlShouldHeal(xl.storageDisks, partsMetadata, errs, bucket, objInfo.Name) {
|
Size: objInfo.Size,
|
||||||
healStat := xlHealStat(xl, partsMetadata, errs)
|
IsDir: false,
|
||||||
result.Objects = append(result.Objects, ObjectInfo{
|
})
|
||||||
Name: objInfo.Name,
|
|
||||||
ModTime: objInfo.ModTime,
|
|
||||||
Size: objInfo.Size,
|
|
||||||
IsDir: false,
|
|
||||||
HealObjectInfo: &healStat,
|
|
||||||
})
|
|
||||||
}
|
|
||||||
objectLock.RUnlock()
|
|
||||||
}
|
}
|
||||||
return result, nil
|
return result, nil
|
||||||
}
|
}
|
||||||
|
@ -54,14 +54,16 @@ func TestListObjectsHeal(t *testing.T) {
|
|||||||
|
|
||||||
// Put 5 objects under sane dir
|
// Put 5 objects under sane dir
|
||||||
for i := 0; i < 5; i++ {
|
for i := 0; i < 5; i++ {
|
||||||
_, err = xl.PutObject(bucketName, "sane/"+objName+strconv.Itoa(i), mustGetHashReader(t, bytes.NewReader([]byte("abcd")), int64(len("abcd")), "", ""), nil)
|
_, err = xl.PutObject(bucketName, "sane/"+objName+strconv.Itoa(i),
|
||||||
|
mustGetHashReader(t, bytes.NewReader([]byte("abcd")), int64(len("abcd")), "", ""), nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
|
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Put 500 objects under unsane/subdir dir
|
// Put 5 objects under unsane/subdir dir
|
||||||
for i := 0; i < 5; i++ {
|
for i := 0; i < 5; i++ {
|
||||||
_, err = xl.PutObject(bucketName, "unsane/subdir/"+objName+strconv.Itoa(i), mustGetHashReader(t, bytes.NewReader([]byte("abcd")), int64(len("abcd")), "", ""), nil)
|
_, err = xl.PutObject(bucketName, "unsane/subdir/"+objName+strconv.Itoa(i),
|
||||||
|
mustGetHashReader(t, bytes.NewReader([]byte("abcd")), int64(len("abcd")), "", ""), nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
|
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
|
||||||
}
|
}
|
||||||
@ -101,7 +103,7 @@ func TestListObjectsHeal(t *testing.T) {
|
|||||||
// Inexistent object
|
// Inexistent object
|
||||||
{bucketName, "inexistentObj", "", "", 1000, nil, 0},
|
{bucketName, "inexistentObj", "", "", 1000, nil, 0},
|
||||||
// Test ListObjectsHeal when all objects are sane
|
// Test ListObjectsHeal when all objects are sane
|
||||||
{bucketName, "", "", "", 1000, nil, 0},
|
{bucketName, "", "", "", 1000, nil, 10},
|
||||||
}
|
}
|
||||||
for i, testCase := range testCases {
|
for i, testCase := range testCases {
|
||||||
testFunc(testCase, i+1)
|
testFunc(testCase, i+1)
|
||||||
@ -119,12 +121,12 @@ func TestListObjectsHeal(t *testing.T) {
|
|||||||
|
|
||||||
testCases = []testData{
|
testCases = []testData{
|
||||||
// Test ListObjectsHeal when all objects under unsane/ need to be healed
|
// Test ListObjectsHeal when all objects under unsane/ need to be healed
|
||||||
{bucketName, "", "", "", 1000, nil, 5},
|
{bucketName, "", "", "", 1000, nil, 10},
|
||||||
// List objects heal under unsane/, should return all elements
|
// List objects heal under unsane/, should return all elements
|
||||||
{bucketName, "unsane/", "", "", 1000, nil, 5},
|
{bucketName, "unsane/", "", "", 1000, nil, 5},
|
||||||
// List healing objects under sane/, should return 0
|
// List healing objects under sane/
|
||||||
{bucketName, "sane/", "", "", 1000, nil, 0},
|
{bucketName, "sane/", "", "", 1000, nil, 5},
|
||||||
// Max Keys == 200
|
// Max Keys == 2
|
||||||
{bucketName, "unsane/", "", "", 2, nil, 2},
|
{bucketName, "unsane/", "", "", 2, nil, 2},
|
||||||
// Max key > 1000
|
// Max key > 1000
|
||||||
{bucketName, "unsane/", "", "", 5000, nil, 5},
|
{bucketName, "unsane/", "", "", 5000, nil, 5},
|
||||||
|
@ -316,7 +316,7 @@ func TestHealing(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, _, err = xl.HealObject(bucket, object)
|
_, err = xl.HealObject(bucket, object, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
@ -340,7 +340,7 @@ func TestHealing(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, _, err = xl.HealObject(bucket, object)
|
_, err = xl.HealObject(bucket, object, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
@ -362,7 +362,7 @@ func TestHealing(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
// This would create the bucket.
|
// This would create the bucket.
|
||||||
err = xl.HealBucket(bucket)
|
_, err = xl.HealBucket(bucket, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
@ -66,17 +66,18 @@ func reduceErrs(errs []error, ignoredErrs []error) (maxCount int, maxErr error)
|
|||||||
// Additionally a special error is provided to be returned in case
|
// Additionally a special error is provided to be returned in case
|
||||||
// quorum is not satisfied.
|
// quorum is not satisfied.
|
||||||
func reduceQuorumErrs(errs []error, ignoredErrs []error, quorum int, quorumErr error) (maxErr error) {
|
func reduceQuorumErrs(errs []error, ignoredErrs []error, quorum int, quorumErr error) (maxErr error) {
|
||||||
maxCount, maxErr := reduceErrs(errs, ignoredErrs)
|
var maxCount int
|
||||||
if maxErr == nil && maxCount >= quorum {
|
maxCount, maxErr = reduceErrs(errs, ignoredErrs)
|
||||||
|
switch {
|
||||||
|
case maxErr == nil && maxCount >= quorum:
|
||||||
// Success in quorum.
|
// Success in quorum.
|
||||||
return nil
|
case maxErr != nil && maxCount >= quorum:
|
||||||
}
|
|
||||||
if maxErr != nil && maxCount >= quorum {
|
|
||||||
// Errors in quorum.
|
// Errors in quorum.
|
||||||
return errors2.Trace(maxErr, errs...)
|
maxErr = errors2.Trace(maxErr, errs...)
|
||||||
|
default:
|
||||||
|
// No quorum satisfied.
|
||||||
|
maxErr = errors2.Trace(quorumErr, errs...)
|
||||||
}
|
}
|
||||||
// No quorum satisfied.
|
|
||||||
maxErr = errors2.Trace(quorumErr, errs...)
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -365,6 +366,17 @@ func shuffleDisks(disks []StorageAPI, distribution []int) (shuffledDisks []Stora
|
|||||||
return shuffledDisks
|
return shuffledDisks
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// unshuffleIndex - performs reverse of the shuffleDisks operations
|
||||||
|
// for a single 0-based index.
|
||||||
|
func unshuffleIndex(n int, distribution []int) int {
|
||||||
|
for i, v := range distribution {
|
||||||
|
if v-1 == n {
|
||||||
|
return i
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return -1
|
||||||
|
}
|
||||||
|
|
||||||
// evalDisks - returns a new slice of disks where nil is set if
|
// evalDisks - returns a new slice of disks where nil is set if
|
||||||
// the corresponding error in errs slice is not nil
|
// the corresponding error in errs slice is not nil
|
||||||
func evalDisks(disks []StorageAPI, errs []error) []StorageAPI {
|
func evalDisks(disks []StorageAPI, errs []error) []StorageAPI {
|
||||||
|
@ -36,13 +36,11 @@ func main() {
|
|||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
| Service operations|LockInfo operations|Healing operations|Config operations| Misc |
|
| Service operations | LockInfo operations | Healing operations | Config operations | Misc |
|
||||||
|:---|:---|:---|:---|:---|
|
|:------------------------------------|:----------------------------|:--------------------------------------|:--------------------------|:------------------------------------|
|
||||||
|[`ServiceStatus`](#ServiceStatus)| [`ListLocks`](#ListLocks)| [`ListObjectsHeal`](#ListObjectsHeal)|[`GetConfig`](#GetConfig)| [`SetCredentials`](#SetCredentials)|
|
| [`ServiceStatus`](#ServiceStatus) | [`ListLocks`](#ListLocks) | [`Heal`](#Heal) | [`GetConfig`](#GetConfig) | [`SetCredentials`](#SetCredentials) |
|
||||||
|[`ServiceRestart`](#ServiceRestart)| [`ClearLocks`](#ClearLocks)| [`ListBucketsHeal`](#ListBucketsHeal)|[`SetConfig`](#SetConfig)||
|
| [`ServiceSendAction`](#ServiceSendAction) | [`ClearLocks`](#ClearLocks) | | [`SetConfig`](#SetConfig) | |
|
||||||
| | |[`HealBucket`](#HealBucket) |||
|
|
||||||
| | |[`HealObject`](#HealObject)|||
|
|
||||||
| | |[`HealFormat`](#HealFormat)|||
|
|
||||||
|
|
||||||
## 1. Constructor
|
## 1. Constructor
|
||||||
<a name="Minio"></a>
|
<a name="Minio"></a>
|
||||||
@ -60,8 +58,25 @@ __Parameters__
|
|||||||
|`secretAccessKey` | _string_ |Secret key for the object storage endpoint. |
|
|`secretAccessKey` | _string_ |Secret key for the object storage endpoint. |
|
||||||
|`ssl` | _bool_ | Set this value to 'true' to enable secure (HTTPS) access. |
|
|`ssl` | _bool_ | Set this value to 'true' to enable secure (HTTPS) access. |
|
||||||
|
|
||||||
|
## 2. Admin API Version
|
||||||
|
|
||||||
## 2. Service operations
|
<a name="VersionInfo"></a>
|
||||||
|
### VersionInfo() (AdminAPIVersionInfo, error)
|
||||||
|
Fetch server's supported Administrative API version.
|
||||||
|
|
||||||
|
__Example__
|
||||||
|
|
||||||
|
``` go
|
||||||
|
|
||||||
|
info, err := madmClnt.VersionInfo()
|
||||||
|
if err != nil {
|
||||||
|
log.Fatalln(err)
|
||||||
|
}
|
||||||
|
log.Printf("%s\n", info.Version)
|
||||||
|
|
||||||
|
```
|
||||||
|
|
||||||
|
## 3. Service operations
|
||||||
|
|
||||||
<a name="ServiceStatus"></a>
|
<a name="ServiceStatus"></a>
|
||||||
### ServiceStatus() (ServiceStatusMetadata, error)
|
### ServiceStatus() (ServiceStatusMetadata, error)
|
||||||
@ -102,17 +117,19 @@ Fetch service status, replies disk space used, backend type and total disks offl
|
|||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
<a name="ServiceRestart"></a>
|
<a name="ServiceSendAction"></a>
|
||||||
### ServiceRestart() (error)
|
### ServiceSendAction(act ServiceActionValue) (error)
|
||||||
If successful restarts the running minio service, for distributed setup restarts all remote minio servers.
|
Sends a service action command to service - possible actions are restarting and stopping the server.
|
||||||
|
|
||||||
__Example__
|
__Example__
|
||||||
|
|
||||||
|
|
||||||
```go
|
```go
|
||||||
|
|
||||||
|
// to restart
|
||||||
st, err := madmClnt.ServiceRestart()
|
st, err := madmClnt.ServiceSendAction(ServiceActionValueRestart)
|
||||||
|
// or to stop
|
||||||
|
// st, err := madmClnt.ServiceSendAction(ServiceActionValueStop)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalln(err)
|
log.Fatalln(err)
|
||||||
}
|
}
|
||||||
@ -120,7 +137,7 @@ If successful restarts the running minio service, for distributed setup restarts
|
|||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
## 3. Info operations
|
## 4. Info operations
|
||||||
|
|
||||||
<a name="ServerInfo"></a>
|
<a name="ServerInfo"></a>
|
||||||
### ServerInfo() ([]ServerInfo, error)
|
### ServerInfo() ([]ServerInfo, error)
|
||||||
@ -143,7 +160,7 @@ Fetch all information for all cluster nodes, such as uptime, region, network sta
|
|||||||
```
|
```
|
||||||
|
|
||||||
|
|
||||||
## 4. Lock operations
|
## 5. Lock operations
|
||||||
|
|
||||||
<a name="ListLocks"></a>
|
<a name="ListLocks"></a>
|
||||||
### ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
### ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
||||||
@ -175,146 +192,95 @@ __Example__
|
|||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
## 5. Heal operations
|
## 6. Heal operations
|
||||||
|
|
||||||
<a name="ListObjectsHeal"></a>
|
<a name="Heal"></a>
|
||||||
### ListObjectsHeal(bucket, prefix string, recursive bool, doneCh <-chan struct{}) (<-chan ObjectInfo, error)
|
### Heal(bucket, prefix string, healOpts HealOpts, clientToken string, forceStart bool) (start HealStartSuccess, status HealTaskStatus, err error)
|
||||||
If successful returns information on the list of objects that need healing in ``bucket`` matching ``prefix``.
|
|
||||||
|
Start a heal sequence that scans data under given (possible empty)
|
||||||
|
`bucket` and `prefix`. The `recursive` bool turns on recursive
|
||||||
|
traversal under the given path. `dryRun` does not mutate on-disk data,
|
||||||
|
but performs data validation. `incomplete` enables healing of
|
||||||
|
multipart uploads that are in progress. `removeBadFiles` removes
|
||||||
|
unrecoverable files. `statisticsOnly` turns off detailed
|
||||||
|
heal-operations reporting in the status call.
|
||||||
|
|
||||||
|
Two heal sequences on overlapping paths may not be initiated.
|
||||||
|
|
||||||
|
The progress of a heal should be followed using the `HealStatus`
|
||||||
|
API. The server accumulates results of the heal traversal and waits
|
||||||
|
for the client to receive and acknowledge them using the status
|
||||||
|
API. When the statistics-only option is set, the server only maintains
|
||||||
|
aggregates statistics - in this case, no acknowledgement of results is
|
||||||
|
required.
|
||||||
|
|
||||||
__Example__
|
__Example__
|
||||||
|
|
||||||
``` go
|
``` go
|
||||||
// Create a done channel to control 'ListObjectsHeal' go routine.
|
|
||||||
doneCh := make(chan struct{})
|
|
||||||
|
|
||||||
// Indicate to our routine to exit cleanly upon return.
|
healPath, err := madmClnt.HealStart("", "", true, false, true, false, false)
|
||||||
defer close(doneCh)
|
|
||||||
|
|
||||||
// Set true if recursive listing is needed.
|
|
||||||
isRecursive := true
|
|
||||||
// List objects that need healing for a given bucket and
|
|
||||||
// prefix.
|
|
||||||
healObjectCh, err := madmClnt.ListObjectsHeal("mybucket", "myprefix", isRecursive, doneCh)
|
|
||||||
if err != nil {
|
|
||||||
fmt.Println(err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
for object := range healObjectsCh {
|
|
||||||
if object.Err != nil {
|
|
||||||
log.Fatalln(err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
if object.HealObjectInfo != nil {
|
|
||||||
switch healInfo := *object.HealObjectInfo; healInfo.Status {
|
|
||||||
case madmin.CanHeal:
|
|
||||||
fmt.Println(object.Key, " can be healed.")
|
|
||||||
case madmin.QuorumUnavailable:
|
|
||||||
fmt.Println(object.Key, " can't be healed until quorum is available.")
|
|
||||||
case madmin.Corrupted:
|
|
||||||
fmt.Println(object.Key, " can't be healed, not enough information.")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
fmt.Println("object: ", object)
|
|
||||||
}
|
|
||||||
```
|
|
||||||
|
|
||||||
<a name="ListBucketsHeal"></a>
|
|
||||||
### ListBucketsHeal() error
|
|
||||||
If successful returns information on the list of buckets that need healing.
|
|
||||||
|
|
||||||
__Example__
|
|
||||||
|
|
||||||
``` go
|
|
||||||
// List buckets that need healing
|
|
||||||
healBucketsList, err := madmClnt.ListBucketsHeal()
|
|
||||||
if err != nil {
|
|
||||||
fmt.Println(err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
for bucket := range healBucketsList {
|
|
||||||
if bucket.HealBucketInfo != nil {
|
|
||||||
switch healInfo := *object.HealBucketInfo; healInfo.Status {
|
|
||||||
case madmin.CanHeal:
|
|
||||||
fmt.Println(bucket.Key, " can be healed.")
|
|
||||||
case madmin.QuorumUnavailable:
|
|
||||||
fmt.Println(bucket.Key, " can't be healed until quorum is available.")
|
|
||||||
case madmin.Corrupted:
|
|
||||||
fmt.Println(bucket.Key, " can't be healed, not enough information.")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
fmt.Println("bucket: ", bucket)
|
|
||||||
}
|
|
||||||
```
|
|
||||||
|
|
||||||
<a name="HealBucket"></a>
|
|
||||||
### HealBucket(bucket string, isDryRun bool) error
|
|
||||||
If bucket is successfully healed returns nil, otherwise returns error indicating the reason for failure. If isDryRun is true, then the bucket is not healed, but heal bucket request is validated by the server. e.g, if the bucket exists, if bucket name is valid etc.
|
|
||||||
|
|
||||||
__Example__
|
|
||||||
|
|
||||||
``` go
|
|
||||||
isDryRun := false
|
|
||||||
err := madmClnt.HealBucket("mybucket", isDryRun)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalln(err)
|
log.Fatalln(err)
|
||||||
}
|
}
|
||||||
log.Println("successfully healed mybucket")
|
log.Printf("Heal sequence started at %s", healPath)
|
||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
<a name="HealObject"></a>
|
#### HealTaskStatus structure
|
||||||
### HealObject(bucket, object string, isDryRun bool) (HealResult, error)
|
|
||||||
If object is successfully healed returns nil, otherwise returns error indicating the reason for failure. If isDryRun is true, then the object is not healed, but heal object request is validated by the server. e.g, if the object exists, if object name is valid etc.
|
|
||||||
|
|
||||||
| Param | Type | Description |
|
| Param | Type | Description |
|
||||||
|---|---|---|
|
|----|--------|--------|
|
||||||
|`h.State` | _HealState_ | Represents the result of heal operation. It could be one of `HealNone`, `HealPartial` or `HealOK`. |
|
| s.Summary | _string_ | Short status of heal sequence |
|
||||||
|
| s.FailureDetail | _string_ | Error message in case of heal sequence failure |
|
||||||
|
| s.HealSettings | _HealOpts_ | Contains the booleans set in the `HealStart` call |
|
||||||
|
| s.Items | _[]HealResultItem_ | Heal records for actions performed by server |
|
||||||
|
| s.Statistics | _HealStatistics_ | Aggregate of heal records from beginning |
|
||||||
|
|
||||||
|
#### HealResultItem structure
|
||||||
|
|
||||||
| Value | Description |
|
| Param | Type | Description |
|
||||||
|---|---|
|
|------|-------|---------|
|
||||||
|`HealNone` | Object wasn't healed on any of the disks |
|
| ResultIndex | _int64_ | Index of the heal-result record |
|
||||||
|`HealPartial` | Object was healed on some of the disks needing heal |
|
| Type | _HealItemType_ | Represents kind of heal operation in the heal record |
|
||||||
| `HealOK` | Object was healed on all the disks needing heal |
|
| Bucket | _string_ | Bucket name |
|
||||||
|
| Object | _string_ | Object name |
|
||||||
|
| Detail | _string_ | Details about heal operation |
|
||||||
|
| DiskInfo.AvailableOn | _[]int_ | List of disks on which the healed entity is present and healthy |
|
||||||
|
| DiskInfo.HealedOn | _[]int_ | List of disks on which the healed entity was restored |
|
||||||
|
|
||||||
|
#### HealStatistics structure
|
||||||
|
|
||||||
|
Most parameters represent the aggregation of heal operations since the
|
||||||
|
start of the heal sequence.
|
||||||
|
|
||||||
|
| Param | Type | Description |
|
||||||
|
|-------|-----|----------|
|
||||||
|
| NumDisks | _int_ | Number of disks configured in the backend |
|
||||||
|
| NumBucketsScanned | _int64_ | Number of buckets scanned |
|
||||||
|
| BucketsMissingByDisk | _map[int]int64_ | Map of disk to number of buckets missing |
|
||||||
|
| BucketsAvailableByDisk | _map[int]int64_ | Map of disk to number of buckets available |
|
||||||
|
| BucketsHealedByDisk | _map[int]int64_ | Map of disk to number of buckets healed on |
|
||||||
|
| NumObjectsScanned | _int64_ | Number of objects scanned |
|
||||||
|
| NumUploadsScanned | _int64_ | Number of uploads scanned |
|
||||||
|
| ObjectsByAvailablePC | _map[int64]_ | Map of available part counts (after heal) to number of objects |
|
||||||
|
| ObjectsByHealedPC | _map[int64]_ | Map of healed part counts to number of objects |
|
||||||
|
| ObjectsMissingByDisk | _map[int64]_ | Map of disk number to number of objects with parts missing on that disk |
|
||||||
|
| ObjectsAvailableByDisk | _map[int64]_ | Map of disk number to number of objects available on that disk |
|
||||||
|
| ObjectsHealedByDisk | _map[int64]_ | Map of disk number to number of objects healed on that disk |
|
||||||
|
|
||||||
__Example__
|
__Example__
|
||||||
|
|
||||||
``` go
|
``` go
|
||||||
isDryRun = false
|
|
||||||
healResult, err := madmClnt.HealObject("mybucket", "myobject", isDryRun)
|
res, err := madmClnt.HealStatus("", "")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatalln(err)
|
log.Fatalln(err)
|
||||||
}
|
}
|
||||||
|
log.Printf("Heal sequence status data %#v", res)
|
||||||
log.Println("Heal-object result: ", healResult)
|
|
||||||
|
|
||||||
```
|
```
|
||||||
|
|
||||||
<a name="HealFormat"></a>
|
## 7. Config operations
|
||||||
### HealFormat(isDryRun bool) error
|
|
||||||
Heal storage format on available disks. This is used when disks were replaced or were found with missing format. This is supported only for erasure-coded backend.
|
|
||||||
|
|
||||||
__Example__
|
|
||||||
|
|
||||||
``` go
|
|
||||||
isDryRun := true
|
|
||||||
err := madmClnt.HealFormat(isDryRun)
|
|
||||||
if err != nil {
|
|
||||||
log.Fatalln(err)
|
|
||||||
}
|
|
||||||
|
|
||||||
isDryRun = false
|
|
||||||
err = madmClnt.HealFormat(isDryRun)
|
|
||||||
if err != nil {
|
|
||||||
log.Fatalln(err)
|
|
||||||
}
|
|
||||||
|
|
||||||
log.Println("successfully healed storage format on available disks.")
|
|
||||||
|
|
||||||
```
|
|
||||||
|
|
||||||
## 6. Config operations
|
|
||||||
|
|
||||||
<a name="GetConfig"></a>
|
<a name="GetConfig"></a>
|
||||||
### GetConfig() ([]byte, error)
|
### GetConfig() ([]byte, error)
|
||||||
@ -373,7 +339,7 @@ __Example__
|
|||||||
log.Println("SetConfig: ", string(buf.Bytes()))
|
log.Println("SetConfig: ", string(buf.Bytes()))
|
||||||
```
|
```
|
||||||
|
|
||||||
## 7. Misc operations
|
## 8. Misc operations
|
||||||
|
|
||||||
<a name="SetCredentials"></a>
|
<a name="SetCredentials"></a>
|
||||||
|
|
||||||
|
@ -54,7 +54,7 @@ func (e ErrorResponse) Error() string {
|
|||||||
}
|
}
|
||||||
|
|
||||||
const (
|
const (
|
||||||
reportIssue = "Please report this issue at https://github.com/minio/minio-go/issues."
|
reportIssue = "Please report this issue at https://github.com/minio/minio/issues."
|
||||||
)
|
)
|
||||||
|
|
||||||
// httpRespToErrorResponse returns a new encoded ErrorResponse
|
// httpRespToErrorResponse returns a new encoded ErrorResponse
|
||||||
@ -65,8 +65,8 @@ func httpRespToErrorResponse(resp *http.Response) error {
|
|||||||
return ErrInvalidArgument(msg)
|
return ErrInvalidArgument(msg)
|
||||||
}
|
}
|
||||||
var errResp ErrorResponse
|
var errResp ErrorResponse
|
||||||
// Decode the xml error
|
// Decode the json error
|
||||||
err := xmlDecoder(resp.Body, &errResp)
|
err := jsonDecoder(resp.Body, &errResp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return ErrorResponse{
|
return ErrorResponse{
|
||||||
Code: resp.Status,
|
Code: resp.Status,
|
||||||
|
@ -71,6 +71,8 @@ type AdminClient struct {
|
|||||||
const (
|
const (
|
||||||
libraryName = "madmin-go"
|
libraryName = "madmin-go"
|
||||||
libraryVersion = "0.0.1"
|
libraryVersion = "0.0.1"
|
||||||
|
|
||||||
|
libraryAdminURLPrefix = "/minio/admin"
|
||||||
)
|
)
|
||||||
|
|
||||||
// User Agent should always following the below style.
|
// User Agent should always following the below style.
|
||||||
@ -176,6 +178,9 @@ type requestData struct {
|
|||||||
customHeaders http.Header
|
customHeaders http.Header
|
||||||
queryValues url.Values
|
queryValues url.Values
|
||||||
|
|
||||||
|
// Url path relative to admin API base endpoint
|
||||||
|
relPath string
|
||||||
|
|
||||||
contentBody io.Reader
|
contentBody io.Reader
|
||||||
contentLength int64
|
contentLength int64
|
||||||
contentSHA256Bytes []byte
|
contentSHA256Bytes []byte
|
||||||
@ -388,7 +393,7 @@ func (c AdminClient) newRequest(method string, reqData requestData) (req *http.R
|
|||||||
location := "us-east-1"
|
location := "us-east-1"
|
||||||
|
|
||||||
// Construct a new target URL.
|
// Construct a new target URL.
|
||||||
targetURL, err := c.makeTargetURL(reqData.queryValues)
|
targetURL, err := c.makeTargetURL(reqData)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -440,16 +445,16 @@ func (c AdminClient) newRequest(method string, reqData requestData) (req *http.R
|
|||||||
}
|
}
|
||||||
|
|
||||||
// makeTargetURL make a new target url.
|
// makeTargetURL make a new target url.
|
||||||
func (c AdminClient) makeTargetURL(queryValues url.Values) (*url.URL, error) {
|
func (c AdminClient) makeTargetURL(r requestData) (*url.URL, error) {
|
||||||
|
|
||||||
host := c.endpointURL.Host
|
host := c.endpointURL.Host
|
||||||
scheme := c.endpointURL.Scheme
|
scheme := c.endpointURL.Scheme
|
||||||
|
|
||||||
urlStr := scheme + "://" + host + "/"
|
urlStr := scheme + "://" + host + libraryAdminURLPrefix + r.relPath
|
||||||
|
|
||||||
// If there are any query values, add them to the end.
|
// If there are any query values, add them to the end.
|
||||||
if len(queryValues) > 0 {
|
if len(r.queryValues) > 0 {
|
||||||
urlStr = urlStr + "?" + s3utils.QueryEncode(queryValues)
|
urlStr = urlStr + "?" + s3utils.QueryEncode(r.queryValues)
|
||||||
}
|
}
|
||||||
u, err := url.Parse(urlStr)
|
u, err := url.Parse(urlStr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -20,14 +20,10 @@ package madmin
|
|||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
|
||||||
)
|
|
||||||
|
|
||||||
const (
|
|
||||||
configQueryParam = "config"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// NodeSummary - represents the result of an operation part of
|
// NodeSummary - represents the result of an operation part of
|
||||||
@ -47,20 +43,14 @@ type SetConfigResult struct {
|
|||||||
|
|
||||||
// GetConfig - returns the config.json of a minio setup.
|
// GetConfig - returns the config.json of a minio setup.
|
||||||
func (adm *AdminClient) GetConfig() ([]byte, error) {
|
func (adm *AdminClient) GetConfig() ([]byte, error) {
|
||||||
queryVal := make(url.Values)
|
// No TLS?
|
||||||
queryVal.Set(configQueryParam, "")
|
if !adm.secure {
|
||||||
|
return nil, fmt.Errorf("credentials/configuration cannot be retrieved over an insecure connection")
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "get")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Execute GET on /?config to get config of a setup.
|
// Execute GET on /minio/admin/v1/config to get config of a setup.
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
resp, err := adm.executeMethod("GET",
|
||||||
|
requestData{relPath: "/v1/config"})
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -75,50 +65,42 @@ func (adm *AdminClient) GetConfig() ([]byte, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// SetConfig - set config supplied as config.json for the setup.
|
// SetConfig - set config supplied as config.json for the setup.
|
||||||
func (adm *AdminClient) SetConfig(config io.Reader) (SetConfigResult, error) {
|
func (adm *AdminClient) SetConfig(config io.Reader) (r SetConfigResult, err error) {
|
||||||
queryVal := url.Values{}
|
// No TLS?
|
||||||
queryVal.Set(configQueryParam, "")
|
if !adm.secure {
|
||||||
|
return r, fmt.Errorf("credentials/configuration cannot be updated over an insecure connection")
|
||||||
// Set x-minio-operation to set.
|
}
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "set")
|
|
||||||
|
|
||||||
// Read config bytes to calculate MD5, SHA256 and content length.
|
// Read config bytes to calculate MD5, SHA256 and content length.
|
||||||
configBytes, err := ioutil.ReadAll(config)
|
configBytes, err := ioutil.ReadAll(config)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return SetConfigResult{}, err
|
return r, err
|
||||||
}
|
}
|
||||||
|
|
||||||
reqData := requestData{
|
reqData := requestData{
|
||||||
queryValues: queryVal,
|
relPath: "/v1/config",
|
||||||
customHeaders: hdrs,
|
|
||||||
contentBody: bytes.NewReader(configBytes),
|
contentBody: bytes.NewReader(configBytes),
|
||||||
contentMD5Bytes: sumMD5(configBytes),
|
contentMD5Bytes: sumMD5(configBytes),
|
||||||
contentSHA256Bytes: sum256(configBytes),
|
contentSHA256Bytes: sum256(configBytes),
|
||||||
}
|
}
|
||||||
|
|
||||||
// Execute PUT on /?config to set config.
|
// Execute PUT on /minio/admin/v1/config to set config.
|
||||||
resp, err := adm.executeMethod("PUT", reqData)
|
resp, err := adm.executeMethod("PUT", reqData)
|
||||||
|
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return SetConfigResult{}, err
|
return r, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
if resp.StatusCode != http.StatusOK {
|
||||||
return SetConfigResult{}, httpRespToErrorResponse(resp)
|
return r, httpRespToErrorResponse(resp)
|
||||||
}
|
}
|
||||||
|
|
||||||
var result SetConfigResult
|
|
||||||
jsonBytes, err := ioutil.ReadAll(resp.Body)
|
jsonBytes, err := ioutil.ReadAll(resp.Body)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return SetConfigResult{}, err
|
return r, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = json.Unmarshal(jsonBytes, &result)
|
err = json.Unmarshal(jsonBytes, &r)
|
||||||
if err != nil {
|
return r, err
|
||||||
return SetConfigResult{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
return result, nil
|
|
||||||
}
|
}
|
||||||
|
@ -19,7 +19,4 @@ package madmin
|
|||||||
const (
|
const (
|
||||||
// Unsigned payload.
|
// Unsigned payload.
|
||||||
unsignedPayload = "UNSIGNED-PAYLOAD"
|
unsignedPayload = "UNSIGNED-PAYLOAD"
|
||||||
|
|
||||||
// Admin operation header.
|
|
||||||
minioAdminOpHeader = "X-Minio-Operation"
|
|
||||||
)
|
)
|
||||||
|
@ -19,46 +19,50 @@ package madmin
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"encoding/xml"
|
"encoding/json"
|
||||||
|
"fmt"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// setCredsReq - xml to send to the server to set new credentials
|
// SetCredsReq - xml to send to the server to set new credentials
|
||||||
type setCredsReq struct {
|
type SetCredsReq struct {
|
||||||
Username string `xml:"username"`
|
AccessKey string `json:"accessKey"`
|
||||||
Password string `xml:"password"`
|
SecretKey string `json:"secretKey"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// SetCredentials - Call Set Credentials API to set new access and secret keys in the specified Minio server
|
// SetCredentials - Call Set Credentials API to set new access and
|
||||||
|
// secret keys in the specified Minio server
|
||||||
func (adm *AdminClient) SetCredentials(access, secret string) error {
|
func (adm *AdminClient) SetCredentials(access, secret string) error {
|
||||||
|
|
||||||
// Setup new request
|
|
||||||
reqData := requestData{}
|
|
||||||
reqData.queryValues = make(url.Values)
|
|
||||||
reqData.queryValues.Set("service", "")
|
|
||||||
reqData.customHeaders = make(http.Header)
|
|
||||||
reqData.customHeaders.Set(minioAdminOpHeader, "set-credentials")
|
|
||||||
|
|
||||||
// Setup request's body
|
// Setup request's body
|
||||||
body, err := xml.Marshal(setCredsReq{Username: access, Password: secret})
|
body, err := json.Marshal(SetCredsReq{access, secret})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
reqData.contentBody = bytes.NewReader(body)
|
|
||||||
reqData.contentLength = int64(len(body))
|
// No TLS?
|
||||||
reqData.contentMD5Bytes = sumMD5(body)
|
if !adm.secure {
|
||||||
reqData.contentSHA256Bytes = sum256(body)
|
return fmt.Errorf("credentials cannot be updated over an insecure connection")
|
||||||
|
}
|
||||||
|
|
||||||
|
// Setup new request
|
||||||
|
reqData := requestData{
|
||||||
|
relPath: "/v1/config/credential",
|
||||||
|
contentBody: bytes.NewReader(body),
|
||||||
|
contentLength: int64(len(body)),
|
||||||
|
contentMD5Bytes: sumMD5(body),
|
||||||
|
contentSHA256Bytes: sum256(body),
|
||||||
|
}
|
||||||
|
|
||||||
// Execute GET on bucket to list objects.
|
// Execute GET on bucket to list objects.
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
resp, err := adm.executeMethod("PUT", reqData)
|
||||||
|
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Return error to the caller if http response code is different from 200
|
// Return error to the caller if http response code is
|
||||||
|
// different from 200
|
||||||
if resp.StatusCode != http.StatusOK {
|
if resp.StatusCode != http.StatusOK {
|
||||||
return httpRespToErrorResponse(resp)
|
return httpRespToErrorResponse(resp)
|
||||||
}
|
}
|
||||||
|
@ -20,456 +20,157 @@
|
|||||||
package madmin
|
package madmin
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"bytes"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"encoding/xml"
|
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
"net/url"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
// listBucketHealResult container for listObjects response.
|
// HealOpts - collection of options for a heal sequence
|
||||||
type listBucketHealResult struct {
|
type HealOpts struct {
|
||||||
// A response can contain CommonPrefixes only if you have
|
Recursive bool `json:"recursive"`
|
||||||
// specified a delimiter.
|
DryRun bool `json:"dryRun"`
|
||||||
CommonPrefixes []commonPrefix
|
|
||||||
// Metadata about each object returned.
|
|
||||||
Contents []ObjectInfo
|
|
||||||
Delimiter string
|
|
||||||
|
|
||||||
// Encoding type used to encode object keys in the response.
|
|
||||||
EncodingType string
|
|
||||||
|
|
||||||
// A flag that indicates whether or not ListObjects returned all of the results
|
|
||||||
// that satisfied the search criteria.
|
|
||||||
IsTruncated bool
|
|
||||||
Marker string
|
|
||||||
MaxKeys int64
|
|
||||||
Name string
|
|
||||||
|
|
||||||
// When response is truncated (the IsTruncated element value in
|
|
||||||
// the response is true), you can use the key name in this field
|
|
||||||
// as marker in the subsequent request to get next set of objects.
|
|
||||||
// Object storage lists objects in alphabetical order Note: This
|
|
||||||
// element is returned only if you have delimiter request
|
|
||||||
// parameter specified. If response does not include the NextMaker
|
|
||||||
// and it is truncated, you can use the value of the last Key in
|
|
||||||
// the response as the marker in the subsequent request to get the
|
|
||||||
// next set of object keys.
|
|
||||||
NextMarker string
|
|
||||||
Prefix string
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// commonPrefix container for prefix response.
|
// HealStartSuccess - holds information about a successfully started
|
||||||
type commonPrefix struct {
|
// heal operation
|
||||||
Prefix string
|
type HealStartSuccess struct {
|
||||||
|
ClientToken string `json:"clientToken"`
|
||||||
|
ClientAddress string `json:"clientAddress"`
|
||||||
|
StartTime time.Time `json:"startTime"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// Owner - bucket owner/principal
|
// HealTaskStatus - status struct for a heal task
|
||||||
type Owner struct {
|
type HealTaskStatus struct {
|
||||||
ID string
|
Summary string `json:"summary"`
|
||||||
DisplayName string
|
FailureDetail string `json:"detail"`
|
||||||
|
StartTime time.Time `json:"startTime"`
|
||||||
|
HealSettings HealOpts `json:"settings"`
|
||||||
|
NumDisks int `json:"numDisks"`
|
||||||
|
|
||||||
|
Items []HealResultItem `json:"items,omitempty"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// Bucket container for bucket metadata
|
// HealItemType - specify the type of heal operation in a healing
|
||||||
type Bucket struct {
|
// result
|
||||||
Name string
|
type HealItemType string
|
||||||
CreationDate string // time string of format "2006-01-02T15:04:05.000Z"
|
|
||||||
|
|
||||||
HealBucketInfo *HealBucketInfo `xml:"HealBucketInfo,omitempty"`
|
|
||||||
}
|
|
||||||
|
|
||||||
// ListBucketsHealResponse - format for list buckets response
|
|
||||||
type ListBucketsHealResponse struct {
|
|
||||||
XMLName xml.Name `xml:"http://s3.amazonaws.com/doc/2006-03-01/ ListAllMyBucketsResult" json:"-"`
|
|
||||||
|
|
||||||
Owner Owner
|
|
||||||
|
|
||||||
// Container for one or more buckets.
|
|
||||||
Buckets struct {
|
|
||||||
Buckets []Bucket `xml:"Bucket"`
|
|
||||||
} // Buckets are nested
|
|
||||||
}
|
|
||||||
|
|
||||||
// HealStatus - represents different states of healing an object could be in.
|
|
||||||
type HealStatus int
|
|
||||||
|
|
||||||
|
// HealItemType constants
|
||||||
const (
|
const (
|
||||||
// Healthy - Object that is already healthy
|
HealItemMetadata HealItemType = "metadata"
|
||||||
Healthy HealStatus = iota
|
HealItemBucket = "bucket"
|
||||||
// CanHeal - Object can be healed
|
HealItemBucketMetadata = "bucket-metadata"
|
||||||
CanHeal
|
HealItemObject = "object"
|
||||||
// Corrupted - Object can't be healed
|
|
||||||
Corrupted
|
|
||||||
// QuorumUnavailable - Object can't be healed until read
|
|
||||||
// quorum is available
|
|
||||||
QuorumUnavailable
|
|
||||||
// CanPartiallyHeal - Object can't be healed completely until
|
|
||||||
// disks with missing parts come online
|
|
||||||
CanPartiallyHeal
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// HealBucketInfo - represents healing related information of a bucket.
|
// Drive state constants
|
||||||
type HealBucketInfo struct {
|
|
||||||
Status HealStatus
|
|
||||||
}
|
|
||||||
|
|
||||||
// BucketInfo - represents bucket metadata.
|
|
||||||
type BucketInfo struct {
|
|
||||||
// Name of the bucket.
|
|
||||||
Name string
|
|
||||||
|
|
||||||
// Date and time when the bucket was created.
|
|
||||||
Created time.Time
|
|
||||||
|
|
||||||
// Healing information
|
|
||||||
HealBucketInfo *HealBucketInfo `xml:"HealBucketInfo,omitempty"`
|
|
||||||
}
|
|
||||||
|
|
||||||
// HealObjectInfo - represents healing related information of an object.
|
|
||||||
type HealObjectInfo struct {
|
|
||||||
Status HealStatus
|
|
||||||
MissingDataCount int
|
|
||||||
MissingParityCount int
|
|
||||||
}
|
|
||||||
|
|
||||||
// ObjectInfo container for object metadata.
|
|
||||||
type ObjectInfo struct {
|
|
||||||
// An ETag is optionally set to md5sum of an object. In case of multipart objects,
|
|
||||||
// ETag is of the form MD5SUM-N where MD5SUM is md5sum of all individual md5sums of
|
|
||||||
// each parts concatenated into one string.
|
|
||||||
ETag string `json:"etag"`
|
|
||||||
|
|
||||||
Key string `json:"name"` // Name of the object
|
|
||||||
LastModified time.Time `json:"lastModified"` // Date and time the object was last modified.
|
|
||||||
Size int64 `json:"size"` // Size in bytes of the object.
|
|
||||||
ContentType string `json:"contentType"` // A standard MIME type describing the format of the object data.
|
|
||||||
|
|
||||||
// Collection of additional metadata on the object.
|
|
||||||
// eg: x-amz-meta-*, content-encoding etc.
|
|
||||||
Metadata http.Header `json:"metadata"`
|
|
||||||
|
|
||||||
// Owner name.
|
|
||||||
Owner struct {
|
|
||||||
DisplayName string `json:"name"`
|
|
||||||
ID string `json:"id"`
|
|
||||||
} `json:"owner"`
|
|
||||||
|
|
||||||
// The class of storage used to store the object.
|
|
||||||
StorageClass string `json:"storageClass"`
|
|
||||||
|
|
||||||
// Error
|
|
||||||
Err error `json:"-"`
|
|
||||||
HealObjectInfo *HealObjectInfo `json:"healObjectInfo,omitempty"`
|
|
||||||
}
|
|
||||||
|
|
||||||
type healQueryKey string
|
|
||||||
|
|
||||||
const (
|
const (
|
||||||
healBucket healQueryKey = "bucket"
|
DriveStateOk string = "ok"
|
||||||
healObject healQueryKey = "object"
|
DriveStateOffline = "offline"
|
||||||
healPrefix healQueryKey = "prefix"
|
DriveStateCorrupt = "corrupt"
|
||||||
healMarker healQueryKey = "marker"
|
DriveStateMissing = "missing"
|
||||||
healDelimiter healQueryKey = "delimiter"
|
|
||||||
healMaxKey healQueryKey = "max-key"
|
|
||||||
healDryRun healQueryKey = "dry-run"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// mkHealQueryVal - helper function to construct heal REST API query params.
|
// HealResultItem - struct for an individual heal result item
|
||||||
func mkHealQueryVal(bucket, prefix, marker, delimiter, maxKeyStr string) url.Values {
|
type HealResultItem struct {
|
||||||
queryVal := make(url.Values)
|
ResultIndex int64 `json:"resultId"`
|
||||||
queryVal.Set("heal", "")
|
Type HealItemType `json:"type"`
|
||||||
queryVal.Set(string(healBucket), bucket)
|
Bucket string `json:"bucket"`
|
||||||
queryVal.Set(string(healPrefix), prefix)
|
Object string `json:"object"`
|
||||||
queryVal.Set(string(healMarker), marker)
|
Detail string `json:"detail"`
|
||||||
queryVal.Set(string(healDelimiter), delimiter)
|
ParityBlocks int `json:"parityBlocks,omitempty"`
|
||||||
queryVal.Set(string(healMaxKey), maxKeyStr)
|
DataBlocks int `json:"dataBlocks,omitempty"`
|
||||||
return queryVal
|
DiskCount int `json:"diskCount"`
|
||||||
|
DriveInfo struct {
|
||||||
|
// below maps are from drive endpoint to drive state
|
||||||
|
Before map[string]string `json:"before"`
|
||||||
|
After map[string]string `json:"after"`
|
||||||
|
} `json:"drives"`
|
||||||
|
ObjectSize int64 `json:"objectSize"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// listObjectsHeal - issues heal list API request for a batch of maxKeys objects to be healed.
|
// InitDrives - initialize maps used to represent drive info
|
||||||
func (adm *AdminClient) listObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (listBucketHealResult, error) {
|
func (hri *HealResultItem) InitDrives() {
|
||||||
// Construct query params.
|
hri.DriveInfo.Before = make(map[string]string)
|
||||||
maxKeyStr := fmt.Sprintf("%d", maxKeys)
|
hri.DriveInfo.After = make(map[string]string)
|
||||||
queryVal := mkHealQueryVal(bucket, prefix, marker, delimiter, maxKeyStr)
|
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "list-objects")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
|
||||||
|
|
||||||
// Empty 'list' of objects to be healed.
|
|
||||||
toBeHealedObjects := listBucketHealResult{}
|
|
||||||
|
|
||||||
// Execute GET on /?heal to list objects needing heal.
|
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
|
||||||
|
|
||||||
defer closeResponse(resp)
|
|
||||||
if err != nil {
|
|
||||||
return listBucketHealResult{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
|
||||||
return toBeHealedObjects, httpRespToErrorResponse(resp)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
err = xml.NewDecoder(resp.Body).Decode(&toBeHealedObjects)
|
|
||||||
return toBeHealedObjects, err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListObjectsHeal - Lists upto maxKeys objects that needing heal matching bucket, prefix, marker, delimiter.
|
// GetOnlineCounts - returns the number of online disks before and
|
||||||
func (adm *AdminClient) ListObjectsHeal(bucket, prefix string, recursive bool, doneCh <-chan struct{}) (<-chan ObjectInfo, error) {
|
// after heal
|
||||||
// Allocate new list objects channel.
|
func (hri *HealResultItem) GetOnlineCounts() (b, a int) {
|
||||||
objectStatCh := make(chan ObjectInfo, 1)
|
if hri == nil {
|
||||||
// Default listing is delimited at "/"
|
return
|
||||||
delimiter := "/"
|
|
||||||
if recursive {
|
|
||||||
// If recursive we do not delimit.
|
|
||||||
delimiter = ""
|
|
||||||
}
|
}
|
||||||
|
for _, v := range hri.DriveInfo.Before {
|
||||||
// Initiate list objects goroutine here.
|
if v == DriveStateOk {
|
||||||
go func(objectStatCh chan<- ObjectInfo) {
|
b++
|
||||||
defer close(objectStatCh)
|
|
||||||
// Save marker for next request.
|
|
||||||
var marker string
|
|
||||||
for {
|
|
||||||
// Get list of objects a maximum of 1000 per request.
|
|
||||||
result, err := adm.listObjectsHeal(bucket, prefix, marker, delimiter, 1000)
|
|
||||||
if err != nil {
|
|
||||||
objectStatCh <- ObjectInfo{
|
|
||||||
Err: err,
|
|
||||||
}
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
// If contents are available loop through and send over channel.
|
|
||||||
for _, object := range result.Contents {
|
|
||||||
// Save the marker.
|
|
||||||
marker = object.Key
|
|
||||||
select {
|
|
||||||
// Send object content.
|
|
||||||
case objectStatCh <- object:
|
|
||||||
// If receives done from the caller, return here.
|
|
||||||
case <-doneCh:
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Send all common prefixes if any.
|
|
||||||
// NOTE: prefixes are only present if the request is delimited.
|
|
||||||
for _, obj := range result.CommonPrefixes {
|
|
||||||
object := ObjectInfo{}
|
|
||||||
object.Key = obj.Prefix
|
|
||||||
object.Size = 0
|
|
||||||
select {
|
|
||||||
// Send object prefixes.
|
|
||||||
case objectStatCh <- object:
|
|
||||||
// If receives done from the caller, return here.
|
|
||||||
case <-doneCh:
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If next marker present, save it for next request.
|
|
||||||
if result.NextMarker != "" {
|
|
||||||
marker = result.NextMarker
|
|
||||||
}
|
|
||||||
|
|
||||||
// Listing ends result is not truncated, return right here.
|
|
||||||
if !result.IsTruncated {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}(objectStatCh)
|
|
||||||
return objectStatCh, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
const timeFormatAMZLong = "2006-01-02T15:04:05.000Z" // Reply date format with nanosecond precision.
|
|
||||||
|
|
||||||
// ListBucketsHeal - issues heal bucket list API request
|
|
||||||
func (adm *AdminClient) ListBucketsHeal() ([]BucketInfo, error) {
|
|
||||||
queryVal := url.Values{}
|
|
||||||
queryVal.Set("heal", "")
|
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "list-buckets")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
}
|
||||||
|
for _, v := range hri.DriveInfo.After {
|
||||||
// Execute GET on /?heal to list objects needing heal.
|
if v == DriveStateOk {
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
a++
|
||||||
|
|
||||||
defer closeResponse(resp)
|
|
||||||
if err != nil {
|
|
||||||
return []BucketInfo{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
|
||||||
return []BucketInfo{}, httpRespToErrorResponse(resp)
|
|
||||||
}
|
|
||||||
|
|
||||||
var listBucketsHealResult ListBucketsHealResponse
|
|
||||||
|
|
||||||
err = xml.NewDecoder(resp.Body).Decode(&listBucketsHealResult)
|
|
||||||
if err != nil {
|
|
||||||
return []BucketInfo{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
var bucketsToBeHealed []BucketInfo
|
|
||||||
|
|
||||||
for _, bucket := range listBucketsHealResult.Buckets.Buckets {
|
|
||||||
creationDate, err := time.Parse(timeFormatAMZLong, bucket.CreationDate)
|
|
||||||
if err != nil {
|
|
||||||
return []BucketInfo{}, err
|
|
||||||
}
|
}
|
||||||
bucketsToBeHealed = append(bucketsToBeHealed,
|
|
||||||
BucketInfo{
|
|
||||||
Name: bucket.Name,
|
|
||||||
Created: creationDate,
|
|
||||||
HealBucketInfo: bucket.HealBucketInfo,
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
|
return
|
||||||
return bucketsToBeHealed, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// HealBucket - Heal the given bucket
|
// Heal - API endpoint to start heal and to fetch status
|
||||||
func (adm *AdminClient) HealBucket(bucket string, dryrun bool) error {
|
func (adm *AdminClient) Heal(bucket, prefix string, healOpts HealOpts,
|
||||||
// Construct query params.
|
clientToken string, forceStart bool) (
|
||||||
queryVal := url.Values{}
|
healStart HealStartSuccess, healTaskStatus HealTaskStatus, err error) {
|
||||||
queryVal.Set("heal", "")
|
|
||||||
queryVal.Set(string(healBucket), bucket)
|
body, err := json.Marshal(healOpts)
|
||||||
if dryrun {
|
if err != nil {
|
||||||
queryVal.Set(string(healDryRun), "")
|
return healStart, healTaskStatus, err
|
||||||
}
|
}
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
path := fmt.Sprintf("/v1/heal/%s", bucket)
|
||||||
hdrs.Set(minioAdminOpHeader, "bucket")
|
if bucket != "" && prefix != "" {
|
||||||
|
path += "/" + prefix
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Execute POST on /?heal&bucket=mybucket to heal a bucket.
|
// execute POST request to heal api
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
queryVals := make(url.Values)
|
||||||
|
var contentBody io.Reader
|
||||||
|
if clientToken != "" {
|
||||||
|
queryVals.Set("clientToken", clientToken)
|
||||||
|
} else {
|
||||||
|
// Set a body only if clientToken is not given
|
||||||
|
contentBody = bytes.NewReader(body)
|
||||||
|
}
|
||||||
|
if forceStart {
|
||||||
|
queryVals.Set("forceStart", "true")
|
||||||
|
}
|
||||||
|
|
||||||
|
resp, err := adm.executeMethod("POST", requestData{
|
||||||
|
relPath: path,
|
||||||
|
contentBody: contentBody,
|
||||||
|
contentSHA256Bytes: sum256(body),
|
||||||
|
queryValues: queryVals,
|
||||||
|
})
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return healStart, healTaskStatus, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
if resp.StatusCode != http.StatusOK {
|
||||||
return httpRespToErrorResponse(resp)
|
return healStart, healTaskStatus, httpRespToErrorResponse(resp)
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
respBytes, err := ioutil.ReadAll(resp.Body)
|
||||||
}
|
|
||||||
|
|
||||||
// HealResult - represents result of heal-object admin API.
|
|
||||||
type HealResult struct {
|
|
||||||
State HealState `json:"state"`
|
|
||||||
}
|
|
||||||
|
|
||||||
// HealState - different states of heal operation
|
|
||||||
type HealState int
|
|
||||||
|
|
||||||
const (
|
|
||||||
// HealNone - none of the disks healed
|
|
||||||
HealNone HealState = iota
|
|
||||||
// HealPartial - some disks were healed, others were offline
|
|
||||||
HealPartial
|
|
||||||
// HealOK - all disks were healed
|
|
||||||
HealOK
|
|
||||||
)
|
|
||||||
|
|
||||||
// HealObject - Heal the given object.
|
|
||||||
func (adm *AdminClient) HealObject(bucket, object string, dryrun bool) (HealResult, error) {
|
|
||||||
// Construct query params.
|
|
||||||
queryVal := url.Values{}
|
|
||||||
queryVal.Set("heal", "")
|
|
||||||
queryVal.Set(string(healBucket), bucket)
|
|
||||||
queryVal.Set(string(healObject), object)
|
|
||||||
if dryrun {
|
|
||||||
queryVal.Set(string(healDryRun), "")
|
|
||||||
}
|
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "object")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
|
||||||
|
|
||||||
// Execute POST on /?heal&bucket=mybucket&object=myobject to heal an object.
|
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
|
||||||
|
|
||||||
defer closeResponse(resp)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return HealResult{}, err
|
return healStart, healTaskStatus, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
// Was it a status request?
|
||||||
return HealResult{}, httpRespToErrorResponse(resp)
|
if clientToken == "" {
|
||||||
|
err = json.Unmarshal(respBytes, &healStart)
|
||||||
|
} else {
|
||||||
|
err = json.Unmarshal(respBytes, &healTaskStatus)
|
||||||
}
|
}
|
||||||
|
return healStart, healTaskStatus, err
|
||||||
// Healing is not performed so heal object result is empty.
|
|
||||||
if dryrun {
|
|
||||||
return HealResult{}, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
jsonBytes, err := ioutil.ReadAll(resp.Body)
|
|
||||||
if err != nil {
|
|
||||||
return HealResult{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
healResult := HealResult{}
|
|
||||||
err = json.Unmarshal(jsonBytes, &healResult)
|
|
||||||
if err != nil {
|
|
||||||
return HealResult{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
return healResult, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// HealFormat - heal storage format on available disks.
|
|
||||||
func (adm *AdminClient) HealFormat(dryrun bool) error {
|
|
||||||
queryVal := url.Values{}
|
|
||||||
queryVal.Set("heal", "")
|
|
||||||
if dryrun {
|
|
||||||
queryVal.Set(string(healDryRun), "")
|
|
||||||
}
|
|
||||||
|
|
||||||
// Set x-minio-operation to format.
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "format")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
|
||||||
|
|
||||||
// Execute POST on /?heal to heal storage format.
|
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
|
||||||
|
|
||||||
defer closeResponse(resp)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
if resp.StatusCode != http.StatusOK {
|
|
||||||
return httpRespToErrorResponse(resp)
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
@ -21,7 +21,6 @@ import (
|
|||||||
"encoding/json"
|
"encoding/json"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -115,13 +114,7 @@ type ServerInfo struct {
|
|||||||
// ServerInfo - Connect to a minio server and call Server Info Management API
|
// ServerInfo - Connect to a minio server and call Server Info Management API
|
||||||
// to fetch server's information represented by ServerInfo structure
|
// to fetch server's information represented by ServerInfo structure
|
||||||
func (adm *AdminClient) ServerInfo() ([]ServerInfo, error) {
|
func (adm *AdminClient) ServerInfo() ([]ServerInfo, error) {
|
||||||
// Prepare web service request
|
resp, err := adm.executeMethod("GET", requestData{relPath: "/v1/info"})
|
||||||
reqData := requestData{}
|
|
||||||
reqData.queryValues = make(url.Values)
|
|
||||||
reqData.queryValues.Set("info", "")
|
|
||||||
reqData.customHeaders = make(http.Header)
|
|
||||||
|
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -82,24 +82,19 @@ func getLockInfos(body io.Reader) ([]VolumeLockInfo, error) {
|
|||||||
|
|
||||||
// ListLocks - Calls List Locks Management API to fetch locks matching
|
// ListLocks - Calls List Locks Management API to fetch locks matching
|
||||||
// bucket, prefix and held before the duration supplied.
|
// bucket, prefix and held before the duration supplied.
|
||||||
func (adm *AdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
func (adm *AdminClient) ListLocks(bucket, prefix string,
|
||||||
|
duration time.Duration) ([]VolumeLockInfo, error) {
|
||||||
|
|
||||||
queryVal := make(url.Values)
|
queryVal := make(url.Values)
|
||||||
queryVal.Set("lock", "")
|
|
||||||
queryVal.Set("bucket", bucket)
|
queryVal.Set("bucket", bucket)
|
||||||
queryVal.Set("prefix", prefix)
|
queryVal.Set("prefix", prefix)
|
||||||
queryVal.Set("duration", duration.String())
|
queryVal.Set("older-than", duration.String())
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "list")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
|
||||||
|
|
||||||
// Execute GET on /?lock to list locks.
|
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
|
||||||
|
|
||||||
|
// Execute GET on /minio/admin/v1/locks to list locks.
|
||||||
|
resp, err := adm.executeMethod("GET", requestData{
|
||||||
|
queryValues: queryVal,
|
||||||
|
relPath: "/v1/locks",
|
||||||
|
})
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -114,24 +109,19 @@ func (adm *AdminClient) ListLocks(bucket, prefix string, duration time.Duration)
|
|||||||
|
|
||||||
// ClearLocks - Calls Clear Locks Management API to clear locks held
|
// ClearLocks - Calls Clear Locks Management API to clear locks held
|
||||||
// on bucket, matching prefix older than duration supplied.
|
// on bucket, matching prefix older than duration supplied.
|
||||||
func (adm *AdminClient) ClearLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
func (adm *AdminClient) ClearLocks(bucket, prefix string,
|
||||||
|
duration time.Duration) ([]VolumeLockInfo, error) {
|
||||||
|
|
||||||
queryVal := make(url.Values)
|
queryVal := make(url.Values)
|
||||||
queryVal.Set("lock", "")
|
|
||||||
queryVal.Set("bucket", bucket)
|
queryVal.Set("bucket", bucket)
|
||||||
queryVal.Set("prefix", prefix)
|
queryVal.Set("prefix", prefix)
|
||||||
queryVal.Set("duration", duration.String())
|
queryVal.Set("duration", duration.String())
|
||||||
|
|
||||||
hdrs := make(http.Header)
|
|
||||||
hdrs.Set(minioAdminOpHeader, "clear")
|
|
||||||
|
|
||||||
reqData := requestData{
|
|
||||||
queryValues: queryVal,
|
|
||||||
customHeaders: hdrs,
|
|
||||||
}
|
|
||||||
|
|
||||||
// Execute POST on /?lock to clear locks.
|
// Execute POST on /?lock to clear locks.
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
resp, err := adm.executeMethod("DELETE", requestData{
|
||||||
|
queryValues: queryVal,
|
||||||
|
relPath: "/v1/locks",
|
||||||
|
})
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -18,69 +18,79 @@
|
|||||||
package madmin
|
package madmin
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"bytes"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
// ServiceStatusMetadata - contains the response of service status API
|
// ServerVersion - server version
|
||||||
type ServiceStatusMetadata struct {
|
type ServerVersion struct {
|
||||||
Uptime time.Duration `json:"uptime"`
|
Version string `json:"version"`
|
||||||
|
CommitID string `json:"commitID"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// ServiceStatus - Connect to a minio server and call Service Status Management API
|
// ServiceStatus - contains the response of service status API
|
||||||
// to fetch server's storage information represented by ServiceStatusMetadata structure
|
type ServiceStatus struct {
|
||||||
func (adm *AdminClient) ServiceStatus() (ServiceStatusMetadata, error) {
|
ServerVersion ServerVersion `json:"serverVersion"`
|
||||||
|
Uptime time.Duration `json:"uptime"`
|
||||||
|
}
|
||||||
|
|
||||||
// Prepare web service request
|
// ServiceStatus - Connect to a minio server and call Service Status
|
||||||
reqData := requestData{}
|
// Management API to fetch server's storage information represented by
|
||||||
reqData.queryValues = make(url.Values)
|
// ServiceStatusMetadata structure
|
||||||
reqData.queryValues.Set("service", "")
|
func (adm *AdminClient) ServiceStatus() (ss ServiceStatus, err error) {
|
||||||
reqData.customHeaders = make(http.Header)
|
// Request API to GET service status
|
||||||
reqData.customHeaders.Set(minioAdminOpHeader, "status")
|
resp, err := adm.executeMethod("GET", requestData{relPath: "/v1/service"})
|
||||||
|
|
||||||
// Execute GET on bucket to list objects.
|
|
||||||
resp, err := adm.executeMethod("GET", reqData)
|
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return ServiceStatusMetadata{}, err
|
return ss, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check response http status code
|
// Check response http status code
|
||||||
if resp.StatusCode != http.StatusOK {
|
if resp.StatusCode != http.StatusOK {
|
||||||
return ServiceStatusMetadata{}, httpRespToErrorResponse(resp)
|
return ss, httpRespToErrorResponse(resp)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unmarshal the server's json response
|
|
||||||
var serviceStatus ServiceStatusMetadata
|
|
||||||
|
|
||||||
respBytes, err := ioutil.ReadAll(resp.Body)
|
respBytes, err := ioutil.ReadAll(resp.Body)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return ServiceStatusMetadata{}, err
|
return ss, err
|
||||||
}
|
}
|
||||||
|
|
||||||
err = json.Unmarshal(respBytes, &serviceStatus)
|
err = json.Unmarshal(respBytes, &ss)
|
||||||
if err != nil {
|
return ss, err
|
||||||
return ServiceStatusMetadata{}, err
|
|
||||||
}
|
|
||||||
|
|
||||||
return serviceStatus, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ServiceRestart - Call Service Restart API to restart a specified Minio server
|
// ServiceActionValue - type to restrict service-action values
|
||||||
func (adm *AdminClient) ServiceRestart() error {
|
type ServiceActionValue string
|
||||||
//
|
|
||||||
reqData := requestData{}
|
|
||||||
reqData.queryValues = make(url.Values)
|
|
||||||
reqData.queryValues.Set("service", "")
|
|
||||||
reqData.customHeaders = make(http.Header)
|
|
||||||
reqData.customHeaders.Set(minioAdminOpHeader, "restart")
|
|
||||||
|
|
||||||
// Execute GET on bucket to list objects.
|
const (
|
||||||
resp, err := adm.executeMethod("POST", reqData)
|
// ServiceActionValueRestart represents restart action
|
||||||
|
ServiceActionValueRestart ServiceActionValue = "restart"
|
||||||
|
// ServiceActionValueStop represents stop action
|
||||||
|
ServiceActionValueStop = "stop"
|
||||||
|
)
|
||||||
|
|
||||||
|
// ServiceAction - represents POST body for service action APIs
|
||||||
|
type ServiceAction struct {
|
||||||
|
Action ServiceActionValue `json:"action"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// ServiceSendAction - Call Service Restart/Stop API to restart/stop a
|
||||||
|
// Minio server
|
||||||
|
func (adm *AdminClient) ServiceSendAction(action ServiceActionValue) error {
|
||||||
|
body, err := json.Marshal(ServiceAction{action})
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Request API to Restart server
|
||||||
|
resp, err := adm.executeMethod("POST", requestData{
|
||||||
|
relPath: "/v1/service",
|
||||||
|
contentBody: bytes.NewReader(body),
|
||||||
|
contentSHA256Bytes: sum256(body),
|
||||||
|
})
|
||||||
defer closeResponse(resp)
|
defer closeResponse(resp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
|
@ -18,7 +18,7 @@ package madmin
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"crypto/md5"
|
"crypto/md5"
|
||||||
"encoding/xml"
|
"encoding/json"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"net"
|
"net"
|
||||||
@ -45,9 +45,9 @@ func sumMD5(data []byte) []byte {
|
|||||||
return hash.Sum(nil)
|
return hash.Sum(nil)
|
||||||
}
|
}
|
||||||
|
|
||||||
// xmlDecoder provide decoded value in xml.
|
// jsonDecoder decode json to go type.
|
||||||
func xmlDecoder(body io.Reader, v interface{}) error {
|
func jsonDecoder(body io.Reader, v interface{}) error {
|
||||||
d := xml.NewDecoder(body)
|
d := json.NewDecoder(body)
|
||||||
return d.Decode(v)
|
return d.Decode(v)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
54
pkg/madmin/version-commands.go
Normal file
54
pkg/madmin/version-commands.go
Normal file
@ -0,0 +1,54 @@
|
|||||||
|
/*
|
||||||
|
* Minio Cloud Storage, (C) 2017 Minio, Inc.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
* you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
|
||||||
|
package madmin
|
||||||
|
|
||||||
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
"io/ioutil"
|
||||||
|
"net/http"
|
||||||
|
)
|
||||||
|
|
||||||
|
// AdminAPIVersionInfo - contains admin API version information
|
||||||
|
type AdminAPIVersionInfo struct {
|
||||||
|
Version string `json:"version"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// VersionInfo - Connect to minio server and call the version API to
|
||||||
|
// retrieve the server API version
|
||||||
|
func (adm *AdminClient) VersionInfo() (verInfo AdminAPIVersionInfo, err error) {
|
||||||
|
var resp *http.Response
|
||||||
|
resp, err = adm.executeMethod("GET", requestData{relPath: "/version"})
|
||||||
|
defer closeResponse(resp)
|
||||||
|
if err != nil {
|
||||||
|
return verInfo, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check response http status code
|
||||||
|
if resp.StatusCode != http.StatusOK {
|
||||||
|
return verInfo, httpRespToErrorResponse(resp)
|
||||||
|
}
|
||||||
|
|
||||||
|
respBytes, err := ioutil.ReadAll(resp.Body)
|
||||||
|
if err != nil {
|
||||||
|
return verInfo, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Unmarshal the server's json response
|
||||||
|
err = json.Unmarshal(respBytes, &verInfo)
|
||||||
|
return verInfo, err
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user