Add large bucket support for erasure coded backend (#5160)

This PR implements an object layer which
combines input erasure sets of XL layers
into a unified namespace.

This object layer extends the existing
erasure coded implementation, it is assumed
in this design that providing > 16 disks is
a static configuration as well i.e if you started
the setup with 32 disks with 4 sets 8 disks per
pack then you would need to provide 4 sets always.

Some design details and restrictions:

- Objects are distributed using consistent ordering
  to a unique erasure coded layer.
- Each pack has its own dsync so locks are synchronized
  properly at pack (erasure layer).
- Each pack still has a maximum of 16 disks
  requirement, you can start with multiple
  such sets statically.
- Static sets set of disks and cannot be
  changed, there is no elastic expansion allowed.
- Static sets set of disks and cannot be
  changed, there is no elastic removal allowed.
- ListObjects() across sets can be noticeably
  slower since List happens on all servers,
  and is merged at this sets layer.

Fixes #5465
Fixes #5464
Fixes #5461
Fixes #5460
Fixes #5459
Fixes #5458
Fixes #5460
Fixes #5488
Fixes #5489
Fixes #5497
Fixes #5496
This commit is contained in:
Harshavardhana
2018-02-15 17:45:57 -08:00
committed by kannappanr
parent dd80256151
commit fb96779a8a
82 changed files with 5046 additions and 4771 deletions

View File

@@ -52,14 +52,15 @@ const (
var (
// This struct literal represents the Admin API version that
// the server uses.
adminAPIVersionInfo = madmin.AdminAPIVersionInfo{"1"}
adminAPIVersionInfo = madmin.AdminAPIVersionInfo{
Version: "1",
}
)
// VersionHandler - GET /minio/admin/version
// -----------
// Returns Administration API version
func (a adminAPIHandlers) VersionHandler(w http.ResponseWriter, r *http.Request) {
adminAPIErr := checkAdminRequestAuthType(r, globalServerConfig.GetRegion())
if adminAPIErr != ErrNone {
writeErrorResponse(w, adminAPIErr, r.URL)
@@ -555,7 +556,6 @@ func (a adminAPIHandlers) HealHandler(w http.ResponseWriter, r *http.Request) {
writeSuccessResponseJSON(w, respBytes)
}
}
return
}
// GetConfigHandler - GET /minio/admin/v1/config

View File

@@ -253,10 +253,22 @@ func (atb *adminXLTestBed) CleanupHealTestData(t *testing.T) {
// initTestObjLayer - Helper function to initialize an XL-based object
// layer and set globalObjectAPI.
func initTestXLObjLayer() (ObjectLayer, []string, error) {
objLayer, xlDirs, xlErr := prepareXL16()
if xlErr != nil {
return nil, nil, xlErr
xlDirs, err := getRandomDisks(16)
if err != nil {
return nil, nil, err
}
endpoints := mustGetNewEndpointList(xlDirs...)
format, err := waitForFormatXL(true, endpoints, 1, 16)
if err != nil {
removeRoots(xlDirs)
return nil, nil, err
}
objLayer, err := newXLSets(endpoints, format, 1, 16)
if err != nil {
return nil, nil, err
}
// Make objLayer available to all internal services via globalObjectAPI.
globalObjLayerMutex.Lock()
globalObjectAPI = objLayer

View File

@@ -528,58 +528,15 @@ func (h *healSequence) healDiskFormat() error {
return errServerNotInitialized
}
// Acquire lock on format.json
formatLock := globalNSMutex.NewNSLock(minioMetaBucket, formatConfigFile)
if err := formatLock.GetLock(globalHealingTimeout); err != nil {
return errFnHealFromAPIErr(err)
}
defer formatLock.Unlock()
// Create a new set of storage instances to heal format.json.
bootstrapDisks, err := initStorageDisks(globalEndpoints)
res, err := objectAPI.HealFormat(h.settings.DryRun)
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)
}
peersReInitFormat(globalAdminPeers, h.settings.DryRun)
// Push format heal result
return h.pushHealResultItem(hres)
return h.pushHealResultItem(res)
}
// healBuckets - check for all buckets heal or just particular bucket.
@@ -601,8 +558,7 @@ func (h *healSequence) healBuckets() error {
}
for _, bucket := range buckets {
err = h.healBucket(bucket.Name)
if err != nil {
if err = h.healBucket(bucket.Name); err != nil {
return err
}
}
@@ -615,19 +571,27 @@ func (h *healSequence) healBucket(bucket string) error {
if h.isQuitting() {
return errHealStopSignalled
}
// Get current object layer instance.
objectAPI := newObjectLayerFn()
if objectAPI == nil {
return errServerNotInitialized
}
bucketLock := globalNSMutex.NewNSLock(bucket, "")
if err := bucketLock.GetLock(globalHealingTimeout); err != nil {
return err
}
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 {
bucketLock.Unlock()
return perr
}
}
bucketLock.Unlock()
// handle heal-bucket error
if err != nil {
return err

View File

@@ -34,8 +34,8 @@ import (
const (
// Admin service names
signalServiceRPC = "Admin.SignalService"
reInitFormatRPC = "Admin.ReInitFormat"
listLocksRPC = "Admin.ListLocks"
reInitDisksRPC = "Admin.ReInitDisks"
serverInfoDataRPC = "Admin.ServerInfoData"
getConfigRPC = "Admin.GetConfig"
writeTmpConfigRPC = "Admin.WriteTmpConfig"
@@ -56,8 +56,8 @@ type remoteAdminClient struct {
// commands like service stop and service restart.
type adminCmdRunner interface {
SignalService(s serviceSignal) error
ReInitFormat(dryRun bool) error
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
ReInitDisks() error
ServerInfoData() (ServerInfoData, error)
GetConfig() ([]byte, error)
WriteTmpConfig(tmpFileName string, configBytes []byte) error
@@ -77,6 +77,16 @@ func (lc localAdminClient) SignalService(s serviceSignal) error {
return nil
}
// ReInitFormat - re-initialize disk format.
func (lc localAdminClient) ReInitFormat(dryRun bool) error {
objectAPI := newObjectLayerFn()
if objectAPI == nil {
return errServerNotInitialized
}
_, err := objectAPI.HealFormat(dryRun)
return err
}
// ListLocks - Fetches lock information from local lock instrumentation.
func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
return listLocksInfo(bucket, prefix, duration), nil
@@ -92,7 +102,14 @@ func (rc remoteAdminClient) SignalService(s serviceSignal) (err error) {
err = errUnsupportedSignal
}
return err
}
// ReInitFormat - re-initialize disk format, remotely.
func (rc remoteAdminClient) ReInitFormat(dryRun bool) error {
reply := AuthRPCReply{}
return rc.Call(reInitFormatRPC, &ReInitFormatArgs{
DryRun: dryRun,
}, &reply)
}
// ListLocks - Sends list locks command to remote server via RPC.
@@ -109,20 +126,6 @@ func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Durat
return reply.VolLocks, nil
}
// ReInitDisks - There is nothing to do here, heal format REST API
// handler has already formatted and reinitialized the local disks.
func (lc localAdminClient) ReInitDisks() error {
return nil
}
// ReInitDisks - Signals peers via RPC to reinitialize their disks and
// object layer.
func (rc remoteAdminClient) ReInitDisks() error {
args := AuthRPCArgs{}
reply := AuthRPCReply{}
return rc.Call(reInitDisksRPC, &args, &reply)
}
// ServerInfoData - Returns the server info of this server.
func (lc localAdminClient) ServerInfoData() (sid ServerInfoData, e error) {
if globalBootTime.IsZero() {
@@ -240,6 +243,7 @@ func (rc remoteAdminClient) CommitConfig(tmpFileName string) error {
type adminPeer struct {
addr string
cmdRunner adminCmdRunner
isLocal bool
}
// type alias for a collection of adminPeer.
@@ -254,6 +258,7 @@ func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
adminPeerList = append(adminPeerList, adminPeer{
thisPeer,
localAdminClient{},
true,
})
hostSet := set.CreateStringSet(globalMinioAddr)
@@ -280,6 +285,26 @@ func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
return adminPeerList
}
// peersReInitFormat - reinitialize remote object layers to new format.
func peersReInitFormat(peers adminPeers, dryRun bool) error {
errs := make([]error, len(peers))
// Send ReInitFormat RPC call to all nodes.
// for local adminPeer this is a no-op.
wg := sync.WaitGroup{}
for i, peer := range peers {
wg.Add(1)
go func(idx int, peer adminPeer) {
defer wg.Done()
if !peer.isLocal {
errs[idx] = peer.cmdRunner.ReInitFormat(dryRun)
}
}(i, peer)
}
wg.Wait()
return nil
}
// Initialize global adminPeer collection.
func initGlobalAdminPeers(endpoints EndpointList) {
globalAdminPeers = makeAdminPeers(endpoints)
@@ -363,24 +388,6 @@ func listPeerLocksInfo(peers adminPeers, bucket, prefix string, duration time.Du
return groupedLockInfos, nil
}
// reInitPeerDisks - reinitialize disks and object layer on peer servers to use the new format.
func reInitPeerDisks(peers adminPeers) error {
errs := make([]error, len(peers))
// Send ReInitDisks RPC call to all nodes.
// for local adminPeer this is a no-op.
wg := sync.WaitGroup{}
for i, peer := range peers {
wg.Add(1)
go func(idx int, peer adminPeer) {
defer wg.Done()
errs[idx] = peer.cmdRunner.ReInitDisks()
}(i, peer)
}
wg.Wait()
return nil
}
// uptimeSlice - used to sort uptimes in chronological order.
type uptimeSlice []struct {
err error

View File

@@ -30,8 +30,6 @@ import (
const adminPath = "/admin"
var errUnsupportedBackend = fmt.Errorf("not supported for non erasure-code backend")
// adminCmd - exports RPC methods for service status, stop and
// restart commands.
type adminCmd struct {
@@ -80,6 +78,21 @@ func (s *adminCmd) SignalService(args *SignalServiceArgs, reply *AuthRPCReply) e
return nil
}
// ReInitFormatArgs - provides dry-run information to re-initialize format.json
type ReInitFormatArgs struct {
AuthRPCArgs
DryRun bool
}
// ReInitFormat - re-init 'format.json'
func (s *adminCmd) ReInitFormat(args *ReInitFormatArgs, reply *AuthRPCReply) error {
if err := args.IsAuthenticated(); err != nil {
return err
}
_, err := newObjectLayerFn().HealFormat(args.DryRun)
return err
}
// ListLocks - lists locks held by requests handled by this server instance.
func (s *adminCmd) ListLocks(query *ListLocksQuery, reply *ListLocksReply) error {
if err := query.IsAuthenticated(); err != nil {
@@ -90,47 +103,6 @@ func (s *adminCmd) ListLocks(query *ListLocksQuery, reply *ListLocksReply) error
return nil
}
// ReInitDisk - reinitialize storage disks and object layer to use the
// new format.
func (s *adminCmd) ReInitDisks(args *AuthRPCArgs, reply *AuthRPCReply) error {
if err := args.IsAuthenticated(); err != nil {
return err
}
if !globalIsXL {
return errUnsupportedBackend
}
// Get the current object layer instance.
objLayer := newObjectLayerFn()
// Initialize new disks to include the newly formatted disks.
bootstrapDisks, err := initStorageDisks(globalEndpoints)
if err != nil {
return err
}
// Wrap into retrying disks
retryingDisks := initRetryableStorageDisks(bootstrapDisks,
time.Millisecond, time.Millisecond*5, globalStorageHealthCheckInterval, globalStorageRetryThreshold)
// Initialize new object layer with newly formatted disks.
newObjectAPI, err := newXLObjects(retryingDisks)
if err != nil {
return err
}
// Replace object layer with newly formatted storage.
globalObjLayerMutex.Lock()
globalObjectAPI = newObjectAPI
globalObjLayerMutex.Unlock()
// Shutdown storage belonging to old object layer instance.
objLayer.Shutdown()
return nil
}
// ServerInfo - returns the server info when object layer was initialized on this server.
func (s *adminCmd) ServerInfoData(args *AuthRPCArgs, reply *ServerInfoDataReply) error {
if err := args.IsAuthenticated(); err != nil {

View File

@@ -91,8 +91,8 @@ func TestAdminStatus(t *testing.T) {
testAdminCmd(statusCmd, t)
}
// TestReInitDisks - test for Admin.ReInitDisks RPC service.
func TestReInitDisks(t *testing.T) {
// TestReInitFormat - test for Admin.ReInitFormat RPC service.
func TestReInitFormat(t *testing.T) {
// Reset global variables to start afresh.
resetTestGlobals()
@@ -138,40 +138,13 @@ func TestReInitDisks(t *testing.T) {
}
authReply := AuthRPCReply{}
err = adminServer.ReInitDisks(&authArgs, &authReply)
err = adminServer.ReInitFormat(&ReInitFormatArgs{
AuthRPCArgs: authArgs,
DryRun: false,
}, &authReply)
if err != nil {
t.Errorf("Expected to pass, but failed with %v", err)
}
token, err = authenticateNode(creds.AccessKey, creds.SecretKey)
if err != nil {
t.Fatal(err)
}
// Negative test case with admin rpc server setup for FS.
globalIsXL = false
fsAdminServer := adminCmd{}
fsArgs := LoginRPCArgs{
AuthToken: token,
Version: globalRPCAPIVersion,
RequestTime: UTCNow(),
}
fsReply := LoginRPCReply{}
err = fsAdminServer.Login(&fsArgs, &fsReply)
if err != nil {
t.Fatalf("Failed to login to fs admin server - %v", err)
}
authArgs = AuthRPCArgs{
AuthToken: token,
Version: globalRPCAPIVersion,
}
authReply = AuthRPCReply{}
// Attempt ReInitDisks service on a FS backend.
err = fsAdminServer.ReInitDisks(&authArgs, &authReply)
if err != errUnsupportedBackend {
t.Errorf("Expected to fail with %v, but received %v",
errUnsupportedBackend, err)
}
}
// TestGetConfig - Test for GetConfig admin RPC.

View File

@@ -127,6 +127,13 @@ func (api objectAPIHandlers) GetBucketLocationHandler(w http.ResponseWriter, r *
return
}
bucketLock := globalNSMutex.NewNSLock(bucket, "")
if err := bucketLock.GetRLock(globalObjectTimeout); err != nil {
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return
}
defer bucketLock.RUnlock()
if _, err := objectAPI.GetBucketInfo(bucket); err != nil {
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return
@@ -397,6 +404,13 @@ func (api objectAPIHandlers) PutBucketHandler(w http.ResponseWriter, r *http.Req
return
}
bucketLock := globalNSMutex.NewNSLock(bucket, "")
if err := bucketLock.GetLock(globalObjectTimeout); err != nil {
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return
}
defer bucketLock.Unlock()
// Proceed to creating a bucket.
err := objectAPI.MakeBucketWithLocation(bucket, "")
if err != nil {

View File

@@ -35,7 +35,6 @@ func TestGetBucketLocationHandler(t *testing.T) {
func testGetBucketLocationHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
initBucketPolicies(obj)
// test cases with sample input and expected output.
testCases := []struct {
@@ -180,7 +179,6 @@ func TestHeadBucketHandler(t *testing.T) {
func testHeadBucketHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
initBucketPolicies(obj)
// test cases with sample input and expected output.
testCases := []struct {
@@ -287,7 +285,6 @@ func TestListMultipartUploadsHandler(t *testing.T) {
// testListMultipartUploadsHandler - Tests validate listing of multipart uploads.
func testListMultipartUploadsHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
initBucketPolicies(obj)
// Collection of non-exhaustive ListMultipartUploads test cases, valid errors
// and success responses.
@@ -618,7 +615,6 @@ func TestAPIDeleteMultipleObjectsHandler(t *testing.T) {
func testAPIDeleteMultipleObjectsHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
initBucketPolicies(obj)
var err error
// register event notifier.
@@ -822,7 +818,6 @@ func testIsBucketActionAllowedHandler(obj ObjectLayer, instanceType, bucketName
{"s3:ListObject", "mybucket", "abc", false, false},
}
for i, testCase := range testCases {
initBucketPolicies(obj)
isAllowed := isBucketActionAllowed(testCase.action, testCase.bucket, testCase.prefix, obj)
if isAllowed != testCase.shouldPass {
t.Errorf("Case %d: Expected the response status to be `%t`, but instead found `%t`", i+1, testCase.shouldPass, isAllowed)

View File

@@ -223,7 +223,7 @@ func TestSendNotificationEvent(t *testing.T) {
go func() {
// Send one empty notification event on listenChan
events := []NotificationEvent{NotificationEvent{}}
events := []NotificationEvent{{}}
l.sendNotificationEvent(events)
testCh <- struct{}{}
}()

View File

@@ -250,7 +250,6 @@ func TestPutBucketPolicyHandler(t *testing.T) {
// testPutBucketPolicyHandler - Test for Bucket policy end point.
func testPutBucketPolicyHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
initBucketPolicies(obj)
bucketName1 := fmt.Sprintf("%s-1", bucketName)
if err := obj.MakeBucketWithLocation(bucketName1, ""); err != nil {
@@ -458,9 +457,6 @@ func TestGetBucketPolicyHandler(t *testing.T) {
// testGetBucketPolicyHandler - Test for end point which fetches the access policy json of the given bucket.
func testGetBucketPolicyHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
// initialize bucket policy.
initBucketPolicies(obj)
// template for constructing HTTP request body for PUT bucket policy.
bucketPolicyTemplate := `{"Version":"2012-10-17","Statement":[{"Action":["s3:GetBucketLocation","s3:ListBucket"],"Effect":"Allow","Principal":{"AWS":["*"]},"Resource":["arn:aws:s3:::%s"],"Sid":""},{"Action":["s3:GetObject"],"Effect":"Allow","Principal":{"AWS":["*"]},"Resource":["arn:aws:s3:::%s/this*"],"Sid":""}]}`
@@ -647,8 +643,6 @@ func TestDeleteBucketPolicyHandler(t *testing.T) {
// testDeleteBucketPolicyHandler - Test for Delete bucket policy end point.
func testDeleteBucketPolicyHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler,
credentials auth.Credentials, t *testing.T) {
// initialize bucket policy.
initBucketPolicies(obj)
// template for constructing HTTP request body for PUT bucket policy.
bucketPolicyTemplate := `{

View File

@@ -76,14 +76,15 @@ func (bp *bucketPolicies) DeleteBucketPolicy(bucket string) error {
}
// Intialize all bucket policies.
func initBucketPolicies(objAPI ObjectLayer) error {
func initBucketPolicies(objAPI ObjectLayer) (*bucketPolicies, error) {
if objAPI == nil {
return errInvalidArgument
return nil, errInvalidArgument
}
// List buckets to proceed loading all notification configuration.
buckets, err := objAPI.ListBuckets()
if err != nil {
return errors.Cause(err)
return nil, errors.Cause(err)
}
policies := make(map[string]policy.BucketAccessPolicy)
@@ -95,7 +96,7 @@ func initBucketPolicies(objAPI ObjectLayer) error {
// other unexpected errors during net.Dial.
if !errors.IsErrIgnored(pErr, errDiskNotFound) {
if !isErrBucketPolicyNotFound(pErr) {
return errors.Cause(pErr)
return nil, errors.Cause(pErr)
}
}
// Continue to load other bucket policies if possible.
@@ -103,20 +104,12 @@ func initBucketPolicies(objAPI ObjectLayer) error {
}
policies[bucket.Name] = bp
}
// Populate global bucket collection.
bPolicies := &bucketPolicies{
// Return all bucket policies.
return &bucketPolicies{
rwMutex: &sync.RWMutex{},
bucketPolicyConfigs: policies,
}
switch objAPI.(type) {
case *fsObjects:
objAPI.(*fsObjects).bucketPolicies = bPolicies
case *xlObjects:
objAPI.(*xlObjects).bucketPolicies = bPolicies
}
// Success.
return nil
}, nil
}
// readBucketPolicyJSON - reads bucket policy for an input bucket, returns BucketPolicyNotFound

251
cmd/endpoint-ellipses.go Normal file
View File

@@ -0,0 +1,251 @@
/*
* Minio Cloud Storage, (C) 2018 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 (
"fmt"
"strings"
"github.com/minio/minio-go/pkg/set"
"github.com/minio/minio/pkg/ellipses"
)
// This file implements and supports ellipses pattern for
// `minio server` command line arguments.
// Maximum number of unique args supported on the command line.
const (
serverCommandLineArgsMax = 32
)
// Endpoint set represents parsed ellipses values, also provides
// methods to get the sets of endpoints.
type endpointSet struct {
argPatterns []ellipses.ArgPattern
endpoints []string // Endpoints saved from previous GetEndpoints().
setIndexes [][]uint64 // All the sets.
}
// Supported set sizes this is used to find the optimal
// single set size.
var setSizes = []uint64{4, 6, 8, 10, 12, 14, 16}
// getDivisibleSize - returns a greatest common divisor of
// all the ellipses sizes.
func getDivisibleSize(totalSizes []uint64) (result uint64) {
gcd := func(x, y uint64) uint64 {
for y != 0 {
x, y = y, x%y
}
return x
}
result = totalSizes[0]
for i := 1; i < len(totalSizes); i++ {
result = gcd(result, totalSizes[i])
}
return result
}
// getSetIndexes returns list of indexes which provides the set size
// on each index, this function also determines the final set size
// The final set size has the affinity towards choosing smaller
// indexes (total sets)
func getSetIndexes(args []string, totalSizes []uint64) (setIndexes [][]uint64, err error) {
if len(totalSizes) == 0 || len(args) == 0 {
return nil, errInvalidArgument
}
setIndexes = make([][]uint64, len(totalSizes))
for i, totalSize := range totalSizes {
// Check if totalSize has minimum range upto setSize
if totalSize < setSizes[0] {
return nil, fmt.Errorf("Invalid inputs (%s). Ellipses range or number of args %d should be atleast divisible by least possible set size %d",
args[i], totalSize, setSizes[0])
}
}
var setSize uint64
commonSize := getDivisibleSize(totalSizes)
if commonSize > setSizes[len(setSizes)-1] {
prevD := commonSize / setSizes[0]
for _, i := range setSizes {
if commonSize%i == 0 {
d := commonSize / i
if d <= prevD {
prevD = d
setSize = i
}
}
}
} else {
setSize = commonSize
}
// isValidSetSize - checks whether given count is a valid set size for erasure coding.
isValidSetSize := func(count uint64) bool {
return (count >= setSizes[0] && count <= setSizes[len(setSizes)-1] && count%2 == 0)
}
// Check whether setSize is with the supported range.
if !isValidSetSize(setSize) {
return nil, fmt.Errorf("Invalid inputs (%s). Ellipses range or number of args %d should be atleast divisible by least possible set size %d",
args, setSize, setSizes[0])
}
for i := range totalSizes {
for j := uint64(0); j < totalSizes[i]/setSize; j++ {
setIndexes[i] = append(setIndexes[i], setSize)
}
}
return setIndexes, nil
}
// Returns all the expanded endpoints, each argument is expanded separately.
func (s endpointSet) getEndpoints() (endpoints []string) {
if len(s.endpoints) != 0 {
return s.endpoints
}
for _, argPattern := range s.argPatterns {
for _, lbls := range argPattern.Expand() {
endpoints = append(endpoints, strings.Join(lbls, ""))
}
}
s.endpoints = endpoints
return endpoints
}
// Get returns the sets representation of the endpoints
// this function also intelligently decides on what will
// be the right set size etc.
func (s endpointSet) Get() (sets [][]string) {
var k = uint64(0)
endpoints := s.getEndpoints()
for i := range s.setIndexes {
for j := range s.setIndexes[i] {
sets = append(sets, endpoints[k:s.setIndexes[i][j]+k])
k = s.setIndexes[i][j] + k
}
}
return sets
}
// Return the total size for each argument patterns.
func getTotalSizes(argPatterns []ellipses.ArgPattern) []uint64 {
var totalSizes []uint64
for _, argPattern := range argPatterns {
var totalSize uint64 = 1
for _, p := range argPattern {
totalSize = totalSize * uint64(len(p.Seq))
}
totalSizes = append(totalSizes, totalSize)
}
return totalSizes
}
// Parses all arguments and returns an endpointSet which is a collection
// of endpoints following the ellipses pattern, this is what is used
// by the object layer for initializing itself.
func parseEndpointSet(args ...string) (ep endpointSet, err error) {
var argPatterns = make([]ellipses.ArgPattern, len(args))
for i, arg := range args {
patterns, err := ellipses.FindEllipsesPatterns(arg)
if err != nil {
return endpointSet{}, err
}
argPatterns[i] = patterns
}
ep.setIndexes, err = getSetIndexes(args, getTotalSizes(argPatterns))
if err != nil {
return endpointSet{}, err
}
ep.argPatterns = argPatterns
return ep, nil
}
// Parses all ellipses input arguments, expands them into corresponding
// list of endpoints chunked evenly in accordance with a specific
// set size.
// For example: {1...64} is divided into 4 sets each of size 16.
// This applies to even distributed setup syntax as well.
func getAllSets(args ...string) ([][]string, error) {
if len(args) == 0 {
return nil, errInvalidArgument
}
var setArgs [][]string
if !ellipses.HasEllipses(args...) {
var setIndexes [][]uint64
// Check if we have more one args.
if len(args) > 1 {
var err error
setIndexes, err = getSetIndexes(args, []uint64{uint64(len(args))})
if err != nil {
return nil, err
}
} else {
// We are in FS setup, proceed forward.
setIndexes = [][]uint64{[]uint64{uint64(len(args))}}
}
s := endpointSet{
endpoints: args,
setIndexes: setIndexes,
}
setArgs = s.Get()
} else {
s, err := parseEndpointSet(args...)
if err != nil {
return nil, err
}
setArgs = s.Get()
}
uniqueArgs := set.NewStringSet()
for _, sargs := range setArgs {
for _, arg := range sargs {
if uniqueArgs.Contains(arg) {
return nil, fmt.Errorf("Input args (%s) has duplicate ellipses", args)
}
uniqueArgs.Add(arg)
}
}
return setArgs, nil
}
// CreateServerEndpoints - validates and creates new endpoints from input args, supports
// both ellipses and without ellipses transparently.
func createServerEndpoints(serverAddr string, args ...string) (string, EndpointList, SetupType, int, int, error) {
setArgs, err := getAllSets(args...)
if err != nil {
return serverAddr, nil, -1, 0, 0, err
}
var endpoints EndpointList
var setupType SetupType
serverAddr, endpoints, setupType, err = CreateEndpoints(serverAddr, setArgs...)
if err != nil {
return serverAddr, nil, -1, 0, 0, err
}
return serverAddr, endpoints, setupType, len(setArgs), len(setArgs[0]), nil
}

View File

@@ -0,0 +1,388 @@
/*
* Minio Cloud Storage, (C) 2018 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 (
"fmt"
"reflect"
"testing"
"github.com/minio/minio/pkg/ellipses"
)
// Tests create endpoints with ellipses and without.
func TestCreateServerEndpoints(t *testing.T) {
testCases := []struct {
serverAddr string
args []string
success bool
}{
// Invalid input.
{"", []string{}, false},
// Range cannot be negative.
{":9000", []string{"/export1{-1...1}"}, false},
// Range cannot start bigger than end.
{":9000", []string{"/export1{64...1}"}, false},
// Range can only be numeric.
{":9000", []string{"/export1{a...z}"}, false},
// Duplicate disks not allowed.
{":9000", []string{"/export1{1...32}", "/export1{1...32}"}, false},
// Same host cannot export same disk on two ports - special case localhost.
{":9001", []string{"http://localhost:900{1...2}/export{1...64}"}, false},
// Valid inputs.
{":9000", []string{"/export1"}, true},
{":9000", []string{"/export1", "/export2", "/export3", "/export4"}, true},
{":9000", []string{"/export1{1...64}"}, true},
{":9000", []string{"/export1{01...64}"}, true},
{":9000", []string{"/export1{1...32}", "/export1{33...64}"}, true},
{":9001", []string{"http://localhost:9001/export{1...64}"}, true},
{":9001", []string{"http://localhost:9001/export{01...64}"}, true},
}
for i, testCase := range testCases {
_, _, _, _, _, err := createServerEndpoints(testCase.serverAddr, testCase.args...)
if err != nil && testCase.success {
t.Errorf("Test %d: Expected success but failed instead %s", i+1, err)
}
if err == nil && !testCase.success {
t.Errorf("Test %d: Expected failure but passed instead", i+1)
}
}
}
// Test tests calculating set indexes.
func TestGetSetIndexes(t *testing.T) {
testCases := []struct {
args []string
totalSizes []uint64
indexes [][]uint64
success bool
}{
// Invalid inputs.
{
[]string{"data{1...27}"},
[]uint64{27},
nil,
false,
},
// Valid inputs.
{
[]string{"data{1...64}"},
[]uint64{64},
[][]uint64{[]uint64{16, 16, 16, 16}},
true,
},
{
[]string{"data{1...24}"},
[]uint64{24},
[][]uint64{[]uint64{12, 12}},
true,
},
{
[]string{"data/controller{1...11}/export{1...8}"},
[]uint64{88},
[][]uint64{[]uint64{8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}},
true,
},
{
[]string{"data{1...4}"},
[]uint64{4},
[][]uint64{[]uint64{4}},
true,
},
}
for i, testCase := range testCases {
t.Run(fmt.Sprintf("Test%d", i+1), func(t *testing.T) {
gotIndexes, err := getSetIndexes(testCase.args, testCase.totalSizes)
if err != nil && testCase.success {
t.Errorf("Expected success but failed instead %s", err)
}
if err == nil && !testCase.success {
t.Errorf("Expected failure but passed instead")
}
if !reflect.DeepEqual(testCase.indexes, gotIndexes) {
t.Errorf("Expected %v, got %v", testCase.indexes, gotIndexes)
}
})
}
}
func getSequences(start int, number int, paddinglen int) (seq []string) {
for i := start; i <= number; i++ {
if paddinglen == 0 {
seq = append(seq, fmt.Sprintf("%d", i))
} else {
seq = append(seq, fmt.Sprintf(fmt.Sprintf("%%0%dd", paddinglen), i))
}
}
return seq
}
// Test tests parses endpoint ellipses input pattern.
func TestParseEndpointSet(t *testing.T) {
testCases := []struct {
arg string
es endpointSet
success bool
}{
// Tests invalid inputs.
{
"...",
endpointSet{},
false,
},
// Indivisible range.
{
"{1...27}",
endpointSet{},
false,
},
// No range specified.
{
"{...}",
endpointSet{},
false,
},
// Invalid range.
{
"http://minio{2...3}/export/set{1...0}",
endpointSet{},
false,
},
// Range cannot be smaller than 4 minimum.
{
"/export{1..2}",
endpointSet{},
false,
},
// Unsupported characters.
{
"/export/test{1...2O}",
endpointSet{},
false,
},
// Tests valid inputs.
{
"/export/set{1...64}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"/export/set",
"",
getSequences(1, 64, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16, 16, 16}},
},
true,
},
// Valid input for distributed setup.
{
"http://minio{2...3}/export/set{1...64}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"",
"",
getSequences(1, 64, 0),
},
{
"http://minio",
"/export/set",
getSequences(2, 3, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16, 16, 16, 16, 16, 16, 16}},
},
true,
},
// Supporting some advanced cases.
{
"http://minio{1...64}.mydomain.net/data",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"http://minio",
".mydomain.net/data",
getSequences(1, 64, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16, 16, 16}},
},
true,
},
{
"http://rack{1...4}.mydomain.minio{1...16}/data",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"",
"/data",
getSequences(1, 16, 0),
},
{
"http://rack",
".mydomain.minio",
getSequences(1, 4, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16, 16, 16}},
},
true,
},
// Supporting kubernetes cases.
{
"http://minio{0...15}.mydomain.net/data{0...1}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"",
"",
getSequences(0, 1, 0),
},
{
"http://minio",
".mydomain.net/data",
getSequences(0, 15, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16}},
},
true,
},
// No host regex, just disks.
{
"http://server1/data{1...32}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"http://server1/data",
"",
getSequences(1, 32, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16}},
},
true,
},
// No host regex, just disks with two position numerics.
{
"http://server1/data{01...32}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"http://server1/data",
"",
getSequences(1, 32, 2),
},
},
},
nil,
[][]uint64{[]uint64{16, 16}},
},
true,
},
// More than 2 ellipses are supported as well.
{
"http://minio{2...3}/export/set{1...64}/test{1...2}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"",
"",
getSequences(1, 2, 0),
},
{
"",
"/test",
getSequences(1, 64, 0),
},
{
"http://minio",
"/export/set",
getSequences(2, 3, 0),
},
},
},
nil,
[][]uint64{[]uint64{16, 16, 16, 16, 16, 16, 16, 16,
16, 16, 16, 16, 16, 16, 16, 16}},
},
true,
},
// More than 1 ellipses per argument for standalone setup.
{
"/export{1...10}/disk{1...10}",
endpointSet{
[]ellipses.ArgPattern{
[]ellipses.Pattern{
{
"",
"",
getSequences(1, 10, 0),
},
{
"/export",
"/disk",
getSequences(1, 10, 0),
},
},
},
nil,
[][]uint64{[]uint64{10, 10, 10, 10, 10, 10, 10, 10, 10, 10}},
},
true,
},
}
for i, testCase := range testCases {
t.Run(fmt.Sprintf("Test%d", i+1), func(t *testing.T) {
gotEs, err := parseEndpointSet(testCase.arg)
if err != nil && testCase.success {
t.Errorf("Expected success but failed instead %s", err)
}
if err == nil && !testCase.success {
t.Errorf("Expected failure but passed instead")
}
if !reflect.DeepEqual(testCase.es, gotEs) {
t.Errorf("Expected %v, got %v", testCase.es, gotEs)
}
})
}
}

View File

@@ -23,7 +23,6 @@ import (
"path"
"path/filepath"
"runtime"
"sort"
"strconv"
"strings"
@@ -45,7 +44,8 @@ const (
// Endpoint - any type of endpoint.
type Endpoint struct {
*url.URL
IsLocal bool
IsLocal bool
SetIndex int
}
func (endpoint Endpoint) String() string {
@@ -166,21 +166,6 @@ func NewEndpoint(arg string) (ep Endpoint, e error) {
// EndpointList - list of same type of endpoint.
type EndpointList []Endpoint
// Swap - helper method for sorting.
func (endpoints EndpointList) Swap(i, j int) {
endpoints[i], endpoints[j] = endpoints[j], endpoints[i]
}
// Len - helper method for sorting.
func (endpoints EndpointList) Len() int {
return len(endpoints)
}
// Less - helper method for sorting.
func (endpoints EndpointList) Less(i, j int) bool {
return endpoints[i].String() < endpoints[j].String()
}
// IsHTTPS - returns true if secure for URLEndpointType.
func (endpoints EndpointList) IsHTTPS() bool {
return endpoints[0].IsHTTPS()
@@ -197,16 +182,6 @@ func (endpoints EndpointList) GetString(i int) 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.
isValidDistribution := func(count int) bool {
return (count >= minErasureBlocks && count <= maxErasureBlocks && count%2 == 0)
}
// Check whether no. of args are valid for XL distribution.
if !isValidDistribution(len(args)) {
return nil, fmt.Errorf("A total of %d endpoints were found. For erasure mode it should be an even number between %d and %d", len(args), minErasureBlocks, maxErasureBlocks)
}
var endpointType EndpointType
var scheme string
@@ -236,8 +211,6 @@ func NewEndpointList(args ...string) (endpoints EndpointList, err error) {
endpoints = append(endpoints, endpoint)
}
sort.Sort(endpoints)
return endpoints, nil
}
@@ -258,7 +231,7 @@ func checkCrossDeviceMounts(endpoints EndpointList) (err error) {
}
// CreateEndpoints - validates and creates new endpoints for given args.
func CreateEndpoints(serverAddr string, args ...string) (string, EndpointList, SetupType, error) {
func CreateEndpoints(serverAddr string, args ...[]string) (string, EndpointList, SetupType, error) {
var endpoints EndpointList
var setupType SetupType
var err error
@@ -271,9 +244,9 @@ func CreateEndpoints(serverAddr string, args ...string) (string, EndpointList, S
_, serverAddrPort := mustSplitHostPort(serverAddr)
// For single arg, return FS setup.
if len(args) == 1 {
if len(args) == 1 && len(args[0]) == 1 {
var endpoint Endpoint
endpoint, err = NewEndpoint(args[0])
endpoint, err = NewEndpoint(args[0][0])
if err != nil {
return serverAddr, endpoints, setupType, err
}
@@ -290,14 +263,25 @@ func CreateEndpoints(serverAddr string, args ...string) (string, EndpointList, S
return serverAddr, endpoints, setupType, nil
}
// Convert args to endpoints
if endpoints, err = NewEndpointList(args...); err != nil {
return serverAddr, endpoints, setupType, err
}
for i, iargs := range args {
var newEndpoints EndpointList
// Convert args to endpoints
var eps EndpointList
eps, err = NewEndpointList(iargs...)
if err != nil {
return serverAddr, endpoints, setupType, err
}
// Check for cross device mounts if any.
if err = checkCrossDeviceMounts(endpoints); err != nil {
return serverAddr, endpoints, setupType, err
// Check for cross device mounts if any.
if err = checkCrossDeviceMounts(eps); err != nil {
return serverAddr, endpoints, setupType, err
}
for _, ep := range eps {
ep.SetIndex = i
newEndpoints = append(newEndpoints, ep)
}
endpoints = append(endpoints, newEndpoints...)
}
// Return XL setup when all endpoints are path style.
@@ -441,6 +425,23 @@ func CreateEndpoints(serverAddr string, args ...string) (string, EndpointList, S
}
}
uniqueArgs := set.NewStringSet()
for _, endpoint := range endpoints {
uniqueArgs.Add(endpoint.Host)
}
// Error out if we have more than serverCommandLineArgsMax unique servers.
if len(uniqueArgs.ToSlice()) > serverCommandLineArgsMax {
err := fmt.Errorf("Unsupported number of endpoints (%s), total number of servers cannot be more than %d", endpoints, serverCommandLineArgsMax)
return serverAddr, endpoints, setupType, err
}
// Error out if we have less than 2 unique servers.
if len(uniqueArgs.ToSlice()) < 2 && setupType == DistXLSetupType {
err := fmt.Errorf("Unsupported number of endpoints (%s), minimum number of servers cannot be less than 2 in distributed setup", endpoints)
return serverAddr, endpoints, setupType, err
}
setupType = DistXLSetupType
return serverAddr, endpoints, setupType, nil
}

View File

@@ -20,7 +20,6 @@ import (
"fmt"
"net/url"
"reflect"
"sort"
"strings"
"testing"
)
@@ -108,7 +107,6 @@ func TestNewEndpointList(t *testing.T) {
{[]string{"d1", "d2", "d3", "d1"}, fmt.Errorf("duplicate endpoints found")},
{[]string{"d1", "d2", "d3", "./d1"}, fmt.Errorf("duplicate endpoints found")},
{[]string{"http://localhost/d1", "http://localhost/d2", "http://localhost/d1", "http://localhost/d4"}, fmt.Errorf("duplicate endpoints found")},
{[]string{"d1", "d2", "d3", "d4", "d5"}, fmt.Errorf("A total of 5 endpoints were found. For erasure mode it should be an even number between 4 and 32")},
{[]string{"ftp://server/d1", "http://server/d2", "http://server/d3", "http://server/d4"}, fmt.Errorf("'ftp://server/d1': invalid URL endpoint format")},
{[]string{"d1", "http://localhost/d2", "d3", "d4"}, fmt.Errorf("mixed style endpoints are not supported")},
{[]string{"http://example.org/d1", "https://example.com/d1", "http://example.net/d1", "https://example.edut/d1"}, fmt.Errorf("mixed scheme is not supported")},
@@ -142,7 +140,6 @@ func TestCreateEndpoints(t *testing.T) {
getExpectedEndpoints := func(args []string, prefix string) ([]*url.URL, []bool) {
var URLs []*url.URL
var localFlags []bool
sort.Strings(args)
for _, arg := range args {
u, _ := url.Parse(arg)
URLs = append(URLs, u)
@@ -157,8 +154,8 @@ func TestCreateEndpoints(t *testing.T) {
args := []string{
"http://" + nonLoopBackIP + ":10000/d1",
"http://" + nonLoopBackIP + ":10000/d2",
"http://example.com:10000/d4",
"http://example.org:10000/d3",
"http://example.com:10000/d4",
}
case1URLs, case1LocalFlags := getExpectedEndpoints(args, "http://"+nonLoopBackIP+":10000/")
@@ -167,26 +164,26 @@ func TestCreateEndpoints(t *testing.T) {
args = []string{
"http://" + nonLoopBackIP + ":10000/d1",
"http://" + nonLoopBackIP + ":9000/d2",
"http://example.com:10000/d4",
"http://example.org:10000/d3",
"http://example.com:10000/d4",
}
case2URLs, case2LocalFlags := getExpectedEndpoints(args, "http://"+nonLoopBackIP+":10000/")
case3Endpoint1 := "http://" + nonLoopBackIP + "/d1"
args = []string{
"http://" + nonLoopBackIP + ":80/d1",
"http://example.org:9000/d2",
"http://example.com:80/d3",
"http://example.net:80/d4",
"http://example.org:9000/d2",
}
case3URLs, case3LocalFlags := getExpectedEndpoints(args, "http://"+nonLoopBackIP+":80/")
case4Endpoint1 := "http://" + nonLoopBackIP + "/d1"
args = []string{
"http://" + nonLoopBackIP + ":9000/d1",
"http://example.org:9000/d2",
"http://example.com:9000/d3",
"http://example.net:9000/d4",
"http://example.org:9000/d2",
}
case4URLs, case4LocalFlags := getExpectedEndpoints(args, "http://"+nonLoopBackIP+":9000/")
@@ -213,29 +210,29 @@ func TestCreateEndpoints(t *testing.T) {
testCases := []struct {
serverAddr string
args []string
args [][]string
expectedServerAddr string
expectedEndpoints EndpointList
expectedSetupType SetupType
expectedErr error
}{
{"localhost", []string{}, "", EndpointList{}, -1, fmt.Errorf("address localhost: missing port in address")},
{"localhost", [][]string{}, "", EndpointList{}, -1, fmt.Errorf("address localhost: missing port in address")},
// FS Setup
{"localhost:9000", []string{"http://localhost/d1"}, "", EndpointList{}, -1, fmt.Errorf("use path style endpoint for FS setup")},
{":443", []string{"d1"}, ":443", EndpointList{Endpoint{URL: &url.URL{Path: "d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", []string{"/d1"}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: "/d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", []string{"./d1"}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: "d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", []string{`\d1`}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: `\d1`}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", []string{`.\d1`}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: `.\d1`}, IsLocal: true}}, FSSetupType, nil},
{":8080", []string{"https://example.org/d1", "https://example.org/d2", "https://example.org/d3", "https://example.org/d4"}, "", EndpointList{}, -1, fmt.Errorf("no endpoint found for this host")},
{":8080", []string{"https://example.org/d1", "https://example.com/d2", "https://example.net:8000/d3", "https://example.edu/d1"}, "", EndpointList{}, -1, fmt.Errorf("no endpoint found for this host")},
{"localhost:9000", []string{"https://127.0.0.1:9000/d1", "https://localhost:9001/d1", "https://example.com/d1", "https://example.com/d2"}, "", EndpointList{}, -1, fmt.Errorf("path '/d1' can not be served by different port on same address")},
{"localhost:9000", []string{"https://127.0.0.1:8000/d1", "https://localhost:9001/d2", "https://example.com/d1", "https://example.com/d2"}, "", EndpointList{}, -1, fmt.Errorf("port number in server address must match with one of the port in local endpoints")},
{"localhost:10000", []string{"https://127.0.0.1:8000/d1", "https://localhost:8000/d2", "https://example.com/d1", "https://example.com/d2"}, "", EndpointList{}, -1, fmt.Errorf("server address and local endpoint have different ports")},
{"localhost:9000", [][]string{[]string{"http://localhost/d1"}}, "", EndpointList{}, -1, fmt.Errorf("use path style endpoint for FS setup")},
{":443", [][]string{[]string{"d1"}}, ":443", EndpointList{Endpoint{URL: &url.URL{Path: "d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", [][]string{[]string{"/d1"}}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: "/d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", [][]string{[]string{"./d1"}}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: "d1"}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", [][]string{[]string{`\d1`}}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: `\d1`}, IsLocal: true}}, FSSetupType, nil},
{"localhost:10000", [][]string{[]string{`.\d1`}}, "localhost:10000", EndpointList{Endpoint{URL: &url.URL{Path: `.\d1`}, IsLocal: true}}, FSSetupType, nil},
{":8080", [][]string{[]string{"https://example.org/d1", "https://example.org/d2", "https://example.org/d3", "https://example.org/d4"}}, "", EndpointList{}, -1, fmt.Errorf("no endpoint found for this host")},
{":8080", [][]string{[]string{"https://example.org/d1", "https://example.com/d2", "https://example.net:8000/d3", "https://example.edu/d1"}}, "", EndpointList{}, -1, fmt.Errorf("no endpoint found for this host")},
{"localhost:9000", [][]string{[]string{"https://127.0.0.1:9000/d1", "https://localhost:9001/d1", "https://example.com/d1", "https://example.com/d2"}}, "", EndpointList{}, -1, fmt.Errorf("path '/d1' can not be served by different port on same address")},
{"localhost:9000", [][]string{[]string{"https://127.0.0.1:8000/d1", "https://localhost:9001/d2", "https://example.com/d1", "https://example.com/d2"}}, "", EndpointList{}, -1, fmt.Errorf("port number in server address must match with one of the port in local endpoints")},
{"localhost:10000", [][]string{[]string{"https://127.0.0.1:8000/d1", "https://localhost:8000/d2", "https://example.com/d1", "https://example.com/d2"}}, "", EndpointList{}, -1, fmt.Errorf("server address and local endpoint have different ports")},
// XL Setup with PathEndpointType
{":1234", []string{"/d1", "/d2", "d3", "d4"}, ":1234",
{":1234", [][]string{[]string{"/d1", "/d2", "d3", "d4"}}, ":1234",
EndpointList{
Endpoint{URL: &url.URL{Path: "/d1"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d2"}, IsLocal: true},
@@ -243,53 +240,55 @@ func TestCreateEndpoints(t *testing.T) {
Endpoint{URL: &url.URL{Path: "d4"}, IsLocal: true},
}, XLSetupType, nil},
// XL Setup with URLEndpointType
{":9000", []string{"http://localhost/d1", "http://localhost/d2", "http://localhost/d3", "http://localhost/d4"}, ":9000", EndpointList{
{":9000", [][]string{[]string{"http://localhost/d1", "http://localhost/d2", "http://localhost/d3", "http://localhost/d4"}}, ":9000", EndpointList{
Endpoint{URL: &url.URL{Path: "/d1"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d2"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d3"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d4"}, IsLocal: true},
}, XLSetupType, nil},
// XL Setup with URLEndpointType having mixed naming to local host.
{"127.0.0.1:10000", []string{"http://localhost/d1", "http://localhost/d2", "http://127.0.0.1/d3", "http://127.0.0.1/d4"}, ":10000", EndpointList{
{"127.0.0.1:10000", [][]string{[]string{"http://localhost/d1", "http://localhost/d2", "http://127.0.0.1/d3", "http://127.0.0.1/d4"}}, ":10000", EndpointList{
Endpoint{URL: &url.URL{Path: "/d1"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d2"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d3"}, IsLocal: true},
Endpoint{URL: &url.URL{Path: "/d4"}, IsLocal: true},
}, XLSetupType, nil},
{":9001", []string{"http://10.0.0.1:9000/export", "http://10.0.0.2:9000/export", "http://" + nonLoopBackIP + ":9001/export", "http://10.0.0.2:9001/export"}, "", EndpointList{}, -1, fmt.Errorf("path '/export' can not be served by different port on same address")},
{":9000", []string{"http://127.0.0.1:9000/export", "http://" + nonLoopBackIP + ":9000/export", "http://10.0.0.1:9000/export", "http://10.0.0.2:9000/export"}, "", EndpointList{}, -1, fmt.Errorf("path '/export' cannot be served by different address on same server")},
{":9000", []string{"http://localhost/d1", "http://localhost/d2", "http://example.org/d3", "http://example.com/d4"}, "", EndpointList{}, -1, fmt.Errorf("'localhost' resolves to loopback address is not allowed for distributed XL")},
{":9001", [][]string{[]string{"http://10.0.0.1:9000/export", "http://10.0.0.2:9000/export", "http://" + nonLoopBackIP + ":9001/export", "http://10.0.0.2:9001/export"}}, "", EndpointList{}, -1, fmt.Errorf("path '/export' can not be served by different port on same address")},
{":9000", [][]string{[]string{"http://127.0.0.1:9000/export", "http://" + nonLoopBackIP + ":9000/export", "http://10.0.0.1:9000/export", "http://10.0.0.2:9000/export"}}, "", EndpointList{}, -1, fmt.Errorf("path '/export' cannot be served by different address on same server")},
{":9000", [][]string{[]string{"http://localhost/d1", "http://localhost/d2", "http://example.org/d3", "http://example.com/d4"}}, "", EndpointList{}, -1, fmt.Errorf("'localhost' resolves to loopback address is not allowed for distributed XL")},
// DistXL type
{"127.0.0.1:10000", []string{case1Endpoint1, case1Endpoint2, "http://example.org/d3", "http://example.com/d4"}, "127.0.0.1:10000", EndpointList{
{"127.0.0.1:10000", [][]string{[]string{case1Endpoint1, case1Endpoint2, "http://example.org/d3", "http://example.com/d4"}}, "127.0.0.1:10000", EndpointList{
Endpoint{URL: case1URLs[0], IsLocal: case1LocalFlags[0]},
Endpoint{URL: case1URLs[1], IsLocal: case1LocalFlags[1]},
Endpoint{URL: case1URLs[2], IsLocal: case1LocalFlags[2]},
Endpoint{URL: case1URLs[3], IsLocal: case1LocalFlags[3]},
}, DistXLSetupType, nil},
{"127.0.0.1:10000", []string{case2Endpoint1, case2Endpoint2, "http://example.org/d3", "http://example.com/d4"}, "127.0.0.1:10000", EndpointList{
{"127.0.0.1:10000", [][]string{[]string{case2Endpoint1, case2Endpoint2, "http://example.org/d3", "http://example.com/d4"}}, "127.0.0.1:10000", EndpointList{
Endpoint{URL: case2URLs[0], IsLocal: case2LocalFlags[0]},
Endpoint{URL: case2URLs[1], IsLocal: case2LocalFlags[1]},
Endpoint{URL: case2URLs[2], IsLocal: case2LocalFlags[2]},
Endpoint{URL: case2URLs[3], IsLocal: case2LocalFlags[3]},
}, DistXLSetupType, nil},
{":80", []string{case3Endpoint1, "http://example.org:9000/d2", "http://example.com/d3", "http://example.net/d4"}, ":80", EndpointList{
{":80", [][]string{[]string{case3Endpoint1, "http://example.org:9000/d2", "http://example.com/d3", "http://example.net/d4"}}, ":80", EndpointList{
Endpoint{URL: case3URLs[0], IsLocal: case3LocalFlags[0]},
Endpoint{URL: case3URLs[1], IsLocal: case3LocalFlags[1]},
Endpoint{URL: case3URLs[2], IsLocal: case3LocalFlags[2]},
Endpoint{URL: case3URLs[3], IsLocal: case3LocalFlags[3]},
}, DistXLSetupType, nil},
{":9000", []string{case4Endpoint1, "http://example.org/d2", "http://example.com/d3", "http://example.net/d4"}, ":9000", EndpointList{
{":9000", [][]string{[]string{case4Endpoint1, "http://example.org/d2", "http://example.com/d3", "http://example.net/d4"}}, ":9000", EndpointList{
Endpoint{URL: case4URLs[0], IsLocal: case4LocalFlags[0]},
Endpoint{URL: case4URLs[1], IsLocal: case4LocalFlags[1]},
Endpoint{URL: case4URLs[2], IsLocal: case4LocalFlags[2]},
Endpoint{URL: case4URLs[3], IsLocal: case4LocalFlags[3]},
}, DistXLSetupType, nil},
{":9000", []string{case5Endpoint1, case5Endpoint2, case5Endpoint3, case5Endpoint4}, ":9000", EndpointList{
{":9000", [][]string{[]string{case5Endpoint1, case5Endpoint2, case5Endpoint3, case5Endpoint4}}, ":9000", EndpointList{
Endpoint{URL: case5URLs[0], IsLocal: case5LocalFlags[0]},
Endpoint{URL: case5URLs[1], IsLocal: case5LocalFlags[1]},
Endpoint{URL: case5URLs[2], IsLocal: case5LocalFlags[2]},
@@ -297,7 +296,7 @@ func TestCreateEndpoints(t *testing.T) {
}, DistXLSetupType, nil},
// DistXL Setup using only local host.
{":9003", []string{"http://localhost:9000/d1", "http://localhost:9001/d2", "http://127.0.0.1:9002/d3", case6Endpoint}, ":9003", EndpointList{
{":9003", [][]string{[]string{"http://localhost:9000/d1", "http://localhost:9001/d2", "http://127.0.0.1:9002/d3", case6Endpoint}}, ":9003", EndpointList{
Endpoint{URL: case6URLs[0], IsLocal: case6LocalFlags[0]},
Endpoint{URL: case6URLs[1], IsLocal: case6LocalFlags[1]},
Endpoint{URL: case6URLs[2], IsLocal: case6LocalFlags[2]},

View File

@@ -37,7 +37,7 @@ func TestInitEventNotifierFaultyDisks(t *testing.T) {
// remove the root directory after the test ends.
defer os.RemoveAll(rootPath)
disks, err := getRandomDisks(4)
disks, err := getRandomDisks(16)
if err != nil {
t.Fatal("Unable to create directories for FS backend. ", err)
}
@@ -64,12 +64,13 @@ func TestInitEventNotifierFaultyDisks(t *testing.T) {
notificationXML += "</NotificationConfiguration>"
size := int64(len([]byte(notificationXML)))
reader := bytes.NewReader([]byte(notificationXML))
if _, err := xl.PutObject(minioMetaBucket, bucketConfigPrefix+"/"+bucketName+"/"+bucketNotificationConfig, mustGetHashReader(t, reader, size, "", ""), nil); err != nil {
bucketConfigPath := bucketConfigPrefix + "/" + bucketName + "/" + bucketNotificationConfig
if _, err := xl.PutObject(minioMetaBucket, bucketConfigPath, mustGetHashReader(t, reader, size, "", ""), nil); err != nil {
t.Fatal("Unexpected error:", err)
}
for i, d := range xl.storageDisks {
xl.storageDisks[i] = newNaughtyDisk(d.(*retryStorage), nil, errFaultyDisk)
xl.storageDisks[i] = newNaughtyDisk(d, nil, errFaultyDisk)
}
// Test initEventNotifier() with faulty disks
for i := 1; i <= 3; i++ {

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@@ -164,7 +164,8 @@ func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
fs.fsFormatRlk = rlk
// Initialize and load bucket policies.
if err = initBucketPolicies(fs); err != nil {
fs.bucketPolicies, err = initBucketPolicies(fs)
if err != nil {
return nil, fmt.Errorf("Unable to load all bucket policies. %s", err)
}
@@ -174,6 +175,7 @@ func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
}
go fs.cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, globalServiceDoneCh)
// Return successfully initialized object layer.
return fs, nil
}
@@ -1048,6 +1050,11 @@ func (fs *fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKe
return result, nil
}
// HealFormat - no-op for fs, Valid only for XL.
func (fs *fsObjects) HealFormat(dryRun bool) (madmin.HealResultItem, error) {
return madmin.HealResultItem{}, errors.Trace(NotImplemented{})
}
// HealObject - no-op for fs. Valid only for XL.
func (fs *fsObjects) HealObject(bucket, object string, dryRun bool) (
res madmin.HealResultItem, err error) {

View File

@@ -78,6 +78,11 @@ func (a GatewayUnsupported) DeleteBucketPolicy(bucket string) error {
return errors.Trace(NotImplemented{})
}
// HealFormat - Not implemented stub
func (a GatewayUnsupported) HealFormat(dryRun bool) (madmin.HealResultItem, error) {
return madmin.HealResultItem{}, errors.Trace(NotImplemented{})
}
// HealBucket - Not implemented stub
func (a GatewayUnsupported) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, error) {
return nil, errors.Trace(NotImplemented{})

View File

@@ -74,6 +74,12 @@ const (
)
var (
// Indicates the total number of erasure coded sets configured.
globalXLSetCount int
// Indicates set drive count.
globalXLSetDriveCount int
// Indicates if the running minio server is distributed setup.
globalIsDistXL = false

View File

@@ -326,7 +326,7 @@ func (l *lockServer) lockMaintenance(interval time.Duration) {
})
// Close the connection regardless of the call response.
c.rpcClient.Close()
c.AuthRPCClient.Close()
// For successful response, verify if lock is indeed active or stale.
if expired {

View File

@@ -33,6 +33,7 @@ func TestListLocksInfo(t *testing.T) {
t.Fatal(err)
}
defer os.RemoveAll(rootPath)
// Initializing new XL objectLayer.
objAPI, _, xlErr := initTestXLObjLayer()
if xlErr != nil {
@@ -49,7 +50,7 @@ func TestListLocksInfo(t *testing.T) {
var nsMutex *nsLockMap
nsMutex = objAPI.(*xlObjects).nsMutex
nsMutex = objAPI.(*xlSets).sets[0].nsMutex
// Acquire a few locks to populate lock instrumentation.
// Take 10 read locks on bucket1/prefix1/obj1

View File

@@ -93,6 +93,7 @@ func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int)
lockMap: make(map[string][]lockRequesterInfo),
},
}
globalLockServer = &localLockServer
clnts = append(clnts, &(localLockServer.ll))
}

View File

@@ -28,7 +28,7 @@ import (
// Programmed errors are stored in errors field.
type naughtyDisk struct {
// The real disk
disk *retryStorage
disk StorageAPI
// Programmed errors: API call number => error to return
errors map[int]error
// The error to return when no error value is programmed
@@ -39,7 +39,7 @@ type naughtyDisk struct {
mu sync.Mutex
}
func newNaughtyDisk(d *retryStorage, errs map[int]error, defaultErr error) *naughtyDisk {
func newNaughtyDisk(d StorageAPI, errs map[int]error, defaultErr error) *naughtyDisk {
return &naughtyDisk{disk: d, errors: errs, defaultErr: defaultErr}
}
@@ -47,11 +47,11 @@ func (d *naughtyDisk) String() string {
return d.disk.String()
}
func (d *naughtyDisk) Init() (err error) {
if err = d.calcError(); err != nil {
return err
func (d *naughtyDisk) IsOnline() bool {
if err := d.calcError(); err != nil {
return err == errDiskNotFound
}
return d.disk.Init()
return d.disk.IsOnline()
}
func (d *naughtyDisk) Close() (err error) {

View File

@@ -85,7 +85,7 @@ func getHostIP4(host string) (ipList set.StringSet, err error) {
// Mark the starting time
startTime := time.Now()
// wait for hosts to resolve in exponentialbackoff manner
for _ = range newRetryTimerSimple(doneCh) {
for range newRetryTimerSimple(doneCh) {
// Retry infinitely on Kubernetes and Docker swarm.
// This is needed as the remote hosts are sometime
// not available immediately.

View File

@@ -159,69 +159,6 @@ func newStorageAPI(endpoint Endpoint) (storage StorageAPI, err error) {
return newStorageRPC(endpoint), nil
}
var initMetaVolIgnoredErrs = append(baseIgnoredErrs, errVolumeExists)
// Initializes meta volume on all input storage disks.
func initMetaVolume(storageDisks []StorageAPI) error {
// This happens for the first time, but keep this here since this
// is the only place where it can be made expensive optimizing all
// other calls. Create minio meta volume, if it doesn't exist yet.
var wg = &sync.WaitGroup{}
// Initialize errs to collect errors inside go-routine.
var errs = make([]error, len(storageDisks))
// Initialize all disks in parallel.
for index, disk := range storageDisks {
if disk == nil {
// Ignore create meta volume on disks which are not found.
continue
}
wg.Add(1)
go func(index int, disk StorageAPI) {
// Indicate this wait group is done.
defer wg.Done()
// Attempt to create `.minio.sys`.
err := disk.MakeVol(minioMetaBucket)
if err != nil {
if !errors.IsErrIgnored(err, initMetaVolIgnoredErrs...) {
errs[index] = err
return
}
}
err = disk.MakeVol(minioMetaTmpBucket)
if err != nil {
if !errors.IsErrIgnored(err, initMetaVolIgnoredErrs...) {
errs[index] = err
return
}
}
err = disk.MakeVol(minioMetaMultipartBucket)
if err != nil {
if !errors.IsErrIgnored(err, initMetaVolIgnoredErrs...) {
errs[index] = err
return
}
}
}(index, disk)
}
// Wait for all cleanup to finish.
wg.Wait()
// Return upon first error.
for _, err := range errs {
if err == nil {
continue
}
return toObjectErr(err, minioMetaBucket)
}
// Return success here.
return nil
}
// Cleanup a directory recursively.
func cleanupDir(storage StorageAPI, volume, dirPath string) error {
var delFunc func(string) error

View File

@@ -16,7 +16,11 @@
package cmd
import "time"
import (
"time"
"github.com/minio/minio/pkg/madmin"
)
// BackendType - represents different backend types.
type BackendType int
@@ -45,8 +49,13 @@ type StorageInfo struct {
// Following fields are only meaningful if BackendType is Erasure.
OnlineDisks int // Online disks during server startup.
OfflineDisks int // Offline disks during server startup.
StandardSCData int // Data disks for currently configured Standard storage class.
StandardSCParity int // Parity disks for currently configured Standard storage class.
RRSCData int // Data disks for currently configured Reduced Redundancy storage class.
RRSCParity int // Parity disks for currently configured Reduced Redundancy storage class.
// List of all disk status, this is only meaningful if BackendType is Erasure.
Sets [][]madmin.DriveInfo
}
}

View File

@@ -56,6 +56,7 @@ type ObjectLayer interface {
CompleteMultipartUpload(bucket, object, uploadID string, uploadedParts []CompletePart) (objInfo ObjectInfo, err error)
// Healing operations.
HealFormat(dryRun bool) (madmin.HealResultItem, error)
HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, error)
HealObject(bucket, object string, dryRun bool) (madmin.HealResultItem, error)
ListBucketsHeal() (buckets []BucketInfo, err error)

View File

@@ -43,6 +43,7 @@ type posix struct {
ioErrCount int32 // ref: https://golang.org/pkg/sync/atomic/#pkg-note-BUG
diskPath string
pool sync.Pool
connected bool
}
// checkPathLength - returns error if given path name length more than 255
@@ -137,6 +138,8 @@ func newPosix(path string) (StorageAPI, error) {
return nil, err
}
st.connected = true
// Success.
return st, nil
}
@@ -218,14 +221,13 @@ func (s *posix) String() string {
return s.diskPath
}
// Init - this is a dummy call.
func (s *posix) Init() error {
func (s *posix) Close() error {
s.connected = false
return nil
}
// Close - this is a dummy call.
func (s *posix) Close() error {
return nil
func (s *posix) IsOnline() bool {
return s.connected
}
// DiskInfo provides current information about disk space usage,
@@ -249,6 +251,9 @@ func (s *posix) getVolDir(volume string) (string, error) {
// checkDiskFound - validates if disk is available,
// returns errDiskNotFound if not found.
func (s *posix) checkDiskFound() (err error) {
if !s.IsOnline() {
return errDiskNotFound
}
_, err = os.Stat((s.diskPath))
if err != nil {
if os.IsNotExist(err) {

View File

@@ -1,147 +0,0 @@
/*
* Minio Cloud Storage, (C) 2016 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 (
"fmt"
"sync"
humanize "github.com/dustin/go-humanize"
)
// Helper to generate integer sequences into a friendlier user consumable format.
func formatInts(i int, t int) string {
if i < 10 {
if t < 10 {
return fmt.Sprintf("0%d/0%d", i, t)
}
return fmt.Sprintf("0%d/%d", i, t)
}
return fmt.Sprintf("%d/%d", i, t)
}
// Print a given message once.
type printOnceFunc func(msg string)
// Print once is a constructor returning a function printing once.
// internally print uses sync.Once to perform exactly one action.
func printOnceFn() printOnceFunc {
var once sync.Once
return func(msg string) {
once.Do(func() {
log.Println(msg)
})
}
}
// Prints custom message when healing is required for XL and Distributed XL backend.
func printHealMsg(endpoints EndpointList, storageDisks []StorageAPI, fn printOnceFunc) {
msg := getHealMsg(endpoints, storageDisks)
fn(msg)
}
// Disks offline and online strings..
const (
diskOffline = "offline"
diskOnline = "online"
)
// Constructs a formatted heal message, when cluster is found to be in state where it requires healing.
// healing is optional, server continues to initialize object layer after printing this message.
// it is upto the end user to perform a heal if needed.
func getHealMsg(endpoints EndpointList, storageDisks []StorageAPI) string {
healFmtCmd := `"mc admin heal myminio"`
msg := fmt.Sprintf("New disk(s) were found, format them by running - %s\n",
healFmtCmd)
disksInfo, _, _ := getDisksInfo(storageDisks)
for i, info := range disksInfo {
if storageDisks[i] == nil {
continue
}
msg += fmt.Sprintf(
"\n[%s] %s - %s %s",
formatInts(i+1, len(storageDisks)),
endpoints[i],
humanize.IBytes(uint64(info.Total)),
func() string {
if info.Total > 0 {
return diskOnline
}
return diskOffline
}(),
)
}
return msg
}
// Prints regular message when we have sufficient disks to start the cluster.
func printRegularMsg(endpoints EndpointList, storageDisks []StorageAPI, fn printOnceFunc) {
msg := getStorageInitMsg("Initializing data volume.", endpoints, storageDisks)
fn(msg)
}
// Constructs a formatted regular message when we have sufficient disks to start the cluster.
func getStorageInitMsg(titleMsg string, endpoints EndpointList, storageDisks []StorageAPI) string {
msg := colorBlue(titleMsg)
disksInfo, _, _ := getDisksInfo(storageDisks)
for i, info := range disksInfo {
if storageDisks[i] == nil {
continue
}
msg += fmt.Sprintf(
"\n[%s] %s - %s %s",
formatInts(i+1, len(storageDisks)),
endpoints[i],
humanize.IBytes(uint64(info.Total)),
func() string {
if info.Total > 0 {
return diskOnline
}
return diskOffline
}(),
)
}
return msg
}
// Prints initialization message when cluster is being initialized for the first time.
func printFormatMsg(endpoints EndpointList, storageDisks []StorageAPI, fn printOnceFunc) {
msg := getStorageInitMsg("Initializing data volume for the first time.", endpoints, storageDisks)
fn(msg)
}
// Combines each disk errors in a newline formatted string.
// this is a helper function in printing messages across
// all disks.
func combineDiskErrs(storageDisks []StorageAPI, sErrs []error) string {
var msg string
for i, disk := range storageDisks {
if disk == nil {
continue
}
if sErrs[i] == nil {
continue
}
msg += fmt.Sprintf(
"\n[%s] %s : %s",
formatInts(i+1, len(storageDisks)),
storageDisks[i],
sErrs[i],
)
}
return msg
}

View File

@@ -1,107 +0,0 @@
/*
* Minio Cloud Storage, (C) 2016, 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 (
"fmt"
"os"
"testing"
)
// Tests heal message to be correct and properly formatted.
func TestHealMsg(t *testing.T) {
rootPath, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatal("Unable to initialize test config", err)
}
defer os.RemoveAll(rootPath)
storageDisks, fsDirs := prepareXLStorageDisks(t)
errs := make([]error, len(storageDisks))
defer removeRoots(fsDirs)
nilDisks := deepCopyStorageDisks(storageDisks)
nilDisks[5] = nil
authErrs := make([]error, len(storageDisks))
authErrs[5] = errAuthentication
args := []string{}
for i := range storageDisks {
args = append(args, fmt.Sprintf("http://10.1.10.%d:9000/d1", i+1))
}
endpoints := mustGetNewEndpointList(args...)
testCases := []struct {
endPoints EndpointList
storageDisks []StorageAPI
serrs []error
}{
// Test - 1 for valid disks and errors.
{endpoints, storageDisks, errs},
// Test - 2 for one of the disks is nil.
{endpoints, nilDisks, errs},
}
for i, testCase := range testCases {
msg := getHealMsg(testCase.endPoints, testCase.storageDisks)
if msg == "" {
t.Fatalf("Test: %d Unable to get heal message.", i+1)
}
msg = getStorageInitMsg("init", testCase.endPoints, testCase.storageDisks)
if msg == "" {
t.Fatalf("Test: %d Unable to get regular message.", i+1)
}
}
}
// Tests disk info, validates if we do return proper disk info structure
// even in case of certain disks not available.
func TestDisksInfo(t *testing.T) {
storageDisks, fsDirs := prepareXLStorageDisks(t)
defer removeRoots(fsDirs)
testCases := []struct {
storageDisks []StorageAPI
onlineDisks int
offlineDisks int
}{
{
storageDisks: storageDisks,
onlineDisks: 16,
offlineDisks: 0,
},
{
storageDisks: prepareNOfflineDisks(deepCopyStorageDisks(storageDisks), 4, t),
onlineDisks: 12,
offlineDisks: 4,
},
{
storageDisks: prepareNOfflineDisks(deepCopyStorageDisks(storageDisks), 16, t),
onlineDisks: 0,
offlineDisks: 16,
},
}
for i, testCase := range testCases {
_, onlineDisks, offlineDisks := getDisksInfo(testCase.storageDisks)
if testCase.onlineDisks != onlineDisks {
t.Errorf("Test %d: Expected online disks %d, got %d", i+1, testCase.onlineDisks, onlineDisks)
}
if testCase.offlineDisks != offlineDisks {
t.Errorf("Test %d: Expected offline disks %d, got %d", i+1, testCase.offlineDisks, offlineDisks)
}
}
}

View File

@@ -18,222 +18,60 @@ package cmd
import (
"fmt"
"os"
"time"
"github.com/minio/mc/pkg/console"
"github.com/minio/minio/pkg/errors"
)
/*
var printEndpointError = func() func(Endpoint, error) {
printOnce := make(map[Endpoint]map[string]bool)
Following table lists different possible states the backend could be in.
return func(endpoint Endpoint, err error) {
m, ok := printOnce[endpoint]
if !ok {
m = make(map[string]bool)
m[err.Error()] = true
printOnce[endpoint] = m
errorIf(err, "%s: %s", endpoint, err)
return
}
if m[err.Error()] {
return
}
m[err.Error()] = true
errorIf(err, "%s: %s", endpoint, err)
}
}()
* In a single-node, multi-disk setup, "Online" would refer to disks' status.
* In a multi-node setup, it could refer to disks' or network connectivity
between the nodes, or both.
+----------+--------------------------+-----------------------+
| Online | Format status | Course of action |
| | | |
-----------+--------------------------+-----------------------+
| All | All Formatted | |
+----------+--------------------------+ initObjectLayer |
| Quorum | Quorum Formatted | |
+----------+--------------------------+-----------------------+
| All | Quorum | Print message saying |
| | Formatted, | "Heal via control" |
| | some unformatted | and initObjectLayer |
+----------+--------------------------+-----------------------+
| All | None Formatted | FormatDisks |
| | | and initObjectLayer |
| | | |
+----------+--------------------------+-----------------------+
| No | | Wait till enough |
| Quorum | _ | nodes are online and |
| | | one of the above |
| | | sections apply |
+----------+--------------------------+-----------------------+
| | | |
| Quorum | Quorum UnFormatted | Abort |
+----------+--------------------------+-----------------------+
A disk can be in one of the following states.
- Unformatted
- Formatted
- Corrupted
- Offline
*/
// InitActions - a type synonym for enumerating initialization activities.
type InitActions int
const (
// FormatDisks - see above table for disk states where it is applicable.
FormatDisks InitActions = iota
// SuggestToHeal - Prints heal message and initialize object layer.
SuggestToHeal
// WaitForQuorum - Wait for quorum number of disks to be online.
WaitForQuorum
// WaitForAll - Wait for all disks to be online.
WaitForAll
// WaitForFormatting - Wait for formatting to be triggered
// from the '1st' server in the cluster.
WaitForFormatting
// WaitForConfig - Wait for all servers to have the same config
// including (credentials, version and time).
WaitForConfig
// InitObjectLayer - Initialize object layer.
InitObjectLayer
// Abort initialization of object layer since there aren't enough good
// copies of format.json to recover.
Abort
)
// configErrs contains the list of configuration errors.
var configErrs = []error{
errInvalidAccessKeyID,
errAuthentication,
errRPCAPIVersionUnsupported,
errServerTimeMismatch,
func formatXLMigrateLocalEndpoints(endpoints EndpointList) error {
for _, endpoint := range endpoints {
if !endpoint.IsLocal {
continue
}
formatPath := pathJoin(endpoint.Path, minioMetaBucket, formatConfigFile)
if _, err := os.Stat(formatPath); err != nil {
if os.IsNotExist(err) {
continue
}
return err
}
if err := formatXLMigrate(endpoint.Path); err != nil {
return err
}
}
return nil
}
// Config errs to actions converts looking for specific config errors
// which need to be returned quickly and server should wait instead.
func configErrsToActions(errMap map[error]int) InitActions {
var action InitActions
for _, configErr := range configErrs {
if errMap[configErr] > 0 {
action = WaitForConfig
break
}
}
return action
}
// reduceInitXLErrs reduces errors found in distributed XL initialization
func reduceInitXLErrs(storageDisks []StorageAPI, sErrs []error) error {
var foundErrs int
for i := range sErrs {
if sErrs[i] != nil {
foundErrs++
}
}
if foundErrs == 0 {
return nil
}
// Early quit if there is a config error
for i := range sErrs {
if contains(configErrs, sErrs[i]) {
return fmt.Errorf("%s: %s", storageDisks[i], sErrs[i])
}
}
// Combine all disk errors otherwise for user inspection
return fmt.Errorf("%s", combineDiskErrs(storageDisks, sErrs))
}
// Preparatory initialization stage for XL validates known errors.
// Converts them into specific actions. These actions have special purpose
// which caller decides on what needs to be done.
// Logic used in this function is as shown below.
//
// ---- Possible states and handled conditions -----
//
// - Formatted setup
// - InitObjectLayer when `disksFormatted >= readQuorum`
// - Wait for quorum when `disksFormatted < readQuorum && disksFormatted + disksOffline >= readQuorum`
// (we don't know yet if there are unformatted disks)
// - Wait for heal when `disksFormatted >= readQuorum && disksUnformatted > 0`
// (here we know there is at least one unformatted disk which requires healing)
//
// - Unformatted setup
// - Format/Wait for format when `disksUnformatted == diskCount`
//
// - Wait for all when `disksUnformatted + disksFormatted + diskOffline == diskCount`
//
// Under all other conditions should lead to server initialization aborted.
func prepForInitXL(firstDisk bool, sErrs []error, diskCount int) InitActions {
// Count errors by error value.
errMap := make(map[error]int)
for _, err := range sErrs {
errMap[errors.Cause(err)]++
// Format disks before initialization of object layer.
func waitForFormatXL(firstDisk bool, endpoints EndpointList, setCount, disksPerSet int) (format *formatXLV2, err error) {
if len(endpoints) == 0 || setCount == 0 || disksPerSet == 0 {
return nil, errInvalidArgument
}
// Validates and converts specific config errors into WaitForConfig.
if configErrsToActions(errMap) == WaitForConfig {
return WaitForConfig
}
readQuorum := diskCount / 2
disksOffline := errMap[errDiskNotFound]
disksFormatted := errMap[nil]
disksUnformatted := errMap[errUnformattedDisk]
// No Quorum lots of offline disks, wait for quorum.
if disksOffline > readQuorum {
return WaitForQuorum
}
// All disks are unformatted, proceed to formatting disks.
if disksUnformatted == diskCount {
// Only the first server formats an uninitialized setup, others wait for notification.
if firstDisk { // First node always initializes.
return FormatDisks
}
return WaitForFormatting
}
// Already formatted and in quorum, proceed to initialization of object layer.
if disksFormatted >= readQuorum {
if disksFormatted+disksOffline == diskCount {
return InitObjectLayer
}
// Some of the formatted disks are possibly corrupted or unformatted,
// let user know to heal them.
return SuggestToHeal
}
// Some unformatted, some disks formatted and some disks are offline but we don't
// quorum to decide. This is an undecisive state - wait for all of offline disks
// to be online to figure out the course of action.
if disksUnformatted+disksFormatted+disksOffline == diskCount {
return WaitForAll
}
// Exhausted all our checks, un-handled situations such as some disks corrupted we Abort.
return Abort
}
// Prints retry message upon a specific retry count.
func printRetryMsg(sErrs []error, storageDisks []StorageAPI) {
for i, sErr := range sErrs {
switch sErr {
case errDiskNotFound, errFaultyDisk, errFaultyRemoteDisk:
errorIf(sErr, "Disk %s is still unreachable", storageDisks[i])
}
}
}
// Maximum retry attempts.
const maxRetryAttempts = 5
// Implements a jitter backoff loop for formatting all disks during
// initialization of the server.
func retryFormattingXLDisks(firstDisk bool, endpoints EndpointList, storageDisks []StorageAPI) error {
if len(endpoints) == 0 {
return errInvalidArgument
}
if storageDisks == nil {
return errInvalidArgument
if err = formatXLMigrateLocalEndpoints(endpoints); err != nil {
return nil, err
}
// Done channel is used to close any lingering retry routine, as soon
@@ -254,138 +92,54 @@ func retryFormattingXLDisks(firstDisk bool, endpoints EndpointList, storageDisks
retryTimerCh := newRetryTimerSimple(doneCh)
for {
select {
case retryCount := <-retryTimerCh:
case _ = <-retryTimerCh:
// Attempt to load all `format.json` from all disks.
formatConfigs, sErrs := loadAllFormats(storageDisks)
if retryCount > maxRetryAttempts {
// After max retry attempts we start printing
// actual errors for disks not being available.
printRetryMsg(sErrs, storageDisks)
}
formatConfigs, sErrs := loadFormatXLAll(endpoints)
// Pre-emptively check if one of the formatted disks
// is invalid. This function returns success for the
// most part unless one of the formats is not consistent
// with expected XL format. For example if a user is
// trying to pool FS backend into an XL set.
if index, err := checkFormatXLValues(formatConfigs); err != nil {
// We will perhaps print and retry for the first 5 attempts
// because in XL initialization first server is the one which
// initializes the erasure set. This check ensures that the
// rest of the other servers do get a chance to see that the
// first server has a wrong format and exit gracefully.
// refer - https://github.com/minio/minio/issues/4140
if retryCount > maxRetryAttempts {
errorIf(err, "%s : Detected disk in unexpected format",
storageDisks[index])
if err = checkFormatXLValues(formatConfigs); err != nil {
return nil, err
}
for i, sErr := range sErrs {
if _, ok := formatCriticalErrors[errors.Cause(sErr)]; ok {
return nil, fmt.Errorf("Disk %s: %s", endpoints[i], sErr)
}
}
if shouldInitXLDisks(sErrs) {
if !firstDisk {
console.Println("Waiting for the first server to format the disks.")
continue
}
return err
return initFormatXL(endpoints, setCount, disksPerSet)
}
// Check if this is a XL or distributed XL, anything > 1 is considered XL backend.
switch prepForInitXL(firstDisk, sErrs, len(storageDisks)) {
case Abort:
return reduceInitXLErrs(storageDisks, sErrs)
case FormatDisks:
printFormatMsg(endpoints, storageDisks, printOnceFn())
return initFormatXL(storageDisks)
case InitObjectLayer:
// Validate formats loaded before proceeding forward.
err := genericFormatCheckXL(formatConfigs, sErrs)
if err == nil {
printRegularMsg(endpoints, storageDisks, printOnceFn())
format, err = getFormatXLInQuorum(formatConfigs)
if err == nil {
for i := range formatConfigs {
if formatConfigs[i] == nil {
continue
}
if err = formatXLV2Check(format, formatConfigs[i]); err != nil {
return nil, fmt.Errorf("%s format error: %s", endpoints[i], err)
}
}
return err
case SuggestToHeal:
// Validate formats loaded before proceeding forward.
err := genericFormatCheckXL(formatConfigs, sErrs)
if err == nil {
printHealMsg(endpoints, storageDisks, printOnceFn())
if len(format.XL.Sets) != globalXLSetCount {
return nil, fmt.Errorf("Current backend format is inconsistent with input args (%s), Expected set count %d, got %d", endpoints, len(format.XL.Sets), globalXLSetCount)
}
return err
case WaitForQuorum:
log.Printf(
"Initializing data volume. Waiting for minimum %d servers to come online. (elapsed %s)\n",
len(storageDisks)/2+1, getElapsedTime(),
)
case WaitForConfig:
// Print configuration errors.
log.Printf(
"Initializing data volume. Waiting for configuration issues to be fixed (%s). (elapsed %s)\n",
reduceInitXLErrs(storageDisks, sErrs), getElapsedTime())
case WaitForAll:
log.Printf("Initializing data volume for first time. Waiting for other servers to come online (elapsed %s)\n", getElapsedTime())
case WaitForFormatting:
log.Printf("Initializing data volume for first time. Waiting for first server to come online (elapsed %s)\n", getElapsedTime())
if len(format.XL.Sets[0]) != globalXLSetDriveCount {
return nil, fmt.Errorf("Current backend format is inconsistent with input args (%s), Expected drive count per set %d, got %d", endpoints, len(format.XL.Sets[0]), globalXLSetDriveCount)
}
return format, nil
}
console.Printf("Waiting for a minimum of %d disks to come online (elapsed %s)\n", len(endpoints)/2, getElapsedTime())
case <-globalOSSignalCh:
return fmt.Errorf("Initializing data volumes gracefully stopped")
return nil, fmt.Errorf("Initializing data volumes gracefully stopped")
}
}
}
// Initialize storage disks based on input arguments.
func initStorageDisks(endpoints EndpointList) ([]StorageAPI, error) {
// Bootstrap disks.
storageDisks := make([]StorageAPI, len(endpoints))
for index, endpoint := range endpoints {
// Intentionally ignore disk not found errors. XL is designed
// to handle these errors internally.
storage, err := newStorageAPI(endpoint)
if err != nil && err != errDiskNotFound {
return nil, err
}
storageDisks[index] = storage
}
return storageDisks, nil
}
// Wrap disks into retryable disks.
func initRetryableStorageDisks(disks []StorageAPI, retryUnit, retryCap, retryInterval time.Duration, retryThreshold int) (outDisks []StorageAPI) {
// Initialize the disk into a retryable-disks wrapper.
outDisks = make([]StorageAPI, len(disks))
for i, disk := range disks {
outDisks[i] = &retryStorage{
remoteStorage: disk,
retryInterval: retryInterval,
maxRetryAttempts: retryThreshold,
retryUnit: retryUnit,
retryCap: retryCap,
offlineTimestamp: UTCNow(), // Set timestamp to prevent immediate marking as offline
}
}
return
}
// Format disks before initialization of object layer.
func waitForFormatXLDisks(firstDisk bool, endpoints EndpointList, storageDisks []StorageAPI) (formattedDisks []StorageAPI, err error) {
if len(endpoints) == 0 {
return nil, errInvalidArgument
}
if storageDisks == nil {
return nil, errInvalidArgument
}
// Retryable disks before formatting, we need to have a larger
// retry window (30 seconds, with once-per-second retries) so
// that we wait enough amount of time before the disks come
// online.
retryDisks := initRetryableStorageDisks(storageDisks, time.Second, time.Second*30,
globalStorageInitHealthCheckInterval, globalStorageInitRetryThreshold)
// Start retry loop retrying until disks are formatted
// properly, until we have reached a conditional quorum of
// formatted disks.
if err = retryFormattingXLDisks(firstDisk, endpoints, retryDisks); err != nil {
return nil, err
}
// Initialize the disk into a formatted disks wrapper. This
// uses a shorter retry window (5ms with once-per-ms retries)
formattedDisks = initRetryableStorageDisks(storageDisks, time.Millisecond, time.Millisecond*5,
globalStorageHealthCheckInterval, globalStorageRetryThreshold)
// Success.
return formattedDisks, nil
}

View File

@@ -1,205 +0,0 @@
/*
* Minio Cloud Storage, (C) 2016 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 (
"os"
"testing"
)
func (action InitActions) String() string {
switch action {
case InitObjectLayer:
return "InitObjectLayer"
case FormatDisks:
return "FormatDisks"
case WaitForFormatting:
return "WaitForFormatting"
case SuggestToHeal:
return "SuggestToHeal"
case WaitForAll:
return "WaitForAll"
case WaitForQuorum:
return "WaitForQuorum"
case WaitForConfig:
return "WaitForConfig"
case Abort:
return "Abort"
default:
return "Unknown"
}
}
func TestReduceInitXLErrs(t *testing.T) {
_, fsDirs, err := prepareXL(4)
if err != nil {
t.Fatalf("Unable to initialize 'XL' object layer.")
}
// Remove all dirs.
for _, dir := range fsDirs {
defer os.RemoveAll(dir)
}
storageDisks, err := initStorageDisks(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal("Unexpected error: ", err)
}
testCases := []struct {
sErrs []error
expectedErr string
}{
{[]error{nil, nil, nil, nil}, ""},
{[]error{errUnformattedDisk, nil, nil, nil}, "\n[01/04] " + storageDisks[0].String() + " : unformatted disk found"},
{[]error{errUnformattedDisk, errUnformattedDisk, nil, nil}, "\n[01/04] " + storageDisks[0].String() + " : unformatted disk found" + "\n[02/04] " + storageDisks[1].String() + " : unformatted disk found"},
{[]error{errUnformattedDisk, errUnformattedDisk, errRPCAPIVersionUnsupported, nil}, storageDisks[2].String() + ": Unsupported rpc API version"},
}
for i, test := range testCases {
actual := reduceInitXLErrs(storageDisks, test.sErrs)
if test.expectedErr == "" && actual != nil {
t.Errorf("Test %d expected no error but received `%s`", i+1, actual.Error())
}
if test.expectedErr != "" && actual.Error() != test.expectedErr {
t.Errorf("Test %d expected `%s` but received `%s`", i+1, test.expectedErr, actual.Error())
}
}
}
func TestPrepForInitXL(t *testing.T) {
// All disks are unformatted, a fresh setup.
allUnformatted := []error{
errUnformattedDisk, errUnformattedDisk, errUnformattedDisk, errUnformattedDisk,
errUnformattedDisk, errUnformattedDisk, errUnformattedDisk, errUnformattedDisk,
}
// All disks are formatted, possible restart of a node in a formatted setup.
allFormatted := []error{
nil, nil, nil, nil,
nil, nil, nil, nil,
}
// Quorum number of disks are formatted and rest are offline.
quorumFormatted := []error{
nil, nil, nil, nil,
nil, errDiskNotFound, errDiskNotFound, errDiskNotFound,
}
// Minority disks are corrupted, can be healed.
minorityCorrupted := []error{
errCorruptedFormat, errCorruptedFormat, errCorruptedFormat, nil,
nil, nil, nil, nil,
}
// Majority disks are corrupted, pretty bad setup.
majorityCorrupted := []error{
errCorruptedFormat, errCorruptedFormat, errCorruptedFormat, errCorruptedFormat,
errCorruptedFormat, nil, nil, nil,
}
// Quorum disks are unformatted, remaining yet to come online.
quorumUnformatted := []error{
errUnformattedDisk, errUnformattedDisk, errUnformattedDisk, errUnformattedDisk,
errUnformattedDisk, errDiskNotFound, errDiskNotFound, errDiskNotFound,
}
quorumUnformattedSomeCorrupted := []error{
errUnformattedDisk, errUnformattedDisk, errUnformattedDisk, errUnformattedDisk,
errUnformattedDisk, errCorruptedFormat, errCorruptedFormat, errDiskNotFound,
}
// Quorum number of disks not online yet.
noQuourm := []error{
errDiskNotFound, errDiskNotFound, errDiskNotFound, errDiskNotFound,
errDiskNotFound, nil, nil, nil,
}
// Invalid access key id.
accessKeyIDErr := []error{
errInvalidAccessKeyID, errInvalidAccessKeyID, errInvalidAccessKeyID, errInvalidAccessKeyID,
errInvalidAccessKeyID, nil, nil, nil,
}
// Authentication error.
authenticationErr := []error{
nil, nil, nil, errAuthentication,
errAuthentication, errAuthentication, errAuthentication, errAuthentication,
}
// Unsupported rpc API version.
rpcUnsupportedVersion := []error{
errRPCAPIVersionUnsupported, errRPCAPIVersionUnsupported, errRPCAPIVersionUnsupported, errRPCAPIVersionUnsupported,
errRPCAPIVersionUnsupported, nil, nil, nil,
}
// Server time mismatch.
serverTimeMismatch := []error{
errServerTimeMismatch, errServerTimeMismatch, errServerTimeMismatch, errServerTimeMismatch,
errServerTimeMismatch, nil, nil, nil,
}
// Collection of config errs.
configErrs := []error{
errServerTimeMismatch, errServerTimeMismatch, errRPCAPIVersionUnsupported, errAuthentication,
errInvalidAccessKeyID, nil, nil, nil,
}
// Suggest to heal under formatted disks in quorum.
formattedDisksInQuorum := []error{
nil, nil, nil, nil,
errUnformattedDisk, errUnformattedDisk, errDiskNotFound, errDiskNotFound,
}
// Wait for all under undecisive state.
undecisiveErrs1 := []error{
errDiskNotFound, nil, nil, nil,
errUnformattedDisk, errUnformattedDisk, errDiskNotFound, errDiskNotFound,
}
undecisiveErrs2 := []error{
errDiskNotFound, errDiskNotFound, errDiskNotFound, errDiskNotFound,
errUnformattedDisk, errUnformattedDisk, errUnformattedDisk, errUnformattedDisk,
}
testCases := []struct {
// Params for prepForInit().
firstDisk bool
errs []error
diskCount int
action InitActions
}{
// Local disks.
{true, allFormatted, 8, InitObjectLayer},
{true, quorumFormatted, 8, InitObjectLayer},
{true, allUnformatted, 8, FormatDisks},
{true, quorumUnformatted, 8, WaitForAll},
{true, quorumUnformattedSomeCorrupted, 8, Abort},
{true, noQuourm, 8, WaitForQuorum},
{true, minorityCorrupted, 8, SuggestToHeal},
{true, majorityCorrupted, 8, Abort},
// Remote disks.
{false, allFormatted, 8, InitObjectLayer},
{false, quorumFormatted, 8, InitObjectLayer},
{false, allUnformatted, 8, WaitForFormatting},
{false, quorumUnformatted, 8, WaitForAll},
{false, quorumUnformattedSomeCorrupted, 8, Abort},
{false, noQuourm, 8, WaitForQuorum},
{false, minorityCorrupted, 8, SuggestToHeal},
{false, formattedDisksInQuorum, 8, SuggestToHeal},
{false, majorityCorrupted, 8, Abort},
{false, undecisiveErrs1, 8, WaitForAll},
{false, undecisiveErrs2, 8, WaitForAll},
// Config mistakes.
{true, accessKeyIDErr, 8, WaitForConfig},
{true, authenticationErr, 8, WaitForConfig},
{true, rpcUnsupportedVersion, 8, WaitForConfig},
{true, serverTimeMismatch, 8, WaitForConfig},
{true, configErrs, 8, WaitForConfig},
}
for i, test := range testCases {
actual := prepForInitXL(test.firstDisk, test.errs, test.diskCount)
if actual != test.action {
t.Errorf("Test %d expected %s but received %s\n", i+1, test.action, actual)
}
}
}

View File

@@ -1,326 +0,0 @@
/*
* Minio Cloud Storage, (C) 2016 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 (
"time"
"github.com/minio/minio/pkg/disk"
)
const (
// NOTE: Values indicated here are based on manual testing and
// for best case scenarios under wide array of setups. If you
// encounter changes in future feel free to change these values.
// Attempt to retry only this many number of times before
// giving up on the remote disk entirely during initialization.
globalStorageInitRetryThreshold = 2
// Attempt to retry only this many number of times before
// giving up on the remote disk entirely after initialization.
globalStorageRetryThreshold = 1
// Interval to check health status of a node whether it has
// come back up online during initialization.
globalStorageInitHealthCheckInterval = 15 * time.Minute
// Interval to check health status of a node whether it has
// come back up online.
globalStorageHealthCheckInterval = 5 * time.Minute
)
// Converts rpc.ServerError to underlying error. This function is
// written so that the storageAPI errors are consistent across network
// disks as well.
func retryToStorageErr(err error) error {
if err == errDiskNotFoundFromNetError || err == errDiskNotFoundFromRPCShutdown {
return errDiskNotFound
}
return err
}
// Retry storage is an instance of StorageAPI which
// additionally verifies upon network shutdown if the
// underlying storage is available and is really
// formatted. After the initialization phase it will
// also cache when the underlying storage is offline
// to prevent needless calls and recheck the health of
// underlying storage in regular intervals.
type retryStorage struct {
remoteStorage StorageAPI
maxRetryAttempts int
retryInterval time.Duration
retryUnit time.Duration
retryCap time.Duration
offline bool // Mark whether node is offline
offlineTimestamp time.Time // Last timestamp of checking status of node
}
// String representation of remoteStorage.
func (f *retryStorage) String() string {
return f.remoteStorage.String()
}
// Reconnects to underlying remote storage.
func (f *retryStorage) Init() (err error) {
return retryToStorageErr(f.remoteStorage.Init())
}
// Closes the underlying remote storage connection.
func (f *retryStorage) Close() (err error) {
return retryToStorageErr(f.remoteStorage.Close())
}
// Return whether the underlying remote storage is offline
// and, if so, try to reconnect at regular intervals to
// restore the connection
func (f *retryStorage) IsOffline() bool {
// Check if offline and whether enough time has lapsed since most recent check
if f.offline && UTCNow().Sub(f.offlineTimestamp) >= f.retryInterval {
f.offlineTimestamp = UTCNow() // reset timestamp
if e := f.reInit(nil); e == nil {
// Connection has been re-established
f.offline = false // Mark node as back online
}
}
return f.offline
}
// DiskInfo - a retryable implementation of disk info.
func (f *retryStorage) DiskInfo() (info disk.Info, err error) {
if f.IsOffline() {
return info, errDiskNotFound
}
info, err = f.remoteStorage.DiskInfo()
if f.reInitUponDiskNotFound(err) {
info, err = f.remoteStorage.DiskInfo()
return info, retryToStorageErr(err)
}
return info, retryToStorageErr(err)
}
// MakeVol - a retryable implementation of creating a volume.
func (f *retryStorage) MakeVol(volume string) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.MakeVol(volume)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.MakeVol(volume))
}
return retryToStorageErr(err)
}
// ListVols - a retryable implementation of listing all the volumes.
func (f *retryStorage) ListVols() (vols []VolInfo, err error) {
if f.IsOffline() {
return vols, errDiskNotFound
}
vols, err = f.remoteStorage.ListVols()
if f.reInitUponDiskNotFound(err) {
vols, err = f.remoteStorage.ListVols()
return vols, retryToStorageErr(err)
}
return vols, retryToStorageErr(err)
}
// StatVol - a retryable implementation of stating a volume.
func (f *retryStorage) StatVol(volume string) (vol VolInfo, err error) {
if f.IsOffline() {
return vol, errDiskNotFound
}
vol, err = f.remoteStorage.StatVol(volume)
if f.reInitUponDiskNotFound(err) {
vol, err = f.remoteStorage.StatVol(volume)
return vol, retryToStorageErr(err)
}
return vol, retryToStorageErr(err)
}
// DeleteVol - a retryable implementation of deleting a volume.
func (f *retryStorage) DeleteVol(volume string) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.DeleteVol(volume)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.DeleteVol(volume))
}
return retryToStorageErr(err)
}
// PrepareFile - a retryable implementation of preparing a file.
func (f *retryStorage) PrepareFile(volume, path string, length int64) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.PrepareFile(volume, path, length)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.PrepareFile(volume, path, length))
}
return retryToStorageErr(err)
}
// AppendFile - a retryable implementation of append to a file.
func (f *retryStorage) AppendFile(volume, path string, buffer []byte) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.AppendFile(volume, path, buffer)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.AppendFile(volume, path, buffer))
}
return retryToStorageErr(err)
}
// StatFile - a retryable implementation of stating a file.
func (f *retryStorage) StatFile(volume, path string) (fileInfo FileInfo, err error) {
if f.IsOffline() {
return fileInfo, errDiskNotFound
}
fileInfo, err = f.remoteStorage.StatFile(volume, path)
if f.reInitUponDiskNotFound(err) {
fileInfo, err = f.remoteStorage.StatFile(volume, path)
return fileInfo, retryToStorageErr(err)
}
return fileInfo, retryToStorageErr(err)
}
// ReadAll - a retryable implementation of reading all the content from a file.
func (f *retryStorage) ReadAll(volume, path string) (buf []byte, err error) {
if f.IsOffline() {
return buf, errDiskNotFound
}
buf, err = f.remoteStorage.ReadAll(volume, path)
if f.reInitUponDiskNotFound(err) {
buf, err = f.remoteStorage.ReadAll(volume, path)
return buf, retryToStorageErr(err)
}
return buf, retryToStorageErr(err)
}
// ReadFile - a retryable implementation of reading at offset from a file.
func (f *retryStorage) ReadFile(volume, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (m int64, err error) {
if f.IsOffline() {
return m, errDiskNotFound
}
m, err = f.remoteStorage.ReadFile(volume, path, offset, buffer, verifier)
if f.reInitUponDiskNotFound(err) {
m, err = f.remoteStorage.ReadFile(volume, path, offset, buffer, verifier)
return m, retryToStorageErr(err)
}
return m, retryToStorageErr(err)
}
// ListDir - a retryable implementation of listing directory entries.
func (f *retryStorage) ListDir(volume, path string) (entries []string, err error) {
if f.IsOffline() {
return entries, errDiskNotFound
}
entries, err = f.remoteStorage.ListDir(volume, path)
if f.reInitUponDiskNotFound(err) {
entries, err = f.remoteStorage.ListDir(volume, path)
return entries, retryToStorageErr(err)
}
return entries, retryToStorageErr(err)
}
// DeleteFile - a retryable implementation of deleting a file.
func (f *retryStorage) DeleteFile(volume, path string) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.DeleteFile(volume, path)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.DeleteFile(volume, path))
}
return retryToStorageErr(err)
}
// RenameFile - a retryable implementation of renaming a file.
func (f *retryStorage) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
if f.IsOffline() {
return errDiskNotFound
}
err = f.remoteStorage.RenameFile(srcVolume, srcPath, dstVolume, dstPath)
if f.reInitUponDiskNotFound(err) {
return retryToStorageErr(f.remoteStorage.RenameFile(srcVolume, srcPath, dstVolume, dstPath))
}
return retryToStorageErr(err)
}
// Try to reinitialize the connection when we have some form of DiskNotFound error
func (f *retryStorage) reInitUponDiskNotFound(err error) bool {
if err == errDiskNotFound || err == errDiskNotFoundFromNetError || err == errDiskNotFoundFromRPCShutdown {
return f.reInit(err) == nil
}
return false
}
// Connect and attempt to load the format from a disconnected node.
// Additionally upon failure, we retry maxRetryAttempts times before
// giving up. Essentially as a whole it would mean we are infact
// performing 1 + maxRetryAttempts times reInit.
func (f *retryStorage) reInit(e error) (err error) {
// Check whether node has gone offline.
if UTCNow().Sub(f.offlineTimestamp) >= f.retryInterval {
if e == errDiskNotFoundFromNetError { // Make node offline due to network error
f.offline = true // Marking node offline
f.offlineTimestamp = UTCNow()
return errDiskNotFound
}
// Continue for other errors like RPC shutdown (and retry connection below)
}
// Close the underlying connection.
f.remoteStorage.Close() // Error here is purposefully ignored.
// Done channel is used to close any lingering retry routine, as soon
// as this function returns.
doneCh := make(chan struct{})
defer close(doneCh)
for i := range newRetryTimer(f.retryUnit, f.retryCap, doneCh) {
// Initialize and make a new login attempt.
err = f.remoteStorage.Init()
if err != nil {
// No need to return error until the retry count
// threshold has reached.
if i < f.maxRetryAttempts {
continue
}
return err
}
// Attempt to load format to see if the disk is really
// a formatted disk and part of the cluster.
if _, err = loadFormat(f.remoteStorage); err != nil {
// No need to return error until the retry count
// threshold has reached.
if i < f.maxRetryAttempts {
continue
}
return err
}
// Login and loading format was a success, break and proceed forward.
break
}
return err
}

View File

@@ -1,455 +0,0 @@
/*
* Minio Cloud Storage, (C) 2016, 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 (
"bytes"
"errors"
"os"
"reflect"
"testing"
"time"
sha256 "github.com/minio/sha256-simd"
)
// Tests retry storage.
func TestRetryStorage(t *testing.T) {
root, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(root)
originalStorageDisks, disks := prepareXLStorageDisks(t)
defer removeRoots(disks)
var storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
// Validate all the conditions for retrying calls.
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
err = disk.Init()
if err != errDiskNotFound {
t.Fatal("Expected errDiskNotFound, got", err)
}
}
for _, disk := range storageDisks {
_, err = disk.DiskInfo()
if err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.MakeVol("existent"); err != nil {
t.Fatal(err)
}
if _, err = disk.StatVol("existent"); err == errVolumeNotFound {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if _, err = disk.StatVol("existent"); err == errVolumeNotFound {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if _, err = disk.ListVols(); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.DeleteVol("existent"); err != nil {
t.Fatal(err)
}
if str := disk.String(); str == "" {
t.Fatal("String method for disk cannot be empty.")
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.MakeVol("existent"); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.PrepareFile("existent", "path", 10); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.AppendFile("existent", "path", []byte("Hello, World")); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
var buf1 []byte
if buf1, err = disk.ReadAll("existent", "path"); err != nil {
t.Fatal(err)
}
if !bytes.Equal(buf1, []byte("Hello, World")) {
t.Fatalf("Expected `Hello, World`, got %s", string(buf1))
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
var buf2 = make([]byte, 5)
var n int64
if n, err = disk.ReadFile("existent", "path", 7, buf2, nil); err != nil {
t.Fatal(err)
}
if err != nil {
t.Error("Error in ReadFile", err)
}
if n != 5 {
t.Fatalf("Expected 5, got %d", n)
}
if !bytes.Equal(buf2, []byte("World")) {
t.Fatalf("Expected `World`, got %s", string(buf2))
}
}
sha256Hash := func(b []byte) []byte {
k := sha256.Sum256(b)
return k[:]
}
for _, disk := range storageDisks {
var buf2 = make([]byte, 5)
verifier := NewBitrotVerifier(SHA256, sha256Hash([]byte("Hello, World")))
var n int64
if n, err = disk.ReadFile("existent", "path", 7, buf2, verifier); err != nil {
t.Fatal(err)
}
if err != nil {
t.Error("Error in ReadFile with bitrot verification", err)
}
if n != 5 {
t.Fatalf("Expected 5, got %d", n)
}
if !bytes.Equal(buf2, []byte("World")) {
t.Fatalf("Expected `World`, got %s", string(buf2))
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.RenameFile("existent", "path", "existent", "new-path"); err != nil {
t.Fatal(err)
}
if _, err = disk.StatFile("existent", "new-path"); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if _, err = disk.StatFile("existent", "new-path"); err != nil {
t.Fatal(err)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
var entries []string
if entries, err = disk.ListDir("existent", ""); err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(entries, []string{"new-path"}) {
t.Fatalf("Expected []string{\"new-path\"}, got %s", entries)
}
}
storageDisks = make([]StorageAPI, len(originalStorageDisks))
for i := range originalStorageDisks {
retryDisk, ok := originalStorageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
storageDisks[i] = &retryStorage{
remoteStorage: newNaughtyDisk(retryDisk, map[int]error{
1: errDiskNotFound,
}, nil),
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}
}
for _, disk := range storageDisks {
if err = disk.DeleteFile("existent", "new-path"); err != nil {
t.Fatal(err)
}
if err = disk.DeleteVol("existent"); err != nil {
t.Fatal(err)
}
}
}
// Tests reply storage error transformation.
func TestReplyStorageErr(t *testing.T) {
unknownErr := errors.New("Unknown error")
testCases := []struct {
expectedErr error
err error
}{
{
expectedErr: errDiskNotFound,
err: errDiskNotFoundFromNetError,
},
{
expectedErr: errDiskNotFound,
err: errDiskNotFoundFromRPCShutdown,
},
{
expectedErr: unknownErr,
err: unknownErr,
},
}
for i, testCase := range testCases {
resultErr := retryToStorageErr(testCase.err)
if testCase.expectedErr != resultErr {
t.Errorf("Test %d: Expected %s, got %s", i+1, testCase.expectedErr, resultErr)
}
}
}

View File

@@ -46,7 +46,16 @@ var serverCmd = cli.Command{
{{.HelpName}} - {{.Usage}}
USAGE:
{{.HelpName}} {{if .VisibleFlags}}[FLAGS] {{end}}PATH [PATH...]
{{.HelpName}} {{if .VisibleFlags}}[FLAGS] {{end}}DIR1 [DIR2..]
{{.HelpName}} {{if .VisibleFlags}}[FLAGS] {{end}}DIR{1...64}
DIR:
DIR points to a directory on a filesystem. When you want to combine
multiple drives into a single large system, pass one directory per
filesystem separated by space. You may also use a '...' convention
to abbreviate the directory arguments. Remote directories in a
distributed setup are encoded as HTTP(s) URIs.
{{if .VisibleFlags}}
FLAGS:
{{range .VisibleFlags}}{{.}}
@@ -72,16 +81,24 @@ EXAMPLES:
2. Start minio server bound to a specific ADDRESS:PORT.
$ {{.HelpName}} --address 192.168.1.101:9000 /home/shared
3. Start erasure coded minio server on a 12 disks server.
3. Start minio server on a 12 disks server.
$ {{.HelpName}} /mnt/export1/ /mnt/export2/ /mnt/export3/ /mnt/export4/ \
/mnt/export5/ /mnt/export6/ /mnt/export7/ /mnt/export8/ /mnt/export9/ \
/mnt/export10/ /mnt/export11/ /mnt/export12/
4. Start erasure coded distributed minio server on a 4 node setup with 1 drive each. Run following commands on all the 4 nodes.
4. Start distributed minio server on a 4 node setup with 1 drive each. Run following commands on all the 4 nodes.
$ export MINIO_ACCESS_KEY=minio
$ export MINIO_SECRET_KEY=miniostorage
$ {{.HelpName}} http://192.168.1.11/mnt/export/ http://192.168.1.12/mnt/export/ \
http://192.168.1.13/mnt/export/ http://192.168.1.14/mnt/export/
5. Start minio server on 64 disks server.
$ {{.HelpName}} /mnt/export{1...64}
6. Start distributed minio server on an 8 node setup with 8 drives each. Run following command on all the 8 nodes.
$ export MINIO_ACCESS_KEY=minio
$ export MINIO_SECRET_KEY=miniostorage
$ {{.HelpName}} http://node{1...8}.example.com/mnt/export/{1...8}
`,
}
@@ -96,8 +113,13 @@ func serverHandleCmdArgs(ctx *cli.Context) {
var setupType SetupType
var err error
globalMinioAddr, globalEndpoints, setupType, err = CreateEndpoints(serverAddr, ctx.Args()...)
if len(ctx.Args()) > serverCommandLineArgsMax {
fatalIf(errInvalidArgument, "Invalid total number of arguments (%d) passed, supported upto 32 unique arguments", len(ctx.Args()))
}
globalMinioAddr, globalEndpoints, setupType, globalXLSetCount, globalXLSetDriveCount, err = createServerEndpoints(serverAddr, ctx.Args()...)
fatalIf(err, "Invalid command line arguments server=%s, args=%s", serverAddr, ctx.Args())
globalMinioHost, globalMinioPort = mustSplitHostPort(globalMinioAddr)
if runtime.GOOS == "darwin" {
// On macOS, if a process already listens on LOCALIPADDR:PORT, net.Listen() falls back
@@ -128,7 +150,7 @@ func serverHandleEnvVars() {
// serverMain handler called for 'minio server' command.
func serverMain(ctx *cli.Context) {
if !ctx.Args().Present() || ctx.Args().First() == "help" {
if (!ctx.IsSet("sets") && !ctx.Args().Present()) || ctx.Args().First() == "help" {
cli.ShowCommandHelpAndExit(ctx, "server", 1)
}
@@ -187,7 +209,7 @@ func serverMain(ctx *cli.Context) {
// Set nodes for dsync for distributed setup.
if globalIsDistXL {
globalDsync, err = dsync.New(newDsyncNodes(globalEndpoints))
fatalIf(err, "Unable to initialize distributed locking clients")
fatalIf(err, "Unable to initialize distributed locking on %s", globalEndpoints)
}
// Initialize name space lock.
@@ -244,39 +266,17 @@ func serverMain(ctx *cli.Context) {
// Initialize object layer with the supplied disks, objectLayer is nil upon any error.
func newObjectLayer(endpoints EndpointList) (newObject ObjectLayer, err error) {
// For FS only, directly use the disk.
isFS := len(endpoints) == 1
if isFS {
// Initialize new FS object layer.
return newFSObjectLayer(endpoints[0].Path)
}
// Initialize storage disks.
storageDisks, err := initStorageDisks(endpoints)
format, err := waitForFormatXL(endpoints[0].IsLocal, endpoints, globalXLSetCount, globalXLSetDriveCount)
if err != nil {
return nil, err
}
// Wait for formatting disks for XL backend.
var formattedDisks []StorageAPI
// First disk argument check if it is local.
firstDisk := endpoints[0].IsLocal
formattedDisks, err = waitForFormatXLDisks(firstDisk, endpoints, storageDisks)
if err != nil {
return nil, err
}
// Cleanup objects that weren't successfully written into the namespace.
if err = houseKeeping(storageDisks); err != nil {
return nil, err
}
// Once XL formatted, initialize object layer.
newObject, err = newXLObjectLayer(formattedDisks)
if err != nil {
return nil, err
}
// XL initialized, return.
return newObject, nil
return newXLSets(endpoints, format, len(format.XL.Sets), len(format.XL.Sets[0]))
}

View File

@@ -51,13 +51,16 @@ func TestNewObjectLayer(t *testing.T) {
}
defer removeRoots(disks)
globalXLSetCount = 1
globalXLSetDriveCount = 16
endpoints = mustGetNewEndpointList(disks...)
obj, err = newObjectLayer(endpoints)
if err != nil {
t.Fatal("Unexpected object layer initialization error", err)
}
_, ok = obj.(*xlObjects)
_, ok = obj.(*xlSets)
if !ok {
t.Fatal("Unexpected object layer detected", reflect.TypeOf(obj))
}

View File

@@ -51,10 +51,6 @@ func printStartupMessage(apiEndPoints []string) {
objAPI := newObjectLayerFn()
if objAPI != nil {
printStorageInfo(objAPI.StorageInfo())
// Storage class info only printed for Erasure backend
if objAPI.StorageInfo().Backend.Type == Erasure {
printStorageClassInfoMsg(objAPI.StorageInfo())
}
}
// Prints credential, region and browser access.
@@ -184,33 +180,6 @@ func getStorageInfoMsg(storageInfo StorageInfo) string {
return msg
}
func printStorageClassInfoMsg(storageInfo StorageInfo) {
standardClassMsg := getStandardStorageClassInfoMsg(storageInfo)
rrsClassMsg := getRRSStorageClassInfoMsg(storageInfo)
storageClassMsg := fmt.Sprintf(getFormatStr(len(standardClassMsg), 3), standardClassMsg) + fmt.Sprintf(getFormatStr(len(rrsClassMsg), 3), rrsClassMsg)
// Print storage class section only if data is present
if storageClassMsg != "" {
log.Println(colorBlue("Storage Class:"))
log.Println(storageClassMsg)
}
}
func getStandardStorageClassInfoMsg(storageInfo StorageInfo) string {
var msg string
if maxDiskFailures := storageInfo.Backend.StandardSCParity - storageInfo.Backend.OfflineDisks; maxDiskFailures >= 0 {
msg += fmt.Sprintf("Objects with "+standardStorageClass+" class can withstand [%d] drive failure(s).\n", maxDiskFailures)
}
return msg
}
func getRRSStorageClassInfoMsg(storageInfo StorageInfo) string {
var msg string
if maxDiskFailures := storageInfo.Backend.RRSCParity - storageInfo.Backend.OfflineDisks; maxDiskFailures >= 0 {
msg += fmt.Sprintf("Objects with "+reducedRedundancyStorageClass+" class can withstand [%d] drive failure(s).\n", maxDiskFailures)
}
return msg
}
// Prints startup message of storage capacity and erasure information.
func printStorageInfo(storageInfo StorageInfo) {
log.Println(getStorageInfoMsg(storageInfo))

View File

@@ -31,17 +31,12 @@ import (
// Tests if we generate storage info.
func TestStorageInfoMsg(t *testing.T) {
infoStorage := StorageInfo{
Total: 10 * humanize.GiByte,
Free: 2 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 7, 1, 4, 5},
}
infoStorage := StorageInfo{}
infoStorage.Total = 10 * humanize.GiByte
infoStorage.Free = 2 * humanize.GiByte
infoStorage.Backend.Type = Erasure
infoStorage.Backend.OnlineDisks = 7
infoStorage.Backend.OfflineDisks = 1
if msg := getStorageInfoMsg(infoStorage); !strings.Contains(msg, "2.0 GiB Free, 10 GiB Total") || !strings.Contains(msg, "7 Online, 1 Offline") {
t.Fatal("Unexpected storage info message, found:", msg)
@@ -155,97 +150,3 @@ func TestPrintStartupMessage(t *testing.T) {
apiEndpoints := []string{"http://127.0.0.1:9000"}
printStartupMessage(apiEndpoints)
}
func TestGetStandardStorageClassInfoMsg(t *testing.T) {
tests := []struct {
name string
args StorageInfo
want string
}{
{"1", StorageInfo{
Total: 20 * humanize.GiByte,
Free: 2 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 15, 1, 5, 3},
}, "Objects with " + standardStorageClass + " class can withstand [4] drive failure(s).\n"},
{"2", StorageInfo{
Total: 30 * humanize.GiByte,
Free: 3 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 10, 0, 5, 3},
}, "Objects with " + standardStorageClass + " class can withstand [5] drive failure(s).\n"},
{"3", StorageInfo{
Total: 15 * humanize.GiByte,
Free: 2 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 12, 3, 6, 2},
}, "Objects with " + standardStorageClass + " class can withstand [3] drive failure(s).\n"},
}
for _, tt := range tests {
if got := getStandardStorageClassInfoMsg(tt.args); got != tt.want {
t.Errorf("Test %s failed, expected %v, got %v", tt.name, tt.want, got)
}
}
}
func TestGetRRSStorageClassInfoMsg(t *testing.T) {
tests := []struct {
name string
args StorageInfo
want string
}{
{"1", StorageInfo{
Total: 20 * humanize.GiByte,
Free: 2 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 15, 1, 5, 3},
}, "Objects with " + reducedRedundancyStorageClass + " class can withstand [2] drive failure(s).\n"},
{"2", StorageInfo{
Total: 30 * humanize.GiByte,
Free: 3 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 16, 0, 5, 3},
}, "Objects with " + reducedRedundancyStorageClass + " class can withstand [3] drive failure(s).\n"},
{"3", StorageInfo{
Total: 15 * humanize.GiByte,
Free: 2 * humanize.GiByte,
Backend: struct {
Type BackendType
OnlineDisks int
OfflineDisks int
StandardSCParity int
RRSCParity int
}{Erasure, 12, 3, 6, 5},
}, "Objects with " + reducedRedundancyStorageClass + " class can withstand [2] drive failure(s).\n"},
}
for _, tt := range tests {
if got := getRRSStorageClassInfoMsg(tt.args); got != tt.want {
t.Errorf("Test %s failed, expected %v, got %v", tt.name, tt.want, got)
}
}
}

View File

@@ -126,23 +126,27 @@ func runAllTests(suite *TestSuiteCommon, c *check) {
func TestServerSuite(t *testing.T) {
testCases := []*TestSuiteCommon{
// Init and run test on FS backend with signature v4.
&TestSuiteCommon{serverType: "FS", signer: signerV4},
{serverType: "FS", signer: signerV4},
// Init and run test on FS backend with signature v2.
&TestSuiteCommon{serverType: "FS", signer: signerV2},
{serverType: "FS", signer: signerV2},
// Init and run test on FS backend, with tls enabled.
&TestSuiteCommon{serverType: "FS", signer: signerV4, secure: true},
{serverType: "FS", signer: signerV4, secure: true},
// Init and run test on XL backend.
&TestSuiteCommon{serverType: "XL", signer: signerV4},
{serverType: "XL", signer: signerV4},
// Init and run test on XLSet backend.
{serverType: "XLSet", signer: signerV4},
}
for _, testCase := range testCases {
runAllTests(testCase, &check{t, testCase.serverType})
for i, testCase := range testCases {
t.Run(fmt.Sprintf("Test: %d, ServerType: %s", i+1, testCase.serverType), func(t *testing.T) {
runAllTests(testCase, &check{t, testCase.serverType})
})
}
}
// Setting up the test suite.
// Starting the Test server with temporary FS backend.
func (s *TestSuiteCommon) SetUpSuite(c *check) {
rootPath, err := newTestConfig("us-east-1")
rootPath, err := newTestConfig(globalMinioDefaultRegion)
c.Assert(err, nil)
if s.secure {

View File

@@ -148,12 +148,12 @@ func validateParity(ssParity, rrsParity int) (err error) {
return fmt.Errorf("Reduced redundancy storage class parity %d should be greater than or equal to %d", rrsParity, minimumParityDisks)
}
if ssParity > len(globalEndpoints)/2 {
return fmt.Errorf("Standard storage class parity %d should be less than or equal to %d", ssParity, len(globalEndpoints)/2)
if ssParity > globalXLSetDriveCount/2 {
return fmt.Errorf("Standard storage class parity %d should be less than or equal to %d", ssParity, globalXLSetDriveCount/2)
}
if rrsParity > len(globalEndpoints)/2 {
return fmt.Errorf("Reduced redundancy storage class parity %d should be less than or equal to %d", rrsParity, len(globalEndpoints)/2)
if rrsParity > globalXLSetDriveCount/2 {
return fmt.Errorf("Reduced redundancy storage class parity %d should be less than or equal to %d", rrsParity, globalXLSetDriveCount/2)
}
if ssParity > 0 && rrsParity > 0 {

View File

@@ -82,19 +82,17 @@ func testValidateParity(obj ObjectLayer, instanceType string, dirs []string, t T
// Reset global storage class flags
resetGlobalStorageEnvs()
// Set globalEndpoints for a single node XL setup.
endpoints := globalEndpoints
// Set proper envs for a single node XL setup.
saveIsXL := globalIsXL
defer func() {
globalEndpoints = endpoints
globalIsXL = saveIsXL
}()
isXL := globalIsXL
defer func() {
globalIsXL = isXL
}()
globalIsXL = true
globalEndpoints = mustGetNewEndpointList(dirs...)
saveSetDriveCount := globalXLSetDriveCount
defer func() {
globalXLSetDriveCount = saveSetDriveCount
}()
globalXLSetCount = len(dirs)
tests := []struct {
rrsParity int
@@ -131,16 +129,16 @@ func testGetRedundancyCount(obj ObjectLayer, instanceType string, dirs []string,
tests := []struct {
sc string
disks []StorageAPI
disksCount int
expectedData int
expectedParity int
}{
{reducedRedundancyStorageClass, xl.storageDisks, 14, 2},
{standardStorageClass, xl.storageDisks, 8, 8},
{"", xl.storageDisks, 8, 8},
{reducedRedundancyStorageClass, xl.storageDisks, 9, 7},
{standardStorageClass, xl.storageDisks, 10, 6},
{"", xl.storageDisks, 9, 7},
{reducedRedundancyStorageClass, len(xl.storageDisks), 14, 2},
{standardStorageClass, len(xl.storageDisks), 8, 8},
{"", len(xl.storageDisks), 8, 8},
{reducedRedundancyStorageClass, len(xl.storageDisks), 9, 7},
{standardStorageClass, len(xl.storageDisks), 10, 6},
{"", len(xl.storageDisks), 9, 7},
}
for i, tt := range tests {
// Set env var for test case 4
@@ -155,7 +153,7 @@ func testGetRedundancyCount(obj ObjectLayer, instanceType string, dirs []string,
if i+1 == 6 {
globalStandardStorageClass.Parity = 7
}
data, parity := getRedundancyCount(tt.sc, len(tt.disks))
data, parity := getRedundancyCount(tt.sc, tt.disksCount)
if data != tt.expectedData {
t.Errorf("Test %d, Expected data disks %d, got %d", i+1, tt.expectedData, data)
return

View File

@@ -36,12 +36,6 @@ var errDiskFull = errors.New("disk path full")
// errDiskNotFound - cannot find the underlying configured disk anymore.
var errDiskNotFound = errors.New("disk not found")
// errDiskNotFoundFromNetError - cannot find the underlying configured disk anymore due to network error.
var errDiskNotFoundFromNetError = errors.New("disk not found from net error")
// errDiskNotFoundFromShutdown - cannot find the underlying configured disk anymore due to rpc shutdown.
var errDiskNotFoundFromRPCShutdown = errors.New("disk not found from rpc shutdown")
// errFaultyRemoteDisk - remote disk is faulty.
var errFaultyRemoteDisk = errors.New("remote disk is faulty")

View File

@@ -28,8 +28,8 @@ type StorageAPI interface {
String() string
// Storage operations.
Init() (err error)
Close() (err error)
IsOnline() bool // Returns true if disk is online.
Close() error
DiskInfo() (info disk.Info, err error)
// Volume operations.

View File

@@ -29,12 +29,26 @@ import (
type networkStorage struct {
rpcClient *AuthRPCClient
connected bool
}
const (
storageRPCPath = "/storage"
)
func isErrorNetworkDisconnect(err error) bool {
if err == nil {
return false
}
if _, ok := err.(*net.OpError); ok {
return true
}
if err == rpc.ErrShutdown {
return true
}
return false
}
// Converts rpc.ServerError to underlying error. This function is
// written so that the storageAPI errors are consistent across network
// disks as well.
@@ -43,13 +57,8 @@ func toStorageErr(err error) error {
return nil
}
switch err.(type) {
case *net.OpError:
return errDiskNotFoundFromNetError
}
if err == rpc.ErrShutdown {
return errDiskNotFoundFromRPCShutdown
if isErrorNetworkDisconnect(err) {
return errDiskNotFound
}
switch err.Error() {
@@ -99,7 +108,7 @@ func newStorageRPC(endpoint Endpoint) StorageAPI {
rpcPath := path.Join(minioReservedBucketPath, storageRPCPath, endpoint.Path)
serverCred := globalServerConfig.GetCredential()
return &networkStorage{
disk := &networkStorage{
rpcClient: newAuthRPCClient(authConfig{
accessKey: serverCred.AccessKey,
secretKey: serverCred.SecretKey,
@@ -110,6 +119,9 @@ func newStorageRPC(endpoint Endpoint) StorageAPI {
disableReconnect: true,
}),
}
// Attempt a remote login.
disk.connected = disk.rpcClient.Login() == nil
return disk
}
// Stringer provides a canonicalized representation of network device.
@@ -126,22 +138,36 @@ func (n *networkStorage) String() string {
return scheme + "://" + n.rpcClient.ServerAddr() + path.Join("/", serviceEndpoint)
}
// Init - attempts a login to reconnect.
func (n *networkStorage) Init() error {
return toStorageErr(n.rpcClient.Login())
func (n *networkStorage) Close() error {
n.connected = false
return toStorageErr(n.rpcClient.Close())
}
// Closes the underlying RPC connection.
func (n *networkStorage) Close() error {
// Close the underlying connection.
return toStorageErr(n.rpcClient.Close())
func (n *networkStorage) IsOnline() bool {
return n.connected
}
func (n *networkStorage) call(handler string, args interface {
SetAuthToken(string)
SetRPCAPIVersion(semVersion)
}, reply interface{}) error {
if !n.connected {
return errDiskNotFound
}
if err := n.rpcClient.Call(handler, args, reply); err != nil {
if isErrorNetworkDisconnect(err) {
n.connected = false
}
return toStorageErr(err)
}
return nil
}
// DiskInfo - fetch disk information for a remote disk.
func (n *networkStorage) DiskInfo() (info disk.Info, err error) {
args := AuthRPCArgs{}
if err = n.rpcClient.Call("Storage.DiskInfoHandler", &args, &info); err != nil {
return disk.Info{}, toStorageErr(err)
if err = n.call("Storage.DiskInfoHandler", &args, &info); err != nil {
return disk.Info{}, err
}
return info, nil
}
@@ -150,18 +176,14 @@ func (n *networkStorage) DiskInfo() (info disk.Info, err error) {
func (n *networkStorage) MakeVol(volume string) (err error) {
reply := AuthRPCReply{}
args := GenericVolArgs{Vol: volume}
if err := n.rpcClient.Call("Storage.MakeVolHandler", &args, &reply); err != nil {
return toStorageErr(err)
}
return nil
return n.call("Storage.MakeVolHandler", &args, &reply)
}
// ListVols - List all volumes on a remote disk.
func (n *networkStorage) ListVols() (vols []VolInfo, err error) {
ListVols := ListVolsReply{}
err = n.rpcClient.Call("Storage.ListVolsHandler", &AuthRPCArgs{}, &ListVols)
if err != nil {
return nil, toStorageErr(err)
if err = n.call("Storage.ListVolsHandler", &AuthRPCArgs{}, &ListVols); err != nil {
return nil, err
}
return ListVols.Vols, nil
}
@@ -169,8 +191,8 @@ func (n *networkStorage) ListVols() (vols []VolInfo, err error) {
// StatVol - get volume info over the network.
func (n *networkStorage) StatVol(volume string) (volInfo VolInfo, err error) {
args := GenericVolArgs{Vol: volume}
if err = n.rpcClient.Call("Storage.StatVolHandler", &args, &volInfo); err != nil {
return VolInfo{}, toStorageErr(err)
if err = n.call("Storage.StatVolHandler", &args, &volInfo); err != nil {
return VolInfo{}, err
}
return volInfo, nil
}
@@ -179,46 +201,37 @@ func (n *networkStorage) StatVol(volume string) (volInfo VolInfo, err error) {
func (n *networkStorage) DeleteVol(volume string) (err error) {
reply := AuthRPCReply{}
args := GenericVolArgs{Vol: volume}
if err := n.rpcClient.Call("Storage.DeleteVolHandler", &args, &reply); err != nil {
return toStorageErr(err)
}
return nil
return n.call("Storage.DeleteVolHandler", &args, &reply)
}
// File operations.
func (n *networkStorage) PrepareFile(volume, path string, length int64) (err error) {
reply := AuthRPCReply{}
if err = n.rpcClient.Call("Storage.PrepareFileHandler", &PrepareFileArgs{
return n.call("Storage.PrepareFileHandler", &PrepareFileArgs{
Vol: volume,
Path: path,
Size: length,
}, &reply); err != nil {
return toStorageErr(err)
}
return nil
}, &reply)
}
// AppendFile - append file writes buffer to a remote network path.
func (n *networkStorage) AppendFile(volume, path string, buffer []byte) (err error) {
reply := AuthRPCReply{}
if err = n.rpcClient.Call("Storage.AppendFileHandler", &AppendFileArgs{
return n.call("Storage.AppendFileHandler", &AppendFileArgs{
Vol: volume,
Path: path,
Buffer: buffer,
}, &reply); err != nil {
return toStorageErr(err)
}
return nil
}, &reply)
}
// StatFile - get latest Stat information for a file at path.
func (n *networkStorage) StatFile(volume, path string) (fileInfo FileInfo, err error) {
if err = n.rpcClient.Call("Storage.StatFileHandler", &StatFileArgs{
if err = n.call("Storage.StatFileHandler", &StatFileArgs{
Vol: volume,
Path: path,
}, &fileInfo); err != nil {
return FileInfo{}, toStorageErr(err)
return FileInfo{}, err
}
return fileInfo, nil
}
@@ -228,11 +241,11 @@ func (n *networkStorage) StatFile(volume, path string) (fileInfo FileInfo, err e
// This API is meant to be used on files which have small memory footprint, do
// not use this on large files as it would cause server to crash.
func (n *networkStorage) ReadAll(volume, path string) (buf []byte, err error) {
if err = n.rpcClient.Call("Storage.ReadAllHandler", &ReadAllArgs{
if err = n.call("Storage.ReadAllHandler", &ReadAllArgs{
Vol: volume,
Path: path,
}, &buf); err != nil {
return nil, toStorageErr(err)
return nil, err
}
return buf, nil
}
@@ -260,22 +273,22 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
}
var result []byte
err = n.rpcClient.Call("Storage.ReadFileHandler", &args, &result)
err = n.call("Storage.ReadFileHandler", &args, &result)
// Copy results to buffer.
copy(buffer, result)
// Return length of result, err if any.
return int64(len(result)), toStorageErr(err)
return int64(len(result)), err
}
// ListDir - list all entries at prefix.
func (n *networkStorage) ListDir(volume, path string) (entries []string, err error) {
if err = n.rpcClient.Call("Storage.ListDirHandler", &ListDirArgs{
if err = n.call("Storage.ListDirHandler", &ListDirArgs{
Vol: volume,
Path: path,
}, &entries); err != nil {
return nil, toStorageErr(err)
return nil, err
}
// Return successfully unmarshalled results.
return entries, nil
@@ -284,25 +297,19 @@ func (n *networkStorage) ListDir(volume, path string) (entries []string, err err
// DeleteFile - Delete a file at path.
func (n *networkStorage) DeleteFile(volume, path string) (err error) {
reply := AuthRPCReply{}
if err = n.rpcClient.Call("Storage.DeleteFileHandler", &DeleteFileArgs{
return n.call("Storage.DeleteFileHandler", &DeleteFileArgs{
Vol: volume,
Path: path,
}, &reply); err != nil {
return toStorageErr(err)
}
return nil
}, &reply)
}
// RenameFile - rename a remote file from source to destination.
func (n *networkStorage) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
reply := AuthRPCReply{}
if err = n.rpcClient.Call("Storage.RenameFileHandler", &RenameFileArgs{
return n.call("Storage.RenameFileHandler", &RenameFileArgs{
SrcVol: srcVolume,
SrcPath: srcPath,
DstVol: dstVolume,
DstPath: dstPath,
}, &reply); err != nil {
return toStorageErr(err)
}
return nil
}, &reply)
}

View File

@@ -126,11 +126,11 @@ func TestStorageErr(t *testing.T) {
err: fmt.Errorf("%s", io.ErrUnexpectedEOF.Error()),
},
{
expectedErr: errDiskNotFoundFromNetError,
expectedErr: errDiskNotFound,
err: &net.OpError{},
},
{
expectedErr: errDiskNotFoundFromRPCShutdown,
expectedErr: errDiskNotFound,
err: rpc.ErrShutdown,
},
{

View File

@@ -55,6 +55,7 @@ import (
"github.com/minio/minio-go/pkg/policy"
"github.com/minio/minio-go/pkg/s3signer"
"github.com/minio/minio/pkg/auth"
"github.com/minio/minio/pkg/bpool"
"github.com/minio/minio/pkg/hash"
)
@@ -71,6 +72,9 @@ func init() {
// Set system resources to maximum.
setMaxResources()
log = NewLogger()
log.EnableQuiet()
}
// concurreny level for certain parallel tests.
@@ -166,6 +170,36 @@ func prepareFS() (ObjectLayer, string, error) {
return obj, fsDirs[0], nil
}
func prepareXL32() (ObjectLayer, []string, error) {
fsDirs1, err := getRandomDisks(16)
if err != nil {
return nil, nil, err
}
endpoints1 := mustGetNewEndpointList(fsDirs1...)
fsDirs2, err := getRandomDisks(16)
if err != nil {
removeRoots(fsDirs1)
return nil, nil, err
}
endpoints2 := mustGetNewEndpointList(fsDirs2...)
endpoints := append(endpoints1, endpoints2...)
fsDirs := append(fsDirs1, fsDirs2...)
format, err := waitForFormatXL(true, endpoints, 2, 16)
if err != nil {
removeRoots(fsDirs)
return nil, nil, err
}
objAPI, err := newXLSets(endpoints, format, 2, 16)
if err != nil {
return nil, nil, err
}
return objAPI, fsDirs, nil
}
func prepareXL(nDisks int) (ObjectLayer, []string, error) {
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
@@ -211,6 +245,9 @@ const (
// XLTestStr is the string which is used as notation for XL ObjectLayer in the unit tests.
XLTestStr string = "XL"
// XLSetsTestStr is the string which is used as notation for XL sets object layer in the unit tests.
XLSetsTestStr string = "XLSet"
)
const letterBytes = "abcdefghijklmnopqrstuvwxyz01234569"
@@ -290,7 +327,9 @@ func UnstartedTestServer(t TestErrHandler, instanceType string) TestServer {
credentials := globalServerConfig.GetCredential()
testServer.Obj = objLayer
testServer.Disks = mustGetNewEndpointList(disks...)
for _, disk := range disks {
testServer.Disks = append(testServer.Disks, mustGetNewEndpointList(disk)...)
}
testServer.Root = root
testServer.AccessKey = credentials.AccessKey
testServer.SecretKey = credentials.SecretKey
@@ -1640,21 +1679,65 @@ func getRandomDisks(N int) ([]string, error) {
return erasureDisks, nil
}
// initObjectLayer - Instantiates object layer and returns it.
func initObjectLayer(endpoints EndpointList) (ObjectLayer, []StorageAPI, error) {
// Initialize object layer with the supplied disks, objectLayer is nil upon any error.
func newTestObjectLayer(endpoints EndpointList) (newObject ObjectLayer, err error) {
// For FS only, directly use the disk.
isFS := len(endpoints) == 1
if isFS {
// Initialize new FS object layer.
return newFSObjectLayer(endpoints[0].Path)
}
_, err = waitForFormatXL(endpoints[0].IsLocal, endpoints, 1, 16)
if err != nil {
return nil, err
}
storageDisks, err := initStorageDisks(endpoints)
if err != nil {
return nil, nil, err
return nil, err
}
formattedDisks, err := waitForFormatXLDisks(true, endpoints, storageDisks)
// Initialize list pool.
listPool := newTreeWalkPool(globalLookupTimeout)
// Initialize xl objects.
xl := &xlObjects{
listPool: listPool,
storageDisks: storageDisks,
nsMutex: newNSLock(false),
bp: bpool.NewBytePoolCap(4, blockSizeV1, blockSizeV1*2),
}
xl.getDisks = func() []StorageAPI {
return xl.storageDisks
}
// Initialize and load bucket policies.
xl.bucketPolicies, err = initBucketPolicies(xl)
if err != nil {
return nil, err
}
// Initialize a new event notifier.
if err = initEventNotifier(xl); err != nil {
return nil, err
}
return xl, nil
}
// initObjectLayer - Instantiates object layer and returns it.
func initObjectLayer(endpoints EndpointList) (ObjectLayer, []StorageAPI, error) {
objLayer, err := newTestObjectLayer(endpoints)
if err != nil {
return nil, nil, err
}
objLayer, err := newXLObjectLayer(formattedDisks)
if err != nil {
return nil, nil, err
var formattedDisks []StorageAPI
// Should use the object layer tests for validating cache.
if xl, ok := objLayer.(*xlObjects); ok {
formattedDisks = xl.storageDisks
}
// Success.
@@ -1678,13 +1761,6 @@ func removeDiskN(disks []string, n int) {
}
}
// Makes a entire new copy of a StorageAPI slice.
func deepCopyStorageDisks(storageDisks []StorageAPI) []StorageAPI {
newStorageDisks := make([]StorageAPI, len(storageDisks))
copy(newStorageDisks, storageDisks)
return newStorageDisks
}
// Initializes storage disks with 'N' errored disks, N disks return 'err' for each disk access.
func prepareNErroredDisks(storageDisks []StorageAPI, offline int, err error, t *testing.T) []StorageAPI {
if offline > len(storageDisks) {
@@ -1692,37 +1768,11 @@ func prepareNErroredDisks(storageDisks []StorageAPI, offline int, err error, t *
}
for i := 0; i < offline; i++ {
storageDisks[i] = &naughtyDisk{disk: &retryStorage{
remoteStorage: storageDisks[i],
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
}, defaultErr: err}
storageDisks[i] = &naughtyDisk{disk: storageDisks[i], defaultErr: err}
}
return storageDisks
}
// Initializes storage disks with 'N' offline disks, N disks returns 'errDiskNotFound' for each disk access.
func prepareNOfflineDisks(storageDisks []StorageAPI, offline int, t *testing.T) []StorageAPI {
return prepareNErroredDisks(storageDisks, offline, errDiskNotFound, t)
}
// Initializes backend storage disks.
func prepareXLStorageDisks(t *testing.T) ([]StorageAPI, []string) {
nDisks := 16
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal("Unexpected error: ", err)
}
_, storageDisks, err := initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
removeRoots(fsDirs)
t.Fatal("Unable to initialize storage disks", err)
}
return storageDisks, fsDirs
}
// creates a bucket for the tests and returns the bucket name.
// initializes the specified API endpoints for the tests.
// initialies the root and returns its path.
@@ -1749,10 +1799,13 @@ func initAPIHandlerTest(obj ObjectLayer, endpoints []string) (string, http.Handl
}
// prepare test backend.
// create FS/XL bankend.
// create FS/XL/XLSet backend.
// return object layer, backend disks.
func prepareTestBackend(instanceType string) (ObjectLayer, []string, error) {
switch instanceType {
// Total number of disks for XL sets backend is set to 32.
case XLSetsTestStr:
return prepareXL32()
// Total number of disks for XL backend is set to 16.
case XLTestStr:
return prepareXL16()
@@ -2388,7 +2441,6 @@ func mustGetNewEndpointList(args ...string) (endpoints EndpointList) {
endpoints, err = NewEndpointList(args...)
fatalIf(err, "unable to create new endpoint list")
}
return endpoints
}

View File

@@ -1667,7 +1667,7 @@ func TestWebObjectLayerFaultyDisks(t *testing.T) {
// Set faulty disks to XL backend
xl := obj.(*xlObjects)
for i, d := range xl.storageDisks {
xl.storageDisks[i] = newNaughtyDisk(d.(*retryStorage), nil, errFaultyDisk)
xl.storageDisks[i] = newNaughtyDisk(d, nil, errFaultyDisk)
}
// Initialize web rpc endpoint.

1378
cmd/xl-sets.go Normal file

File diff suppressed because it is too large Load Diff

191
cmd/xl-sets_test.go Normal file
View File

@@ -0,0 +1,191 @@
/*
* 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 (
"os"
"path/filepath"
"testing"
)
// TestCrcHashMod - test crc hash.
func TestCrcHashMod(t *testing.T) {
testCases := []struct {
objectName string
crcHash int
}{
// cases which should pass the test.
// passing in valid object name.
{"object", 12},
{"The Shining Script <v1>.pdf", 14},
{"Cost Benefit Analysis (2009-2010).pptx", 13},
{"117Gn8rfHL2ACARPAhaFd0AGzic9pUbIA/5OCn5A", 1},
{"SHØRT", 9},
{"There are far too many object names, and far too few bucket names!", 13},
{"a/b/c/", 1},
{"/a/b/c", 4},
{string([]byte{0xff, 0xfe, 0xfd}), 13},
}
// Tests hashing order to be consistent.
for i, testCase := range testCases {
if crcHashElement := hashKey("CRCMOD", testCase.objectName, 16); crcHashElement != testCase.crcHash {
t.Errorf("Test case %d: Expected \"%v\" but failed \"%v\"", i+1, testCase.crcHash, crcHashElement)
}
}
if crcHashElement := hashKey("CRCMOD", "This will fail", -1); crcHashElement != -1 {
t.Errorf("Test: Expected \"-1\" but got \"%v\"", crcHashElement)
}
if crcHashElement := hashKey("CRCMOD", "This will fail", 0); crcHashElement != -1 {
t.Errorf("Test: Expected \"-1\" but got \"%v\"", crcHashElement)
}
if crcHashElement := hashKey("UNKNOWN", "This will fail", 0); crcHashElement != -1 {
t.Errorf("Test: Expected \"-1\" but got \"%v\"", crcHashElement)
}
}
// TestNewXL - tests initialization of all input disks
// and constructs a valid `XL` object
func TestNewXLSets(t *testing.T) {
var nDisks = 16 // Maximum disks.
var erasureDisks []string
for i := 0; i < nDisks; i++ {
// Do not attempt to create this path, the test validates
// so that newXLSets initializes non existing paths
// and successfully returns initialized object layer.
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
erasureDisks = append(erasureDisks, disk)
defer os.RemoveAll(disk)
}
endpoints := mustGetNewEndpointList(erasureDisks...)
_, err := waitForFormatXL(true, endpoints, 0, 16)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
_, err = waitForFormatXL(true, nil, 1, 16)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
// Initializes all erasure disks
format, err := waitForFormatXL(true, endpoints, 1, 16)
if err != nil {
t.Fatalf("Unable to format disks for erasure, %s", err)
}
if _, err := newXLSets(endpoints, format, 1, 16); err != nil {
t.Fatalf("Unable to initialize erasure")
}
}
// TestStorageInfoSets - tests storage info for erasure coded sets of disks.
func TestStorageInfoSets(t *testing.T) {
var nDisks = 16 // Maximum disks.
var erasureDisks []string
for i := 0; i < nDisks; i++ {
// Do not attempt to create this path, the test validates
// so that newXLSets initializes non existing paths
// and successfully returns initialized object layer.
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
erasureDisks = append(erasureDisks, disk)
defer os.RemoveAll(disk)
}
endpoints := mustGetNewEndpointList(erasureDisks...)
// Initializes all erasure disks
format, err := waitForFormatXL(true, endpoints, 1, 16)
if err != nil {
t.Fatalf("Unable to format disks for erasure, %s", err)
}
objLayer, err := newXLSets(endpoints, format, 1, 16)
if err != nil {
t.Fatal(err)
}
// Get storage info first attempt.
disks16Info := objLayer.StorageInfo()
// This test assumes homogeneity between all disks,
// i.e if we loose one disk the effective storage
// usage values is assumed to decrease. If we have
// heterogenous environment this is not true all the time.
if disks16Info.Free <= 0 {
t.Fatalf("Diskinfo total free values should be greater 0")
}
if disks16Info.Total <= 0 {
t.Fatalf("Diskinfo total values should be greater 0")
}
}
// TestHashedLayer - tests the hashed layer which will be returned
// consistently for a given object name.
func TestHashedLayer(t *testing.T) {
rootPath, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(rootPath)
var objs []*xlObjects
for i := 0; i < 16; i++ {
obj, fsDirs, err := prepareXL16()
if err != nil {
t.Fatal("Unable to initialize 'XL' object layer.", err)
}
// Remove all dirs.
for _, dir := range fsDirs {
defer os.RemoveAll(dir)
}
objs = append(objs, obj.(*xlObjects))
}
sets := &xlSets{sets: objs, distributionAlgo: "CRCMOD"}
testCases := []struct {
objectName string
expectedObj *xlObjects
}{
// cases which should pass the test.
// passing in valid object name.
{"object", objs[12]},
{"The Shining Script <v1>.pdf", objs[14]},
{"Cost Benefit Analysis (2009-2010).pptx", objs[13]},
{"117Gn8rfHL2ACARPAhaFd0AGzic9pUbIA/5OCn5A", objs[1]},
{"SHØRT", objs[9]},
{"There are far too many object names, and far too few bucket names!", objs[13]},
{"a/b/c/", objs[1]},
{"/a/b/c", objs[4]},
{string([]byte{0xff, 0xfe, 0xfd}), objs[13]},
}
// Tests hashing order to be consistent.
for i, testCase := range testCases {
gotObj := sets.getHashedSet(testCase.objectName)
if gotObj != testCase.expectedObj {
t.Errorf("Test case %d: Expected \"%#v\" but failed \"%#v\"", i+1, testCase.expectedObj, gotObj)
}
}
}

View File

@@ -35,11 +35,6 @@ var bucketMetadataOpIgnoredErrs = append(bucketOpIgnoredErrs, errVolumeNotFound)
// MakeBucket - make a bucket.
func (xl xlObjects) MakeBucketWithLocation(bucket, location string) error {
bucketLock := xl.nsMutex.NewNSLock(bucket, "")
if err := bucketLock.GetLock(globalObjectTimeout); err != nil {
return err
}
defer bucketLock.Unlock()
// Verify if bucket is valid.
if !IsValidBucketName(bucket) {
return errors.Trace(BucketNameInvalid{Bucket: bucket})
@@ -49,10 +44,10 @@ func (xl xlObjects) MakeBucketWithLocation(bucket, location string) error {
var wg = &sync.WaitGroup{}
// Initialize list of errors.
var dErrs = make([]error, len(xl.storageDisks))
var dErrs = make([]error, len(xl.getDisks()))
// Make a volume entry on all underlying storage disks.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
dErrs[index] = errors.Trace(errDiskNotFound)
continue
@@ -71,11 +66,11 @@ func (xl xlObjects) MakeBucketWithLocation(bucket, location string) error {
// Wait for all make vol to finish.
wg.Wait()
writeQuorum := len(xl.storageDisks)/2 + 1
writeQuorum := len(xl.getDisks())/2 + 1
err := reduceWriteQuorumErrs(dErrs, bucketOpIgnoredErrs, writeQuorum)
if errors.Cause(err) == errXLWriteQuorum {
// Purge successfully created buckets if we don't have writeQuorum.
undoMakeBucket(xl.storageDisks, bucket)
undoMakeBucket(xl.getDisks(), bucket)
}
return toObjectErr(err, bucket)
}
@@ -84,7 +79,7 @@ func (xl xlObjects) undoDeleteBucket(bucket string) {
// Initialize sync waitgroup.
var wg = &sync.WaitGroup{}
// Undo previous make bucket entry on all underlying storage disks.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
continue
}
@@ -150,7 +145,7 @@ func (xl xlObjects) getBucketInfo(bucketName string) (bucketInfo BucketInfo, err
// reduce to one error based on read quorum.
// `nil` is deliberately passed for ignoredErrs
// because these errors were already ignored.
readQuorum := len(xl.storageDisks) / 2
readQuorum := len(xl.getDisks()) / 2
return BucketInfo{}, reduceReadQuorumErrs(bucketErrs, nil, readQuorum)
}
@@ -240,10 +235,10 @@ func (xl xlObjects) DeleteBucket(bucket string) error {
// Collect if all disks report volume not found.
var wg = &sync.WaitGroup{}
var dErrs = make([]error, len(xl.storageDisks))
var dErrs = make([]error, len(xl.getDisks()))
// Remove a volume entry on all underlying storage disks.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
dErrs[index] = errors.Trace(errDiskNotFound)
continue
@@ -273,7 +268,8 @@ func (xl xlObjects) DeleteBucket(bucket string) error {
// Wait for all the delete vols to finish.
wg.Wait()
writeQuorum := len(xl.storageDisks)/2 + 1
writeQuorum := len(xl.getDisks())/2 + 1
err := reduceWriteQuorumErrs(dErrs, bucketOpIgnoredErrs, writeQuorum)
if errors.Cause(err) == errXLWriteQuorum {
xl.undoDeleteBucket(bucket)
@@ -282,9 +278,6 @@ func (xl xlObjects) DeleteBucket(bucket string) error {
return toObjectErr(err, bucket)
}
// Delete all bucket metadata.
deleteBucketMetadata(bucket, xl)
return nil
}

View File

@@ -25,8 +25,8 @@ import (
// getLoadBalancedDisks - fetches load balanced (sufficiently randomized) disk slice.
func (xl xlObjects) getLoadBalancedDisks() (disks []StorageAPI) {
// Based on the random shuffling return back randomized disks.
for _, i := range hashOrder(UTCNow().String(), len(xl.storageDisks)) {
disks = append(disks, xl.storageDisks[i-1])
for _, i := range hashOrder(UTCNow().String(), len(xl.getDisks())) {
disks = append(disks, xl.getDisks()[i-1])
}
return disks
}

View File

@@ -272,7 +272,6 @@ func TestListOnlineDisks(t *testing.T) {
t.Fatalf("Test %d: disk (%v) with part.1 missing is not a disk with available data",
i+1, xlDisks[tamperedIndex])
}
}
}

View File

@@ -19,115 +19,14 @@ package cmd
import (
"fmt"
"path"
"sort"
"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, dryRun bool) (res madmin.HealResultItem,
err error) {
// Attempt to load all `format.json`.
formatConfigs, sErrs := loadAllFormats(storageDisks)
// Generic format check.
// - if (no quorum) return error
// - if (disks not recognized) // Always error.
if err = genericFormatCheckXL(formatConfigs, sErrs); err != nil {
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)
_, unformattedDiskCount, diskNotFoundCount,
corruptedFormatCount, otherErrCount := formatErrsSummary(sErrs)
switch {
case unformattedDiskCount == numDisks:
// all unformatted.
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 res, fmt.Errorf("cannot proceed with heal as %s",
errSomeDiskOffline)
case otherErrCount > 0:
return res, fmt.Errorf("cannot proceed with heal as some disks had unhandled errors")
case corruptedFormatCount > 0:
// 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:
// 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 res, nil
func (xl xlObjects) HealFormat(dryRun bool) (madmin.HealResultItem, error) {
return madmin.HealResultItem{}, errors.Trace(NotImplemented{})
}
// Heals a bucket if it doesn't exist on one of the disks, additionally
@@ -141,17 +40,13 @@ func (xl xlObjects) HealBucket(bucket string, dryRun bool) (
}
// 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 nil, err
}
defer bucketLock.Unlock()
writeQuorum := len(xl.getDisks())/2 + 1
// Heal bucket.
result, err := healBucket(xl.storageDisks, bucket, writeQuorum, dryRun)
var result madmin.HealResultItem
result, err = healBucket(xl.getDisks(), bucket, writeQuorum, dryRun)
if err != nil {
return results, err
return nil, err
}
results = append(results, result)
@@ -189,10 +84,16 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int,
go func(index int, disk StorageAPI) {
defer wg.Done()
if _, err := disk.StatVol(bucket); err != nil {
if errors.Cause(err) == errDiskNotFound {
beforeState[index] = madmin.DriveStateOffline
afterState[index] = madmin.DriveStateOffline
dErrs[index] = err
return
}
if errors.Cause(err) != errVolumeNotFound {
beforeState[index] = madmin.DriveStateCorrupt
afterState[index] = madmin.DriveStateCorrupt
dErrs[index] = errors.Trace(err)
dErrs[index] = err
return
}
@@ -205,14 +106,14 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int,
}
makeErr := disk.MakeVol(bucket)
dErrs[index] = errors.Trace(makeErr)
dErrs[index] = makeErr
if makeErr == nil {
afterState[index] = madmin.DriveStateOk
}
} else {
beforeState[index] = madmin.DriveStateOk
afterState[index] = madmin.DriveStateOk
return
}
beforeState[index] = madmin.DriveStateOk
afterState[index] = madmin.DriveStateOk
}(index, disk)
}
@@ -225,11 +126,31 @@ func healBucket(storageDisks []StorageAPI, bucket string, writeQuorum int,
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]
if storageDisks[i] == nil {
res.Before.Drives = append(res.Before.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: "",
State: before,
})
res.After.Drives = append(res.After.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: "",
State: afterState[i],
})
continue
}
drive := storageDisks[i].String()
res.Before.Drives = append(res.Before.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: drive,
State: before,
})
res.After.Drives = append(res.After.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: drive,
State: afterState[i],
})
}
reducedErr := reduceWriteQuorumErrs(dErrs, bucketOpIgnoredErrs, writeQuorum)
@@ -319,62 +240,6 @@ func listAllBuckets(storageDisks []StorageAPI) (buckets map[string]VolInfo,
return buckets, bucketsOcc, 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, _, err := listAllBuckets(xl.storageDisks)
if err != nil {
return listBuckets, err
}
// Iterate over all buckets
for _, currBucket := range buckets {
listBuckets = append(listBuckets,
BucketInfo{currBucket.Name, currBucket.Created})
}
// Sort found buckets
sort.Sort(byBucketName(listBuckets))
return listBuckets, nil
}
// This function is meant for all the healing that needs to be done
// during startup i.e healing of buckets, bucket metadata (policy.json,
// notification.xml, listeners.json) etc. Currently this function
// supports quick healing of buckets, bucket metadata.
func quickHeal(xlObj xlObjects, writeQuorum int, readQuorum int) error {
// List all bucket name occurrence from all disks.
_, bucketOcc, err := listAllBuckets(xlObj.storageDisks)
if err != nil {
return err
}
// All bucket names and bucket metadata that should be healed.
for bucketName, occCount := range bucketOcc {
// Heal bucket only if healing is needed.
if occCount != len(xlObj.storageDisks) {
bucketLock := xlObj.nsMutex.NewNSLock(bucketName, "")
if perr := bucketLock.GetLock(globalHealingTimeout); perr != nil {
return perr
}
defer bucketLock.Unlock()
// Heal bucket and then proceed to heal bucket metadata if any.
if _, err = healBucket(xlObj.storageDisks, bucketName, writeQuorum, false); err == nil {
if _, err = healBucketMetadata(xlObj, bucketName, false); err == nil {
continue
}
return err
}
return err
}
}
// Success.
return nil
}
// 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) {
@@ -409,7 +274,6 @@ func healObject(storageDisks []StorageAPI, bucket string, object string,
// 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
@@ -438,10 +302,6 @@ func healObject(storageDisks []StorageAPI, bucket string, object string,
// 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
// an online disk without valid data/metadata is
// outdated and can be healed.
@@ -449,6 +309,30 @@ func healObject(storageDisks []StorageAPI, bucket string, object string,
outDatedDisks[i] = storageDisks[i]
disksToHealCount++
}
if v == nil {
result.Before.Drives = append(result.Before.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: "",
State: driveState,
})
result.After.Drives = append(result.After.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: "",
State: driveState,
})
continue
}
drive := v.String()
result.Before.Drives = append(result.Before.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: drive,
State: driveState,
})
result.After.Drives = append(result.After.Drives, madmin.HealDriveInfo{
UUID: "",
Endpoint: drive,
State: driveState,
})
}
// If less than read quorum number of disks have all the parts
@@ -591,10 +475,14 @@ func healObject(storageDisks []StorageAPI, bucket string, object string,
return result, toObjectErr(errors.Trace(aErr), bucket, object)
}
realDiskIdx := unshuffleIndex(diskIndex,
latestMeta.Erasure.Distribution)
drive := globalEndpoints.GetString(realDiskIdx)
result.DriveInfo.After[drive] = madmin.DriveStateOk
realDiskIdx := unshuffleIndex(diskIndex, latestMeta.Erasure.Distribution)
if outDatedDisks[realDiskIdx] != nil {
for i, v := range result.After.Drives {
if v.Endpoint == outDatedDisks[realDiskIdx].String() {
result.After.Drives[i].State = madmin.DriveStateOk
}
}
}
}
// Set the size of the object in the heal result
@@ -608,12 +496,11 @@ func healObject(storageDisks []StorageAPI, bucket string, object string,
// 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, dryRun bool) (
hr madmin.HealResultItem, err 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)
partsMetadata, errs := readAllXLMetadata(xl.getDisks(), bucket, object)
// get read quorum for this object
var readQuorum int
@@ -630,5 +517,5 @@ func (xl xlObjects) HealObject(bucket, object string, dryRun bool) (
defer objectLock.RUnlock()
// Heal the object.
return healObject(xl.storageDisks, bucket, object, readQuorum, dryRun)
return healObject(xl.getDisks(), bucket, object, readQuorum, dryRun)
}

View File

@@ -18,249 +18,13 @@ package cmd
import (
"bytes"
"fmt"
"os"
"path/filepath"
"testing"
"github.com/minio/minio-go/pkg/set"
"github.com/minio/minio/pkg/errors"
)
// Tests healing of format XL.
func TestHealFormatXL(t *testing.T) {
root, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(root)
nDisks := 16
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// Everything is fine, should return nil
obj, _, err := initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl := obj.(*xlObjects)
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// Disks 0..15 are nil
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 15; i++ {
xl.storageDisks[i] = nil
}
if _, err = healFormatXL(xl.storageDisks, false); err != errXLReadQuorum {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// One disk returns Faulty Disk
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := range xl.storageDisks {
posixDisk, ok := xl.storageDisks[i].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
xl.storageDisks[i] = newNaughtyDisk(posixDisk, nil, errDiskFull)
}
if _, err = healFormatXL(xl.storageDisks, false); err != errXLReadQuorum {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// One disk is not found, heal corrupted disks should return
// error for offline disk
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
xl.storageDisks[0] = nil
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)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// Remove format.json of all disks
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 15; i++ {
if err = xl.storageDisks[i].DeleteFile(minioMetaBucket, formatConfigFile); err != nil {
t.Fatal(err)
}
}
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// Corrupted format json in one disk
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 15; i++ {
if err = xl.storageDisks[i].AppendFile(minioMetaBucket, formatConfigFile, []byte("corrupted data")); err != nil {
t.Fatal(err)
}
}
if _, err = healFormatXL(xl.storageDisks, false); err == nil {
t.Fatal("Should get a json parsing error, ")
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// Remove format.json on 3 disks.
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteFile(minioMetaBucket, formatConfigFile); err != nil {
t.Fatal(err)
}
}
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// One disk is not found, heal corrupted disks should return nil
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteFile(minioMetaBucket, formatConfigFile); err != nil {
t.Fatal(err)
}
}
posixDisk, ok := xl.storageDisks[3].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
xl.storageDisks[3] = newNaughtyDisk(posixDisk, nil, errDiskNotFound)
expectedErr := fmt.Errorf("cannot proceed with heal as %s", errSomeDiskOffline)
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
if err.Error() != expectedErr.Error() {
t.Fatal("Got an unexpected error: ", err)
}
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// One disk has access denied error, heal should return
// appropriate error
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteFile(minioMetaBucket, formatConfigFile); err != nil {
t.Fatal(err)
}
}
posixDisk, ok = xl.storageDisks[3].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
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, false); err != nil {
if err.Error() != expectedErr.Error() {
t.Fatal("Got an unexpected error: ", err)
}
}
removeRoots(fsDirs)
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
// One disk is not found, heal corrupted disks should return nil
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
if err = obj.MakeBucketWithLocation(getRandomBucketName(), ""); err != nil {
t.Fatal(err)
}
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteFile(minioMetaBucket, formatConfigFile); err != nil {
t.Fatal(err)
}
}
if _, err = healFormatXL(xl.storageDisks, false); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
removeRoots(fsDirs)
}
// Tests undoes and validates if the undoing completes successfully.
func TestUndoMakeBucket(t *testing.T) {
root, err := newTestConfig(globalMinioDefaultRegion)
@@ -301,167 +65,6 @@ func TestUndoMakeBucket(t *testing.T) {
}
}
// Tests quick healing of bucket and bucket metadata.
func TestQuickHeal(t *testing.T) {
root, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(root)
nDisks := 16
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
defer removeRoots(fsDirs)
// Remove format.json on 16 disks.
obj, _, err := initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
bucketName := getRandomBucketName()
if err = obj.MakeBucketWithLocation(bucketName, ""); err != nil {
t.Fatal(err)
}
xl := obj.(*xlObjects)
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteVol(bucketName); err != nil {
t.Fatal(err)
}
}
// figure out read and write quorum
readQuorum := len(xl.storageDisks) / 2
writeQuorum := len(xl.storageDisks)/2 + 1
// Heal the missing buckets.
if err = quickHeal(*xl, writeQuorum, readQuorum); err != nil {
t.Fatal(err)
}
// Validate if buckets were indeed healed.
for i := 0; i <= 2; i++ {
if _, err = xl.storageDisks[i].StatVol(bucketName); err != nil {
t.Fatal(err)
}
}
// Corrupt one of the disks to return unformatted disk.
posixDisk, ok := xl.storageDisks[0].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errUnformattedDisk)
if err = quickHeal(*xl, writeQuorum, readQuorum); err != errUnformattedDisk {
t.Fatal(err)
}
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
defer removeRoots(fsDirs)
// One disk is not found, heal corrupted disks should return nil
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
xl.storageDisks[0] = nil
if err = quickHeal(*xl, writeQuorum, readQuorum); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
fsDirs, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
defer removeRoots(fsDirs)
// One disk is not found, heal corrupted disks should return nil
obj, _, err = initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
xl = obj.(*xlObjects)
// Corrupt one of the disks to return unformatted disk.
posixDisk, ok = xl.storageDisks[0].(*retryStorage)
if !ok {
t.Fatal("storage disk is not *retryStorage type")
}
xl.storageDisks[0] = newNaughtyDisk(posixDisk, nil, errDiskNotFound)
if err = quickHeal(*xl, writeQuorum, readQuorum); err != nil {
t.Fatal("Got an unexpected error: ", err)
}
}
// TestListBucketsHeal lists buckets heal result
func TestListBucketsHeal(t *testing.T) {
root, err := newTestConfig("us-east-1")
if err != nil {
t.Fatal(err)
}
defer os.RemoveAll(root)
nDisks := 16
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal(err)
}
defer removeRoots(fsDirs)
obj, _, err := initObjectLayer(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal(err)
}
// Create a bucket that won't get corrupted
saneBucket := "sanebucket"
if err = obj.MakeBucketWithLocation(saneBucket, ""); err != nil {
t.Fatal(err)
}
// Create a bucket that will be removed in some disks
corruptedBucketName := getRandomBucketName()
if err = obj.MakeBucketWithLocation(corruptedBucketName, ""); err != nil {
t.Fatal(err)
}
xl := obj.(*xlObjects)
// Remove bucket in disk 0, 1 and 2
for i := 0; i <= 2; i++ {
if err = xl.storageDisks[i].DeleteVol(corruptedBucketName); err != nil {
t.Fatal(err)
}
}
// List the missing buckets.
buckets, err := xl.ListBucketsHeal()
if err != nil {
t.Fatal(err)
}
bucketSet := set.CreateStringSet(saneBucket, corruptedBucketName)
// Check the number of buckets in list buckets heal result
if len(buckets) != len(bucketSet) {
t.Fatalf("Length of missing buckets is incorrect, expected: 2, found: %d", len(buckets))
}
// 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)
}
}
}
// Tests healing of object.
func TestHealObjectXL(t *testing.T) {
root, err := newTestConfig(globalMinioDefaultRegion)

View File

@@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -16,175 +16,12 @@
package cmd
import (
"sort"
"strings"
"github.com/minio/minio/pkg/errors"
)
func listDirHealFactory(isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
// Returns sorted merged entries from all the disks.
listDir := func(bucket, prefixDir, prefixEntry string) (mergedEntries []string, delayIsLeaf bool, err error) {
for _, disk := range disks {
if disk == nil {
continue
}
var entries []string
var newEntries []string
entries, err = disk.ListDir(bucket, prefixDir)
if err != nil {
continue
}
// isLeaf() check has to happen here so that
// trailing "/" for objects can be removed.
for i, entry := range entries {
if isLeaf(bucket, pathJoin(prefixDir, entry)) {
entries[i] = strings.TrimSuffix(entry, slashSeparator)
}
}
// Find elements in entries which are not in mergedEntries
for _, entry := range entries {
idx := sort.SearchStrings(mergedEntries, entry)
// if entry is already present in mergedEntries don't add.
if idx < len(mergedEntries) && mergedEntries[idx] == entry {
continue
}
newEntries = append(newEntries, entry)
}
if len(newEntries) > 0 {
// Merge the entries and sort it.
mergedEntries = append(mergedEntries, newEntries...)
sort.Strings(mergedEntries)
}
// Filter entries that have the prefix prefixEntry.
mergedEntries = filterMatchingPrefix(mergedEntries, prefixEntry)
}
return mergedEntries, false, nil
}
return listDir
// This is not implemented/needed anymore, look for xl-sets.ListBucketHeal()
func (xl xlObjects) ListBucketsHeal() ([]BucketInfo, error) {
return nil, nil
}
// listObjectsHeal - wrapper function implemented over file tree walk.
func (xl xlObjects) listObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) {
// Default is recursive, if delimiter is set then list non recursive.
recursive := true
if delimiter == slashSeparator {
recursive = false
}
// "heal" true for listObjectsHeal() and false for listObjects()
heal := true
walkResultCh, endWalkCh := xl.listPool.Release(listParams{bucket, recursive, marker, prefix, heal})
if walkResultCh == nil {
endWalkCh = make(chan struct{})
isLeaf := xl.isObject
listDir := listDirHealFactory(isLeaf, xl.storageDisks...)
walkResultCh = startTreeWalk(bucket, prefix, marker, recursive, listDir, nil, endWalkCh)
}
var objInfos []ObjectInfo
var eof bool
var nextMarker string
for i := 0; i < maxKeys; {
walkResult, ok := <-walkResultCh
if !ok {
// Closed channel.
eof = true
break
}
// For any walk error return right away.
if walkResult.err != nil {
return loi, toObjectErr(walkResult.err, bucket, prefix)
}
entry := walkResult.entry
var objInfo ObjectInfo
if hasSuffix(entry, slashSeparator) {
// Object name needs to be full path.
objInfo.Bucket = bucket
objInfo.Name = entry
objInfo.IsDir = true
} else {
var err error
objInfo, err = xl.getObjectInfo(bucket, entry)
if err != nil {
// Ignore errFileNotFound
if errors.Cause(err) == errFileNotFound {
continue
}
return loi, toObjectErr(err, bucket, prefix)
}
}
nextMarker = objInfo.Name
objInfos = append(objInfos, objInfo)
i++
if walkResult.end {
eof = true
break
}
}
params := listParams{bucket, recursive, nextMarker, prefix, heal}
if !eof {
xl.listPool.Set(params, walkResultCh, endWalkCh)
}
result := ListObjectsInfo{IsTruncated: !eof}
for _, objInfo := range objInfos {
result.NextMarker = objInfo.Name
if objInfo.IsDir {
result.Prefixes = append(result.Prefixes, objInfo.Name)
continue
}
// 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
}
// ListObjects - list all objects at prefix, delimited by '/'.
// This is not implemented/needed anymore, look for xl-sets.ListObjectsHeal()
func (xl xlObjects) ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) {
if err := checkListObjsArgs(bucket, prefix, marker, delimiter, xl); err != nil {
return loi, err
}
// With max keys of zero we have reached eof, return right here.
if maxKeys == 0 {
return loi, nil
}
// For delimiter and prefix as '/' we do not list anything at all
// since according to s3 spec we stop at the 'delimiter' along
// with the prefix. On a flat namespace with 'prefix' as '/'
// we don't have any entries, since all the keys are of form 'keyName/...'
if delimiter == slashSeparator && prefix == slashSeparator {
return loi, nil
}
// Over flowing count - reset to maxObjectList.
if maxKeys < 0 || maxKeys > maxObjectList {
maxKeys = maxObjectList
}
// Initiate a list operation, if successful filter and return quickly.
listObjInfo, err := xl.listObjectsHeal(bucket, prefix, marker, delimiter, maxKeys)
if err == nil {
// We got the entries successfully return.
return listObjInfo, nil
}
// Return error at the end.
return loi, toObjectErr(err, bucket, prefix)
return loi, nil
}

View File

@@ -1,144 +0,0 @@
/*
* Minio Cloud Storage (C) 2016, 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 (
"bytes"
"os"
"strconv"
"testing"
)
// TestListObjectsHeal - Tests ListObjectsHeal API for XL
func TestListObjectsHeal(t *testing.T) {
initNSLock(false)
rootPath, err := newTestConfig(globalMinioDefaultRegion)
if err != nil {
t.Fatalf("Init Test config failed")
}
// remove the root directory after the test ends.
defer os.RemoveAll(rootPath)
// Create an instance of xl backend
xl, fsDirs, err := prepareXL16()
if err != nil {
t.Fatal(err)
}
// Cleanup backend directories
defer removeRoots(fsDirs)
bucketName := "bucket"
objName := "obj"
// Create test bucket
err = xl.MakeBucketWithLocation(bucketName, "")
if err != nil {
t.Fatal(err)
}
// 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)
if err != nil {
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
}
}
// 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)
if err != nil {
t.Fatalf("XL Object upload failed: <ERROR> %s", err)
}
}
// Structure for testing
type testData struct {
bucket string
object string
marker string
delimiter string
maxKeys int
expectedErr error
foundObjs int
}
// Generic function for testing ListObjectsHeal, needs testData as a parameter
testFunc := func(testCase testData, testRank int) {
objectsNeedHeal, foundErr := xl.ListObjectsHeal(testCase.bucket, testCase.object, testCase.marker, testCase.delimiter, testCase.maxKeys)
if testCase.expectedErr == nil && foundErr != nil {
t.Fatalf("Test %d: Expected nil error, found: %v", testRank, foundErr)
}
if testCase.expectedErr != nil && foundErr.Error() != testCase.expectedErr.Error() {
t.Fatalf("Test %d: Found unexpected error: %v, expected: %v", testRank, foundErr, testCase.expectedErr)
}
if len(objectsNeedHeal.Objects) != testCase.foundObjs {
t.Fatalf("Test %d: Found unexpected number of objects: %d, expected: %v", testRank, len(objectsNeedHeal.Objects), testCase.foundObjs)
}
}
// Start tests
testCases := []testData{
// Wrong bucket name
{"foobucket", "", "", "", 1000, BucketNotFound{Bucket: "foobucket"}, 0},
// Inexistent object
{bucketName, "inexistentObj", "", "", 1000, nil, 0},
// Test ListObjectsHeal when all objects are sane
{bucketName, "", "", "", 1000, nil, 10},
}
for i, testCase := range testCases {
testFunc(testCase, i+1)
}
// Test ListObjectsHeal when all objects under unsane need healing
xlObj := xl.(*xlObjects)
for i := 0; i < 5; i++ {
if err = xlObj.storageDisks[0].DeleteFile(bucketName, "unsane/subdir/"+objName+strconv.Itoa(i)+"/xl.json"); err != nil {
t.Fatal(err)
}
}
// Start tests again with some objects that need healing
testCases = []testData{
// Test ListObjectsHeal when all objects under unsane/ need to be healed
{bucketName, "", "", "", 1000, nil, 10},
// List objects heal under unsane/, should return all elements
{bucketName, "unsane/", "", "", 1000, nil, 5},
// 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},
// Prefix == Delimiter == "/"
{bucketName, "/", "", "/", 1000, nil, 0},
// Max Keys == 0
{bucketName, "", "", "", 0, nil, 0},
// Testing with marker parameter
{bucketName, "", "unsane/subdir/" + objName + "0", "", 1000, nil, 4},
}
for i, testCase := range testCases {
testFunc(testCase, i+1)
}
}

View File

@@ -110,10 +110,11 @@ func (xl xlObjects) listObjects(bucket, prefix, marker, delimiter string, maxKey
var err error
objInfo, err = xl.getObjectInfo(bucket, entry)
if err != nil {
// Ignore errFileNotFound as the object might have got deleted in the interim period
// of listing and getObjectInfo()
// Ignore quorum error as it might be an entry from an outdated disk.
if errors.Cause(err) == errFileNotFound || errors.Cause(err) == errXLReadQuorum {
// Ignore errFileNotFound as the object might have got
// deleted in the interim period of listing and getObjectInfo(),
// ignore quorum error as it might be an entry from an outdated disk.
switch errors.Cause(err) {
case errFileNotFound, errXLReadQuorum:
continue
}
return loi, toObjectErr(err, bucket, prefix)

View File

@@ -410,7 +410,7 @@ func (xl xlObjects) readXLMetaParts(bucket, object string) (xlMetaParts []object
}
// If all errors were ignored, reduce to maximal occurrence
// based on the read quorum.
readQuorum := len(xl.storageDisks) / 2
readQuorum := len(xl.getDisks()) / 2
return nil, reduceReadQuorumErrs(ignoredErrs, nil, readQuorum)
}
@@ -438,7 +438,7 @@ func (xl xlObjects) readXLMetaStat(bucket, object string) (xlStat statInfo, xlMe
}
// If all errors were ignored, reduce to maximal occurrence
// based on the read quorum.
readQuorum := len(xl.storageDisks) / 2
readQuorum := len(xl.getDisks()) / 2
return statInfo{}, nil, reduceReadQuorumErrs(ignoredErrs, nil, readQuorum)
}

View File

@@ -36,12 +36,12 @@ func (xl xlObjects) updateUploadJSON(bucket, object, uploadID string, initiated
tmpUploadsPath := mustGetUUID()
// slice to store errors from disks
errs := make([]error, len(xl.storageDisks))
errs := make([]error, len(xl.getDisks()))
// slice to store if it is a delete operation on a disk
isDelete := make([]bool, len(xl.storageDisks))
isDelete := make([]bool, len(xl.getDisks()))
wg := sync.WaitGroup{}
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
errs[index] = errors.Trace(errDiskNotFound)
continue
@@ -108,7 +108,7 @@ func (xl xlObjects) updateUploadJSON(bucket, object, uploadID string, initiated
//
// 2. uploads.json was deleted -> in this case since
// the delete failed, we restore from tmp.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil || errs[index] != nil {
continue
}
@@ -134,7 +134,7 @@ func (xl xlObjects) updateUploadJSON(bucket, object, uploadID string, initiated
// we do have quorum, so in case of delete upload.json file
// operation, we purge from tmp.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil || !isDelete[index] {
continue
}
@@ -188,7 +188,7 @@ func (xl xlObjects) isUploadIDExists(bucket, object, uploadID string) bool {
func (xl xlObjects) removeObjectPart(bucket, object, uploadID, partName string) {
curpartPath := path.Join(bucket, object, uploadID, partName)
wg := sync.WaitGroup{}
for i, disk := range xl.storageDisks {
for i, disk := range xl.getDisks() {
if disk == nil {
continue
}
@@ -227,7 +227,7 @@ func (xl xlObjects) statPart(bucket, object, uploadID, partName string) (fileInf
}
// If all errors were ignored, reduce to maximal occurrence
// based on the read quorum.
readQuorum := len(xl.storageDisks) / 2
readQuorum := len(xl.getDisks()) / 2
return FileInfo{}, reduceReadQuorumErrs(ignoredErrs, nil, readQuorum)
}
@@ -506,7 +506,7 @@ func (xl xlObjects) ListMultipartUploads(bucket, object, keyMarker, uploadIDMark
// operation(s) on the object.
func (xl xlObjects) newMultipartUpload(bucket string, object string, meta map[string]string) (string, error) {
dataBlocks, parityBlocks := getRedundancyCount(meta[amzStorageClass], len(xl.storageDisks))
dataBlocks, parityBlocks := getRedundancyCount(meta[amzStorageClass], len(xl.getDisks()))
xlMeta := newXLMetaV1(object, dataBlocks, parityBlocks)
@@ -542,7 +542,7 @@ func (xl xlObjects) newMultipartUpload(bucket string, object string, meta map[st
tempUploadIDPath := uploadID
// Write updated `xl.json` to all disks.
disks, err := writeSameXLMetadata(xl.storageDisks, minioMetaTmpBucket, tempUploadIDPath, xlMeta, writeQuorum)
disks, err := writeSameXLMetadata(xl.getDisks(), minioMetaTmpBucket, tempUploadIDPath, xlMeta, writeQuorum)
if err != nil {
return "", toObjectErr(err, minioMetaTmpBucket, tempUploadIDPath)
}
@@ -678,7 +678,7 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, d
}
// Read metadata associated with the object from all disks.
partsMetadata, errs = readAllXLMetadata(xl.storageDisks, minioMetaMultipartBucket,
partsMetadata, errs = readAllXLMetadata(xl.getDisks(), minioMetaMultipartBucket,
uploadIDPath)
// get Quorum for this object
@@ -695,7 +695,7 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, d
preUploadIDLock.RUnlock()
// List all online disks.
onlineDisks, modTime := listOnlineDisks(xl.storageDisks, partsMetadata, errs)
onlineDisks, modTime := listOnlineDisks(xl.getDisks(), partsMetadata, errs)
// Pick one from the first valid metadata.
xlMeta, err := pickValidXLMeta(partsMetadata, modTime)
@@ -725,7 +725,11 @@ func (xl xlObjects) PutObjectPart(bucket, object, uploadID string, partID int, d
if err != nil {
return pi, toObjectErr(err, bucket, object)
}
buffer := make([]byte, xlMeta.Erasure.BlockSize, 2*xlMeta.Erasure.BlockSize) // alloc additional space for parity blocks created while erasure coding
// Fetch buffer for I/O, returns from the pool if not allocates a new one and returns.
buffer := xl.bp.Get()
defer xl.bp.Put(buffer)
file, err := storage.CreateFile(data, minioMetaTmpBucket, tmpPartPath, buffer, DefaultBitrotAlgorithm, writeQuorum)
if err != nil {
return pi, toObjectErr(err, bucket, object)
@@ -961,7 +965,7 @@ func (xl xlObjects) CompleteMultipartUpload(bucket string, object string, upload
uploadIDPath := pathJoin(bucket, object, uploadID)
// Read metadata associated with the object from all disks.
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, minioMetaMultipartBucket, uploadIDPath)
partsMetadata, errs := readAllXLMetadata(xl.getDisks(), minioMetaMultipartBucket, uploadIDPath)
// get Quorum for this object
_, writeQuorum, err := objectQuorumFromMeta(xl, partsMetadata, errs)
@@ -974,7 +978,7 @@ func (xl xlObjects) CompleteMultipartUpload(bucket string, object string, upload
return oi, toObjectErr(reducedErr, bucket, object)
}
onlineDisks, modTime := listOnlineDisks(xl.storageDisks, partsMetadata, errs)
onlineDisks, modTime := listOnlineDisks(xl.getDisks(), partsMetadata, errs)
// Calculate full object size.
var objectSize int64
@@ -1077,7 +1081,7 @@ func (xl xlObjects) CompleteMultipartUpload(bucket string, object string, upload
// NOTE: Do not use online disks slice here.
// The reason is that existing object should be purged
// regardless of `xl.json` status and rolled back in case of errors.
_, err = renameObject(xl.storageDisks, bucket, object, minioMetaTmpBucket, newUniqueID, writeQuorum)
_, err = renameObject(xl.getDisks(), bucket, object, minioMetaTmpBucket, newUniqueID, writeQuorum)
if err != nil {
return oi, toObjectErr(err, bucket, object)
}
@@ -1134,11 +1138,11 @@ func (xl xlObjects) CompleteMultipartUpload(bucket string, object string, upload
// Wrapper which removes all the uploaded parts.
func (xl xlObjects) cleanupUploadedParts(uploadIDPath string, writeQuorum int) error {
var errs = make([]error, len(xl.storageDisks))
var errs = make([]error, len(xl.getDisks()))
var wg = &sync.WaitGroup{}
// Cleanup uploadID for all disks.
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
errs[index] = errors.Trace(errDiskNotFound)
continue
@@ -1169,7 +1173,7 @@ func (xl xlObjects) abortMultipartUpload(bucket, object, uploadID string) (err e
uploadIDPath := path.Join(bucket, object, uploadID)
// Read metadata associated with the object from all disks.
partsMetadata, errs := readAllXLMetadata(xl.storageDisks, minioMetaMultipartBucket, uploadIDPath)
partsMetadata, errs := readAllXLMetadata(xl.getDisks(), minioMetaMultipartBucket, uploadIDPath)
// get Quorum for this object
_, writeQuorum, err := objectQuorumFromMeta(xl, partsMetadata, errs)

View File

@@ -164,7 +164,7 @@ func TestUpdateUploadJSON(t *testing.T) {
// make some disks faulty to simulate a failure.
for i := range xl.storageDisks[:9] {
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i].(*retryStorage), nil, errFaultyDisk)
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i], nil, errFaultyDisk)
}
testErrVal := xl.updateUploadJSON(bucket, object, "222abc", UTCNow(), 10, false)

View File

@@ -36,9 +36,9 @@ var objectOpIgnoredErrs = append(baseIgnoredErrs, errDiskAccessDenied)
func (xl xlObjects) putObjectDir(bucket, object string, writeQuorum int) error {
var wg = &sync.WaitGroup{}
errs := make([]error, len(xl.storageDisks))
errs := make([]error, len(xl.getDisks()))
// Prepare object creation in all disks
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
continue
}
@@ -114,7 +114,7 @@ func (xl xlObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string
}
// Read metadata associated with the object from all disks.
metaArr, errs := readAllXLMetadata(xl.storageDisks, srcBucket, srcObject)
metaArr, errs := readAllXLMetadata(xl.getDisks(), srcBucket, srcObject)
// get Quorum for this object
readQuorum, writeQuorum, err := objectQuorumFromMeta(xl, metaArr, errs)
@@ -127,7 +127,7 @@ func (xl xlObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string
}
// List all online disks.
onlineDisks, modTime := listOnlineDisks(xl.storageDisks, metaArr, errs)
onlineDisks, modTime := listOnlineDisks(xl.getDisks(), metaArr, errs)
// Pick latest valid metadata.
xlMeta, err := pickValidXLMeta(metaArr, modTime)
@@ -145,7 +145,7 @@ func (xl xlObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string
cpMetadataOnly := isStringEqual(pathJoin(srcBucket, srcObject), pathJoin(dstBucket, dstObject))
if cpMetadataOnly {
xlMeta.Meta = metadata
partsMetadata := make([]xlMetaV1, len(xl.storageDisks))
partsMetadata := make([]xlMetaV1, len(xl.getDisks()))
// Update `xl.json` content on each disks.
for index := range partsMetadata {
partsMetadata[index] = xlMeta
@@ -233,7 +233,7 @@ func (xl xlObjects) getObject(bucket, object string, startOffset int64, length i
}
// Read metadata associated with the object from all disks.
metaArr, errs := readAllXLMetadata(xl.storageDisks, bucket, object)
metaArr, errs := readAllXLMetadata(xl.getDisks(), bucket, object)
// get Quorum for this object
readQuorum, _, err := objectQuorumFromMeta(xl, metaArr, errs)
@@ -246,7 +246,7 @@ func (xl xlObjects) getObject(bucket, object string, startOffset int64, length i
}
// List all online disks.
onlineDisks, modTime := listOnlineDisks(xl.storageDisks, metaArr, errs)
onlineDisks, modTime := listOnlineDisks(xl.getDisks(), metaArr, errs)
// Pick latest valid metadata.
xlMeta, err := pickValidXLMeta(metaArr, modTime)
@@ -340,9 +340,9 @@ func (xl xlObjects) getObject(bucket, object string, startOffset int64, length i
func (xl xlObjects) getObjectInfoDir(bucket, object string) (oi ObjectInfo, err error) {
var wg = &sync.WaitGroup{}
errs := make([]error, len(xl.storageDisks))
errs := make([]error, len(xl.getDisks()))
// Prepare object creation in a all disks
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
continue
}
@@ -368,7 +368,7 @@ func (xl xlObjects) getObjectInfoDir(bucket, object string) (oi ObjectInfo, err
wg.Wait()
readQuorum := len(xl.storageDisks) / 2
readQuorum := len(xl.getDisks()) / 2
return dirObjectInfo(bucket, object, 0, map[string]string{}), reduceReadQuorumErrs(errs, objectOpIgnoredErrs, readQuorum)
}
@@ -403,7 +403,7 @@ func (xl xlObjects) GetObjectInfo(bucket, object string) (oi ObjectInfo, e error
// getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo.
func (xl xlObjects) getObjectInfo(bucket, object string) (objInfo ObjectInfo, err error) {
// Read metadata associated with the object from all disks.
metaArr, errs := readAllXLMetadata(xl.storageDisks, bucket, object)
metaArr, errs := readAllXLMetadata(xl.getDisks(), bucket, object)
// get Quorum for this object
readQuorum, _, err := objectQuorumFromMeta(xl, metaArr, errs)
@@ -509,7 +509,7 @@ func rename(disks []StorageAPI, srcBucket, srcEntry, dstBucket, dstEntry string,
// Wait for all renames to finish.
wg.Wait()
// We can safely allow RenameFile errors up to len(xl.storageDisks) - writeQuorum
// We can safely allow RenameFile errors up to len(xl.getDisks()) - writeQuorum
// otherwise return failure. Cleanup successful renames.
err := reduceWriteQuorumErrs(errs, objectOpIgnoredErrs, writeQuorum)
if errors.Cause(err) == errXLWriteQuorum {
@@ -566,7 +566,7 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
}
// Get parity and data drive count based on storage class metadata
dataDrives, parityDrives := getRedundancyCount(metadata[amzStorageClass], len(xl.storageDisks))
dataDrives, parityDrives := getRedundancyCount(metadata[amzStorageClass], len(xl.getDisks()))
// we now know the number of blocks this object needs for data and parity.
// writeQuorum is dataBlocks + 1
@@ -593,7 +593,7 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
}
// Rename the successfully written temporary object to final location.
if _, err = renameObject(xl.storageDisks, minioMetaTmpBucket, tempObj, bucket, object, writeQuorum); err != nil {
if _, err = renameObject(xl.getDisks(), minioMetaTmpBucket, tempObj, bucket, object, writeQuorum); err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
@@ -621,7 +621,7 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
var reader io.Reader = data
// Initialize parts metadata
partsMetadata := make([]xlMetaV1, len(xl.storageDisks))
partsMetadata := make([]xlMetaV1, len(xl.getDisks()))
xlMeta := newXLMetaV1(object, dataDrives, parityDrives)
@@ -631,7 +631,7 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
}
// Order disks according to erasure distribution
onlineDisks := shuffleDisks(xl.storageDisks, partsMetadata[0].Erasure.Distribution)
onlineDisks := shuffleDisks(xl.getDisks(), partsMetadata[0].Erasure.Distribution)
// Total size of the written object
var sizeWritten int64
@@ -641,8 +641,9 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
// Alloc additional space for parity blocks created while erasure coding
buffer := make([]byte, xlMeta.Erasure.BlockSize, 2*xlMeta.Erasure.BlockSize)
// Fetch buffer for I/O, returns from the pool if not allocates a new one and returns.
buffer := xl.bp.Get()
defer xl.bp.Put(buffer)
// Read data and split into parts - similar to multipart mechanism
for partIdx := 1; ; partIdx++ {
@@ -723,7 +724,7 @@ func (xl xlObjects) putObject(bucket string, object string, data *hash.Reader, m
// NOTE: Do not use online disks slice here.
// The reason is that existing object should be purged
// regardless of `xl.json` status and rolled back in case of errors.
_, err = renameObject(xl.storageDisks, bucket, object, minioMetaTmpBucket, newUniqueID, writeQuorum)
_, err = renameObject(xl.getDisks(), bucket, object, minioMetaTmpBucket, newUniqueID, writeQuorum)
if err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
@@ -779,7 +780,7 @@ func (xl xlObjects) deleteObject(bucket, object string) error {
// If its a directory request, no need to read metadata.
if !hasSuffix(object, slashSeparator) {
// Read metadata associated with the object from all disks.
metaArr, errs := readAllXLMetadata(xl.storageDisks, bucket, object)
metaArr, errs := readAllXLMetadata(xl.getDisks(), bucket, object)
// get Quorum for this object
_, writeQuorum, err = objectQuorumFromMeta(xl, metaArr, errs)
@@ -787,13 +788,13 @@ func (xl xlObjects) deleteObject(bucket, object string) error {
return err
}
} else {
writeQuorum = len(xl.storageDisks)/2 + 1
writeQuorum = len(xl.getDisks())/2 + 1
}
// Initialize list of errors.
var dErrs = make([]error, len(xl.storageDisks))
var dErrs = make([]error, len(xl.getDisks()))
for index, disk := range xl.storageDisks {
for index, disk := range xl.getDisks() {
if disk == nil {
dErrs[index] = errors.Trace(errDiskNotFound)
continue

View File

@@ -136,7 +136,7 @@ func TestXLDeleteObjectDiskNotFound(t *testing.T) {
// for a 16 disk setup, quorum is 9. To simulate disks not found yet
// quorum is available, we remove disks leaving quorum disks behind.
for i := range xl.storageDisks[:7] {
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i].(*retryStorage), nil, errFaultyDisk)
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i], nil, errFaultyDisk)
}
err = obj.DeleteObject(bucket, object)
if err != nil {
@@ -196,10 +196,10 @@ func TestGetObjectNoQuorum(t *testing.T) {
}
for i := range xl.storageDisks[:9] {
switch diskType := xl.storageDisks[i].(type) {
case *retryStorage:
xl.storageDisks[i] = newNaughtyDisk(diskType, diskErrors, errFaultyDisk)
case *naughtyDisk:
xl.storageDisks[i] = newNaughtyDisk(diskType.disk, diskErrors, errFaultyDisk)
default:
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i], diskErrors, errFaultyDisk)
}
}
// Fetch object from store.
@@ -247,10 +247,10 @@ func TestPutObjectNoQuorum(t *testing.T) {
}
for i := range xl.storageDisks[:9] {
switch diskType := xl.storageDisks[i].(type) {
case *retryStorage:
xl.storageDisks[i] = newNaughtyDisk(diskType, diskErrors, errFaultyDisk)
case *naughtyDisk:
xl.storageDisks[i] = newNaughtyDisk(diskType.disk, diskErrors, errFaultyDisk)
default:
xl.storageDisks[i] = newNaughtyDisk(xl.storageDisks[i], diskErrors, errFaultyDisk)
}
}
// Upload new content to same object "object"

View File

@@ -112,10 +112,11 @@ func diskCount(disks []StorageAPI) int {
// NOTE: collisions are fine, we are not looking for uniqueness
// in the slices returned.
func hashOrder(key string, cardinality int) []int {
if cardinality < 0 {
// Returns an empty int slice for negative cardinality.
if cardinality <= 0 {
// Returns an empty int slice for cardinality < 0.
return nil
}
nums := make([]int, cardinality)
keyCrc := crc32.Checksum([]byte(key), crc32.IEEETable)

View File

@@ -125,7 +125,7 @@ func TestHashOrder(t *testing.T) {
for i, testCase := range testCases {
hashedOrder := hashOrder(testCase.objectName, 16)
if !reflect.DeepEqual(testCase.hashedOrder, hashedOrder) {
t.Errorf("Test case %d: Expected \"%#v\" but failed \"%#v\"", i+1, testCase.hashedOrder, hashedOrder)
t.Errorf("Test case %d: Expected \"%v\" but failed \"%v\"", i+1, testCase.hashedOrder, hashedOrder)
}
}
@@ -133,6 +133,10 @@ func TestHashOrder(t *testing.T) {
if hashedOrder := hashOrder("This will fail", -1); hashedOrder != nil {
t.Errorf("Test: Expect \"nil\" but failed \"%#v\"", hashedOrder)
}
if hashedOrder := hashOrder("This will fail", 0); hashedOrder != nil {
t.Errorf("Test: Expect \"nil\" but failed \"%#v\"", hashedOrder)
}
}
// newTestXLMetaV1 - initializes new xlMetaV1, adds version, allocates a fresh erasure info and metadata.

View File

@@ -17,11 +17,10 @@
package cmd
import (
"fmt"
"sort"
"sync"
"time"
"github.com/minio/minio/pkg/bpool"
"github.com/minio/minio/pkg/disk"
"github.com/minio/minio/pkg/errors"
)
@@ -33,107 +32,36 @@ const (
// Uploads metadata file carries per multipart object metadata.
uploadsJSONFile = "uploads.json"
// Maximum erasure blocks.
maxErasureBlocks = 32
// Minimum erasure blocks.
minErasureBlocks = 4
)
// xlObjects - Implements XL object layer.
type xlObjects struct {
mutex *sync.Mutex
storageDisks []StorageAPI // Collection of initialized backend disks.
// ListObjects pool management.
listPool *treeWalkPool
// name space mutex for object layer
// name space mutex for object layer.
nsMutex *nsLockMap
// getDisks returns list of storageAPIs.
getDisks func() []StorageAPI
// Byte pools used for temporary i/o buffers.
bp *bpool.BytePoolCap
// Variable represents bucket policies in memory.
bucketPolicies *bucketPolicies
// TODO: Deprecated only kept here for tests, should be removed in future.
storageDisks []StorageAPI
// TODO: ListObjects pool management, should be removed in future.
listPool *treeWalkPool
}
// list of all errors that can be ignored in tree walk operation in XL
var xlTreeWalkIgnoredErrs = append(baseIgnoredErrs, errDiskAccessDenied, errVolumeNotFound, errFileNotFound)
// newXLObjectLayer - initialize any object layer depending on the number of disks.
func newXLObjectLayer(storageDisks []StorageAPI) (ObjectLayer, error) {
// Initialize XL object layer.
objAPI, err := newXLObjects(storageDisks)
fatalIf(err, "Unable to initialize XL object layer.")
// Initialize and load bucket policies.
err = initBucketPolicies(objAPI)
fatalIf(err, "Unable to load all bucket policies.")
// Initialize a new event notifier.
err = initEventNotifier(objAPI)
fatalIf(err, "Unable to initialize event notification.")
// Success.
return objAPI, nil
}
// newXLObjects - initialize new xl object layer.
func newXLObjects(storageDisks []StorageAPI) (ObjectLayer, error) {
if storageDisks == nil {
return nil, errInvalidArgument
}
// figure out readQuorum for erasure format.json
readQuorum := len(storageDisks) / 2
writeQuorum := len(storageDisks)/2 + 1
// Load saved XL format.json and validate.
newStorageDisks, err := loadFormatXL(storageDisks, readQuorum)
if err != nil {
return nil, fmt.Errorf("Unable to recognize backend format, %s", err)
}
// Initialize list pool.
listPool := newTreeWalkPool(globalLookupTimeout)
// Initialize xl objects.
xl := &xlObjects{
mutex: &sync.Mutex{},
storageDisks: newStorageDisks,
listPool: listPool,
nsMutex: newNSLock(globalIsDistXL),
}
// Initialize meta volume, if volume already exists ignores it.
if err = initMetaVolume(xl.storageDisks); err != nil {
return nil, fmt.Errorf("Unable to initialize '.minio.sys' meta volume, %s", err)
}
// If the number of offline servers is equal to the readQuorum
// (i.e. the number of online servers also equals the
// readQuorum), we cannot perform quick-heal (no
// write-quorum). However reads may still be possible, so we
// skip quick-heal in this case, and continue.
offlineCount := len(newStorageDisks) - diskCount(newStorageDisks)
if offlineCount == readQuorum {
return xl, nil
}
// Perform a quick heal on the buckets and bucket metadata for any discrepancies.
if err = quickHeal(*xl, writeQuorum, readQuorum); err != nil {
return nil, err
}
// Start background process to cleanup old multipart objects in `.minio.sys`.
go cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, xl, xl.listMultipartUploadsCleanup, globalServiceDoneCh)
return xl, nil
}
// Shutdown function for object storage interface.
func (xl xlObjects) Shutdown() error {
// Add any object layer shutdown activities here.
for _, disk := range xl.storageDisks {
for _, disk := range xl.getDisks() {
// This closes storage rpc client connections if any.
// Otherwise this is a no-op.
if disk == nil {
@@ -291,6 +219,5 @@ func getStorageInfo(disks []StorageAPI) StorageInfo {
// StorageInfo - returns underlying storage statistics.
func (xl xlObjects) StorageInfo() StorageInfo {
storageInfo := getStorageInfo(xl.storageDisks)
return storageInfo
return getStorageInfo(xl.getDisks())
}

View File

@@ -18,7 +18,6 @@ package cmd
import (
"os"
"path/filepath"
"reflect"
"testing"
@@ -50,30 +49,6 @@ func TestStorageInfo(t *testing.T) {
if disks16Info.Total <= 0 {
t.Fatalf("Diskinfo total values should be greater 0")
}
storageDisks, err := initStorageDisks(mustGetNewEndpointList(fsDirs...))
if err != nil {
t.Fatal("Unexpected error: ", err)
}
objLayer, err = newXLObjects(storageDisks)
if err != nil {
t.Fatalf("Unable to initialize 'XL' object layer with ignored disks %s. error %s", fsDirs[:4], err)
}
// Get storage info first attempt.
disks16Info = objLayer.StorageInfo()
// This test assumes homogenity between all disks,
// i.e if we loose one disk the effective storage
// usage values is assumed to decrease. If we have
// heterogenous environment this is not true all the time.
if disks16Info.Free <= 0 {
t.Fatalf("Diskinfo total free values should be greater 0")
}
if disks16Info.Total <= 0 {
t.Fatalf("Diskinfo total values should be greater 0")
}
}
// Sort valid disks info.
@@ -120,50 +95,3 @@ func TestSortingValidDisks(t *testing.T) {
}
}
}
// TestNewXL - tests initialization of all input disks
// and constructs a valid `XL` object
func TestNewXL(t *testing.T) {
var nDisks = 16 // Maximum disks.
var erasureDisks []string
for i := 0; i < nDisks; i++ {
// Do not attempt to create this path, the test validates
// so that newXLObjects initializes non existing paths
// and successfully returns initialized object layer.
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
erasureDisks = append(erasureDisks, disk)
defer os.RemoveAll(disk)
}
// No disks input.
_, err := newXLObjects(nil)
if err != errInvalidArgument {
t.Fatalf("Unable to initialize erasure, %s", err)
}
endpoints := mustGetNewEndpointList(erasureDisks...)
storageDisks, err := initStorageDisks(endpoints)
if err != nil {
t.Fatal("Unexpected error: ", err)
}
_, err = waitForFormatXLDisks(true, endpoints, nil)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
_, err = waitForFormatXLDisks(true, nil, storageDisks)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
// Initializes all erasure disks
formattedDisks, err := waitForFormatXLDisks(true, endpoints, storageDisks)
if err != nil {
t.Fatalf("Unable to format disks for erasure, %s", err)
}
_, err = newXLObjects(formattedDisks)
if err != nil {
t.Fatalf("Unable to initialize erasure, %s", err)
}
}