Merge pull request #1850 from harshavardhana/list-rewrite

XL: Implement ListObjects channel and pool management.
master
Harshavardhana 8 years ago
commit 73ddb5be75
  1. 1
      object-api-multipart_test.go
  2. 136
      tree-walk-pool.go
  3. 125
      tree-walk-xl.go
  4. 18
      xl-v1-list-objects.go
  5. 17
      xl-v1-multipart.go
  6. 15
      xl-v1.go

@ -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)
}

@ -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,10 +111,8 @@ func (xl xlObjects) listMultipartUploads(bucket, prefix, keyMarker, uploadIDMark
})
maxUploads--
if maxUploads == 0 {
if walkResult.end {
eof = true
break
}
eof = true
break
}
continue
}
@ -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
}

@ -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.
@ -159,12 +157,11 @@ func newXLObjects(disks []string) (ObjectLayer, error) {
// Initialize xl objects.
xl := xlObjects{
physicalDisks: disks,
storageDisks: newPosixDisks,
dataBlocks: dataBlocks,
parityBlocks: parityBlocks,
listObjectMap: make(map[listParams][]*treeWalker),
listObjectMapMutex: &sync.Mutex{},
physicalDisks: disks,
storageDisks: newPosixDisks,
dataBlocks: dataBlocks,
parityBlocks: parityBlocks,
listPool: newTreeWalkerPool(globalLookupTimeout),
}
// Figure out read and write quorum based on number of storage disks.

Loading…
Cancel
Save