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

View file

@ -22,12 +22,12 @@ We found the following APIs to be redundant or less useful outside of AWS. If yo
|:---|:---|
|Maximum number of buckets| no-limit|
|Maximum number of objects per bucket| no-limit|
|Maximum object size| 5 TB|
|Maximum object size| 5 TiB|
|Minimum object size| 0 B|
|Maximum object size per PUT operation| 5 GB|
|Maximum object size per PUT operation| 5 GiB|
|Maximum number of parts per upload| 10,000|
|Part size|5 MB to 5 GB. Last part can be 0 B to 5 GB|
|Maximum number of parts returned per list parts request| 1000|
|Part size|5 MiB to 5 GiB. Last part can be 0 B to 5 GiB|
|Maximum number of parts returned per list parts request| 1000|
|Maximum number of objects returned per list objects request| 1000|
|Maximum number of multipart uploads returned per list multipart uploads request| 1000|

View file

@ -0,0 +1,137 @@
Introduction [![Slack](https://slack.minio.io/slack?type=svg)](https://slack.minio.io)
------------
This feature allows Minio to serve a shared NAS drive across multiple Minio instances. There are no special configuration changes required to enable this feature. Access to files stored on NAS volume are locked and synchronized by default.
Motivation
----------
Since Minio instances serve the purpose of a single tenant there is an increasing requirement where users want to run multiple Minio instances on a same backend which is managed by an existing NAS (NFS, GlusterFS, Other distributed filesystems) rather than a local disk. This feature is implemented also with minimal disruption in mind for the user and overall UI.
Restrictions
------------
* A PutObject() is blocked and waits if another GetObject() is in progress.
* A CompleteMultipartUpload() is blocked and waits if another PutObject() or GetObject() is in progress.
* Cannot run FS mode as a remote disk RPC.
## How To Run?
Running Minio instances on shared backend is no different than running on a stand-alone disk. There are no special configuration changes required to enable this feature. Access to files stored on NAS volume are locked and synchronized by default. Following examples will clarify this further for each operating system of your choice:
### Ubuntu 16.04 LTS
Example 1: Start Minio instance on a shared backend mounted and available at `/mnt/nfs`.
On linux server1
```shell
minio server /mnt/nfs
```
On linux server2
```shell
minio server /mnt/nfs
```
### Windows 2012 Server
Example 1: Start Minio instance on a shared backend mounted and available at `\\remote-server\cifs`.
On windows server1
```cmd
minio.exe server \\remote-server\cifs\export
```
On windows server2
```cmd
minio.exe server \\remote-server\cifs\export
```
Alternatively if `\\remote-server\cifs` is mounted as `D:\` drive.
On windows server1
```cmd
minio.exe server D:\export
```
On windows server2
```cmd
minio.exe server D:\export
```
Architecture
------------------
## POSIX/Win32 Locks
### Lock process
With in the same Minio instance locking is handled by existing in-memory namespace locks (**sync.RWMutex** et. al). To synchronize locks between many Minio instances we leverage POSIX `fcntl()` locks on Unixes and on Windows `LockFileEx()` Win32 API. Requesting write lock block if there are any read locks held by neighboring Minio instance on the same path. So does the read lock if there are any active write locks in-progress.
### Unlock process
Unlocking happens on filesystems locks by just closing the file descriptor (fd) which was initially requested for lock operation. Closing the fd tells the kernel to relinquish all the locks held on the path by the current process. This gets trickier when there are many readers on the same path by the same process, it would mean that closing an fd relinquishes locks for all concurrent readers as well. To properly manage this situation a simple fd reference count is implemented, the same fd is shared between many readers. When readers start closing on the fd we start reducing the reference count, once reference count has reached zero we can be sure that there are no more readers active. So we proceed and close the underlying file descriptor which would relinquish the read lock held on the path.
This doesn't apply for the writes because there is always one writer and many readers for any unique object.
## Handling Concurrency.
An example here shows how the contention is handled with GetObject().
GetObject() holds a read lock on `fs.json`.
```go
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, object, fsMetaJSONFile)
rlk, err := fs.rwPool.Open(fsMetaPath)
if err != nil {
return toObjectErr(traceError(err), bucket, object)
}
defer rlk.Close()
... you can perform other operations here ...
_, err = io.CopyBuffer(writer, reader, buf)
... after successful copy operation unlocks the read lock ...
```
A concurrent PutObject is requested on the same object, PutObject() attempts a write lock on `fs.json`.
```go
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()
```
Now from the above snippet the following code one can notice that until the GetObject() returns writing to the client. Following portion of the code will block.
```go
wlk, err := fs.rwPool.Create(fsMetaPath)
```
This restriction is needed so that corrupted data is not returned to the client in between I/O. The logic works vice-versa as well an on-going PutObject(), GetObject() would wait for the PutObject() to complete.
### Caveats (concurrency)
Consider for example 3 servers sharing the same backend
On minio1
- DeleteObject(object1) --> lock acquired on `fs.json` while object1 is being deleted.
On minio2
- PutObject(object1) --> lock waiting until DeleteObject finishes.
On minio3
- PutObject(object1) --> (concurrent request during PutObject minio2 checking if `fs.json` exists)
Once lock is acquired the minio2 validates if the file really exists to avoid obtaining lock on an fd which is already deleted. But this situation calls for a race with a third server which is also attempting to write the same file before the minio2 can validate if the file exists. It might be potentially possible `fs.json` is created so the lock acquired by minio2 might be invalid and can lead to a potential inconsistency.
This is a known problem and cannot be solved by POSIX fcntl locks. These are considered to be the limits of shared filesystem.

View file

@ -0,0 +1,92 @@
# Shared Backend Minio Quickstart Guide [![Slack](https://slack.minio.io/slack?type=svg)](https://slack.minio.io) [![Go Report Card](https://goreportcard.com/badge/minio/minio)](https://goreportcard.com/report/minio/minio) [![Docker Pulls](https://img.shields.io/docker/pulls/minio/minio.svg?maxAge=604800)](https://hub.docker.com/r/minio/minio/) [![codecov](https://codecov.io/gh/minio/minio/branch/master/graph/badge.svg)](https://codecov.io/gh/minio/minio)
Minio now supports shared backend across multiple instances. This solves certain specific use cases.
## Use Cases
- Minio on NAS
- Minio on Distributed Filesystems
- Multi-user Shared Backend.
## Why Minio On Shared Backend?
This feature allows Minio to serve a shared NAS drive across multiple Minio instances. There are no special configuration changes required to enable this feature. Access to files stored on NAS volume are locked and synchronized by default.
# Get started
If you're aware of stand-alone Minio set up, the installation and running remains the same.
## 1. Prerequisites
Install Minio - [Minio Quickstart Guide](https://docs.minio.io/docs/minio).
## 2. Run Minio On Shared Backend
Below examples will clarify further for each operating system of your choice:
### Ubuntu 16.04 LTS
Run the following commands on all the object storage gateway servers where your NAS volume is accessible. By explicitly passing access and secret keys through the environment variable you make sure that all the gateway servers share the same key across.
Example 1: Start Minio instance on a shared backend mounted and available at `/mnt/nfs`.
On linux server1
```sh
minio server /mnt/nfs
```
On linux server2
```sh
minio server /mnt/nfs
```
### Windows 2012 Server
Run the following commands on all the object storage gateway servers where your NAS volume is accessible. By explicitly passing access and secret keys through the environment variable you make sure that all the gateway servers share the same key across.
Example 1: Start Minio instance on a shared backend mounted and available at `\\remote-server\smb`.
On windows server1
```cmd
set MINIO_ACCESS_KEY=my-username
set MINIO_SECRET_KEY=my-password
minio.exe server \\remote-server\smb\export
```
On windows server2
```cmd
set MINIO_ACCESS_KEY=my-username
set MINIO_SECRET_KEY=my-password
minio.exe server \\remote-server\smb\export
```
Alternatively if `\\remote-server\smb` is mounted as `M:\` drive.
On windows server1
```cmd
set MINIO_ACCESS_KEY=my-username
set MINIO_SECRET_KEY=my-password
net use m: \\remote-server\smb\export /P:Yes
minio.exe server M:\export
```
On windows server2
```cmd
set MINIO_ACCESS_KEY=my-username
set MINIO_SECRET_KEY=my-password
net use m: \\remote-server\smb\export /P:Yes
minio.exe server M:\export
```
## 3. Test your setup
To test this setup, access the Minio server via browser or [`mc`](https://docs.minio.io/docs/minio-client-quickstart-guide). Youll see the uploaded files are accessible from the node2 endpoint as well.
## Explore Further
- [Use `mc` with Minio Server](https://docs.minio.io/docs/minio-client-quickstart-guide)
- [Use `aws-cli` with Minio Server](https://docs.minio.io/docs/aws-cli-with-minio)
- [Use `s3cmd` with Minio Server](https://docs.minio.io/docs/s3cmd-with-minio)
- [Use `minio-go` SDK with Minio Server](https://docs.minio.io/docs/golang-client-quickstart-guide)
- [The Minio documentation website](https://docs.minio.io)

102
pkg/lock/lock.go Normal file
View file

@ -0,0 +1,102 @@
/*
* 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 lock - implements filesystem locking wrappers around an
// open file descriptor.
package lock
import (
"os"
"sync"
)
// RLockedFile represents a read locked file, implements a special
// closer which only closes the associated *os.File when the ref count.
// has reached zero, i.e when all the readers have given up their locks.
type RLockedFile struct {
*LockedFile
mutex sync.Mutex
refs int // Holds read lock refs.
}
// IsClosed - Check if the rlocked file is already closed.
func (r *RLockedFile) IsClosed() bool {
r.mutex.Lock()
defer r.mutex.Unlock()
return r.refs == 0
}
// IncLockRef - is used by called to indicate lock refs.
func (r *RLockedFile) IncLockRef() {
r.mutex.Lock()
r.refs++
r.mutex.Unlock()
}
// Close - this closer implements a special closer
// closes the underlying fd only when the refs
// reach zero.
func (r *RLockedFile) Close() (err error) {
r.mutex.Lock()
defer r.mutex.Unlock()
if r.refs == 0 {
return os.ErrInvalid
}
r.refs--
if r.refs == 0 {
err = r.File.Close()
}
return err
}
// Provides a new initialized read locked struct from *os.File
func newRLockedFile(lkFile *LockedFile) (*RLockedFile, error) {
if lkFile == nil {
return nil, os.ErrInvalid
}
return &RLockedFile{
LockedFile: lkFile,
refs: 1,
}, nil
}
// RLockedOpenFile - returns a wrapped read locked file, if the file
// doesn't exist at path returns an error.
func RLockedOpenFile(path string) (*RLockedFile, error) {
lkFile, err := LockedOpenFile(path, os.O_RDONLY, 0666)
if err != nil {
return nil, err
}
return newRLockedFile(lkFile)
}
// LockedFile represents a locked file, implements a helper
// method Size(), represents the size of the underlying object.
type LockedFile struct {
*os.File
size int64
}
// Size - size of the underlying locked file.
func (l *LockedFile) Size() int64 {
return l.size
}

75
pkg/lock/lock_nix.go Normal file
View file

@ -0,0 +1,75 @@
// +build !windows,!plan9,!solaris
/*
* 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 lock
import (
"fmt"
"os"
"syscall"
)
// LockedOpenFile - initializes a new lock and protects
// the file from concurrent access across mount points.
// This implementation doesn't support all the open
// flags and shouldn't be considered as replacement
// for os.OpenFile().
func LockedOpenFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
var lockType int
switch flag {
case syscall.O_RDONLY:
lockType = syscall.LOCK_SH
case syscall.O_WRONLY:
fallthrough
case syscall.O_RDWR:
fallthrough
case syscall.O_WRONLY | syscall.O_CREAT:
fallthrough
case syscall.O_RDWR | syscall.O_CREAT:
lockType = syscall.LOCK_EX
default:
return nil, fmt.Errorf("Unsupported flag (%d)", flag)
}
f, err := os.OpenFile(path, flag|syscall.O_SYNC, perm)
if err != nil {
return nil, err
}
if err = syscall.Flock(int(f.Fd()), lockType); err != nil {
f.Close()
return nil, err
}
st, err := os.Stat(path)
if err != nil {
f.Close()
return nil, err
}
if st.IsDir() {
f.Close()
return nil, &os.PathError{
Op: "open",
Path: path,
Err: syscall.EISDIR,
}
}
return &LockedFile{File: f, size: st.Size()}, nil
}

192
pkg/lock/lock_test.go Normal file
View file

@ -0,0 +1,192 @@
/*
* 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 lock
import (
"io/ioutil"
"os"
"testing"
"time"
)
// Test lock fails.
func TestLockFail(t *testing.T) {
f, err := ioutil.TempFile("", "lock")
if err != nil {
t.Fatal(err)
}
f.Close()
defer func() {
err = os.Remove(f.Name())
if err != nil {
t.Fatal(err)
}
}()
_, err = LockedOpenFile(f.Name(), os.O_APPEND, 0600)
if err == nil {
t.Fatal("Should fail here")
}
}
// Tests lock directory fail.
func TestLockDirFail(t *testing.T) {
d, err := ioutil.TempDir("", "lockDir")
if err != nil {
t.Fatal(err)
}
defer func() {
err = os.Remove(d)
if err != nil {
t.Fatal(err)
}
}()
_, err = LockedOpenFile(d, os.O_APPEND, 0600)
if err == nil {
t.Fatal("Should fail here")
}
}
// Tests rwlock methods.
func TestRWLockedFile(t *testing.T) {
f, err := ioutil.TempFile("", "lock")
if err != nil {
t.Fatal(err)
}
f.Close()
defer func() {
err = os.Remove(f.Name())
if err != nil {
t.Fatal(err)
}
}()
rlk, err := RLockedOpenFile(f.Name())
if err != nil {
t.Fatal(err)
}
if rlk.Size() != 0 {
t.Fatal("File size should be zero", rlk.Size())
}
isClosed := rlk.IsClosed()
if isClosed {
t.Fatal("File ref count shouldn't be zero")
}
// Increase reference count to 2.
rlk.IncLockRef()
isClosed = rlk.IsClosed()
if isClosed {
t.Fatal("File ref count shouldn't be zero")
}
// Decrease reference count by 1.
if err = rlk.Close(); err != nil {
t.Fatal(err)
}
isClosed = rlk.IsClosed()
if isClosed {
t.Fatal("File ref count shouldn't be zero")
}
// Decrease reference count by 1.
if err = rlk.Close(); err != nil {
t.Fatal(err)
}
// Now file should be closed.
isClosed = rlk.IsClosed()
if !isClosed {
t.Fatal("File ref count should be zero")
}
// Closing a file again should result in invalid argument.
if err = rlk.Close(); err != os.ErrInvalid {
t.Fatal(err)
}
_, err = newRLockedFile(nil)
if err != os.ErrInvalid {
t.Fatal("Unexpected error", err)
}
}
// Tests lock and unlock semantics.
func TestLockAndUnlock(t *testing.T) {
f, err := ioutil.TempFile("", "lock")
if err != nil {
t.Fatal(err)
}
f.Close()
defer func() {
err = os.Remove(f.Name())
if err != nil {
t.Fatal(err)
}
}()
// lock the file
l, err := LockedOpenFile(f.Name(), os.O_WRONLY, 0600)
if err != nil {
t.Fatal(err)
}
// unlock the file
if err = l.Close(); err != nil {
t.Fatal(err)
}
// try lock the unlocked file
dupl, err := LockedOpenFile(f.Name(), os.O_WRONLY|os.O_CREATE, 0600)
if err != nil {
t.Errorf("err = %v, want %v", err, nil)
}
// blocking on locked file
locked := make(chan struct{}, 1)
go func() {
bl, blerr := LockedOpenFile(f.Name(), os.O_WRONLY, 0600)
if blerr != nil {
t.Fatal(blerr)
}
locked <- struct{}{}
if blerr = bl.Close(); blerr != nil {
t.Fatal(blerr)
}
}()
select {
case <-locked:
t.Error("unexpected unblocking")
case <-time.After(100 * time.Millisecond):
}
// unlock
if err = dupl.Close(); err != nil {
t.Fatal(err)
}
// the previously blocked routine should be unblocked
select {
case <-locked:
case <-time.After(1 * time.Second):
t.Error("unexpected blocking")
}
}

172
pkg/lock/lock_windows.go Normal file
View file

@ -0,0 +1,172 @@
// +build windows
/*
* 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 lock
import (
"errors"
"os"
"syscall"
"unsafe"
)
var (
modkernel32 = syscall.NewLazyDLL("kernel32.dll")
procLockFileEx = modkernel32.NewProc("LockFileEx")
errLocked = errors.New("The process cannot access the file because another process has locked a portion of the file.")
)
const (
// see https://msdn.microsoft.com/en-us/library/windows/desktop/ms681382(v=vs.85).aspx
errLockViolation syscall.Errno = 0x21
)
// LockedOpenFile - initializes a new lock and protects
// the file from concurrent access.
func LockedOpenFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
f, err := open(path, flag, perm)
if err != nil {
return nil, err
}
if err = lockFile(syscall.Handle(f.Fd()), 0); err != nil {
f.Close()
return nil, err
}
st, err := os.Stat(path)
if err != nil {
f.Close()
return nil, err
}
if st.IsDir() {
f.Close()
return nil, &os.PathError{
Op: "open",
Path: path,
Err: syscall.EISDIR,
}
}
return &LockedFile{File: f, size: st.Size()}, nil
}
func makeInheritSa() *syscall.SecurityAttributes {
var sa syscall.SecurityAttributes
sa.Length = uint32(unsafe.Sizeof(sa))
sa.InheritHandle = 1
return &sa
}
// perm param is ignored, on windows file perms/NT acls
// are not octet combinations. Providing access to NT
// acls is out of scope here.
func open(path string, flag int, perm os.FileMode) (*os.File, error) {
if path == "" {
return nil, syscall.ERROR_FILE_NOT_FOUND
}
pathp, err := syscall.UTF16PtrFromString(path)
if err != nil {
return nil, err
}
var access uint32
switch flag {
case syscall.O_RDONLY:
access = syscall.GENERIC_READ
case syscall.O_WRONLY:
access = syscall.GENERIC_WRITE
case syscall.O_RDWR:
access = syscall.GENERIC_READ | syscall.GENERIC_WRITE
case syscall.O_RDWR | syscall.O_CREAT:
access = syscall.GENERIC_ALL
case syscall.O_WRONLY | syscall.O_CREAT:
access = syscall.GENERIC_ALL
}
if flag&syscall.O_APPEND != 0 {
access &^= syscall.GENERIC_WRITE
access |= syscall.FILE_APPEND_DATA
}
var sa *syscall.SecurityAttributes
if flag&syscall.O_CLOEXEC == 0 {
sa = makeInheritSa()
}
var createflag uint32
switch {
case flag&(syscall.O_CREAT|syscall.O_EXCL) == (syscall.O_CREAT | syscall.O_EXCL):
createflag = syscall.CREATE_NEW
case flag&(syscall.O_CREAT|syscall.O_TRUNC) == (syscall.O_CREAT | syscall.O_TRUNC):
createflag = syscall.CREATE_ALWAYS
case flag&syscall.O_CREAT == syscall.O_CREAT:
createflag = syscall.OPEN_ALWAYS
case flag&syscall.O_TRUNC == syscall.O_TRUNC:
createflag = syscall.TRUNCATE_EXISTING
default:
createflag = syscall.OPEN_EXISTING
}
shareflag := uint32(syscall.FILE_SHARE_READ | syscall.FILE_SHARE_WRITE | syscall.FILE_SHARE_DELETE)
accessAttr := uint32(syscall.FILE_ATTRIBUTE_NORMAL | 0x80000000)
fd, err := syscall.CreateFile(pathp, access, shareflag, sa, createflag, accessAttr, 0)
if err != nil {
return nil, err
}
return os.NewFile(uintptr(fd), path), nil
}
func lockFile(fd syscall.Handle, flags uint32) error {
// https://msdn.microsoft.com/en-us/library/windows/desktop/aa365203(v=vs.85).aspx
var flag uint32 = 2 // Lockfile exlusive.
flag |= flags
if fd == syscall.InvalidHandle {
return nil
}
err := lockFileEx(fd, flag, 1, 0, &syscall.Overlapped{})
if err == nil {
return nil
} else if err.Error() == errLocked.Error() {
return errors.New("lock already acquired")
} else if err != errLockViolation {
return err
}
return nil
}
func lockFileEx(h syscall.Handle, flags, locklow, lockhigh uint32, ol *syscall.Overlapped) (err error) {
var reserved = uint32(0)
r1, _, e1 := syscall.Syscall6(procLockFileEx.Addr(), 6, uintptr(h), uintptr(flags), uintptr(reserved), uintptr(locklow), uintptr(lockhigh), uintptr(unsafe.Pointer(ol)))
if r1 == 0 {
if e1 != 0 {
err = error(e1)
} else {
err = syscall.EINVAL
}
}
return
}

View file

@ -298,7 +298,7 @@ func (d config) Diff(c Config) ([]structs.Field, error) {
currFields := structs.Fields(d.Data())
newFields := structs.Fields(c.Data())
found := false
var found bool
for _, currField := range currFields {
found = false
for _, newField := range newFields {
@ -324,7 +324,7 @@ func (d config) DeepDiff(c Config) ([]structs.Field, error) {
currFields := structs.Fields(d.Data())
newFields := structs.Fields(c.Data())
found := false
var found bool
for _, currField := range currFields {
found = false
for _, newField := range newFields {