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:
Aditya Manthramurthy 2018-01-22 14:54:55 -08:00 committed by Harshavardhana
parent f3f09ed14e
commit a337ea4d11
43 changed files with 2414 additions and 2319 deletions

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
View 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)
}

View file

@ -16,7 +16,15 @@
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.
type adminAPIHandlers struct {
@ -27,46 +35,44 @@ func registerAdminRouter(mux *router.Router) {
adminAPI := adminAPIHandlers{}
// 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 status
adminRouter.Methods("GET").Queries("service", "").Headers(minioAdminOpHeader, "status").HandlerFunc(adminAPI.ServiceStatusHandler)
adminV1Router.Methods(http.MethodGet).Path("/service").HandlerFunc(adminAPI.ServiceStatusHandler)
// Service restart
adminRouter.Methods("POST").Queries("service", "").Headers(minioAdminOpHeader, "restart").HandlerFunc(adminAPI.ServiceRestartHandler)
// Service update credentials
adminRouter.Methods("POST").Queries("service", "").Headers(minioAdminOpHeader, "set-credentials").HandlerFunc(adminAPI.ServiceCredentialsHandler)
// Service restart and stop - TODO
adminV1Router.Methods(http.MethodPost).Path("/service").HandlerFunc(adminAPI.ServiceStopNRestartHandler)
// Info operations
adminRouter.Methods("GET").Queries("info", "").HandlerFunc(adminAPI.ServerInfoHandler)
adminV1Router.Methods(http.MethodGet).Path("/info").HandlerFunc(adminAPI.ServerInfoHandler)
/// Lock operations
// List Locks
adminRouter.Methods("GET").Queries("lock", "").Headers(minioAdminOpHeader, "list").HandlerFunc(adminAPI.ListLocksHandler)
adminV1Router.Methods(http.MethodGet).Path("/locks").HandlerFunc(adminAPI.ListLocksHandler)
// Clear locks
adminRouter.Methods("POST").Queries("lock", "").Headers(minioAdminOpHeader, "clear").HandlerFunc(adminAPI.ClearLocksHandler)
adminV1Router.Methods(http.MethodDelete).Path("/locks").HandlerFunc(adminAPI.ClearLocksHandler)
/// Heal operations
// List Objects needing heal.
adminRouter.Methods("GET").Queries("heal", "").Headers(minioAdminOpHeader, "list-objects").HandlerFunc(adminAPI.ListObjectsHealHandler)
// List Buckets needing heal.
adminRouter.Methods("GET").Queries("heal", "").Headers(minioAdminOpHeader, "list-buckets").HandlerFunc(adminAPI.ListBucketsHealHandler)
// 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)
// Heal processing endpoint.
adminV1Router.Methods(http.MethodPost).Path("/heal/").HandlerFunc(adminAPI.HealHandler)
adminV1Router.Methods(http.MethodPost).Path("/heal/{bucket}").HandlerFunc(adminAPI.HealHandler)
adminV1Router.Methods(http.MethodPost).Path("/heal/{bucket}/{prefix:.*}").HandlerFunc(adminAPI.HealHandler)
/// Config operations
// Update credentials
adminV1Router.Methods(http.MethodPut).Path("/config/credential").HandlerFunc(adminAPI.UpdateCredentialsHandler)
// Get config
adminRouter.Methods("GET").Queries("config", "").Headers(minioAdminOpHeader, "get").HandlerFunc(adminAPI.GetConfigHandler)
// Set Config
adminRouter.Methods("PUT").Queries("config", "").Headers(minioAdminOpHeader, "set").HandlerFunc(adminAPI.SetConfigHandler)
adminV1Router.Methods(http.MethodGet).Path("/config").HandlerFunc(adminAPI.GetConfigHandler)
// Set config
adminV1Router.Methods(http.MethodPut).Path("/config").HandlerFunc(adminAPI.SetConfigHandler)
}

View file

@ -34,7 +34,7 @@ import (
const (
// Admin service names
serviceRestartRPC = "Admin.Restart"
signalServiceRPC = "Admin.SignalService"
listLocksRPC = "Admin.ListLocks"
reInitDisksRPC = "Admin.ReInitDisks"
serverInfoDataRPC = "Admin.ServerInfoData"
@ -56,7 +56,7 @@ type remoteAdminClient struct {
// adminCmdRunner - abstracts local and remote execution of admin
// commands like service stop and service restart.
type adminCmdRunner interface {
Restart() error
SignalService(s serviceSignal) error
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
ReInitDisks() error
ServerInfoData() (ServerInfoData, error)
@ -65,10 +65,16 @@ type adminCmdRunner interface {
CommitConfig(tmpFileName string) error
}
// Restart - Sends a message over channel to the go-routine
// responsible for restarting the process.
func (lc localAdminClient) Restart() error {
globalServiceSignalCh <- serviceRestart
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
// SignalService - sends a restart or stop signal to the local server
func (lc localAdminClient) SignalService(s serviceSignal) error {
switch s {
case serviceRestart, serviceStop:
globalServiceSignalCh <- s
default:
return errUnsupportedSignal
}
return nil
}
@ -77,25 +83,31 @@ func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Durati
return listLocksInfo(bucket, prefix, duration), nil
}
// Restart - Sends restart command to remote server via RPC.
func (rc remoteAdminClient) Restart() error {
args := AuthRPCArgs{}
reply := AuthRPCReply{}
return rc.Call(serviceRestartRPC, &args, &reply)
func (rc remoteAdminClient) SignalService(s serviceSignal) (err error) {
switch s {
case serviceRestart, serviceStop:
reply := AuthRPCReply{}
err = rc.Call(signalServiceRPC, &SignalServiceArgs{Sig: s},
&reply)
default:
err = errUnsupportedSignal
}
return err
}
// ListLocks - Sends list locks command to remote server via RPC.
func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
listArgs := ListLocksQuery{
bucket: bucket,
prefix: prefix,
duration: duration,
Bucket: bucket,
Prefix: prefix,
Duration: duration,
}
var reply ListLocksReply
if err := rc.Call(listLocksRPC, &listArgs, &reply); err != nil {
return nil, err
}
return reply.volLocks, nil
return reply.VolLocks, nil
}
// ReInitDisks - There is nothing to do here, heal format REST API
@ -225,7 +237,7 @@ func (rc remoteAdminClient) CommitConfig(tmpFileName string) error {
return nil
}
// adminPeer - represents an entity that implements Restart methods.
// adminPeer - represents an entity that implements admin API RPCs.
type adminPeer struct {
addr string
cmdRunner adminCmdRunner
@ -274,11 +286,11 @@ func initGlobalAdminPeers(endpoints EndpointList) {
globalAdminPeers = makeAdminPeers(endpoints)
}
// invokeServiceCmd - Invoke Restart command.
// invokeServiceCmd - Invoke Restart/Stop command.
func invokeServiceCmd(cp adminPeer, cmd serviceSignal) (err error) {
switch cmd {
case serviceRestart:
err = cp.cmdRunner.Restart()
case serviceRestart, serviceStop:
err = cp.cmdRunner.SignalService(cmd)
}
return err
}

View file

@ -38,18 +38,24 @@ type adminCmd struct {
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.
type ListLocksQuery struct {
AuthRPCArgs
bucket string
prefix string
duration time.Duration
Bucket string
Prefix string
Duration time.Duration
}
// ListLocksReply - wraps ListLocks response over RPC.
type ListLocksReply struct {
AuthRPCReply
volLocks []VolumeLockInfo
VolLocks []VolumeLockInfo
}
// ServerInfoDataReply - wraps the server info response over RPC.
@ -64,13 +70,13 @@ type ConfigReply struct {
Config []byte // json-marshalled bytes of serverConfigV13
}
// Restart - Restart this instance of minio server.
func (s *adminCmd) Restart(args *AuthRPCArgs, reply *AuthRPCReply) error {
// SignalService - Send a restart or stop signal to the service
func (s *adminCmd) SignalService(args *SignalServiceArgs, reply *AuthRPCReply) error {
if err := args.IsAuthenticated(); err != nil {
return err
}
globalServiceSignalCh <- serviceRestart
globalServiceSignalCh <- args.Sig
return nil
}
@ -79,8 +85,8 @@ func (s *adminCmd) ListLocks(query *ListLocksQuery, reply *ListLocksReply) error
if err := query.IsAuthenticated(); err != nil {
return err
}
volLocks := listLocksInfo(query.bucket, query.prefix, query.duration)
*reply = ListLocksReply{volLocks: volLocks}
volLocks := listLocksInfo(query.Bucket, query.Prefix, query.Duration)
*reply = ListLocksReply{VolLocks: volLocks}
return nil
}

View file

@ -23,8 +23,8 @@ import (
)
func testAdminCmd(cmd cmdType, t *testing.T) {
// reset globals.
// this is to make sure that the tests are not affected by modified globals.
// reset globals. this is to make sure that the tests are not
// affected by modified globals.
resetTestGlobals()
rootPath, err := newTestConfig(globalMinioDefaultRegion)
@ -55,12 +55,22 @@ func testAdminCmd(cmd cmdType, t *testing.T) {
<-globalServiceSignalCh
}()
ga := AuthRPCArgs{AuthToken: token}
sa := SignalServiceArgs{
AuthRPCArgs: AuthRPCArgs{AuthToken: token},
Sig: cmd.toServiceSignal(),
}
genReply := AuthRPCReply{}
switch cmd {
case restartCmd:
if err = adminServer.Restart(&ga, &genReply); err != nil {
t.Errorf("restartCmd: Expected: <nil>, got: %v", err)
case restartCmd, stopCmd:
if err = adminServer.SignalService(&sa, &genReply); err != nil {
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)
}
// 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.
func TestReInitDisks(t *testing.T) {
// Reset global variables to start afresh.

View file

@ -41,8 +41,8 @@ type APIErrorResponse struct {
Key string
BucketName string
Resource string
RequestID string `xml:"RequestId"`
HostID string `xml:"HostId"`
RequestID string `xml:"RequestId" json:"RequestId"`
HostID string `xml:"HostId" json:"HostId"`
}
// APIErrorCode type of error status.
@ -158,6 +158,7 @@ const (
ErrReadQuorum
ErrWriteQuorum
ErrStorageFull
ErrRequestBodyParse
ErrObjectExistsAsDirectory
ErrPolicyNesting
ErrInvalidObjectName
@ -174,6 +175,7 @@ const (
// Please open a https://github.com/minio/minio/issues before adding
// new error codes here.
ErrMalformedJSON
ErrAdminInvalidAccessKey
ErrAdminInvalidSecretKey
ErrAdminConfigNoQuorum
@ -183,6 +185,11 @@ const (
ErrInsecureClientRequest
ErrObjectTampered
ErrHealNotImplemented
ErrHealNoSuchProcess
ErrHealInvalidClientToken
ErrHealMissingBucket
ErrHealAlreadyRunning
ErrHealOverlappingPaths
)
// 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.",
HTTPStatusCode: http.StatusInternalServerError,
},
ErrRequestBodyParse: {
Code: "XMinioRequestBodyParse",
Description: "The request body failed to parse.",
HTTPStatusCode: http.StatusBadRequest,
},
ErrObjectExistsAsDirectory: {
Code: "XMinioObjectExistsAsDirectory",
Description: "Object name already exists as a directory.",
@ -708,6 +720,11 @@ var errorCodeResponse = map[APIErrorCode]APIError{
Description: "Server not initialized, please try again.",
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: {
Code: "XMinioAdminInvalidAccessKey",
Description: "The access key is invalid.",
@ -764,11 +781,6 @@ var errorCodeResponse = map[APIErrorCode]APIError{
Description: errObjectTampered.Error(),
HTTPStatusCode: http.StatusPartialContent,
},
ErrHealNotImplemented: {
Code: "XMinioHealNotImplemented",
Description: "This server does not implement heal functionality.",
HTTPStatusCode: http.StatusBadRequest,
},
ErrMaximumExpires: {
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",
@ -782,6 +794,36 @@ var errorCodeResponse = map[APIErrorCode]APIError{
Description: "Invalid Request",
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.
}

View file

@ -18,6 +18,7 @@ package cmd
import (
"bytes"
"encoding/json"
"encoding/xml"
"fmt"
"net/http"
@ -53,6 +54,14 @@ func encodeResponse(response interface{}) []byte {
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
func setObjectHeaders(w http.ResponseWriter, objInfo ObjectInfo, contentRange *httpRange) {
// set common headers

View file

@ -166,13 +166,12 @@ type ListBucketsResponse struct {
// Upload container for in progress multipart upload
type Upload struct {
Key string
UploadID string `xml:"UploadId"`
Initiator Initiator
Owner Owner
StorageClass string
Initiated string
HealUploadInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
Key string
UploadID string `xml:"UploadId"`
Initiator Initiator
Owner Owner
StorageClass string
Initiated string
}
// CommonPrefix container for prefix response in ListObjectsResponse
@ -182,9 +181,8 @@ type CommonPrefix struct {
// Bucket container for bucket metadata
type Bucket struct {
Name string
CreationDate string // time string of format "2006-01-02T15:04:05.000Z"
HealBucketInfo *HealBucketInfo `xml:"HealBucketInfo,omitempty"`
Name string
CreationDate string // time string of format "2006-01-02T15:04:05.000Z"
}
// Object container for object metadata
@ -198,8 +196,7 @@ type Object struct {
Owner Owner
// The class of storage used to store the object.
StorageClass string
HealObjectInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
StorageClass string
}
// CopyObjectResponse container returns ETag and LastModified of the successfully copied object
@ -308,7 +305,6 @@ func generateListBucketsResponse(buckets []BucketInfo) ListBucketsResponse {
var listbucket = Bucket{}
listbucket.Name = bucket.Name
listbucket.CreationDate = bucket.Created.UTC().Format(timeFormatAMZLong)
listbucket.HealBucketInfo = bucket.HealBucketInfo
listbuckets = append(listbuckets, listbucket)
}
@ -339,8 +335,6 @@ func generateListObjectsV1Response(bucket, prefix, marker, delimiter string, max
content.Size = object.Size
content.StorageClass = globalMinioDefaultStorageClass
content.Owner = owner
// object.HealObjectInfo is non-empty only when resp is constructed in ListObjectsHeal.
content.HealObjectInfo = object.HealObjectInfo
contents = append(contents, content)
}
// TODO - support EncodingType in xml decoding
@ -498,7 +492,6 @@ func generateListMultipartUploadsResponse(bucket string, multipartsInfo ListMult
newUpload.UploadID = upload.UploadID
newUpload.Key = upload.Object
newUpload.Initiated = upload.Initiated.UTC().Format(timeFormatAMZLong)
newUpload.HealUploadInfo = upload.HealUploadInfo
listMultipartUploadsResponse.Uploads[index] = newUpload
}
return listMultipartUploadsResponse
@ -584,3 +577,31 @@ func writeErrorResponseHeadersOnly(w http.ResponseWriter, errorCode APIErrorCode
apiError := getAPIError(errorCode)
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)
}

View file

@ -186,6 +186,15 @@ func (endpoints EndpointList) IsHTTPS() bool {
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.
func NewEndpointList(args ...string) (endpoints EndpointList, err error) {
// isValidDistribution - checks whether given count is a valid distribution for erasure coding.

View file

@ -143,14 +143,14 @@ func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatXLV1, []error) {
// Initialize format configs.
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 {
if disk == nil {
sErrs[index] = errDiskNotFound
continue
}
wg.Add(1)
// Make a volume inside a go-routine.
// Launch go-routine per disk.
go func(index int, disk StorageAPI) {
defer wg.Done()
format, lErr := loadFormat(disk)
@ -162,15 +162,9 @@ func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatXLV1, []error) {
}(index, disk)
}
// Wait for all make vol to finish.
// Wait for all go-routines to finish.
wg.Wait()
for _, err := range sErrs {
if err != nil {
// Return all formats and errors.
return formats, sErrs
}
}
// Return all formats and nil
return formats, sErrs
}
@ -374,8 +368,10 @@ func loadFormat(disk StorageAPI) (format *formatXLV1, err error) {
if err != nil {
return nil, err
}
if len(vols) > 1 {
// 'format.json' not found, but we found user data.
if len(vols) > 1 || (len(vols) == 1 &&
vols[0].Name != minioMetaBucket) {
// 'format.json' not found, but we
// found user data.
return nil, errCorruptedFormat
}
// No other data found, its a fresh disk.
@ -394,11 +390,10 @@ func loadFormat(disk StorageAPI) (format *formatXLV1, err error) {
return format, nil
}
// collectNSaveNewFormatConfigs - creates new format configs based on
// the reference config and saves it on all disks, this is to be
// called from healFormatXL* functions.
// collectNSaveNewFormatConfigs - generates new format configs based on
// the given ref. config and saves on each disk
func collectNSaveNewFormatConfigs(referenceConfig *formatXLV1,
orderedDisks []StorageAPI) error {
orderedDisks []StorageAPI, dryRun bool) error {
// Collect new format configs that need to be written.
var newFormatConfigs = make([]*formatXLV1, len(orderedDisks))
@ -420,16 +415,19 @@ func collectNSaveNewFormatConfigs(referenceConfig *formatXLV1,
}
// 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
// unexpected errors as regular errors can be ignored since there
// might be enough quorum to be operational. Heals only fresh disks.
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,
formats, true)
if err != nil {
@ -441,21 +439,24 @@ func healFormatXLFreshDisks(storageDisks []StorageAPI,
// and allowed fresh disks to be arranged anywhere.
// Following block facilitates to put fresh disks.
for index, format := range formats {
if format != nil {
continue
}
// Format is missing so we go through ordered disks.
if format == nil {
// At this point when disk is missing the fresh disk
// in the stack get it back from storageDisks.
for oIndex, disk := range orderedDisks {
if disk == nil {
orderedDisks[oIndex] = storageDisks[index]
break
}
// At this point when disk is missing the fresh disk
// in the stack get it back from storageDisks.
for oIndex, disk := range orderedDisks {
if disk == nil {
orderedDisks[oIndex] = storageDisks[index]
break
}
}
}
// apply new format config and save to all disks
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks)
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks,
dryRun)
}
// collectUnAssignedDisks - collect disks unassigned to orderedDisks
@ -540,9 +541,9 @@ func reorderDisksByInspection(orderedDisks, storageDisks []StorageAPI,
// Heals corrupted format json in all disks
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,
formats, true)
if err != nil {
@ -570,8 +571,9 @@ func healFormatXLCorruptedDisks(storageDisks []StorageAPI,
}
}
// apply new format config and save to all disks
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks)
// generate and write new configs to all disks
return collectNSaveNewFormatConfigs(referenceConfig, orderedDisks,
dryRun)
}
// loadFormatXL - loads XL `format.json` and returns back properly

View file

@ -278,7 +278,7 @@ func TestFormatXLHealFreshDisks(t *testing.T) {
formatConfigs, _ := loadAllFormats(storageDisks)
// Start healing disks
err = healFormatXLFreshDisks(storageDisks, formatConfigs)
err = healFormatXLFreshDisks(storageDisks, formatConfigs, false)
if err != nil {
t.Fatal("healing corrupted disk failed: ", err)
}
@ -352,7 +352,7 @@ func TestFormatXLHealCorruptedDisks(t *testing.T) {
formatConfigs, _ := loadAllFormats(permutedStorageDisks)
// Start healing disks
err = healFormatXLCorruptedDisks(permutedStorageDisks, formatConfigs)
err = healFormatXLCorruptedDisks(permutedStorageDisks, formatConfigs, false)
if err != nil {
t.Fatal("healing corrupted disk failed: ", err)
}
@ -761,7 +761,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
xl := obj.(*xlObjects)
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)
}
@ -784,7 +784,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
}
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
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)
}
removeRoots(fsDirs)
@ -806,7 +806,7 @@ func TestHealFormatXLCorruptedDisksErrs(t *testing.T) {
}
}
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, ")
}
removeRoots(fsDirs)
@ -833,7 +833,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
}
xl := obj.(*xlObjects)
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)
}
removeRoots(fsDirs)
@ -855,7 +855,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
}
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errFaultyDisk)
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)
}
removeRoots(fsDirs)
@ -873,7 +873,7 @@ func TestHealFormatXLFreshDisksErrs(t *testing.T) {
xl = obj.(*xlObjects)
xl.storageDisks[0] = nil
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)
}
removeRoots(fsDirs)

View file

@ -31,6 +31,7 @@ import (
"github.com/minio/minio/pkg/errors"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/lock"
"github.com/minio/minio/pkg/madmin"
)
// 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.
func (fs fsObjects) HealObject(bucket, object string) (int, int, error) {
return 0, 0, errors.Trace(NotImplemented{})
func (fs fsObjects) HealObject(bucket, object string, dryRun bool) (
res madmin.HealResultItem, err error) {
return res, errors.Trace(NotImplemented{})
}
// HealBucket - no-op for fs, Valid only for XL.
func (fs fsObjects) HealBucket(bucket string) error {
return errors.Trace(NotImplemented{})
func (fs fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem,
error) {
return nil, errors.Trace(NotImplemented{})
}
// ListObjectsHeal - list all objects to be healed. Valid only for XL

View file

@ -400,7 +400,7 @@ func TestFSHealObject(t *testing.T) {
defer os.RemoveAll(disk)
obj := initFSObjects(disk, t)
_, _, err := obj.HealObject("bucket", "object")
_, err := obj.HealObject("bucket", "object", false)
if err == nil || !isSameType(errors.Cause(err), NotImplemented{}) {
t.Fatalf("Heal Object should return NotImplemented error ")
}

View file

@ -23,6 +23,7 @@ import (
"github.com/minio/minio-go/pkg/policy"
"github.com/minio/minio/pkg/errors"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
)
// 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{})
}
// CopyObjectPart copy part of object to other bucket and 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) {
// CopyObjectPart copy part of object to uploadID for another object
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{})
}
@ -79,8 +80,8 @@ func (a GatewayUnsupported) DeleteBucketPolicies(bucket string) error {
}
// HealBucket - Not implemented stub
func (a GatewayUnsupported) HealBucket(bucket string) error {
return errors.Trace(NotImplemented{})
func (a GatewayUnsupported) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, error) {
return nil, errors.Trace(NotImplemented{})
}
// ListBucketsHeal - Not implemented stub
@ -89,8 +90,8 @@ func (a GatewayUnsupported) ListBucketsHeal() (buckets []BucketInfo, err error)
}
// HealObject - Not implemented stub
func (a GatewayUnsupported) HealObject(bucket, object string) (int, int, error) {
return 0, 0, errors.Trace(NotImplemented{})
func (a GatewayUnsupported) HealObject(bucket, object string, dryRun bool) (h madmin.HealResultItem, e error) {
return h, errors.Trace(NotImplemented{})
}
// ListObjectsV2 - Not implemented stub

View file

@ -246,6 +246,12 @@ func (h cacheControlHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
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.
type minioReservedBucketHandler struct {
handler http.Handler
@ -256,8 +262,12 @@ func setReservedBucketHandler(h http.Handler) http.Handler {
}
func (h minioReservedBucketHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if !guessIsRPCReq(r) && !guessIsBrowserReq(r) {
// For all non browser, non RPC requests, reject access to 'minioReservedBucketPath'.
switch {
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)
if isMinioReservedBucket(bucketName) || isMinioMetaBucket(bucketName) {
writeErrorResponse(w, ErrAllAccessDisabled, r.URL)
@ -433,6 +443,11 @@ func (h resourceHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
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.
h.handler.ServeHTTP(w, r)

View file

@ -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.
type BucketInfo struct {
// Name of the bucket.
@ -72,16 +57,6 @@ type BucketInfo struct {
// 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 - represents object metadata.
@ -113,8 +88,7 @@ type ObjectInfo struct {
ContentEncoding string
// User-Defined metadata
UserDefined map[string]string
HealObjectInfo *HealObjectInfo `xml:"HealObjectInfo,omitempty"`
UserDefined map[string]string
}
// ListPartsInfo - represents list of all parts.
@ -273,8 +247,6 @@ type MultipartInfo struct {
Initiated time.Time
StorageClass string // Not supported yet.
HealUploadInfo *HealObjectInfo `xml:"HealUploadInfo,omitempty"`
}
// CompletePart - represents the part that was completed, this is sent by the client

View file

@ -153,9 +153,6 @@ func checkPutObjectArgs(bucket, object string, obj ObjectLayer) error {
// Checks whether bucket exists and returns appropriate error if not.
func checkBucketExist(bucket string, obj ObjectLayer) error {
if !IsValidBucketName(bucket) {
return BucketNameInvalid{Bucket: bucket}
}
_, err := obj.GetBucketInfo(bucket)
if err != nil {
return errors.Cause(err)

View file

@ -21,6 +21,7 @@ import (
"time"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
)
// 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)
// 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)
HealObject(bucket, object string) (int, int, error)
ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (ListObjectsInfo, error)
// Locking operations

View file

@ -70,6 +70,9 @@ func configureServerHandler(endpoints EndpointList) (http.Handler, error) {
return nil, err
}
// Add Admin router.
registerAdminRouter(mux)
// Register web router when its enabled.
if globalIsBrowserEnabled {
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.
registerAPIRouter(mux)

View file

@ -193,6 +193,9 @@ func serverMain(ctx *cli.Context) {
// Initialize name space lock.
initNSLock(globalIsDistXL)
// Init global heal state
initAllHealState(globalIsXL)
// Configure server.
var handler http.Handler
handler, err = configureServerHandler(globalEndpoints)

View file

@ -497,6 +497,17 @@ func resetGlobalStorageEnvs() {
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.
// Resetting ensures that the changes made to globals by one test doesn't affect others.
func resetTestGlobals() {
@ -518,6 +529,8 @@ func resetTestGlobals() {
resetGlobalIsEnvs()
// Reset global storage class flags
resetGlobalStorageEnvs()
// Reset global heal state
resetGlobalHealState()
}
// Configure the server for the test run.

View file

@ -37,11 +37,7 @@ func commonTime(modTimes []time.Time) (modTime time.Time, count int) {
// Find the common cardinality from previously collected
// occurrences of elements.
for time, count := range timeOccurenceMap {
if count == maxima && time.After(modTime) {
maxima = count
modTime = time
} else if count > maxima {
if count > maxima || (count == maxima && time.After(modTime)) {
maxima = count
modTime = time
}
@ -126,7 +122,7 @@ func getLatestXLMeta(partsMetadata []xlMetaV1, errs []error) (xlMetaV1, int) {
// List all the file commit ids from parts metadata.
modTimes := listObjectModtimes(partsMetadata, errs)
// Count all lastest updated xlMeta values
// Count all latest updated xlMeta values
var count int
var latestXLMeta xlMetaV1
@ -144,140 +140,13 @@ func getLatestXLMeta(partsMetadata []xlMetaV1, errs []error) (xlMetaV1, int) {
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
// []StorageAPI returned by listOnlineDisks. Returns,
//
// - disks which have all parts specified in the latest xl.json.
//
// - errs updated to have errFileNotFound in place of disks that had
// missing or corrupted parts.
// - slice of errors about the state of data files on disk - can have
// a not-found error or a hash-mismatch error.
//
// - non-nil error if any of the disks failed unexpectedly (i.e. 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))
buffer := []byte{}
dataErrs := make([]error, len(onlineDisks))
for i, onlineDisk := range onlineDisks {
if onlineDisk == OfflineDisk {
if onlineDisk == nil {
continue
}
// disk has a valid xl.json but may not have all the
// parts. This is considered an outdated disk, since
// it needs healing too.
@ -302,22 +173,25 @@ func disksWithAllParts(onlineDisks []StorageAPI, partsMetadata []xlMetaV1, errs
// verification happens even if a 0-length
// buffer is passed
_, hErr := onlineDisk.ReadFile(bucket, partPath, 0, buffer, verifier)
if hErr != nil {
_, isCorrupted := hErr.(hashMismatchError)
if isCorrupted || hErr == errFileNotFound {
errs[i] = errFileNotFound
availableDisks[i] = OfflineDisk
break
}
_, isCorrupt := hErr.(hashMismatchError)
switch {
case isCorrupt:
fallthrough
case hErr == errFileNotFound, hErr == errVolumeNotFound:
dataErrs[i] = hErr
break
case hErr != nil:
// abort on unhandled errors
return nil, nil, errors.Trace(hErr)
}
}
if errs[i] == nil {
if dataErrs[i] == nil {
// All parts verified, mark it as all data available.
availableDisks[i] = onlineDisk
}
}
return availableDisks, errs, nil
return availableDisks, dataErrs, nil
}

View file

@ -107,20 +107,6 @@ func partsMetaFromModTimes(modTimes []time.Time, algorithm BitrotAlgorithm, chec
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
// are consistent with each other.
func TestListOnlineDisks(t *testing.T) {
@ -273,62 +259,22 @@ func TestListOnlineDisks(t *testing.T) {
partsMetadata := partsMetaFromModTimes(test.modTimes, DefaultBitrotAlgorithm, xlMeta.Erasure.Checksums)
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) {
t.Fatalf("Test %d: Expected modTime to be equal to %v but was found to be %v",
i+1, test.expectedTime, modTime)
}
// Check if a disk is considered both online and outdated,
// which is a contradiction, except if parts are missing.
overlappingDisks := make(map[string]*posix)
for _, availableDisk := range availableDisks {
if availableDisk == nil {
continue
availableDisks, newErrs, _ := disksWithAllParts(onlineDisks, partsMetadata, test.errs, bucket, object)
test.errs = newErrs
if test._tamperBackend != noTamper {
if tamperedIndex != -1 && availableDisks[tamperedIndex] != nil {
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
// meta data
// Test that all disks are returned without any failures with
// unmodified meta data
partsMetadata, errs = readAllXLMetadata(xlDisks, bucket, object)
if err != nil {
t.Fatalf("Failed to read xl meta data %v", err)

View file

@ -23,11 +23,14 @@ import (
"sync"
"github.com/minio/minio/pkg/errors"
"github.com/minio/minio/pkg/madmin"
)
// healFormatXL - heals missing `format.json` on freshly or corrupted
// 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`.
formatConfigs, sErrs := loadAllFormats(storageDisks)
@ -35,7 +38,34 @@ func healFormatXL(storageDisks []StorageAPI) (err error) {
// - if (no quorum) return error
// - if (disks not recognized) // Always error.
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)
@ -45,61 +75,95 @@ func healFormatXL(storageDisks []StorageAPI) (err error) {
switch {
case unformattedDiskCount == numDisks:
// all unformatted.
if err = initFormatXL(storageDisks); err != nil {
return err
if !dryRun {
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:
return fmt.Errorf("cannot proceed with heal as %s",
return res, fmt.Errorf("cannot proceed with heal as %s",
errSomeDiskOffline)
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:
if err = healFormatXLCorruptedDisks(storageDisks, formatConfigs); err != nil {
return fmt.Errorf("Unable to repair corrupted format, %s", err)
// heal corrupted disks
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:
// All drives online but some report missing format.json.
if err = healFormatXLFreshDisks(storageDisks, formatConfigs); err != nil {
// There was an unexpected unrecoverable error
// during healing.
return fmt.Errorf("Unable to heal backend %s", err)
// heal unformatted disks
err = healFormatXLFreshDisks(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
}
return nil
return res, nil
}
// Heals a bucket if it doesn't exist on one of the disks, additionally
// also heals the missing entries for bucket metadata files
// `policy.json, notification.xml, listeners.json`.
func (xl xlObjects) HealBucket(bucket string) error {
if err := checkBucketExist(bucket, xl); err != nil {
return err
func (xl xlObjects) HealBucket(bucket string, dryRun bool) (
results []madmin.HealResultItem, err error) {
if err = checkBucketExist(bucket, xl); err != nil {
return nil, err
}
// get write quorum for an object
writeQuorum := len(xl.storageDisks)/2 + 1
bucketLock := xl.nsMutex.NewNSLock(bucket, "")
if err := bucketLock.GetLock(globalHealingTimeout); err != nil {
return err
if err = bucketLock.GetLock(globalHealingTimeout); err != nil {
return nil, err
}
defer bucketLock.Unlock()
// Heal bucket.
if err := healBucket(xl.storageDisks, bucket, writeQuorum); err != nil {
return err
result, err := healBucket(xl.storageDisks, bucket, writeQuorum, dryRun)
if err != nil {
return results, err
}
results = append(results, result)
// 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.
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.
var wg = &sync.WaitGroup{}
@ -107,24 +171,47 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int) error
// Initialize list of errors.
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.
for index, disk := range storageDisks {
if disk == nil {
dErrs[index] = errors.Trace(errDiskNotFound)
beforeState[index] = madmin.DriveStateOffline
afterState[index] = madmin.DriveStateOffline
continue
}
wg.Add(1)
// Make a volume inside a go-routine.
go func(index int, disk StorageAPI) {
defer wg.Done()
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)
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)
}
@ -132,44 +219,75 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int) error
// Wait for all make vol to finish.
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)
if errors.Cause(reducedErr) == errXLWriteQuorum {
// Purge successfully created buckets if we don't have writeQuorum.
undoMakeBucket(storageDisks, bucket)
}
return reducedErr
return res, reducedErr
}
// Heals all the metadata associated for a given bucket, this function
// 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 {
if _, _, err := xlObj.HealObject(minioMetaBucket, metaPath); err != nil && !isErrObjectNotFound(err) {
return err
result, healErr := xl.HealObject(minioMetaBucket, metaPath, dryRun)
// 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
}
// 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)
if err := healBucketMetaFn(policyPath); err != nil {
return err
err = healBucketMetaFn(policyPath)
if err != nil {
return results, err
}
// Heal `notification.xml` for missing entries, ignores if `notification.xml` is not found.
nConfigPath := path.Join(bucketConfigPrefix, bucket, bucketNotificationConfig)
if err := healBucketMetaFn(nConfigPath); err != nil {
return err
// Heal `notification.xml` for missing entries, ignores if
// `notification.xml` is not found.
nConfigPath := path.Join(bucketConfigPrefix, bucket,
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)
return healBucketMetaFn(lConfigPath)
err = healBucketMetaFn(lConfigPath)
return results, err
}
// listAllBuckets lists all buckets from all disks. It also
// 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)
bucketsOcc = make(map[string]int)
for _, disk := range storageDisks {
@ -178,122 +296,42 @@ func listAllBuckets(storageDisks []StorageAPI) (buckets map[string]VolInfo, buck
}
var volsInfo []VolInfo
volsInfo, err = disk.ListVols()
if err == nil {
for _, volInfo := range volsInfo {
// StorageAPI can send volume names which are
// 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
if err != nil {
if errors.IsErrIgnored(err, bucketMetadataOpIgnoredErrs...) {
continue
}
continue
break
}
// Ignore any disks not found.
if errors.IsErrIgnored(err, bucketMetadataOpIgnoredErrs...) {
continue
for _, volInfo := range volsInfo {
// StorageAPI can send volume names which are
// 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
}
// 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
func (xl xlObjects) ListBucketsHeal() ([]BucketInfo, error) {
listBuckets := []BucketInfo{}
// List all buckets that can be found in all disks
buckets, occ, err := listAllBuckets(xl.storageDisks)
buckets, _, err := listAllBuckets(xl.storageDisks)
if err != nil {
return listBuckets, err
}
// Iterate over all buckets
for _, currBucket := range buckets {
// Check the status of bucket metadata
bucketHealStatus, err := xl.bucketHealStatus(currBucket.Name)
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},
})
}
listBuckets = append(listBuckets,
BucketInfo{currBucket.Name, currBucket.Created})
}
// Sort found buckets
@ -323,8 +361,8 @@ func quickHeal(xlObj xlObjects, writeQuorum int, readQuorum int) error {
defer bucketLock.Unlock()
// Heal bucket and then proceed to heal bucket metadata if any.
if err = healBucket(xlObj.storageDisks, bucketName, writeQuorum); err == nil {
if err = healBucketMetadata(xlObj, bucketName); err == nil {
if _, err = healBucket(xlObj.storageDisks, bucketName, writeQuorum, false); err == nil {
if _, err = healBucketMetadata(xlObj, bucketName, false); err == nil {
continue
}
return err
@ -337,77 +375,108 @@ func quickHeal(xlObj xlObjects, writeQuorum int, readQuorum int) error {
return nil
}
// Heals an object only the corrupted/missing erasure blocks.
func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (int, int, error) {
// Heals an object by re-writing corrupt/missing erasure blocks.
func healObject(storageDisks []StorageAPI, bucket string, object string,
quorum int, dryRun bool) (result madmin.HealResultItem, err error) {
partsMetadata, errs := readAllXLMetadata(storageDisks, bucket, object)
// readQuorum suffices for xl.json since we use monotonic
// system time to break the tie when a split-brain situation
// arises.
if rErr := reduceReadQuorumErrs(errs, nil, quorum); rErr != nil {
return 0, 0, toObjectErr(rErr, bucket, object)
if reducedErr := reduceReadQuorumErrs(errs, nil, quorum); reducedErr != nil {
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)
// List of disks having all parts as per latest xl.json - this
// does a full pass over the data and verifies all part files
// on disk
availableDisks, errs, aErr := disksWithAllParts(latestDisks, partsMetadata, errs, bucket,
object)
// List of disks having all parts as per latest xl.json.
availableDisks, dataErrs, aErr := disksWithAllParts(latestDisks, partsMetadata, errs, bucket, object)
if aErr != nil {
return 0, 0, toObjectErr(aErr, bucket, object)
return result, toObjectErr(aErr, bucket, object)
}
// Number of disks which don't serve data.
numOfflineDisks := 0
for index, disk := range storageDisks {
if disk == nil || errs[index] == errDiskNotFound {
numOfflineDisks++
}
}
// Initialize heal result object
result = madmin.HealResultItem{
Type: madmin.HealItemObject,
Bucket: bucket,
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
for _, disk := range availableDisks {
if disk != nil {
disksToHealCount := 0
for i, v := range availableDisks {
driveState := ""
switch {
case v != nil:
driveState = madmin.DriveStateOk
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) {
// nothing to heal in this case
return 0, 0, nil
// an online disk without valid data/metadata is
// outdated and can be healed.
if errs[i] != errDiskNotFound && v == nil {
outDatedDisks[i] = storageDisks[i]
disksToHealCount++
}
}
// If less than read quorum number of disks have all the parts
// of the data, we can't reconstruct the erasure-coded data.
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.
outDatedDisks := outDatedDisks(storageDisks, availableDisks, errs, partsMetadata, bucket,
object)
if disksToHealCount == 0 {
// Nothing to heal!
return result, nil
}
// Number of disks that had outdated content of the given
// object and are online to be healed.
numHealedDisks := 0
for _, disk := range outDatedDisks {
if disk != nil {
numHealedDisks++
}
// After this point, only have to repair data on disk - so
// return if it is a dry-run
if dryRun {
return result, nil
}
// Latest xlMetaV1 for reference. If a valid metadata is not
// present, it is as good as object not found.
latestMeta, pErr := pickValidXLMeta(partsMetadata, modTime)
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
// xl.json and part files from "bucket/object/" so
// that rename(minioMetaBucket, "tmp/tmpuuid/",
@ -417,18 +486,10 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
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
// errors.
files, err := disk.ListDir(bucket, object)
if err == nil {
files, derr := disk.ListDir(bucket, object)
if derr == nil {
for _, entry := range files {
_ = disk.DeleteFile(bucket,
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
// part to .minio/tmp/uuid/ which needs to be renamed later to
// the final location.
storage, err := NewErasureStorage(latestDisks,
latestMeta.Erasure.DataBlocks, latestMeta.Erasure.ParityBlocks, latestMeta.Erasure.BlockSize)
storage, err := NewErasureStorage(latestDisks, latestMeta.Erasure.DataBlocks,
latestMeta.Erasure.ParityBlocks, latestMeta.Erasure.BlockSize)
if err != nil {
return 0, 0, toObjectErr(err, bucket, object)
return result, toObjectErr(err, bucket, object)
}
checksums := make([][]byte, len(latestDisks))
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,
algorithm, checksums)
if hErr != nil {
return 0, 0, toObjectErr(hErr, bucket, object)
return result, toObjectErr(hErr, bucket, object)
}
// outDatedDisks that had write errors should not be
// 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.
if file.Checksums[i] == nil {
outDatedDisks[i] = nil
numHealedDisks--
disksToHealCount--
continue
}
// 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 numHealedDisks == 0 {
return 0, 0, fmt.Errorf("all disks without up-to-date data had write errors")
if disksToHealCount == 0 {
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,
partsMetadata, diskCount(outDatedDisks))
if aErr != nil {
return 0, 0, toObjectErr(aErr, bucket, object)
return result, toObjectErr(aErr, bucket, object)
}
// Rename from tmp location to the actual location.
for _, disk := range outDatedDisks {
for diskIndex, disk := range outDatedDisks {
if disk == nil {
continue
}
@ -527,33 +588,47 @@ func healObject(storageDisks []StorageAPI, bucket, object string, quorum int) (i
aErr = disk.RenameFile(minioMetaTmpBucket, retainSlash(tmpID), bucket,
retainSlash(object))
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,
// and later the disk comes back up again, heal on the object
// 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
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, bucket, 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 {
return 0, 0, err
return hr, err
}
// Lock the object before healing.
objectLock := xl.nsMutex.NewNSLock(bucket, object)
if err := objectLock.GetRLock(globalHealingTimeout); err != nil {
return 0, 0, err
if lerr := objectLock.GetRLock(globalHealingTimeout); lerr != nil {
return hr, lerr
}
defer objectLock.RUnlock()
// Heal the object.
return healObject(xl.storageDisks, bucket, object, readQuorum)
return healObject(xl.storageDisks, bucket, object, readQuorum, dryRun)
}

View file

@ -23,6 +23,7 @@ import (
"path/filepath"
"testing"
"github.com/minio/minio-go/pkg/set"
"github.com/minio/minio/pkg/errors"
)
@ -46,7 +47,7 @@ func TestHealFormatXL(t *testing.T) {
t.Fatal(err)
}
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)
}
@ -67,7 +68,7 @@ func TestHealFormatXL(t *testing.T) {
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)
}
removeRoots(fsDirs)
@ -90,7 +91,7 @@ func TestHealFormatXL(t *testing.T) {
}
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)
}
removeRoots(fsDirs)
@ -108,7 +109,7 @@ func TestHealFormatXL(t *testing.T) {
}
xl = obj.(*xlObjects)
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)
}
removeRoots(fsDirs)
@ -129,7 +130,7 @@ func TestHealFormatXL(t *testing.T) {
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)
}
removeRoots(fsDirs)
@ -150,7 +151,7 @@ func TestHealFormatXL(t *testing.T) {
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, ")
}
removeRoots(fsDirs)
@ -171,7 +172,7 @@ func TestHealFormatXL(t *testing.T) {
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)
}
removeRoots(fsDirs)
@ -198,7 +199,7 @@ func TestHealFormatXL(t *testing.T) {
}
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskNotFound)
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() {
t.Fatal("Got an unexpected error: ", err)
}
@ -228,7 +229,7 @@ func TestHealFormatXL(t *testing.T) {
}
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskAccessDenied)
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() {
t.Fatal("Got an unexpected error: ", err)
}
@ -254,7 +255,7 @@ func TestHealFormatXL(t *testing.T) {
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)
}
removeRoots(fsDirs)
@ -446,14 +447,18 @@ func TestListBucketsHeal(t *testing.T) {
t.Fatal(err)
}
bucketSet := set.CreateStringSet(saneBucket, corruptedBucketName)
// Check the number of buckets in list buckets heal result
if len(buckets) != 1 {
t.Fatalf("Length of missing buckets is incorrect, expected: 1, found: %d", len(buckets))
if len(buckets) != len(bucketSet) {
t.Fatalf("Length of missing buckets is incorrect, expected: 2, found: %d", len(buckets))
}
// Check the name of bucket in list buckets heal result
if buckets[0].Name != corruptedBucketName {
t.Fatalf("Name of missing bucket is incorrect, expected: %s, found: %s", corruptedBucketName, buckets[0].Name)
// Check each bucket name is in `bucketSet`v
for _, b := range buckets {
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)
}
_, _, err = obj.HealObject(bucket, object)
_, err = obj.HealObject(bucket, object, false)
if err != nil {
t.Fatalf("Failed to heal object - %v", err)
}
@ -536,7 +541,7 @@ func TestHealObjectXL(t *testing.T) {
}
// 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
if errors.Cause(err) != errXLReadQuorum {
t.Errorf("Expected %v but received %v", errDiskNotFound, err)

View file

@ -37,9 +37,6 @@ func listDirHealFactory(isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
continue
}
// Filter entries that have the prefix prefixEntry.
entries = filterMatchingPrefix(entries, prefixEntry)
// isLeaf() check has to happen here so that
// trailing "/" for objects can be removed.
for i, entry := range entries {
@ -63,6 +60,9 @@ func listDirHealFactory(isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
mergedEntries = append(mergedEntries, newEntries...)
sort.Strings(mergedEntries)
}
// Filter entries that have the prefix prefixEntry.
mergedEntries = filterMatchingPrefix(mergedEntries, prefixEntry)
}
return mergedEntries, false, nil
}
@ -141,23 +141,15 @@ func (xl xlObjects) listObjectsHeal(bucket, prefix, marker, delimiter string, ma
continue
}
// Check if the current object needs healing
objectLock := xl.nsMutex.NewNSLock(bucket, objInfo.Name)
if err := objectLock.GetRLock(globalHealingTimeout); err != nil {
return loi, err
}
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, bucket, objInfo.Name)
if xlShouldHeal(xl.storageDisks, partsMetadata, errs, bucket, objInfo.Name) {
healStat := xlHealStat(xl, partsMetadata, errs)
result.Objects = append(result.Objects, ObjectInfo{
Name: objInfo.Name,
ModTime: objInfo.ModTime,
Size: objInfo.Size,
IsDir: false,
HealObjectInfo: &healStat,
})
}
objectLock.RUnlock()
// Add each object seen to the result - objects are
// checked for healing later.
result.Objects = append(result.Objects, ObjectInfo{
Bucket: bucket,
Name: objInfo.Name,
ModTime: objInfo.ModTime,
Size: objInfo.Size,
IsDir: false,
})
}
return result, nil
}

View file

@ -54,14 +54,16 @@ func TestListObjectsHeal(t *testing.T) {
// Put 5 objects under sane dir
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 {
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++ {
_, 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 {
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
}
@ -101,7 +103,7 @@ func TestListObjectsHeal(t *testing.T) {
// Inexistent object
{bucketName, "inexistentObj", "", "", 1000, nil, 0},
// Test ListObjectsHeal when all objects are sane
{bucketName, "", "", "", 1000, nil, 0},
{bucketName, "", "", "", 1000, nil, 10},
}
for i, testCase := range testCases {
testFunc(testCase, i+1)
@ -119,12 +121,12 @@ func TestListObjectsHeal(t *testing.T) {
testCases = []testData{
// 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
{bucketName, "unsane/", "", "", 1000, nil, 5},
// List healing objects under sane/, should return 0
{bucketName, "sane/", "", "", 1000, nil, 0},
// Max Keys == 200
// List healing objects under sane/
{bucketName, "sane/", "", "", 1000, nil, 5},
// Max Keys == 2
{bucketName, "unsane/", "", "", 2, nil, 2},
// Max key > 1000
{bucketName, "unsane/", "", "", 5000, nil, 5},

View file

@ -316,7 +316,7 @@ func TestHealing(t *testing.T) {
t.Fatal(err)
}
_, _, err = xl.HealObject(bucket, object)
_, err = xl.HealObject(bucket, object, false)
if err != nil {
t.Fatal(err)
}
@ -340,7 +340,7 @@ func TestHealing(t *testing.T) {
t.Fatal(err)
}
_, _, err = xl.HealObject(bucket, object)
_, err = xl.HealObject(bucket, object, false)
if err != nil {
t.Fatal(err)
}
@ -362,7 +362,7 @@ func TestHealing(t *testing.T) {
t.Fatal(err)
}
// This would create the bucket.
err = xl.HealBucket(bucket)
_, err = xl.HealBucket(bucket, false)
if err != nil {
t.Fatal(err)
}

View file

@ -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
// quorum is not satisfied.
func reduceQuorumErrs(errs []error, ignoredErrs []error, quorum int, quorumErr error) (maxErr error) {
maxCount, maxErr := reduceErrs(errs, ignoredErrs)
if maxErr == nil && maxCount >= quorum {
var maxCount int
maxCount, maxErr = reduceErrs(errs, ignoredErrs)
switch {
case maxErr == nil && maxCount >= quorum:
// Success in quorum.
return nil
}
if maxErr != nil && maxCount >= quorum {
case maxErr != nil && maxCount >= 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
}
@ -365,6 +366,17 @@ func shuffleDisks(disks []StorageAPI, distribution []int) (shuffledDisks []Stora
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
// the corresponding error in errs slice is not nil
func evalDisks(disks []StorageAPI, errs []error) []StorageAPI {

View file

@ -36,13 +36,11 @@ func main() {
```
| Service operations|LockInfo operations|Healing operations|Config operations| Misc |
|:---|:---|:---|:---|:---|
|[`ServiceStatus`](#ServiceStatus)| [`ListLocks`](#ListLocks)| [`ListObjectsHeal`](#ListObjectsHeal)|[`GetConfig`](#GetConfig)| [`SetCredentials`](#SetCredentials)|
|[`ServiceRestart`](#ServiceRestart)| [`ClearLocks`](#ClearLocks)| [`ListBucketsHeal`](#ListBucketsHeal)|[`SetConfig`](#SetConfig)||
| | |[`HealBucket`](#HealBucket) |||
| | |[`HealObject`](#HealObject)|||
| | |[`HealFormat`](#HealFormat)|||
| Service operations | LockInfo operations | Healing operations | Config operations | Misc |
|:------------------------------------|:----------------------------|:--------------------------------------|:--------------------------|:------------------------------------|
| [`ServiceStatus`](#ServiceStatus) | [`ListLocks`](#ListLocks) | [`Heal`](#Heal) | [`GetConfig`](#GetConfig) | [`SetCredentials`](#SetCredentials) |
| [`ServiceSendAction`](#ServiceSendAction) | [`ClearLocks`](#ClearLocks) | | [`SetConfig`](#SetConfig) | |
## 1. Constructor
<a name="Minio"></a>
@ -60,8 +58,25 @@ __Parameters__
|`secretAccessKey` | _string_ |Secret key for the object storage endpoint. |
|`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>
### ServiceStatus() (ServiceStatusMetadata, error)
@ -102,17 +117,19 @@ Fetch service status, replies disk space used, backend type and total disks offl
```
<a name="ServiceRestart"></a>
### ServiceRestart() (error)
If successful restarts the running minio service, for distributed setup restarts all remote minio servers.
<a name="ServiceSendAction"></a>
### ServiceSendAction(act ServiceActionValue) (error)
Sends a service action command to service - possible actions are restarting and stopping the server.
__Example__
```go
st, err := madmClnt.ServiceRestart()
// to restart
st, err := madmClnt.ServiceSendAction(ServiceActionValueRestart)
// or to stop
// st, err := madmClnt.ServiceSendAction(ServiceActionValueStop)
if err != nil {
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>
### 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>
### ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
@ -175,146 +192,95 @@ __Example__
```
## 5. Heal operations
## 6. Heal operations
<a name="ListObjectsHeal"></a>
### ListObjectsHeal(bucket, prefix string, recursive bool, doneCh <-chan struct{}) (<-chan ObjectInfo, error)
If successful returns information on the list of objects that need healing in ``bucket`` matching ``prefix``.
<a name="Heal"></a>
### Heal(bucket, prefix string, healOpts HealOpts, clientToken string, forceStart bool) (start HealStartSuccess, status HealTaskStatus, err error)
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__
``` go
// Create a done channel to control 'ListObjectsHeal' go routine.
doneCh := make(chan struct{})
// Indicate to our routine to exit cleanly upon return.
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)
healPath, err := madmClnt.HealStart("", "", true, false, true, false, false)
if err != nil {
log.Fatalln(err)
}
log.Println("successfully healed mybucket")
log.Printf("Heal sequence started at %s", healPath)
```
<a name="HealObject"></a>
### 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.
#### HealTaskStatus structure
| Param | Type | Description |
|---|---|---|
|`h.State` | _HealState_ | Represents the result of heal operation. It could be one of `HealNone`, `HealPartial` or `HealOK`. |
| Param | Type | Description |
|----|--------|--------|
| 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 |
|---|---|
|`HealNone` | Object wasn't healed on any of the disks |
|`HealPartial` | Object was healed on some of the disks needing heal |
| `HealOK` | Object was healed on all the disks needing heal |
| Param | Type | Description |
|------|-------|---------|
| ResultIndex | _int64_ | Index of the heal-result record |
| Type | _HealItemType_ | Represents kind of heal operation in the heal record |
| 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__
``` go
isDryRun = false
healResult, err := madmClnt.HealObject("mybucket", "myobject", isDryRun)
res, err := madmClnt.HealStatus("", "")
if err != nil {
log.Fatalln(err)
}
log.Println("Heal-object result: ", healResult)
log.Printf("Heal sequence status data %#v", res)
```
<a name="HealFormat"></a>
### 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
## 7. Config operations
<a name="GetConfig"></a>
### GetConfig() ([]byte, error)
@ -373,7 +339,7 @@ __Example__
log.Println("SetConfig: ", string(buf.Bytes()))
```
## 7. Misc operations
## 8. Misc operations
<a name="SetCredentials"></a>

View file

@ -54,7 +54,7 @@ func (e ErrorResponse) Error() string {
}
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
@ -65,8 +65,8 @@ func httpRespToErrorResponse(resp *http.Response) error {
return ErrInvalidArgument(msg)
}
var errResp ErrorResponse
// Decode the xml error
err := xmlDecoder(resp.Body, &errResp)
// Decode the json error
err := jsonDecoder(resp.Body, &errResp)
if err != nil {
return ErrorResponse{
Code: resp.Status,

View file

@ -71,6 +71,8 @@ type AdminClient struct {
const (
libraryName = "madmin-go"
libraryVersion = "0.0.1"
libraryAdminURLPrefix = "/minio/admin"
)
// User Agent should always following the below style.
@ -176,6 +178,9 @@ type requestData struct {
customHeaders http.Header
queryValues url.Values
// Url path relative to admin API base endpoint
relPath string
contentBody io.Reader
contentLength int64
contentSHA256Bytes []byte
@ -388,7 +393,7 @@ func (c AdminClient) newRequest(method string, reqData requestData) (req *http.R
location := "us-east-1"
// Construct a new target URL.
targetURL, err := c.makeTargetURL(reqData.queryValues)
targetURL, err := c.makeTargetURL(reqData)
if err != nil {
return nil, err
}
@ -440,16 +445,16 @@ func (c AdminClient) newRequest(method string, reqData requestData) (req *http.R
}
// 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
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 len(queryValues) > 0 {
urlStr = urlStr + "?" + s3utils.QueryEncode(queryValues)
if len(r.queryValues) > 0 {
urlStr = urlStr + "?" + s3utils.QueryEncode(r.queryValues)
}
u, err := url.Parse(urlStr)
if err != nil {

View file

@ -20,14 +20,10 @@ package madmin
import (
"bytes"
"encoding/json"
"fmt"
"io"
"io/ioutil"
"net/http"
"net/url"
)
const (
configQueryParam = "config"
)
// 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.
func (adm *AdminClient) GetConfig() ([]byte, error) {
queryVal := make(url.Values)
queryVal.Set(configQueryParam, "")
hdrs := make(http.Header)
hdrs.Set(minioAdminOpHeader, "get")
reqData := requestData{
queryValues: queryVal,
customHeaders: hdrs,
// No TLS?
if !adm.secure {
return nil, fmt.Errorf("credentials/configuration cannot be retrieved over an insecure connection")
}
// Execute GET on /?config to get config of a setup.
resp, err := adm.executeMethod("GET", reqData)
// Execute GET on /minio/admin/v1/config to get config of a setup.
resp, err := adm.executeMethod("GET",
requestData{relPath: "/v1/config"})
defer closeResponse(resp)
if err != nil {
return nil, err
@ -75,50 +65,42 @@ func (adm *AdminClient) GetConfig() ([]byte, error) {
}
// SetConfig - set config supplied as config.json for the setup.
func (adm *AdminClient) SetConfig(config io.Reader) (SetConfigResult, error) {
queryVal := url.Values{}
queryVal.Set(configQueryParam, "")
// Set x-minio-operation to set.
hdrs := make(http.Header)
hdrs.Set(minioAdminOpHeader, "set")
func (adm *AdminClient) SetConfig(config io.Reader) (r SetConfigResult, err error) {
// No TLS?
if !adm.secure {
return r, fmt.Errorf("credentials/configuration cannot be updated over an insecure connection")
}
// Read config bytes to calculate MD5, SHA256 and content length.
configBytes, err := ioutil.ReadAll(config)
if err != nil {
return SetConfigResult{}, err
return r, err
}
reqData := requestData{
queryValues: queryVal,
customHeaders: hdrs,
relPath: "/v1/config",
contentBody: bytes.NewReader(configBytes),
contentMD5Bytes: sumMD5(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)
defer closeResponse(resp)
if err != nil {
return SetConfigResult{}, err
return r, err
}
if resp.StatusCode != http.StatusOK {
return SetConfigResult{}, httpRespToErrorResponse(resp)
return r, httpRespToErrorResponse(resp)
}
var result SetConfigResult
jsonBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return SetConfigResult{}, err
return r, err
}
err = json.Unmarshal(jsonBytes, &result)
if err != nil {
return SetConfigResult{}, err
}
return result, nil
err = json.Unmarshal(jsonBytes, &r)
return r, err
}

View file

@ -19,7 +19,4 @@ package madmin
const (
// Unsigned payload.
unsignedPayload = "UNSIGNED-PAYLOAD"
// Admin operation header.
minioAdminOpHeader = "X-Minio-Operation"
)

View file

@ -19,46 +19,50 @@ package madmin
import (
"bytes"
"encoding/xml"
"encoding/json"
"fmt"
"net/http"
"net/url"
)
// setCredsReq - xml to send to the server to set new credentials
type setCredsReq struct {
Username string `xml:"username"`
Password string `xml:"password"`
// SetCredsReq - xml to send to the server to set new credentials
type SetCredsReq struct {
AccessKey string `json:"accessKey"`
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 {
// 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
body, err := xml.Marshal(setCredsReq{Username: access, Password: secret})
body, err := json.Marshal(SetCredsReq{access, secret})
if err != nil {
return err
}
reqData.contentBody = bytes.NewReader(body)
reqData.contentLength = int64(len(body))
reqData.contentMD5Bytes = sumMD5(body)
reqData.contentSHA256Bytes = sum256(body)
// No TLS?
if !adm.secure {
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.
resp, err := adm.executeMethod("POST", reqData)
resp, err := adm.executeMethod("PUT", reqData)
defer closeResponse(resp)
if err != nil {
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 {
return httpRespToErrorResponse(resp)
}

View file

@ -20,456 +20,157 @@
package madmin
import (
"bytes"
"encoding/json"
"encoding/xml"
"fmt"
"io"
"io/ioutil"
"net/http"
"net/url"
"time"
)
// listBucketHealResult container for listObjects response.
type listBucketHealResult struct {
// A response can contain CommonPrefixes only if you have
// specified a delimiter.
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
// HealOpts - collection of options for a heal sequence
type HealOpts struct {
Recursive bool `json:"recursive"`
DryRun bool `json:"dryRun"`
}
// commonPrefix container for prefix response.
type commonPrefix struct {
Prefix string
// HealStartSuccess - holds information about a successfully started
// heal operation
type HealStartSuccess struct {
ClientToken string `json:"clientToken"`
ClientAddress string `json:"clientAddress"`
StartTime time.Time `json:"startTime"`
}
// Owner - bucket owner/principal
type Owner struct {
ID string
DisplayName string
// HealTaskStatus - status struct for a heal task
type HealTaskStatus struct {
Summary string `json:"summary"`
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
type Bucket struct {
Name 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 - specify the type of heal operation in a healing
// result
type HealItemType string
// HealItemType constants
const (
// Healthy - Object that is already healthy
Healthy HealStatus = iota
// CanHeal - Object can be healed
CanHeal
// 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
HealItemMetadata HealItemType = "metadata"
HealItemBucket = "bucket"
HealItemBucketMetadata = "bucket-metadata"
HealItemObject = "object"
)
// HealBucketInfo - represents healing related information of a bucket.
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
// Drive state constants
const (
healBucket healQueryKey = "bucket"
healObject healQueryKey = "object"
healPrefix healQueryKey = "prefix"
healMarker healQueryKey = "marker"
healDelimiter healQueryKey = "delimiter"
healMaxKey healQueryKey = "max-key"
healDryRun healQueryKey = "dry-run"
DriveStateOk string = "ok"
DriveStateOffline = "offline"
DriveStateCorrupt = "corrupt"
DriveStateMissing = "missing"
)
// mkHealQueryVal - helper function to construct heal REST API query params.
func mkHealQueryVal(bucket, prefix, marker, delimiter, maxKeyStr string) url.Values {
queryVal := make(url.Values)
queryVal.Set("heal", "")
queryVal.Set(string(healBucket), bucket)
queryVal.Set(string(healPrefix), prefix)
queryVal.Set(string(healMarker), marker)
queryVal.Set(string(healDelimiter), delimiter)
queryVal.Set(string(healMaxKey), maxKeyStr)
return queryVal
// HealResultItem - struct for an individual heal result item
type HealResultItem struct {
ResultIndex int64 `json:"resultId"`
Type HealItemType `json:"type"`
Bucket string `json:"bucket"`
Object string `json:"object"`
Detail string `json:"detail"`
ParityBlocks int `json:"parityBlocks,omitempty"`
DataBlocks int `json:"dataBlocks,omitempty"`
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.
func (adm *AdminClient) listObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (listBucketHealResult, error) {
// Construct query params.
maxKeyStr := fmt.Sprintf("%d", maxKeys)
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
// InitDrives - initialize maps used to represent drive info
func (hri *HealResultItem) InitDrives() {
hri.DriveInfo.Before = make(map[string]string)
hri.DriveInfo.After = make(map[string]string)
}
// ListObjectsHeal - Lists upto maxKeys objects that needing heal matching bucket, prefix, marker, delimiter.
func (adm *AdminClient) ListObjectsHeal(bucket, prefix string, recursive bool, doneCh <-chan struct{}) (<-chan ObjectInfo, error) {
// Allocate new list objects channel.
objectStatCh := make(chan ObjectInfo, 1)
// Default listing is delimited at "/"
delimiter := "/"
if recursive {
// If recursive we do not delimit.
delimiter = ""
// GetOnlineCounts - returns the number of online disks before and
// after heal
func (hri *HealResultItem) GetOnlineCounts() (b, a int) {
if hri == nil {
return
}
// Initiate list objects goroutine here.
go func(objectStatCh chan<- ObjectInfo) {
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
}
for _, v := range hri.DriveInfo.Before {
if v == DriveStateOk {
b++
}
}(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,
}
// Execute GET on /?heal to list objects needing heal.
resp, err := adm.executeMethod("GET", reqData)
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
for _, v := range hri.DriveInfo.After {
if v == DriveStateOk {
a++
}
bucketsToBeHealed = append(bucketsToBeHealed,
BucketInfo{
Name: bucket.Name,
Created: creationDate,
HealBucketInfo: bucket.HealBucketInfo,
})
}
return bucketsToBeHealed, nil
return
}
// HealBucket - Heal the given bucket
func (adm *AdminClient) HealBucket(bucket string, dryrun bool) error {
// Construct query params.
queryVal := url.Values{}
queryVal.Set("heal", "")
queryVal.Set(string(healBucket), bucket)
if dryrun {
queryVal.Set(string(healDryRun), "")
// Heal - API endpoint to start heal and to fetch status
func (adm *AdminClient) Heal(bucket, prefix string, healOpts HealOpts,
clientToken string, forceStart bool) (
healStart HealStartSuccess, healTaskStatus HealTaskStatus, err error) {
body, err := json.Marshal(healOpts)
if err != nil {
return healStart, healTaskStatus, err
}
hdrs := make(http.Header)
hdrs.Set(minioAdminOpHeader, "bucket")
reqData := requestData{
queryValues: queryVal,
customHeaders: hdrs,
path := fmt.Sprintf("/v1/heal/%s", bucket)
if bucket != "" && prefix != "" {
path += "/" + prefix
}
// Execute POST on /?heal&bucket=mybucket to heal a bucket.
resp, err := adm.executeMethod("POST", reqData)
// execute POST request to heal api
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)
if err != nil {
return err
return healStart, healTaskStatus, err
}
if resp.StatusCode != http.StatusOK {
return httpRespToErrorResponse(resp)
return healStart, healTaskStatus, httpRespToErrorResponse(resp)
}
return nil
}
// 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)
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return HealResult{}, err
return healStart, healTaskStatus, err
}
if resp.StatusCode != http.StatusOK {
return HealResult{}, httpRespToErrorResponse(resp)
// Was it a status request?
if clientToken == "" {
err = json.Unmarshal(respBytes, &healStart)
} else {
err = json.Unmarshal(respBytes, &healTaskStatus)
}
// 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
return healStart, healTaskStatus, err
}

View file

@ -21,7 +21,6 @@ import (
"encoding/json"
"io/ioutil"
"net/http"
"net/url"
"time"
)
@ -115,13 +114,7 @@ type ServerInfo struct {
// ServerInfo - Connect to a minio server and call Server Info Management API
// to fetch server's information represented by ServerInfo structure
func (adm *AdminClient) ServerInfo() ([]ServerInfo, error) {
// Prepare web service request
reqData := requestData{}
reqData.queryValues = make(url.Values)
reqData.queryValues.Set("info", "")
reqData.customHeaders = make(http.Header)
resp, err := adm.executeMethod("GET", reqData)
resp, err := adm.executeMethod("GET", requestData{relPath: "/v1/info"})
defer closeResponse(resp)
if err != nil {
return nil, err

View file

@ -82,24 +82,19 @@ func getLockInfos(body io.Reader) ([]VolumeLockInfo, error) {
// ListLocks - Calls List Locks Management API to fetch locks matching
// 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.Set("lock", "")
queryVal.Set("bucket", bucket)
queryVal.Set("prefix", prefix)
queryVal.Set("duration", 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)
queryVal.Set("older-than", duration.String())
// Execute GET on /minio/admin/v1/locks to list locks.
resp, err := adm.executeMethod("GET", requestData{
queryValues: queryVal,
relPath: "/v1/locks",
})
defer closeResponse(resp)
if err != nil {
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
// 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.Set("lock", "")
queryVal.Set("bucket", bucket)
queryVal.Set("prefix", prefix)
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.
resp, err := adm.executeMethod("POST", reqData)
resp, err := adm.executeMethod("DELETE", requestData{
queryValues: queryVal,
relPath: "/v1/locks",
})
defer closeResponse(resp)
if err != nil {
return nil, err

View file

@ -18,69 +18,79 @@
package madmin
import (
"bytes"
"encoding/json"
"io/ioutil"
"net/http"
"net/url"
"time"
)
// ServiceStatusMetadata - contains the response of service status API
type ServiceStatusMetadata struct {
Uptime time.Duration `json:"uptime"`
// ServerVersion - server version
type ServerVersion struct {
Version string `json:"version"`
CommitID string `json:"commitID"`
}
// ServiceStatus - Connect to a minio server and call Service Status Management API
// to fetch server's storage information represented by ServiceStatusMetadata structure
func (adm *AdminClient) ServiceStatus() (ServiceStatusMetadata, error) {
// ServiceStatus - contains the response of service status API
type ServiceStatus struct {
ServerVersion ServerVersion `json:"serverVersion"`
Uptime time.Duration `json:"uptime"`
}
// Prepare web service request
reqData := requestData{}
reqData.queryValues = make(url.Values)
reqData.queryValues.Set("service", "")
reqData.customHeaders = make(http.Header)
reqData.customHeaders.Set(minioAdminOpHeader, "status")
// Execute GET on bucket to list objects.
resp, err := adm.executeMethod("GET", reqData)
// ServiceStatus - Connect to a minio server and call Service Status
// Management API to fetch server's storage information represented by
// ServiceStatusMetadata structure
func (adm *AdminClient) ServiceStatus() (ss ServiceStatus, err error) {
// Request API to GET service status
resp, err := adm.executeMethod("GET", requestData{relPath: "/v1/service"})
defer closeResponse(resp)
if err != nil {
return ServiceStatusMetadata{}, err
return ss, err
}
// Check response http status code
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)
if err != nil {
return ServiceStatusMetadata{}, err
return ss, err
}
err = json.Unmarshal(respBytes, &serviceStatus)
if err != nil {
return ServiceStatusMetadata{}, err
}
return serviceStatus, nil
err = json.Unmarshal(respBytes, &ss)
return ss, err
}
// ServiceRestart - Call Service Restart API to restart a specified Minio server
func (adm *AdminClient) ServiceRestart() error {
//
reqData := requestData{}
reqData.queryValues = make(url.Values)
reqData.queryValues.Set("service", "")
reqData.customHeaders = make(http.Header)
reqData.customHeaders.Set(minioAdminOpHeader, "restart")
// ServiceActionValue - type to restrict service-action values
type ServiceActionValue string
// Execute GET on bucket to list objects.
resp, err := adm.executeMethod("POST", reqData)
const (
// 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)
if err != nil {
return err

View file

@ -18,7 +18,7 @@ package madmin
import (
"crypto/md5"
"encoding/xml"
"encoding/json"
"io"
"io/ioutil"
"net"
@ -45,9 +45,9 @@ func sumMD5(data []byte) []byte {
return hash.Sum(nil)
}
// xmlDecoder provide decoded value in xml.
func xmlDecoder(body io.Reader, v interface{}) error {
d := xml.NewDecoder(body)
// jsonDecoder decode json to go type.
func jsonDecoder(body io.Reader, v interface{}) error {
d := json.NewDecoder(body)
return d.Decode(v)
}

View 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
}