From 620e4624130b3f828c2e028b6f42021205938de4 Mon Sep 17 00:00:00 2001 From: Harshavardhana Date: Wed, 17 Apr 2019 09:52:08 -0700 Subject: [PATCH] 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 --- cmd/disk-cache-fs.go | 2 +- cmd/disk-cache.go | 6 +- cmd/fs-v1.go | 144 +---- cmd/gateway-common.go | 15 + cmd/gateway/gateway.go | 1 + cmd/gateway/hdfs/gateway-hdfs-utils.go | 67 +++ cmd/gateway/hdfs/gateway-hdfs.go | 700 +++++++++++++++++++++++++ cmd/object-api-common.go | 126 ++++- cmd/test-utils_test.go | 2 +- cmd/tree-walk-pool.go | 18 +- cmd/tree-walk-pool_test.go | 8 +- cmd/tree-walk.go | 32 +- cmd/tree-walk_test.go | 8 +- cmd/xl-sets.go | 119 +---- cmd/xl-v1-list-objects.go | 2 +- cmd/xl-v1.go | 2 +- docs/gateway/hdfs.md | 65 +++ go.mod | 13 +- go.sum | 51 +- 19 files changed, 1089 insertions(+), 292 deletions(-) create mode 100644 cmd/gateway/hdfs/gateway-hdfs-utils.go create mode 100644 cmd/gateway/hdfs/gateway-hdfs.go create mode 100644 docs/gateway/hdfs.md diff --git a/cmd/disk-cache-fs.go b/cmd/disk-cache-fs.go index 741cb4315..9c0a1ba17 100644 --- a/cmd/disk-cache-fs.go +++ b/cmd/disk-cache-fs.go @@ -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), } diff --git a/cmd/disk-cache.go b/cmd/disk-cache.go index 476ce5695..3f02be045 100644 --- a/cmd/disk-cache.go +++ b/cmd/disk-cache.go @@ -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) }, diff --git a/cmd/fs-v1.go b/cmd/fs-v1.go index 1d99b64f3..7aa46a985 100644 --- a/cmd/fs-v1.go +++ b/cmd/fs-v1.go @@ -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 - } - // 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 - } - } - if _, err := fs.statBucketDir(ctx, bucket); err != nil { - return loi, err - } - - // 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 + 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) - // 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. diff --git a/cmd/gateway-common.go b/cmd/gateway-common.go index 7f3901d42..c37e6ea0c 100644 --- a/cmd/gateway-common.go +++ b/cmd/gateway-common.go @@ -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 diff --git a/cmd/gateway/gateway.go b/cmd/gateway/gateway.go index 8d7bb2a51..3e61ed6bd 100644 --- a/cmd/gateway/gateway.go +++ b/cmd/gateway/gateway.go @@ -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" diff --git a/cmd/gateway/hdfs/gateway-hdfs-utils.go b/cmd/gateway/hdfs/gateway-hdfs-utils.go new file mode 100644 index 000000000..df9d2d1fa --- /dev/null +++ b/cmd/gateway/hdfs/gateway-hdfs-utils.go @@ -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 } diff --git a/cmd/gateway/hdfs/gateway-hdfs.go b/cmd/gateway/hdfs/gateway-hdfs.go new file mode 100644 index 000000000..aae3e27ac --- /dev/null +++ b/cmd/gateway/hdfs/gateway-hdfs.go @@ -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) +} diff --git a/cmd/object-api-common.go b/cmd/object-api-common.go index 53a0b7c68..6ec4ac633 100644 --- a/cmd/object-api-common.go +++ b/cmd/object-api-common.go @@ -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 +} diff --git a/cmd/test-utils_test.go b/cmd/test-utils_test.go index 9af384b2a..d3b07aee7 100644 --- a/cmd/test-utils_test.go +++ b/cmd/test-utils_test.go @@ -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{ diff --git a/cmd/tree-walk-pool.go b/cmd/tree-walk-pool.go index bdaa22716..7ed4cc821 100644 --- a/cmd/tree-walk-pool.go +++ b/cmd/tree-walk-pool.go @@ -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() diff --git a/cmd/tree-walk-pool_test.go b/cmd/tree-walk-pool_test.go index 9ad947836..9243d4ceb 100644 --- a/cmd/tree-walk-pool_test.go +++ b/cmd/tree-walk-pool_test.go @@ -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) } diff --git a/cmd/tree-walk.go b/cmd/tree-walk.go index 0f2f845e5..5bc2f26ae 100644 --- a/cmd/tree-walk.go +++ b/cmd/tree-walk.go @@ -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) diff --git a/cmd/tree-walk_test.go b/cmd/tree-walk_test.go index 2f4079ad4..a6305c955 100644 --- a/cmd/tree-walk_test.go +++ b/cmd/tree-walk_test.go @@ -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 { diff --git a/cmd/xl-sets.go b/cmd/xl-sets.go index b3db5c292..cf37c0e25 100644 --- a/cmd/xl-sets.go +++ b/cmd/xl-sets.go @@ -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 - } - - // For any walk error return right away. - if walkResult.err != nil { - return result, toObjectErr(walkResult.err, bucket, prefix) - } + listDir := listDirSetsFactory(ctx, isLeaf, isLeafDir, s.sets...) - 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) { diff --git a/cmd/xl-v1-list-objects.go b/cmd/xl-v1-list-objects.go index 069d04dc7..1711090c7 100644 --- a/cmd/xl-v1-list-objects.go +++ b/cmd/xl-v1-list-objects.go @@ -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 { diff --git a/cmd/xl-v1.go b/cmd/xl-v1.go index 096813d2e..712936a1b 100644 --- a/cmd/xl-v1.go +++ b/cmd/xl-v1.go @@ -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. diff --git a/docs/gateway/hdfs.md b/docs/gateway/hdfs.md new file mode 100644 index 000000000..9ca2664b9 --- /dev/null +++ b/docs/gateway/hdfs.md @@ -0,0 +1,65 @@ +# MinIO HDFS Gateway [![Slack](https://slack.minio.io/slack?type=svg)](https://slack.minio.io) +MinIO HDFS gateway adds Amazon S3 API support to Hadoop HDFS filesystem. Applications can use both the S3 and file APIs concurrently without requiring any data migration. Since the gateway is stateless and shared-nothing, you may elastically provision as many MinIO instances as needed to distribute the load. + +## Run MinIO Gateway for HDFS Storage + +### Using Docker + +Please ensure to replace `hdfs://namenode:8200` with *an* actual name node ip/hostname and port. + +``` +docker run -p 9000:9000 --name hdfs-s3 \ + -e "MINIO_ACCESS_KEY=minio" \ + -e "MINIO_SECRET_KEY=minio123" \ + minio/minio gateway hdfs hdfs://namenode:8200 +``` + +### Using Binary + +``` +export MINIO_ACCESS_KEY=minio +export MINIO_SECRET_KEY=minio123 +minio gateway hdfs hdfs://namenode:8200 +``` + +## Test using MinIO Browser + +*MinIO gateway* comes with an embedded web based object browser. Point your web browser to http://127.0.0.1:9000 to ensure that your server has started successfully. + +![Screenshot](https://raw.githubusercontent.com/minio/minio/master/docs/screenshots/minio-browser-gateway.png) + +## Test using MinIO Client `mc` + +`mc` provides a modern alternative to UNIX commands such as ls, cat, cp, mirror, diff etc. It supports filesystems and Amazon S3 compatible cloud storage services. + +### Configure `mc` + +``` +mc config host add myhdfs http://gateway-ip:9000 access_key secret_key +``` + +### List buckets on hdfs + +``` +mc ls myhdfs +[2017-02-22 01:50:43 PST] 0B user/ +[2017-02-26 21:43:51 PST] 0B datasets/ +[2017-02-26 22:10:11 PST] 0B assets/ +``` + +### Known limitations +Gateway inherits the following limitations of HDFS storage layer: +- No bucket policy support (HDFS has no such concept) +- No bucket notification APIs are not supported (HDFS has no support for fsnotify) +- No server side encryption support (Intentionally not implemented) +- No server side compression support (Intentionally not implemented) + +## Roadmap +- Additional metadata support for PutObject operations +- Additional metadata support for Multipart operations +- Background append to provide concurrency support for multipart operations + +## Explore Further +- [`mc` command-line interface](https://docs.minio.io/docs/minio-client-quickstart-guide) +- [`aws` command-line interface](https://docs.minio.io/docs/aws-cli-with-minio) +- [`minio-go` Go SDK](https://docs.minio.io/docs/golang-client-quickstart-guide) diff --git a/go.mod b/go.mod index 155d33e13..598885b28 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,7 @@ go 1.12 require ( cloud.google.com/go v0.37.2 - contrib.go.opencensus.io/exporter/ocagent v0.4.9 // indirect + contrib.go.opencensus.io/exporter/ocagent v0.4.10 // indirect github.com/Azure/azure-sdk-for-go v27.0.0+incompatible github.com/Azure/go-autorest v11.7.0+incompatible github.com/DataDog/zstd v1.3.5 // indirect @@ -12,6 +12,7 @@ require ( github.com/aliyun/aliyun-oss-go-sdk v0.0.0-20190307165228-86c17b95fcd5 github.com/bcicen/jstream v0.0.0-20190220045926-16c1f8af81c2 github.com/cheggaaa/pb v1.0.28 + github.com/colinmarc/hdfs/v2 v2.0.0 github.com/coredns/coredns v1.4.0 github.com/coreos/bbolt v1.3.2 // indirect github.com/coreos/etcd v3.3.12+incompatible @@ -23,7 +24,6 @@ require ( github.com/elazarl/go-bindata-assetfs v1.0.0 github.com/fatih/color v1.7.0 github.com/fatih/structs v1.1.0 - github.com/go-ini/ini v1.42.0 // indirect github.com/go-sql-driver/mysql v1.4.1 github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef // indirect github.com/golang/snappy v0.0.1 @@ -54,7 +54,7 @@ require ( github.com/minio/blazer v0.0.0-20171126203752-2081f5bf0465 github.com/minio/cli v1.3.0 github.com/minio/dsync v0.0.0-20190131060523-fb604afd87b2 - github.com/minio/highwayhash v0.0.0-20190131021015-02ca4b43caa3 + github.com/minio/highwayhash v1.0.0 github.com/minio/lsync v0.0.0-20190207022115-a4e43e3d0887 github.com/minio/mc v0.0.0-20190401030144-a1355e50e2e8 github.com/minio/minio-go v0.0.0-20190327203652-5325257a208f @@ -79,10 +79,10 @@ require ( github.com/segmentio/go-prompt v1.2.1-0.20161017233205-f0d19b6901ad github.com/sirupsen/logrus v1.3.0 // indirect github.com/skyrings/skyring-common v0.0.0-20160929130248-d1c0bb1cbd5e - github.com/smartystreets/assertions v0.0.0-20190401200700-3f99fa72afbb // indirect + github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3 // indirect github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a // indirect github.com/soheilhy/cmux v0.1.4 // indirect - github.com/streadway/amqp v0.0.0-20190312223743-14f78b41ce6d + github.com/streadway/amqp v0.0.0-20190402114354-16ed540749f6 github.com/tidwall/gjson v1.2.1 github.com/tidwall/pretty v0.0.0-20190325153808-1166b9ac2b65 // indirect github.com/tidwall/sjson v1.0.4 @@ -97,9 +97,10 @@ require ( golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3 golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67 golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 - golang.org/x/tools v0.0.0-20190408220357-e5b8258f4918 // indirect google.golang.org/api v0.3.0 gopkg.in/Shopify/sarama.v1 v1.20.0 + gopkg.in/cheggaaa/pb.v1 v1.0.28 // indirect + gopkg.in/jcmturner/goidentity.v2 v2.0.0 // indirect gopkg.in/olivere/elastic.v5 v5.0.80 gopkg.in/yaml.v2 v2.2.2 ) diff --git a/go.sum b/go.sum index 1198f6dd7..cee7fe370 100644 --- a/go.sum +++ b/go.sum @@ -5,8 +5,8 @@ cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.37.2 h1:4y4L7BdHenTfZL0HervofNTHh9Ad6mNX72cQvl+5eH0= cloud.google.com/go v0.37.2/go.mod h1:H8IAquKe2L30IxoupDgqTaQvKSwF/c8prYHynGIWQbA= contrib.go.opencensus.io/exporter/ocagent v0.4.7/go.mod h1:+KkYrcvvEN0E5ls626sqMv8PdMx2931feKtzIwP01qI= -contrib.go.opencensus.io/exporter/ocagent v0.4.9 h1:8ZbMXpyd04/3LILa/9Tzr8N4HzZNj6Vb2xsaSuR4nQI= -contrib.go.opencensus.io/exporter/ocagent v0.4.9/go.mod h1:ueLzZcP7LPhPulEBukGn4aLh7Mx9YJwpVJ9nL2FYltw= +contrib.go.opencensus.io/exporter/ocagent v0.4.10 h1:Trr4zF3bbDxrde1svPSW0PkGwCzoHY7f3JL8g5Gl+hM= +contrib.go.opencensus.io/exporter/ocagent v0.4.10/go.mod h1:ueLzZcP7LPhPulEBukGn4aLh7Mx9YJwpVJ9nL2FYltw= contrib.go.opencensus.io/exporter/stackdriver v0.0.0-20180919222851-d1e19f5c23e9/go.mod h1:hNe5qQofPbg6bLQY5wHCvQ7o+2E5P8PkegEuQ+MyRw0= git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= git.apache.org/thrift.git v0.12.0 h1:CMxsZlAmxKs+VAZMlDDL0wXciMblJcutQbEe3A9CYUM= @@ -70,6 +70,8 @@ github.com/cheggaaa/pb v1.0.28 h1:kWGpdAcSp3MxMU9CCHOwz/8V0kCHN4+9yQm2MzWuI98= github.com/cheggaaa/pb v1.0.28/go.mod h1:pQciLPpbU0oxA0h+VJYYLxO+XeDQb5pZijXscXHm81s= github.com/chrismalek/oktasdk-go v0.0.0-20181212195951-3430665dfaa0/go.mod h1:5d8DqS60xkj9k3aXfL3+mXBH0DPYO0FQjcKosxl+b/Q= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/colinmarc/hdfs/v2 v2.0.0 h1:zXrYPcGbdTS8li50eel904btC8XI35dwDYlTvtFKQe4= +github.com/colinmarc/hdfs/v2 v2.0.0/go.mod h1:uU5kspcuAB3y3XVcBew3IEXsYDTeyOHBbiBFSqCQHnk= github.com/containerd/continuity v0.0.0-20181203112020-004b46473808/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= github.com/coredns/coredns v0.0.0-20180121192821-d4bf076ccf4e/go.mod h1:zASH/MVDgR6XZTbxvOnsZfffS+31vg6Ackf/wo1+AM0= github.com/coredns/coredns v1.4.0 h1:RubBkYmkByUqZWWkjRHvNLnUHgkRVqAWgSMmRFvpE1A= @@ -120,8 +122,6 @@ github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1 github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/eclipse/paho.mqtt.golang v0.0.0-20181129145454-379fd9f99ba5/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= -github.com/eclipse/paho.mqtt.golang v1.1.1 h1:iPJYXJLaViCshRTW/PSqImSS6HJ2Rf671WR0bXZ2GIU= -github.com/eclipse/paho.mqtt.golang v1.1.1/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= github.com/eclipse/paho.mqtt.golang v1.1.2-0.20190322152051-20337d8c3947 h1:sgqVE7PzNVkD3Fkb2XPXMmZD4+tRRIpMVTkVryEeG6M= github.com/eclipse/paho.mqtt.golang v1.1.2-0.20190322152051-20337d8c3947/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= github.com/elazarl/go-bindata-assetfs v0.0.0-20151224045452-57eb5e1fc594/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= @@ -200,6 +200,7 @@ github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORR github.com/gopherjs/gopherjs v0.0.0-20181103185306-d547d1d9531e/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20190309154008-847fc94819f9 h1:Z0f701LpR4dqO92bP6TnIe3ZURClzJtBhds8R8u1HBE= github.com/gopherjs/gopherjs v0.0.0-20190309154008-847fc94819f9/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gopherjs/gopherjs v0.0.0-20190328170749-bb2674552d8f h1:4Gslotqbs16iAg+1KR/XdabIfq8TlAWHdwS5QJFksLc= github.com/gopherjs/gopherjs v0.0.0-20190328170749-bb2674552d8f/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorhill/cronexpr v0.0.0-20180427100037-88b0669f7d75/go.mod h1:g2644b03hfBX9Ov0ZBDgXXens4rxSxmqFBbhvKv2yVA= github.com/gorilla/context v0.0.0-20160525203319-aed02d124ae4/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= @@ -256,6 +257,7 @@ github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerX github.com/hashicorp/go-sockaddr v1.0.2 h1:ztczhD1jLxIRjVejw8gFomI1BQZOe2WoVOu0SyteCQc= github.com/hashicorp/go-sockaddr v1.0.2/go.mod h1:rB4wwRAUzs07qva3c5SdrY/NEtAUjGlgmH/UkBUC97A= github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= +github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -295,6 +297,8 @@ github.com/howeyc/gopass v0.0.0-20170109162249-bf9dde6d0d2c/go.mod h1:lADxMC39cJ github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/go-update v0.0.0-20160112193335-8152e7eb6ccf h1:WfD7VjIE6z8dIvMsI4/s+1qr5EL+zoIGev1BQj1eoJ8= github.com/inconshreveable/go-update v0.0.0-20160112193335-8152e7eb6ccf/go.mod h1:hyb9oH7vZsitZCiBt0ZvifOrB+qc8PS5IiilCIb87rg= +github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930 h1:v4CYlQ+HeysPHsr2QFiEO60gKqnvn1xwvuKhhAhuEkk= +github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jeffchao/backoff v0.0.0-20140404060208-9d7fd7aa17f2/go.mod h1:xkfESuHriIekR+4RoV+fu91j/CfnYM29Zi2tMFw5iD4= github.com/jefferai/jsonx v1.0.0/go.mod h1:OGmqmi2tTeI/PS+qQfBDToLHHJIy/RMp24fPo8vFvoQ= github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= @@ -366,8 +370,8 @@ github.com/minio/dsync v0.0.0-20190104003057-61c41ffdeea2/go.mod h1:eLQe3mXL0h02 github.com/minio/dsync v0.0.0-20190131060523-fb604afd87b2 h1:5Aq4Aro/PSNVgoWWTLPX+zcfDY87VhtKPv+7x1ERJ1w= github.com/minio/dsync v0.0.0-20190131060523-fb604afd87b2/go.mod h1:eLQe3mXL0h02kNpPtBJiLr1fIEIJftgXRAjncjQbxJo= github.com/minio/highwayhash v0.0.0-20181220011308-93ed73d64169/go.mod h1:NL8wme5P5MoscwAkXfGroz3VgpCdhBw3KYOu5mEsvpU= -github.com/minio/highwayhash v0.0.0-20190131021015-02ca4b43caa3 h1:qt0ysgOZnWt6g6WA/gDpLfl+iV77hEYWLu2dleBw3i4= -github.com/minio/highwayhash v0.0.0-20190131021015-02ca4b43caa3/go.mod h1:xQboMTeM9nY9v/LlAOxFctujiv5+Aq2hR5dxBpaMbdc= +github.com/minio/highwayhash v1.0.0 h1:iMSDhgUILCr0TNm8LWlSjF8N0ZIj2qbO8WHp6Q/J2BA= +github.com/minio/highwayhash v1.0.0/go.mod h1:xQboMTeM9nY9v/LlAOxFctujiv5+Aq2hR5dxBpaMbdc= github.com/minio/lsync v0.0.0-20190207022115-a4e43e3d0887 h1:MIpCDz3d2FR2a+FjdizuFdjsoeHuLlSkl3YNQJ55jV8= github.com/minio/lsync v0.0.0-20190207022115-a4e43e3d0887/go.mod h1:ni10+iSX7FO8N2rv41XM444V6w4rYO0dZo5KIkbn/YA= github.com/minio/mc v0.0.0-20190311071728-2e612b23d665/go.mod h1:7qLZXNjCD55DJ3iqe1uWoUh1MASRVd1M6wnqSdyhx7Y= @@ -379,8 +383,6 @@ github.com/minio/minio-go v0.0.0-20190227180923-59af836a7e6d/go.mod h1:/haSOWG8h github.com/minio/minio-go v0.0.0-20190313212832-5d20267d970d/go.mod h1:/haSOWG8hQNx2+JOfLJ9GKp61EAmgPwRVw/Sac0NzaM= github.com/minio/minio-go v0.0.0-20190327203652-5325257a208f h1:u+iNxfkLrfyWp7KxSTV+ZhO4SMHT6qUFxSZ6yhYMQ0Q= github.com/minio/minio-go v0.0.0-20190327203652-5325257a208f/go.mod h1:/haSOWG8hQNx2+JOfLJ9GKp61EAmgPwRVw/Sac0NzaM= -github.com/minio/minio-go v6.0.14+incompatible h1:fnV+GD28LeqdN6vT2XdGKW8Qe/IfjJDswNVuni6km9o= -github.com/minio/minio-go v6.0.14+incompatible/go.mod h1:7guKYtitv8dktvNUGrhzmNlA5wrAABTQXCoesZdFQO8= github.com/minio/parquet-go v0.0.0-20190318185229-9d767baf1679 h1:OMKaN/82sBHUZPvjYNBFituHExa1OGY63eACDGtetKs= github.com/minio/parquet-go v0.0.0-20190318185229-9d767baf1679/go.mod h1:J+goXSuzlte5imWMqb6cUWC/tbYYysUHctwmKXomYzM= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= @@ -449,6 +451,7 @@ github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FI github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY= github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ= +github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= @@ -516,19 +519,21 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1 github.com/smartystreets/assertions v0.0.0-20190116191733-b6c0e53d7304/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac h1:wbW+Bybf9pXxnCFAOWZTqkRjAc7rAIwo2e1ArUhiHxg= github.com/smartystreets/assertions v0.0.0-20190215210624-980c5ac6f3ac/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/assertions v0.0.0-20190401200700-3f99fa72afbb/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3 h1:hBSHahWMEgzwRyS6dRpxY0XyjZsHyQ61s084wo5PJe0= +github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/go-aws-auth v0.0.0-20180515143844-0c1422d1fdb9/go.mod h1:SnhjPscd9TpLiy1LpzGSKh3bXCfxxXuqd9xmQJy3slM= github.com/smartystreets/goconvey v0.0.0-20181108003508-044398e4856c/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s= github.com/smartystreets/goconvey v0.0.0-20190222223459-a17d461953aa/go.mod h1:2RVY1rIf+2J2o/IM9+vPq9RzmHDSseB7FoXiSNIUsoU= github.com/smartystreets/goconvey v0.0.0-20190306220146-200a235640ff h1:86HlEv0yBCry9syNuylzqznKXDK11p6D0DT596yNMys= github.com/smartystreets/goconvey v0.0.0-20190306220146-200a235640ff/go.mod h1:KSQcGKpxUMHk3nbYzs/tIBAM2iDooCn0BmttHOJEbLs= +github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a h1:pa8hGb/2YqsZKovtsgrwcDH1RZhVbTKCjLp47XpqCDs= github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/streadway/amqp v0.0.0-20160311215503-2e25825abdbd/go.mod h1:1WNBiOZtZQLpVAyu0iTduoJL9hEsMloAK5XWrtW0xdY= -github.com/streadway/amqp v0.0.0-20190312223743-14f78b41ce6d h1:ToACqFOOYVdz7PswtVcAawttvtdGlLhoAsXdhYFQeEI= -github.com/streadway/amqp v0.0.0-20190312223743-14f78b41ce6d/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= +github.com/streadway/amqp v0.0.0-20190402114354-16ed540749f6 h1:D8lgxQkWwQ6cloDE8Qql7XKmxYgbReNY1KhQUsBQvBk= +github.com/streadway/amqp v0.0.0-20190402114354-16ed540749f6/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -573,6 +578,7 @@ go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= golang.org/x/build v0.0.0-20190314133821-5284462c4bec/go.mod h1:atTaCNAy0f16Ah5aV1gMSwgiKVHwu/JncqDpuRr7lS4= +golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180820150726-614d502a4dac/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -613,10 +619,6 @@ golang.org/x/net v0.0.0-20190301231341-16b79f2e4e95/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190318221613-d196dffd7c2b/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190324223953-e3b2ff56ed87/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190328230028-74de082e2cca h1:hyA6yiAgbUwuWqtscNvWAI7U1CtlaD1KilQ6iudt1aI= -golang.org/x/net v0.0.0-20190328230028-74de082e2cca/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190403144856-b630fd6fe46b h1:/zjbcJPEGAyu6Is/VBOALsgdi4z9+kz/Vtdm6S+beD0= -golang.org/x/net v0.0.0-20190403144856-b630fd6fe46b/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3 h1:0GoQqolDA55aaLxZyTzK/Y2ePZzZTUrRacwib7cNsYQ= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/oauth2 v0.0.0-20180603041954-1e0a3fa8ba9a/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -653,8 +655,6 @@ golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190304154630-e844e0132e93/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190318195719-6c81ef8f67ca/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190322080309-f49334f85ddc/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190329044733-9eb1bfa1ce65 h1:hOY+O8MxdkPV10pNf7/XEHaySCiPKxixMKUshfHsGn0= -golang.org/x/sys v0.0.0-20190329044733-9eb1bfa1ce65/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e h1:nFYrTHrdrAOpShe27kaFHjsqYSEQ0KWqdWLu3xuZJts= golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67 h1:1Fzlr8kkDLQwqMP8GxrhptBLqZG/EDpiATneiZHY998= @@ -675,11 +675,6 @@ golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3 golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190318200714-bb1270c20edf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190401201229-1bac838f5b88/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190404132500-923d25813098 h1:MtqjsZmyGRgMmLUgxnmMJ6RYdvd2ib8ipiayHhqSxs4= -golang.org/x/tools v0.0.0-20190404132500-923d25813098/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190408220357-e5b8258f4918 h1:Nc6hX4GCe+yXisksIiO0/6ToNDV42upjoQwt2IwHKUM= -golang.org/x/tools v0.0.0-20190408220357-e5b8258f4918/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= google.golang.org/api v0.0.0-20180603000442-8e296ef26005/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.0.0-20180916000451-19ff8768a5c0/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= @@ -723,6 +718,8 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/cheggaaa/pb.v1 v1.0.27/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= +gopkg.in/cheggaaa/pb.v1 v1.0.28 h1:n1tBJnnK2r7g9OW2btFH91V92STTUevLXYFb8gy9EMk= +gopkg.in/cheggaaa/pb.v1 v1.0.28/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/cheggaaa/pb.v2 v2.0.6 h1:L2KAo2l2ZQTzxmh8b9RdQpzgLpK2mX3paGCMJSUugBk= gopkg.in/cheggaaa/pb.v2 v2.0.6/go.mod h1:0CiZ1p8pvtxBlQpLXkHuUTpdJ1shm3OqCF1QugkjHL4= gopkg.in/fatih/color.v1 v1.7.0 h1:bYGjb+HezBM6j/QmgBfgm1adxHpzzrss6bj4r9ROppk= @@ -736,6 +733,16 @@ gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= gopkg.in/ini.v1 v1.41.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.42.0 h1:7N3gPTt50s8GuLortA00n8AqRTk75qOP98+mTPpgzRk= gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw= +gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= +gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM= +gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= +gopkg.in/jcmturner/goidentity.v2 v2.0.0 h1:6Bmcdaxb0dD3HyHbo/MtJ2Q1wXLDuZJFwXZmuZvM+zw= +gopkg.in/jcmturner/goidentity.v2 v2.0.0/go.mod h1:vCwK9HeXksMeUmQ4SxDd1tRz4LejrKh3KRVjQWhjvZI= +gopkg.in/jcmturner/gokrb5.v5 v5.3.0 h1:RS1MYApX27Hx1Xw7NECs7XxGxxrm69/4OmaRuX9kwec= +gopkg.in/jcmturner/gokrb5.v5 v5.3.0/go.mod h1:oQz8Wc5GsctOTgCVyKad1Vw4TCWz5G6gfIQr88RPv4k= +gopkg.in/jcmturner/rpc.v0 v0.0.2 h1:wBTgrbL1qmLBUPsYVCqdJiI5aJgQhexmK+JkTHPUNJI= +gopkg.in/jcmturner/rpc.v0 v0.0.2/go.mod h1:NzMq6cRzR9lipgw7WxRBHNx5N8SifBuaCQsOT1kWY/E= gopkg.in/mattn/go-colorable.v0 v0.1.0 h1:WYuADWvfvYC07fm8ygYB3LMcsc5CunpxfMGKawHkAos= gopkg.in/mattn/go-colorable.v0 v0.1.0/go.mod h1:BVJlBXzARQxdi3nZo6f6bnl5yR20/tOL6p+V0KejgSY= gopkg.in/mattn/go-isatty.v0 v0.0.4 h1:NtS1rQGQr4IaFWBGz4Cz4BhB///gyys4gDVtKA7hIsc=