fs: Re-implement object layer to remember the fd (#3509)

This patch re-writes FS backend to support shared backend sharing locks for safe concurrent access across multiple servers.
This commit is contained in:
Harshavardhana
2017-01-16 17:05:00 -08:00
committed by GitHub
parent a054c73e22
commit 1c699d8d3f
68 changed files with 3860 additions and 1580 deletions

View File

@@ -134,11 +134,11 @@ func testServicesCmdHandler(cmd cmdType, t *testing.T) {
if cmd == statusCmd {
// Initializing objectLayer and corresponding
// []StorageAPI since DiskInfo() method requires it.
objLayer, fsDir, fsErr := prepareFS()
objLayer, fsDirs, fsErr := prepareXL()
if fsErr != nil {
t.Fatalf("failed to initialize XL based object layer - %v.", fsErr)
}
defer removeRoots([]string{fsDir})
defer removeRoots(fsDirs)
globalObjLayerMutex.Lock()
globalObjectAPI = objLayer
globalObjLayerMutex.Unlock()

View File

@@ -30,32 +30,23 @@ import (
// Prepare benchmark backend
func prepareBenchmarkBackend(instanceType string) (ObjectLayer, []string, error) {
var nDisks int
switch instanceType {
// Total number of disks for FS backend is set to 1.
case FSTestStr:
nDisks = 1
// Total number of disks for FS backend is set to 16.
obj, disk, err := prepareFS()
if err != nil {
return nil, nil, err
}
return obj, []string{disk}, nil
// Total number of disks for XL backend is set to 16.
case XLTestStr:
nDisks = 16
default:
nDisks = 1
return prepareXL()
}
// get `nDisks` random disks.
disks, err := getRandomDisks(nDisks)
obj, disk, err := prepareFS()
if err != nil {
return nil, nil, err
}
endpoints, err := parseStorageEndpoints(disks)
if err != nil {
return nil, nil, err
}
// initialize object layer.
obj, _, err := initObjectLayer(endpoints)
if err != nil {
return nil, nil, err
}
return obj, disks, nil
return obj, []string{disk}, nil
}
// Benchmark utility functions for ObjectLayer.PutObject().

View File

@@ -1,85 +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 (
"bytes"
"io/ioutil"
"os"
"path/filepath"
"strings"
)
const policyJSON = "policy.json"
func getBucketFromPolicyPath(oldPolicyPath string) string {
bucketPrefix, _ := filepath.Split(oldPolicyPath)
_, bucketName := filepath.Split(strings.TrimSuffix(bucketPrefix, slashSeparator))
return bucketName
}
func cleanupOldBucketPolicyConfigs() error {
// Get old bucket policy config directory.
oldBucketsConfigDir, err := getOldBucketsConfigPath()
fatalIf(err, "Unable to fetch buckets config path to migrate bucket policy")
// Recursively remove configDir/buckets/ - old bucket policy config location.
// N B This is called only if all bucket policies were successfully migrated.
return os.RemoveAll(oldBucketsConfigDir)
}
func migrateBucketPolicyConfig(objAPI ObjectLayer) error {
// Get old bucket policy config directory.
oldBucketsConfigDir, err := getOldBucketsConfigPath()
fatalIf(err, "Unable to fetch buckets config path to migrate bucket policy")
// Check if config directory holding bucket policy exists before
// migration.
_, err = os.Stat(oldBucketsConfigDir)
if os.IsNotExist(err) {
return nil
}
if err != nil {
return err
}
// WalkFunc that migrates access-policy.json to
// .minio.sys/buckets/bucketName/policy.json on all disks.
migrateBucketPolicy := func(policyPath string, fileInfo os.FileInfo, err error) error {
// policyFile - e.g /configDir/sample-bucket/access-policy.json
if err != nil {
return err
}
// Skip entries that aren't bucket policy files.
if fileInfo.Name() != "access-policy.json" {
return nil
}
// Get bucketName from old policy file path.
bucketName := getBucketFromPolicyPath(policyPath)
// Read bucket policy config from old location.
policyBytes, err := ioutil.ReadFile(policyPath)
fatalIf(err, "Unable to read bucket policy to migrate bucket policy", policyPath)
newPolicyPath := retainSlash(bucketConfigPrefix) + retainSlash(bucketName) + policyJSON
var metadata map[string]string
sha256sum := ""
// Erasure code the policy config to all the disks.
_, err = objAPI.PutObject(minioMetaBucket, newPolicyPath, int64(len(policyBytes)), bytes.NewReader(policyBytes), metadata, sha256sum)
fatalIf(err, "Unable to write bucket policy during migration.", newPolicyPath)
return nil
}
return filepath.Walk(oldBucketsConfigDir, migrateBucketPolicy)
}

View File

@@ -20,7 +20,6 @@ import (
"bytes"
"encoding/json"
"io"
"path"
"sync"
)
@@ -28,6 +27,9 @@ const (
// Static prefix to be used while constructing bucket ARN.
// refer to S3 docs for more info.
bucketARNPrefix = "arn:" + eventSource + ":::"
// Bucket policy config name.
bucketPolicyConfig = "policy.json"
)
// Variable represents bucket policies in memory.
@@ -137,19 +139,10 @@ func initBucketPolicies(objAPI ObjectLayer) error {
return nil
}
// getOldBucketsConfigPath - get old buckets config path. (Only used for migrating old bucket policies)
func getOldBucketsConfigPath() (string, error) {
configPath, err := getConfigPath()
if err != nil {
return "", err
}
return path.Join(configPath, "buckets"), nil
}
// readBucketPolicyJSON - reads bucket policy for an input bucket, returns BucketPolicyNotFound
// if bucket policy is not found.
func readBucketPolicyJSON(bucket string, objAPI ObjectLayer) (bucketPolicyReader io.Reader, err error) {
policyPath := pathJoin(bucketConfigPrefix, bucket, policyJSON)
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
// Acquire a read lock on policy config before reading.
objLock := globalNSMutex.NewNSLock(minioMetaBucket, policyPath)
@@ -191,7 +184,7 @@ func readBucketPolicy(bucket string, objAPI ObjectLayer) (*bucketPolicy, error)
// removeBucketPolicy - removes any previously written bucket policy. Returns BucketPolicyNotFound
// if no policies are found.
func removeBucketPolicy(bucket string, objAPI ObjectLayer) error {
policyPath := pathJoin(bucketConfigPrefix, bucket, policyJSON)
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
// Acquire a write lock on policy config before modifying.
objLock := globalNSMutex.NewNSLock(minioMetaBucket, policyPath)
objLock.Lock()
@@ -214,7 +207,7 @@ func writeBucketPolicy(bucket string, objAPI ObjectLayer, bpy *bucketPolicy) err
errorIf(err, "Unable to marshal bucket policy '%v' to JSON", *bpy)
return err
}
policyPath := pathJoin(bucketConfigPrefix, bucket, policyJSON)
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
// Acquire a write lock on policy config before modifying.
objLock := globalNSMutex.NewNSLock(minioMetaBucket, policyPath)
objLock.Lock()

41
cmd/commands_test.go Normal file
View File

@@ -0,0 +1,41 @@
/*
* 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 (
"testing"
"github.com/minio/cli"
)
// Tests register command function.
func TestRegisterCommand(t *testing.T) {
registerCommand(cli.Command{
Name: "test1",
})
ccount := len(commands)
if ccount != 1 {
t.Fatalf("Unexpected number of commands found %d", ccount)
}
registerCommand(cli.Command{
Name: "test2",
})
ccount = len(commands)
if ccount != 2 {
t.Fatalf("Unexpected number of commands found %d", ccount)
}
}

View File

@@ -21,6 +21,9 @@ import "net/http"
// Standard cross domain policy information located at https://s3.amazonaws.com/crossdomain.xml
const crossDomainXML = `<?xml version="1.0"?><!DOCTYPE cross-domain-policy SYSTEM "http://www.adobe.com/xml/dtds/cross-domain-policy.dtd"><cross-domain-policy><allow-access-from domain="*" secure="false" /></cross-domain-policy>`
// Standard path where an app would find cross domain policy information.
const crossDomainXMLEntity = "/crossdomain.xml"
// Cross domain policy implements http.Handler interface, implementing a custom ServerHTTP.
type crossDomainPolicy struct {
handler http.Handler
@@ -38,7 +41,7 @@ func setCrossDomainPolicy(h http.Handler) http.Handler {
func (c crossDomainPolicy) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// Look for 'crossdomain.xml' in the incoming request.
switch r.URL.Path {
case "/crossdomain.xml":
case crossDomainXMLEntity:
// Write the standard cross domain policy xml.
w.Write([]byte(crossDomainXML))
// Request completed, no need to serve to other handlers.

View File

@@ -0,0 +1,41 @@
/*
* 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 (
"net/http"
"net/http/httptest"
"testing"
router "github.com/gorilla/mux"
)
// Test cross domain xml handler.
func TestCrossXMLHandler(t *testing.T) {
// Server initialization.
mux := router.NewRouter().SkipClean(true)
handler := setCrossDomainPolicy(mux)
srv := httptest.NewServer(handler)
resp, err := http.Get(srv.URL + crossDomainXMLEntity)
if err != nil {
t.Fatal(err)
}
if resp.StatusCode != http.StatusOK {
t.Fatal("Unexpected http status received", resp.Status)
}
}

View File

@@ -17,6 +17,7 @@
package cmd
import (
"bytes"
"fmt"
"net"
"reflect"
@@ -34,11 +35,11 @@ func TestInitEventNotifierFaultyDisks(t *testing.T) {
// remove the root directory after the test ends.
defer removeAll(rootPath)
disks, err := getRandomDisks(1)
disks, err := getRandomDisks(4)
if err != nil {
t.Fatal("Unable to create directories for FS backend. ", err)
}
defer removeAll(disks[0])
defer removeRoots(disks)
endpoints, err := parseStorageEndpoints(disks)
if err != nil {
t.Fatal(err)
@@ -53,8 +54,7 @@ func TestInitEventNotifierFaultyDisks(t *testing.T) {
t.Fatal("Unexpected error:", err)
}
fs := obj.(fsObjects)
fsstorage := fs.storage.(*retryStorage)
xl := obj.(*xlObjects)
listenARN := "arn:minio:sns:us-east-1:1:listen"
queueARN := "arn:minio:sqs:us-east-1:1:redis"
@@ -64,14 +64,18 @@ func TestInitEventNotifierFaultyDisks(t *testing.T) {
notificationXML += "<TopicConfiguration><Event>s3:ObjectRemoved:*</Event><Event>s3:ObjectRemoved:*</Event><Topic>" + listenARN + "</Topic></TopicConfiguration>"
notificationXML += "<QueueConfiguration><Event>s3:ObjectRemoved:*</Event><Event>s3:ObjectRemoved:*</Event><Queue>" + queueARN + "</Queue></QueueConfiguration>"
notificationXML += "</NotificationConfiguration>"
if err := fsstorage.AppendFile(minioMetaBucket, bucketConfigPrefix+"/"+bucketName+"/"+bucketNotificationConfig, []byte(notificationXML)); err != nil {
size := int64(len([]byte(notificationXML)))
reader := bytes.NewReader([]byte(notificationXML))
if _, err := xl.PutObject(minioMetaBucket, bucketConfigPrefix+"/"+bucketName+"/"+bucketNotificationConfig, size, reader, nil, ""); err != nil {
t.Fatal("Unexpected error:", err)
}
for i, d := range xl.storageDisks {
xl.storageDisks[i] = newNaughtyDisk(d.(*retryStorage), nil, errFaultyDisk)
}
// Test initEventNotifier() with faulty disks
for i := 1; i <= 3; i++ {
fs.storage = newNaughtyDisk(fsstorage, map[int]error{i: errFaultyDisk}, nil)
if err := initEventNotifier(fs); errorCause(err) != errFaultyDisk {
if err := initEventNotifier(xl); errorCause(err) != errFaultyDisk {
t.Fatal("Unexpected error:", err)
}
}
@@ -387,7 +391,7 @@ func TestInitEventNotifier(t *testing.T) {
}
// needed to load listener config from disk for testing (in
// single peer mode, the listener config is ingored, but here
// single peer mode, the listener config is ignored, but here
// we want to test the loading from disk too.)
globalIsDistXL = true

View File

@@ -24,8 +24,12 @@ import "syscall"
// sure that subsequent writes on a file just created will not fail,
// in addition, file allocation will be contigous on the disk
func Fallocate(fd int, offset int64, len int64) error {
return syscall.Fallocate(fd,
1, // FALLOC_FL_KEEP_SIZE
offset,
len)
// No need to attempt fallocate for 0 length.
if len == 0 {
return nil
}
// Don't extend size of file even if offset + len is
// greater than file size from <bits/fcntl-linux.h>.
fallocFLKeepSize := uint32(1)
return syscall.Fallocate(fd, fallocFLKeepSize, offset, len)
}

View File

@@ -1,3 +1,5 @@
// +build linux
/*
* Minio Cloud Storage, (C) 2016 Minio, Inc.
*
@@ -16,7 +18,12 @@
package cmd
import "errors"
import "testing"
// errFSDiskFormat - returned when given disk format is other than FS format.
var errFSDiskFormat = errors.New("Disk is not in FS format")
// Tests allocate.
func TestFallocate(t *testing.T) {
err := Fallocate(0, 0, 0)
if err != nil {
t.Fatal("Unexpected error in fallocate for length 0:", err)
}
}

View File

@@ -149,20 +149,6 @@ func reduceFormatErrs(errs []error, diskCount int) (err error) {
return nil
}
// creates format.json, the FS format info in minioMetaBucket.
func initFormatFS(storageDisk StorageAPI) error {
// Initialize meta volume, if volume already exists ignores it.
if err := initMetaVolume([]StorageAPI{storageDisk}); err != nil {
return fmt.Errorf("Unable to initialize '.minio.sys' meta volume, %s", err)
}
return saveFSFormatData(storageDisk, newFSFormatV1())
}
// loads format.json from minioMetaBucket if it exists.
func loadFormatFS(storageDisk StorageAPI) (format *formatConfigV1, err error) {
return loadFormat(storageDisk)
}
// loadAllFormats - load all format config from all input disks in parallel.
func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatConfigV1, []error) {
// Initialize sync waitgroup.
@@ -206,18 +192,6 @@ func loadAllFormats(bootstrapDisks []StorageAPI) ([]*formatConfigV1, []error) {
return formatConfigs, sErrs
}
// genericFormatCheckFS - validates format config and returns an error if any.
func genericFormatCheckFS(formatConfig *formatConfigV1, sErr error) (err error) {
if sErr != nil {
return sErr
}
// Successfully read, validate if FS.
if !isFSFormat(formatConfig) {
return errFSDiskFormat
}
return nil
}
// genericFormatCheckXL - validates and returns error.
// if (no quorum) return error
// if (any disk is corrupt) return error // phase2

View File

@@ -664,28 +664,6 @@ func TestReduceFormatErrs(t *testing.T) {
}
}
// Tests for genericFormatCheckFS()
func TestGenericFormatCheckFS(t *testing.T) {
// Generate format configs for XL.
formatConfigs := genFormatXLInvalidJBOD()
// Validate disk format is fs, should fail.
if err := genericFormatCheckFS(formatConfigs[0], nil); err != errFSDiskFormat {
t.Fatalf("Unexpected error, expected %s, got %s", errFSDiskFormat, err)
}
// Validate disk is unformatted, should fail.
if err := genericFormatCheckFS(nil, errUnformattedDisk); err != errUnformattedDisk {
t.Fatalf("Unexpected error, expected %s, got %s", errUnformattedDisk, err)
}
// Validate when disk is in FS format.
format := newFSFormatV1()
if err := genericFormatCheckFS(format, nil); err != nil {
t.Fatalf("Unexpected error should pass, failed with %s", err)
}
}
// Tests for genericFormatCheckXL()
func TestGenericFormatCheckXL(t *testing.T) {
var errs []error

View File

@@ -1,91 +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 "io"
// Reads from the requested local location uses a staging buffer. Restricts
// reads upto requested range of length and offset. If successful staging
// buffer is written to the incoming stream. Returns errors if any.
func fsReadFile(disk StorageAPI, bucket, object string, writer io.Writer, totalLeft, startOffset int64, buf []byte) (err error) {
bufSize := int64(len(buf))
// Start the read loop until requested range.
for {
// Figure out the right size for the buffer.
curLeft := bufSize
if totalLeft < bufSize {
curLeft = totalLeft
}
// Reads the file at offset.
nr, er := disk.ReadFile(bucket, object, startOffset, buf[:curLeft])
if nr > 0 {
// Write to response writer.
nw, ew := writer.Write(buf[0:nr])
if nw > 0 {
// Decrement whats left to write.
totalLeft -= int64(nw)
// Progress the offset
startOffset += int64(nw)
}
if ew != nil {
err = traceError(ew)
break
}
if nr != int64(nw) {
err = traceError(io.ErrShortWrite)
break
}
}
if er == io.EOF || er == io.ErrUnexpectedEOF {
break
}
if er != nil {
err = traceError(er)
break
}
if totalLeft == 0 {
break
}
}
return err
}
// Reads from input stream until end of file, takes an input buffer for staging reads.
// The staging buffer is then written to the disk. Returns for any error that occurs
// while reading the stream or writing to disk. Caller should cleanup partial files.
// Upon errors total data written will be 0 and returns error, on success returns
// total data written to disk.
func fsCreateFile(disk StorageAPI, reader io.Reader, buf []byte, bucket, object string) (int64, error) {
bytesWritten := int64(0)
// Read the buffer till io.EOF and appends data to path at bucket/object.
for {
n, rErr := reader.Read(buf)
if rErr != nil && rErr != io.EOF {
return 0, traceError(rErr)
}
bytesWritten += int64(n)
wErr := disk.AppendFile(bucket, object, buf[0:n])
if wErr != nil {
return 0, traceError(wErr)
}
if rErr == io.EOF {
break
}
}
return bytesWritten, nil
}

View File

@@ -18,6 +18,8 @@ package cmd
import (
"errors"
"io"
"os"
"reflect"
"sync"
"time"
@@ -30,15 +32,16 @@ import (
var errPartsMissing = errors.New("required parts missing")
// Error sent when appendParts go-routine has waited long enough and timedout.
var errAppendPartsTimeout = errors.New("appendParts goroutine timeout")
var errAppendPartsTimeout = errors.New("appendParts go-routine timeout")
// Timeout value for the appendParts go-routine.
var appendPartsTimeout = 24 * 60 * 60 * time.Second // 24 hours.
var appendPartsTimeout = 24 * 60 * 60 * time.Second // 24 Hours.
// Holds a map of uploadID->appendParts go-routine
type backgroundAppend struct {
infoMap map[string]bgAppendPartsInfo
sync.Mutex
infoMap map[string]bgAppendPartsInfo
appendFile io.WriteCloser
}
// Input to the appendParts go-routine
@@ -56,9 +59,9 @@ type bgAppendPartsInfo struct {
}
// Called after a part is uploaded so that it can be appended in the background.
func (b *backgroundAppend) append(disk StorageAPI, bucket, object, uploadID string, meta fsMetaV1) chan error {
b.Lock()
info, ok := b.infoMap[uploadID]
func (fs fsObjects) append(bucket, object, uploadID string, meta fsMetaV1) chan error {
fs.bgAppend.Lock()
info, ok := fs.bgAppend.infoMap[uploadID]
if !ok {
// Corresponding appendParts go-routine was not found, create a new one. Would happen when the first
// part of a multipart upload is uploaded.
@@ -68,11 +71,12 @@ func (b *backgroundAppend) append(disk StorageAPI, bucket, object, uploadID stri
completeCh := make(chan struct{})
info = bgAppendPartsInfo{inputCh, timeoutCh, abortCh, completeCh}
b.infoMap[uploadID] = info
fs.bgAppend.infoMap[uploadID] = info
go b.appendParts(disk, bucket, object, uploadID, info)
go fs.appendParts(bucket, object, uploadID, info)
}
b.Unlock()
fs.bgAppend.Unlock()
errCh := make(chan error)
go func() {
// send input in a goroutine as send on the inputCh can block if appendParts go-routine
@@ -85,19 +89,23 @@ func (b *backgroundAppend) append(disk StorageAPI, bucket, object, uploadID stri
case info.inputCh <- bgAppendPartsInput{meta, errCh}:
}
}()
return errCh
}
// Called on complete-multipart-upload. Returns nil if the required parts have been appended.
func (b *backgroundAppend) complete(disk StorageAPI, bucket, object, uploadID string, meta fsMetaV1) error {
b.Lock()
defer b.Unlock()
info, ok := b.infoMap[uploadID]
delete(b.infoMap, uploadID)
func (fs *fsObjects) complete(bucket, object, uploadID string, meta fsMetaV1) error {
fs.bgAppend.Lock()
defer fs.bgAppend.Unlock()
info, ok := fs.bgAppend.infoMap[uploadID]
delete(fs.bgAppend.infoMap, uploadID)
if !ok {
return errPartsMissing
}
errCh := make(chan error)
select {
case <-info.timeoutCh:
// This is to handle a rare race condition where we found info in b.infoMap
@@ -105,6 +113,7 @@ func (b *backgroundAppend) complete(disk StorageAPI, bucket, object, uploadID st
return errAppendPartsTimeout
case info.inputCh <- bgAppendPartsInput{meta, errCh}:
}
err := <-errCh
close(info.completeCh)
@@ -113,21 +122,26 @@ func (b *backgroundAppend) complete(disk StorageAPI, bucket, object, uploadID st
}
// Called after complete-multipart-upload or abort-multipart-upload so that the appendParts go-routine is not left dangling.
func (b *backgroundAppend) abort(uploadID string) {
b.Lock()
defer b.Unlock()
info, ok := b.infoMap[uploadID]
func (fs fsObjects) abort(uploadID string) {
fs.bgAppend.Lock()
defer fs.bgAppend.Unlock()
info, ok := fs.bgAppend.infoMap[uploadID]
if !ok {
return
}
delete(b.infoMap, uploadID)
delete(fs.bgAppend.infoMap, uploadID)
info.abortCh <- struct{}{}
}
// This is run as a go-routine that appends the parts in the background.
func (b *backgroundAppend) appendParts(disk StorageAPI, bucket, object, uploadID string, info bgAppendPartsInfo) {
func (fs fsObjects) appendParts(bucket, object, uploadID string, info bgAppendPartsInfo) {
appendPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, uploadID)
// Holds the list of parts that is already appended to the "append" file.
appendMeta := fsMetaV1{}
// Allocate staging read buffer.
buf := make([]byte, readSizeV1)
for {
@@ -136,6 +150,7 @@ func (b *backgroundAppend) appendParts(disk StorageAPI, bucket, object, uploadID
// We receive on this channel when new part gets uploaded or when complete-multipart sends
// a value on this channel to confirm if all the required parts are appended.
meta := input.meta
for {
// Append should be done such a way that if part-3 and part-2 is uploaded before part-1, we
// wait till part-1 is uploaded after which we append part-2 and part-3 as well in this for-loop.
@@ -152,18 +167,23 @@ func (b *backgroundAppend) appendParts(disk StorageAPI, bucket, object, uploadID
}
break
}
if err := appendPart(disk, bucket, object, uploadID, part, buf); err != nil {
disk.DeleteFile(minioMetaTmpBucket, uploadID)
if err := fs.appendPart(bucket, object, uploadID, part, buf); err != nil {
fsRemoveFile(appendPath)
appendMeta.Parts = nil
input.errCh <- err
break
}
appendMeta.AddObjectPart(part.Number, part.Name, part.ETag, part.Size)
}
case <-info.abortCh:
// abort-multipart-upload closed abortCh to end the appendParts go-routine.
disk.DeleteFile(minioMetaTmpBucket, uploadID)
close(info.timeoutCh) // So that any racing PutObjectPart does not leave a dangling go-routine.
fsRemoveFile(appendPath)
// So that any racing PutObjectPart does not leave a dangling go-routine.
close(info.timeoutCh)
return
case <-info.completeCh:
// complete-multipart-upload closed completeCh to end the appendParts go-routine.
@@ -172,11 +192,12 @@ func (b *backgroundAppend) appendParts(disk StorageAPI, bucket, object, uploadID
case <-time.After(appendPartsTimeout):
// Timeout the goroutine to garbage collect its resources. This would happen if the client initiates
// a multipart upload and does not complete/abort it.
b.Lock()
delete(b.infoMap, uploadID)
b.Unlock()
fs.bgAppend.Lock()
delete(fs.bgAppend.infoMap, uploadID)
fs.bgAppend.Unlock()
// Delete the temporary append file as well.
disk.DeleteFile(minioMetaTmpBucket, uploadID)
fsRemoveFile(appendPath)
close(info.timeoutCh)
return
@@ -186,29 +207,34 @@ func (b *backgroundAppend) appendParts(disk StorageAPI, bucket, object, uploadID
// Appends the "part" to the append-file inside "tmp/" that finally gets moved to the actual location
// upon complete-multipart-upload.
func appendPart(disk StorageAPI, bucket, object, uploadID string, part objectPartInfo, buf []byte) error {
partPath := pathJoin(bucket, object, uploadID, part.Name)
func (fs fsObjects) appendPart(bucket, object, uploadID string, part objectPartInfo, buf []byte) error {
partPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object, uploadID, part.Name)
offset := int64(0)
totalLeft := part.Size
for totalLeft > 0 {
curLeft := int64(readSizeV1)
if totalLeft < readSizeV1 {
curLeft = totalLeft
// Read each file part to start writing to the temporary concatenated object.
file, size, err := fsOpenFile(partPath, offset)
if err != nil {
if err == errFileNotFound {
return errPartsMissing
}
n, err := disk.ReadFile(minioMetaMultipartBucket, partPath, offset, buf[:curLeft])
if err != nil {
// Check for EOF/ErrUnexpectedEOF not needed as it should never happen as we know
// the exact size of the file and hence know the size of buf[]
// EOF/ErrUnexpectedEOF indicates that the length of file was shorter than part.Size and
// hence considered as an error condition.
return err
}
if err = disk.AppendFile(minioMetaTmpBucket, uploadID, buf[:n]); err != nil {
return err
}
offset += n
totalLeft -= n
return err
}
return nil
defer file.Close()
tmpObjPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, uploadID)
// No need to hold a lock, this is a unique file and will be only written
// to one one process per uploadID per minio process.
wfile, err := os.OpenFile(preparePath(tmpObjPath), os.O_CREATE|os.O_APPEND|os.O_WRONLY, 0666)
if err != nil {
return err
}
defer wfile.Close()
// Fallocate more space as we concatenate.
if err = fsFAllocate(int(wfile.Fd()), 0, size); err != nil {
return err
}
_, err = io.CopyBuffer(wfile, file, buf)
return err
}

373
cmd/fs-v1-helpers.go Normal file
View File

@@ -0,0 +1,373 @@
/*
* 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 (
"io"
"os"
pathutil "path"
)
// Removes only the file at given path does not remove
// any parent directories, handles long paths for
// windows automatically.
func fsRemoveFile(filePath string) (err error) {
if filePath == "" {
return errInvalidArgument
}
if err = checkPathLength(filePath); err != nil {
return err
}
if err = os.Remove(preparePath(filePath)); err != nil {
if os.IsNotExist(err) {
return errFileNotFound
} else if os.IsPermission(err) {
return errFileAccessDenied
}
return err
}
return nil
}
// Removes all files and folders at a given path, handles
// long paths for windows automatically.
func fsRemoveAll(dirPath string) (err error) {
if dirPath == "" {
return errInvalidArgument
}
if err = checkPathLength(dirPath); err != nil {
return err
}
if err = removeAll(dirPath); err != nil {
if os.IsPermission(err) {
return errVolumeAccessDenied
}
}
return err
}
// Removes a directory only if its empty, handles long
// paths for windows automatically.
func fsRemoveDir(dirPath string) (err error) {
if dirPath == "" {
return errInvalidArgument
}
if err = checkPathLength(dirPath); err != nil {
return err
}
if err = os.Remove(preparePath(dirPath)); err != nil {
if os.IsNotExist(err) {
return errVolumeNotFound
} else if isSysErrNotEmpty(err) {
return errVolumeNotEmpty
}
}
return err
}
// Creates a new directory, parent dir should exist
// otherwise returns an error. If directory already
// exists returns an error. Windows long paths
// are handled automatically.
func fsMkdir(dirPath string) (err error) {
if dirPath == "" {
return errInvalidArgument
}
if err = checkPathLength(dirPath); err != nil {
return err
}
if err = os.Mkdir(preparePath(dirPath), 0777); err != nil {
if os.IsExist(err) {
return errVolumeExists
} else if os.IsPermission(err) {
return errDiskAccessDenied
} else if isSysErrNotDir(err) {
// File path cannot be verified since
// one of the parents is a file.
return errDiskAccessDenied
} else if isSysErrPathNotFound(err) {
// Add specific case for windows.
return errDiskAccessDenied
}
}
return nil
}
// Lookup if directory exists, returns directory
// attributes upon success.
func fsStatDir(statDir string) (os.FileInfo, error) {
if statDir == "" {
return nil, errInvalidArgument
}
if err := checkPathLength(statDir); err != nil {
return nil, err
}
fi, err := os.Stat(preparePath(statDir))
if err != nil {
if os.IsNotExist(err) {
return nil, errVolumeNotFound
} else if os.IsPermission(err) {
return nil, errVolumeAccessDenied
}
return nil, err
}
if !fi.IsDir() {
return nil, errVolumeAccessDenied
}
return fi, nil
}
// Lookup if file exists, returns file attributes upon success
func fsStatFile(statFile string) (os.FileInfo, error) {
if statFile == "" {
return nil, errInvalidArgument
}
if err := checkPathLength(statFile); err != nil {
return nil, err
}
fi, err := os.Stat(preparePath(statFile))
if err != nil {
if os.IsNotExist(err) {
return nil, errFileNotFound
} else if os.IsPermission(err) {
return nil, errFileAccessDenied
} else if isSysErrNotDir(err) {
return nil, errFileAccessDenied
} else if isSysErrPathNotFound(err) {
return nil, errFileNotFound
}
return nil, err
}
if fi.IsDir() {
return nil, errFileNotFound
}
return fi, nil
}
// Opens the file at given path, optionally from an offset. Upon success returns
// a readable stream and the size of the readable stream.
func fsOpenFile(readPath string, offset int64) (io.ReadCloser, int64, error) {
if readPath == "" || offset < 0 {
return nil, 0, errInvalidArgument
}
if err := checkPathLength(readPath); err != nil {
return nil, 0, err
}
fr, err := os.Open(preparePath(readPath))
if err != nil {
if os.IsNotExist(err) {
return nil, 0, errFileNotFound
} else if os.IsPermission(err) {
return nil, 0, errFileAccessDenied
} else if isSysErrNotDir(err) {
// File path cannot be verified since one of the parents is a file.
return nil, 0, errFileAccessDenied
} else if isSysErrPathNotFound(err) {
// Add specific case for windows.
return nil, 0, errFileNotFound
}
return nil, 0, err
}
// Stat to get the size of the file at path.
st, err := fr.Stat()
if err != nil {
return nil, 0, err
}
// Verify if its not a regular file, since subsequent Seek is undefined.
if !st.Mode().IsRegular() {
return nil, 0, errIsNotRegular
}
// Seek to the requested offset.
if offset > 0 {
_, err = fr.Seek(offset, os.SEEK_SET)
if err != nil {
return nil, 0, err
}
}
// Success.
return fr, st.Size(), nil
}
// Creates a file and copies data from incoming reader. Staging buffer is used by io.CopyBuffer.
func fsCreateFile(tempObjPath string, reader io.Reader, buf []byte, fallocSize int64) (int64, error) {
if tempObjPath == "" || reader == nil || buf == nil {
return 0, errInvalidArgument
}
if err := checkPathLength(tempObjPath); err != nil {
return 0, err
}
if err := mkdirAll(pathutil.Dir(tempObjPath), 0777); err != nil {
return 0, err
}
writer, err := os.OpenFile(preparePath(tempObjPath), os.O_CREATE|os.O_WRONLY, 0666)
if err != nil {
// File path cannot be verified since one of the parents is a file.
if isSysErrNotDir(err) {
return 0, errFileAccessDenied
}
return 0, err
}
defer writer.Close()
// Fallocate only if the size is final object is known.
if fallocSize > 0 {
if err = fsFAllocate(int(writer.Fd()), 0, fallocSize); err != nil {
return 0, err
}
}
bytesWritten, err := io.CopyBuffer(writer, reader, buf)
if err != nil {
return 0, err
}
return bytesWritten, nil
}
// Removes uploadID at destination path.
func fsRemoveUploadIDPath(basePath, uploadIDPath string) error {
if basePath == "" || uploadIDPath == "" {
return errInvalidArgument
}
// List all the entries in uploadID.
entries, err := readDir(uploadIDPath)
if err != nil && err != errFileNotFound {
return err
}
// Delete all the entries obtained from previous readdir.
for _, entryPath := range entries {
err = fsDeleteFile(basePath, pathJoin(uploadIDPath, entryPath))
if err != nil && err != errFileNotFound {
return err
}
}
return nil
}
// fsFAllocate is similar to Fallocate but provides a convenient
// wrapper to handle various operating system specific errors.
func fsFAllocate(fd int, offset int64, len int64) (err error) {
e := Fallocate(fd, offset, len)
// Ignore errors when Fallocate is not supported in the current system
if e != nil && !isSysErrNoSys(e) && !isSysErrOpNotSupported(e) {
switch {
case isSysErrNoSpace(e):
err = errDiskFull
case isSysErrIO(e):
err = e
default:
// For errors: EBADF, EINTR, EINVAL, ENODEV, EPERM, ESPIPE and ETXTBSY
// Appending was failed anyway, returns unexpected error
err = errUnexpected
}
return err
}
return nil
}
// Renames source path to destination path, creates all the
// missing parents if they don't exist.
func fsRenameFile(sourcePath, destPath string) error {
if err := mkdirAll(pathutil.Dir(destPath), 0777); err != nil {
return traceError(err)
}
if err := os.Rename(preparePath(sourcePath), preparePath(destPath)); err != nil {
return traceError(err)
}
return nil
}
// Delete a file and its parent if it is empty at the destination path.
// this function additionally protects the basePath from being deleted.
func fsDeleteFile(basePath, deletePath string) error {
if err := checkPathLength(basePath); err != nil {
return err
}
if err := checkPathLength(deletePath); err != nil {
return err
}
if basePath == deletePath {
return nil
}
// Verify if the path exists.
pathSt, err := os.Stat(preparePath(deletePath))
if err != nil {
if os.IsNotExist(err) {
return errFileNotFound
} else if os.IsPermission(err) {
return errFileAccessDenied
}
return err
}
if pathSt.IsDir() && !isDirEmpty(deletePath) {
// Verify if directory is empty.
return nil
}
// Attempt to remove path.
if err = os.Remove(preparePath(deletePath)); err != nil {
if os.IsNotExist(err) {
return errFileNotFound
} else if os.IsPermission(err) {
return errFileAccessDenied
} else if isSysErrNotEmpty(err) {
return errVolumeNotEmpty
}
return err
}
// Recursively go down the next path and delete again.
if err := fsDeleteFile(basePath, pathutil.Dir(deletePath)); err != nil {
return err
}
return nil
}

405
cmd/fs-v1-helpers_test.go Normal file
View File

@@ -0,0 +1,405 @@
/*
* 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 (
"bytes"
"runtime"
"testing"
)
func TestFSStats(t *testing.T) {
// create posix test setup
_, path, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer removeAll(path)
// Setup test environment.
if err = fsMkdir(""); err != errInvalidArgument {
t.Fatal("Unexpected error", err)
}
if err = fsMkdir(pathJoin(path, "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001")); err != errFileNameTooLong {
t.Fatal("Unexpected error", err)
}
if err = fsMkdir(pathJoin(path, "success-vol")); err != nil {
t.Fatalf("Unable to create volume, %s", err)
}
var buf = make([]byte, 4096)
var reader = bytes.NewReader([]byte("Hello, world"))
if _, err = fsCreateFile(pathJoin(path, "success-vol", "success-file"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
if err = fsMkdir(pathJoin(path, "success-vol", "success-file")); err != errVolumeExists {
t.Fatal("Unexpected error", err)
}
if _, err = fsCreateFile(pathJoin(path, "success-vol", "path/to/success-file"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
testCases := []struct {
srcFSPath string
srcVol string
srcPath string
expectedErr error
}{
// Test case - 1.
// Test case with valid inputs, expected to pass.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "success-file",
expectedErr: nil,
},
// Test case - 2.
// Test case with valid inputs, expected to pass.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "path/to/success-file",
expectedErr: nil,
},
// Test case - 3.
// Test case with non-existent file.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "nonexistent-file",
expectedErr: errFileNotFound,
},
// Test case - 4.
// Test case with non-existent file path.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "path/2/success-file",
expectedErr: errFileNotFound,
},
// Test case - 5.
// Test case with path being a directory.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "path",
expectedErr: errFileNotFound,
},
// Test case - 6.
// Test case with src path segment > 255.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
expectedErr: errFileNameTooLong,
},
// Test case - 7.
// Test case validate only srcVol exists.
{
srcFSPath: path,
srcVol: "success-vol",
expectedErr: nil,
},
// Test case - 8.
// Test case validate only srcVol doesn't exist.
{
srcFSPath: path,
srcVol: "success-vol-non-existent",
expectedErr: errVolumeNotFound,
},
// Test case - 9.
// Test case validate invalid argument.
{
expectedErr: errInvalidArgument,
},
}
for i, testCase := range testCases {
if testCase.srcPath != "" {
if _, err := fsStatFile(pathJoin(testCase.srcFSPath, testCase.srcVol, testCase.srcPath)); err != testCase.expectedErr {
t.Fatalf("TestPosix case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
} else {
if _, err := fsStatDir(pathJoin(testCase.srcFSPath, testCase.srcVol)); err != testCase.expectedErr {
t.Fatalf("TestPosix case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
}
}
func TestFSCreateAndOpen(t *testing.T) {
// Setup test environment.
_, path, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer removeAll(path)
if err = fsMkdir(pathJoin(path, "success-vol")); err != nil {
t.Fatalf("Unable to create directory, %s", err)
}
if _, err = fsCreateFile("", nil, nil, 0); err != errInvalidArgument {
t.Fatal("Unexpected error", err)
}
if _, _, err = fsOpenFile("", -1); err != errInvalidArgument {
t.Fatal("Unexpected error", err)
}
var buf = make([]byte, 4096)
var reader = bytes.NewReader([]byte("Hello, world"))
if _, err = fsCreateFile(pathJoin(path, "success-vol", "success-file"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
testCases := []struct {
srcVol string
srcPath string
expectedErr error
}{
// Test case - 1.
// Test case with segment of the volume name > 255.
{
srcVol: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
srcPath: "success-file",
expectedErr: errFileNameTooLong,
},
// Test case - 2.
// Test case with src path segment > 255.
{
srcVol: "success-vol",
srcPath: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
expectedErr: errFileNameTooLong,
},
}
for i, testCase := range testCases {
_, err = fsCreateFile(pathJoin(path, testCase.srcVol, testCase.srcPath), reader, buf, reader.Size())
if err != testCase.expectedErr {
t.Errorf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
_, _, err = fsOpenFile(pathJoin(path, testCase.srcVol, testCase.srcPath), 0)
if err != testCase.expectedErr {
t.Errorf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
// Attempt to open a directory.
if _, _, err = fsOpenFile(pathJoin(path), 0); err != errIsNotRegular {
t.Fatal("Unexpected error", err)
}
}
func TestFSDeletes(t *testing.T) {
// create posix test setup
_, path, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer removeAll(path)
// Setup test environment.
if err = fsMkdir(pathJoin(path, "success-vol")); err != nil {
t.Fatalf("Unable to create directory, %s", err)
}
var buf = make([]byte, 4096)
var reader = bytes.NewReader([]byte("Hello, world"))
if _, err = fsCreateFile(pathJoin(path, "success-vol", "success-file"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
testCases := []struct {
srcVol string
srcPath string
expectedErr error
}{
// Test case - 1.
// valid case with existing volume and file to delete.
{
srcVol: "success-vol",
srcPath: "success-file",
expectedErr: nil,
},
// Test case - 2.
// The file was deleted in the last case, so DeleteFile should fail.
{
srcVol: "success-vol",
srcPath: "success-file",
expectedErr: errFileNotFound,
},
// Test case - 3.
// Test case with segment of the volume name > 255.
{
srcVol: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
srcPath: "success-file",
expectedErr: errFileNameTooLong,
},
// Test case - 4.
// Test case with src path segment > 255.
{
srcVol: "success-vol",
srcPath: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
expectedErr: errFileNameTooLong,
},
}
for i, testCase := range testCases {
if err = fsDeleteFile(path, pathJoin(path, testCase.srcVol, testCase.srcPath)); err != testCase.expectedErr {
t.Errorf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
}
// Tests fs removes.
func TestFSRemoves(t *testing.T) {
// create posix test setup
_, path, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer removeAll(path)
// Setup test environment.
if err = fsMkdir(pathJoin(path, "success-vol")); err != nil {
t.Fatalf("Unable to create directory, %s", err)
}
var buf = make([]byte, 4096)
var reader = bytes.NewReader([]byte("Hello, world"))
if _, err = fsCreateFile(pathJoin(path, "success-vol", "success-file"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
if _, err = fsCreateFile(pathJoin(path, "success-vol", "success-file-new"), reader, buf, reader.Size()); err != nil {
t.Fatalf("Unable to create file, %s", err)
}
// Seek back.
reader.Seek(0, 0)
testCases := []struct {
srcFSPath string
srcVol string
srcPath string
expectedErr error
}{
// Test case - 1.
// valid case with existing volume and file to delete.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "success-file",
expectedErr: nil,
},
// Test case - 2.
// The file was deleted in the last case, so DeleteFile should fail.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "success-file",
expectedErr: errFileNotFound,
},
// Test case - 3.
// Test case with segment of the volume name > 255.
{
srcFSPath: path,
srcVol: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
srcPath: "success-file",
expectedErr: errFileNameTooLong,
},
// Test case - 4.
// Test case with src path segment > 255.
{
srcFSPath: path,
srcVol: "success-vol",
srcPath: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
expectedErr: errFileNameTooLong,
},
// Test case - 5.
// Test case with src path empty.
{
srcFSPath: path,
srcVol: "success-vol",
expectedErr: errVolumeNotEmpty,
},
// Test case - 6.
// Test case with src path empty.
{
srcFSPath: path,
srcVol: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
expectedErr: errFileNameTooLong,
},
// Test case - 7.
// Test case with src path empty.
{
srcFSPath: path,
srcVol: "non-existent",
expectedErr: errVolumeNotFound,
},
// Test case - 8.
// Test case with src and volume path empty.
{
expectedErr: errInvalidArgument,
},
}
for i, testCase := range testCases {
if testCase.srcPath != "" {
if err = fsRemoveFile(pathJoin(testCase.srcFSPath, testCase.srcVol, testCase.srcPath)); err != testCase.expectedErr {
t.Errorf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
} else {
if err = fsRemoveDir(pathJoin(testCase.srcFSPath, testCase.srcVol, testCase.srcPath)); err != testCase.expectedErr {
t.Error(err)
}
}
}
if err = fsRemoveAll(pathJoin(path, "success-vol")); err != nil {
t.Fatal(err)
}
if err = fsRemoveAll(""); err != errInvalidArgument {
t.Fatal(err)
}
if err = fsRemoveAll("my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001"); err != errFileNameTooLong {
t.Fatal(err)
}
if runtime.GOOS != "windows" {
if err = fsRemoveAll("/usr"); err != errVolumeAccessDenied {
t.Fatal(err)
}
}
}

View File

@@ -18,7 +18,15 @@ package cmd
import (
"encoding/json"
"io"
"io/ioutil"
"os"
pathutil "path"
"sort"
"strings"
"github.com/minio/minio/pkg/lock"
"github.com/minio/minio/pkg/mimedb"
)
const (
@@ -38,6 +46,50 @@ type fsMetaV1 struct {
Parts []objectPartInfo `json:"parts,omitempty"`
}
// Converts metadata to object info.
func (m fsMetaV1) ToObjectInfo(bucket, object string, fi os.FileInfo) ObjectInfo {
if len(m.Meta) == 0 {
m.Meta = make(map[string]string)
}
// Guess content-type from the extension if possible.
if m.Meta["content-type"] == "" {
if objectExt := pathutil.Ext(object); objectExt != "" {
if content, ok := mimedb.DB[strings.ToLower(strings.TrimPrefix(objectExt, "."))]; ok {
m.Meta["content-type"] = content.ContentType
}
}
}
objInfo := ObjectInfo{
Bucket: bucket,
Name: object,
}
// We set file into only if its valid.
objInfo.ModTime = timeSentinel
if fi != nil {
objInfo.ModTime = fi.ModTime()
objInfo.Size = fi.Size()
objInfo.IsDir = fi.IsDir()
}
objInfo.MD5Sum = m.Meta["md5Sum"]
objInfo.ContentType = m.Meta["content-type"]
objInfo.ContentEncoding = m.Meta["content-encoding"]
// md5Sum has already been extracted into objInfo.MD5Sum. We
// need to remove it from m.Meta to avoid it from appearing as
// part of response headers. e.g, X-Minio-* or X-Amz-*.
delete(m.Meta, "md5Sum")
// Save all the other userdefined API.
objInfo.UserDefined = m.Meta
// Success..
return objInfo
}
// ObjectPartIndex - returns the index of matching object part number.
func (m fsMetaV1) ObjectPartIndex(partNumber int) (partIndex int) {
for i, part := range m.Parts {
@@ -73,41 +125,43 @@ func (m *fsMetaV1) AddObjectPart(partNumber int, partName string, partETag strin
sort.Sort(byObjectPartNumber(m.Parts))
}
// readFSMetadata - returns the object metadata `fs.json` content.
func readFSMetadata(disk StorageAPI, bucket, filePath string) (fsMeta fsMetaV1, err error) {
// Read all `fs.json`.
buf, err := disk.ReadAll(bucket, filePath)
func (m *fsMetaV1) WriteTo(writer io.Writer) (n int64, err error) {
var metadataBytes []byte
metadataBytes, err = json.Marshal(m)
if err != nil {
return fsMetaV1{}, traceError(err)
return 0, traceError(err)
}
// Decode `fs.json` into fsMeta structure.
if err = json.Unmarshal(buf, &fsMeta); err != nil {
return fsMetaV1{}, traceError(err)
if err = writer.(*lock.LockedFile).Truncate(0); err != nil {
return 0, traceError(err)
}
if _, err = writer.Write(metadataBytes); err != nil {
return 0, traceError(err)
}
// Success.
return fsMeta, nil
return int64(len(metadataBytes)), nil
}
// Write fsMeta to fs.json or fs-append.json.
func writeFSMetadata(disk StorageAPI, bucket, filePath string, fsMeta fsMetaV1) error {
tmpPath := mustGetUUID()
metadataBytes, err := json.Marshal(fsMeta)
func (m *fsMetaV1) ReadFrom(reader io.Reader) (n int64, err error) {
var metadataBytes []byte
metadataBytes, err = ioutil.ReadAll(reader)
if err != nil {
return traceError(err)
return 0, traceError(err)
}
if err = disk.AppendFile(minioMetaTmpBucket, tmpPath, metadataBytes); err != nil {
return traceError(err)
if len(metadataBytes) == 0 {
return 0, traceError(io.EOF)
}
err = disk.RenameFile(minioMetaTmpBucket, tmpPath, bucket, filePath)
if err != nil {
err = disk.DeleteFile(minioMetaTmpBucket, tmpPath)
if err != nil {
return traceError(err)
}
// Decode `fs.json` into fsMeta structure.
if err = json.Unmarshal(metadataBytes, m); err != nil {
return 0, traceError(err)
}
return nil
// Success.
return int64(len(metadataBytes)), nil
}
// newFSMetaV1 - initializes new fsMetaV1.
@@ -130,21 +184,49 @@ func newFSFormatV1() (format *formatConfigV1) {
}
}
// isFSFormat - returns whether given formatConfigV1 is FS type or not.
func isFSFormat(format *formatConfigV1) bool {
return format.Format == "fs"
// loads format.json from minioMetaBucket if it exists.
func loadFormatFS(fsPath string) (*formatConfigV1, error) {
rlk, err := lock.RLockedOpenFile(pathJoin(fsPath, minioMetaBucket, fsFormatJSONFile))
if err != nil {
if os.IsNotExist(err) {
return nil, errUnformattedDisk
}
return nil, err
}
defer rlk.Close()
formatBytes, err := ioutil.ReadAll(rlk)
if err != nil {
return nil, err
}
format := &formatConfigV1{}
if err = json.Unmarshal(formatBytes, format); err != nil {
return nil, err
}
return format, nil
}
// writes FS format (format.json) into minioMetaBucket.
func saveFSFormatData(storage StorageAPI, fsFormat *formatConfigV1) error {
func saveFormatFS(formatPath string, fsFormat *formatConfigV1) error {
metadataBytes, err := json.Marshal(fsFormat)
if err != nil {
return err
}
// fsFormatJSONFile - format.json file stored in minioMetaBucket(.minio) directory.
if err = storage.AppendFile(minioMetaBucket, fsFormatJSONFile, metadataBytes); err != nil {
// fsFormatJSONFile - format.json file stored in minioMetaBucket(.minio.sys) directory.
lk, err := lock.LockedOpenFile(preparePath(formatPath), os.O_CREATE|os.O_WRONLY, 0600)
if err != nil {
return err
}
defer lk.Close()
if _, err = lk.Write(metadataBytes); err != nil {
return err
}
// Success.
return nil
}
@@ -153,11 +235,13 @@ func isPartsSame(uploadedParts []objectPartInfo, completeParts []completePart) b
if len(uploadedParts) != len(completeParts) {
return false
}
for i := range completeParts {
if uploadedParts[i].Number != completeParts[i].PartNumber ||
uploadedParts[i].ETag != completeParts[i].ETag {
return false
}
}
return true
}

View File

@@ -18,30 +18,34 @@ package cmd
import (
"bytes"
"io"
"os"
"path/filepath"
"testing"
)
func initFSObjects(disk string, t *testing.T) (obj ObjectLayer) {
endpoints, err := parseStorageEndpoints([]string{disk})
if err != nil {
t.Fatal(err)
// Tests ToObjectInfo function.
func TestFSV1MetadataObjInfo(t *testing.T) {
fsMeta := newFSMetaV1()
objInfo := fsMeta.ToObjectInfo("testbucket", "testobject", nil)
if objInfo.Size != 0 {
t.Fatal("Unexpected object info value for Size", objInfo.Size)
}
obj, _, err = initObjectLayer(endpoints)
if err != nil {
t.Fatal("Unexpected err: ", err)
if objInfo.ModTime != timeSentinel {
t.Fatal("Unexpected object info value for ModTime ", objInfo.ModTime)
}
if objInfo.IsDir {
t.Fatal("Unexpected object info value for IsDir", objInfo.IsDir)
}
return obj
}
// TestReadFsMetadata - readFSMetadata testing with a healthy and faulty disk
// TestReadFSMetadata - readFSMetadata testing with a healthy and faulty disk
func TestReadFSMetadata(t *testing.T) {
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
@@ -56,37 +60,42 @@ func TestReadFSMetadata(t *testing.T) {
}
// Construct the full path of fs.json
fsPath := "buckets/" + bucketName + "/" + objectName + "/fs.json"
fsPath := pathJoin("buckets", bucketName, objectName, "fs.json")
fsPath = pathJoin(fs.fsPath, minioMetaBucket, fsPath)
rlk, err := fs.rwPool.Open(fsPath)
if err != nil {
t.Fatal("Unexpected error ", err)
}
defer rlk.Close()
sectionReader := io.NewSectionReader(rlk, 0, rlk.Size())
// Regular fs metadata reading, no errors expected
if _, err := readFSMetadata(fs.storage, ".minio.sys", fsPath); err != nil {
fsMeta := fsMetaV1{}
if _, err = fsMeta.ReadFrom(sectionReader); err != nil {
t.Fatal("Unexpected error ", err)
}
// Corrupted fs.json
if err := fs.storage.AppendFile(".minio.sys", fsPath, []byte{'a'}); err != nil {
file, err := os.OpenFile(preparePath(fsPath), os.O_APPEND|os.O_WRONLY, 0666)
if err != nil {
t.Fatal("Unexpected error ", err)
}
if _, err := readFSMetadata(fs.storage, ".minio.sys", fsPath); err == nil {
file.Write([]byte{'a'})
file.Close()
fsMeta = fsMetaV1{}
if _, err := fsMeta.ReadFrom(sectionReader); err == nil {
t.Fatal("Should fail", err)
}
// Test with corrupted disk
fsStorage := fs.storage.(*retryStorage)
naughty := newNaughtyDisk(fsStorage, nil, errFaultyDisk)
fs.storage = naughty
if _, err := readFSMetadata(fs.storage, ".minio.sys", fsPath); errorCause(err) != errFaultyDisk {
t.Fatal("Should fail", err)
}
}
// TestWriteFsMetadata - tests of writeFSMetadata with healthy and faulty disks
// TestWriteFSMetadata - tests of writeFSMetadata with healthy disk.
func TestWriteFSMetadata(t *testing.T) {
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
@@ -100,24 +109,27 @@ func TestWriteFSMetadata(t *testing.T) {
t.Fatal("Unexpected err: ", err)
}
// Construct the complete path of fs.json
fsPath := "buckets/" + bucketName + "/" + objectName + "/fs.json"
// Construct the full path of fs.json
fsPath := pathJoin("buckets", bucketName, objectName, "fs.json")
fsPath = pathJoin(fs.fsPath, minioMetaBucket, fsPath)
// Fs metadata reading, no errors expected (healthy disk)
fsMeta, err := readFSMetadata(fs.storage, ".minio.sys", fsPath)
rlk, err := fs.rwPool.Open(fsPath)
if err != nil {
t.Fatal("Unexpected error ", err)
}
defer rlk.Close()
// Reading metadata with a corrupted disk
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 2; i++ {
naughty := newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk, i + 1: errFaultyDisk}, nil)
fs.storage = naughty
if err = writeFSMetadata(fs.storage, ".minio.sys", fsPath, fsMeta); errorCause(err) != errFaultyDisk {
t.Fatal("Unexpected error", i, err)
}
sectionReader := io.NewSectionReader(rlk, 0, rlk.Size())
// FS metadata reading, no errors expected (healthy disk)
fsMeta := fsMetaV1{}
_, err = fsMeta.ReadFrom(sectionReader)
if err != nil {
t.Fatal("Unexpected error ", err)
}
if fsMeta.Version != "1.0.0" {
t.Fatalf("Unexpected version %s", fsMeta.Version)
}
if fsMeta.Format != "fs" {
t.Fatalf("Unexpected format %s", fsMeta.Format)
}
}

View File

@@ -17,72 +17,126 @@
package cmd
import (
"path"
"fmt"
"io"
"runtime"
"time"
pathutil "path"
"github.com/minio/minio/pkg/lock"
)
// Returns if the prefix is a multipart upload.
func (fs fsObjects) isMultipartUpload(bucket, prefix string) bool {
_, err := fs.storage.StatFile(bucket, pathJoin(prefix, uploadsJSONFile))
return err == nil
}
// isUploadIDExists - verify if a given uploadID exists and is valid.
func (fs fsObjects) isUploadIDExists(bucket, object, uploadID string) bool {
uploadIDPath := path.Join(bucket, object, uploadID)
_, err := fs.storage.StatFile(minioMetaMultipartBucket, path.Join(uploadIDPath, fsMetaJSONFile))
uploadsIDPath := pathJoin(fs.fsPath, bucket, prefix, uploadsJSONFile)
_, err := fsStatFile(uploadsIDPath)
if err != nil {
if err == errFileNotFound {
return false
}
errorIf(err, "Unable to access upload id "+pathJoin(minioMetaMultipartBucket, uploadIDPath))
errorIf(err, "Unable to access uploads.json "+uploadsIDPath)
return false
}
return true
}
// updateUploadJSON - add or remove upload ID info in all `uploads.json`.
func (fs fsObjects) updateUploadJSON(bucket, object, uploadID string, initiated time.Time, isRemove bool) error {
uploadsPath := path.Join(bucket, object, uploadsJSONFile)
tmpUploadsPath := mustGetUUID()
// Delete uploads.json file wrapper handling a tricky case on windows.
func (fs fsObjects) deleteUploadsJSON(bucket, object, uploadID string) error {
timeID := fmt.Sprintf("%X", time.Now().UTC().UnixNano())
tmpPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, uploadID+"+"+timeID)
uploadsJSON, err := readUploadsJSON(bucket, object, fs.storage)
if errorCause(err) == errFileNotFound {
// If file is not found, we assume a default (empty)
// upload info.
uploadsJSON, err = newUploadsV1("fs"), nil
multipartBucketPath := pathJoin(fs.fsPath, minioMetaMultipartBucket)
uploadPath := pathJoin(multipartBucketPath, bucket, object)
uploadsMetaPath := pathJoin(uploadPath, uploadsJSONFile)
// Special case for windows please read through.
if runtime.GOOS == "windows" {
// Ordinarily windows does not permit deletion or renaming of files still
// in use, but if all open handles to that file were opened with FILE_SHARE_DELETE
// then it can permit renames and deletions of open files.
//
// There are however some gotchas with this, and it is worth listing them here.
// Firstly, Windows never allows you to really delete an open file, rather it is
// flagged as delete pending and its entry in its directory remains visible
// (though no new file handles may be opened to it) and when the very last
// open handle to the file in the system is closed, only then is it truly
// deleted. Well, actually only sort of truly deleted, because Windows only
// appears to remove the file entry from the directory, but in fact that
// entry is merely hidden and actually still exists and attempting to create
// a file with the same name will return an access denied error. How long it
// silently exists for depends on a range of factors, but put it this way:
// if your code loops creating and deleting the same file name as you might
// when operating a lock file, you're going to see lots of random spurious
// access denied errors and truly dismal lock file performance compared to POSIX.
//
// We work-around these un-POSIX file semantics by taking a dual step to
// deleting files. Firstly, it renames the file to tmp location into multipartTmpBucket
// We always open files with FILE_SHARE_DELETE permission enabled, with that
// flag Windows permits renaming and deletion, and because the name was changed
// to a very random name somewhere not in its origin directory before deletion,
// you don't see those unexpected random errors when creating files with the
// same name as a recently deleted file as you do anywhere else on Windows.
// Because the file is probably not in its original containing directory any more,
// deletions of that directory will not fail with “directory not empty” as they
// otherwise normally would either.
fsRenameFile(uploadsMetaPath, tmpPath)
// Proceed to deleting the directory.
if err := fsDeleteFile(multipartBucketPath, uploadPath); err != nil {
return err
}
// Finally delete the renamed file.
return fsDeleteFile(pathutil.Dir(tmpPath), tmpPath)
}
return fsDeleteFile(multipartBucketPath, uploadsMetaPath)
}
// Removes the uploadID, called either by CompleteMultipart of AbortMultipart. If the resuling uploads
// slice is empty then we remove/purge the file.
func (fs fsObjects) removeUploadID(bucket, object, uploadID string, rwlk *lock.LockedFile) error {
uploadIDs := uploadsV1{}
_, err := uploadIDs.ReadFrom(io.NewSectionReader(rwlk, 0, rwlk.Size()))
if err != nil {
return err
}
// update the uploadsJSON struct
if !isRemove {
// Add the uploadID
uploadsJSON.AddUploadID(uploadID, initiated)
} else {
// Remove the upload ID
uploadsJSON.RemoveUploadID(uploadID)
}
// Removes upload id from the uploads list.
uploadIDs.RemoveUploadID(uploadID)
// update the file or delete it?
if len(uploadsJSON.Uploads) > 0 {
err = writeUploadJSON(&uploadsJSON, uploadsPath, tmpUploadsPath, fs.storage)
} else {
// no uploads, so we delete the file.
if err = fs.storage.DeleteFile(minioMetaMultipartBucket, uploadsPath); err != nil {
return toObjectErr(traceError(err), minioMetaMultipartBucket, uploadsPath)
}
}
// Check this is the last entry.
if uploadIDs.IsEmpty() {
// No more uploads left, so we delete `uploads.json` file.
return fs.deleteUploadsJSON(bucket, object, uploadID)
} // else not empty
// Write update `uploads.json`.
_, err = uploadIDs.WriteTo(rwlk)
return err
}
// addUploadID - add upload ID and its initiated time to 'uploads.json'.
func (fs fsObjects) addUploadID(bucket, object string, uploadID string, initiated time.Time) error {
return fs.updateUploadJSON(bucket, object, uploadID, initiated, false)
}
// Adds a new uploadID if no previous `uploads.json` is
// found we initialize a new one.
func (fs fsObjects) addUploadID(bucket, object, uploadID string, initiated time.Time, rwlk *lock.LockedFile) error {
uploadIDs := uploadsV1{}
// removeUploadID - remove upload ID in 'uploads.json'.
func (fs fsObjects) removeUploadID(bucket, object string, uploadID string) error {
return fs.updateUploadJSON(bucket, object, uploadID, time.Time{}, true)
_, err := uploadIDs.ReadFrom(io.NewSectionReader(rwlk, 0, rwlk.Size()))
// For all unexpected errors, we return.
if err != nil && errorCause(err) != io.EOF {
return err
}
// If we couldn't read anything, we assume a default
// (empty) upload info.
if errorCause(err) == io.EOF {
uploadIDs = newUploadsV1("fs")
}
// Adds new upload id to the list.
uploadIDs.AddUploadID(uploadID, initiated)
// Write update `uploads.json`.
_, err = uploadIDs.WriteTo(rwlk)
return err
}

View File

@@ -19,53 +19,8 @@ package cmd
import (
"path/filepath"
"testing"
"time"
)
// TestFSIsUploadExists - complete test with valid and invalid cases
func TestFSIsUploadExists(t *testing.T) {
// Prepare for testing
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
bucketName := "bucket"
objectName := "object"
if err := obj.MakeBucket(bucketName); err != nil {
t.Fatal("Unexpected err: ", err)
}
uploadID, err := obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
// Test with valid upload id
if exists := fs.isUploadIDExists(bucketName, objectName, uploadID); !exists {
t.Fatal("Wrong result, expected: ", exists)
}
// Test with inexistant bucket/object names
if exists := fs.isUploadIDExists("bucketfoo", "objectfoo", uploadID); exists {
t.Fatal("Wrong result, expected: ", !exists)
}
// Test with inexistant upload ID
if exists := fs.isUploadIDExists(bucketName, objectName, uploadID+"-ff"); exists {
t.Fatal("Wrong result, expected: ", !exists)
}
// isUploadIdExists with a faulty disk should return false
fsStorage := fs.storage.(*retryStorage)
naughty := newNaughtyDisk(fsStorage, nil, errFaultyDisk)
fs.storage = naughty
if exists := fs.isUploadIDExists(bucketName, objectName, uploadID); exists {
t.Fatal("Wrong result, expected: ", !exists)
}
}
// TestFSWriteUploadJSON - tests for writeUploadJSON for FS
func TestFSWriteUploadJSON(t *testing.T) {
// Prepare for tests
@@ -73,31 +28,21 @@ func TestFSWriteUploadJSON(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
bucketName := "bucket"
objectName := "object"
obj.MakeBucket(bucketName)
uploadID, err := obj.NewMultipartUpload(bucketName, objectName, nil)
_, err := obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
// newMultipartUpload will fail.
removeAll(disk) // Remove disk.
_, err = obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
if err := fs.addUploadID(bucketName, objectName, uploadID, time.Now().UTC()); err != nil {
t.Fatal("Unexpected err: ", err)
}
// isUploadIdExists with a faulty disk should return false
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 3; i++ {
naughty := newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
fs.storage = naughty
if err := fs.addUploadID(bucketName, objectName, uploadID, time.Now().UTC()); errorCause(err) != errFaultyDisk {
if _, ok := errorCause(err).(BucketNotFound); !ok {
t.Fatal("Unexpected err: ", err)
}
}

View File

@@ -22,13 +22,68 @@ import (
"fmt"
"hash"
"io"
"path"
"os"
pathutil "path"
"strings"
"time"
"github.com/minio/sha256-simd"
)
// listMultipartUploadIDs - list all the upload ids from a marker up to 'count'.
func (fs fsObjects) listMultipartUploadIDs(bucketName, objectName, uploadIDMarker string, count int) ([]uploadMetadata, bool, error) {
var uploads []uploadMetadata
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, pathJoin(bucketName, objectName))
objectMPartPathLock.RLock()
defer objectMPartPathLock.RUnlock()
uploadsPath := pathJoin(bucketName, objectName, uploadsJSONFile)
rlk, err := fs.rwPool.Open(pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadsPath))
if err != nil {
if err == errFileNotFound || err == errFileAccessDenied {
return nil, true, nil
}
return nil, false, traceError(err)
}
defer fs.rwPool.Close(pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadsPath))
// Read `uploads.json`.
uploadIDs := uploadsV1{}
if _, err = uploadIDs.ReadFrom(io.NewSectionReader(rlk, 0, rlk.Size())); err != nil {
return nil, false, err
}
index := 0
if uploadIDMarker != "" {
for ; index < len(uploadIDs.Uploads); index++ {
if uploadIDs.Uploads[index].UploadID == uploadIDMarker {
// Skip the uploadID as it would already be listed in previous listing.
index++
break
}
}
}
for index < len(uploadIDs.Uploads) {
uploads = append(uploads, uploadMetadata{
Object: objectName,
UploadID: uploadIDs.Uploads[index].UploadID,
Initiated: uploadIDs.Uploads[index].Initiated,
})
count--
index++
if count == 0 {
break
}
}
end := (index == len(uploadIDs.Uploads))
return uploads, end, nil
}
// listMultipartUploads - lists all multipart uploads.
func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMarker, delimiter string, maxUploads int) (ListMultipartsInfo, error) {
result := ListMultipartsInfo{}
@@ -54,31 +109,40 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
if keyMarker != "" {
multipartMarkerPath = pathJoin(bucket, keyMarker)
}
var uploads []uploadMetadata
var err error
var eof bool
if uploadIDMarker != "" {
keyMarkerLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, keyMarker))
keyMarkerLock.RLock()
uploads, _, err = listMultipartUploadIDs(bucket, keyMarker, uploadIDMarker, maxUploads, fs.storage)
keyMarkerLock.RUnlock()
uploads, _, err = fs.listMultipartUploadIDs(bucket, keyMarker, uploadIDMarker, maxUploads)
if err != nil {
return ListMultipartsInfo{}, err
}
maxUploads = maxUploads - len(uploads)
}
var walkResultCh chan treeWalkResult
var endWalkCh chan struct{}
heal := false // true only for xl.ListObjectsHeal()
// true only for xl.ListObjectsHeal(), set to false.
heal := false
// Proceed to list only if we have more uploads to be listed.
if maxUploads > 0 {
walkResultCh, endWalkCh = fs.listPool.Release(listParams{minioMetaMultipartBucket, recursive, multipartMarkerPath, multipartPrefixPath, heal})
listPrms := listParams{minioMetaMultipartBucket, recursive, multipartMarkerPath, multipartPrefixPath, heal}
// Pop out any previously waiting marker.
walkResultCh, endWalkCh = fs.listPool.Release(listPrms)
if walkResultCh == nil {
endWalkCh = make(chan struct{})
isLeaf := fs.isMultipartUpload
listDir := listDirFactory(isLeaf, fsTreeWalkIgnoredErrs, fs.storage)
walkResultCh = startTreeWalk(minioMetaMultipartBucket, multipartPrefixPath, multipartMarkerPath, recursive, listDir, isLeaf, endWalkCh)
listDir := fs.listDirFactory(isLeaf)
walkResultCh = startTreeWalk(minioMetaMultipartBucket, multipartPrefixPath,
multipartMarkerPath, recursive, listDir, isLeaf, endWalkCh)
}
// List until maxUploads requested.
for maxUploads > 0 {
walkResult, ok := <-walkResultCh
if !ok {
@@ -86,6 +150,7 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
eof = true
break
}
// For any walk error return right away.
if walkResult.err != nil {
// File not found or Disk not found is a valid case.
@@ -95,6 +160,7 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
}
return ListMultipartsInfo{}, walkResult.err
}
entry := strings.TrimPrefix(walkResult.entry, retainSlash(bucket))
if strings.HasSuffix(walkResult.entry, slashSeparator) {
uploads = append(uploads, uploadMetadata{
@@ -109,18 +175,16 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
}
continue
}
var tmpUploads []uploadMetadata
var end bool
uploadIDMarker = ""
entryLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, entry))
entryLock.RLock()
tmpUploads, end, err = listMultipartUploadIDs(bucket, entry, uploadIDMarker, maxUploads, fs.storage)
entryLock.RUnlock()
tmpUploads, end, err = fs.listMultipartUploadIDs(bucket, entry, uploadIDMarker, maxUploads)
if err != nil {
return ListMultipartsInfo{}, err
}
uploads = append(uploads, tmpUploads...)
maxUploads -= len(tmpUploads)
if walkResult.end && end {
@@ -129,6 +193,7 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
}
}
}
// Loop through all the received uploads fill in the multiparts result.
for _, upload := range uploads {
var objectName string
@@ -158,6 +223,8 @@ func (fs fsObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
result.NextKeyMarker = ""
result.NextUploadIDMarker = ""
}
// Success.
return result, nil
}
@@ -174,6 +241,11 @@ func (fs fsObjects) ListMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
if err := checkListMultipartArgs(bucket, prefix, keyMarker, uploadIDMarker, delimiter, fs); err != nil {
return ListMultipartsInfo{}, err
}
if _, err := fs.statBucketDir(bucket); err != nil {
return ListMultipartsInfo{}, toObjectErr(err, bucket)
}
return fs.listMultipartUploads(bucket, prefix, keyMarker, uploadIDMarker, delimiter, maxUploads)
}
@@ -191,23 +263,35 @@ func (fs fsObjects) newMultipartUpload(bucket string, object string, meta map[st
// Save additional metadata.
fsMeta.Meta = meta
// This lock needs to be held for any changes to the directory
// contents of ".minio.sys/multipart/object/"
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
uploadID = mustGetUUID()
initiated := time.Now().UTC()
// Add upload ID to uploads.json
if err = fs.addUploadID(bucket, object, uploadID, initiated); err != nil {
return "", err
uploadsPath := pathJoin(bucket, object, uploadsJSONFile)
rwlk, err := fs.rwPool.Create(pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadsPath))
if err != nil {
return "", toObjectErr(traceError(err), bucket, object)
}
uploadIDPath := path.Join(bucket, object, uploadID)
if err = writeFSMetadata(fs.storage, minioMetaMultipartBucket, path.Join(uploadIDPath, fsMetaJSONFile), fsMeta); err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, uploadIDPath)
defer rwlk.Close()
uploadIDPath := pathJoin(bucket, object, uploadID)
fsMetaPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, fsMetaJSONFile)
metaFile, err := fs.rwPool.Create(fsMetaPath)
if err != nil {
return "", toObjectErr(traceError(err), bucket, object)
}
defer metaFile.Close()
// Add a new upload id.
if err = fs.addUploadID(bucket, object, uploadID, initiated, rwlk); err != nil {
return "", toObjectErr(err, bucket, object)
}
// Write all the set metadata.
if _, err = fsMeta.WriteTo(metaFile); err != nil {
return "", toObjectErr(err, bucket, object)
}
// Return success.
return uploadID, nil
}
@@ -221,6 +305,17 @@ func (fs fsObjects) NewMultipartUpload(bucket, object string, meta map[string]st
if err := checkNewMultipartArgs(bucket, object, fs); err != nil {
return "", err
}
if _, err := fs.statBucketDir(bucket); err != nil {
return "", toObjectErr(err, bucket)
}
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
return fs.newMultipartUpload(bucket, object, meta)
}
@@ -229,12 +324,14 @@ func partToAppend(fsMeta fsMetaV1, fsAppendMeta fsMetaV1) (part objectPartInfo,
if len(fsMeta.Parts) == 0 {
return
}
// As fsAppendMeta.Parts will be sorted len(fsAppendMeta.Parts) will naturally be the next part number
nextPartNum := len(fsAppendMeta.Parts) + 1
nextPartIndex := fsMeta.ObjectPartIndex(nextPartNum)
if nextPartIndex == -1 {
return
}
return fsMeta.Parts[nextPartIndex], true
}
@@ -247,15 +344,43 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
return "", err
}
uploadIDPath := path.Join(bucket, object, uploadID)
if _, err := fs.statBucketDir(bucket); err != nil {
return "", toObjectErr(err, bucket)
}
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
// Disallow any parallel abort or complete multipart operations.
uploadsPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object, uploadsJSONFile)
if _, err := fs.rwPool.Open(uploadsPath); err != nil {
if err == errFileNotFound || err == errFileAccessDenied {
return "", traceError(InvalidUploadID{UploadID: uploadID})
}
return "", toObjectErr(traceError(err), bucket, object)
}
defer fs.rwPool.Close(uploadsPath)
uploadIDPath := pathJoin(bucket, object, uploadID)
preUploadIDLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, uploadIDPath)
preUploadIDLock.RLock()
// Just check if the uploadID exists to avoid copy if it doesn't.
uploadIDExists := fs.isUploadIDExists(bucket, object, uploadID)
preUploadIDLock.RUnlock()
if !uploadIDExists {
return "", traceError(InvalidUploadID{UploadID: uploadID})
fsMetaPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, fsMetaJSONFile)
rwlk, err := fs.rwPool.Write(fsMetaPath)
if err != nil {
if err == errFileNotFound || err == errFileAccessDenied {
return "", traceError(InvalidUploadID{UploadID: uploadID})
}
return "", toObjectErr(traceError(err), bucket, object)
}
defer rwlk.Close()
fsMeta := fsMetaV1{}
_, err = fsMeta.ReadFrom(io.NewSectionReader(rwlk, 0, rwlk.Size()))
if err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, fsMetaPath)
}
partSuffix := fmt.Sprintf("object%d", partID)
@@ -263,7 +388,6 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
// Initialize md5 writer.
md5Writer := md5.New()
hashWriters := []io.Writer{md5Writer}
var sha256Writer hash.Hash
@@ -272,6 +396,7 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
hashWriters = append(hashWriters, sha256Writer)
}
multiWriter := io.MultiWriter(hashWriters...)
// Limit the reader to its provided size if specified.
var limitDataReader io.Reader
if size > 0 {
@@ -289,31 +414,24 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
}
buf := make([]byte, int(bufSize))
if size > 0 {
// Prepare file to avoid disk fragmentation
err := fs.storage.PrepareFile(minioMetaTmpBucket, tmpPartPath, size)
if err != nil {
return "", toObjectErr(err, minioMetaTmpBucket, tmpPartPath)
}
}
bytesWritten, cErr := fsCreateFile(fs.storage, teeReader, buf, minioMetaTmpBucket, tmpPartPath)
fsPartPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, tmpPartPath)
bytesWritten, cErr := fsCreateFile(fsPartPath, teeReader, buf, size)
if cErr != nil {
fs.storage.DeleteFile(minioMetaTmpBucket, tmpPartPath)
fsRemoveFile(fsPartPath)
return "", toObjectErr(cErr, minioMetaTmpBucket, tmpPartPath)
}
// Should return IncompleteBody{} error when reader has fewer
// bytes than specified in request header.
if bytesWritten < size {
fs.storage.DeleteFile(minioMetaTmpBucket, tmpPartPath)
fsRemoveFile(fsPartPath)
return "", traceError(IncompleteBody{})
}
// Delete temporary part in case of failure. If
// PutObjectPart succeeds then there would be nothing to
// delete.
defer fs.storage.DeleteFile(minioMetaTmpBucket, tmpPartPath)
defer fsRemoveFile(fsPartPath)
newMD5Hex := hex.EncodeToString(md5Writer.Sum(nil))
if md5Hex != "" {
@@ -329,45 +447,32 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
}
}
// Hold write lock as we are updating fs.json
postUploadIDLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, uploadIDPath)
postUploadIDLock.Lock()
defer postUploadIDLock.Unlock()
// Just check if the uploadID exists to avoid copy if it doesn't.
if !fs.isUploadIDExists(bucket, object, uploadID) {
return "", traceError(InvalidUploadID{UploadID: uploadID})
}
fsMetaPath := pathJoin(uploadIDPath, fsMetaJSONFile)
fsMeta, err := readFSMetadata(fs.storage, minioMetaMultipartBucket, fsMetaPath)
if err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, fsMetaPath)
}
fsMeta.AddObjectPart(partID, partSuffix, newMD5Hex, size)
partPath := path.Join(bucket, object, uploadID, partSuffix)
partPath := pathJoin(bucket, object, uploadID, partSuffix)
// Lock the part so that another part upload with same part-number gets blocked
// while the part is getting appended in the background.
partLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, partPath)
partLock.Lock()
err = fs.storage.RenameFile(minioMetaTmpBucket, tmpPartPath, minioMetaMultipartBucket, partPath)
if err != nil {
fsNSPartPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, partPath)
if err = fsRenameFile(fsPartPath, fsNSPartPath); err != nil {
partLock.Unlock()
return "", toObjectErr(traceError(err), minioMetaMultipartBucket, partPath)
return "", toObjectErr(err, minioMetaMultipartBucket, partPath)
}
uploadIDPath = path.Join(bucket, object, uploadID)
if err = writeFSMetadata(fs.storage, minioMetaMultipartBucket, path.Join(uploadIDPath, fsMetaJSONFile), fsMeta); err != nil {
// Save the object part info in `fs.json`.
fsMeta.AddObjectPart(partID, partSuffix, newMD5Hex, size)
if _, err = fsMeta.WriteTo(rwlk); err != nil {
partLock.Unlock()
return "", toObjectErr(err, minioMetaMultipartBucket, uploadIDPath)
}
// Append the part in background.
errCh := fs.bgAppend.append(fs.storage, bucket, object, uploadID, fsMeta)
errCh := fs.append(bucket, object, uploadID, fsMeta)
go func() {
// Also receive the error so that the appendParts go-routine does not block on send.
// But the error received is ignored as fs.PutObjectPart() would have already
// returned success to the client.
// Also receive the error so that the appendParts go-routine
// does not block on send. But the error received is ignored
// as fs.PutObjectPart() would have already returned success
// to the client.
<-errCh
partLock.Unlock()
}()
@@ -381,36 +486,51 @@ func (fs fsObjects) PutObjectPart(bucket, object, uploadID string, partID int, s
func (fs fsObjects) listObjectParts(bucket, object, uploadID string, partNumberMarker, maxParts int) (ListPartsInfo, error) {
result := ListPartsInfo{}
fsMetaPath := path.Join(bucket, object, uploadID, fsMetaJSONFile)
fsMeta, err := readFSMetadata(fs.storage, minioMetaMultipartBucket, fsMetaPath)
uploadIDPath := pathJoin(bucket, object, uploadID)
fsMetaPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, fsMetaJSONFile)
metaFile, err := fs.rwPool.Open(fsMetaPath)
if err != nil {
if err == errFileNotFound || err == errFileAccessDenied {
// On windows oddly this is returned.
return ListPartsInfo{}, traceError(InvalidUploadID{UploadID: uploadID})
}
return ListPartsInfo{}, toObjectErr(traceError(err), bucket, object)
}
defer fs.rwPool.Close(fsMetaPath)
fsMeta := fsMetaV1{}
_, err = fsMeta.ReadFrom((io.NewSectionReader(metaFile, 0, metaFile.Size())))
if err != nil {
return ListPartsInfo{}, toObjectErr(err, minioMetaBucket, fsMetaPath)
}
// Only parts with higher part numbers will be listed.
partIdx := fsMeta.ObjectPartIndex(partNumberMarker)
parts := fsMeta.Parts
if partIdx != -1 {
parts = fsMeta.Parts[partIdx+1:]
}
count := maxParts
for _, part := range parts {
var fi FileInfo
partNamePath := path.Join(bucket, object, uploadID, part.Name)
fi, err = fs.storage.StatFile(minioMetaMultipartBucket, partNamePath)
var fi os.FileInfo
partNamePath := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, part.Name)
fi, err = fsStatFile(partNamePath)
if err != nil {
return ListPartsInfo{}, toObjectErr(traceError(err), minioMetaMultipartBucket, partNamePath)
}
result.Parts = append(result.Parts, partInfo{
PartNumber: part.Number,
ETag: part.ETag,
LastModified: fi.ModTime,
Size: fi.Size,
LastModified: fi.ModTime(),
Size: fi.Size(),
})
count--
if count == 0 {
break
}
}
// If listed entries are more than maxParts, we set IsTruncated as true.
if len(parts) > len(result.Parts) {
result.IsTruncated = true
@@ -423,6 +543,8 @@ func (fs fsObjects) listObjectParts(bucket, object, uploadID string, partNumberM
result.Object = object
result.UploadID = uploadID
result.MaxParts = maxParts
// Success.
return result, nil
}
@@ -438,29 +560,23 @@ func (fs fsObjects) ListObjectParts(bucket, object, uploadID string, partNumberM
return ListPartsInfo{}, err
}
// Hold lock so that there is no competing
// abort-multipart-upload or complete-multipart-upload.
uploadIDLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, object, uploadID))
uploadIDLock.Lock()
defer uploadIDLock.Unlock()
if !fs.isUploadIDExists(bucket, object, uploadID) {
return ListPartsInfo{}, traceError(InvalidUploadID{UploadID: uploadID})
if _, err := fs.statBucketDir(bucket); err != nil {
return ListPartsInfo{}, toObjectErr(err, bucket)
}
return fs.listObjectParts(bucket, object, uploadID, partNumberMarker, maxParts)
}
func (fs fsObjects) totalObjectSize(fsMeta fsMetaV1, parts []completePart) (int64, error) {
objSize := int64(0)
for _, part := range parts {
partIdx := fsMeta.ObjectPartIndex(part.PartNumber)
if partIdx == -1 {
return 0, InvalidPart{}
}
objSize += fsMeta.Parts[partIdx].Size
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, pathJoin(bucket, object))
objectMPartPathLock.RLock()
defer objectMPartPathLock.RUnlock()
listPartsInfo, err := fs.listObjectParts(bucket, object, uploadID, partNumberMarker, maxParts)
if err != nil {
return ListPartsInfo{}, toObjectErr(err, bucket, object)
}
return objSize, nil
// Success.
return listPartsInfo, nil
}
// CompleteMultipartUpload - completes an ongoing multipart
@@ -474,18 +590,8 @@ func (fs fsObjects) CompleteMultipartUpload(bucket string, object string, upload
return "", err
}
uploadIDPath := path.Join(bucket, object, uploadID)
// Hold lock so that
// 1) no one aborts this multipart upload
// 2) no one does a parallel complete-multipart-upload on this
// multipart upload
uploadIDLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, uploadIDPath)
uploadIDLock.Lock()
defer uploadIDLock.Unlock()
if !fs.isUploadIDExists(bucket, object, uploadID) {
return "", traceError(InvalidUploadID{UploadID: uploadID})
if _, err := fs.statBucketDir(bucket); err != nil {
return "", toObjectErr(err, bucket)
}
// Calculate s3 compatible md5sum for complete multipart.
@@ -494,22 +600,65 @@ func (fs fsObjects) CompleteMultipartUpload(bucket string, object string, upload
return "", err
}
// Read saved fs metadata for ongoing multipart.
fsMetaPath := pathJoin(uploadIDPath, fsMetaJSONFile)
fsMeta, err := readFSMetadata(fs.storage, minioMetaMultipartBucket, fsMetaPath)
uploadIDPath := pathJoin(bucket, object, uploadID)
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket, pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
fsMetaPathMultipart := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, fsMetaJSONFile)
rlk, err := fs.rwPool.Open(fsMetaPathMultipart)
if err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, fsMetaPath)
if err == errFileNotFound || err == errFileAccessDenied {
return "", traceError(InvalidUploadID{UploadID: uploadID})
}
return "", toObjectErr(traceError(err), bucket, object)
}
// Disallow any parallel abort or complete multipart operations.
rwlk, err := fs.rwPool.Write(pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object, uploadsJSONFile))
if err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
if err == errFileNotFound || err == errFileAccessDenied {
return "", traceError(InvalidUploadID{UploadID: uploadID})
}
return "", toObjectErr(traceError(err), bucket, object)
}
defer rwlk.Close()
fsMeta := fsMetaV1{}
// Read saved fs metadata for ongoing multipart.
_, err = fsMeta.ReadFrom(io.NewSectionReader(rlk, 0, rlk.Size()))
if err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(err, minioMetaMultipartBucket, fsMetaPathMultipart)
}
// Wait for any competing PutObject() operation on bucket/object, since same namespace
// would be acquired for `fs.json`.
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
metaFile, err := fs.rwPool.Create(fsMetaPath)
if err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(traceError(err), bucket, object)
}
defer metaFile.Close()
// This lock is held during rename of the appended tmp file to the actual
// location so that any competing GetObject/PutObject/DeleteObject do not race.
appendFallback := true // In case background-append did not append the required parts.
if isPartsSame(fsMeta.Parts, parts) {
err = fs.bgAppend.complete(fs.storage, bucket, object, uploadID, fsMeta)
err = fs.complete(bucket, object, uploadID, fsMeta)
if err == nil {
appendFallback = false
if err = fs.storage.RenameFile(minioMetaTmpBucket, uploadID, bucket, object); err != nil {
return "", toObjectErr(traceError(err), minioMetaTmpBucket, uploadID)
fsTmpObjPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, uploadID)
fsNSObjPath := pathJoin(fs.fsPath, bucket, object)
if err = fsRenameFile(fsTmpObjPath, fsNSObjPath); err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(err, minioMetaTmpBucket, uploadID)
}
}
}
@@ -518,77 +667,79 @@ func (fs fsObjects) CompleteMultipartUpload(bucket string, object string, upload
// background append could not do append all the required parts, hence we do it here.
tempObj := uploadID + "-" + "part.1"
fsTmpObjPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, tempObj)
// Delete the temporary object in the case of a
// failure. If PutObject succeeds, then there would be
// nothing to delete.
defer fsRemoveFile(fsTmpObjPath)
// Allocate staging buffer.
var buf = make([]byte, readSizeV1)
var objSize int64
objSize, err = fs.totalObjectSize(fsMeta, parts)
if err != nil {
return "", traceError(err)
}
if objSize > 0 {
// Prepare file to avoid disk fragmentation
err = fs.storage.PrepareFile(minioMetaTmpBucket, tempObj, objSize)
if err != nil {
return "", traceError(err)
}
}
// Loop through all parts, validate them and then commit to disk.
// Validate all parts and then commit to disk.
for i, part := range parts {
partIdx := fsMeta.ObjectPartIndex(part.PartNumber)
if partIdx == -1 {
fs.rwPool.Close(fsMetaPathMultipart)
return "", traceError(InvalidPart{})
}
if fsMeta.Parts[partIdx].ETag != part.ETag {
fs.rwPool.Close(fsMetaPathMultipart)
return "", traceError(BadDigest{})
}
// All parts except the last part has to be atleast 5MB.
if (i < len(parts)-1) && !isMinAllowedPartSize(fsMeta.Parts[partIdx].Size) {
fs.rwPool.Close(fsMetaPathMultipart)
return "", traceError(PartTooSmall{
PartNumber: part.PartNumber,
PartSize: fsMeta.Parts[partIdx].Size,
PartETag: part.ETag,
})
}
// Construct part suffix.
partSuffix := fmt.Sprintf("object%d", part.PartNumber)
multipartPartFile := path.Join(bucket, object, uploadID, partSuffix)
multipartPartFile := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, partSuffix)
var reader io.ReadCloser
offset := int64(0)
totalLeft := fsMeta.Parts[partIdx].Size
for totalLeft > 0 {
curLeft := int64(readSizeV1)
if totalLeft < readSizeV1 {
curLeft = totalLeft
reader, _, err = fsOpenFile(multipartPartFile, offset)
if err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
if err == errFileNotFound {
return "", traceError(InvalidPart{})
}
var n int64
n, err = fs.storage.ReadFile(minioMetaMultipartBucket, multipartPartFile, offset, buf[:curLeft])
if n > 0 {
if err = fs.storage.AppendFile(minioMetaTmpBucket, tempObj, buf[:n]); err != nil {
return "", toObjectErr(traceError(err), minioMetaTmpBucket, tempObj)
}
}
if err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
break
}
if err == errFileNotFound {
return "", traceError(InvalidPart{})
}
return "", toObjectErr(traceError(err), minioMetaMultipartBucket, multipartPartFile)
}
offset += n
totalLeft -= n
return "", toObjectErr(traceError(err), minioMetaMultipartBucket, partSuffix)
}
// No need to hold a lock, this is a unique file and will be only written
// to one one process per uploadID per minio process.
var wfile *os.File
wfile, err = os.OpenFile(preparePath(fsTmpObjPath), os.O_CREATE|os.O_APPEND|os.O_WRONLY, 0666)
if err != nil {
reader.Close()
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(traceError(err), bucket, object)
}
_, err = io.CopyBuffer(wfile, reader, buf)
if err != nil {
wfile.Close()
reader.Close()
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(traceError(err), bucket, object)
}
wfile.Close()
reader.Close()
}
// Rename the file back to original location, if not delete the temporary object.
err = fs.storage.RenameFile(minioMetaTmpBucket, tempObj, bucket, object)
if err != nil {
if dErr := fs.storage.DeleteFile(minioMetaTmpBucket, tempObj); dErr != nil {
return "", toObjectErr(traceError(dErr), minioMetaTmpBucket, tempObj)
}
return "", toObjectErr(traceError(err), bucket, object)
fsNSObjPath := pathJoin(fs.fsPath, bucket, object)
if err = fsRenameFile(fsTmpObjPath, fsNSObjPath); err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(err, minioMetaTmpBucket, uploadID)
}
}
@@ -601,54 +752,33 @@ func (fs fsObjects) CompleteMultipartUpload(bucket string, object string, upload
}
fsMeta.Meta["md5Sum"] = s3MD5
fsMetaPath = path.Join(bucketMetaPrefix, bucket, object, fsMetaJSONFile)
// Write the metadata to a temp file and rename it to the actual location.
if err = writeFSMetadata(fs.storage, minioMetaBucket, fsMetaPath, fsMeta); err != nil {
// Write all the set metadata.
if _, err = fsMeta.WriteTo(metaFile); err != nil {
fs.rwPool.Close(fsMetaPathMultipart)
return "", toObjectErr(err, bucket, object)
}
// Close lock held on bucket/object/uploadid/fs.json,
// this needs to be done for windows so that we can happily
// delete the bucket/object/uploadid
fs.rwPool.Close(fsMetaPathMultipart)
// Cleanup all the parts if everything else has been safely committed.
if err = cleanupUploadedParts(bucket, object, uploadID, fs.storage); err != nil {
return "", toObjectErr(err, bucket, object)
multipartObjectDir := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object)
multipartUploadIDDir := pathJoin(multipartObjectDir, uploadID)
if err = fsRemoveUploadIDPath(multipartObjectDir, multipartUploadIDDir); err != nil {
return "", toObjectErr(traceError(err), bucket, object)
}
// Hold the lock so that two parallel
// complete-multipart-uploads do not leave a stale
// uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
// remove entry from uploads.json
if err := fs.removeUploadID(bucket, object, uploadID); err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, path.Join(bucket, object))
// Remove entry from `uploads.json`.
if err = fs.removeUploadID(bucket, object, uploadID, rwlk); err != nil {
return "", toObjectErr(err, minioMetaMultipartBucket, pathutil.Join(bucket, object))
}
// Return md5sum.
return s3MD5, nil
}
// abortMultipartUpload - wrapper for purging an ongoing multipart
// transaction, deletes uploadID entry from `uploads.json` and purges
// the directory at '.minio.sys/multipart/bucket/object/uploadID' holding
// all the upload parts.
func (fs fsObjects) abortMultipartUpload(bucket, object, uploadID string) error {
// Signal appendParts routine to stop waiting for new parts to arrive.
fs.bgAppend.abort(uploadID)
// Cleanup all uploaded parts.
if err := cleanupUploadedParts(bucket, object, uploadID, fs.storage); err != nil {
return err
}
// remove entry from uploads.json with quorum
if err := fs.removeUploadID(bucket, object, uploadID); err != nil {
return toObjectErr(err, bucket, object)
}
// success
return nil
}
// AbortMultipartUpload - aborts an ongoing multipart operation
// signified by the input uploadID. This is an atomic operation
// doesn't require clients to initiate multiple such requests.
@@ -666,17 +796,57 @@ func (fs fsObjects) AbortMultipartUpload(bucket, object, uploadID string) error
return err
}
// Hold lock so that there is no competing
// complete-multipart-upload or put-object-part.
uploadIDLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, object, uploadID))
uploadIDLock.Lock()
defer uploadIDLock.Unlock()
if !fs.isUploadIDExists(bucket, object, uploadID) {
return traceError(InvalidUploadID{UploadID: uploadID})
if _, err := fs.statBucketDir(bucket); err != nil {
return toObjectErr(err, bucket)
}
err := fs.abortMultipartUpload(bucket, object, uploadID)
return err
uploadIDPath := pathJoin(bucket, object, uploadID)
// Hold the lock so that two parallel complete-multipart-uploads
// do not leave a stale uploads.json behind.
objectMPartPathLock := globalNSMutex.NewNSLock(minioMetaMultipartBucket,
pathJoin(bucket, object))
objectMPartPathLock.Lock()
defer objectMPartPathLock.Unlock()
fsMetaPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadIDPath, fsMetaJSONFile)
if _, err := fs.rwPool.Open(fsMetaPath); err != nil {
if err == errFileNotFound || err == errFileAccessDenied {
return traceError(InvalidUploadID{UploadID: uploadID})
}
return toObjectErr(traceError(err), bucket, object)
}
uploadsPath := pathJoin(bucket, object, uploadsJSONFile)
rwlk, err := fs.rwPool.Write(pathJoin(fs.fsPath, minioMetaMultipartBucket, uploadsPath))
if err != nil {
fs.rwPool.Close(fsMetaPath)
if err == errFileNotFound || err == errFileAccessDenied {
return traceError(InvalidUploadID{UploadID: uploadID})
}
return toObjectErr(traceError(err), bucket, object)
}
defer rwlk.Close()
// Signal appendParts routine to stop waiting for new parts to arrive.
fs.abort(uploadID)
// Close lock held on bucket/object/uploadid/fs.json,
// this needs to be done for windows so that we can happily
// delete the bucket/object/uploadid
fs.rwPool.Close(fsMetaPath)
// Cleanup all uploaded parts and abort the upload.
multipartObjectDir := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object)
multipartUploadIDDir := pathJoin(multipartObjectDir, uploadID)
if err = fsRemoveUploadIDPath(multipartObjectDir, multipartUploadIDDir); err != nil {
return toObjectErr(traceError(err), bucket, object)
}
// Remove entry from `uploads.json`.
if err = fs.removeUploadID(bucket, object, uploadID, rwlk); err != nil {
return toObjectErr(err, bucket, object)
}
return nil
}

View File

@@ -19,7 +19,6 @@ package cmd
import (
"bytes"
"path/filepath"
"reflect"
"testing"
)
@@ -30,7 +29,7 @@ func TestNewMultipartUploadFaultyDisk(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
@@ -38,20 +37,11 @@ func TestNewMultipartUploadFaultyDisk(t *testing.T) {
t.Fatal("Cannot create bucket, err: ", err)
}
// Test with faulty disk
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 5; i++ {
// Faulty disk generates errFaultyDisk at 'i' storage api call number
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
if _, err := fs.NewMultipartUpload(bucketName, objectName, map[string]string{"X-Amz-Meta-xid": "3f"}); errorCause(err) != errFaultyDisk {
switch i {
case 1:
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
default:
t.Fatal("Unexpected error ", err)
}
// Test with disk removed.
removeAll(disk) // remove disk.
if _, err := fs.NewMultipartUpload(bucketName, objectName, map[string]string{"X-Amz-Meta-xid": "3f"}); err != nil {
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
}
}
@@ -68,7 +58,7 @@ func TestPutObjectPartFaultyDisk(t *testing.T) {
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
data := []byte("12345")
@@ -86,30 +76,10 @@ func TestPutObjectPartFaultyDisk(t *testing.T) {
md5Hex := getMD5Hash(data)
sha256sum := ""
// Test with faulty disk
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 7; i++ {
// Faulty disk generates errFaultyDisk at 'i' storage api call number
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
md5sum, err := fs.PutObjectPart(bucketName, objectName, uploadID, 1, dataLen, bytes.NewReader(data), md5Hex, sha256sum)
if errorCause(err) != errFaultyDisk {
if errorCause(err) == nil {
t.Fatalf("Test %d shouldn't succeed, md5sum = %s\n", i, md5sum)
}
switch i {
case 1:
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
case 3:
case 2, 4, 5, 6:
if !isSameType(errorCause(err), InvalidUploadID{}) {
t.Fatal("Unexpected error ", err)
}
default:
t.Fatal("Unexpected error ", i, err, reflect.TypeOf(errorCause(err)), reflect.TypeOf(errFaultyDisk))
}
}
removeAll(disk) // Disk not found.
_, err = fs.PutObjectPart(bucketName, objectName, uploadID, 1, dataLen, bytes.NewReader(data), md5Hex, sha256sum)
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
}
@@ -120,7 +90,7 @@ func TestCompleteMultipartUploadFaultyDisk(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
data := []byte("12345")
@@ -143,23 +113,10 @@ func TestCompleteMultipartUploadFaultyDisk(t *testing.T) {
parts := []completePart{{PartNumber: 1, ETag: md5Hex}}
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 3; i++ {
// Faulty disk generates errFaultyDisk at 'i' storage api call number
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
if _, err := fs.CompleteMultipartUpload(bucketName, objectName, uploadID, parts); errorCause(err) != errFaultyDisk {
switch i {
case 1:
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
case 2:
if !isSameType(errorCause(err), InvalidUploadID{}) {
t.Fatal("Unexpected error ", err)
}
default:
t.Fatal("Unexpected error ", i, err, reflect.TypeOf(errorCause(err)), reflect.TypeOf(errFaultyDisk))
}
removeAll(disk) // Disk not found.
if _, err := fs.CompleteMultipartUpload(bucketName, objectName, uploadID, parts); err != nil {
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
}
}
@@ -169,8 +126,10 @@ func TestListMultipartUploadsFaultyDisk(t *testing.T) {
// Prepare for tests
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
data := []byte("12345")
@@ -191,27 +150,10 @@ func TestListMultipartUploadsFaultyDisk(t *testing.T) {
t.Fatal("Unexpected error ", err)
}
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 4; i++ {
// Faulty disk generates errFaultyDisk at 'i' storage api call number
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
if _, err := fs.ListMultipartUploads(bucketName, objectName, "", "", "", 1000); errorCause(err) != errFaultyDisk {
switch i {
case 1:
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
case 2:
if !isSameType(errorCause(err), InvalidUploadID{}) {
t.Fatal("Unexpected error ", err)
}
case 3:
if errorCause(err) != errFileNotFound {
t.Fatal("Unexpected error ", err)
}
default:
t.Fatal("Unexpected error ", i, err, reflect.TypeOf(errorCause(err)), reflect.TypeOf(errFaultyDisk))
}
removeAll(disk) // Disk not found.
if _, err := fs.ListMultipartUploads(bucketName, objectName, "", "", "", 1000); err != nil {
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error ", err)
}
}
}

193
cmd/fs-v1-rwpool.go Normal file
View File

@@ -0,0 +1,193 @@
/*
* 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"
pathutil "path"
"sync"
"github.com/minio/minio/pkg/lock"
)
// fsIOPool represents a protected list to keep track of all
// the concurrent readers at a given path.
type fsIOPool struct {
sync.Mutex
readersMap map[string]*lock.RLockedFile
}
// Open is a wrapper call to read locked file which
// returns a ReadAtCloser.
//
// ReaderAt is provided so that the fd is non seekable, since
// we are sharing fd's with concurrent threads, we don't want
// all readers to change offsets on each other during such
// concurrent operations. Using ReadAt allows us to read from
// any offsets.
//
// Closer is implemented to track total readers and to close
// only when there no more readers, the fd is purged if the lock
// count has reached zero.
func (fsi *fsIOPool) Open(path string) (*lock.RLockedFile, error) {
if err := checkPathLength(path); err != nil {
return nil, err
}
fsi.Lock()
rlkFile, ok := fsi.readersMap[path]
// File reference exists on map, validate if its
// really closed and we are safe to purge it.
if ok && rlkFile != nil {
// If the file is closed and not removed from map is a bug.
if rlkFile.IsClosed() {
// Log this as an error.
errorIf(errUnexpected, "Unexpected entry found on the map %s", path)
// Purge the cached lock path from map.
delete(fsi.readersMap, path)
// Indicate that we can populate the new fd.
ok = false
} else {
// Increment the lock ref, since the file is not closed yet
// and caller requested to read the file again.
rlkFile.IncLockRef()
}
}
fsi.Unlock()
// Locked path reference doesn't exist, freshly open the file in
// read lock mode.
if !ok {
var err error
// Open file for reading.
rlkFile, err = lock.RLockedOpenFile(preparePath(path))
if err != nil {
if os.IsNotExist(err) {
return nil, errFileNotFound
} else if os.IsPermission(err) {
return nil, errFileAccessDenied
} else if isSysErrIsDir(err) {
return nil, errIsNotRegular
} else if isSysErrNotDir(err) {
return nil, errFileAccessDenied
} else if isSysErrPathNotFound(err) {
return nil, errFileNotFound
}
return nil, err
}
// Save new reader on the map.
fsi.Lock()
fsi.readersMap[path] = rlkFile
fsi.Unlock()
}
// Success.
return rlkFile, nil
}
// Write - Attempt to lock the file if it exists,
// - if the file exists. Then we try to get a write lock this
// will block if we can't get a lock perhaps another write
// or read is in progress. Concurrent calls are protected
// by the global namspace lock within the same process.
func (fsi *fsIOPool) Write(path string) (wlk *lock.LockedFile, err error) {
if err = checkPathLength(path); err != nil {
return nil, err
}
wlk, err = lock.LockedOpenFile(preparePath(path), os.O_RDWR, 0666)
if err != nil {
if os.IsNotExist(err) {
return nil, errFileNotFound
} else if os.IsPermission(err) {
return nil, errFileAccessDenied
} else if isSysErrIsDir(err) {
return nil, errIsNotRegular
}
return nil, err
}
return wlk, nil
}
// Create - creates a new write locked file instance.
// - if the file doesn't exist. We create the file and hold lock.
func (fsi *fsIOPool) Create(path string) (wlk *lock.LockedFile, err error) {
if err = checkPathLength(path); err != nil {
return nil, err
}
// Creates parent if missing.
if err = mkdirAll(pathutil.Dir(path), 0777); err != nil {
if os.IsPermission(err) {
return nil, errFileAccessDenied
} else if isSysErrNotDir(err) {
return nil, errFileAccessDenied
}
return nil, err
}
// Attempt to create the file.
wlk, err = lock.LockedOpenFile(preparePath(path), os.O_RDWR|os.O_CREATE, 0666)
if err != nil {
if os.IsPermission(err) {
return nil, errFileAccessDenied
} else if isSysErrIsDir(err) {
return nil, errIsNotRegular
} else if isSysErrPathNotFound(err) {
return nil, errFileAccessDenied
}
return nil, err
}
// Success.
return wlk, err
}
// Close implements closing the path referenced by the reader in such
// a way that it makes sure to remove entry from the map immediately
// if no active readers are present.
func (fsi *fsIOPool) Close(path string) error {
fsi.Lock()
defer fsi.Unlock()
if err := checkPathLength(path); err != nil {
return err
}
// Pop readers from path.
rlkFile, ok := fsi.readersMap[path]
if !ok {
return nil
}
// Close the reader.
rlkFile.Close()
// If the file is closed, remove it from the reader pool map.
if rlkFile.IsClosed() {
// Purge the cached lock path from map.
delete(fsi.readersMap, path)
}
// Success.
return nil
}

112
cmd/fs-v1-rwpool_test.go Normal file
View File

@@ -0,0 +1,112 @@
/*
* 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 (
"runtime"
"testing"
"github.com/minio/minio/pkg/lock"
)
// Tests long path calls.
func TestRWPoolLongPath(t *testing.T) {
rwPool := &fsIOPool{
readersMap: make(map[string]*lock.RLockedFile),
}
longPath := "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001"
if _, err := rwPool.Create(longPath); err != errFileNameTooLong {
t.Fatal(err)
}
if _, err := rwPool.Write(longPath); err != errFileNameTooLong {
t.Fatal(err)
}
if _, err := rwPool.Open(longPath); err != errFileNameTooLong {
t.Fatal(err)
}
}
// Tests all RWPool methods.
func TestRWPool(t *testing.T) {
// create posix test setup
_, path, err := newPosixTestSetup()
if err != nil {
t.Fatalf("Unable to create posix test setup, %s", err)
}
defer removeAll(path)
rwPool := &fsIOPool{
readersMap: make(map[string]*lock.RLockedFile),
}
wlk, err := rwPool.Create(pathJoin(path, "success-vol", "file/path/1.txt"))
if err != nil {
t.Fatal(err)
}
wlk.Close()
// Fails to create a parent directory if there is a file.
_, err = rwPool.Create(pathJoin(path, "success-vol", "file/path/1.txt/test"))
if err != errFileAccessDenied {
t.Fatal("Unexpected error", err)
}
// Fails to create a file if there is a directory.
_, err = rwPool.Create(pathJoin(path, "success-vol", "file"))
if runtime.GOOS == "windows" {
if err != errFileAccessDenied {
t.Fatal("Unexpected error", err)
}
} else {
if err != errIsNotRegular {
t.Fatal("Unexpected error", err)
}
}
rlk, err := rwPool.Open(pathJoin(path, "success-vol", "file/path/1.txt"))
if err != nil {
t.Fatal("Unexpected error", err)
}
rlk.Close()
// Fails to read a directory.
_, err = rwPool.Open(pathJoin(path, "success-vol", "file"))
if runtime.GOOS == "windows" {
if err != errFileAccessDenied {
t.Fatal("Unexpected error", err)
}
} else {
if err != errIsNotRegular {
t.Fatal("Unexpected error", err)
}
}
// Fails to open a file which has a parent as file.
_, err = rwPool.Open(pathJoin(path, "success-vol", "file/path/1.txt/test"))
if runtime.GOOS != "windows" {
if err != errFileAccessDenied {
t.Fatal("Unexpected error", err)
}
} else {
if err != errFileNotFound {
t.Fatal("Unexpected error", err)
}
}
}

View File

@@ -23,110 +23,203 @@ import (
"fmt"
"hash"
"io"
"path"
"os"
"path/filepath"
"runtime"
"sort"
"strings"
"syscall"
"github.com/minio/minio/pkg/mimedb"
"github.com/minio/minio/pkg/disk"
"github.com/minio/minio/pkg/lock"
"github.com/minio/sha256-simd"
)
// fsObjects - Implements fs object layer.
type fsObjects struct {
storage StorageAPI
// Path to be exported over S3 API.
fsPath string
// List pool management.
// Unique value to be used for all
// temporary transactions.
fsUUID string
minFreeSpace int64
minFreeInodes int64
// FS rw pool.
rwPool *fsIOPool
// ListObjects pool management.
listPool *treeWalkPool
// To manage the appendRoutine go0routines
bgAppend *backgroundAppend
}
// list of all errors that can be ignored in tree walk operation in FS
var fsTreeWalkIgnoredErrs = []error{
errFileNotFound,
errVolumeNotFound,
// Initializes meta volume on all the fs path.
func initMetaVolumeFS(fsPath, fsUUID string) error {
// This happens for the first time, but keep this here since this
// is the only place where it can be made less expensive
// optimizing all other calls. Create minio meta volume,
// if it doesn't exist yet.
metaBucketPath := pathJoin(fsPath, minioMetaBucket)
if err := mkdirAll(metaBucketPath, 0777); err != nil {
return err
}
metaTmpPath := pathJoin(fsPath, minioMetaTmpBucket, fsUUID)
if err := mkdirAll(metaTmpPath, 0777); err != nil {
return err
}
metaMultipartPath := pathJoin(fsPath, minioMetaMultipartBucket)
if err := mkdirAll(metaMultipartPath, 0777); err != nil {
return err
}
// Return success here.
return nil
}
// newFSObjects - initialize new fs object layer.
func newFSObjects(storage StorageAPI) (ObjectLayer, error) {
if storage == nil {
// newFSObjectLayer - initialize new fs object layer.
func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
if fsPath == "" {
return nil, errInvalidArgument
}
// Load format and validate.
_, err := loadFormatFS(storage)
var err error
// Disallow relative paths, figure out absolute paths.
fsPath, err = filepath.Abs(fsPath)
if err != nil {
return nil, fmt.Errorf("Unable to recognize backend format, %s", err)
return nil, err
}
fi, err := os.Stat(preparePath(fsPath))
if err == nil {
if !fi.IsDir() {
return nil, syscall.ENOTDIR
}
}
if os.IsNotExist(err) {
// Disk not found create it.
err = mkdirAll(fsPath, 0777)
if err != nil {
return nil, err
}
}
// Assign a new UUID for FS minio mode. Each server instance
// gets its own UUID for temporary file transaction.
fsUUID := mustGetUUID()
// Initialize meta volume, if volume already exists ignores it.
if err = initMetaVolume([]StorageAPI{storage}); err != nil {
if err = initMetaVolumeFS(fsPath, fsUUID); err != nil {
return nil, fmt.Errorf("Unable to initialize '.minio.sys' meta volume, %s", err)
}
// Load `format.json`.
format, err := loadFormatFS(fsPath)
if err != nil && err != errUnformattedDisk {
return nil, fmt.Errorf("Unable to load 'format.json', %s", err)
}
// If the `format.json` doesn't exist create one.
if err == errUnformattedDisk {
fsFormatPath := pathJoin(fsPath, minioMetaBucket, fsFormatJSONFile)
// Initialize format.json, if already exists overwrite it.
if serr := saveFormatFS(fsFormatPath, newFSFormatV1()); serr != nil {
return nil, fmt.Errorf("Unable to initialize 'format.json', %s", serr)
}
}
// Validate if we have the same format.
if err == nil && format.Format != "fs" {
return nil, fmt.Errorf("Unable to recognize backend format, Disk is not in FS format. %s", format.Format)
}
// Initialize fs objects.
fs := fsObjects{
storage: storage,
fs := &fsObjects{
fsPath: fsPath,
fsUUID: fsUUID,
minFreeSpace: fsMinFreeSpace,
minFreeInodes: fsMinFreeInodes,
rwPool: &fsIOPool{
readersMap: make(map[string]*lock.RLockedFile),
},
listPool: newTreeWalkPool(globalLookupTimeout),
bgAppend: &backgroundAppend{
infoMap: make(map[string]bgAppendPartsInfo),
},
}
// Validate if disk has enough free space to use.
if err = fs.checkDiskFree(); err != nil {
return nil, err
}
// Initialize and load bucket policies.
err = initBucketPolicies(fs)
if err != nil {
return nil, fmt.Errorf("Unable to load all bucket policies. %s", err)
}
// Initialize a new event notifier.
err = initEventNotifier(fs)
if err != nil {
return nil, fmt.Errorf("Unable to initialize event notification. %s", err)
}
// Return successfully initialized object layer.
return fs, nil
}
// Should be called when process shuts down.
func (fs fsObjects) Shutdown() error {
// List if there are any multipart entries.
prefix := ""
entries, err := fs.storage.ListDir(minioMetaMultipartBucket, prefix)
if err != nil {
// A non nil err means that an unexpected error occurred
return toObjectErr(traceError(err))
}
if len(entries) > 0 {
// Should not remove .minio.sys if there are any multipart
// uploads were found.
// checkDiskFree verifies if disk path has sufficient minimum free disk space and files.
func (fs fsObjects) checkDiskFree() (err error) {
// We don't validate disk space or inode utilization on windows.
// Each windows calls to 'GetVolumeInformationW' takes around 3-5seconds.
if runtime.GOOS == "windows" {
return nil
}
if err = fs.storage.DeleteVol(minioMetaMultipartBucket); err != nil {
return toObjectErr(traceError(err))
}
// List if there are any bucket configuration entries.
_, err = fs.storage.ListDir(minioMetaBucket, bucketConfigPrefix)
if err != errFileNotFound {
// A nil err means that bucket config directory is not empty hence do not remove '.minio.sys' volume.
// A non nil err means that an unexpected error occurred
return toObjectErr(traceError(err))
}
// Cleanup and delete tmp bucket.
if err = cleanupDir(fs.storage, minioMetaTmpBucket, prefix); err != nil {
var di disk.Info
di, err = getDiskInfo(preparePath(fs.fsPath))
if err != nil {
return err
}
if err = fs.storage.DeleteVol(minioMetaTmpBucket); err != nil {
return toObjectErr(traceError(err))
// Remove 5% from free space for cumulative disk space used for journalling, inodes etc.
availableDiskSpace := float64(di.Free) * 0.95
if int64(availableDiskSpace) <= fs.minFreeSpace {
return errDiskFull
}
// Remove format.json and delete .minio.sys bucket
if err = fs.storage.DeleteFile(minioMetaBucket, fsFormatJSONFile); err != nil {
return toObjectErr(traceError(err))
}
if err = fs.storage.DeleteVol(minioMetaBucket); err != nil {
if err != errVolumeNotEmpty {
return toObjectErr(traceError(err))
// Some filesystems do not implement a way to provide total inodes available, instead inodes
// are allocated based on available disk space. For example CephFS, StoreNext CVFS, AzureFile driver.
// Allow for the available disk to be separately validate and we will validate inodes only if
// total inodes are provided by the underlying filesystem.
if di.Files != 0 {
availableFiles := int64(di.Ffree)
if availableFiles <= fs.minFreeInodes {
return errDiskFull
}
}
// Successful.
// Success.
return nil
}
// Should be called when process shuts down.
func (fs fsObjects) Shutdown() error {
// Cleanup and delete tmp uuid.
return fsRemoveAll(pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID))
}
// StorageInfo - returns underlying storage statistics.
func (fs fsObjects) StorageInfo() StorageInfo {
info, err := fs.storage.DiskInfo()
errorIf(err, "Unable to get disk info %#v", fs.storage)
info, err := getDiskInfo(preparePath(fs.fsPath))
errorIf(err, "Unable to get disk info %#v", fs.fsPath)
storageInfo := StorageInfo{
Total: info.Total,
Free: info.Free,
@@ -137,81 +230,141 @@ func (fs fsObjects) StorageInfo() StorageInfo {
/// Bucket operations
// MakeBucket - make a bucket.
func (fs fsObjects) MakeBucket(bucket string) error {
// getBucketDir - will convert incoming bucket names to
// corresponding valid bucket names on the backend in a platform
// compatible way for all operating systems.
func (fs fsObjects) getBucketDir(bucket string) (string, error) {
// Verify if bucket is valid.
if !IsValidBucketName(bucket) {
return traceError(BucketNameInvalid{Bucket: bucket})
return "", traceError(BucketNameInvalid{Bucket: bucket})
}
if err := fs.storage.MakeVol(bucket); err != nil {
bucketDir := pathJoin(fs.fsPath, bucket)
return bucketDir, nil
}
func (fs fsObjects) statBucketDir(bucket string) (os.FileInfo, error) {
bucketDir, err := fs.getBucketDir(bucket)
if err != nil {
return nil, err
}
st, err := fsStatDir(bucketDir)
if err != nil {
return nil, traceError(err)
}
return st, nil
}
// MakeBucket - create a new bucket, returns if it
// already exists.
func (fs fsObjects) MakeBucket(bucket string) error {
bucketDir, err := fs.getBucketDir(bucket)
if err != nil {
return toObjectErr(err, bucket)
}
if err = fsMkdir(bucketDir); err != nil {
return toObjectErr(traceError(err), bucket)
}
return nil
}
// GetBucketInfo - get bucket info.
// GetBucketInfo - fetch bucket metadata info.
func (fs fsObjects) GetBucketInfo(bucket string) (BucketInfo, error) {
// Verify if bucket is valid.
if !IsValidBucketName(bucket) {
return BucketInfo{}, traceError(BucketNameInvalid{Bucket: bucket})
}
vi, err := fs.storage.StatVol(bucket)
st, err := fs.statBucketDir(bucket)
if err != nil {
return BucketInfo{}, toObjectErr(traceError(err), bucket)
return BucketInfo{}, toObjectErr(err, bucket)
}
// As os.Stat() doesn't carry other than ModTime(), use ModTime() as CreatedTime.
createdTime := st.ModTime()
return BucketInfo{
Name: bucket,
Created: vi.Created,
Created: createdTime,
}, nil
}
// ListBuckets - list buckets.
// ListBuckets - list all s3 compatible buckets (directories) at fsPath.
func (fs fsObjects) ListBuckets() ([]BucketInfo, error) {
var bucketInfos []BucketInfo
vols, err := fs.storage.ListVols()
if err != nil {
return nil, toObjectErr(traceError(err))
if err := checkPathLength(fs.fsPath); err != nil {
return nil, err
}
var bucketInfos []BucketInfo
entries, err := readDir(preparePath(fs.fsPath))
if err != nil {
return nil, toObjectErr(traceError(errDiskNotFound))
}
var invalidBucketNames []string
for _, vol := range vols {
// StorageAPI can send volume names which are incompatible
// with buckets, handle it and skip them.
if !IsValidBucketName(vol.Name) {
invalidBucketNames = append(invalidBucketNames, vol.Name)
for _, entry := range entries {
if entry == minioMetaBucket+"/" || !strings.HasSuffix(entry, slashSeparator) {
continue
}
// Ignore the volume special bucket.
if vol.Name == minioMetaBucket {
var fi os.FileInfo
fi, err = fsStatDir(pathJoin(fs.fsPath, entry))
if err != nil {
// If the directory does not exist, skip the entry.
if err == errVolumeNotFound {
continue
} else if err == errVolumeAccessDenied {
// Skip the entry if its a file.
continue
}
return nil, err
}
if !IsValidBucketName(fi.Name()) {
invalidBucketNames = append(invalidBucketNames, fi.Name())
continue
}
bucketInfos = append(bucketInfos, BucketInfo{
Name: vol.Name,
Created: vol.Created,
Name: fi.Name(),
// As os.Stat() doesn't carry other than ModTime(), use ModTime() as CreatedTime.
Created: fi.ModTime(),
})
}
// Print a user friendly message if we indeed skipped certain directories which are
// incompatible with S3's bucket name restrictions.
if len(invalidBucketNames) > 0 {
errorIf(errors.New("One or more invalid bucket names found"), "Skipping %s", invalidBucketNames)
}
// Sort bucket infos by bucket name.
sort.Sort(byBucketName(bucketInfos))
// Succes.
return bucketInfos, nil
}
// DeleteBucket - delete a bucket.
// DeleteBucket - delete a bucket and all the metadata associated
// with the bucket including pending multipart, object metadata.
func (fs fsObjects) DeleteBucket(bucket string) error {
// Verify if bucket is valid.
if !IsValidBucketName(bucket) {
return traceError(BucketNameInvalid{Bucket: bucket})
}
// Attempt to delete regular bucket.
if err := fs.storage.DeleteVol(bucket); err != nil {
return toObjectErr(traceError(err), bucket)
}
// Cleanup all the previously incomplete multiparts.
if err := cleanupDir(fs.storage, minioMetaMultipartBucket, bucket); err != nil && errorCause(err) != errVolumeNotFound {
bucketDir, err := fs.getBucketDir(bucket)
if err != nil {
return toObjectErr(err, bucket)
}
// Attempt to delete regular bucket.
if err = fsRemoveDir(bucketDir); err != nil {
return toObjectErr(err, bucket)
}
// Cleanup all the previously incomplete multiparts.
minioMetaMultipartBucketDir := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket)
if err = fsRemoveAll(minioMetaMultipartBucketDir); err != nil {
return toObjectErr(err, bucket)
}
// Cleanup all the bucket metadata.
minioMetadataBucketDir := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket)
if err = fsRemoveAll(minioMetadataBucketDir); err != nil {
return toObjectErr(err, bucket)
}
return nil
}
@@ -221,8 +374,12 @@ func (fs fsObjects) DeleteBucket(bucket string) error {
// if source object and destination object are same we only
// update metadata.
func (fs fsObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string, metadata map[string]string) (ObjectInfo, error) {
if _, err := fs.statBucketDir(srcBucket); err != nil {
return ObjectInfo{}, toObjectErr(err, srcBucket)
}
// Stat the file to get file size.
fi, err := fs.storage.StatFile(srcBucket, srcObject)
fi, err := fsStatFile(pathJoin(fs.fsPath, srcBucket, srcObject))
if err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), srcBucket, srcObject)
}
@@ -230,21 +387,28 @@ func (fs fsObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string
// Check if this request is only metadata update.
cpMetadataOnly := strings.EqualFold(pathJoin(srcBucket, srcObject), pathJoin(dstBucket, dstObject))
if cpMetadataOnly {
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, srcBucket, srcObject, fsMetaJSONFile)
var wlk *lock.LockedFile
wlk, err = fs.rwPool.Write(fsMetaPath)
if err != nil {
return ObjectInfo{}, toObjectErr(err, srcBucket, srcObject)
}
// This close will allow for locks to be synchronized on `fs.json`.
defer wlk.Close()
// Save objects' metadata in `fs.json`.
fsMeta := newFSMetaV1()
fsMeta.Meta = metadata
fsMetaPath := pathJoin(bucketMetaPrefix, dstBucket, dstObject, fsMetaJSONFile)
if err = writeFSMetadata(fs.storage, minioMetaBucket, fsMetaPath, fsMeta); err != nil {
return ObjectInfo{}, toObjectErr(err, dstBucket, dstObject)
if _, err = fsMeta.WriteTo(wlk); err != nil {
return ObjectInfo{}, toObjectErr(err, srcBucket, srcObject)
}
// Get object info.
return fs.getObjectInfo(dstBucket, dstObject)
// Return the new object info.
return fsMeta.ToObjectInfo(srcBucket, srcObject, fi), nil
}
// Length of the file to read.
length := fi.Size
length := fi.Size()
// Initialize pipe.
pipeReader, pipeWriter := io.Pipe()
@@ -280,88 +444,89 @@ func (fs fsObjects) GetObject(bucket, object string, offset int64, length int64,
if err = checkGetObjArgs(bucket, object); err != nil {
return err
}
if _, err = fs.statBucketDir(bucket); err != nil {
return toObjectErr(err, bucket)
}
// Offset cannot be negative.
if offset < 0 {
return toObjectErr(traceError(errUnexpected), bucket, object)
}
// Writer cannot be nil.
if writer == nil {
return toObjectErr(traceError(errUnexpected), bucket, object)
}
// Stat the file to get file size.
fi, err := fs.storage.StatFile(bucket, object)
if bucket != minioMetaBucket {
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
_, err = fs.rwPool.Open(fsMetaPath)
if err != nil && err != errFileNotFound {
return toObjectErr(traceError(err), bucket, object)
}
defer fs.rwPool.Close(fsMetaPath)
}
// Read the object, doesn't exist returns an s3 compatible error.
fsObjPath := pathJoin(fs.fsPath, bucket, object)
reader, size, err := fsOpenFile(fsObjPath, offset)
if err != nil {
return toObjectErr(traceError(err), bucket, object)
}
defer reader.Close()
// For negative length we read everything.
if length < 0 {
length = fi.Size - offset
}
// Reply back invalid range if the input offset and length fall out of range.
if offset > fi.Size || offset+length > fi.Size {
return traceError(InvalidRange{offset, length, fi.Size})
}
var totalLeft = length
bufSize := int64(readSizeV1)
if length > 0 && bufSize > length {
bufSize = length
}
// For negative length we read everything.
if length < 0 {
length = size - offset
}
// Reply back invalid range if the input offset and length fall out of range.
if offset > size || offset+length > size {
return traceError(InvalidRange{offset, length, size})
}
// Allocate a staging buffer.
buf := make([]byte, int(bufSize))
if err = fsReadFile(fs.storage, bucket, object, writer, totalLeft, offset, buf); err != nil {
// Returns any error.
return toObjectErr(err, bucket, object)
}
return nil
_, err = io.CopyBuffer(writer, io.LimitReader(reader, length), buf)
return toObjectErr(traceError(err), bucket, object)
}
// getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo.
func (fs fsObjects) getObjectInfo(bucket, object string) (ObjectInfo, error) {
fi, err := fs.storage.StatFile(bucket, object)
if err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, object)
}
fsMeta, err := readFSMetadata(fs.storage, minioMetaBucket, path.Join(bucketMetaPrefix, bucket, object, fsMetaJSONFile))
// Ignore error if the metadata file is not found, other errors must be returned.
if err != nil && errorCause(err) != errFileNotFound {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
fsMeta := fsMetaV1{}
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
if len(fsMeta.Meta) == 0 {
fsMeta.Meta = make(map[string]string)
}
// Guess content-type from the extension if possible.
if fsMeta.Meta["content-type"] == "" {
if objectExt := path.Ext(object); objectExt != "" {
if content, ok := mimedb.DB[strings.ToLower(strings.TrimPrefix(objectExt, "."))]; ok {
fsMeta.Meta["content-type"] = content.ContentType
}
// Read `fs.json` to perhaps contend with
// parallel Put() operations.
rlk, err := fs.rwPool.Open(fsMetaPath)
if err == nil {
// Read from fs metadata only if it exists.
defer fs.rwPool.Close(fsMetaPath)
if _, rerr := fsMeta.ReadFrom(io.NewSectionReader(rlk, 0, rlk.Size())); rerr != nil {
return ObjectInfo{}, toObjectErr(rerr, bucket, object)
}
}
objInfo := ObjectInfo{
Bucket: bucket,
Name: object,
ModTime: fi.ModTime,
Size: fi.Size,
IsDir: fi.Mode.IsDir(),
MD5Sum: fsMeta.Meta["md5Sum"],
ContentType: fsMeta.Meta["content-type"],
ContentEncoding: fsMeta.Meta["content-encoding"],
// Ignore if `fs.json` is not available, this is true for pre-existing data.
if err != nil && err != errFileNotFound {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, object)
}
// md5Sum has already been extracted into objInfo.MD5Sum. We
// need to remove it from fsMeta.Meta to avoid it from appearing as
// part of response headers. e.g, X-Minio-* or X-Amz-*.
delete(fsMeta.Meta, "md5Sum")
objInfo.UserDefined = fsMeta.Meta
// Stat the file to get file size.
fi, err := fsStatFile(pathJoin(fs.fsPath, bucket, object))
if err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, object)
}
return objInfo, nil
return fsMeta.ToObjectInfo(bucket, object, fi), nil
}
// GetObjectInfo - reads object metadata and replies back ObjectInfo.
@@ -369,6 +534,11 @@ func (fs fsObjects) GetObjectInfo(bucket, object string) (ObjectInfo, error) {
if err := checkGetObjArgs(bucket, object); err != nil {
return ObjectInfo{}, err
}
if _, err := fs.statBucketDir(bucket); err != nil {
return ObjectInfo{}, toObjectErr(err, bucket)
}
return fs.getObjectInfo(bucket, object)
}
@@ -380,17 +550,34 @@ func (fs fsObjects) PutObject(bucket string, object string, size int64, data io.
if err = checkPutObjectArgs(bucket, object, fs); err != nil {
return ObjectInfo{}, err
}
if _, err = fs.statBucketDir(bucket); err != nil {
return ObjectInfo{}, toObjectErr(err, bucket)
}
// No metadata is set, allocate a new one.
if metadata == nil {
metadata = make(map[string]string)
}
uniqueID := mustGetUUID()
fsMeta := newFSMetaV1()
fsMeta.Meta = metadata
var wlk *lock.LockedFile
if bucket != minioMetaBucket {
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
wlk, err = fs.rwPool.Create(fsMetaPath)
if err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
// This close will allow for locks to be synchronized on `fs.json`.
defer wlk.Close()
}
// Uploaded object will first be written to the temporary location which will eventually
// be renamed to the actual location. It is first written to the temporary location
// so that cleaning it up will be easy if the server goes down.
tempObj := uniqueID
tempObj := mustGetUUID()
// Initialize md5 writer.
md5Writer := md5.New()
@@ -414,26 +601,17 @@ func (fs fsObjects) PutObject(bucket string, object string, size int64, data io.
limitDataReader = data
}
// Prepare file to avoid disk fragmentation
if size > 0 {
err = fs.storage.PrepareFile(minioMetaTmpBucket, tempObj, size)
if err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
}
// Allocate a buffer to Read() from request body
bufSize := int64(readSizeV1)
if size > 0 && bufSize > size {
bufSize = size
}
buf := make([]byte, int(bufSize))
teeReader := io.TeeReader(limitDataReader, multiWriter)
var bytesWritten int64
bytesWritten, err = fsCreateFile(fs.storage, teeReader, buf, minioMetaTmpBucket, tempObj)
fsTmpObjPath := pathJoin(fs.fsPath, minioMetaTmpBucket, fs.fsUUID, tempObj)
bytesWritten, err := fsCreateFile(fsTmpObjPath, teeReader, buf, size)
if err != nil {
fs.storage.DeleteFile(minioMetaTmpBucket, tempObj)
fsRemoveFile(fsTmpObjPath)
errorIf(err, "Failed to create object %s/%s", bucket, object)
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
@@ -441,14 +619,14 @@ func (fs fsObjects) PutObject(bucket string, object string, size int64, data io.
// Should return IncompleteBody{} error when reader has fewer
// bytes than specified in request header.
if bytesWritten < size {
fs.storage.DeleteFile(minioMetaTmpBucket, tempObj)
fsRemoveFile(fsTmpObjPath)
return ObjectInfo{}, traceError(IncompleteBody{})
}
// Delete the temporary object in the case of a
// failure. If PutObject succeeds, then there would be
// nothing to delete.
defer fs.storage.DeleteFile(minioMetaTmpBucket, tempObj)
defer fsRemoveFile(fsTmpObjPath)
newMD5Hex := hex.EncodeToString(md5Writer.Sum(nil))
// Update the md5sum if not set with the newly calculated one.
@@ -473,25 +651,26 @@ func (fs fsObjects) PutObject(bucket string, object string, size int64, data io.
}
// Entire object was written to the temp location, now it's safe to rename it to the actual location.
err = fs.storage.RenameFile(minioMetaTmpBucket, tempObj, bucket, object)
fsNSObjPath := pathJoin(fs.fsPath, bucket, object)
if err = fsRenameFile(fsTmpObjPath, fsNSObjPath); err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
if bucket != minioMetaBucket {
// Write FS metadata after a successful namespace operation.
if _, err = fsMeta.WriteTo(wlk); err != nil {
return ObjectInfo{}, toObjectErr(err, bucket, object)
}
}
// Stat the file to fetch timestamp, size.
fi, err := fsStatFile(pathJoin(fs.fsPath, bucket, object))
if err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, object)
}
if bucket != minioMetaBucket {
// Save objects' metadata in `fs.json`.
// Skip creating fs.json if bucket is .minio.sys as the object would have been created
// by minio's S3 layer (ex. policy.json)
fsMeta := newFSMetaV1()
fsMeta.Meta = metadata
fsMetaPath := path.Join(bucketMetaPrefix, bucket, object, fsMetaJSONFile)
if err = writeFSMetadata(fs.storage, minioMetaBucket, fsMetaPath, fsMeta); err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, object)
}
}
return fs.getObjectInfo(bucket, object)
// Success.
return fsMeta.ToObjectInfo(bucket, object, fi), nil
}
// DeleteObject - deletes an object from a bucket, this operation is destructive
@@ -501,38 +680,96 @@ func (fs fsObjects) DeleteObject(bucket, object string) error {
return err
}
if _, err := fs.statBucketDir(bucket); err != nil {
return toObjectErr(err, bucket)
}
minioMetaBucketDir := pathJoin(fs.fsPath, minioMetaBucket)
fsMetaPath := pathJoin(minioMetaBucketDir, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
if bucket != minioMetaBucket {
// We don't store fs.json for minio-S3-layer created files like policy.json,
// hence we don't try to delete fs.json for such files.
err := fs.storage.DeleteFile(minioMetaBucket, path.Join(bucketMetaPrefix, bucket, object, fsMetaJSONFile))
rwlk, lerr := fs.rwPool.Write(fsMetaPath)
if lerr == nil {
// This close will allow for fs locks to be synchronized on `fs.json`.
defer rwlk.Close()
}
if lerr != nil && lerr != errFileNotFound {
return toObjectErr(lerr, bucket, object)
}
}
// Delete the object.
if err := fsDeleteFile(pathJoin(fs.fsPath, bucket), pathJoin(fs.fsPath, bucket, object)); err != nil {
return toObjectErr(traceError(err), bucket, object)
}
if bucket != minioMetaBucket {
// Delete the metadata object.
err := fsDeleteFile(minioMetaBucketDir, fsMetaPath)
if err != nil && err != errFileNotFound {
return toObjectErr(traceError(err), bucket, object)
}
}
if err := fs.storage.DeleteFile(bucket, object); err != nil {
return toObjectErr(traceError(err), bucket, object)
}
return nil
}
// list of all errors that can be ignored in tree walk operation in FS
var fsTreeWalkIgnoredErrs = append(baseIgnoredErrs, []error{
errFileNotFound,
errVolumeNotFound,
}...)
// Returns function "listDir" of the type listDirFunc.
// isLeaf - is used by listDir function to check if an entry
// is a leaf or non-leaf entry.
func (fs fsObjects) listDirFactory(isLeaf isLeafFunc) listDirFunc {
// listDir - lists all the entries at a given prefix and given entry in the prefix.
listDir := func(bucket, prefixDir, prefixEntry string) (entries []string, delayIsLeaf bool, err error) {
entries, err = readDir(pathJoin(fs.fsPath, bucket, prefixDir))
if err == nil {
// Listing needs to be sorted.
sort.Strings(entries)
// Filter entries that have the prefix prefixEntry.
entries = filterMatchingPrefix(entries, prefixEntry)
// Can isLeaf() check be delayed till when it has to be sent down the
// treeWalkResult channel?
delayIsLeaf = delayIsLeafCheck(entries)
if delayIsLeaf {
return entries, delayIsLeaf, nil
}
// 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)
}
}
// Sort again after removing trailing "/" for objects as the previous sort
// does not hold good anymore.
sort.Strings(entries)
// Succes.
return entries, delayIsLeaf, nil
} // Return error at the end.
// Error.
return nil, false, err
}
// Return list factory instance.
return listDir
}
// ListObjects - list all objects at prefix upto maxKeys., optionally delimited by '/'. Maintains the list pool
// state for future re-entrant list requests.
func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKeys int) (ListObjectsInfo, error) {
// Convert entry to ObjectInfo
entryToObjectInfo := func(entry string) (objInfo ObjectInfo, err error) {
if strings.HasSuffix(entry, slashSeparator) {
// Object name needs to be full path.
objInfo.Name = entry
objInfo.IsDir = true
return
}
if objInfo, err = fs.getObjectInfo(bucket, entry); err != nil {
return ObjectInfo{}, err
}
return
if err := checkListObjsArgs(bucket, prefix, marker, delimiter, fs); err != nil {
return ListObjectsInfo{}, err
}
if err := checkListObjsArgs(bucket, prefix, marker, delimiter, fs); err != nil {
if _, err := fs.statBucketDir(bucket); err != nil {
return ListObjectsInfo{}, err
}
@@ -561,6 +798,24 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
recursive = false
}
// Convert entry to ObjectInfo
entryToObjectInfo := func(entry string) (objInfo ObjectInfo, err error) {
if strings.HasSuffix(entry, slashSeparator) {
// Object name needs to be full path.
objInfo.Name = entry
objInfo.IsDir = true
return
}
// Stat the file to get file size.
var fi os.FileInfo
fi, err = fsStatFile(pathJoin(fs.fsPath, bucket, entry))
if err != nil {
return ObjectInfo{}, toObjectErr(traceError(err), bucket, entry)
}
fsMeta := fsMetaV1{}
return fsMeta.ToObjectInfo(bucket, entry, fi), nil
}
heal := false // true only for xl.ListObjectsHeal()
walkResultCh, endWalkCh := fs.listPool.Release(listParams{bucket, recursive, marker, prefix, heal})
if walkResultCh == nil {
@@ -571,12 +826,15 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
// object string does not end with "/".
return !strings.HasSuffix(object, slashSeparator)
}
listDir := listDirFactory(isLeaf, fsTreeWalkIgnoredErrs, fs.storage)
listDir := fs.listDirFactory(isLeaf)
walkResultCh = startTreeWalk(bucket, prefix, marker, recursive, listDir, isLeaf, endWalkCh)
}
var objInfos []ObjectInfo
var eof bool
var nextMarker string
// List until maxKeys requested.
for i := 0; i < maxKeys; {
walkResult, ok := <-walkResultCh
if !ok {
@@ -604,6 +862,8 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
}
i++
}
// Save list routine for the next marker if we haven't reached EOF.
params := listParams{bucket, recursive, nextMarker, prefix, heal}
if !eof {
fs.listPool.Set(params, walkResultCh, endWalkCh)
@@ -618,6 +878,8 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
}
result.Objects = append(result.Objects, objInfo)
}
// Success.
return result, nil
}

View File

@@ -18,81 +18,26 @@ package cmd
import (
"bytes"
"fmt"
"os"
"path/filepath"
"testing"
"time"
)
// TestNewFS - tests initialization of all input disks
// and constructs a valid `FS` object layer.
func TestNewFS(t *testing.T) {
// Do not attempt to create this path, the test validates
// so that newFSObjects initializes non existing paths
// so that newFSObjectLayer initializes non existing paths
// and successfully returns initialized object layer.
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
// Setup to test errFSDiskFormat.
disks := []string{}
for i := 0; i < 6; i++ {
xlDisk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(xlDisk)
disks = append(disks, xlDisk)
}
endpoints, err := parseStorageEndpoints([]string{disk})
if err != nil {
t.Fatal("Uexpected error: ", err)
}
fsStorageDisks, err := initStorageDisks(endpoints)
if err != nil {
t.Fatal("Uexpected error: ", err)
}
endpoints, err = parseStorageEndpoints(disks)
if err != nil {
t.Fatal("Uexpected error: ", err)
}
xlStorageDisks, err := initStorageDisks(endpoints)
if err != nil {
t.Fatal("Uexpected error: ", err)
}
// Initializes all disks with XL
formattedDisks, err := waitForFormatDisks(true, endpoints, xlStorageDisks)
if err != nil {
t.Fatalf("Unable to format XL %s", err)
}
_, err = newXLObjects(formattedDisks)
if err != nil {
t.Fatalf("Unable to initialize XL object, %s", err)
}
testCases := []struct {
disk StorageAPI
expectedErr error
}{
{fsStorageDisks[0], nil},
{xlStorageDisks[0], errFSDiskFormat},
}
for _, testCase := range testCases {
if _, err = waitForFormatDisks(true, endpoints, []StorageAPI{testCase.disk}); err != testCase.expectedErr {
t.Errorf("expected: %s, got :%s", testCase.expectedErr, err)
}
}
_, err = newFSObjects(nil)
_, err := newFSObjectLayer("")
if err != errInvalidArgument {
t.Errorf("Expecting error invalid argument, got %s", err)
}
_, err = newFSObjects(&retryStorage{
remoteStorage: xlStorageDisks[0],
maxRetryAttempts: 1,
retryUnit: time.Millisecond,
retryCap: time.Millisecond * 10,
})
_, err = newFSObjectLayer(disk)
if err != nil {
errMsg := "Unable to recognize backend format, Disk is not in FS format."
if err.Error() == errMsg {
@@ -113,10 +58,10 @@ func TestFSShutdown(t *testing.T) {
bucketName := "testbucket"
objectName := "object"
// Create and return an fsObject with its path in the disk
prepareTest := func() (fsObjects, string) {
prepareTest := func() (*fsObjects, string) {
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
objectContent := "12345"
obj.MakeBucket(bucketName)
sha256sum := ""
@@ -135,12 +80,10 @@ func TestFSShutdown(t *testing.T) {
for i := 1; i <= 5; i++ {
fs, disk := prepareTest()
fs.DeleteObject(bucketName, objectName)
fsStorage := fs.storage.(*retryStorage)
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
if err := fs.Shutdown(); errorCause(err) != errFaultyDisk {
removeAll(disk)
if err := fs.Shutdown(); err != nil {
t.Fatal(i, ", Got unexpected fs shutdown error: ", err)
}
removeAll(disk)
}
}
@@ -150,26 +93,38 @@ func TestFSLoadFormatFS(t *testing.T) {
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
// Assign a new UUID.
uuid := mustGetUUID()
// Regular format loading
_, err := loadFormatFS(fs.storage)
// Initialize meta volume, if volume already exists ignores it.
if err := initMetaVolumeFS(disk, uuid); err != nil {
t.Fatal(err)
}
fsFormatPath := pathJoin(disk, minioMetaBucket, fsFormatJSONFile)
if err := saveFormatFS(preparePath(fsFormatPath), newFSFormatV1()); err != nil {
t.Fatal("Should not fail here", err)
}
_, err := loadFormatFS(disk)
if err != nil {
t.Fatal("Should not fail here", err)
}
// Loading corrupted format file
fs.storage.AppendFile(minioMetaBucket, fsFormatJSONFile, []byte{'b'})
_, err = loadFormatFS(fs.storage)
file, err := os.OpenFile(preparePath(fsFormatPath), os.O_APPEND|os.O_WRONLY|os.O_CREATE, 0666)
if err != nil {
t.Fatal("Should not fail here", err)
}
file.Write([]byte{'b'})
file.Close()
_, err = loadFormatFS(disk)
if err == nil {
t.Fatal("Should return an error here")
}
// Loading format file from faulty disk
fsStorage := fs.storage.(*retryStorage)
fs.storage = newNaughtyDisk(fsStorage, nil, errFaultyDisk)
_, err = loadFormatFS(fs.storage)
if err != errFaultyDisk {
t.Fatal("Should return faulty disk error")
// Loading format file from disk not found.
removeAll(disk)
_, err = loadFormatFS(disk)
if err != nil && err != errUnformattedDisk {
t.Fatal("Should return unformatted disk, but got", err)
}
}
@@ -180,7 +135,7 @@ func TestFSGetBucketInfo(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
obj.MakeBucket(bucketName)
@@ -200,14 +155,12 @@ func TestFSGetBucketInfo(t *testing.T) {
t.Fatal("BucketNameInvalid error not returned")
}
// Loading format file from faulty disk
fsStorage := fs.storage.(*retryStorage)
fs.storage = newNaughtyDisk(fsStorage, nil, errFaultyDisk)
// Check for buckets and should get disk not found.
removeAll(disk)
_, err = fs.GetBucketInfo(bucketName)
if errorCause(err) != errFaultyDisk {
t.Fatal("errFaultyDisk error not returned")
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("BucketNotFound error not returned")
}
}
// TestFSDeleteObject - test fs.DeleteObject() with healthy and corrupted disks
@@ -217,7 +170,7 @@ func TestFSDeleteObject(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
@@ -229,12 +182,16 @@ func TestFSDeleteObject(t *testing.T) {
if err := fs.DeleteObject("fo", objectName); !isSameType(errorCause(err), BucketNameInvalid{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with bucket does not exist
if err := fs.DeleteObject("foobucket", "fooobject"); !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with invalid object name
if err := fs.DeleteObject(bucketName, "\\"); !isSameType(errorCause(err), ObjectNameInvalid{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with inexist bucket/object
if err := fs.DeleteObject("foobucket", "fooobject"); !isSameType(errorCause(err), BucketNotFound{}) {
// Test with object does not exist.
if err := fs.DeleteObject(bucketName, "foooobject"); !isSameType(errorCause(err), ObjectNotFound{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with valid condition
@@ -242,11 +199,12 @@ func TestFSDeleteObject(t *testing.T) {
t.Fatal("Unexpected error: ", err)
}
// Loading format file from faulty disk
fsStorage := fs.storage.(*retryStorage)
fs.storage = newNaughtyDisk(fsStorage, nil, errFaultyDisk)
if err := fs.DeleteObject(bucketName, objectName); errorCause(err) != errFaultyDisk {
t.Fatal("Unexpected error: ", err)
// Delete object should err disk not found.
removeAll(disk)
if err := fs.DeleteObject(bucketName, objectName); err != nil {
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error: ", err)
}
}
}
@@ -258,7 +216,7 @@ func TestFSDeleteBucket(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
err := obj.MakeBucket(bucketName)
@@ -267,29 +225,27 @@ func TestFSDeleteBucket(t *testing.T) {
}
// Test with an invalid bucket name
if err := fs.DeleteBucket("fo"); !isSameType(errorCause(err), BucketNameInvalid{}) {
if err = fs.DeleteBucket("fo"); !isSameType(errorCause(err), BucketNameInvalid{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with an inexistant bucket
if err := fs.DeleteBucket("foobucket"); !isSameType(errorCause(err), BucketNotFound{}) {
if err = fs.DeleteBucket("foobucket"); !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error: ", err)
}
// Test with a valid case
if err := fs.DeleteBucket(bucketName); err != nil {
if err = fs.DeleteBucket(bucketName); err != nil {
t.Fatal("Unexpected error: ", err)
}
obj.MakeBucket(bucketName)
// Loading format file from faulty disk
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 2; i++ {
fs.storage = newNaughtyDisk(fsStorage, map[int]error{i: errFaultyDisk}, nil)
if err := fs.DeleteBucket(bucketName); errorCause(err) != errFaultyDisk {
// Delete bucker should get error disk not found.
removeAll(disk)
if err = fs.DeleteBucket(bucketName); err != nil {
if !isSameType(errorCause(err), BucketNotFound{}) {
t.Fatal("Unexpected error: ", err)
}
}
}
// TestFSListBuckets - tests for fs ListBuckets
@@ -299,7 +255,7 @@ func TestFSListBuckets(t *testing.T) {
defer removeAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(fsObjects)
fs := obj.(*fsObjects)
bucketName := "bucket"
if err := obj.MakeBucket(bucketName); err != nil {
@@ -307,28 +263,40 @@ func TestFSListBuckets(t *testing.T) {
}
// Create a bucket with invalid name
if err := fs.storage.MakeVol("vo^"); err != nil {
if err := mkdirAll(pathJoin(fs.fsPath, "vo^"), 0777); err != nil {
t.Fatal("Unexpected error: ", err)
}
f, err := os.Create(pathJoin(fs.fsPath, "test"))
if err != nil {
t.Fatal("Unexpected error: ", err)
}
f.Close()
// Test
// Test list buckets to have only one entry.
buckets, err := fs.ListBuckets()
if err != nil {
t.Fatal("Unexpected error: ", err)
}
if len(buckets) != 1 {
t.Fatal("ListBuckets not working properly")
t.Fatal("ListBuckets not working properly", buckets)
}
// Test ListBuckets with faulty disks
fsStorage := fs.storage.(*retryStorage)
for i := 1; i <= 2; i++ {
fs.storage = newNaughtyDisk(fsStorage, nil, errFaultyDisk)
if _, err := fs.ListBuckets(); errorCause(err) != errFaultyDisk {
// Test ListBuckets with disk not found.
removeAll(disk)
if _, err := fs.ListBuckets(); err != nil {
if errorCause(err) != errDiskNotFound {
t.Fatal("Unexpected error: ", err)
}
}
longPath := fmt.Sprintf("%0256d", 1)
fs.fsPath = longPath
if _, err := fs.ListBuckets(); err != nil {
if errorCause(err) != errFileNameTooLong {
t.Fatal("Unexpected error: ", err)
}
}
}
// TestFSHealObject - tests for fs HealObject

View File

@@ -62,7 +62,8 @@ var (
globalConfigDir = mustGetConfigPath() // config-dir flag set via command line
// Add new global flags here.
globalIsDistXL = false // "Is Distributed?" flag.
// Indicates if the running minio server is distributed setup.
globalIsDistXL = false
// This flag is set to 'true' by default, it is set to `false`
// when MINIO_BROWSER env is set to 'off'.

View File

@@ -147,7 +147,7 @@ func (n *nsLockMap) statusBlockedToRunning(param nsParam, lockSource, opsID stri
// newDebugLockInfo - Constructs a debugLockInfo value given lock source, status and type.
func newDebugLockInfo(lockSource string, status statusType, readLock bool) debugLockInfo {
lType := debugRLockStr
var lType lockType
if readLock {
lType = debugRLockStr
} else {

View File

@@ -119,6 +119,39 @@ func verifyRPCLockInfoResponse(l lockStateCase, rpcLockInfoMap map[string]*Syste
}
}
// Read entire state of the locks in the system and return.
func getSystemLockState() (SystemLockState, error) {
globalNSMutex.lockMapMutex.Lock()
defer globalNSMutex.lockMapMutex.Unlock()
lockState := SystemLockState{}
lockState.TotalBlockedLocks = globalNSMutex.counters.blocked
lockState.TotalLocks = globalNSMutex.counters.total
lockState.TotalAcquiredLocks = globalNSMutex.counters.granted
for param, debugLock := range globalNSMutex.debugLockMap {
volLockInfo := VolumeLockInfo{}
volLockInfo.Bucket = param.volume
volLockInfo.Object = param.path
volLockInfo.LocksOnObject = debugLock.counters.total
volLockInfo.TotalBlockedLocks = debugLock.counters.blocked
volLockInfo.LocksAcquiredOnObject = debugLock.counters.granted
for opsID, lockInfo := range debugLock.lockInfo {
volLockInfo.LockDetailsOnObject = append(volLockInfo.LockDetailsOnObject, OpsLockState{
OperationID: opsID,
LockSource: lockInfo.lockSource,
LockType: lockInfo.lType,
Status: lockInfo.status,
Since: lockInfo.since,
Duration: time.Now().UTC().Sub(lockInfo.since),
})
}
lockState.LocksInfoPerObject = append(lockState.LocksInfoPerObject, volLockInfo)
}
return lockState, nil
}
// Asserts the lock counter from the global globalNSMutex inmemory lock with the expected one.
func verifyGlobalLockStats(l lockStateCase, t *testing.T, testNum int) {
globalNSMutex.lockMapMutex.Lock()

View File

@@ -0,0 +1,68 @@
/*
* 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 (
"fmt"
"testing"
"time"
"github.com/minio/dsync"
)
// Tests lock rpc client.
func TestLockRPCClient(t *testing.T) {
lkClient := newLockRPCClient(authConfig{
accessKey: "abcd",
secretKey: "abcd123",
serverAddr: fmt.Sprintf("%X", time.Now().UTC().UnixNano()),
serviceEndpoint: pathJoin(lockRPCPath, "/test/1"),
secureConn: false,
serviceName: "Dsync",
})
// Attempt all calls.
_, err := lkClient.RLock(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for Rlock to fail")
}
_, err = lkClient.Lock(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for Lock to fail")
}
_, err = lkClient.RUnlock(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for RUnlock to fail")
}
_, err = lkClient.Unlock(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for Unlock to fail")
}
_, err = lkClient.ForceUnlock(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for ForceUnlock to fail")
}
_, err = lkClient.Expired(dsync.LockArgs{})
if err == nil {
t.Fatal("Expected for Expired to fail")
}
}

View File

@@ -68,52 +68,6 @@ type OpsLockState struct {
Duration time.Duration `json:"duration"` // Duration since the lock was held.
}
// Read entire state of the locks in the system and return.
func getSystemLockState() (SystemLockState, error) {
globalNSMutex.lockMapMutex.Lock()
defer globalNSMutex.lockMapMutex.Unlock()
// Fetch current time once instead of fetching system time for every lock.
timeNow := time.Now().UTC()
lockState := SystemLockState{
TotalAcquiredLocks: globalNSMutex.counters.granted,
TotalLocks: globalNSMutex.counters.total,
TotalBlockedLocks: globalNSMutex.counters.blocked,
}
var totalReadLocks, totalWriteLocks int64
for param, debugLock := range globalNSMutex.debugLockMap {
volLockInfo := VolumeLockInfo{}
volLockInfo.Bucket = param.volume
volLockInfo.Object = param.path
volLockInfo.LocksOnObject = debugLock.counters.total
volLockInfo.TotalBlockedLocks = debugLock.counters.blocked
volLockInfo.LocksAcquiredOnObject = debugLock.counters.granted
for opsID, lockInfo := range debugLock.lockInfo {
volLockInfo.LockDetailsOnObject = append(volLockInfo.LockDetailsOnObject, OpsLockState{
OperationID: opsID,
LockSource: lockInfo.lockSource,
LockType: lockInfo.lType,
Status: lockInfo.status,
Since: lockInfo.since,
Duration: timeNow.Sub(lockInfo.since),
})
switch lockInfo.lType {
case debugRLockStr:
totalReadLocks++
case debugWLockStr:
totalWriteLocks++
}
}
volLockInfo.TotalReadLocks = totalReadLocks
volLockInfo.TotalWriteLocks = totalWriteLocks
lockState.LocksInfoPerObject = append(lockState.LocksInfoPerObject, volLockInfo)
}
return lockState, nil
}
// listLocksInfo - Fetches locks held on bucket, matching prefix older than relTime.
func listLocksInfo(bucket, prefix string, relTime time.Duration) []VolumeLockInfo {
globalNSMutex.lockMapMutex.Lock()

View File

@@ -28,6 +28,14 @@ import (
// Global name space lock.
var globalNSMutex *nsLockMap
// RWLocker - locker interface extends sync.Locker
// to introduce RLock, RUnlock.
type RWLocker interface {
sync.Locker
RLock()
RUnlock()
}
// Initialize distributed locking only in case of distributed setup.
// Returns if the setup is distributed or not on success.
func initDsyncNodes(eps []*url.URL) error {
@@ -68,13 +76,6 @@ func initNSLock(isDistXL bool) {
globalNSMutex.debugLockMap = make(map[nsParam]*debugLockInfoPerVolumePath)
}
// RWLocker - interface that any read-write locking library should implement.
type RWLocker interface {
sync.Locker
RLock()
RUnlock()
}
// nsParam - carries name space resource.
type nsParam struct {
volume string
@@ -94,8 +95,7 @@ type nsLockMap struct {
counters *lockStat
debugLockMap map[nsParam]*debugLockInfoPerVolumePath // Info for instrumentation on locks.
// Indicates whether the locking service is part
// of a distributed setup or not.
// Indicates if namespace is part of a distributed setup.
isDistXL bool
lockMap map[nsParam]*nsLock
lockMapMutex sync.Mutex
@@ -256,14 +256,14 @@ func (n *nsLockMap) ForceUnlock(volume, path string) {
// lockInstance - frontend/top-level interface for namespace locks.
type lockInstance struct {
n *nsLockMap
ns *nsLockMap
volume, path, opsID string
}
// NewNSLock - returns a lock instance for a given volume and
// path. The returned lockInstance object encapsulates the nsLockMap,
// volume, path and operation ID.
func (n *nsLockMap) NewNSLock(volume, path string) *lockInstance {
func (n *nsLockMap) NewNSLock(volume, path string) RWLocker {
return &lockInstance{n, volume, path, getOpsID()}
}
@@ -271,24 +271,24 @@ func (n *nsLockMap) NewNSLock(volume, path string) *lockInstance {
func (li *lockInstance) Lock() {
lockSource := callerSource()
readLock := false
li.n.lock(li.volume, li.path, lockSource, li.opsID, readLock)
li.ns.lock(li.volume, li.path, lockSource, li.opsID, readLock)
}
// Unlock - block until write lock is released.
func (li *lockInstance) Unlock() {
readLock := false
li.n.unlock(li.volume, li.path, li.opsID, readLock)
li.ns.unlock(li.volume, li.path, li.opsID, readLock)
}
// RLock - block until read lock is taken.
func (li *lockInstance) RLock() {
lockSource := callerSource()
readLock := true
li.n.lock(li.volume, li.path, lockSource, li.opsID, readLock)
li.ns.lock(li.volume, li.path, lockSource, li.opsID, readLock)
}
// RUnlock - block until read lock is released.
func (li *lockInstance) RUnlock() {
readLock := true
li.n.unlock(li.volume, li.path, li.opsID, readLock)
li.ns.unlock(li.volume, li.path, li.opsID, readLock)
}

View File

@@ -48,6 +48,12 @@ func init() {
globalObjLayerMutex = &sync.Mutex{}
}
// Check if the disk is remote.
func isRemoteDisk(disk StorageAPI) bool {
_, ok := disk.(*networkStorage)
return ok
}
// House keeping code for FS/XL and distributed Minio setup.
func houseKeeping(storageDisks []StorageAPI) error {
var wg = &sync.WaitGroup{}
@@ -60,8 +66,8 @@ func houseKeeping(storageDisks []StorageAPI) error {
if disk == nil {
continue
}
if _, ok := disk.(*networkStorage); ok {
// Skip remote disks.
// Skip remote disks.
if isRemoteDisk(disk) {
continue
}
wg.Add(1)

View File

@@ -352,16 +352,6 @@ func isErrBucketPolicyNotFound(err error) bool {
return false
}
// Check if error type is ObjectNameInvalid.
func isErrObjectNameInvalid(err error) bool {
err = errorCause(err)
switch err.(type) {
case ObjectNameInvalid:
return true
}
return false
}
// Check if error type is ObjectNotFound.
func isErrObjectNotFound(err error) bool {
err = errorCause(err)

View File

@@ -571,16 +571,11 @@ func testListObjects(obj ObjectLayer, instanceType string, t TestErrHandler) {
// Initialize FS backend for the benchmark.
func initFSObjectsB(disk string, t *testing.B) (obj ObjectLayer) {
endPoints, err := parseStorageEndpoints([]string{disk})
var err error
obj, err = newFSObjectLayer(disk)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
obj, _, err = initObjectLayer(endPoints)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
return obj
}

View File

@@ -18,10 +18,14 @@ package cmd
import (
"encoding/json"
"io"
"io/ioutil"
"path"
"sort"
"sync"
"time"
"github.com/minio/minio/pkg/lock"
)
// A uploadInfo represents the s3 compatible spec.
@@ -67,6 +71,44 @@ func (u *uploadsV1) RemoveUploadID(uploadID string) {
}
}
// IsEmpty - is true if no more uploads available.
func (u *uploadsV1) IsEmpty() bool {
return len(u.Uploads) == 0
}
func (u *uploadsV1) WriteTo(writer io.Writer) (n int64, err error) {
// Serialize to prepare to write to disk.
var uplBytes []byte
uplBytes, err = json.Marshal(u)
if err != nil {
return 0, traceError(err)
}
if err = writer.(*lock.LockedFile).Truncate(0); err != nil {
return 0, traceError(err)
}
_, err = writer.Write(uplBytes)
if err != nil {
return 0, traceError(err)
}
return int64(len(uplBytes)), nil
}
func (u *uploadsV1) ReadFrom(reader io.Reader) (n int64, err error) {
var uploadIDBytes []byte
uploadIDBytes, err = ioutil.ReadAll(reader)
if err != nil {
return 0, traceError(err)
}
if len(uploadIDBytes) == 0 {
return 0, traceError(io.EOF)
}
// Decode `uploads.json`.
if err = json.Unmarshal(uploadIDBytes, u); err != nil {
return 0, traceError(err)
}
return int64(len(uploadIDBytes)), nil
}
// readUploadsJSON - get all the saved uploads JSON.
func readUploadsJSON(bucket, object string, disk StorageAPI) (uploadIDs uploadsV1, err error) {
uploadJSONPath := path.Join(bucket, object, uploadsJSONFile)
@@ -100,8 +142,7 @@ func writeUploadJSON(u *uploadsV1, uploadsPath, tmpPath string, disk StorageAPI)
return traceError(wErr)
}
// Write `uploads.json` to disk. First to tmp location and
// then rename.
// Write `uploads.json` to disk. First to tmp location and then rename.
if wErr = disk.AppendFile(minioMetaTmpBucket, tmpPath, uplBytes); wErr != nil {
return traceError(wErr)
}

View File

@@ -1207,7 +1207,7 @@ func testListMultipartUploads(obj ObjectLayer, instanceType string, t TestErrHan
}
for i, testCase := range testCases {
// fmt.Println(testCase) // uncomment to peek into the test cases.
// fmt.Println(i+1, testCase) // uncomment to peek into the test cases.
actualResult, actualErr := obj.ListMultipartUploads(testCase.bucket, testCase.prefix, testCase.keyMarker, testCase.uploadIDMarker, testCase.delimiter, testCase.maxUploads)
if actualErr != nil && testCase.shouldPass {
t.Errorf("Test %d: %s: Expected to pass, but failed with: <ERROR> %s", i+1, instanceType, actualErr.Error())
@@ -1520,7 +1520,7 @@ func TestListObjectParts(t *testing.T) {
ExecObjectLayerTest(t, testListObjectParts)
}
// testListMultipartUploads - Tests validate listing of multipart uploads.
// testListObjectParts - test validate listing of object parts.
func testListObjectParts(obj ObjectLayer, instanceType string, t TestErrHandler) {
bucketNames := []string{"minio-bucket", "minio-2-bucket"}

View File

@@ -770,8 +770,8 @@ func (api objectAPIHandlers) CompleteMultipartUploadHandler(w http.ResponseWrite
md5Sum, err = objectAPI.CompleteMultipartUpload(bucket, object, uploadID, completeParts)
if err != nil {
err = errorCause(err)
errorIf(err, "Unable to complete multipart upload.")
err = errorCause(err)
switch oErr := err.(type) {
case PartTooSmall:
// Write part too small error.

View File

@@ -715,6 +715,16 @@ func testNonExistantObjectInBucket(obj ObjectLayer, instanceType string, c TestE
}
}
// Check if error type is ObjectNameInvalid.
func isErrObjectNameInvalid(err error) bool {
err = errorCause(err)
switch err.(type) {
case ObjectNameInvalid:
return true
}
return false
}
// Wrapper for calling testGetDirectoryReturnsObjectNotFound for both XL and FS.
func (s *ObjectLayerAPISuite) TestGetDirectoryReturnsObjectNotFound(c *C) {
ExecObjectLayerTest(c, testGetDirectoryReturnsObjectNotFound)

View File

@@ -42,6 +42,17 @@ func isSysErrIO(err error) bool {
return err == syscall.EIO
}
// Check if the given error corresponds to EISDIR (is a directory).
func isSysErrIsDir(err error) bool {
if pathErr, ok := err.(*os.PathError); ok {
switch pathErr.Err {
case syscall.EISDIR:
return true
}
}
return false
}
// Check if the given error corresponds to ENOTDIR (is not a directory).
func isSysErrNotDir(err error) bool {
if pathErr, ok := err.(*os.PathError); ok {

View File

@@ -652,7 +652,6 @@ func (s *posix) createFile(volume, path string) (f *os.File, err error) {
// PrepareFile - run prior actions before creating a new file for optimization purposes
// Currently we use fallocate when available to avoid disk fragmentation as much as possible
func (s *posix) PrepareFile(volume, path string, fileSize int64) (err error) {
// It doesn't make sense to create a negative-sized file
if fileSize <= 0 {
return errInvalidArgument

View File

@@ -43,8 +43,8 @@ func newPosixTestSetup() (StorageAPI, string, error) {
return posixStorage, diskPath, nil
}
// Tests posix.getDiskInfo()
func TestGetDiskInfo(t *testing.T) {
// TestPosixs posix.getDiskInfo()
func TestPosixGetDiskInfo(t *testing.T) {
path, err := ioutil.TempDir(globalTestTmpDir, "minio-")
if err != nil {
t.Fatalf("Unable to create a temporary directory, %s", err)
@@ -67,8 +67,8 @@ func TestGetDiskInfo(t *testing.T) {
}
}
// TestReadAll - Tests the functionality implemented by posix ReadAll storage API.
func TestReadAll(t *testing.T) {
// TestPosixReadAll - TestPosixs the functionality implemented by posix ReadAll storage API.
func TestPosixReadAll(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -91,27 +91,27 @@ func TestReadAll(t *testing.T) {
t.Fatalf("Unable to create a file \"as-file-parent\", %s", err)
}
// Testcases to validate different conditions for ReadAll API.
// TestPosixcases to validate different conditions for ReadAll API.
testCases := []struct {
volume string
path string
err error
}{
// Test case - 1.
// TestPosix case - 1.
// Validate volume does not exist.
{
volume: "i-dont-exist",
path: "",
err: errVolumeNotFound,
},
// Test case - 2.
// TestPosix case - 2.
// Validate bad condition file does not exist.
{
volume: "exists",
path: "as-file-not-found",
err: errFileNotFound,
},
// Test case - 3.
// TestPosix case - 3.
// Validate bad condition file exists as prefix/directory and
// we are attempting to read it.
{
@@ -119,21 +119,21 @@ func TestReadAll(t *testing.T) {
path: "as-directory",
err: errFileNotFound,
},
// Test case - 4.
// TestPosix case - 4.
{
volume: "exists",
path: "as-file-parent/as-file",
err: errFileNotFound,
},
// Test case - 5.
// TestPosix case - 5.
// Validate the good condition file exists and we are able to read it.
{
volume: "exists",
path: "as-file",
err: nil,
},
// Test case - 6.
// Test case with invalid volume name.
// TestPosix case - 6.
// TestPosix case with invalid volume name.
{
volume: "ab",
path: "as-file",
@@ -146,15 +146,15 @@ func TestReadAll(t *testing.T) {
for i, testCase := range testCases {
dataRead, err = posixStorage.ReadAll(testCase.volume, testCase.path)
if err != testCase.err {
t.Fatalf("Test %d: Expected err \"%s\", got err \"%s\"", i+1, testCase.err, err)
t.Fatalf("TestPosix %d: Expected err \"%s\", got err \"%s\"", i+1, testCase.err, err)
}
if err == nil {
if string(dataRead) != string([]byte("Hello, World")) {
t.Errorf("Test %d: Expected the data read to be \"%s\", but instead got \"%s\"", i+1, "Hello, World", string(dataRead))
t.Errorf("TestPosix %d: Expected the data read to be \"%s\", but instead got \"%s\"", i+1, "Hello, World", string(dataRead))
}
}
}
// Testing for faulty disk.
// TestPosixing for faulty disk.
// Setting ioErrCount > maxAllowedIOError.
if posixType, ok := posixStorage.(*posix); ok {
// setting the io error count from as specified in the test case.
@@ -168,8 +168,8 @@ func TestReadAll(t *testing.T) {
}
}
// TestNewPosix all the cases handled in posix storage layer initialization.
func TestNewPosix(t *testing.T) {
// TestPosixNewPosix all the cases handled in posix storage layer initialization.
func TestPosixNewPosix(t *testing.T) {
// Temporary dir name.
tmpDirName := globalTestTmpDir + "/" + "minio-" + nextSuffix()
// Temporary file name.
@@ -207,14 +207,14 @@ func TestNewPosix(t *testing.T) {
// Initialize a new posix layer.
_, err := newPosix(testCase.name)
if err != testCase.err {
t.Fatalf("Test %d failed wanted: %s, got: %s", i+1, err, testCase.err)
t.Fatalf("TestPosix %d failed wanted: %s, got: %s", i+1, err, testCase.err)
}
}
}
// TestMakeVol - Test validate the logic for creation of new posix volume.
// TestPosixMakeVol - TestPosix validate the logic for creation of new posix volume.
// Asserts the failures too against the expected failures.
func TestMakeVol(t *testing.T) {
func TestPosixMakeVol(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -237,35 +237,35 @@ func TestMakeVol(t *testing.T) {
ioErrCount int
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
// A valid case, volume creation is expected to succeed.
{
volName: "success-vol",
ioErrCount: 0,
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
// Case where a file exists by the name of the volume to be created.
{
volName: "vol-as-file",
ioErrCount: 0,
expectedErr: errVolumeExists,
},
// Test case - 3.
// TestPosix case - 3.
{
volName: "existing-vol",
ioErrCount: 0,
expectedErr: errVolumeExists,
},
// Test case - 4.
// TestPosix case - 4.
// IO error > maxAllowedIOError, should fail with errFaultyDisk.
{
volName: "vol",
ioErrCount: 6,
expectedErr: errFaultyDisk,
},
// Test case - 5.
// Test case with invalid volume name.
// TestPosix case - 5.
// TestPosix case with invalid volume name.
{
volName: "ab",
ioErrCount: 0,
@@ -281,11 +281,11 @@ func TestMakeVol(t *testing.T) {
t.Errorf("Expected the StorageAPI to be of type *posix")
}
if err := posixStorage.MakeVol(testCase.volName); err != testCase.expectedErr {
t.Fatalf("Test %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posix, err := newPosix("/usr")
@@ -299,8 +299,8 @@ func TestMakeVol(t *testing.T) {
}
}
// TestDeleteVol - Validates the expected behaviour of posix.DeleteVol for various cases.
func TestDeleteVol(t *testing.T) {
// TestPosixDeleteVol - Validates the expected behaviour of posix.DeleteVol for various cases.
func TestPosixDeleteVol(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -313,7 +313,7 @@ func TestDeleteVol(t *testing.T) {
t.Fatalf("Unable to create volume, %s", err)
}
// Test failure cases.
// TestPosix failure cases.
vol := slashpath.Join(path, "nonempty-vol")
if err = os.Mkdir(vol, 0777); err != nil {
t.Fatalf("Unable to create directory, %s", err)
@@ -327,35 +327,35 @@ func TestDeleteVol(t *testing.T) {
ioErrCount int
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
// A valida case. Empty vol, should be possible to delete.
{
volName: "success-vol",
ioErrCount: 0,
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
// volume is non-existent.
{
volName: "nonexistent-vol",
ioErrCount: 0,
expectedErr: errVolumeNotFound,
},
// Test case - 3.
// TestPosix case - 3.
// It shouldn't be possible to delete an non-empty volume, validating the same.
{
volName: "nonempty-vol",
ioErrCount: 0,
expectedErr: errVolumeNotEmpty,
},
// Test case - 4.
// TestPosix case - 4.
// IO error > maxAllowedIOError, should fail with errFaultyDisk.
{
volName: "my-disk",
ioErrCount: 6,
expectedErr: errFaultyDisk,
},
// Test case - 5.
// TestPosix case - 5.
// Invalid volume name.
{
volName: "ab",
@@ -372,11 +372,11 @@ func TestDeleteVol(t *testing.T) {
t.Errorf("Expected the StorageAPI to be of type *posix")
}
if err = posixStorage.DeleteVol(testCase.volName); err != testCase.expectedErr {
t.Fatalf("Test: %d, expected: %s, got: %s", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix: %d, expected: %s, got: %s", i+1, testCase.expectedErr, err)
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/usr")
@@ -396,7 +396,7 @@ func TestDeleteVol(t *testing.T) {
// removing the disk, used to recreate disk not found error.
removeAll(diskPath)
// Test for delete on an removed disk.
// TestPosix for delete on an removed disk.
// should fail with disk not found.
err = posixDeletedStorage.DeleteVol("Del-Vol")
if err != errDiskNotFound {
@@ -404,8 +404,8 @@ func TestDeleteVol(t *testing.T) {
}
}
// TestStatVol - Tests validate the volume info returned by posix.StatVol() for various inputs.
func TestStatVol(t *testing.T) {
// TestPosixStatVol - TestPosixs validate the volume info returned by posix.StatVol() for various inputs.
func TestPosixStatVol(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -423,25 +423,25 @@ func TestStatVol(t *testing.T) {
ioErrCount int
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
{
volName: "success-vol",
ioErrCount: 0,
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
{
volName: "nonexistent-vol",
ioErrCount: 0,
expectedErr: errVolumeNotFound,
},
// Test case - 3.
// TestPosix case - 3.
{
volName: "success-vol",
ioErrCount: 6,
expectedErr: errFaultyDisk,
},
// Test case - 4.
// TestPosix case - 4.
{
volName: "ab",
ioErrCount: 0,
@@ -460,12 +460,12 @@ func TestStatVol(t *testing.T) {
}
volInfo, err = posixStorage.StatVol(testCase.volName)
if err != testCase.expectedErr {
t.Fatalf("Test case : %d, Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix case : %d, Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
if err == nil {
if volInfo.Name != volInfo.Name {
t.Errorf("Test case %d: Expected the volume name to be \"%s\", instead found \"%s\"", i+1, volInfo.Name, volInfo.Name)
t.Errorf("TestPosix case %d: Expected the volume name to be \"%s\", instead found \"%s\"", i+1, volInfo.Name, volInfo.Name)
}
}
}
@@ -477,7 +477,7 @@ func TestStatVol(t *testing.T) {
// removing the disk, used to recreate disk not found error.
removeAll(diskPath)
// Test for delete on an removed disk.
// TestPosix for delete on an removed disk.
// should fail with disk not found.
_, err = posixDeletedStorage.StatVol("Stat vol")
if err != errDiskNotFound {
@@ -485,8 +485,8 @@ func TestStatVol(t *testing.T) {
}
}
// TestListVols - Validates the result and the error output for posix volume listing functionality posix.ListVols().
func TestListVols(t *testing.T) {
// TestPosixListVols - Validates the result and the error output for posix volume listing functionality posix.ListVols().
func TestPosixListVols(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -494,14 +494,14 @@ func TestListVols(t *testing.T) {
}
var volInfo []VolInfo
// Test empty list vols.
// TestPosix empty list vols.
if volInfo, err = posixStorage.ListVols(); err != nil {
t.Fatalf("expected: <nil>, got: %s", err)
} else if len(volInfo) != 0 {
t.Fatalf("expected: [], got: %s", volInfo)
}
// Test non-empty list vols.
// TestPosix non-empty list vols.
if err = posixStorage.MakeVol("success-vol"); err != nil {
t.Fatalf("Unable to create volume, %s", err)
}
@@ -538,8 +538,8 @@ func TestListVols(t *testing.T) {
}
}
// TestPosixListDir - Tests validate the directory listing functionality provided by posix.ListDir .
func TestPosixListDir(t *testing.T) {
// TestPosixPosixListDir - TestPosixs validate the directory listing functionality provided by posix.ListDir .
func TestPosixPosixListDir(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -573,7 +573,7 @@ func TestPosixListDir(t *testing.T) {
expectedListDir []string
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
// valid case with existing volume and file to delete.
{
srcVol: "success-vol",
@@ -582,7 +582,7 @@ func TestPosixListDir(t *testing.T) {
expectedListDir: []string{"def/", "xyz/"},
expectedErr: nil,
},
// Test case - 1.
// TestPosix case - 1.
// valid case with existing volume and file to delete.
{
srcVol: "success-vol",
@@ -591,7 +591,7 @@ func TestPosixListDir(t *testing.T) {
expectedListDir: []string{"ghi/"},
expectedErr: nil,
},
// Test case - 1.
// TestPosix case - 1.
// valid case with existing volume and file to delete.
{
srcVol: "success-vol",
@@ -600,31 +600,31 @@ func TestPosixListDir(t *testing.T) {
expectedListDir: []string{"success-file"},
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
{
srcVol: "success-vol",
srcPath: "abcdef",
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 3.
// Test case with invalid volume name.
// TestPosix case - 3.
// TestPosix case with invalid volume name.
{
srcVol: "ab",
srcPath: "success-file",
ioErrCnt: 0,
expectedErr: errInvalidArgument,
},
// Test case - 4.
// Test case with io error count > max limit.
// TestPosix case - 4.
// TestPosix case with io error count > max limit.
{
srcVol: "success-vol",
srcPath: "success-file",
ioErrCnt: 6,
expectedErr: errFaultyDisk,
},
// Test case - 5.
// Test case with non existent volume.
// TestPosix case - 5.
// TestPosix case with non existent volume.
{
srcVol: "non-existent-vol",
srcPath: "success-file",
@@ -644,18 +644,18 @@ func TestPosixListDir(t *testing.T) {
}
dirList, err = posixStorage.ListDir(testCase.srcVol, testCase.srcPath)
if err != testCase.expectedErr {
t.Fatalf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
if err == nil {
for _, expected := range testCase.expectedListDir {
if !strings.Contains(strings.Join(dirList, ","), expected) {
t.Errorf("Test case %d: Expected the directory listing to be \"%v\", but got \"%v\"", i+1, testCase.expectedListDir, dirList)
t.Errorf("TestPosix case %d: Expected the directory listing to be \"%v\", but got \"%v\"", i+1, testCase.expectedListDir, dirList)
}
}
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/usr")
@@ -668,7 +668,7 @@ func TestPosixListDir(t *testing.T) {
}
}
// Test for delete on an removed disk.
// TestPosix for delete on an removed disk.
// should fail with disk not found.
err = posixDeletedStorage.DeleteFile("del-vol", "my-file")
if err != errDiskNotFound {
@@ -676,8 +676,8 @@ func TestPosixListDir(t *testing.T) {
}
}
// TestDeleteFile - Series of test cases construct valid and invalid input data and validates the result and the error response.
func TestDeleteFile(t *testing.T) {
// TestPosixDeleteFile - Series of test cases construct valid and invalid input data and validates the result and the error response.
func TestPosixDeleteFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -706,7 +706,7 @@ func TestDeleteFile(t *testing.T) {
ioErrCnt int
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
// valid case with existing volume and file to delete.
{
srcVol: "success-vol",
@@ -714,7 +714,7 @@ func TestDeleteFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
// The file was deleted in the last case, so DeleteFile should fail.
{
srcVol: "success-vol",
@@ -722,32 +722,32 @@ func TestDeleteFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 3.
// Test case with io error count > max limit.
// TestPosix case - 3.
// TestPosix case with io error count > max limit.
{
srcVol: "success-vol",
srcPath: "success-file",
ioErrCnt: 6,
expectedErr: errFaultyDisk,
},
// Test case - 4.
// Test case with segment of the volume name > 255.
// TestPosix case - 4.
// TestPosix case with segment of the volume name > 255.
{
srcVol: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
srcPath: "success-file",
ioErrCnt: 0,
expectedErr: errInvalidArgument,
},
// Test case - 5.
// Test case with non-existent volume.
// TestPosix case - 5.
// TestPosix case with non-existent volume.
{
srcVol: "non-existent-vol",
srcPath: "success-file",
ioErrCnt: 0,
expectedErr: errVolumeNotFound,
},
// Test case - 6.
// Test case with src path segment > 255.
// TestPosix case - 6.
// TestPosix case with src path segment > 255.
{
srcVol: "success-vol",
srcPath: "my-obj-del-0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001",
@@ -765,11 +765,11 @@ func TestDeleteFile(t *testing.T) {
t.Errorf("Expected the StorageAPI to be of type *posix")
}
if err = posixStorage.DeleteFile(testCase.srcVol, testCase.srcPath); err != testCase.expectedErr {
t.Errorf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
t.Errorf("TestPosix case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/usr")
@@ -782,7 +782,7 @@ func TestDeleteFile(t *testing.T) {
}
}
// Test for delete on an removed disk.
// TestPosix for delete on an removed disk.
// should fail with disk not found.
err = posixDeletedStorage.DeleteFile("del-vol", "my-file")
if err != errDiskNotFound {
@@ -790,8 +790,8 @@ func TestDeleteFile(t *testing.T) {
}
}
// TestReadFile - Tests posix.ReadFile with wide range of cases and asserts the result and error response.
func TestReadFile(t *testing.T) {
// TestPosixReadFile - TestPosixs posix.ReadFile with wide range of cases and asserts the result and error response.
func TestPosixReadFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -984,7 +984,7 @@ func TestReadFile(t *testing.T) {
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS == "linux" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/")
@@ -1000,7 +1000,7 @@ func TestReadFile(t *testing.T) {
}
}
// Testing for faulty disk.
// TestPosixing for faulty disk.
// setting ioErrCnt to 6.
// should fail with errFaultyDisk.
if posixType, ok := posixStorage.(*posix); ok {
@@ -1017,8 +1017,8 @@ func TestReadFile(t *testing.T) {
}
}
// Test posix.AppendFile()
func TestAppendFile(t *testing.T) {
// TestPosix posix.AppendFile()
func TestPosixAppendFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -1042,9 +1042,9 @@ func TestAppendFile(t *testing.T) {
}{
{"myobject", nil},
{"path/to/my/object", nil},
// Test to append to previously created file.
// TestPosix to append to previously created file.
{"myobject", nil},
// Test to use same path of previously created file.
// TestPosix to use same path of previously created file.
{"path/to/my/testobject", nil},
// One path segment length is 255 chars long.
{"path/to/my/object000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001", nil},
@@ -1074,7 +1074,7 @@ func TestAppendFile(t *testing.T) {
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/usr")
@@ -1086,14 +1086,14 @@ func TestAppendFile(t *testing.T) {
t.Errorf("expected: Permission error, got: %s", err)
}
}
// Test case with invalid volume name.
// TestPosix case with invalid volume name.
// A valid volume name should be atleast of size 3.
err = posixStorage.AppendFile("bn", "yes", []byte("hello, world"))
if err != errInvalidArgument {
t.Fatalf("expected: \"Invalid argument error\", got: \"%s\"", err)
}
// Test case with IO error count > max limit.
// TestPosix case with IO error count > max limit.
// setting ioErrCnt to 6.
// should fail with errFaultyDisk.
@@ -1109,8 +1109,8 @@ func TestAppendFile(t *testing.T) {
}
}
// Test posix.PrepareFile()
func TestPrepareFile(t *testing.T) {
// TestPosix posix.PrepareFile()
func TestPosixPrepareFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -1133,9 +1133,9 @@ func TestPrepareFile(t *testing.T) {
}{
{"myobject", nil},
{"path/to/my/object", nil},
// Test to append to previously created file.
// TestPosix to append to previously created file.
{"myobject", nil},
// Test to use same path of previously created file.
// TestPosix to use same path of previously created file.
{"path/to/my/testobject", nil},
{"object-as-dir", errIsNotRegular},
// path segment uses previously uploaded object.
@@ -1161,7 +1161,7 @@ func TestPrepareFile(t *testing.T) {
}
}
// Test for permission denied.
// TestPosix for permission denied.
if runtime.GOOS != "windows" {
// Initialize posix storage layer for permission denied error.
posixStorage, err = newPosix("/usr")
@@ -1174,20 +1174,20 @@ func TestPrepareFile(t *testing.T) {
}
}
// Test case with invalid file size which should be strictly positive
// TestPosix case with invalid file size which should be strictly positive
err = posixStorage.PrepareFile("bn", "yes", -3)
if err != errInvalidArgument {
t.Fatalf("should fail: %v", err)
}
// Test case with invalid volume name.
// TestPosix case with invalid volume name.
// A valid volume name should be atleast of size 3.
err = posixStorage.PrepareFile("bn", "yes", 16)
if err != errInvalidArgument {
t.Fatalf("expected: \"Invalid argument error\", got: \"%s\"", err)
}
// Test case with IO error count > max limit.
// TestPosix case with IO error count > max limit.
// setting ioErrCnt to 6.
// should fail with errFaultyDisk.
@@ -1203,8 +1203,8 @@ func TestPrepareFile(t *testing.T) {
}
}
// Test posix.RenameFile()
func TestRenameFile(t *testing.T) {
// TestPosix posix.RenameFile()
func TestPosixRenameFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -1250,7 +1250,7 @@ func TestRenameFile(t *testing.T) {
ioErrCnt int
expectedErr error
}{
// Test case - 1.
// TestPosix case - 1.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1259,7 +1259,7 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 2.
// TestPosix case - 2.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1268,8 +1268,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 3.
// Test to overwrite destination file.
// TestPosix case - 3.
// TestPosix to overwrite destination file.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1278,8 +1278,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 4.
// Test case with io error count set to 1.
// TestPosix case - 4.
// TestPosix case with io error count set to 1.
// expected not to fail.
{
srcVol: "src-vol",
@@ -1289,8 +1289,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 1,
expectedErr: nil,
},
// Test case - 5.
// Test case with io error count set to maximum allowed count.
// TestPosix case - 5.
// TestPosix case with io error count set to maximum allowed count.
// expected not to fail.
{
srcVol: "src-vol",
@@ -1300,8 +1300,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 5,
expectedErr: nil,
},
// Test case - 6.
// Test case with non-existent source file.
// TestPosix case - 6.
// TestPosix case with non-existent source file.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1310,8 +1310,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 7.
// Test to check failure of source and destination are not same type.
// TestPosix case - 7.
// TestPosix to check failure of source and destination are not same type.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1320,8 +1320,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileAccessDenied,
},
// Test case - 8.
// Test to check failure of destination directory exists.
// TestPosix case - 8.
// TestPosix to check failure of destination directory exists.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1330,8 +1330,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileAccessDenied,
},
// Test case - 9.
// Test case with io error count is greater than maxAllowedIOError.
// TestPosix case - 9.
// TestPosix case with io error count is greater than maxAllowedIOError.
{
srcVol: "src-vol",
destVol: "dest-vol",
@@ -1340,8 +1340,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 6,
expectedErr: errFaultyDisk,
},
// Test case - 10.
// Test case with source being a file and destination being a directory.
// TestPosix case - 10.
// TestPosix case with source being a file and destination being a directory.
// Either both have to be files or directories.
// Expecting to fail with `errFileAccessDenied`.
{
@@ -1352,8 +1352,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileAccessDenied,
},
// Test case - 11.
// Test case with non-existent source volume.
// TestPosix case - 11.
// TestPosix case with non-existent source volume.
// Expecting to fail with `errVolumeNotFound`.
{
srcVol: "src-vol-non-existent",
@@ -1363,8 +1363,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errVolumeNotFound,
},
// Test case - 12.
// Test case with non-existent destination volume.
// TestPosix case - 12.
// TestPosix case with non-existent destination volume.
// Expecting to fail with `errVolumeNotFound`.
{
srcVol: "src-vol",
@@ -1374,8 +1374,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errVolumeNotFound,
},
// Test case - 13.
// Test case with invalid src volume name. Length should be atleast 3.
// TestPosix case - 13.
// TestPosix case with invalid src volume name. Length should be atleast 3.
// Expecting to fail with `errInvalidArgument`.
{
srcVol: "ab",
@@ -1385,8 +1385,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errInvalidArgument,
},
// Test case - 14.
// Test case with invalid destination volume name. Length should be atleast 3.
// TestPosix case - 14.
// TestPosix case with invalid destination volume name. Length should be atleast 3.
// Expecting to fail with `errInvalidArgument`.
{
srcVol: "abcd",
@@ -1396,8 +1396,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errInvalidArgument,
},
// Test case - 15.
// Test case with invalid destination volume name. Length should be atleast 3.
// TestPosix case - 15.
// TestPosix case with invalid destination volume name. Length should be atleast 3.
// Expecting to fail with `errInvalidArgument`.
{
srcVol: "abcd",
@@ -1407,8 +1407,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errInvalidArgument,
},
// Test case - 16.
// Test case with the parent of the destination being a file.
// TestPosix case - 16.
// TestPosix case with the parent of the destination being a file.
// expected to fail with `errFileAccessDenied`.
{
srcVol: "src-vol",
@@ -1418,8 +1418,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileAccessDenied,
},
// Test case - 17.
// Test case with segment of source file name more than 255.
// TestPosix case - 17.
// TestPosix case with segment of source file name more than 255.
// expected not to fail.
{
srcVol: "src-vol",
@@ -1429,8 +1429,8 @@ func TestRenameFile(t *testing.T) {
ioErrCnt: 0,
expectedErr: errFileNameTooLong,
},
// Test case - 18.
// Test case with segment of destination file name more than 255.
// TestPosix case - 18.
// TestPosix case with segment of destination file name more than 255.
// expected not to fail.
{
srcVol: "src-vol",
@@ -1452,13 +1452,13 @@ func TestRenameFile(t *testing.T) {
}
if err := posixStorage.RenameFile(testCase.srcVol, testCase.srcPath, testCase.destVol, testCase.destPath); err != testCase.expectedErr {
t.Fatalf("Test %d: Expected the error to be : \"%v\", got: \"%v\".", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix %d: Expected the error to be : \"%v\", got: \"%v\".", i+1, testCase.expectedErr, err)
}
}
}
// Test posix.StatFile()
func TestStatFile(t *testing.T) {
// TestPosix posix.StatFile()
func TestPosixStatFile(t *testing.T) {
// create posix test setup
posixStorage, path, err := newPosixTestSetup()
if err != nil {
@@ -1485,56 +1485,56 @@ func TestStatFile(t *testing.T) {
ioErrCnt int
expectedErr error
}{
// Test case - 1.
// Test case with valid inputs, expected to pass.
// TestPosix case - 1.
// TestPosix case with valid inputs, expected to pass.
{
srcVol: "success-vol",
srcPath: "success-file",
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 2.
// Test case with valid inputs, expected to pass.
// TestPosix case - 2.
// TestPosix case with valid inputs, expected to pass.
{
srcVol: "success-vol",
srcPath: "path/to/success-file",
ioErrCnt: 0,
expectedErr: nil,
},
// Test case - 3.
// Test case with non-existent file.
// TestPosix case - 3.
// TestPosix case with non-existent file.
{
srcVol: "success-vol",
srcPath: "nonexistent-file",
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 4.
// Test case with non-existent file path.
// TestPosix case - 4.
// TestPosix case with non-existent file path.
{
srcVol: "success-vol",
srcPath: "path/2/success-file",
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 5.
// Test case with path being a directory.
// TestPosix case - 5.
// TestPosix case with path being a directory.
{
srcVol: "success-vol",
srcPath: "path",
ioErrCnt: 0,
expectedErr: errFileNotFound,
},
// Test case - 6.
// Test case with io error count > max limit.
// TestPosix case - 6.
// TestPosix case with io error count > max limit.
{
srcVol: "success-vol",
srcPath: "success-file",
ioErrCnt: 6,
expectedErr: errFaultyDisk,
},
// Test case - 7.
// Test case with non existent volume.
// TestPosix case - 7.
// TestPosix case with non existent volume.
{
srcVol: "non-existent-vol",
srcPath: "success-file",
@@ -1552,7 +1552,7 @@ func TestStatFile(t *testing.T) {
t.Errorf("Expected the StorageAPI to be of type *posix")
}
if _, err := posixStorage.StatFile(testCase.srcVol, testCase.srcPath); err != testCase.expectedErr {
t.Fatalf("Test case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
t.Fatalf("TestPosix case %d: Expected: \"%s\", got: \"%s\"", i+1, testCase.expectedErr, err)
}
}
}

View File

@@ -70,7 +70,8 @@ import (
type InitActions int
const (
// FormatDisks - see above table for disk states where it is applicable.
// FormatDisks - see above table for disk states where it
// is applicable.
FormatDisks InitActions = iota
// WaitForHeal - Wait for disks to heal.
@@ -82,10 +83,12 @@ const (
// WaitForAll - Wait for all disks to be online.
WaitForAll
// WaitForFormatting - Wait for formatting to be triggered from the '1st' server in the cluster.
// 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 - Wait for all servers to have the same config
// including (credentials, version and time).
WaitForConfig
// InitObjectLayer - Initialize object layer.
@@ -96,8 +99,8 @@ const (
Abort
)
// Quick error to actions converts looking for specific errors which need to
// be returned quickly and server should wait instead.
// Quick error to actions converts looking for specific errors
// which need to be returned quickly and server should wait instead.
func quickErrToActions(errMap map[error]int) InitActions {
var action InitActions
switch {
@@ -187,7 +190,7 @@ func printRetryMsg(sErrs []error, storageDisks []StorageAPI) {
// Implements a jitter backoff loop for formatting all disks during
// initialization of the server.
func retryFormattingDisks(firstDisk bool, endpoints []*url.URL, storageDisks []StorageAPI) error {
func retryFormattingXLDisks(firstDisk bool, endpoints []*url.URL, storageDisks []StorageAPI) error {
if len(endpoints) == 0 {
return errInvalidArgument
}
@@ -220,17 +223,6 @@ func retryFormattingDisks(firstDisk bool, endpoints []*url.URL, storageDisks []S
// for disks not being available.
printRetryMsg(sErrs, storageDisks)
}
if len(formatConfigs) == 1 {
err := genericFormatCheckFS(formatConfigs[0], sErrs[0])
if err != nil {
// For an new directory or existing data.
if err == errUnformattedDisk || err == errCorruptedFormat {
return initFormatFS(storageDisks[0])
}
return err
}
return nil
} // Check if this is a XL or distributed XL, anything > 1 is considered XL backend.
// 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
@@ -239,6 +231,7 @@ func retryFormattingDisks(firstDisk bool, endpoints []*url.URL, storageDisks []S
if err := checkFormatXLValues(formatConfigs); err != nil {
return err
}
// Check if this is a XL or distributed XL, anything > 1 is considered XL backend.
switch prepForInitXL(firstDisk, sErrs, len(storageDisks)) {
case Abort:
return errCorruptedFormat
@@ -300,7 +293,7 @@ func initStorageDisks(endpoints []*url.URL) ([]StorageAPI, error) {
}
// Format disks before initialization object layer.
func waitForFormatDisks(firstDisk bool, endpoints []*url.URL, storageDisks []StorageAPI) (formattedDisks []StorageAPI, err error) {
func waitForFormatXLDisks(firstDisk bool, endpoints []*url.URL, storageDisks []StorageAPI) (formattedDisks []StorageAPI, err error) {
if len(endpoints) == 0 {
return nil, errInvalidArgument
}
@@ -327,7 +320,7 @@ func waitForFormatDisks(firstDisk bool, endpoints []*url.URL, storageDisks []Sto
// Start retry loop retrying until disks are formatted properly, until we have reached
// a conditional quorum of formatted disks.
err = retryFormattingDisks(firstDisk, endpoints, retryDisks)
err = retryFormattingXLDisks(firstDisk, endpoints, retryDisks)
if err != nil {
return nil, err
}

View File

@@ -28,53 +28,6 @@ func newObjectLayerFn() ObjectLayer {
return globalObjectAPI
}
// newObjectLayer - initialize any object layer depending on the number of disks.
func newObjectLayer(storageDisks []StorageAPI) (ObjectLayer, error) {
var objAPI ObjectLayer
var err error
if len(storageDisks) == 1 {
// Initialize FS object layer.
objAPI, err = newFSObjects(storageDisks[0])
} else {
// Initialize XL object layer.
objAPI, err = newXLObjects(storageDisks)
}
if err != nil {
return nil, err
}
// The following actions are performed here, so that any
// requests coming in early in the bootup sequence don't fail
// unexpectedly - e.g. if initEventNotifier was initialized
// after this function completes, an event could be generated
// before the notification system is ready, causing event
// drops or crashes.
// Migrate bucket policy from configDir to .minio.sys/buckets/
err = migrateBucketPolicyConfig(objAPI)
if err != nil {
errorIf(err, "Unable to migrate bucket policy from config directory")
return nil, err
}
err = cleanupOldBucketPolicyConfigs()
if err != nil {
errorIf(err, "Unable to clean up bucket policy from config directory.")
return nil, err
}
// 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
}
// Composed function registering routers for only distributed XL setup.
func registerDistXLRouters(mux *router.Router, srvCmdConfig serverCmdConfig) error {
// Register storage rpc router only if its a distributed setup.

View File

@@ -84,9 +84,8 @@ EXAMPLES:
}
type serverCmdConfig struct {
serverAddr string
endpoints []*url.URL
storageDisks []StorageAPI
serverAddr string
endpoints []*url.URL
}
// Parse an array of end-points (from the command line)
@@ -248,6 +247,8 @@ func checkServerSyntax(c *cli.Context) {
// Verify syntax for all the XL disks.
disks := c.Args()
// Parse disks check if they comply with expected URI style.
endpoints, err := parseStorageEndpoints(disks)
fatalIf(err, "Unable to parse storage endpoints %s", strings.Join(disks, " "))
@@ -262,7 +263,7 @@ func checkServerSyntax(c *cli.Context) {
if len(endpoints) > 1 {
// Validate if we have sufficient disks for XL setup.
err = checkSufficientDisks(endpoints)
fatalIf(err, "Invalid number of disks supplied.")
fatalIf(err, "Insufficient number of disks.")
} else {
// Validate if we have invalid disk for FS setup.
if endpoints[0].Host != "" && endpoints[0].Scheme != "" {
@@ -360,7 +361,7 @@ func serverMain(c *cli.Context) {
// Initialization routine, such as config loading, enable logging, ..
minioInit(c)
// Check for minio updates from dl.minio.io
// Check for new updates from dl.minio.io.
checkUpdate()
// Server address.
@@ -371,10 +372,13 @@ func serverMain(c *cli.Context) {
fatalIf(err, "Unable to extract host and port %s", serverAddr)
// Check server syntax and exit in case of errors.
// Done after globalMinioHost and globalMinioPort is set as parseStorageEndpoints()
// depends on it.
// Done after globalMinioHost and globalMinioPort is set
// as parseStorageEndpoints() depends on it.
checkServerSyntax(c)
// Initialize server config.
initServerConfig(c)
// Disks to be used in server init.
endpoints, err := parseStorageEndpoints(c.Args())
fatalIf(err, "Unable to parse storage endpoints %s", c.Args())
@@ -391,39 +395,32 @@ func serverMain(c *cli.Context) {
// on all nodes.
sort.Sort(byHostPath(endpoints))
storageDisks, err := initStorageDisks(endpoints)
fatalIf(err, "Unable to initialize storage disk(s).")
// Cleanup objects that weren't successfully written into the namespace.
fatalIf(houseKeeping(storageDisks), "Unable to purge temporary files.")
// Initialize server config.
initServerConfig(c)
// First disk argument check if it is local.
firstDisk := isLocalStorage(endpoints[0])
// Configure server.
srvConfig := serverCmdConfig{
serverAddr: serverAddr,
endpoints: endpoints,
}
// Check if endpoints are part of distributed setup.
globalIsDistXL = isDistributedSetup(endpoints)
// Configure server.
srvConfig := serverCmdConfig{
serverAddr: serverAddr,
endpoints: endpoints,
storageDisks: storageDisks,
// Set nodes for dsync for distributed setup.
if globalIsDistXL {
fatalIf(initDsyncNodes(endpoints), "Unable to initialize distributed locking clients")
}
// Initialize name space lock.
initNSLock(globalIsDistXL)
// Configure server.
handler, err := configureServerHandler(srvConfig)
fatalIf(err, "Unable to configure one of server's RPC services.")
// Set nodes for dsync for distributed setup.
if globalIsDistXL {
fatalIf(initDsyncNodes(endpoints), "Unable to initialize distributed locking")
}
// Initialize S3 Peers inter-node communication only in distributed setup.
initGlobalS3Peers(endpoints)
// Initialize name space lock.
initNSLock(globalIsDistXL)
// Initialize Admin Peers inter-node communication only in distributed setup.
initGlobalAdminPeers(endpoints)
// Initialize a new HTTP server.
apiServer := NewServerMux(serverAddr, handler)
@@ -438,12 +435,6 @@ func serverMain(c *cli.Context) {
// Set the global API endpoints value.
globalAPIEndpoints = apiEndPoints
// Initialize S3 Peers inter-node communication
initGlobalS3Peers(endpoints)
// Initialize Admin Peers inter-node communication
initGlobalAdminPeers(endpoints)
// Start server, automatically configures TLS if certs are available.
go func() {
cert, key := "", ""
@@ -453,13 +444,8 @@ func serverMain(c *cli.Context) {
fatalIf(apiServer.ListenAndServe(cert, key), "Failed to start minio server.")
}()
// Wait for formatting of disks.
formattedDisks, err := waitForFormatDisks(firstDisk, endpoints, storageDisks)
fatalIf(err, "formatting storage disks failed")
// Once formatted, initialize object layer.
newObject, err := newObjectLayer(formattedDisks)
fatalIf(err, "intializing object layer failed")
newObject, err := newObjectLayer(srvConfig)
fatalIf(err, "Initializing object layer failed")
globalObjLayerMutex.Lock()
globalObjectAPI = newObject
@@ -471,3 +457,57 @@ func serverMain(c *cli.Context) {
// Waits on the server.
<-globalServiceDoneCh
}
// Initialize object layer with the supplied disks, objectLayer is nil upon any error.
func newObjectLayer(srvCmdCfg serverCmdConfig) (newObject ObjectLayer, err error) {
// For FS only, directly use the disk.
isFS := len(srvCmdCfg.endpoints) == 1
if isFS {
// Unescape is needed for some UNC paths on windows
// which are of this form \\127.0.0.1\\export\test.
var fsPath string
fsPath, err = url.QueryUnescape(srvCmdCfg.endpoints[0].String())
if err != nil {
return nil, err
}
// Initialize new FS object layer.
newObject, err = newFSObjectLayer(fsPath)
if err != nil {
return nil, err
}
// FS initialized, return.
return newObject, nil
}
// First disk argument check if it is local.
firstDisk := isLocalStorage(srvCmdCfg.endpoints[0])
// Initialize storage disks.
storageDisks, err := initStorageDisks(srvCmdCfg.endpoints)
if err != nil {
return nil, err
}
// Wait for formatting disks for XL backend.
var formattedDisks []StorageAPI
formattedDisks, err = waitForFormatXLDisks(firstDisk, srvCmdCfg.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
}

View File

@@ -21,6 +21,7 @@ import (
"flag"
"net/http"
"os"
"reflect"
"runtime"
"testing"
@@ -222,6 +223,63 @@ func TestCheckSufficientDisks(t *testing.T) {
}
}
// Tests initializing new object layer.
func TestNewObjectLayer(t *testing.T) {
// Tests for FS object layer.
nDisks := 1
disks, err := getRandomDisks(nDisks)
if err != nil {
t.Fatal("Failed to create disks for the backend")
}
defer removeRoots(disks)
endpoints, err := parseStorageEndpoints(disks)
if err != nil {
t.Fatal("Unexpected parse error", err)
}
obj, err := newObjectLayer(serverCmdConfig{
serverAddr: ":9000",
endpoints: endpoints,
})
if err != nil {
t.Fatal("Unexpected object layer initialization error", err)
}
_, ok := obj.(*fsObjects)
if !ok {
t.Fatal("Unexpected object layer detected", reflect.TypeOf(obj))
}
// Tests for XL object layer initialization.
// Create temporary backend for the test server.
nDisks = 16
disks, err = getRandomDisks(nDisks)
if err != nil {
t.Fatal("Failed to create disks for the backend")
}
defer removeRoots(disks)
endpoints, err = parseStorageEndpoints(disks)
if err != nil {
t.Fatal("Unexpected parse error", err)
}
obj, err = newObjectLayer(serverCmdConfig{
serverAddr: ":9000",
endpoints: endpoints,
})
if err != nil {
t.Fatal("Unexpected object layer initialization error", err)
}
_, ok = obj.(*xlObjects)
if !ok {
t.Fatal("Unexpected object layer detected", reflect.TypeOf(obj))
}
}
// Tests parsing various types of input endpoints and paths.
func TestParseStorageEndpoints(t *testing.T) {
testCases := []struct {
globalMinioHost string

View File

@@ -109,18 +109,20 @@ func dial(addr string) error {
// Tests initializing listeners.
func TestInitListeners(t *testing.T) {
portTest1 := getFreePort()
portTest2 := getFreePort()
testCases := []struct {
serverAddr string
shouldPass bool
}{
// Test 1 with ip and port.
{
serverAddr: "127.0.0.1:" + getFreePort(),
serverAddr: "127.0.0.1:" + portTest1,
shouldPass: true,
},
// Test 2 only port.
{
serverAddr: ":" + getFreePort(),
serverAddr: ":" + portTest2,
shouldPass: true,
},
// Test 3 with no port error.

View File

@@ -1145,8 +1145,8 @@ func (s *TestSuiteCommon) TestSHA256Mismatch(c *C) {
}
}
// TestNotBeAbleToCreateObjectInNonexistentBucket - Validates the error response
// on an attempt to upload an object into a non-existent bucket.
// TestPutObjectLongName - Validates the error response
// on an attempt to upload an object with long name.
func (s *TestSuiteCommon) TestPutObjectLongName(c *C) {
// generate a random bucket name.
bucketName := getRandomBucketName()

View File

@@ -80,7 +80,7 @@ func TestDoesPresignedV2SignatureMatch(t *testing.T) {
// (5) Should error when the signature does not match.
{
queryParams: map[string]string{
"Expires": fmt.Sprintf("%d", now.Unix()),
"Expires": fmt.Sprintf("%d", now.Unix()+60),
"Signature": "zOM2YrY/yAQe15VWmT78OlBrK6g=",
"AWSAccessKeyId": serverConfig.GetCredential().AccessKey,
},

View File

@@ -106,13 +106,6 @@ func getURLEncodedName(name string) string {
return encodedName
}
func findHost(signedHeaders []string) APIErrorCode {
if contains(signedHeaders, "host") {
return ErrNone
}
return ErrUnsignedHeaders
}
// extractSignedHeaders extract signed headers from Authorization header
func extractSignedHeaders(signedHeaders []string, reqHeaders http.Header) (http.Header, APIErrorCode) {
// find whether "host" is part of list of signed headers.

View File

@@ -194,25 +194,6 @@ func TestExtractSignedHeaders(t *testing.T) {
}
}
// TestFindHost - tests the logic to find whether "host" is part of signed headers.
func TestFindHost(t *testing.T) {
// doesn't contain "host".
signedHeaders := []string{"x-amz-content-sha256", "x-amz-date"}
errCode := findHost(signedHeaders)
// expected to error out with code ErrUnsignedHeaders .
if errCode != ErrUnsignedHeaders {
t.Fatalf("Expected the APIErrorCode to be %d, but got %d", ErrUnsignedHeaders, errCode)
}
// adding "host".
signedHeaders = append(signedHeaders, "host")
// epxected to pass.
errCode = findHost(signedHeaders)
if errCode != ErrNone {
t.Fatalf("Expected the APIErrorCode to be %d, but got %d", ErrNone, errCode)
}
}
// TestSignV4TrimAll - tests the logic of TrimAll() function
func TestSignV4TrimAll(t *testing.T) {
testCases := []struct {

View File

@@ -351,6 +351,9 @@ func (s *TestRPCStorageSuite) testRPCStorageListDir(t *testing.T) {
}
}
dirs, err := storageDisk.ListDir("myvol", "")
if err != nil {
t.Error(err)
}
if len(dirs) != dirCount {
t.Errorf("Expected %d directories but found only %d", dirCount, len(dirs))
}
@@ -361,6 +364,9 @@ func (s *TestRPCStorageSuite) testRPCStorageListDir(t *testing.T) {
}
}
dirs, err = storageDisk.ListDir("myvol", "")
if err != nil {
t.Error(err)
}
if len(dirs) != 0 {
t.Errorf("Expected no directories but found %d", dirCount)
}
@@ -370,6 +376,9 @@ func (s *TestRPCStorageSuite) testRPCStorageListDir(t *testing.T) {
t.Error("Unable to initiate DeleteVol", err)
}
vols, err := storageDisk.ListVols()
if err != nil {
t.Error(err)
}
if len(vols) != 0 {
t.Errorf("Expected no volumes but found %d", dirCount)
}

View File

@@ -70,7 +70,8 @@ func init() {
}
func prepareFS() (ObjectLayer, string, error) {
fsDirs, err := getRandomDisks(1)
nDisks := 1
fsDirs, err := getRandomDisks(nDisks)
if err != nil {
return nil, "", err
}
@@ -78,12 +79,15 @@ func prepareFS() (ObjectLayer, string, error) {
if err != nil {
return nil, "", err
}
obj, _, err := initObjectLayer(endpoints)
fsPath, err := url.QueryUnescape(endpoints[0].String())
if err != nil {
removeRoots(fsDirs)
return nil, "", err
}
return obj, fsDirs[0], nil
obj, err := newFSObjectLayer(fsPath)
if err != nil {
return nil, "", err
}
return obj, endpoints[0].Path, nil
}
func prepareXL() (ObjectLayer, []string, error) {
@@ -104,6 +108,17 @@ func prepareXL() (ObjectLayer, []string, error) {
return obj, fsDirs, nil
}
// Initialize FS objects.
func initFSObjects(disk string, t *testing.T) (obj ObjectLayer) {
newTestConfig("us-east-1")
var err error
obj, err = newFSObjectLayer(disk)
if err != nil {
t.Fatal(err)
}
return obj
}
// TestErrHandler - Golang Testing.T and Testing.B, and gocheck.C satisfy this interface.
// This makes it easy to run the TestServer from any of the tests.
// Using this interface, functionalities to be used in tests can be made generalized, and can be integrated in benchmarks/unit tests/go check suite tests.
@@ -118,6 +133,7 @@ type TestErrHandler interface {
const (
// FSTestStr is the string which is used as notation for Single node ObjectLayer in the unit tests.
FSTestStr string = "FS"
// XLTestStr is the string which is used as notation for XL ObjectLayer in the unit tests.
XLTestStr string = "XL"
)
@@ -204,15 +220,15 @@ func UnstartedTestServer(t TestErrHandler, instanceType string) TestServer {
testServer.AccessKey = credentials.AccessKey
testServer.SecretKey = credentials.SecretKey
objLayer, storageDisks, err := initObjectLayer(testServer.Disks)
objLayer, _, err := initObjectLayer(testServer.Disks)
if err != nil {
t.Fatalf("Failed obtaining Temp Backend: <ERROR> %s", err)
}
srvCmdCfg := serverCmdConfig{
endpoints: testServer.Disks,
storageDisks: storageDisks,
endpoints: testServer.Disks,
}
httpHandler, err := configureServerHandler(
srvCmdCfg,
)
@@ -338,7 +354,7 @@ func initTestStorageRPCEndPoint(srvCmdConfig serverCmdConfig) http.Handler {
return muxRouter
}
// StartTestStorageRPCServer - Creates a temp XL/FS backend and initializes storage RPC end points,
// StartTestStorageRPCServer - Creates a temp XL backend and initializes storage RPC end points,
// then starts a test server with those storage RPC end points registered.
func StartTestStorageRPCServer(t TestErrHandler, instanceType string, diskN int) TestServer {
// create temporary backend for the test server.
@@ -402,7 +418,7 @@ func StartTestPeersRPCServer(t TestErrHandler, instanceType string) TestServer {
testRPCServer.SecretKey = credentials.SecretKey
// create temporary backend for the test server.
objLayer, storageDisks, err := initObjectLayer(endpoints)
objLayer, _, err := initObjectLayer(endpoints)
if err != nil {
t.Fatalf("Failed obtaining Temp Backend: <ERROR> %s", err)
}
@@ -413,8 +429,7 @@ func StartTestPeersRPCServer(t TestErrHandler, instanceType string) TestServer {
globalObjLayerMutex.Unlock()
srvCfg := serverCmdConfig{
endpoints: endpoints,
storageDisks: storageDisks,
endpoints: endpoints,
}
mux := router.NewRouter()
@@ -1620,12 +1635,12 @@ func initObjectLayer(endpoints []*url.URL) (ObjectLayer, []StorageAPI, error) {
return nil, nil, err
}
formattedDisks, err := waitForFormatDisks(true, endpoints, storageDisks)
formattedDisks, err := waitForFormatXLDisks(true, endpoints, storageDisks)
if err != nil {
return nil, nil, err
}
objLayer, err := newObjectLayer(formattedDisks)
objLayer, err := newXLObjectLayer(formattedDisks)
if err != nil {
return nil, nil, err
}
@@ -1722,7 +1737,7 @@ func initAPIHandlerTest(obj ObjectLayer, endpoints []string) (bucketName string,
// failed to create newbucket, return err.
return "", nil, err
}
// Register the API end points with XL/FS object layer.
// Register the API end points with XL object layer.
// Registering only the GetObject handler.
apiRouter = initTestAPIEndPoints(obj, endpoints)
return bucketName, apiRouter, nil
@@ -1928,7 +1943,6 @@ func ExecObjectLayerAPITest(t *testing.T, objAPITest objAPITestType, endpoints [
if err != nil {
t.Fatalf("Initialzation of API handler tests failed: <ERROR> %s", err)
}
credentials = serverConfig.GetCredential()
// Executing the object layer tests for XL.
objAPITest(objLayer, XLTestStr, bucketXL, xlAPIRouter, credentials, t)
// clean up the temporary test backend.
@@ -2118,7 +2132,7 @@ func registerAPIFunctions(muxRouter *router.Router, objLayer ObjectLayer, apiFun
registerBucketLevelFunc(bucketRouter, api, apiFunctions...)
}
// Takes in XL/FS object layer, and the list of API end points to be tested/required, registers the API end points and returns the HTTP handler.
// Takes in XL object layer, and the list of API end points to be tested/required, registers the API end points and returns the HTTP handler.
// Need isolated registration of API end points while writing unit tests for end points.
// All the API end points are registered only for the default case.
func initTestAPIEndPoints(objLayer ObjectLayer, apiFunctions []string) http.Handler {

View File

@@ -194,29 +194,6 @@ func contains(stringList []string, element string) bool {
return false
}
// Contains endpoint returns true if endpoint found in the list of input endpoints.
func containsEndpoint(endpoints []*url.URL, endpoint *url.URL) bool {
for _, ep := range endpoints {
if *ep == *endpoint {
return true
}
}
return false
}
// urlPathSplit - split url path into bucket and object components.
func urlPathSplit(urlPath string) (bucketName, prefixName string) {
if urlPath == "" {
return urlPath, ""
}
urlPath = strings.TrimPrefix(urlPath, "/")
i := strings.Index(urlPath, "/")
if i != -1 {
return urlPath[:i], urlPath[i+1:]
}
return urlPath, ""
}
// Starts a profiler returns nil if profiler is not enabled, caller needs to handle this.
func startProfiler(profiler string) interface {
Stop()

View File

@@ -126,48 +126,6 @@ func TestMaxObjectSize(t *testing.T) {
}
}
// Test urlPathSplit.
func TestURLPathSplit(t *testing.T) {
type test struct {
urlPath string
bucketName string
prefixName string
}
testCases := []test{
{
urlPath: "/b/c/",
bucketName: "b",
prefixName: "c/",
},
{
urlPath: "c/aa",
bucketName: "c",
prefixName: "aa",
},
{
urlPath: "",
bucketName: "",
prefixName: "",
},
{
urlPath: "/b",
bucketName: "b",
prefixName: "",
},
}
for i, testCase := range testCases {
bucketName, prefixName := urlPathSplit(testCase.urlPath)
if bucketName != testCase.bucketName {
t.Errorf("Tets %d: Expected %s, %s", i+1, testCase.bucketName, bucketName)
}
if prefixName != testCase.prefixName {
t.Errorf("Tets %d: Expected %s, %s", i+1, testCase.bucketName, bucketName)
}
}
}
// Tests minimum allowed part size.
func TestMinAllowedPartSize(t *testing.T) {
sizes := []struct {

View File

@@ -461,7 +461,7 @@ func testListObjectsWebHandler(obj ObjectLayer, instanceType string, t TestErrHa
verifyReply(reply)
// Unauthenticated ListObjects should fail.
err, reply = test("")
err, _ = test("")
if err == nil {
t.Fatalf("Expected error `%s`", err)
}
@@ -870,7 +870,7 @@ func testDownloadWebHandler(obj ObjectLayer, instanceType string, t TestErrHandl
}
// Unauthenticated download should fail.
code, bodyContent = test("")
code, _ = test("")
if code != http.StatusForbidden {
t.Fatalf("Expected the response status to be 403, but instead found `%d`", code)
}

View File

@@ -137,7 +137,7 @@ func healBucketMetadata(storageDisks []StorageAPI, bucket string, readQuorum int
}
// Heal `policy.json` for missing entries, ignores if `policy.json` is not found.
policyPath := pathJoin(bucketConfigPrefix, bucket, policyJSON)
policyPath := pathJoin(bucketConfigPrefix, bucket, bucketPolicyConfig)
if err := healBucketMetaFn(policyPath); err != nil {
return err
}

View File

@@ -505,9 +505,9 @@ func (xl xlObjects) PutObject(bucket string, object string, size int64, data io.
onlineDisks := getOrderedDisks(xlMeta.Erasure.Distribution, xl.storageDisks)
// Delete temporary object in the event of failure. If
// PutObject succeeded there would be no temporary object to
// delete.
// Delete temporary object in the event of failure.
// If PutObject succeeded there would be no temporary
// object to delete.
defer xl.deleteObject(minioMetaTmpBucket, tempObj)
if size > 0 {

View File

@@ -76,6 +76,24 @@ type xlObjects struct {
// 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 {

View File

@@ -132,7 +132,7 @@ func TestNewXL(t *testing.T) {
var erasureDisks []string
for i := 0; i < nDisks; i++ {
// Do not attempt to create this path, the test validates
// so that newFSObjects initializes non existing paths
// so that newXLObjects initializes non existing paths
// and successfully returns initialized object layer.
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
erasureDisks = append(erasureDisks, disk)
@@ -155,18 +155,18 @@ func TestNewXL(t *testing.T) {
t.Fatal("Unexpected error: ", err)
}
_, err = waitForFormatDisks(true, endpoints, nil)
_, err = waitForFormatXLDisks(true, endpoints, nil)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
_, err = waitForFormatDisks(true, nil, storageDisks)
_, err = waitForFormatXLDisks(true, nil, storageDisks)
if err != errInvalidArgument {
t.Fatalf("Expecting error, got %s", err)
}
// Initializes all erasure disks
formattedDisks, err := waitForFormatDisks(true, endpoints, storageDisks)
formattedDisks, err := waitForFormatXLDisks(true, endpoints, storageDisks)
if err != nil {
t.Fatalf("Unable to format disks for erasure, %s", err)
}