Implement S3-HDFS gateway (#7440)

- [x] Support bucket and regular object operations
- [x] Supports Select API on HDFS
- [x] Implement multipart API support
- [x] Completion of ListObjects support
This commit is contained in:
Harshavardhana
2019-04-17 09:52:08 -07:00
committed by kannappanr
parent 1d49295943
commit 620e462413
19 changed files with 1088 additions and 291 deletions

View File

@@ -87,7 +87,7 @@ func newCacheFSObjects(dir string, expiry int, maxDiskUsagePct int) (*cacheFSObj
readersMap: make(map[string]*lock.RLockedFile),
},
nsMutex: newNSLock(false),
listPool: newTreeWalkPool(globalLookupTimeout),
listPool: NewTreeWalkPool(globalLookupTimeout),
appendFileMap: make(map[string]*fsAppendFile),
}

View File

@@ -53,7 +53,7 @@ type cacheObjects struct {
// pointer to disk cache
cache *diskCache
// ListObjects pool management.
listPool *treeWalkPool
listPool *TreeWalkPool
// file path patterns to exclude from cache
exclude []string
// Object functions pointing to the corresponding functions of backend implementation.
@@ -372,7 +372,7 @@ func (c cacheObjects) GetObjectInfo(ctx context.Context, bucket, object string,
// 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.
// disks - list of fsObjects
func listDirCacheFactory(isLeaf isLeafFunc, disks []*cacheFSObjects) listDirFunc {
func listDirCacheFactory(isLeaf IsLeafFunc, disks []*cacheFSObjects) ListDirFunc {
listCacheDirs := func(bucket, prefixDir, prefixEntry string) (dirs []string) {
var entries []string
for _, disk := range disks {
@@ -962,7 +962,7 @@ func newServerCacheObjects(config CacheConfig) (CacheObjectLayer, error) {
return &cacheObjects{
cache: dcache,
exclude: config.Exclude,
listPool: newTreeWalkPool(globalLookupTimeout),
listPool: NewTreeWalkPool(globalLookupTimeout),
GetObjectFn: func(ctx context.Context, bucket, object string, startOffset int64, length int64, writer io.Writer, etag string, opts ObjectOptions) error {
return newObjectLayerFn().GetObject(ctx, bucket, object, startOffset, length, writer, etag, opts)
},

View File

@@ -62,7 +62,7 @@ type FSObjects struct {
rwPool *fsIOPool
// ListObjects pool management.
listPool *treeWalkPool
listPool *TreeWalkPool
diskMount bool
@@ -141,7 +141,7 @@ func NewFSObjectLayer(fsPath string) (ObjectLayer, error) {
readersMap: make(map[string]*lock.RLockedFile),
},
nsMutex: newNSLock(false),
listPool: newTreeWalkPool(globalLookupTimeout),
listPool: NewTreeWalkPool(globalLookupTimeout),
appendFileMap: make(map[string]*fsAppendFile),
diskMount: mountinfo.IsLikelyMountPoint(fsPath),
}
@@ -1001,7 +1001,7 @@ func (fs *FSObjects) DeleteObject(ctx context.Context, bucket, object string) er
// 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) {
var err error
@@ -1097,134 +1097,22 @@ func (fs *FSObjects) getObjectETag(ctx context.Context, bucket, entry string, lo
// 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(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, e error) {
if err := checkListObjsArgs(ctx, bucket, prefix, marker, delimiter, fs); err != nil {
return loi, err
isLeaf := func(bucket, object string) bool {
// bucket argument is unused as we don't need to StatFile
// to figure if it's a file, just need to check that the
// object string does not end with "/".
return !hasSuffix(object, slashSeparator)
}
// Marker is set validate pre-condition.
if marker != "" {
// Marker not common with prefix is not implemented.Send an empty response
if !hasPrefix(marker, prefix) {
return ListObjectsInfo{}, e
// Return true if the specified object is an empty directory
isLeafDir := func(bucket, object string) bool {
if !hasSuffix(object, slashSeparator) {
return false
}
return fs.isObjectDir(bucket, object)
}
if _, err := fs.statBucketDir(ctx, bucket); err != nil {
return loi, err
}
listDir := fs.listDirFactory(isLeaf)
// With max keys of zero we have reached eof, return right here.
if maxKeys == 0 {
return loi, nil
}
// For delimiter and prefix as '/' we do not list anything at all
// since according to s3 spec we stop at the 'delimiter'
// along // with the prefix. On a flat namespace with 'prefix'
// as '/' we don't have any entries, since all the keys are
// of form 'keyName/...'
if delimiter == slashSeparator && prefix == slashSeparator {
return loi, nil
}
// Over flowing count - reset to maxObjectList.
if maxKeys < 0 || maxKeys > maxObjectList {
maxKeys = maxObjectList
}
// Default is recursive, if delimiter is set then list non recursive.
recursive := true
if delimiter == slashSeparator {
recursive = false
}
// Convert entry to ObjectInfo
entryToObjectInfo := func(entry string) (objInfo ObjectInfo, err error) {
// Protect the entry from concurrent deletes, or renames.
objectLock := fs.nsMutex.NewNSLock(bucket, entry)
if err = objectLock.GetRLock(globalListingTimeout); err != nil {
logger.LogIf(ctx, err)
return ObjectInfo{}, err
}
defer objectLock.RUnlock()
return fs.getObjectInfo(ctx, bucket, entry)
}
walkResultCh, endWalkCh := fs.listPool.Release(listParams{bucket, recursive, marker, prefix})
if walkResultCh == nil {
endWalkCh = make(chan struct{})
isLeaf := func(bucket, object string) bool {
// bucket argument is unused as we don't need to StatFile
// to figure if it's a file, just need to check that the
// object string does not end with "/".
return !hasSuffix(object, slashSeparator)
}
// Return true if the specified object is an empty directory
isLeafDir := func(bucket, object string) bool {
if !hasSuffix(object, slashSeparator) {
return false
}
return fs.isObjectDir(bucket, object)
}
listDir := fs.listDirFactory(isLeaf)
walkResultCh = startTreeWalk(ctx, bucket, prefix, marker, recursive, listDir, isLeaf, isLeafDir, endWalkCh)
}
var objInfos []ObjectInfo
var eof bool
var nextMarker string
// List until maxKeys requested.
for i := 0; i < maxKeys; {
walkResult, ok := <-walkResultCh
if !ok {
// Closed channel.
eof = true
break
}
// For any walk error return right away.
if walkResult.err != nil {
// File not found is a valid case.
if walkResult.err == errFileNotFound {
return loi, nil
}
return loi, toObjectErr(walkResult.err, bucket, prefix)
}
objInfo, err := entryToObjectInfo(walkResult.entry)
if err != nil {
return loi, nil
}
nextMarker = objInfo.Name
objInfos = append(objInfos, objInfo)
if walkResult.end {
eof = true
break
}
i++
}
// Save list routine for the next marker if we haven't reached EOF.
params := listParams{bucket, recursive, nextMarker, prefix}
if !eof {
fs.listPool.Set(params, walkResultCh, endWalkCh)
}
result := ListObjectsInfo{}
for _, objInfo := range objInfos {
if objInfo.IsDir && delimiter == slashSeparator {
result.Prefixes = append(result.Prefixes, objInfo.Name)
continue
}
result.Objects = append(result.Objects, objInfo)
}
if !eof {
result.IsTruncated = true
if len(objInfos) > 0 {
result.NextMarker = objInfos[len(objInfos)-1].Name
}
}
// Success.
return result, nil
return listObjects(ctx, fs, bucket, prefix, marker, delimiter, maxKeys, fs.listPool, isLeaf, isLeafDir, listDir, fs.getObjectInfo, fs.getObjectInfo)
}
// ReloadFormat - no-op for fs, Valid only for XL.

View File

@@ -36,6 +36,21 @@ var (
// CleanMetadataKeys provides cleanMetadataKeys function alias.
CleanMetadataKeys = cleanMetadataKeys
// PathJoin function alias.
PathJoin = pathJoin
// ListObjects function alias.
ListObjects = listObjects
// FilterListEntries function alias.
FilterListEntries = filterListEntries
// IsStringEqual is string equal.
IsStringEqual = isStringEqual
// GetCompleteMultipartMD5 returns multipart MD5
GetCompleteMultipartMD5 = getCompleteMultipartMD5
)
// StatInfo - alias for statInfo

View File

@@ -20,6 +20,7 @@ import (
// Import all gateways.
_ "github.com/minio/minio/cmd/gateway/azure"
_ "github.com/minio/minio/cmd/gateway/gcs"
_ "github.com/minio/minio/cmd/gateway/hdfs"
_ "github.com/minio/minio/cmd/gateway/nas"
_ "github.com/minio/minio/cmd/gateway/oss"
_ "github.com/minio/minio/cmd/gateway/s3"

View File

@@ -0,0 +1,67 @@
/*
* Minio Cloud Storage, (C) 2019 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 hdfs
import (
"strings"
"github.com/minio/minio-go/pkg/s3utils"
minio "github.com/minio/minio/cmd"
)
const (
// Minio meta bucket.
minioMetaBucket = ".minio.sys"
// Minio Tmp meta prefix.
minioMetaTmpBucket = minioMetaBucket + "/tmp"
// Minio reserved bucket name.
minioReservedBucket = "minio"
)
// Ignores all reserved bucket names or invalid bucket names.
func isReservedOrInvalidBucket(bucketEntry string, strict bool) bool {
bucketEntry = strings.TrimSuffix(bucketEntry, "/")
if strict {
if err := s3utils.CheckValidBucketNameStrict(bucketEntry); err != nil {
return true
}
} else {
if err := s3utils.CheckValidBucketName(bucketEntry); err != nil {
return true
}
}
return isMinioMetaBucket(bucketEntry) || isMinioReservedBucket(bucketEntry)
}
// Returns true if input bucket is a reserved minio meta bucket '.minio.sys'.
func isMinioMetaBucket(bucketName string) bool {
return bucketName == minioMetaBucket
}
// Returns true if input bucket is a reserved minio bucket 'minio'.
func isMinioReservedBucket(bucketName string) bool {
return bucketName == minioReservedBucket
}
// byBucketName is a collection satisfying sort.Interface.
type byBucketName []minio.BucketInfo
func (d byBucketName) Len() int { return len(d) }
func (d byBucketName) Swap(i, j int) { d[i], d[j] = d[j], d[i] }
func (d byBucketName) Less(i, j int) bool { return d[i].Name < d[j].Name }

View File

@@ -0,0 +1,700 @@
/*
* Minio Cloud Storage, (C) 2019 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 hdfs
import (
"context"
"io"
"net"
"net/http"
"os"
"os/user"
"path"
"sort"
"strings"
"syscall"
"time"
"github.com/colinmarc/hdfs/v2"
"github.com/minio/cli"
"github.com/minio/minio-go/pkg/s3utils"
minio "github.com/minio/minio/cmd"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/auth"
xnet "github.com/minio/minio/pkg/net"
)
const (
hdfsBackend = "hdfs"
hdfsSeparator = "/"
)
func init() {
const hdfsGatewayTemplate = `NAME:
{{.HelpName}} - {{.Usage}}
USAGE:
{{.HelpName}} {{if .VisibleFlags}}[FLAGS]{{end}} HDFS-NAMENODE [HDFS-NAMENODE...]
{{if .VisibleFlags}}
FLAGS:
{{range .VisibleFlags}}{{.}}
{{end}}{{end}}
HDFS-NAMENODE:
HDFS namenode URI
ENVIRONMENT VARIABLES:
ACCESS:
MINIO_ACCESS_KEY: Username or access key of minimum 3 characters in length.
MINIO_SECRET_KEY: Password or secret key of minimum 8 characters in length.
BROWSER:
MINIO_BROWSER: To disable web browser access, set this value to "off".
DOMAIN:
MINIO_DOMAIN: To enable virtual-host-style requests, set this value to Minio host domain name.
CACHE:
MINIO_CACHE_DRIVES: List of mounted drives or directories delimited by ";".
MINIO_CACHE_EXCLUDE: List of cache exclusion patterns delimited by ";".
MINIO_CACHE_EXPIRY: Cache expiry duration in days.
MINIO_CACHE_MAXUSE: Maximum permitted usage of the cache in percentage (0-100).
EXAMPLES:
1. Start minio gateway server for HDFS backend.
$ export MINIO_ACCESS_KEY=accesskey
$ export MINIO_SECRET_KEY=secretkey
$ {{.HelpName}} hdfs://namenode:8200
2. Start minio gateway server for HDFS with edge caching enabled.
$ export MINIO_ACCESS_KEY=accesskey
$ export MINIO_SECRET_KEY=secretkey
$ export MINIO_CACHE_DRIVES="/mnt/drive1;/mnt/drive2;/mnt/drive3;/mnt/drive4"
$ export MINIO_CACHE_EXCLUDE="bucket1/*;*.png"
$ export MINIO_CACHE_EXPIRY=40
$ export MINIO_CACHE_MAXUSE=80
$ {{.HelpName}} hdfs://namenode:8200
`
minio.RegisterGatewayCommand(cli.Command{
Name: hdfsBackend,
Usage: "Hadoop Distributed File System (HDFS)",
Action: hdfsGatewayMain,
CustomHelpTemplate: hdfsGatewayTemplate,
HideHelpCommand: true,
})
}
// Handler for 'minio gateway hdfs' command line.
func hdfsGatewayMain(ctx *cli.Context) {
// Validate gateway arguments.
if !ctx.Args().Present() || ctx.Args().First() == "help" {
cli.ShowCommandHelpAndExit(ctx, hdfsBackend, 1)
}
minio.StartGateway(ctx, &HDFS{args: ctx.Args()})
}
// HDFS implements Gateway.
type HDFS struct {
args []string
}
// Name implements Gateway interface.
func (g *HDFS) Name() string {
return hdfsBackend
}
// NewGatewayLayer returns hdfs gatewaylayer.
func (g *HDFS) NewGatewayLayer(creds auth.Credentials) (minio.ObjectLayer, error) {
dialFunc := (&net.Dialer{
Timeout: 30 * time.Second,
KeepAlive: 30 * time.Second,
DualStack: true,
}).DialContext
var addresses []string
for _, s := range g.args {
u, err := xnet.ParseURL(s)
if err != nil {
return nil, err
}
addresses = append(addresses, u.Host)
}
user, err := user.Current()
if err != nil {
return nil, err
}
opts := hdfs.ClientOptions{
Addresses: addresses,
User: user.Username,
NamenodeDialFunc: dialFunc,
DatanodeDialFunc: dialFunc,
}
clnt, err := hdfs.NewClient(opts)
if err != nil {
return nil, err
}
if err = clnt.MkdirAll(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket), os.FileMode(0755)); err != nil {
return nil, err
}
return &hdfsObjects{clnt: clnt, listPool: minio.NewTreeWalkPool(time.Minute * 30)}, nil
}
// Production - hdfs gateway is production ready.
func (g *HDFS) Production() bool {
return false
}
func (n *hdfsObjects) Shutdown(ctx context.Context) error {
return n.clnt.Close()
}
func (n *hdfsObjects) StorageInfo(ctx context.Context) minio.StorageInfo {
fsInfo, err := n.clnt.StatFs()
if err != nil {
return minio.StorageInfo{}
}
sinfo := minio.StorageInfo{}
sinfo.Used = fsInfo.Used
sinfo.Backend.Type = minio.Unknown
return sinfo
}
// hdfsObjects implements gateway for Minio and S3 compatible object storage servers.
type hdfsObjects struct {
minio.GatewayUnsupported
clnt *hdfs.Client
listPool *minio.TreeWalkPool
}
func hdfsToObjectErr(ctx context.Context, err error, params ...string) error {
if err == nil {
return nil
}
bucket := ""
object := ""
uploadID := ""
switch len(params) {
case 3:
uploadID = params[2]
fallthrough
case 2:
object = params[1]
fallthrough
case 1:
bucket = params[0]
}
switch {
case os.IsNotExist(err):
if uploadID != "" {
return minio.InvalidUploadID{
UploadID: uploadID,
}
}
if object != "" {
return minio.ObjectNotFound{Bucket: bucket, Object: object}
}
return minio.BucketNotFound{Bucket: bucket}
case os.IsExist(err):
if object != "" {
return minio.PrefixAccessDenied{Bucket: bucket, Object: object}
}
return minio.BucketAlreadyOwnedByYou{Bucket: bucket}
case isSysErrNotEmpty(err):
if object != "" {
return minio.PrefixAccessDenied{Bucket: bucket, Object: object}
}
return minio.BucketNotEmpty{Bucket: bucket}
default:
logger.LogIf(ctx, err)
return err
}
}
// hdfsIsValidBucketName verifies whether a bucket name is valid.
func hdfsIsValidBucketName(bucket string) bool {
return s3utils.CheckValidBucketNameStrict(bucket) == nil
}
func (n *hdfsObjects) DeleteBucket(ctx context.Context, bucket string) error {
if !hdfsIsValidBucketName(bucket) {
return minio.BucketNameInvalid{Bucket: bucket}
}
return hdfsToObjectErr(ctx, n.clnt.Remove(minio.PathJoin(hdfsSeparator, bucket)), bucket)
}
func (n *hdfsObjects) MakeBucketWithLocation(ctx context.Context, bucket, location string) error {
if !hdfsIsValidBucketName(bucket) {
return minio.BucketNameInvalid{Bucket: bucket}
}
return hdfsToObjectErr(ctx, n.clnt.Mkdir(minio.PathJoin(hdfsSeparator, bucket), os.FileMode(0755)), bucket)
}
func (n *hdfsObjects) GetBucketInfo(ctx context.Context, bucket string) (bi minio.BucketInfo, err error) {
fi, err := n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return bi, hdfsToObjectErr(ctx, err, bucket)
}
// As hdfs.Stat() doesn't carry anything other than ModTime(), use ModTime() as CreatedTime.
return minio.BucketInfo{
Name: bucket,
Created: fi.ModTime(),
}, nil
}
func (n *hdfsObjects) ListBuckets(ctx context.Context) (buckets []minio.BucketInfo, err error) {
entries, err := n.clnt.ReadDir(hdfsSeparator)
if err != nil {
logger.LogIf(ctx, err)
return nil, hdfsToObjectErr(ctx, err)
}
for _, entry := range entries {
// Ignore all reserved bucket names and invalid bucket names.
if isReservedOrInvalidBucket(entry.Name(), false) {
continue
}
buckets = append(buckets, minio.BucketInfo{
Name: entry.Name(),
// As hdfs.Stat() doesnt carry CreatedTime, use ModTime() as CreatedTime.
Created: entry.ModTime(),
})
}
// Sort bucket infos by bucket name.
sort.Sort(byBucketName(buckets))
return buckets, nil
}
func (n *hdfsObjects) isObjectDir(bucket, object string) bool {
f, err := n.clnt.Open(minio.PathJoin(hdfsSeparator, bucket, object))
if err != nil {
return false
}
defer f.Close()
entries, err := f.Readdir(1)
if err != nil {
return false
}
return len(entries) == 0
}
func (n *hdfsObjects) listDirFactory(isLeaf minio.IsLeafFunc) minio.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) {
f, err := n.clnt.Open(minio.PathJoin(hdfsSeparator, bucket, prefixDir))
if err != nil {
if os.IsNotExist(err) {
err = nil
}
logger.LogIf(context.Background(), err)
return
}
defer f.Close()
fis, err := f.Readdir(0)
if err != nil {
logger.LogIf(context.Background(), err)
return
}
for _, fi := range fis {
if fi.IsDir() {
entries = append(entries, fi.Name()+hdfsSeparator)
} else {
entries = append(entries, fi.Name())
}
}
fis = nil
entries, delayIsLeaf = minio.FilterListEntries(bucket, prefixDir, entries, prefixEntry, isLeaf)
return entries, delayIsLeaf
}
// Return list factory instance.
return listDir
}
// ListObjects lists all blobs in HDFS bucket filtered by prefix.
func (n *hdfsObjects) ListObjects(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (loi minio.ListObjectsInfo, err error) {
isLeaf := func(bucket, object string) bool {
// bucket argument is unused as we don't need to StatFile
// to figure if it's a file, just need to check that the
// object string does not end with "/".
return !strings.HasSuffix(object, hdfsSeparator)
}
// Return true if the specified object is an empty directory
isLeafDir := func(bucket, object string) bool {
if !strings.HasSuffix(object, hdfsSeparator) {
return false
}
return n.isObjectDir(bucket, object)
}
listDir := n.listDirFactory(isLeaf)
getObjectInfo := func(ctx context.Context, bucket, entry string) (minio.ObjectInfo, error) {
return n.GetObjectInfo(ctx, bucket, entry, minio.ObjectOptions{})
}
return minio.ListObjects(ctx, n, bucket, prefix, marker, delimiter, maxKeys, n.listPool, isLeaf, isLeafDir, listDir, getObjectInfo, getObjectInfo)
}
// Check if the given error corresponds to ENOTEMPTY for unix
// and ERROR_DIR_NOT_EMPTY for windows (directory not empty).
func isSysErrNotEmpty(err error) bool {
if err == syscall.ENOTEMPTY {
return true
}
if pathErr, ok := err.(*os.PathError); ok {
if pathErr.Err == syscall.ENOTEMPTY {
return true
}
}
return false
}
// deleteObject deletes a file path if its empty. If it's successfully deleted,
// it will recursively move up the tree, deleting empty parent directories
// until it finds one with files in it. Returns nil for a non-empty directory.
func (n *hdfsObjects) deleteObject(basePath, deletePath string) error {
if basePath == deletePath {
return nil
}
// Attempt to remove path.
if err := n.clnt.Remove(deletePath); err != nil {
switch {
case err == syscall.ENOTEMPTY:
case isSysErrNotEmpty(err):
// Ignore errors if the directory is not empty. The server relies on
// this functionality, and sometimes uses recursion that should not
// error on parent directories.
return nil
default:
return err
}
}
// Trailing slash is removed when found to ensure
// slashpath.Dir() to work as intended.
deletePath = strings.TrimSuffix(deletePath, hdfsSeparator)
deletePath = path.Dir(deletePath)
// Delete parent directory. Errors for parent directories shouldn't trickle down.
n.deleteObject(basePath, deletePath)
return nil
}
// ListObjectsV2 lists all blobs in HDFS bucket filtered by prefix
func (n *hdfsObjects) ListObjectsV2(ctx context.Context, bucket, prefix, continuationToken, delimiter string, maxKeys int,
fetchOwner bool, startAfter string) (loi minio.ListObjectsV2Info, err error) {
// fetchOwner is not supported and unused.
marker := continuationToken
if marker == "" {
marker = startAfter
}
resultV1, err := n.ListObjects(ctx, bucket, prefix, marker, delimiter, maxKeys)
if err != nil {
return loi, err
}
return minio.ListObjectsV2Info{
Objects: resultV1.Objects,
Prefixes: resultV1.Prefixes,
ContinuationToken: continuationToken,
NextContinuationToken: resultV1.NextMarker,
IsTruncated: resultV1.IsTruncated,
}, nil
}
func (n *hdfsObjects) DeleteObject(ctx context.Context, bucket, object string) error {
return hdfsToObjectErr(ctx, n.deleteObject(minio.PathJoin(hdfsSeparator, bucket), minio.PathJoin(hdfsSeparator, bucket, object)), bucket, object)
}
func (n *hdfsObjects) GetObjectNInfo(ctx context.Context, bucket, object string, rs *minio.HTTPRangeSpec, h http.Header, lockType minio.LockType, opts minio.ObjectOptions) (gr *minio.GetObjectReader, err error) {
objInfo, err := n.GetObjectInfo(ctx, bucket, object, opts)
if err != nil {
return nil, err
}
var startOffset, length int64
startOffset, length, err = rs.GetOffsetLength(objInfo.Size)
if err != nil {
return nil, err
}
pr, pw := io.Pipe()
go func() {
nerr := n.GetObject(ctx, bucket, object, startOffset, length, pw, objInfo.ETag, opts)
pw.CloseWithError(nerr)
}()
// Setup cleanup function to cause the above go-routine to
// exit in case of partial read
pipeCloser := func() { pr.Close() }
return minio.NewGetObjectReaderFromReader(pr, objInfo, opts.CheckCopyPrecondFn, pipeCloser)
}
func (n *hdfsObjects) CopyObject(ctx context.Context, srcBucket, srcObject, dstBucket, dstObject string, srcInfo minio.ObjectInfo, srcOpts, dstOpts minio.ObjectOptions) (minio.ObjectInfo, error) {
cpSrcDstSame := minio.IsStringEqual(minio.PathJoin(hdfsSeparator, srcBucket, srcObject), minio.PathJoin(hdfsSeparator, dstBucket, dstObject))
if cpSrcDstSame {
return n.GetObjectInfo(ctx, srcBucket, srcObject, minio.ObjectOptions{})
}
return n.PutObject(ctx, dstBucket, dstObject, srcInfo.PutObjReader, minio.ObjectOptions{
ServerSideEncryption: dstOpts.ServerSideEncryption,
UserDefined: srcInfo.UserDefined,
})
}
func (n *hdfsObjects) GetObject(ctx context.Context, bucket, key string, startOffset, length int64, writer io.Writer, etag string, opts minio.ObjectOptions) error {
if _, err := n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket)); err != nil {
return hdfsToObjectErr(ctx, err, bucket)
}
rd, err := n.clnt.Open(minio.PathJoin(hdfsSeparator, bucket, key))
if err != nil {
return hdfsToObjectErr(ctx, err, bucket, key)
}
if _, err = rd.Seek(startOffset, io.SeekStart); err != nil {
return hdfsToObjectErr(ctx, err, bucket, key)
}
if _, err = io.Copy(writer, io.LimitReader(rd, length)); err != nil {
return hdfsToObjectErr(ctx, err, bucket, key)
}
return nil
}
// GetObjectInfo reads object info and replies back ObjectInfo.
func (n *hdfsObjects) GetObjectInfo(ctx context.Context, bucket, object string, opts minio.ObjectOptions) (objInfo minio.ObjectInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket)
}
fi, err := n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket, object))
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
return minio.ObjectInfo{
Bucket: bucket,
Name: object,
ModTime: fi.ModTime(),
Size: fi.Size(),
IsDir: fi.IsDir(),
AccTime: fi.(*hdfs.FileInfo).AccessTime(),
}, nil
}
func (n *hdfsObjects) PutObject(ctx context.Context, bucket string, object string, r *minio.PutObjReader, opts minio.ObjectOptions) (objInfo minio.ObjectInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket)
}
name := minio.PathJoin(hdfsSeparator, bucket, object)
// If its a directory create a prefix {
if strings.HasSuffix(object, hdfsSeparator) {
if err = n.clnt.MkdirAll(name, os.FileMode(0755)); err != nil {
n.deleteObject(minio.PathJoin(hdfsSeparator, bucket), name)
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
} else {
tmpname := minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, minio.MustGetUUID())
var w *hdfs.FileWriter
w, err = n.clnt.Create(tmpname)
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
defer n.deleteObject(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket), tmpname)
defer w.Close()
if _, err = io.Copy(w, r); err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
dir := path.Dir(name)
if dir != "" {
if err = n.clnt.MkdirAll(dir, os.FileMode(0755)); err != nil {
n.deleteObject(minio.PathJoin(hdfsSeparator, bucket), dir)
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
}
if err = n.clnt.Rename(tmpname, name); err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
}
fi, err := n.clnt.Stat(name)
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
return minio.ObjectInfo{
Bucket: bucket,
Name: object,
ETag: r.MD5CurrentHexString(),
ModTime: fi.ModTime(),
Size: fi.Size(),
IsDir: fi.IsDir(),
AccTime: fi.(*hdfs.FileInfo).AccessTime(),
}, nil
}
func (n *hdfsObjects) NewMultipartUpload(ctx context.Context, bucket string, object string, opts minio.ObjectOptions) (uploadID string, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return uploadID, hdfsToObjectErr(ctx, err, bucket)
}
uploadID = minio.MustGetUUID()
if err = n.clnt.CreateEmptyFile(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID)); err != nil {
return uploadID, hdfsToObjectErr(ctx, err, bucket)
}
return uploadID, nil
}
func (n *hdfsObjects) ListMultipartUploads(ctx context.Context, bucket string, prefix string, keyMarker string, uploadIDMarker string, delimiter string, maxUploads int) (lmi minio.ListMultipartsInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return lmi, hdfsToObjectErr(ctx, err, bucket)
}
// It's decided not to support List Multipart Uploads, hence returning empty result.
return lmi, nil
}
func (n *hdfsObjects) checkUploadIDExists(ctx context.Context, bucket, object, uploadID string) (err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID))
if err != nil {
return hdfsToObjectErr(ctx, err, bucket, object, uploadID)
}
return nil
}
func (n *hdfsObjects) ListObjectParts(ctx context.Context, bucket, object, uploadID string, partNumberMarker int, maxParts int, opts minio.ObjectOptions) (result minio.ListPartsInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return result, hdfsToObjectErr(ctx, err, bucket)
}
if err = n.checkUploadIDExists(ctx, bucket, object, uploadID); err != nil {
return result, err
}
// It's decided not to support List parts, hence returning empty result.
return result, nil
}
func (n *hdfsObjects) CopyObjectPart(ctx context.Context, srcBucket, srcObject, dstBucket, dstObject, uploadID string, partID int,
startOffset int64, length int64, srcInfo minio.ObjectInfo, srcOpts, dstOpts minio.ObjectOptions) (minio.PartInfo, error) {
return n.PutObjectPart(ctx, dstBucket, dstObject, uploadID, partID, srcInfo.PutObjReader, dstOpts)
}
func (n *hdfsObjects) PutObjectPart(ctx context.Context, bucket, object, uploadID string, partID int, r *minio.PutObjReader, opts minio.ObjectOptions) (info minio.PartInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return info, hdfsToObjectErr(ctx, err, bucket)
}
var w *hdfs.FileWriter
w, err = n.clnt.Append(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID))
if err != nil {
return info, hdfsToObjectErr(ctx, err, bucket, object, uploadID)
}
defer w.Close()
_, err = io.Copy(w, r.Reader)
if err != nil {
return info, hdfsToObjectErr(ctx, err, bucket, object, uploadID)
}
info.PartNumber = partID
info.ETag = r.MD5CurrentHexString()
info.LastModified = minio.UTCNow()
info.Size = r.Reader.Size()
return info, nil
}
func (n *hdfsObjects) CompleteMultipartUpload(ctx context.Context, bucket, object, uploadID string, parts []minio.CompletePart, opts minio.ObjectOptions) (objInfo minio.ObjectInfo, err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket)
}
if err = n.checkUploadIDExists(ctx, bucket, object, uploadID); err != nil {
return objInfo, err
}
name := minio.PathJoin(hdfsSeparator, bucket, object)
dir := path.Dir(name)
if dir != "" {
if err = n.clnt.MkdirAll(dir, os.FileMode(0755)); err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
}
err = n.clnt.Rename(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID), name)
// Object already exists is an error on HDFS
// remove it and then create it again.
if os.IsExist(err) {
if err = n.clnt.Remove(name); err != nil {
if dir != "" {
n.deleteObject(minio.PathJoin(hdfsSeparator, bucket), dir)
}
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
if err = n.clnt.Rename(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID), name); err != nil {
if dir != "" {
n.deleteObject(minio.PathJoin(hdfsSeparator, bucket), dir)
}
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
}
fi, err := n.clnt.Stat(name)
if err != nil {
return objInfo, hdfsToObjectErr(ctx, err, bucket, object)
}
// Calculate s3 compatible md5sum for complete multipart.
s3MD5, err := minio.GetCompleteMultipartMD5(ctx, parts)
if err != nil {
return objInfo, err
}
return minio.ObjectInfo{
Bucket: bucket,
Name: object,
ETag: s3MD5,
ModTime: fi.ModTime(),
Size: fi.Size(),
IsDir: fi.IsDir(),
AccTime: fi.(*hdfs.FileInfo).AccessTime(),
}, nil
}
func (n *hdfsObjects) AbortMultipartUpload(ctx context.Context, bucket, object, uploadID string) (err error) {
_, err = n.clnt.Stat(minio.PathJoin(hdfsSeparator, bucket))
if err != nil {
return hdfsToObjectErr(ctx, err, bucket)
}
return hdfsToObjectErr(ctx, n.clnt.Remove(minio.PathJoin(hdfsSeparator, minioMetaTmpBucket, uploadID)), bucket, object, uploadID)
}

View File

@@ -1,5 +1,5 @@
/*
* MinIO Cloud Storage, (C) 2016, 2017, 2018 MinIO, Inc.
* MinIO Cloud Storage, (C) 2016-2019 MinIO, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -161,3 +161,127 @@ func removeListenerConfig(ctx context.Context, objAPI ObjectLayer, bucket string
lcPath := path.Join(bucketConfigPrefix, bucket, bucketListenerConfig)
return objAPI.DeleteObject(ctx, minioMetaBucket, lcPath)
}
func listObjects(ctx context.Context, obj ObjectLayer, bucket, prefix, marker, delimiter string, maxKeys int, tpool *TreeWalkPool, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc, listDir ListDirFunc, getObjInfo func(context.Context, string, string) (ObjectInfo, error), getObjectInfoDirs ...func(context.Context, string, string) (ObjectInfo, error)) (loi ListObjectsInfo, err error) {
if err := checkListObjsArgs(ctx, bucket, prefix, marker, delimiter, obj); err != nil {
return loi, err
}
// Marker is set validate pre-condition.
if marker != "" {
// Marker not common with prefix is not implemented. Send an empty response
if !hasPrefix(marker, prefix) {
return loi, nil
}
}
// With max keys of zero we have reached eof, return right here.
if maxKeys == 0 {
return loi, nil
}
// For delimiter and prefix as '/' we do not list anything at all
// since according to s3 spec we stop at the 'delimiter'
// along // with the prefix. On a flat namespace with 'prefix'
// as '/' we don't have any entries, since all the keys are
// of form 'keyName/...'
if delimiter == slashSeparator && prefix == slashSeparator {
return loi, nil
}
// Over flowing count - reset to maxObjectList.
if maxKeys < 0 || maxKeys > maxObjectList {
maxKeys = maxObjectList
}
// Default is recursive, if delimiter is set then list non recursive.
recursive := true
if delimiter == slashSeparator {
recursive = false
}
walkResultCh, endWalkCh := tpool.Release(listParams{bucket, recursive, marker, prefix})
if walkResultCh == nil {
endWalkCh = make(chan struct{})
walkResultCh = startTreeWalk(ctx, bucket, prefix, marker, recursive, listDir, isLeaf, isLeafDir, endWalkCh)
}
var objInfos []ObjectInfo
var eof bool
var nextMarker string
// List until maxKeys requested.
for i := 0; i < maxKeys; {
walkResult, ok := <-walkResultCh
if !ok {
// Closed channel.
eof = true
break
}
// For any walk error return right away.
if walkResult.err != nil {
// File not found is a valid case.
if walkResult.err == errFileNotFound {
continue
}
return loi, toObjectErr(walkResult.err, bucket, prefix)
}
var objInfo ObjectInfo
var err error
if hasSuffix(walkResult.entry, slashSeparator) {
for _, getObjectInfoDir := range getObjectInfoDirs {
objInfo, err = getObjectInfoDir(ctx, bucket, walkResult.entry)
if err == nil {
break
}
}
} else {
objInfo, err = getObjInfo(ctx, bucket, walkResult.entry)
}
if err != nil {
// Ignore errFileNotFound as the object might have got
// deleted in the interim period of listing and getObjectInfo(),
// ignore quorum error as it might be an entry from an outdated disk.
if IsErrIgnored(err, []error{
errFileNotFound,
errXLReadQuorum,
}...) {
continue
}
return loi, toObjectErr(err, bucket, prefix)
}
nextMarker = objInfo.Name
objInfos = append(objInfos, objInfo)
if walkResult.end {
eof = true
break
}
i++
}
// Save list routine for the next marker if we haven't reached EOF.
params := listParams{bucket, recursive, nextMarker, prefix}
if !eof {
tpool.Set(params, walkResultCh, endWalkCh)
}
result := ListObjectsInfo{}
for _, objInfo := range objInfos {
if objInfo.IsDir && delimiter == slashSeparator {
result.Prefixes = append(result.Prefixes, objInfo.Name)
continue
}
result.Objects = append(result.Objects, objInfo)
}
if !eof {
result.IsTruncated = true
if len(objInfos) > 0 {
result.NextMarker = objInfos[len(objInfos)-1].Name
}
}
// Success.
return result, nil
}

View File

@@ -1614,7 +1614,7 @@ func newTestObjectLayer(endpoints EndpointList) (newObject ObjectLayer, err erro
}
// Initialize list pool.
listPool := newTreeWalkPool(globalLookupTimeout)
listPool := NewTreeWalkPool(globalLookupTimeout)
// Initialize xl objects.
xl := &xlObjects{

View File

@@ -43,25 +43,25 @@ var errWalkAbort = errors.New("treeWalk abort")
// treeWalk - represents the go routine that does the file tree walk.
type treeWalk struct {
resultCh chan treeWalkResult
resultCh chan TreeWalkResult
endWalkCh chan struct{} // To signal when treeWalk go-routine should end.
endTimerCh chan<- struct{} // To signal when timer go-routine should end.
}
// treeWalkPool - pool of treeWalk go routines.
// TreeWalkPool - pool of treeWalk go routines.
// A treeWalk is added to the pool by Set() and removed either by
// doing a Release() or if the concerned timer goes off.
// treeWalkPool's purpose is to maintain active treeWalk go-routines in a map so that
// it can be looked up across related list calls.
type treeWalkPool struct {
type TreeWalkPool struct {
pool map[listParams][]treeWalk
timeOut time.Duration
lock *sync.Mutex
}
// newTreeWalkPool - initialize new tree walk pool.
func newTreeWalkPool(timeout time.Duration) *treeWalkPool {
tPool := &treeWalkPool{
// NewTreeWalkPool - initialize new tree walk pool.
func NewTreeWalkPool(timeout time.Duration) *TreeWalkPool {
tPool := &TreeWalkPool{
pool: make(map[listParams][]treeWalk),
timeOut: timeout,
lock: &sync.Mutex{},
@@ -70,10 +70,10 @@ func newTreeWalkPool(timeout time.Duration) *treeWalkPool {
}
// Release - selects a treeWalk from the pool based on the input
// listParams, removes it from the pool, and returns the treeWalkResult
// listParams, removes it from the pool, and returns the TreeWalkResult
// channel.
// Returns nil if listParams does not have an asccociated treeWalk.
func (t treeWalkPool) Release(params listParams) (resultCh chan treeWalkResult, endWalkCh chan struct{}) {
func (t TreeWalkPool) Release(params listParams) (resultCh chan TreeWalkResult, endWalkCh chan struct{}) {
t.lock.Lock()
defer t.lock.Unlock()
walks, ok := t.pool[params] // Pick the valid walks.
@@ -103,7 +103,7 @@ func (t treeWalkPool) Release(params listParams) (resultCh chan treeWalkResult,
// 2) Relase() signals the timer go-routine to end on endTimerCh.
// During listing the timer should not timeout and end the treeWalk go-routine, hence the
// timer go-routine should be ended.
func (t treeWalkPool) Set(params listParams, resultCh chan treeWalkResult, endWalkCh chan struct{}) {
func (t TreeWalkPool) Set(params listParams, resultCh chan TreeWalkResult, endWalkCh chan struct{}) {
t.lock.Lock()
defer t.lock.Unlock()

View File

@@ -25,7 +25,7 @@ import (
// and that is available in the pool before the timeout.
func TestTreeWalkPoolBasic(t *testing.T) {
// Create a treeWalkPool
tw := newTreeWalkPool(1 * time.Second)
tw := NewTreeWalkPool(1 * time.Second)
// Create sample params
params := listParams{
@@ -33,7 +33,7 @@ func TestTreeWalkPoolBasic(t *testing.T) {
}
// Add a treeWalk to the pool
resultCh := make(chan treeWalkResult)
resultCh := make(chan TreeWalkResult)
endWalkCh := make(chan struct{})
tw.Set(params, resultCh, endWalkCh)
@@ -60,7 +60,7 @@ func TestTreeWalkPoolBasic(t *testing.T) {
// Test if multiple tree walkers for the same listParams are managed as expected by the pool.
func TestManyWalksSameParam(t *testing.T) {
// Create a treeWalkPool.
tw := newTreeWalkPool(5 * time.Second)
tw := NewTreeWalkPool(5 * time.Second)
// Create sample params.
params := listParams{
@@ -75,7 +75,7 @@ func TestManyWalksSameParam(t *testing.T) {
default:
// Create many treeWalk go-routines for the same params.
for i := 0; i < 10; i++ {
resultCh := make(chan treeWalkResult)
resultCh := make(chan TreeWalkResult)
endWalkCh := make(chan struct{})
tw.Set(params, resultCh, endWalkCh)
}

View File

@@ -22,8 +22,8 @@ import (
"strings"
)
// Tree walk result carries results of tree walking.
type treeWalkResult struct {
// TreeWalkResult - Tree walk result carries results of tree walking.
type TreeWalkResult struct {
entry string
err error
end bool
@@ -36,7 +36,7 @@ type treeWalkResult struct {
// sorting with out trailing "/" = ["a", "a-b"]
// sorting with trailing "/" = ["a-b/", "a/"]
// Hence if entries[] does not have a case like the above example then isLeaf() check
// can be delayed till the entry is pushed into the treeWalkResult channel.
// can be delayed till the entry is pushed into the TreeWalkResult channel.
// delayIsLeafCheck() returns true if isLeaf can be delayed or false if
// isLeaf should be done in listDir()
func delayIsLeafCheck(entries []string) bool {
@@ -85,21 +85,21 @@ func filterMatchingPrefix(entries []string, prefixEntry string) []string {
return entries[start:end]
}
// "listDir" function of type listDirFunc returned by listDirFactory() - explained below.
type listDirFunc func(bucket, prefixDir, prefixEntry string) (entries []string, delayIsLeaf bool)
// ListDirFunc - "listDir" function of type listDirFunc returned by listDirFactory() - explained below.
type ListDirFunc func(bucket, prefixDir, prefixEntry string) (entries []string, delayIsLeaf bool)
// A function isLeaf of type isLeafFunc is used to detect if an entry is a leaf entry. There are four scenarios
// IsLeafFunc - A function isLeaf of type isLeafFunc is used to detect if an entry is a leaf entry. There are four scenarios
// where isLeaf should behave differently:
// 1. FS backend object listing - isLeaf is true if the entry has a trailing "/"
// 2. FS backend multipart listing - isLeaf is true if the entry is a directory and contains uploads.json
// 3. XL backend object listing - isLeaf is true if the entry is a directory and contains xl.json
// 4. XL backend multipart listing - isLeaf is true if the entry is a directory and contains uploads.json
type isLeafFunc func(string, string) bool
type IsLeafFunc func(string, string) bool
// A function isLeafDir of type isLeafDirFunc is used to detect if an entry represents an empty directory.
type isLeafDirFunc func(string, string) bool
// IsLeafDirFunc - A function isLeafDir of type isLeafDirFunc is used to detect if an entry represents an empty directory.
type IsLeafDirFunc func(string, string) bool
func filterListEntries(bucket, prefixDir string, entries []string, prefixEntry string, isLeaf isLeafFunc) ([]string, bool) {
func filterListEntries(bucket, prefixDir string, entries []string, prefixEntry string, isLeaf IsLeafFunc) ([]string, bool) {
// Listing needs to be sorted.
sort.Strings(entries)
@@ -107,7 +107,7 @@ func filterListEntries(bucket, prefixDir string, entries []string, prefixEntry s
entries = filterMatchingPrefix(entries, prefixEntry)
// Can isLeaf() check be delayed till when it has to be sent down the
// treeWalkResult channel?
// TreeWalkResult channel?
delayIsLeaf := delayIsLeafCheck(entries)
if delayIsLeaf {
return entries, true
@@ -125,8 +125,8 @@ func filterListEntries(bucket, prefixDir string, entries []string, prefixEntry s
return entries, false
}
// treeWalk walks directory tree recursively pushing treeWalkResult into the channel as and when it encounters files.
func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker string, recursive bool, listDir listDirFunc, isLeaf isLeafFunc, isLeafDir isLeafDirFunc, resultCh chan treeWalkResult, endWalkCh chan struct{}, isEnd bool) error {
// treeWalk walks directory tree recursively pushing TreeWalkResult into the channel as and when it encounters files.
func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker string, recursive bool, listDir ListDirFunc, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc, resultCh chan TreeWalkResult, endWalkCh chan struct{}, isEnd bool) error {
// Example:
// if prefixDir="one/two/three/" and marker="four/five.txt" treeWalk is recursively
// called with prefixDir="one/two/three/four/" and marker="five.txt"
@@ -219,7 +219,7 @@ func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker
select {
case <-endWalkCh:
return errWalkAbort
case resultCh <- treeWalkResult{entry: pathJoin(prefixDir, entry), end: isEOF}:
case resultCh <- TreeWalkResult{entry: pathJoin(prefixDir, entry), end: isEOF}:
}
}
@@ -228,7 +228,7 @@ func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker
}
// Initiate a new treeWalk in a goroutine.
func startTreeWalk(ctx context.Context, bucket, prefix, marker string, recursive bool, listDir listDirFunc, isLeaf isLeafFunc, isLeafDir isLeafDirFunc, endWalkCh chan struct{}) chan treeWalkResult {
func startTreeWalk(ctx context.Context, bucket, prefix, marker string, recursive bool, listDir ListDirFunc, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc, endWalkCh chan struct{}) chan TreeWalkResult {
// Example 1
// If prefix is "one/two/three/" and marker is "one/two/three/four/five.txt"
// treeWalk is called with prefixDir="one/two/three/" and marker="four/five.txt"
@@ -239,7 +239,7 @@ func startTreeWalk(ctx context.Context, bucket, prefix, marker string, recursive
// treeWalk is called with prefixDir="one/two/" and marker="three/four/five.txt"
// and entryPrefixMatch="th"
resultCh := make(chan treeWalkResult, maxObjectList)
resultCh := make(chan TreeWalkResult, maxObjectList)
entryPrefixMatch := prefix
prefixDir := ""
lastIndex := strings.LastIndex(prefix, slashSeparator)

View File

@@ -128,7 +128,7 @@ func createNamespace(disk StorageAPI, volume string, files []string) error {
// Test if tree walker returns entries matching prefix alone are received
// when a non empty prefix is supplied.
func testTreeWalkPrefix(t *testing.T, listDir listDirFunc, isLeaf isLeafFunc, isLeafDir isLeafDirFunc) {
func testTreeWalkPrefix(t *testing.T, listDir ListDirFunc, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc) {
// Start the tree walk go-routine.
prefix := "d/"
endWalkCh := make(chan struct{})
@@ -143,7 +143,7 @@ func testTreeWalkPrefix(t *testing.T, listDir listDirFunc, isLeaf isLeafFunc, is
}
// Test if entries received on tree walk's channel appear after the supplied marker.
func testTreeWalkMarker(t *testing.T, listDir listDirFunc, isLeaf isLeafFunc, isLeafDir isLeafDirFunc) {
func testTreeWalkMarker(t *testing.T, listDir ListDirFunc, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc) {
// Start the tree walk go-routine.
prefix := ""
endWalkCh := make(chan struct{})
@@ -243,7 +243,7 @@ func TestTreeWalkTimeout(t *testing.T) {
listDir := listDirFactory(context.Background(), isLeaf, disk)
// TreeWalk pool with 2 seconds timeout for tree-walk go routines.
pool := newTreeWalkPool(2 * time.Second)
pool := NewTreeWalkPool(2 * time.Second)
endWalkCh := make(chan struct{})
prefix := ""
@@ -625,7 +625,7 @@ func TestTreeWalkIsEnd(t *testing.T) {
{"d/", "d/e", true, "d/g/h"},
}
for i, test := range testCases {
var entry treeWalkResult
var entry TreeWalkResult
for entry = range startTreeWalk(context.Background(), volume, test.prefix, test.marker, test.recursive, listDir, isLeaf, isLeafDir, endWalkCh) {
}
if entry.entry != test.expectedEntry {

View File

@@ -77,7 +77,7 @@ type xlSets struct {
distributionAlgo string
// Pack level listObjects pool management.
listPool *treeWalkPool
listPool *TreeWalkPool
}
// isConnected - checks if the endpoint is connected or not.
@@ -270,7 +270,7 @@ func newXLSets(endpoints EndpointList, format *formatXLV3, setCount int, drivesP
format: format,
disksConnectDoneCh: make(chan struct{}),
distributionAlgo: format.XL.DistributionAlgo,
listPool: newTreeWalkPool(globalLookupTimeout),
listPool: NewTreeWalkPool(globalLookupTimeout),
}
mutex := newNSLock(globalIsDistXL)
@@ -643,7 +643,7 @@ func (s *xlSets) CopyObject(ctx context.Context, srcBucket, srcObject, destBucke
// 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.
// disks - used for doing disk.ListDir(). Sets passes set of disks.
func listDirSetsFactory(ctx context.Context, isLeaf isLeafFunc, isLeafDir isLeafDirFunc, sets ...*xlObjects) listDirFunc {
func listDirSetsFactory(ctx context.Context, isLeaf IsLeafFunc, isLeafDir IsLeafDirFunc, sets ...*xlObjects) ListDirFunc {
listDirInternal := func(bucket, prefixDir, prefixEntry string, disks []StorageAPI) (mergedEntries []string) {
var diskEntries = make([][]string, len(disks))
var wg sync.WaitGroup
@@ -712,109 +712,38 @@ func listDirSetsFactory(ctx context.Context, isLeaf isLeafFunc, isLeafDir isLeaf
// listed and subsequently merge lexically sorted inside listDirSetsFactory(). Resulting
// value through the walk channel receives the data properly lexically sorted.
func (s *xlSets) ListObjects(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (ListObjectsInfo, error) {
var result ListObjectsInfo
// validate all the inputs for listObjects
if err := checkListObjsArgs(ctx, bucket, prefix, marker, delimiter, s); err != nil {
return result, err
isLeaf := func(bucket, entry string) bool {
entry = strings.TrimSuffix(entry, slashSeparator)
// Verify if we are at the leaf, a leaf is where we
// see `xl.json` inside a directory.
return s.getHashedSet(entry).isObject(bucket, entry)
}
var objInfos []ObjectInfo
var eof bool
var nextMarker string
recursive := true
if delimiter == slashSeparator {
recursive = false
}
walkResultCh, endWalkCh := s.listPool.Release(listParams{bucket, recursive, marker, prefix})
if walkResultCh == nil {
endWalkCh = make(chan struct{})
isLeaf := func(bucket, entry string) bool {
entry = strings.TrimSuffix(entry, slashSeparator)
// Verify if we are at the leaf, a leaf is where we
// see `xl.json` inside a directory.
return s.getHashedSet(entry).isObject(bucket, entry)
}
isLeafDir := func(bucket, entry string) bool {
// Verify prefixes in all sets.
var ok bool
for _, set := range s.sets {
ok = set.isObjectDir(bucket, entry)
if ok {
return true
}
isLeafDir := func(bucket, entry string) bool {
// Verify prefixes in all sets.
var ok bool
for _, set := range s.sets {
ok = set.isObjectDir(bucket, entry)
if ok {
return true
}
return false
}
listDir := listDirSetsFactory(ctx, isLeaf, isLeafDir, s.sets...)
walkResultCh = startTreeWalk(ctx, bucket, prefix, marker, recursive, listDir, isLeaf, isLeafDir, endWalkCh)
return false
}
for i := 0; i < maxKeys; {
walkResult, ok := <-walkResultCh
if !ok {
// Closed channel.
eof = true
break
}
listDir := listDirSetsFactory(ctx, isLeaf, isLeafDir, s.sets...)
// For any walk error return right away.
if walkResult.err != nil {
return result, toObjectErr(walkResult.err, bucket, prefix)
}
var objInfo ObjectInfo
var err error
if hasSuffix(walkResult.entry, slashSeparator) {
// Verify prefixes in all sets.
for _, set := range s.sets {
objInfo, err = set.getObjectInfoDir(ctx, bucket, walkResult.entry)
if err == nil {
break
}
}
} else {
objInfo, err = s.getHashedSet(walkResult.entry).getObjectInfo(ctx, bucket, walkResult.entry)
}
if err != nil {
// Ignore errFileNotFound as the object might have got
// deleted in the interim period of listing and getObjectInfo(),
// ignore quorum error as it might be an entry from an outdated disk.
if IsErrIgnored(err, []error{
errFileNotFound,
errXLReadQuorum,
}...) {
continue
}
return result, toObjectErr(err, bucket, prefix)
}
nextMarker = objInfo.Name
objInfos = append(objInfos, objInfo)
i++
if walkResult.end {
eof = true
break
}
var getObjectInfoDirs []func(context.Context, string, string) (ObjectInfo, error)
// Verify prefixes in all sets.
for _, set := range s.sets {
getObjectInfoDirs = append(getObjectInfoDirs, set.getObjectInfoDir)
}
params := listParams{bucket, recursive, nextMarker, prefix}
if !eof {
s.listPool.Set(params, walkResultCh, endWalkCh)
var getObjectInfo = func(ctx context.Context, bucket string, entry string) (ObjectInfo, error) {
return s.getHashedSet(entry).getObjectInfo(ctx, bucket, entry)
}
result = ListObjectsInfo{IsTruncated: !eof}
for _, objInfo := range objInfos {
result.NextMarker = objInfo.Name
if objInfo.IsDir && delimiter == slashSeparator {
result.Prefixes = append(result.Prefixes, objInfo.Name)
continue
}
result.Objects = append(result.Objects, objInfo)
}
return result, nil
return listObjects(ctx, s, bucket, prefix, marker, delimiter, maxKeys, s.listPool, isLeaf, isLeafDir, listDir, getObjectInfo, getObjectInfoDirs...)
}
func (s *xlSets) ListMultipartUploads(ctx context.Context, bucket, prefix, keyMarker, uploadIDMarker, delimiter string, maxUploads int) (result ListMultipartsInfo, err error) {

View File

@@ -24,7 +24,7 @@ import (
// 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.
// disks - used for doing disk.ListDir()
func listDirFactory(ctx context.Context, isLeaf isLeafFunc, disks ...StorageAPI) listDirFunc {
func listDirFactory(ctx context.Context, isLeaf IsLeafFunc, disks ...StorageAPI) ListDirFunc {
// Returns sorted merged entries from all the disks.
listDir := func(bucket, prefixDir, prefixEntry string) (mergedEntries []string, delayIsLeaf bool) {
for _, disk := range disks {

View File

@@ -48,7 +48,7 @@ type xlObjects struct {
storageDisks []StorageAPI
// TODO: ListObjects pool management, should be removed in future.
listPool *treeWalkPool
listPool *TreeWalkPool
}
// Shutdown function for object storage interface.