Add data usage collect with its new admin API (#8553)
Admin data usage info API returns the following (Only FS & XL, for now) - Number of buckets - Number of objects - The total size of objects - Objects histogram - Bucket sizesmaster
parent
e2c5d29017
commit
555969ee42
@ -0,0 +1,142 @@ |
||||
/* |
||||
* 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 cmd |
||||
|
||||
import ( |
||||
"bytes" |
||||
"context" |
||||
"encoding/json" |
||||
"time" |
||||
|
||||
"github.com/minio/minio/cmd/logger" |
||||
"github.com/minio/minio/pkg/hash" |
||||
) |
||||
|
||||
const ( |
||||
dataUsageObjName = "data-usage" |
||||
dataUsageCrawlInterval = 12 * time.Hour |
||||
) |
||||
|
||||
func initDataUsageStats() { |
||||
go runDataUsageInfoUpdateRoutine() |
||||
} |
||||
|
||||
func runDataUsageInfoUpdateRoutine() { |
||||
// Wait until the object layer is ready
|
||||
var objAPI ObjectLayer |
||||
for { |
||||
objAPI = newObjectLayerWithoutSafeModeFn() |
||||
if objAPI == nil { |
||||
time.Sleep(time.Second) |
||||
continue |
||||
} |
||||
break |
||||
} |
||||
|
||||
ctx := context.Background() |
||||
|
||||
switch v := objAPI.(type) { |
||||
case *xlZones: |
||||
runDataUsageInfoForXLZones(ctx, v, GlobalServiceDoneCh) |
||||
case *FSObjects: |
||||
runDataUsageInfoForFS(ctx, v, GlobalServiceDoneCh) |
||||
default: |
||||
return |
||||
} |
||||
} |
||||
|
||||
func runDataUsageInfoForFS(ctx context.Context, fsObj *FSObjects, endCh <-chan struct{}) { |
||||
t := time.NewTicker(dataUsageCrawlInterval) |
||||
defer t.Stop() |
||||
for { |
||||
// Get data usage info of the FS Object
|
||||
usageInfo := fsObj.crawlAndGetDataUsageInfo(ctx, endCh) |
||||
// Save the data usage in the disk
|
||||
err := storeDataUsageInBackend(ctx, fsObj, usageInfo) |
||||
if err != nil { |
||||
logger.LogIf(ctx, err) |
||||
} |
||||
select { |
||||
case <-endCh: |
||||
return |
||||
// Wait until the next crawl interval
|
||||
case <-t.C: |
||||
} |
||||
} |
||||
} |
||||
|
||||
func runDataUsageInfoForXLZones(ctx context.Context, z *xlZones, endCh <-chan struct{}) { |
||||
locker := z.NewNSLock(ctx, minioMetaBucket, "leader-data-usage-info") |
||||
for { |
||||
err := locker.GetLock(newDynamicTimeout(time.Millisecond, time.Millisecond)) |
||||
if err != nil { |
||||
time.Sleep(5 * time.Minute) |
||||
continue |
||||
} |
||||
// Break without locking
|
||||
break |
||||
} |
||||
|
||||
t := time.NewTicker(dataUsageCrawlInterval) |
||||
defer t.Stop() |
||||
for { |
||||
usageInfo := z.crawlAndGetDataUsage(ctx, endCh) |
||||
err := storeDataUsageInBackend(ctx, z, usageInfo) |
||||
if err != nil { |
||||
logger.LogIf(ctx, err) |
||||
} |
||||
select { |
||||
case <-endCh: |
||||
locker.Unlock() |
||||
return |
||||
case <-t.C: |
||||
} |
||||
} |
||||
} |
||||
|
||||
func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, dataUsageInfo DataUsageInfo) error { |
||||
dataUsageJSON, err := json.Marshal(dataUsageInfo) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
|
||||
size := int64(len(dataUsageJSON)) |
||||
r, err := hash.NewReader(bytes.NewReader(dataUsageJSON), size, "", "", size, false) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
|
||||
_, err = objAPI.PutObject(ctx, minioMetaBackgroundOpsBucket, dataUsageObjName, NewPutObjReader(r, nil, nil), ObjectOptions{}) |
||||
return err |
||||
} |
||||
|
||||
func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (DataUsageInfo, error) { |
||||
var dataUsageInfoJSON bytes.Buffer |
||||
|
||||
err := objAPI.GetObject(ctx, minioMetaBackgroundOpsBucket, dataUsageObjName, 0, -1, &dataUsageInfoJSON, "", ObjectOptions{}) |
||||
if err != nil { |
||||
return DataUsageInfo{}, nil |
||||
} |
||||
|
||||
var dataUsageInfo DataUsageInfo |
||||
err = json.Unmarshal(dataUsageInfoJSON.Bytes(), &dataUsageInfo) |
||||
if err != nil { |
||||
return DataUsageInfo{}, err |
||||
} |
||||
|
||||
return dataUsageInfo, nil |
||||
} |
@ -0,0 +1,226 @@ |
||||
// Copyright 2016 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// This code is imported from "golang.org/x/tools/internal/fastwalk",
|
||||
// only fastwalk.go is imported since we already implement readDir()
|
||||
// with some little tweaks.
|
||||
|
||||
package cmd |
||||
|
||||
import ( |
||||
"errors" |
||||
"os" |
||||
"path/filepath" |
||||
"runtime" |
||||
"strings" |
||||
"sync" |
||||
) |
||||
|
||||
// ErrTraverseLink is used as a return value from WalkFuncs to indicate that the
|
||||
// symlink named in the call may be traversed.
|
||||
var ErrTraverseLink = errors.New("fastwalk: traverse symlink, assuming target is a directory") |
||||
|
||||
// ErrSkipFiles is a used as a return value from WalkFuncs to indicate that the
|
||||
// callback should not be called for any other files in the current directory.
|
||||
// Child directories will still be traversed.
|
||||
var ErrSkipFiles = errors.New("fastwalk: skip remaining files in directory") |
||||
|
||||
// Walk is a faster implementation of filepath.Walk.
|
||||
//
|
||||
// filepath.Walk's design necessarily calls os.Lstat on each file,
|
||||
// even if the caller needs less info.
|
||||
// Many tools need only the type of each file.
|
||||
// On some platforms, this information is provided directly by the readdir
|
||||
// system call, avoiding the need to stat each file individually.
|
||||
// fastwalk_unix.go contains a fork of the syscall routines.
|
||||
//
|
||||
// See golang.org/issue/16399
|
||||
//
|
||||
// Walk walks the file tree rooted at root, calling walkFn for
|
||||
// each file or directory in the tree, including root.
|
||||
//
|
||||
// If fastWalk returns filepath.SkipDir, the directory is skipped.
|
||||
//
|
||||
// Unlike filepath.Walk:
|
||||
// * file stat calls must be done by the user.
|
||||
// The only provided metadata is the file type, which does not include
|
||||
// any permission bits.
|
||||
// * multiple goroutines stat the filesystem concurrently. The provided
|
||||
// walkFn must be safe for concurrent use.
|
||||
// * fastWalk can follow symlinks if walkFn returns the TraverseLink
|
||||
// sentinel error. It is the walkFn's responsibility to prevent
|
||||
// fastWalk from going into symlink cycles.
|
||||
func fastWalk(root string, walkFn func(path string, typ os.FileMode) error) error { |
||||
// TODO(bradfitz): make numWorkers configurable? We used a
|
||||
// minimum of 4 to give the kernel more info about multiple
|
||||
// things we want, in hopes its I/O scheduling can take
|
||||
// advantage of that. Hopefully most are in cache. Maybe 4 is
|
||||
// even too low of a minimum. Profile more.
|
||||
numWorkers := 4 |
||||
if n := runtime.NumCPU(); n > numWorkers { |
||||
numWorkers = n |
||||
} |
||||
|
||||
// Make sure to wait for all workers to finish, otherwise
|
||||
// walkFn could still be called after returning. This Wait call
|
||||
// runs after close(e.donec) below.
|
||||
var wg sync.WaitGroup |
||||
defer wg.Wait() |
||||
|
||||
w := &walker{ |
||||
fn: walkFn, |
||||
enqueuec: make(chan walkItem, numWorkers), // buffered for performance
|
||||
workc: make(chan walkItem, numWorkers), // buffered for performance
|
||||
donec: make(chan struct{}), |
||||
|
||||
// buffered for correctness & not leaking goroutines:
|
||||
resc: make(chan error, numWorkers), |
||||
} |
||||
defer close(w.donec) |
||||
|
||||
for i := 0; i < numWorkers; i++ { |
||||
wg.Add(1) |
||||
go w.doWork(&wg) |
||||
} |
||||
todo := []walkItem{{dir: root}} |
||||
out := 0 |
||||
for { |
||||
workc := w.workc |
||||
var workItem walkItem |
||||
if len(todo) == 0 { |
||||
workc = nil |
||||
} else { |
||||
workItem = todo[len(todo)-1] |
||||
} |
||||
select { |
||||
case workc <- workItem: |
||||
todo = todo[:len(todo)-1] |
||||
out++ |
||||
case it := <-w.enqueuec: |
||||
todo = append(todo, it) |
||||
case err := <-w.resc: |
||||
out-- |
||||
if err != nil { |
||||
return err |
||||
} |
||||
if out == 0 && len(todo) == 0 { |
||||
// It's safe to quit here, as long as the buffered
|
||||
// enqueue channel isn't also readable, which might
|
||||
// happen if the worker sends both another unit of
|
||||
// work and its result before the other select was
|
||||
// scheduled and both w.resc and w.enqueuec were
|
||||
// readable.
|
||||
select { |
||||
case it := <-w.enqueuec: |
||||
todo = append(todo, it) |
||||
default: |
||||
return nil |
||||
} |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
// doWork reads directories as instructed (via workc) and runs the
|
||||
// user's callback function.
|
||||
func (w *walker) doWork(wg *sync.WaitGroup) { |
||||
defer wg.Done() |
||||
for { |
||||
select { |
||||
case <-w.donec: |
||||
return |
||||
case it := <-w.workc: |
||||
select { |
||||
case <-w.donec: |
||||
return |
||||
case w.resc <- w.walk(it.dir, !it.callbackDone): |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
type walker struct { |
||||
fn func(path string, typ os.FileMode) error |
||||
|
||||
donec chan struct{} // closed on fastWalk's return
|
||||
workc chan walkItem // to workers
|
||||
enqueuec chan walkItem // from workers
|
||||
resc chan error // from workers
|
||||
} |
||||
|
||||
type walkItem struct { |
||||
dir string |
||||
callbackDone bool // callback already called; don't do it again
|
||||
} |
||||
|
||||
func (w *walker) enqueue(it walkItem) { |
||||
select { |
||||
case w.enqueuec <- it: |
||||
case <-w.donec: |
||||
} |
||||
} |
||||
|
||||
func (w *walker) onDirEnt(dirName, baseName string, typ os.FileMode) error { |
||||
joined := dirName + string(os.PathSeparator) + baseName |
||||
if typ == os.ModeDir { |
||||
w.enqueue(walkItem{dir: joined}) |
||||
return nil |
||||
} |
||||
|
||||
err := w.fn(joined, typ) |
||||
if typ == os.ModeSymlink { |
||||
if err == ErrTraverseLink { |
||||
// Set callbackDone so we don't call it twice for both the
|
||||
// symlink-as-symlink and the symlink-as-directory later:
|
||||
w.enqueue(walkItem{dir: joined, callbackDone: true}) |
||||
return nil |
||||
} |
||||
if err == filepath.SkipDir { |
||||
// Permit SkipDir on symlinks too.
|
||||
return nil |
||||
} |
||||
} |
||||
return err |
||||
} |
||||
|
||||
func readDirFn(dirName string, fn func(dirName, entName string, typ os.FileMode) error) error { |
||||
fis, err := readDir(dirName) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
skipFiles := false |
||||
for _, fi := range fis { |
||||
var mode os.FileMode |
||||
if strings.HasSuffix(fi, SlashSeparator) { |
||||
mode |= os.ModeDir |
||||
} |
||||
|
||||
if mode == 0 && skipFiles { |
||||
continue |
||||
} |
||||
|
||||
if err := fn(dirName, fi, mode); err != nil { |
||||
if err == ErrSkipFiles { |
||||
skipFiles = true |
||||
continue |
||||
} |
||||
return err |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
func (w *walker) walk(root string, runUserCallback bool) error { |
||||
if runUserCallback { |
||||
err := w.fn(root, os.ModeDir) |
||||
if err == filepath.SkipDir { |
||||
return nil |
||||
} |
||||
if err != nil { |
||||
return err |
||||
} |
||||
} |
||||
|
||||
return readDirFn(root, w.onDirEnt) |
||||
} |
@ -0,0 +1,44 @@ |
||||
// +build ignore
|
||||
|
||||
/* |
||||
* 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 main |
||||
|
||||
import ( |
||||
"log" |
||||
|
||||
"github.com/minio/minio/pkg/madmin" |
||||
) |
||||
|
||||
func main() { |
||||
// Note: YOUR-ACCESSKEYID, YOUR-SECRETACCESSKEY and my-bucketname are
|
||||
// dummy values, please replace them with original values.
|
||||
|
||||
// API requests are secure (HTTPS) if secure=true and insecure (HTTPS) otherwise.
|
||||
// New returns an MinIO Admin client object.
|
||||
madmClnt, err := madmin.New("your-minio.example.com:9000", "YOUR-ACCESSKEYID", "YOUR-SECRETACCESSKEY", true) |
||||
if err != nil { |
||||
log.Fatalln(err) |
||||
} |
||||
|
||||
dataUsageInfo, err := madmClnt.DataUsageInfo() |
||||
if err != nil { |
||||
log.Fatalln(err) |
||||
} |
||||
log.Println(dataUsageInfo) |
||||
} |
Loading…
Reference in new issue