mirror of https://github.com/minio/minio.git
Lock free multipart backend implementation for FS (#5401)
This commit is contained in:
parent
018813b98f
commit
3b2486ebaf
|
@ -21,6 +21,7 @@ import (
|
|||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path"
|
||||
"time"
|
||||
|
||||
errors2 "github.com/minio/minio/pkg/errors"
|
||||
|
@ -31,6 +32,7 @@ import (
|
|||
const (
|
||||
formatBackendFS = "fs"
|
||||
formatFSVersionV1 = "1"
|
||||
formatFSVersionV2 = "2"
|
||||
)
|
||||
|
||||
// formatFSV1 - structure holds format version '1'.
|
||||
|
@ -41,6 +43,12 @@ type formatFSV1 struct {
|
|||
} `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.
|
||||
type formatFSVersionDetect struct {
|
||||
FS struct {
|
||||
|
@ -48,7 +56,7 @@ type formatFSVersionDetect struct {
|
|||
} `json:"fs"`
|
||||
}
|
||||
|
||||
// Returns the latest "fs" format.
|
||||
// Returns the latest "fs" format V1
|
||||
func newFormatFSV1() (format *formatFSV1) {
|
||||
f := &formatFSV1{}
|
||||
f.Version = formatMetaVersionV1
|
||||
|
@ -57,6 +65,15 @@ func newFormatFSV1() (format *formatFSV1) {
|
|||
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
|
||||
func formatFSSave(f *os.File, data interface{}) error {
|
||||
b, err := json.Marshal(data)
|
||||
|
@ -98,20 +115,57 @@ func formatFSGetVersion(r io.ReadSeeker) (string, error) {
|
|||
return format.FS.Version, nil
|
||||
}
|
||||
|
||||
// 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) 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.
|
||||
// Migrate from V1 to V2. V2 implements new backend format for multipart
|
||||
// uploads. Delete the previous multipart directory.
|
||||
func formatFSMigrateV1ToV2(wlk *lock.LockedFile, fsPath string) error {
|
||||
version, err := formatFSGetVersion(wlk)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
@ -196,7 +250,7 @@ func initFormatFS(fsPath string) (rlk *lock.RLockedFile, err error) {
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if version != formatFSVersionV1 {
|
||||
if version != formatFSVersionV2 {
|
||||
// Format needs migration
|
||||
rlk.Close()
|
||||
// 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 {
|
||||
return nil, err
|
||||
}
|
||||
err = formatFSMigrate(wlk)
|
||||
err = formatFSMigrate(wlk, fsPath)
|
||||
wlk.Close()
|
||||
if err != nil {
|
||||
// Migration failed, bail out so that the user can observe what happened.
|
||||
|
|
|
@ -62,8 +62,8 @@ func TestFSFormatFS(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if version != formatFSVersionV1 {
|
||||
t.Fatalf(`expected: %s, got: %s`, formatFSVersionV1, version)
|
||||
if version != formatFSVersionV2 {
|
||||
t.Fatalf(`expected: %s, got: %s`, formatFSVersionV2, version)
|
||||
}
|
||||
|
||||
// Corrupt the format.json file and test the functions.
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -23,6 +23,7 @@ import (
|
|||
"runtime"
|
||||
|
||||
"github.com/minio/minio/pkg/errors"
|
||||
"github.com/minio/minio/pkg/lock"
|
||||
)
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
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 {
|
||||
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 bytesWritten, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ import (
|
|||
"io/ioutil"
|
||||
"os"
|
||||
pathutil "path"
|
||||
"sort"
|
||||
"strings"
|
||||
|
||||
"github.com/minio/minio/pkg/errors"
|
||||
|
@ -127,41 +126,6 @@ func (m fsMetaV1) ToObjectInfo(bucket, object string, fi os.FileInfo) ObjectInfo
|
|||
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) {
|
||||
var metadataBytes []byte
|
||||
metadataBytes, err = json.Marshal(m)
|
||||
|
@ -203,21 +167,6 @@ func parseFSMetaMap(fsMetaBuf []byte) map[string]string {
|
|||
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) {
|
||||
var fsMetaBuf []byte
|
||||
fi, err := lk.Stat()
|
||||
|
@ -249,9 +198,6 @@ func (m *fsMetaV1) ReadFrom(lk *lock.LockedFile) (n int64, err error) {
|
|||
// obtain metadata.
|
||||
m.Meta = parseFSMetaMap(fsMetaBuf)
|
||||
|
||||
// obtain parts info list.
|
||||
m.Parts = parseFSParts(fsMetaBuf)
|
||||
|
||||
// obtain minio release date.
|
||||
m.Minio.Release = parseFSRelease(fsMetaBuf)
|
||||
|
||||
|
@ -267,19 +213,3 @@ func newFSMetaV1() (fsMeta fsMetaV1) {
|
|||
fsMeta.Minio.Release = ReleaseTag
|
||||
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
|
@ -48,7 +48,7 @@ func TestFSCleanupMultipartUploadsInRoutine(t *testing.T) {
|
|||
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
|
||||
// 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
|
||||
func TestNewMultipartUploadFaultyDisk(t *testing.T) {
|
||||
// Prepare for tests
|
||||
|
@ -215,12 +150,7 @@ func TestCompleteMultipartUploadFaultyDisk(t *testing.T) {
|
|||
|
||||
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}}
|
||||
|
||||
fs.fsPath = filepath.Join(globalTestTmpDir, "minio-"+nextSuffix())
|
||||
if _, err := fs.CompleteMultipartUpload(bucketName, objectName, uploadID, parts); err != nil {
|
||||
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) {
|
||||
// Prepare for tests
|
||||
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) {
|
||||
// Prepare for tests
|
||||
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 {
|
||||
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 {
|
||||
t.Fatal("Unexpected error ", err)
|
||||
}
|
||||
|
@ -306,24 +236,16 @@ func TestListMultipartUploadsFaultyDisk(t *testing.T) {
|
|||
fs := obj.(*fsObjects)
|
||||
bucketName := "bucket"
|
||||
objectName := "object"
|
||||
data := []byte("12345")
|
||||
|
||||
if err := obj.MakeBucketWithLocation(bucketName, ""); err != nil {
|
||||
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 {
|
||||
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())
|
||||
if _, err := fs.ListMultipartUploads(bucketName, objectName, "", "", "", 1000); err != nil {
|
||||
if !isSameType(errors.Cause(err), BucketNotFound{}) {
|
||||
|
|
80
cmd/fs-v1.go
80
cmd/fs-v1.go
|
@ -25,6 +25,7 @@ import (
|
|||
"path"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"sync"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
|
@ -52,13 +53,20 @@ type fsObjects struct {
|
|||
// ListObjects pool management.
|
||||
listPool *treeWalkPool
|
||||
|
||||
// To manage the appendRoutine go0routines
|
||||
bgAppend *backgroundAppend
|
||||
appendFileMap map[string]*fsAppendFile
|
||||
appendFileMapMu sync.Mutex
|
||||
|
||||
// name space mutex for object layer
|
||||
// To manage the appendRoutine go-routines
|
||||
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.
|
||||
func initMetaVolumeFS(fsPath, fsUUID string) error {
|
||||
// This happens for the first time, but keep this here since this
|
||||
|
@ -139,11 +147,9 @@ func newFSObjectLayer(fsPath string) (ObjectLayer, error) {
|
|||
rwPool: &fsIOPool{
|
||||
readersMap: make(map[string]*lock.RLockedFile),
|
||||
},
|
||||
listPool: newTreeWalkPool(globalLookupTimeout),
|
||||
bgAppend: &backgroundAppend{
|
||||
infoMap: make(map[string]bgAppendPartsInfo),
|
||||
},
|
||||
nsMutex: newNSLock(false),
|
||||
nsMutex: newNSLock(false),
|
||||
listPool: newTreeWalkPool(globalLookupTimeout),
|
||||
appendFileMap: make(map[string]*fsAppendFile),
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// Start background process to cleanup old multipart objects in `.minio.sys`.
|
||||
go cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, fs, fs.listMultipartUploadsCleanup, globalServiceDoneCh)
|
||||
|
||||
go fs.cleanupStaleMultipartUploads(multipartCleanupInterval, multipartExpiry, globalServiceDoneCh)
|
||||
// Return successfully initialized object layer.
|
||||
return fs, nil
|
||||
}
|
||||
|
||||
// Should be called when process shuts down.
|
||||
func (fs fsObjects) Shutdown() error {
|
||||
func (fs *fsObjects) Shutdown() error {
|
||||
fs.fsFormatRlk.Close()
|
||||
|
||||
// Cleanup and delete tmp uuid.
|
||||
|
@ -178,7 +182,7 @@ func (fs fsObjects) Shutdown() error {
|
|||
}
|
||||
|
||||
// StorageInfo - returns underlying storage statistics.
|
||||
func (fs fsObjects) StorageInfo() StorageInfo {
|
||||
func (fs *fsObjects) StorageInfo() StorageInfo {
|
||||
info, err := getDiskInfo((fs.fsPath))
|
||||
errorIf(err, "Unable to get disk info %#v", fs.fsPath)
|
||||
storageInfo := StorageInfo{
|
||||
|
@ -192,12 +196,12 @@ func (fs fsObjects) StorageInfo() StorageInfo {
|
|||
// Locking operations
|
||||
|
||||
// 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{}
|
||||
}
|
||||
|
||||
// Clear namespace locks held in object layer
|
||||
func (fs fsObjects) ClearLocks([]VolumeLockInfo) error {
|
||||
func (fs *fsObjects) ClearLocks([]VolumeLockInfo) error {
|
||||
return NotImplemented{}
|
||||
}
|
||||
|
||||
|
@ -206,7 +210,7 @@ func (fs fsObjects) ClearLocks([]VolumeLockInfo) 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) {
|
||||
func (fs *fsObjects) getBucketDir(bucket string) (string, error) {
|
||||
// Verify if bucket is valid.
|
||||
if !IsValidBucketName(bucket) {
|
||||
return "", errors.Trace(BucketNameInvalid{Bucket: bucket})
|
||||
|
@ -216,7 +220,7 @@ func (fs fsObjects) getBucketDir(bucket string) (string, error) {
|
|||
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)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -230,7 +234,7 @@ func (fs fsObjects) statBucketDir(bucket string) (os.FileInfo, error) {
|
|||
|
||||
// MakeBucket - create a new bucket, returns if it
|
||||
// already exists.
|
||||
func (fs fsObjects) MakeBucketWithLocation(bucket, location string) error {
|
||||
func (fs *fsObjects) MakeBucketWithLocation(bucket, location string) error {
|
||||
bucketLock := fs.nsMutex.NewNSLock(bucket, "")
|
||||
if err := bucketLock.GetLock(globalObjectTimeout); err != nil {
|
||||
return err
|
||||
|
@ -249,7 +253,7 @@ func (fs fsObjects) MakeBucketWithLocation(bucket, location string) error {
|
|||
}
|
||||
|
||||
// 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)
|
||||
if err != nil {
|
||||
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.
|
||||
func (fs fsObjects) ListBuckets() ([]BucketInfo, error) {
|
||||
func (fs *fsObjects) ListBuckets() ([]BucketInfo, error) {
|
||||
if err := checkPathLength(fs.fsPath); err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
@ -305,7 +309,7 @@ func (fs fsObjects) ListBuckets() ([]BucketInfo, error) {
|
|||
|
||||
// DeleteBucket - delete a bucket and all the metadata associated
|
||||
// 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)
|
||||
if err != nil {
|
||||
return toObjectErr(err, bucket)
|
||||
|
@ -336,7 +340,7 @@ func (fs fsObjects) DeleteBucket(bucket string) error {
|
|||
// CopyObject - copy object source object to destination object.
|
||||
// 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, 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
|
||||
// Hold write lock on destination since in both cases
|
||||
// - 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.
|
||||
// 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 {
|
||||
return err
|
||||
}
|
||||
|
@ -454,7 +458,7 @@ func (fs fsObjects) GetObject(bucket, object string, offset int64, length int64,
|
|||
}
|
||||
|
||||
// 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 {
|
||||
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.
|
||||
func (fs fsObjects) getObjectInfo(bucket, object string) (oi ObjectInfo, e error) {
|
||||
func (fs *fsObjects) getObjectInfo(bucket, object string) (oi ObjectInfo, e error) {
|
||||
fsMeta := fsMetaV1{}
|
||||
fi, err := fsStatDir(pathJoin(fs.fsPath, bucket, object))
|
||||
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.
|
||||
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.
|
||||
objectLock := fs.nsMutex.NewNSLock(bucket, object)
|
||||
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
|
||||
// object is "a/b/c/d", stat makes sure that objects ""a/b/c""
|
||||
// "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
|
||||
isParentDirObject = func(p string) bool {
|
||||
if p == "." || p == "/" {
|
||||
|
@ -616,7 +620,7 @@ func (fs fsObjects) parentDirIsObject(bucket, parent string) bool {
|
|||
// until EOF, writes data directly to configured filesystem path.
|
||||
// Additionally writes `fs.json` which carries the necessary metadata
|
||||
// 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.
|
||||
objectLock := fs.nsMutex.NewNSLock(bucket, object)
|
||||
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
|
||||
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.
|
||||
if metadata == nil {
|
||||
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
|
||||
// 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.
|
||||
objectLock := fs.nsMutex.NewNSLock(bucket, object)
|
||||
if err := objectLock.GetLock(globalOperationTimeout); err != nil {
|
||||
|
@ -804,7 +808,7 @@ var fsTreeWalkIgnoredErrs = append(baseIgnoredErrs, []error{
|
|||
// 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 {
|
||||
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))
|
||||
|
@ -821,7 +825,7 @@ func (fs fsObjects) listDirFactory(isLeaf isLeafFunc) listDirFunc {
|
|||
|
||||
// getObjectETag is a helper function, which returns only the md5sum
|
||||
// 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)
|
||||
|
||||
// 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
|
||||
// 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 {
|
||||
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.
|
||||
func (fs fsObjects) HealObject(bucket, object string, dryRun bool) (
|
||||
func (fs *fsObjects) HealObject(bucket, object string, dryRun bool) (
|
||||
res madmin.HealResultItem, err error) {
|
||||
|
||||
return res, errors.Trace(NotImplemented{})
|
||||
}
|
||||
|
||||
// HealBucket - no-op for fs, Valid only for XL.
|
||||
func (fs fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem,
|
||||
func (fs *fsObjects) HealBucket(bucket string, dryRun bool) ([]madmin.HealResultItem,
|
||||
error) {
|
||||
|
||||
return nil, errors.Trace(NotImplemented{})
|
||||
}
|
||||
|
||||
// ListObjectsHeal - list all objects to be healed. Valid only for XL
|
||||
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{})
|
||||
}
|
||||
|
||||
// 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{})
|
||||
}
|
||||
|
|
|
@ -1374,11 +1374,12 @@ func testListObjectPartsDiskNotFound(obj ObjectLayer, instanceType string, disks
|
|||
},
|
||||
// partinfos - 2.
|
||||
{
|
||||
Bucket: bucketNames[0],
|
||||
Object: objectNames[0],
|
||||
MaxParts: 2,
|
||||
IsTruncated: false,
|
||||
UploadID: uploadIDs[0],
|
||||
Bucket: bucketNames[0],
|
||||
Object: objectNames[0],
|
||||
MaxParts: 2,
|
||||
IsTruncated: false,
|
||||
UploadID: uploadIDs[0],
|
||||
PartNumberMarker: 3,
|
||||
Parts: []PartInfo{
|
||||
{
|
||||
PartNumber: 4,
|
||||
|
@ -1611,11 +1612,12 @@ func testListObjectParts(obj ObjectLayer, instanceType string, t TestErrHandler)
|
|||
},
|
||||
// partinfos - 2.
|
||||
{
|
||||
Bucket: bucketNames[0],
|
||||
Object: objectNames[0],
|
||||
MaxParts: 2,
|
||||
IsTruncated: false,
|
||||
UploadID: uploadIDs[0],
|
||||
Bucket: bucketNames[0],
|
||||
Object: objectNames[0],
|
||||
MaxParts: 2,
|
||||
IsTruncated: false,
|
||||
UploadID: uploadIDs[0],
|
||||
PartNumberMarker: 3,
|
||||
Parts: []PartInfo{
|
||||
{
|
||||
PartNumber: 4,
|
||||
|
|
|
@ -909,6 +909,7 @@ func (api objectAPIHandlers) AbortMultipartUploadHandler(w http.ResponseWriter,
|
|||
|
||||
uploadID, _, _, _ := getObjectResources(r.URL.Query())
|
||||
if err := objectAPI.AbortMultipartUpload(bucket, object, uploadID); err != nil {
|
||||
errorIf(err, "AbortMultipartUpload failed")
|
||||
writeErrorResponse(w, toAPIErrorCode(err), r.URL)
|
||||
return
|
||||
}
|
||||
|
|
|
@ -833,6 +833,7 @@ func (xl xlObjects) listObjectParts(bucket, object, uploadID string, partNumberM
|
|||
result.Object = object
|
||||
result.UploadID = uploadID
|
||||
result.MaxParts = maxParts
|
||||
result.PartNumberMarker = partNumberMarker
|
||||
|
||||
// For empty number of parts or maxParts as zero, return right here.
|
||||
if len(xlParts) == 0 || maxParts == 0 {
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -20,8 +20,13 @@ package ioutil
|
|||
|
||||
import (
|
||||
"io"
|
||||
|
||||
humanize "github.com/dustin/go-humanize"
|
||||
)
|
||||
|
||||
// defaultAppendBufferSize - Default buffer size for the AppendFile
|
||||
const defaultAppendBufferSize = humanize.MiByte
|
||||
|
||||
// WriteOnCloser implements io.WriteCloser and always
|
||||
// exectues at least one write operation if it is closed.
|
||||
//
|
||||
|
|
|
@ -18,6 +18,7 @@ package ioutil
|
|||
|
||||
import (
|
||||
goioutil "io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
)
|
||||
|
||||
|
@ -37,3 +38,38 @@ func TestCloseOnWriter(t *testing.T) {
|
|||
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))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
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)
|
||||
}
|
||||
|
|
|
@ -42,7 +42,7 @@ const (
|
|||
|
||||
// lockedOpenFile is an internal function.
|
||||
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 {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -168,7 +168,7 @@ func fixLongPath(path string) string {
|
|||
// 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) {
|
||||
func Open(path string, flag int, perm os.FileMode) (*os.File, error) {
|
||||
if path == "" {
|
||||
return nil, syscall.ERROR_FILE_NOT_FOUND
|
||||
}
|
||||
|
@ -190,6 +190,8 @@ func open(path string, flag int, perm os.FileMode) (*os.File, error) {
|
|||
fallthrough
|
||||
case syscall.O_WRONLY | syscall.O_CREAT:
|
||||
access = syscall.GENERIC_READ | syscall.GENERIC_WRITE
|
||||
case syscall.O_WRONLY | syscall.O_CREAT | syscall.O_APPEND:
|
||||
access = syscall.FILE_APPEND_DATA
|
||||
default:
|
||||
return nil, fmt.Errorf("Unsupported flag (%d)", flag)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue