mirror of
https://github.com/minio/minio.git
synced 2025-04-04 03:40:30 -04:00
Lock free multipart backend implementation for FS (#5401)
This commit is contained in:
parent
018813b98f
commit
3b2486ebaf
@ -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.
|
||||||
|
@ -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.
|
||||||
|
@ -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"
|
"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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
@ -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{}) {
|
||||||
|
80
cmd/fs-v1.go
80
cmd/fs-v1.go
@ -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{})
|
||||||
}
|
}
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
}
|
}
|
||||||
|
@ -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 {
|
||||||
|
43
pkg/ioutil/append-file_nix.go
Normal file
43
pkg/ioutil/append-file_nix.go
Normal 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
|
||||||
|
}
|
43
pkg/ioutil/append-file_windows.go
Normal file
43
pkg/ioutil/append-file_windows.go
Normal 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
|
||||||
|
}
|
@ -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.
|
||||||
//
|
//
|
||||||
|
@ -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))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -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)
|
||||||
|
}
|
||||||
|
@ -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)
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user