mirror of https://github.com/minio/minio.git
Merge pull request #1850 from harshavardhana/list-rewrite
XL: Implement ListObjects channel and pool management.
This commit is contained in:
commit
73ddb5be75
|
@ -1056,6 +1056,7 @@ func testListMultipartUploads(obj ObjectLayer, instanceType string, t *testing.T
|
|||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
// fmt.Println(testCase) // uncomment to peek into the test cases.
|
||||
actualResult, actualErr := obj.ListMultipartUploads(testCase.bucket, testCase.prefix, testCase.keyMarker, testCase.uploadIDMarker, testCase.delimiter, testCase.maxUploads)
|
||||
if actualErr != nil && testCase.shouldPass {
|
||||
t.Errorf("Test %d: %s: Expected to pass, but failed with: <ERROR> %s", i+1, instanceType, actualErr.Error())
|
||||
|
|
|
@ -0,0 +1,136 @@
|
|||
/*
|
||||
* 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 main
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Global lookup timeout.
|
||||
const (
|
||||
globalLookupTimeout = time.Minute * 30 // 30minutes.
|
||||
)
|
||||
|
||||
// errWalkAbort - returned by the treeWalker routine, it signals the end of treeWalk.
|
||||
var errWalkAbort = errors.New("treeWalk abort")
|
||||
|
||||
// treeWalkerPoolInfo - tree walker pool info carries temporary walker
|
||||
// channel stored until timeout is called.
|
||||
type treeWalkerPoolInfo struct {
|
||||
treeWalkerCh chan treeWalker
|
||||
treeWalkerDoneCh chan struct{}
|
||||
doneCh chan<- struct{}
|
||||
}
|
||||
|
||||
// treeWalkerPool - tree walker pool is a set of temporary tree walker
|
||||
// objects. Any item stored in the pool will be removed automatically at
|
||||
// a given timeOut value. This pool is safe for use by multiple
|
||||
// goroutines simultaneously. pool's purpose is to cache tree walker
|
||||
// channels for later reuse.
|
||||
type treeWalkerPool struct {
|
||||
pool map[listParams][]treeWalkerPoolInfo
|
||||
timeOut time.Duration
|
||||
lock *sync.Mutex
|
||||
}
|
||||
|
||||
// newTreeWalkerPool - initialize new tree walker pool.
|
||||
func newTreeWalkerPool(timeout time.Duration) *treeWalkerPool {
|
||||
tPool := &treeWalkerPool{
|
||||
pool: make(map[listParams][]treeWalkerPoolInfo),
|
||||
timeOut: timeout,
|
||||
lock: &sync.Mutex{},
|
||||
}
|
||||
return tPool
|
||||
}
|
||||
|
||||
// Release - selects an item from the pool based on the input
|
||||
// listParams, removes it from the pool, and returns treeWalker
|
||||
// channels. Release will return nil, if listParams is not
|
||||
// recognized.
|
||||
func (t treeWalkerPool) Release(params listParams) (treeWalkerCh chan treeWalker, treeWalkerDoneCh chan struct{}) {
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
walks, ok := t.pool[params] // Pick the valid walks.
|
||||
if ok {
|
||||
if len(walks) > 0 {
|
||||
// Pop out the first valid walk entry.
|
||||
walk := walks[0]
|
||||
walks = walks[1:]
|
||||
if len(walks) > 0 {
|
||||
t.pool[params] = walks
|
||||
} else {
|
||||
delete(t.pool, params)
|
||||
}
|
||||
walk.doneCh <- struct{}{}
|
||||
return walk.treeWalkerCh, walk.treeWalkerDoneCh
|
||||
}
|
||||
}
|
||||
// Release return nil if params not found.
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Set - adds new list params along with treeWalker channel to the
|
||||
// pool for future. Additionally this also starts a go routine which
|
||||
// waits at the configured timeout. Additionally this go-routine is
|
||||
// also closed pro-actively by 'Release' call when the treeWalker
|
||||
// item is obtained from the pool.
|
||||
func (t treeWalkerPool) Set(params listParams, treeWalkerCh chan treeWalker, treeWalkerDoneCh chan struct{}) {
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
|
||||
// Should be a buffered channel so that Release() never blocks.
|
||||
var doneCh = make(chan struct{}, 1)
|
||||
walkInfo := treeWalkerPoolInfo{
|
||||
treeWalkerCh: treeWalkerCh,
|
||||
treeWalkerDoneCh: treeWalkerDoneCh,
|
||||
doneCh: doneCh,
|
||||
}
|
||||
// Append new walk info.
|
||||
t.pool[params] = append(t.pool[params], walkInfo)
|
||||
|
||||
// Safe expiry of treeWalkerCh after timeout.
|
||||
go func(doneCh <-chan struct{}) {
|
||||
select {
|
||||
// Wait until timeOut
|
||||
case <-time.After(t.timeOut):
|
||||
t.lock.Lock()
|
||||
walks, ok := t.pool[params] // Look for valid walks.
|
||||
if ok {
|
||||
// Look for walkInfo, remove it from the walks list.
|
||||
for i, walk := range walks {
|
||||
if walk == walkInfo {
|
||||
walks = append(walks[:i], walks[i+1:]...)
|
||||
}
|
||||
}
|
||||
// Walks is empty we have no more pending requests.
|
||||
// Remove map entry.
|
||||
if len(walks) == 0 {
|
||||
delete(t.pool, params)
|
||||
} else { // Save the updated walks.
|
||||
t.pool[params] = walks
|
||||
}
|
||||
}
|
||||
// Close tree walker for the backing go-routine to die.
|
||||
close(treeWalkerDoneCh)
|
||||
t.lock.Unlock()
|
||||
case <-doneCh:
|
||||
return
|
||||
}
|
||||
}(doneCh)
|
||||
}
|
125
tree-walk-xl.go
125
tree-walk-xl.go
|
@ -19,7 +19,6 @@ package main
|
|||
import (
|
||||
"sort"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
// listParams - list object params used for list object map
|
||||
|
@ -31,21 +30,19 @@ type listParams struct {
|
|||
}
|
||||
|
||||
// Tree walk result carries results of tree walking.
|
||||
type treeWalkResult struct {
|
||||
type treeWalker struct {
|
||||
entry string
|
||||
err error
|
||||
end bool
|
||||
}
|
||||
|
||||
// Tree walk notify carries a channel which notifies tree walk
|
||||
// results, additionally it also carries information if treeWalk
|
||||
// should be timedOut.
|
||||
type treeWalker struct {
|
||||
ch <-chan treeWalkResult
|
||||
timedOut bool
|
||||
}
|
||||
|
||||
// listDir - listDir.
|
||||
// listDir - lists all the entries at a given prefix, takes additional params as filter and leaf detection.
|
||||
// filter is required to filter out the listed entries usually this function is supposed to return
|
||||
// true or false.
|
||||
// isLeaf is required to differentiate between directories and objects, this is a special requirement for XL
|
||||
// backend since objects are kept as directories, the only way to know if a directory is truly an object
|
||||
// we validate if 'xl.json' exists at the leaf. isLeaf replies true/false based on the outcome of a Stat
|
||||
// operation.
|
||||
func (xl xlObjects) listDir(bucket, prefixDir string, filter func(entry string) bool, isLeaf func(string, string) bool) (entries []string, err error) {
|
||||
for _, disk := range xl.getLoadBalancedQuorumDisks() {
|
||||
if disk == nil {
|
||||
|
@ -62,7 +59,7 @@ func (xl xlObjects) listDir(bucket, prefixDir string, filter func(entry string)
|
|||
}
|
||||
// Skip the entries which do not match the filter.
|
||||
for i, entry := range entries {
|
||||
if filter(entry) {
|
||||
if !filter(entry) {
|
||||
entries[i] = ""
|
||||
continue
|
||||
}
|
||||
|
@ -83,7 +80,7 @@ func (xl xlObjects) listDir(bucket, prefixDir string, filter func(entry string)
|
|||
}
|
||||
|
||||
// treeWalk walks directory tree recursively pushing fileInfo into the channel as and when it encounters files.
|
||||
func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string, recursive bool, send func(treeWalkResult) bool, count *int, isLeaf func(string, string) bool) bool {
|
||||
func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string, recursive bool, isLeaf func(string, string) bool, treeWalkCh chan treeWalker, doneCh 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"
|
||||
|
@ -99,14 +96,19 @@ func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string,
|
|||
}
|
||||
}
|
||||
entries, err := xl.listDir(bucket, prefixDir, func(entry string) bool {
|
||||
return !strings.HasPrefix(entry, entryPrefixMatch)
|
||||
return strings.HasPrefix(entry, entryPrefixMatch)
|
||||
}, isLeaf)
|
||||
if err != nil {
|
||||
send(treeWalkResult{err: err})
|
||||
return false
|
||||
select {
|
||||
case <-doneCh:
|
||||
return errWalkAbort
|
||||
case treeWalkCh <- treeWalker{err: err}:
|
||||
return err
|
||||
}
|
||||
}
|
||||
// For an empty list return right here.
|
||||
if len(entries) == 0 {
|
||||
return true
|
||||
return nil
|
||||
}
|
||||
|
||||
// example:
|
||||
|
@ -116,12 +118,14 @@ func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string,
|
|||
return entries[i] >= markerDir
|
||||
})
|
||||
entries = entries[idx:]
|
||||
*count += len(entries)
|
||||
// For an empty list after search through the entries, return right here.
|
||||
if len(entries) == 0 {
|
||||
return nil
|
||||
}
|
||||
for i, entry := range entries {
|
||||
if i == 0 && markerDir == entry {
|
||||
if !recursive {
|
||||
// Skip as the marker would already be listed in the previous listing.
|
||||
*count--
|
||||
continue
|
||||
}
|
||||
if recursive && !strings.HasSuffix(entry, slashSeparator) {
|
||||
|
@ -130,11 +134,9 @@ func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string,
|
|||
// should not be skipped, instead it will need to be treeWalk()'ed into.
|
||||
|
||||
// Skip if it is a file though as it would be listed in previous listing.
|
||||
*count--
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
if recursive && strings.HasSuffix(entry, slashSeparator) {
|
||||
// If the entry is a directory, we will need recurse into it.
|
||||
markerArg := ""
|
||||
|
@ -143,23 +145,30 @@ func (xl xlObjects) treeWalk(bucket, prefixDir, entryPrefixMatch, marker string,
|
|||
// recursing into "four/"
|
||||
markerArg = markerBase
|
||||
}
|
||||
*count--
|
||||
prefixMatch := "" // Valid only for first level treeWalk and empty for subdirectories.
|
||||
if !xl.treeWalk(bucket, pathJoin(prefixDir, entry), prefixMatch, markerArg, recursive, send, count, isLeaf) {
|
||||
return false
|
||||
// markIsEnd is passed to this entry's treeWalk() so that treeWalker.end can be marked
|
||||
// true at the end of the treeWalk stream.
|
||||
markIsEnd := i == len(entries)-1 && isEnd
|
||||
if tErr := xl.treeWalk(bucket, pathJoin(prefixDir, entry), prefixMatch, markerArg, recursive, isLeaf, treeWalkCh, doneCh, markIsEnd); tErr != nil {
|
||||
return tErr
|
||||
}
|
||||
continue
|
||||
}
|
||||
*count--
|
||||
if !send(treeWalkResult{entry: pathJoin(prefixDir, entry)}) {
|
||||
return false
|
||||
// EOF is set if we are at last entry and the caller indicated we at the end.
|
||||
isEOF := ((i == len(entries)-1) && isEnd)
|
||||
select {
|
||||
case <-doneCh:
|
||||
return errWalkAbort
|
||||
case treeWalkCh <- treeWalker{entry: pathJoin(prefixDir, entry), end: isEOF}:
|
||||
}
|
||||
}
|
||||
return true
|
||||
|
||||
// Everything is listed.
|
||||
return nil
|
||||
}
|
||||
|
||||
// Initiate a new treeWalk in a goroutine.
|
||||
func (xl xlObjects) startTreeWalk(bucket, prefix, marker string, recursive bool, isLeaf func(string, string) bool) *treeWalker {
|
||||
func (xl xlObjects) startTreeWalk(bucket, prefix, marker string, recursive bool, isLeaf func(string, string) bool, doneCh chan struct{}) chan treeWalker {
|
||||
// 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"
|
||||
|
@ -170,8 +179,7 @@ func (xl xlObjects) startTreeWalk(bucket, prefix, marker string, recursive bool,
|
|||
// treeWalk is called with prefixDir="one/two/" and marker="three/four/five.txt"
|
||||
// and entryPrefixMatch="th"
|
||||
|
||||
ch := make(chan treeWalkResult, maxObjectList)
|
||||
walkNotify := treeWalker{ch: ch}
|
||||
treeWalkCh := make(chan treeWalker, maxObjectList)
|
||||
entryPrefixMatch := prefix
|
||||
prefixDir := ""
|
||||
lastIndex := strings.LastIndex(prefix, slashSeparator)
|
||||
|
@ -179,58 +187,11 @@ func (xl xlObjects) startTreeWalk(bucket, prefix, marker string, recursive bool,
|
|||
entryPrefixMatch = prefix[lastIndex+1:]
|
||||
prefixDir = prefix[:lastIndex+1]
|
||||
}
|
||||
count := 0
|
||||
marker = strings.TrimPrefix(marker, prefixDir)
|
||||
go func() {
|
||||
defer close(ch)
|
||||
send := func(walkResult treeWalkResult) bool {
|
||||
if count == 0 {
|
||||
walkResult.end = true
|
||||
}
|
||||
timer := time.After(time.Second * 60)
|
||||
select {
|
||||
case ch <- walkResult:
|
||||
return true
|
||||
case <-timer:
|
||||
walkNotify.timedOut = true
|
||||
return false
|
||||
}
|
||||
}
|
||||
xl.treeWalk(bucket, prefixDir, entryPrefixMatch, marker, recursive, send, &count, isLeaf)
|
||||
isEnd := true // Indication to start walking the tree with end as true.
|
||||
xl.treeWalk(bucket, prefixDir, entryPrefixMatch, marker, recursive, isLeaf, treeWalkCh, doneCh, isEnd)
|
||||
close(treeWalkCh)
|
||||
}()
|
||||
return &walkNotify
|
||||
}
|
||||
|
||||
// Save the goroutine reference in the map
|
||||
func (xl xlObjects) saveTreeWalk(params listParams, walker *treeWalker) {
|
||||
xl.listObjectMapMutex.Lock()
|
||||
defer xl.listObjectMapMutex.Unlock()
|
||||
|
||||
walkers, _ := xl.listObjectMap[params]
|
||||
walkers = append(walkers, walker)
|
||||
|
||||
xl.listObjectMap[params] = walkers
|
||||
}
|
||||
|
||||
// Lookup the goroutine reference from map
|
||||
func (xl xlObjects) lookupTreeWalk(params listParams) *treeWalker {
|
||||
xl.listObjectMapMutex.Lock()
|
||||
defer xl.listObjectMapMutex.Unlock()
|
||||
|
||||
if walkChs, ok := xl.listObjectMap[params]; ok {
|
||||
for i, walkCh := range walkChs {
|
||||
if !walkCh.timedOut {
|
||||
newWalkChs := walkChs[i+1:]
|
||||
if len(newWalkChs) > 0 {
|
||||
xl.listObjectMap[params] = newWalkChs
|
||||
} else {
|
||||
delete(xl.listObjectMap, params)
|
||||
}
|
||||
return walkCh
|
||||
}
|
||||
}
|
||||
// As all channels are timed out, delete the map entry
|
||||
delete(xl.listObjectMap, params)
|
||||
}
|
||||
return nil
|
||||
return treeWalkCh
|
||||
}
|
||||
|
|
|
@ -26,15 +26,17 @@ func (xl xlObjects) listObjects(bucket, prefix, marker, delimiter string, maxKey
|
|||
recursive = false
|
||||
}
|
||||
|
||||
walker := xl.lookupTreeWalk(listParams{bucket, recursive, marker, prefix})
|
||||
if walker == nil {
|
||||
walker = xl.startTreeWalk(bucket, prefix, marker, recursive, xl.isObject)
|
||||
walkerCh, walkerDoneCh := xl.listPool.Release(listParams{bucket, recursive, marker, prefix})
|
||||
if walkerCh == nil {
|
||||
walkerDoneCh = make(chan struct{})
|
||||
walkerCh = xl.startTreeWalk(bucket, prefix, marker, recursive, xl.isObject, walkerDoneCh)
|
||||
}
|
||||
|
||||
var objInfos []ObjectInfo
|
||||
var eof bool
|
||||
var nextMarker string
|
||||
for i := 0; i < maxKeys; {
|
||||
walkResult, ok := <-walker.ch
|
||||
walkResult, ok := <-walkerCh
|
||||
if !ok {
|
||||
// Closed channel.
|
||||
eof = true
|
||||
|
@ -63,18 +65,18 @@ func (xl xlObjects) listObjects(bucket, prefix, marker, delimiter string, maxKey
|
|||
return ListObjectsInfo{}, toObjectErr(err, bucket, prefix)
|
||||
}
|
||||
}
|
||||
|
||||
nextMarker = objInfo.Name
|
||||
objInfos = append(objInfos, objInfo)
|
||||
if walkResult.end {
|
||||
i++
|
||||
if walkResult.end == true {
|
||||
eof = true
|
||||
break
|
||||
}
|
||||
i++
|
||||
}
|
||||
|
||||
params := listParams{bucket, recursive, nextMarker, prefix}
|
||||
if !eof {
|
||||
xl.saveTreeWalk(params, walker)
|
||||
xl.listPool.Set(params, walkerCh, walkerDoneCh)
|
||||
}
|
||||
|
||||
result := ListObjectsInfo{IsTruncated: !eof}
|
||||
|
|
|
@ -81,13 +81,14 @@ func (xl xlObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
|
|||
}
|
||||
// Validate if we need to list further depending on maxUploads.
|
||||
if maxUploads > 0 {
|
||||
walker := xl.lookupTreeWalk(listParams{minioMetaBucket, recursive, multipartMarkerPath, multipartPrefixPath})
|
||||
if walker == nil {
|
||||
walker = xl.startTreeWalk(minioMetaBucket, multipartPrefixPath, multipartMarkerPath, recursive, xl.isMultipartUpload)
|
||||
walkerCh, walkerDoneCh := xl.listPool.Release(listParams{minioMetaBucket, recursive, multipartMarkerPath, multipartPrefixPath})
|
||||
if walkerCh == nil {
|
||||
walkerDoneCh = make(chan struct{})
|
||||
walkerCh = xl.startTreeWalk(minioMetaBucket, multipartPrefixPath, multipartMarkerPath, recursive, xl.isMultipartUpload, walkerDoneCh)
|
||||
}
|
||||
// Collect uploads until we have reached maxUploads count to 0.
|
||||
for maxUploads > 0 {
|
||||
walkResult, ok := <-walker.ch
|
||||
walkResult, ok := <-walkerCh
|
||||
if !ok {
|
||||
// Closed channel.
|
||||
eof = true
|
||||
|
@ -110,11 +111,9 @@ func (xl xlObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
|
|||
})
|
||||
maxUploads--
|
||||
if maxUploads == 0 {
|
||||
if walkResult.end {
|
||||
eof = true
|
||||
break
|
||||
}
|
||||
}
|
||||
continue
|
||||
}
|
||||
var newUploads []uploadMetadata
|
||||
|
@ -142,7 +141,7 @@ func (xl xlObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
|
|||
}
|
||||
uploads = append(uploads, newUploads...)
|
||||
maxUploads -= len(newUploads)
|
||||
if walkResult.end && end {
|
||||
if end && walkResult.end {
|
||||
eof = true
|
||||
break
|
||||
}
|
||||
|
|
7
xl-v1.go
7
xl-v1.go
|
@ -20,7 +20,6 @@ import (
|
|||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/minio/minio/pkg/disk"
|
||||
)
|
||||
|
@ -45,8 +44,7 @@ type xlObjects struct {
|
|||
writeQuorum int // writeQuorum minimum required disks to write data.
|
||||
|
||||
// List pool management.
|
||||
listObjectMap map[listParams][]*treeWalker
|
||||
listObjectMapMutex *sync.Mutex
|
||||
listPool *treeWalkerPool
|
||||
}
|
||||
|
||||
// errXLMaxDisks - returned for reached maximum of disks.
|
||||
|
@ -163,8 +161,7 @@ func newXLObjects(disks []string) (ObjectLayer, error) {
|
|||
storageDisks: newPosixDisks,
|
||||
dataBlocks: dataBlocks,
|
||||
parityBlocks: parityBlocks,
|
||||
listObjectMap: make(map[listParams][]*treeWalker),
|
||||
listObjectMapMutex: &sync.Mutex{},
|
||||
listPool: newTreeWalkerPool(globalLookupTimeout),
|
||||
}
|
||||
|
||||
// Figure out read and write quorum based on number of storage disks.
|
||||
|
|
Loading…
Reference in New Issue