Lock free multipart backend implementation for FS (#5401)

This commit is contained in:
Krishna Srinivas 2018-01-31 13:17:24 -08:00 committed by kannappanr
parent 018813b98f
commit 3b2486ebaf
17 changed files with 668 additions and 1215 deletions

View File

@ -21,6 +21,7 @@ import (
"fmt" "fmt"
"io" "io"
"os" "os"
"path"
"time" "time"
errors2 "github.com/minio/minio/pkg/errors" errors2 "github.com/minio/minio/pkg/errors"
@ -31,6 +32,7 @@ import (
const ( const (
formatBackendFS = "fs" formatBackendFS = "fs"
formatFSVersionV1 = "1" formatFSVersionV1 = "1"
formatFSVersionV2 = "2"
) )
// formatFSV1 - structure holds format version '1'. // formatFSV1 - structure holds format version '1'.
@ -41,6 +43,12 @@ type formatFSV1 struct {
} `json:"fs"` } `json:"fs"`
} }
// formatFSV2 - structure is same as formatFSV1. But the multipart backend
// structure is flat instead of hierarchy now.
// In .minio.sys/multipart we have:
// sha256(bucket/object)/uploadID/[fs.json, 1.etag, 2.etag ....]
type formatFSV2 = formatFSV1
// Used to detect the version of "fs" format. // Used to detect the version of "fs" format.
type formatFSVersionDetect struct { type formatFSVersionDetect struct {
FS struct { FS struct {
@ -48,7 +56,7 @@ type formatFSVersionDetect struct {
} `json:"fs"` } `json:"fs"`
} }
// Returns the latest "fs" format. // Returns the latest "fs" format V1
func newFormatFSV1() (format *formatFSV1) { func newFormatFSV1() (format *formatFSV1) {
f := &formatFSV1{} f := &formatFSV1{}
f.Version = formatMetaVersionV1 f.Version = formatMetaVersionV1
@ -57,6 +65,15 @@ func newFormatFSV1() (format *formatFSV1) {
return f return f
} }
// Returns the latest "fs" format V2
func newFormatFSV2() (format *formatFSV2) {
f := &formatFSV2{}
f.Version = formatMetaVersionV1
f.Format = formatBackendFS
f.FS.Version = formatFSVersionV2
return f
}
// Save to fs format.json // Save to fs format.json
func formatFSSave(f *os.File, data interface{}) error { func formatFSSave(f *os.File, data interface{}) error {
b, err := json.Marshal(data) b, err := json.Marshal(data)
@ -98,20 +115,57 @@ func formatFSGetVersion(r io.ReadSeeker) (string, error) {
return format.FS.Version, nil return format.FS.Version, nil
} }
// Migrate the "fs" backend. // Migrate from V1 to V2. V2 implements new backend format for multipart
// Migration should happen when formatFSV1.FS.Version changes. This version // uploads. Delete the previous multipart directory.
// can change when there is a change to the struct formatFSV1.FS or if there func formatFSMigrateV1ToV2(wlk *lock.LockedFile, fsPath string) error {
// is any change in the backend file system tree structure.
func formatFSMigrate(wlk *lock.LockedFile) error {
// Add any migration code here in case we bump format.FS.Version
// Make sure that the version is what we expect after the migration.
version, err := formatFSGetVersion(wlk) version, err := formatFSGetVersion(wlk)
if err != nil { if err != nil {
return err return err
} }
if version != formatFSVersionV1 { if version != formatFSVersionV1 {
return fmt.Errorf(`%s file: expected FS version: %s, found FS version: %s`, formatConfigFile, formatFSVersionV1, version) return fmt.Errorf(`format.json version expected %s, found %s`, formatFSVersionV1, version)
}
if err = fsRemoveAll(path.Join(fsPath, minioMetaMultipartBucket)); err != nil {
return err
}
if err = os.MkdirAll(path.Join(fsPath, minioMetaMultipartBucket), 0755); err != nil {
return err
}
return formatFSSave(wlk.File, newFormatFSV2())
}
// Migrate the "fs" backend.
// Migration should happen when formatFSV1.FS.Version changes. This version
// can change when there is a change to the struct formatFSV1.FS or if there
// is any change in the backend file system tree structure.
func formatFSMigrate(wlk *lock.LockedFile, fsPath string) error {
// Add any migration code here in case we bump format.FS.Version
version, err := formatFSGetVersion(wlk)
if err != nil {
return err
}
switch version {
case formatFSVersionV1:
if err = formatFSMigrateV1ToV2(wlk, fsPath); err != nil {
return err
}
fallthrough
case formatFSVersionV2:
// We are at the latest version.
}
// Make sure that the version is what we expect after the migration.
version, err = formatFSGetVersion(wlk)
if err != nil {
return err
}
if version != formatFSVersionV2 {
return fmt.Errorf(`%s file: expected FS version: %s, found FS version: %s`, formatConfigFile, formatFSVersionV2, version)
} }
return nil return nil
} }
@ -196,7 +250,7 @@ func initFormatFS(fsPath string) (rlk *lock.RLockedFile, err error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
if version != formatFSVersionV1 { if version != formatFSVersionV2 {
// Format needs migration // Format needs migration
rlk.Close() rlk.Close()
// Hold write lock during migration so that we do not disturb any // Hold write lock during migration so that we do not disturb any
@ -211,7 +265,7 @@ func initFormatFS(fsPath string) (rlk *lock.RLockedFile, err error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
err = formatFSMigrate(wlk) err = formatFSMigrate(wlk, fsPath)
wlk.Close() wlk.Close()
if err != nil { if err != nil {
// Migration failed, bail out so that the user can observe what happened. // Migration failed, bail out so that the user can observe what happened.

View File

@ -62,8 +62,8 @@ func TestFSFormatFS(t *testing.T) {
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
if version != formatFSVersionV1 { if version != formatFSVersionV2 {
t.Fatalf(`expected: %s, got: %s`, formatFSVersionV1, version) t.Fatalf(`expected: %s, got: %s`, formatFSVersionV2, version)
} }
// Corrupt the format.json file and test the functions. // Corrupt the format.json file and test the functions.

View File

@ -1,240 +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 (
"errors"
"io"
"os"
"reflect"
"sync"
"time"
)
// Error sent by appendParts go-routine when there are holes in parts.
// For ex. let's say client uploads part-2 before part-1 in which case we
// can not append and have to wait till part-1 is uploaded. Hence we return
// this error. Currently this error is not used in the caller.
var errPartsMissing = errors.New("required parts missing")
// Error sent when appendParts go-routine has waited long enough and timedout.
var errAppendPartsTimeout = errors.New("appendParts go-routine timeout")
// Timeout value for the appendParts go-routine.
var appendPartsTimeout = 24 * 60 * 60 * time.Second // 24 Hours.
// Holds a map of uploadID->appendParts go-routine
type backgroundAppend struct {
sync.Mutex
infoMap map[string]bgAppendPartsInfo
appendFile io.WriteCloser
}
// Input to the appendParts go-routine
type bgAppendPartsInput struct {
meta fsMetaV1 // list of parts that need to be appended
errCh chan error // error sent by appendParts go-routine
}
// Identifies an appendParts go-routine.
type bgAppendPartsInfo struct {
inputCh chan bgAppendPartsInput
timeoutCh chan struct{} // closed by appendParts go-routine when it timesout
abortCh chan struct{} // closed after abort of upload to end the appendParts go-routine
completeCh chan struct{} // closed after complete of upload to end the appendParts go-routine
}
// Called after a part is uploaded so that it can be appended in the background.
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.
inputCh := make(chan bgAppendPartsInput)
timeoutCh := make(chan struct{})
abortCh := make(chan struct{})
completeCh := make(chan struct{})
info = bgAppendPartsInfo{inputCh, timeoutCh, abortCh, completeCh}
fs.bgAppend.infoMap[uploadID] = info
go fs.appendParts(bucket, object, uploadID, info)
}
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
// is busy appending a part.
select {
case <-info.timeoutCh:
// This is to handle a rare race condition where we found info in b.infoMap
// but soon after that appendParts go-routine timed out.
errCh <- errAppendPartsTimeout
case info.inputCh <- bgAppendPartsInput{meta, errCh}:
}
}()
return errCh
}
// Called on complete-multipart-upload. Returns nil if the required parts have been appended.
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
// but soon after that appendParts go-routine timedouted out.
return errAppendPartsTimeout
case info.inputCh <- bgAppendPartsInput{meta, errCh}:
}
err := <-errCh
close(info.completeCh)
return err
}
// Called after complete-multipart-upload or abort-multipart-upload so that the appendParts go-routine is not left dangling.
func (fs fsObjects) abort(uploadID string) {
fs.bgAppend.Lock()
defer fs.bgAppend.Unlock()
info, ok := fs.bgAppend.infoMap[uploadID]
if !ok {
return
}
delete(fs.bgAppend.infoMap, uploadID)
info.abortCh <- struct{}{}
}
// This is run as a go-routine that appends the parts in the background.
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 {
select {
case input := <-info.inputCh:
// 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.
part, appendNeeded := partToAppend(meta, appendMeta)
if !appendNeeded {
if reflect.DeepEqual(meta.Parts, appendMeta.Parts) {
// Sending nil is useful so that the complete-multipart-upload knows that
// all the required parts have been appended.
input.errCh <- nil
} else {
// Sending error is useful so that complete-multipart-upload can fall-back to
// its own append process.
input.errCh <- errPartsMissing
}
break
}
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.
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.
close(info.timeoutCh) // So that any racing PutObjectPart does not leave a dangling go-routine.
return
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.
fs.bgAppend.Lock()
delete(fs.bgAppend.infoMap, uploadID)
fs.bgAppend.Unlock()
// Delete the temporary append file as well.
fsRemoveFile(appendPath)
close(info.timeoutCh)
return
}
}
}
// Appends the "part" to the append-file inside "tmp/" that finally gets moved to the actual location
// upon complete-multipart-upload.
func (fs fsObjects) appendPart(bucket, object, uploadID string, part objectPartInfo, buf []byte) error {
partPath := pathJoin(fs.fsPath, minioMetaMultipartBucket, bucket, object, uploadID, part.Name)
var offset int64
// 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
}
return err
}
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((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
}

View File

@ -23,6 +23,7 @@ import (
"runtime" "runtime"
"github.com/minio/minio/pkg/errors" "github.com/minio/minio/pkg/errors"
"github.com/minio/minio/pkg/lock"
) )
// Removes only the file at given path does not remove // Removes only the file at given path does not remove
@ -266,7 +267,7 @@ func fsCreateFile(filePath string, reader io.Reader, buf []byte, fallocSize int6
return 0, errors.Trace(err) return 0, errors.Trace(err)
} }
writer, err := os.OpenFile((filePath), os.O_CREATE|os.O_WRONLY, 0666) writer, err := lock.Open(filePath, os.O_CREATE|os.O_WRONLY, 0666)
if err != nil { if err != nil {
return 0, osErrToFSFileErr(err) return 0, osErrToFSFileErr(err)
} }
@ -291,6 +292,7 @@ func fsCreateFile(filePath string, reader io.Reader, buf []byte, fallocSize int6
return 0, errors.Trace(err) return 0, errors.Trace(err)
} }
} }
return bytesWritten, nil return bytesWritten, nil
} }

View File

@ -22,7 +22,6 @@ import (
"io/ioutil" "io/ioutil"
"os" "os"
pathutil "path" pathutil "path"
"sort"
"strings" "strings"
"github.com/minio/minio/pkg/errors" "github.com/minio/minio/pkg/errors"
@ -127,41 +126,6 @@ func (m fsMetaV1) ToObjectInfo(bucket, object string, fi os.FileInfo) ObjectInfo
return objInfo 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 {
if partNumber == part.Number {
partIndex = i
return partIndex
}
}
return -1
}
// AddObjectPart - add a new object part in order.
func (m *fsMetaV1) AddObjectPart(partNumber int, partName string, partETag string, partSize int64) {
partInfo := objectPartInfo{
Number: partNumber,
Name: partName,
ETag: partETag,
Size: partSize,
}
// Update part info if it already exists.
for i, part := range m.Parts {
if partNumber == part.Number {
m.Parts[i] = partInfo
return
}
}
// Proceed to include new part info.
m.Parts = append(m.Parts, partInfo)
// Parts in fsMeta should be in sorted order by part number.
sort.Sort(byObjectPartNumber(m.Parts))
}
func (m *fsMetaV1) WriteTo(lk *lock.LockedFile) (n int64, err error) { func (m *fsMetaV1) WriteTo(lk *lock.LockedFile) (n int64, err error) {
var metadataBytes []byte var metadataBytes []byte
metadataBytes, err = json.Marshal(m) metadataBytes, err = json.Marshal(m)
@ -203,21 +167,6 @@ func parseFSMetaMap(fsMetaBuf []byte) map[string]string {
return metaMap return metaMap
} }
func parseFSParts(fsMetaBuf []byte) []objectPartInfo {
// Parse the FS Parts.
partsResult := gjson.GetBytes(fsMetaBuf, "parts").Array()
partInfo := make([]objectPartInfo, len(partsResult))
for i, p := range partsResult {
info := objectPartInfo{}
info.Number = int(p.Get("number").Int())
info.Name = p.Get("name").String()
info.ETag = p.Get("etag").String()
info.Size = p.Get("size").Int()
partInfo[i] = info
}
return partInfo
}
func (m *fsMetaV1) ReadFrom(lk *lock.LockedFile) (n int64, err error) { func (m *fsMetaV1) ReadFrom(lk *lock.LockedFile) (n int64, err error) {
var fsMetaBuf []byte var fsMetaBuf []byte
fi, err := lk.Stat() fi, err := lk.Stat()
@ -249,9 +198,6 @@ func (m *fsMetaV1) ReadFrom(lk *lock.LockedFile) (n int64, err error) {
// obtain metadata. // obtain metadata.
m.Meta = parseFSMetaMap(fsMetaBuf) m.Meta = parseFSMetaMap(fsMetaBuf)
// obtain parts info list.
m.Parts = parseFSParts(fsMetaBuf)
// obtain minio release date. // obtain minio release date.
m.Minio.Release = parseFSRelease(fsMetaBuf) m.Minio.Release = parseFSRelease(fsMetaBuf)
@ -267,19 +213,3 @@ func newFSMetaV1() (fsMeta fsMetaV1) {
fsMeta.Minio.Release = ReleaseTag fsMeta.Minio.Release = ReleaseTag
return fsMeta return fsMeta
} }
// Return if the part info in uploadedParts and CompleteParts are same.
func isPartsSame(uploadedParts []objectPartInfo, CompleteParts []CompletePart) bool {
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
}

File diff suppressed because it is too large Load Diff

View File

@ -48,7 +48,7 @@ func TestFSCleanupMultipartUploadsInRoutine(t *testing.T) {
t.Fatal("Unexpected err: ", err) t.Fatal("Unexpected err: ", err)
} }
go cleanupStaleMultipartUploads(20*time.Millisecond, 0, obj, fs.listMultipartUploadsCleanup, globalServiceDoneCh) go fs.cleanupStaleMultipartUploads(20*time.Millisecond, 0, globalServiceDoneCh)
// Wait for 40ms such that - we have given enough time for // Wait for 40ms such that - we have given enough time for
// cleanup routine to kick in. // cleanup routine to kick in.
@ -66,71 +66,6 @@ func TestFSCleanupMultipartUploadsInRoutine(t *testing.T) {
} }
} }
// Tests cleanup of stale upload ids.
func TestFSCleanupMultipartUpload(t *testing.T) {
// Prepare for tests
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer os.RemoveAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(*fsObjects)
// Close the multipart cleanup go-routine.
// In this test we are going to manually call
// the function which actually cleans the stale
// uploads.
globalServiceDoneCh <- struct{}{}
bucketName := "bucket"
objectName := "object"
obj.MakeBucketWithLocation(bucketName, "")
uploadID, err := obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
if err = cleanupStaleMultipartUpload(bucketName, 0, obj, fs.listMultipartUploadsCleanup); err != nil {
t.Fatal("Unexpected err: ", err)
}
// Check if upload id was already purged.
if err = obj.AbortMultipartUpload(bucketName, objectName, uploadID); err != nil {
err = errors.Cause(err)
if _, ok := err.(InvalidUploadID); !ok {
t.Fatal("Unexpected err: ", err)
}
}
}
// TestFSWriteUploadJSON - tests for writeUploadJSON for FS
func TestFSWriteUploadJSON(t *testing.T) {
// Prepare for tests
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
defer os.RemoveAll(disk)
obj := initFSObjects(disk, t)
fs := obj.(*fsObjects)
bucketName := "bucket"
objectName := "object"
obj.MakeBucketWithLocation(bucketName, "")
_, err := obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
t.Fatal("Unexpected err: ", err)
}
// newMultipartUpload will fail.
fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
_, err = obj.NewMultipartUpload(bucketName, objectName, nil)
if err != nil {
if _, ok := errors.Cause(err).(BucketNotFound); !ok {
t.Fatal("Unexpected err: ", err)
}
}
}
// TestNewMultipartUploadFaultyDisk - test NewMultipartUpload with faulty disks // TestNewMultipartUploadFaultyDisk - test NewMultipartUpload with faulty disks
func TestNewMultipartUploadFaultyDisk(t *testing.T) { func TestNewMultipartUploadFaultyDisk(t *testing.T) {
// Prepare for tests // Prepare for tests
@ -215,12 +150,7 @@ func TestCompleteMultipartUploadFaultyDisk(t *testing.T) {
md5Hex := getMD5Hash(data) md5Hex := getMD5Hash(data)
if _, err := fs.PutObjectPart(bucketName, objectName, uploadID, 1, mustGetHashReader(t, bytes.NewReader(data), 5, md5Hex, "")); err != nil {
t.Fatal("Unexpected error ", err)
}
parts := []CompletePart{{PartNumber: 1, ETag: md5Hex}} parts := []CompletePart{{PartNumber: 1, ETag: md5Hex}}
fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix()) fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
if _, err := fs.CompleteMultipartUpload(bucketName, objectName, uploadID, parts); err != nil { if _, err := fs.CompleteMultipartUpload(bucketName, objectName, uploadID, parts); err != nil {
if !isSameType(errors.Cause(err), BucketNotFound{}) { if !isSameType(errors.Cause(err), BucketNotFound{}) {
@ -229,7 +159,7 @@ func TestCompleteMultipartUploadFaultyDisk(t *testing.T) {
} }
} }
// TestCompleteMultipartUploadFaultyDisk - test CompleteMultipartUpload with faulty disks // TestCompleteMultipartUpload - test CompleteMultipartUpload
func TestCompleteMultipartUpload(t *testing.T) { func TestCompleteMultipartUpload(t *testing.T) {
// Prepare for tests // Prepare for tests
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix()) disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
@ -263,7 +193,7 @@ func TestCompleteMultipartUpload(t *testing.T) {
} }
} }
// TestCompleteMultipartUploadFaultyDisk - test CompleteMultipartUpload with faulty disks // TestCompleteMultipartUpload - test CompleteMultipartUpload
func TestAbortMultipartUpload(t *testing.T) { func TestAbortMultipartUpload(t *testing.T) {
// Prepare for tests // Prepare for tests
disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix()) disk := filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
@ -289,7 +219,7 @@ func TestAbortMultipartUpload(t *testing.T) {
if _, err := fs.PutObjectPart(bucketName, objectName, uploadID, 1, mustGetHashReader(t, bytes.NewReader(data), 5, md5Hex, "")); err != nil { if _, err := fs.PutObjectPart(bucketName, objectName, uploadID, 1, mustGetHashReader(t, bytes.NewReader(data), 5, md5Hex, "")); err != nil {
t.Fatal("Unexpected error ", err) t.Fatal("Unexpected error ", err)
} }
time.Sleep(time.Second) // Without Sleep on windows, the fs.AbortMultipartUpload() fails with "The process cannot access the file because it is being used by another process."
if err := fs.AbortMultipartUpload(bucketName, objectName, uploadID); err != nil { if err := fs.AbortMultipartUpload(bucketName, objectName, uploadID); err != nil {
t.Fatal("Unexpected error ", err) t.Fatal("Unexpected error ", err)
} }
@ -306,24 +236,16 @@ func TestListMultipartUploadsFaultyDisk(t *testing.T) {
fs := obj.(*fsObjects) fs := obj.(*fsObjects)
bucketName := "bucket" bucketName := "bucket"
objectName := "object" objectName := "object"
data := []byte("12345")
if err := obj.MakeBucketWithLocation(bucketName, ""); err != nil { if err := obj.MakeBucketWithLocation(bucketName, ""); err != nil {
t.Fatal("Cannot create bucket, err: ", err) t.Fatal("Cannot create bucket, err: ", err)
} }
uploadID, err := fs.NewMultipartUpload(bucketName, objectName, map[string]string{"X-Amz-Meta-xid": "3f"}) _, err := fs.NewMultipartUpload(bucketName, objectName, map[string]string{"X-Amz-Meta-xid": "3f"})
if err != nil { if err != nil {
t.Fatal("Unexpected error ", err) t.Fatal("Unexpected error ", err)
} }
md5Hex := getMD5Hash(data)
sha256sum := ""
if _, err := fs.PutObjectPart(bucketName, objectName, uploadID, 1, mustGetHashReader(t, bytes.NewReader(data), 5, md5Hex, sha256sum)); err != nil {
t.Fatal("Unexpected error ", err)
}
fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix()) fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
if _, err := fs.ListMultipartUploads(bucketName, objectName, "", "", "", 1000); err != nil { if _, err := fs.ListMultipartUploads(bucketName, objectName, "", "", "", 1000); err != nil {
if !isSameType(errors.Cause(err), BucketNotFound{}) { if !isSameType(errors.Cause(err), BucketNotFound{}) {

View File

@ -25,6 +25,7 @@ import (
"path" "path"
"path/filepath" "path/filepath"
"sort" "sort"
"sync"
"syscall" "syscall"
"time" "time"
@ -52,13 +53,20 @@ type fsObjects struct {
// ListObjects pool management. // ListObjects pool management.
listPool *treeWalkPool listPool *treeWalkPool
// To manage the appendRoutine go0routines appendFileMap map[string]*fsAppendFile
bgAppend *backgroundAppend appendFileMapMu sync.Mutex
// name space mutex for object layer // To manage the appendRoutine go-routines
nsMutex *nsLockMap nsMutex *nsLockMap
} }
// Represents the background append file.
type fsAppendFile struct {
sync.Mutex
parts []PartInfo // List of parts appended.
filePath string // Absolute path of the file in the temp location.
}
// Initializes meta volume on all the fs path. // Initializes meta volume on all the fs path.
func initMetaVolumeFS(fsPath, fsUUID string) error { func initMetaVolumeFS(fsPath, fsUUID string) error {
// This happens for the first time, but keep this here since this // This happens for the first time, but keep this here since this
@ -139,11 +147,9 @@ func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
rwPool: &fsIOPool{ rwPool: &fsIOPool{
readersMap: make(map[string]*lock.RLockedFile), readersMap: make(map[string]*lock.RLockedFile),
}, },
listPool: newTreeWalkPool(globalLookupTimeout), nsMutex: newNSLock(false),
bgAppend: &backgroundAppend{ listPool: newTreeWalkPool(globalLookupTimeout),
infoMap: make(map[string]bgAppendPartsInfo), appendFileMap: make(map[string]*fsAppendFile),
},
nsMutex: newNSLock(false),
} }
// Once the filesystem has initialized hold the read lock for // Once the filesystem has initialized hold the read lock for
@ -162,15 +168,13 @@ func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
return nil, fmt.Errorf("Unable to initialize event notification. %s", err) return nil, fmt.Errorf("Unable to initialize event notification. %s", err)
} }
// Start background process to cleanup old multipart objects in `.minio.sys`. go fs.cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, globalServiceDoneCh)
go cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, fs, fs.listMultipartUploadsCleanup, globalServiceDoneCh)
// Return successfully initialized object layer. // Return successfully initialized object layer.
return fs, nil return fs, nil
} }
// Should be called when process shuts down. // Should be called when process shuts down.
func (fs fsObjects) Shutdown() error { func (fs *fsObjects) Shutdown() error {
fs.fsFormatRlk.Close() fs.fsFormatRlk.Close()
// Cleanup and delete tmp uuid. // Cleanup and delete tmp uuid.
@ -178,7 +182,7 @@ func (fs fsObjects) Shutdown() error {
} }
// StorageInfo - returns underlying storage statistics. // StorageInfo - returns underlying storage statistics.
func (fs fsObjects) StorageInfo() StorageInfo { func (fs *fsObjects) StorageInfo() StorageInfo {
info, err := getDiskInfo((fs.fsPath)) info, err := getDiskInfo((fs.fsPath))
errorIf(err, "Unable to get disk info %#v", fs.fsPath) errorIf(err, "Unable to get disk info %#v", fs.fsPath)
storageInfo := StorageInfo{ storageInfo := StorageInfo{
@ -192,12 +196,12 @@ func (fs fsObjects) StorageInfo() StorageInfo {
// Locking operations // Locking operations
// List namespace locks held in object layer // List namespace locks held in object layer
func (fs fsObjects) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) { func (fs *fsObjects) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
return []VolumeLockInfo{}, NotImplemented{} return []VolumeLockInfo{}, NotImplemented{}
} }
// Clear namespace locks held in object layer // Clear namespace locks held in object layer
func (fs fsObjects) ClearLocks([]VolumeLockInfo) error { func (fs *fsObjects) ClearLocks([]VolumeLockInfo) error {
return NotImplemented{} return NotImplemented{}
} }
@ -206,7 +210,7 @@ func (fs fsObjects) ClearLocks([]VolumeLockInfo) error {
// getBucketDir - will convert incoming bucket names to // getBucketDir - will convert incoming bucket names to
// corresponding valid bucket names on the backend in a platform // corresponding valid bucket names on the backend in a platform
// compatible way for all operating systems. // compatible way for all operating systems.
func (fs fsObjects) getBucketDir(bucket string) (string, error) { func (fs *fsObjects) getBucketDir(bucket string) (string, error) {
// Verify if bucket is valid. // Verify if bucket is valid.
if !IsValidBucketName(bucket) { if !IsValidBucketName(bucket) {
return "", errors.Trace(BucketNameInvalid{Bucket: bucket}) return "", errors.Trace(BucketNameInvalid{Bucket: bucket})
@ -216,7 +220,7 @@ func (fs fsObjects) getBucketDir(bucket string) (string, error) {
return bucketDir, nil return bucketDir, nil
} }
func (fs fsObjects) statBucketDir(bucket string) (os.FileInfo, error) { func (fs *fsObjects) statBucketDir(bucket string) (os.FileInfo, error) {
bucketDir, err := fs.getBucketDir(bucket) bucketDir, err := fs.getBucketDir(bucket)
if err != nil { if err != nil {
return nil, err return nil, err
@ -230,7 +234,7 @@ func (fs fsObjects) statBucketDir(bucket string) (os.FileInfo, error) {
// MakeBucket - create a new bucket, returns if it // MakeBucket - create a new bucket, returns if it
// already exists. // already exists.
func (fs fsObjects) MakeBucketWithLocation(bucket, location string) error { func (fs *fsObjects) MakeBucketWithLocation(bucket, location string) error {
bucketLock := fs.nsMutex.NewNSLock(bucket, "") bucketLock := fs.nsMutex.NewNSLock(bucket, "")
if err := bucketLock.GetLock(globalObjectTimeout); err != nil { if err := bucketLock.GetLock(globalObjectTimeout); err != nil {
return err return err
@ -249,7 +253,7 @@ func (fs fsObjects) MakeBucketWithLocation(bucket, location string) error {
} }
// GetBucketInfo - fetch bucket metadata info. // GetBucketInfo - fetch bucket metadata info.
func (fs fsObjects) GetBucketInfo(bucket string) (bi BucketInfo, e error) { func (fs *fsObjects) GetBucketInfo(bucket string) (bi BucketInfo, e error) {
st, err := fs.statBucketDir(bucket) st, err := fs.statBucketDir(bucket)
if err != nil { if err != nil {
return bi, toObjectErr(err, bucket) return bi, toObjectErr(err, bucket)
@ -264,7 +268,7 @@ func (fs fsObjects) GetBucketInfo(bucket string) (bi BucketInfo, e error) {
} }
// ListBuckets - list all s3 compatible buckets (directories) at fsPath. // ListBuckets - list all s3 compatible buckets (directories) at fsPath.
func (fs fsObjects) ListBuckets() ([]BucketInfo, error) { func (fs *fsObjects) ListBuckets() ([]BucketInfo, error) {
if err := checkPathLength(fs.fsPath); err != nil { if err := checkPathLength(fs.fsPath); err != nil {
return nil, errors.Trace(err) return nil, errors.Trace(err)
} }
@ -305,7 +309,7 @@ func (fs fsObjects) ListBuckets() ([]BucketInfo, error) {
// DeleteBucket - delete a bucket and all the metadata associated // DeleteBucket - delete a bucket and all the metadata associated
// with the bucket including pending multipart, object metadata. // with the bucket including pending multipart, object metadata.
func (fs fsObjects) DeleteBucket(bucket string) error { func (fs *fsObjects) DeleteBucket(bucket string) error {
bucketDir, err := fs.getBucketDir(bucket) bucketDir, err := fs.getBucketDir(bucket)
if err != nil { if err != nil {
return toObjectErr(err, bucket) return toObjectErr(err, bucket)
@ -336,7 +340,7 @@ func (fs fsObjects) DeleteBucket(bucket string) error {
// CopyObject - copy object source object to destination object. // CopyObject - copy object source object to destination object.
// if source object and destination object are same we only // if source object and destination object are same we only
// update metadata. // update metadata.
func (fs fsObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string, metadata map[string]string, srcEtag string) (oi ObjectInfo, e error) { func (fs *fsObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string, metadata map[string]string, srcEtag string) (oi ObjectInfo, e error) {
cpSrcDstSame := srcBucket == dstBucket && srcObject == dstObject cpSrcDstSame := srcBucket == dstBucket && srcObject == dstObject
// Hold write lock on destination since in both cases // Hold write lock on destination since in both cases
// - if source and destination are same // - if source and destination are same
@ -439,7 +443,7 @@ func (fs fsObjects) CopyObject(srcBucket, srcObject, dstBucket, dstObject string
// //
// startOffset indicates the starting read location of the object. // startOffset indicates the starting read location of the object.
// length indicates the total length of the object. // length indicates the total length of the object.
func (fs fsObjects) GetObject(bucket, object string, offset int64, length int64, writer io.Writer, etag string) (err error) { func (fs *fsObjects) GetObject(bucket, object string, offset int64, length int64, writer io.Writer, etag string) (err error) {
if err = checkGetObjArgs(bucket, object); err != nil { if err = checkGetObjArgs(bucket, object); err != nil {
return err return err
} }
@ -454,7 +458,7 @@ func (fs fsObjects) GetObject(bucket, object string, offset int64, length int64,
} }
// getObject - wrapper for GetObject // getObject - wrapper for GetObject
func (fs fsObjects) getObject(bucket, object string, offset int64, length int64, writer io.Writer, etag string) (err error) { func (fs *fsObjects) getObject(bucket, object string, offset int64, length int64, writer io.Writer, etag string) (err error) {
if _, err = fs.statBucketDir(bucket); err != nil { if _, err = fs.statBucketDir(bucket); err != nil {
return toObjectErr(err, bucket) return toObjectErr(err, bucket)
} }
@ -525,7 +529,7 @@ func (fs fsObjects) getObject(bucket, object string, offset int64, length int64,
} }
// getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo. // getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo.
func (fs fsObjects) getObjectInfo(bucket, object string) (oi ObjectInfo, e error) { func (fs *fsObjects) getObjectInfo(bucket, object string) (oi ObjectInfo, e error) {
fsMeta := fsMetaV1{} fsMeta := fsMetaV1{}
fi, err := fsStatDir(pathJoin(fs.fsPath, bucket, object)) fi, err := fsStatDir(pathJoin(fs.fsPath, bucket, object))
if err != nil && errors.Cause(err) != errFileAccessDenied { if err != nil && errors.Cause(err) != errFileAccessDenied {
@ -573,7 +577,7 @@ func (fs fsObjects) getObjectInfo(bucket, object string) (oi ObjectInfo, e error
} }
// GetObjectInfo - reads object metadata and replies back ObjectInfo. // GetObjectInfo - reads object metadata and replies back ObjectInfo.
func (fs fsObjects) GetObjectInfo(bucket, object string) (oi ObjectInfo, e error) { func (fs *fsObjects) GetObjectInfo(bucket, object string) (oi ObjectInfo, e error) {
// Lock the object before reading. // Lock the object before reading.
objectLock := fs.nsMutex.NewNSLock(bucket, object) objectLock := fs.nsMutex.NewNSLock(bucket, object)
if err := objectLock.GetRLock(globalObjectTimeout); err != nil { if err := objectLock.GetRLock(globalObjectTimeout); err != nil {
@ -595,7 +599,7 @@ func (fs fsObjects) GetObjectInfo(bucket, object string) (oi ObjectInfo, e error
// This function does the following check, suppose // This function does the following check, suppose
// object is "a/b/c/d", stat makes sure that objects ""a/b/c"" // object is "a/b/c/d", stat makes sure that objects ""a/b/c""
// "a/b" and "a" do not exist. // "a/b" and "a" do not exist.
func (fs fsObjects) parentDirIsObject(bucket, parent string) bool { func (fs *fsObjects) parentDirIsObject(bucket, parent string) bool {
var isParentDirObject func(string) bool var isParentDirObject func(string) bool
isParentDirObject = func(p string) bool { isParentDirObject = func(p string) bool {
if p == "." || p == "/" { if p == "." || p == "/" {
@ -616,7 +620,7 @@ func (fs fsObjects) parentDirIsObject(bucket, parent string) bool {
// until EOF, writes data directly to configured filesystem path. // until EOF, writes data directly to configured filesystem path.
// Additionally writes `fs.json` which carries the necessary metadata // Additionally writes `fs.json` which carries the necessary metadata
// for future object operations. // for future object operations.
func (fs fsObjects) PutObject(bucket string, object string, data *hash.Reader, metadata map[string]string) (objInfo ObjectInfo, retErr error) { func (fs *fsObjects) PutObject(bucket string, object string, data *hash.Reader, metadata map[string]string) (objInfo ObjectInfo, retErr error) {
// Lock the object. // Lock the object.
objectLock := fs.nsMutex.NewNSLock(bucket, object) objectLock := fs.nsMutex.NewNSLock(bucket, object)
if err := objectLock.GetLock(globalObjectTimeout); err != nil { if err := objectLock.GetLock(globalObjectTimeout); err != nil {
@ -627,7 +631,7 @@ func (fs fsObjects) PutObject(bucket string, object string, data *hash.Reader, m
} }
// putObject - wrapper for PutObject // putObject - wrapper for PutObject
func (fs fsObjects) putObject(bucket string, object string, data *hash.Reader, metadata map[string]string) (objInfo ObjectInfo, retErr error) { func (fs *fsObjects) putObject(bucket string, object string, data *hash.Reader, metadata map[string]string) (objInfo ObjectInfo, retErr error) {
// No metadata is set, allocate a new one. // No metadata is set, allocate a new one.
if metadata == nil { if metadata == nil {
metadata = make(map[string]string) metadata = make(map[string]string)
@ -751,7 +755,7 @@ func (fs fsObjects) putObject(bucket string, object string, data *hash.Reader, m
// DeleteObject - deletes an object from a bucket, this operation is destructive // DeleteObject - deletes an object from a bucket, this operation is destructive
// and there are no rollbacks supported. // and there are no rollbacks supported.
func (fs fsObjects) DeleteObject(bucket, object string) error { func (fs *fsObjects) DeleteObject(bucket, object string) error {
// Acquire a write lock before deleting the object. // Acquire a write lock before deleting the object.
objectLock := fs.nsMutex.NewNSLock(bucket, object) objectLock := fs.nsMutex.NewNSLock(bucket, object)
if err := objectLock.GetLock(globalOperationTimeout); err != nil { if err := objectLock.GetLock(globalOperationTimeout); err != nil {
@ -804,7 +808,7 @@ var fsTreeWalkIgnoredErrs = append(baseIgnoredErrs, []error{
// Returns function "listDir" of the type listDirFunc. // Returns function "listDir" of the type listDirFunc.
// isLeaf - is used by listDir function to check if an entry // isLeaf - is used by listDir function to check if an entry
// is a leaf or non-leaf entry. // is a leaf or non-leaf entry.
func (fs fsObjects) listDirFactory(isLeaf isLeafFunc) listDirFunc { func (fs *fsObjects) listDirFactory(isLeaf isLeafFunc) listDirFunc {
// listDir - lists all the entries at a given prefix and given entry in the prefix. // 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) { listDir := func(bucket, prefixDir, prefixEntry string) (entries []string, delayIsLeaf bool, err error) {
entries, err = readDir(pathJoin(fs.fsPath, bucket, prefixDir)) entries, err = readDir(pathJoin(fs.fsPath, bucket, prefixDir))
@ -821,7 +825,7 @@ func (fs fsObjects) listDirFactory(isLeaf isLeafFunc) listDirFunc {
// getObjectETag is a helper function, which returns only the md5sum // getObjectETag is a helper function, which returns only the md5sum
// of the file on the disk. // of the file on the disk.
func (fs fsObjects) getObjectETag(bucket, entry string) (string, error) { func (fs *fsObjects) getObjectETag(bucket, entry string) (string, error) {
fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, entry, fsMetaJSONFile) fsMetaPath := pathJoin(fs.fsPath, minioMetaBucket, bucketMetaPrefix, bucket, entry, fsMetaJSONFile)
// Read `fs.json` to perhaps contend with // Read `fs.json` to perhaps contend with
@ -870,7 +874,7 @@ func (fs fsObjects) getObjectETag(bucket, entry string) (string, error) {
// ListObjects - list all objects at prefix upto maxKeys., optionally delimited by '/'. Maintains the list pool // ListObjects - list all objects at prefix upto maxKeys., optionally delimited by '/'. Maintains the list pool
// state for future re-entrant list requests. // state for future re-entrant list requests.
func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) { func (fs *fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) {
if err := checkListObjsArgs(bucket, prefix, marker, delimiter, fs); err != nil { if err := checkListObjsArgs(bucket, prefix, marker, delimiter, fs); err != nil {
return loi, err return loi, err
} }
@ -1024,25 +1028,23 @@ func (fs fsObjects) ListObjects(bucket, prefix, marker, delimiter string, maxKey
} }
// HealObject - no-op for fs. Valid only for XL. // HealObject - no-op for fs. Valid only for XL.
func (fs fsObjects) HealObject(bucket, object string, dryRun bool) ( func (fs *fsObjects) HealObject(bucket, object string, dryRun bool) (
res madmin.HealResultItem, err error) { res madmin.HealResultItem, err error) {
return res, errors.Trace(NotImplemented{}) return res, errors.Trace(NotImplemented{})
} }
// HealBucket - no-op for fs, Valid only for XL. // HealBucket - no-op for fs, Valid only for XL.
func (fs fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem, func (fs *fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem,
error) { error) {
return nil, errors.Trace(NotImplemented{}) return nil, errors.Trace(NotImplemented{})
} }
// ListObjectsHeal - list all objects to be healed. Valid only for XL // ListObjectsHeal - list all objects to be healed. Valid only for XL
func (fs fsObjects) ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) { func (fs *fsObjects) ListObjectsHeal(bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) {
return loi, errors.Trace(NotImplemented{}) return loi, errors.Trace(NotImplemented{})
} }
// ListBucketsHeal - list all buckets to be healed. Valid only for XL // ListBucketsHeal - list all buckets to be healed. Valid only for XL
func (fs fsObjects) ListBucketsHeal() ([]BucketInfo, error) { func (fs *fsObjects) ListBucketsHeal() ([]BucketInfo, error) {
return []BucketInfo{}, errors.Trace(NotImplemented{}) return []BucketInfo{}, errors.Trace(NotImplemented{})
} }

View File

@ -1374,11 +1374,12 @@ func testListObjectPartsDiskNotFound(obj ObjectLayer, instanceType string, disks
}, },
// partinfos - 2. // partinfos - 2.
{ {
Bucket: bucketNames[0], Bucket: bucketNames[0],
Object: objectNames[0], Object: objectNames[0],
MaxParts: 2, MaxParts: 2,
IsTruncated: false, IsTruncated: false,
UploadID: uploadIDs[0], UploadID: uploadIDs[0],
PartNumberMarker: 3,
Parts: []PartInfo{ Parts: []PartInfo{
{ {
PartNumber: 4, PartNumber: 4,
@ -1611,11 +1612,12 @@ func testListObjectParts(obj ObjectLayer, instanceType string, t TestErrHandler)
}, },
// partinfos - 2. // partinfos - 2.
{ {
Bucket: bucketNames[0], Bucket: bucketNames[0],
Object: objectNames[0], Object: objectNames[0],
MaxParts: 2, MaxParts: 2,
IsTruncated: false, IsTruncated: false,
UploadID: uploadIDs[0], UploadID: uploadIDs[0],
PartNumberMarker: 3,
Parts: []PartInfo{ Parts: []PartInfo{
{ {
PartNumber: 4, PartNumber: 4,

View File

@ -909,6 +909,7 @@ func (api objectAPIHandlers) AbortMultipartUploadHandler(w http.ResponseWriter,
uploadID, _, _, _ := getObjectResources(r.URL.Query()) uploadID, _, _, _ := getObjectResources(r.URL.Query())
if err := objectAPI.AbortMultipartUpload(bucket, object, uploadID); err != nil { if err := objectAPI.AbortMultipartUpload(bucket, object, uploadID); err != nil {
errorIf(err, "AbortMultipartUpload failed")
writeErrorResponse(w, toAPIErrorCode(err), r.URL) writeErrorResponse(w, toAPIErrorCode(err), r.URL)
return return
} }

View File

@ -833,6 +833,7 @@ func (xl xlObjects) listObjectParts(bucket, object, uploadID string, partNumberM
result.Object = object result.Object = object
result.UploadID = uploadID result.UploadID = uploadID
result.MaxParts = maxParts result.MaxParts = maxParts
result.PartNumberMarker = partNumberMarker
// For empty number of parts or maxParts as zero, return right here. // For empty number of parts or maxParts as zero, return right here.
if len(xlParts) == 0 || maxParts == 0 { if len(xlParts) == 0 || maxParts == 0 {

View File

@ -0,0 +1,43 @@
// +build !windows
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package ioutil
import (
"io"
"os"
)
// AppendFile - appends the file "src" to the file "dst"
func AppendFile(dst string, src string) error {
appendFile, err := os.OpenFile(dst, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644)
if err != nil {
return err
}
defer appendFile.Close()
srcFile, err := os.Open(src)
if err != nil {
return err
}
defer srcFile.Close()
// Allocate staging buffer.
var buf = make([]byte, defaultAppendBufferSize)
_, err = io.CopyBuffer(appendFile, srcFile, buf)
return err
}

View File

@ -0,0 +1,43 @@
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package ioutil
import (
"io"
"os"
"github.com/minio/minio/pkg/lock"
)
// AppendFile - appends the file "src" to the file "dst"
func AppendFile(dst string, src string) error {
appendFile, err := lock.Open(dst, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644)
if err != nil {
return err
}
defer appendFile.Close()
srcFile, err := lock.Open(src, os.O_RDONLY, 0644)
if err != nil {
return err
}
defer srcFile.Close()
// Allocate staging buffer.
var buf = make([]byte, defaultAppendBufferSize)
_, err = io.CopyBuffer(appendFile, srcFile, buf)
return err
}

View File

@ -20,8 +20,13 @@ package ioutil
import ( import (
"io" "io"
humanize "github.com/dustin/go-humanize"
) )
// defaultAppendBufferSize - Default buffer size for the AppendFile
const defaultAppendBufferSize = humanize.MiByte
// WriteOnCloser implements io.WriteCloser and always // WriteOnCloser implements io.WriteCloser and always
// exectues at least one write operation if it is closed. // exectues at least one write operation if it is closed.
// //

View File

@ -18,6 +18,7 @@ package ioutil
import ( import (
goioutil "io/ioutil" goioutil "io/ioutil"
"os"
"testing" "testing"
) )
@ -37,3 +38,38 @@ func TestCloseOnWriter(t *testing.T) {
t.Error("WriteOnCloser must be marked as HasWritten") t.Error("WriteOnCloser must be marked as HasWritten")
} }
} }
// Test for AppendFile.
func TestAppendFile(t *testing.T) {
f, err := goioutil.TempFile("", "")
if err != nil {
t.Fatal(err)
}
name1 := f.Name()
defer os.Remove(name1)
f.WriteString("aaaaaaaaaa")
f.Close()
f, err = goioutil.TempFile("", "")
if err != nil {
t.Fatal(err)
}
name2 := f.Name()
defer os.Remove(name2)
f.WriteString("bbbbbbbbbb")
f.Close()
if err = AppendFile(name1, name2); err != nil {
t.Error(err)
}
b, err := goioutil.ReadFile(name1)
if err != nil {
t.Error(err)
}
expected := "aaaaaaaaaabbbbbbbbbb"
if string(b) != expected {
t.Errorf("AppendFile() failed, expected: %s, got %s", expected, string(b))
}
}

View File

@ -90,3 +90,8 @@ func TryLockedOpenFile(path string, flag int, perm os.FileMode) (*LockedFile, er
func LockedOpenFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { func LockedOpenFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
return lockedOpenFile(path, flag, perm, 0) return lockedOpenFile(path, flag, perm, 0)
} }
// Open - Call os.OpenFile
func Open(path string, flag int, perm os.FileMode) (*os.File, error) {
return os.OpenFile(path, flag, perm)
}

View File

@ -42,7 +42,7 @@ const (
// lockedOpenFile is an internal function. // lockedOpenFile is an internal function.
func lockedOpenFile(path string, flag int, perm os.FileMode, lockType uint32) (*LockedFile, error) { func lockedOpenFile(path string, flag int, perm os.FileMode, lockType uint32) (*LockedFile, error) {
f, err := open(path, flag, perm) f, err := Open(path, flag, perm)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -168,7 +168,7 @@ func fixLongPath(path string) string {
// perm param is ignored, on windows file perms/NT acls // perm param is ignored, on windows file perms/NT acls
// are not octet combinations. Providing access to NT // are not octet combinations. Providing access to NT
// acls is out of scope here. // acls is out of scope here.
func open(path string, flag int, perm os.FileMode) (*os.File, error) { func Open(path string, flag int, perm os.FileMode) (*os.File, error) {
if path == "" { if path == "" {
return nil, syscall.ERROR_FILE_NOT_FOUND return nil, syscall.ERROR_FILE_NOT_FOUND
} }
@ -190,6 +190,8 @@ func open(path string, flag int, perm os.FileMode) (*os.File, error) {
fallthrough fallthrough
case syscall.O_WRONLY | syscall.O_CREAT: case syscall.O_WRONLY | syscall.O_CREAT:
access = syscall.GENERIC_READ | syscall.GENERIC_WRITE access = syscall.GENERIC_READ | syscall.GENERIC_WRITE
case syscall.O_WRONLY | syscall.O_CREAT | syscall.O_APPEND:
access = syscall.FILE_APPEND_DATA
default: default:
return nil, fmt.Errorf("Unsupported flag (%d)", flag) return nil, fmt.Errorf("Unsupported flag (%d)", flag)
} }