| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | /* | 
					
						
							|  |  |  |  * 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" | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	"errors" | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 	"os" | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	"path" | 
					
						
							|  |  |  | 	"strconv" | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	"time" | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-01-08 19:31:43 +08:00
										 |  |  | 	jsoniter "github.com/json-iterator/go" | 
					
						
							| 
									
										
										
										
											2020-03-05 07:51:03 +08:00
										 |  |  | 	"github.com/minio/minio/cmd/config" | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	"github.com/minio/minio/cmd/logger" | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	"github.com/minio/minio/pkg/color" | 
					
						
							| 
									
										
										
										
											2020-03-05 07:51:03 +08:00
										 |  |  | 	"github.com/minio/minio/pkg/env" | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	"github.com/minio/minio/pkg/hash" | 
					
						
							|  |  |  | ) | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | const ( | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	dataUsageObjName         = ".usage.json" | 
					
						
							|  |  |  | 	dataUsageCacheName       = ".usage-cache.bin" | 
					
						
							|  |  |  | 	envDataUsageCrawlConf    = "MINIO_DISK_USAGE_CRAWL_ENABLE" | 
					
						
							|  |  |  | 	envDataUsageCrawlDelay   = "MINIO_DISK_USAGE_CRAWL_DELAY" | 
					
						
							|  |  |  | 	envDataUsageCrawlDebug   = "MINIO_DISK_USAGE_CRAWL_DEBUG" | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	dataUsageSleepPerFolder  = 1 * time.Millisecond | 
					
						
							|  |  |  | 	dataUsageSleepDefMult    = 10.0 | 
					
						
							|  |  |  | 	dataUsageUpdateDirCycles = 16 | 
					
						
							|  |  |  | 	dataUsageRoot            = SlashSeparator | 
					
						
							|  |  |  | 	dataUsageBucket          = minioMetaBucket + SlashSeparator + bucketMetaPrefix | 
					
						
							|  |  |  | 	dataUsageStartDelay      = 5 * time.Minute // Time to wait on startup and between cycles.
 | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | ) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | // initDataUsageStats will start the crawler unless disabled.
 | 
					
						
							| 
									
										
										
										
											2020-03-23 03:16:36 +08:00
										 |  |  | func initDataUsageStats(ctx context.Context, objAPI ObjectLayer) { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	if env.Get(envDataUsageCrawlConf, config.EnableOn) == config.EnableOn { | 
					
						
							| 
									
										
										
										
											2020-03-23 03:16:36 +08:00
										 |  |  | 		go runDataUsageInfo(ctx, objAPI) | 
					
						
							| 
									
										
										
										
											2020-03-05 07:51:03 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | func runDataUsageInfo(ctx context.Context, objAPI ObjectLayer) { | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	for { | 
					
						
							|  |  |  | 		select { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		case <-ctx.Done(): | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 			return | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		case <-time.NewTimer(dataUsageStartDelay).C: | 
					
						
							| 
									
										
										
										
											2020-03-23 03:16:36 +08:00
										 |  |  | 			// Wait before starting next cycle and wait on startup.
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			results := make(chan DataUsageInfo, 1) | 
					
						
							|  |  |  | 			go storeDataUsageInBackend(ctx, objAPI, results) | 
					
						
							|  |  |  | 			err := objAPI.CrawlAndGetDataUsage(ctx, results) | 
					
						
							|  |  |  | 			close(results) | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 			logger.LogIf(ctx, err) | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 		} | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | // storeDataUsageInBackend will store all objects sent on the gui channel until closed.
 | 
					
						
							|  |  |  | func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, gui <-chan DataUsageInfo) { | 
					
						
							|  |  |  | 	for dataUsageInfo := range gui { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		dataUsageJSON, err := json.Marshal(dataUsageInfo) | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		size := int64(len(dataUsageJSON)) | 
					
						
							|  |  |  | 		r, err := hash.NewReader(bytes.NewReader(dataUsageJSON), size, "", "", size, false) | 
					
						
							|  |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		_, err = objAPI.PutObject(ctx, dataUsageBucket, dataUsageObjName, NewPutObjReader(r, nil, nil), ObjectOptions{}) | 
					
						
							|  |  |  | 		logger.LogIf(ctx, err) | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (DataUsageInfo, error) { | 
					
						
							|  |  |  | 	var dataUsageInfoJSON bytes.Buffer | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	err := objAPI.GetObject(ctx, dataUsageBucket, dataUsageObjName, 0, -1, &dataUsageInfoJSON, "", ObjectOptions{}) | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	if err != nil { | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 		if isErrObjectNotFound(err) { | 
					
						
							|  |  |  | 			return DataUsageInfo{}, nil | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		return DataUsageInfo{}, toObjectErr(err, dataUsageBucket, dataUsageObjName) | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	var dataUsageInfo DataUsageInfo | 
					
						
							| 
									
										
										
										
											2020-01-08 19:31:43 +08:00
										 |  |  | 	var json = jsoniter.ConfigCompatibleWithStandardLibrary | 
					
						
							| 
									
										
										
										
											2019-12-12 22:02:37 +08:00
										 |  |  | 	err = json.Unmarshal(dataUsageInfoJSON.Bytes(), &dataUsageInfo) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		return DataUsageInfo{}, err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	return dataUsageInfo, nil | 
					
						
							|  |  |  | } | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | // Item represents each file while walking.
 | 
					
						
							|  |  |  | type Item struct { | 
					
						
							|  |  |  | 	Path string | 
					
						
							|  |  |  | 	Typ  os.FileMode | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | type getSizeFn func(item Item) (int64, error) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | type cachedFolder struct { | 
					
						
							|  |  |  | 	name   string | 
					
						
							|  |  |  | 	parent *dataUsageHash | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | type folderScanner struct { | 
					
						
							|  |  |  | 	root               string | 
					
						
							|  |  |  | 	getSize            getSizeFn | 
					
						
							|  |  |  | 	oldCache           dataUsageCache | 
					
						
							|  |  |  | 	newCache           dataUsageCache | 
					
						
							|  |  |  | 	waitForLowActiveIO func() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	dataUsageCrawlMult  float64 | 
					
						
							|  |  |  | 	dataUsageCrawlDebug bool | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	newFolders      []cachedFolder | 
					
						
							|  |  |  | 	existingFolders []cachedFolder | 
					
						
							|  |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // sleepDuration multiplies the duration d by x and sleeps if is more than 100 micro seconds.
 | 
					
						
							|  |  |  | // sleep is limited to max 1 second.
 | 
					
						
							|  |  |  | func sleepDuration(d time.Duration, x float64) { | 
					
						
							|  |  |  | 	// Don't sleep for really small amount of time
 | 
					
						
							|  |  |  | 	if d := time.Duration(float64(d) * x); d > time.Microsecond*100 { | 
					
						
							|  |  |  | 		if d > time.Second { | 
					
						
							|  |  |  | 			d = time.Second | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		time.Sleep(d) | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | } | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | // scanQueuedLevels will scan the provided folders.
 | 
					
						
							|  |  |  | // Files found in the folders will be added to f.newCache.
 | 
					
						
							|  |  |  | // If final is provided folders will be put into f.newFolders or f.existingFolders.
 | 
					
						
							|  |  |  | // If final is not provided the folders found are returned from the function.
 | 
					
						
							|  |  |  | func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFolder, final bool) ([]cachedFolder, error) { | 
					
						
							|  |  |  | 	var nextFolders []cachedFolder | 
					
						
							|  |  |  | 	done := ctx.Done() | 
					
						
							|  |  |  | 	for _, folder := range folders { | 
					
						
							|  |  |  | 		select { | 
					
						
							|  |  |  | 		case <-done: | 
					
						
							|  |  |  | 			return nil, ctx.Err() | 
					
						
							|  |  |  | 		default: | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		f.waitForLowActiveIO() | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		sleepDuration(dataUsageSleepPerFolder, f.dataUsageCrawlMult) | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 		cache := dataUsageEntry{} | 
					
						
							|  |  |  | 		thisHash := hashPath(folder.name) | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 		err := readDirFn(path.Join(f.root, folder.name), func(entName string, typ os.FileMode) error { | 
					
						
							|  |  |  | 			// Parse
 | 
					
						
							|  |  |  | 			entName = path.Clean(path.Join(folder.name, entName)) | 
					
						
							|  |  |  | 			bucket, _ := path2BucketObjectWithBasePath(f.root, entName) | 
					
						
							|  |  |  | 			if bucket == "" { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 				if f.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 					logger.Info(color.Green("data-usage:")+" no bucket (%s,%s)", f.root, entName) | 
					
						
							|  |  |  | 				} | 
					
						
							|  |  |  | 				return nil | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 			if isReservedOrInvalidBucket(bucket, false) { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 				if f.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 					logger.Info(color.Green("data-usage:")+" invalid bucket: %v, entry: %v", bucket, entName) | 
					
						
							|  |  |  | 				} | 
					
						
							|  |  |  | 				return nil | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 			select { | 
					
						
							|  |  |  | 			case <-done: | 
					
						
							|  |  |  | 				return ctx.Err() | 
					
						
							|  |  |  | 			default: | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 			if typ&os.ModeDir != 0 { | 
					
						
							|  |  |  | 				h := hashPath(entName) | 
					
						
							|  |  |  | 				_, exists := f.oldCache.Cache[h] | 
					
						
							|  |  |  | 				cache.addChildString(entName) | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 				this := cachedFolder{name: entName, parent: &thisHash} | 
					
						
							|  |  |  | 				cache.addChild(h) | 
					
						
							|  |  |  | 				if final { | 
					
						
							|  |  |  | 					if exists { | 
					
						
							|  |  |  | 						f.existingFolders = append(f.existingFolders, this) | 
					
						
							|  |  |  | 					} else { | 
					
						
							|  |  |  | 						f.newFolders = append(f.newFolders, this) | 
					
						
							|  |  |  | 					} | 
					
						
							|  |  |  | 				} else { | 
					
						
							|  |  |  | 					nextFolders = append(nextFolders, this) | 
					
						
							|  |  |  | 				} | 
					
						
							|  |  |  | 				return nil | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 			f.waitForLowActiveIO() | 
					
						
							|  |  |  | 			// Dynamic time delay.
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 			t := UTCNow() | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			// Get file size, ignore errors.
 | 
					
						
							|  |  |  | 			size, err := f.getSize(Item{Path: path.Join(f.root, entName), Typ: typ}) | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 			sleepDuration(time.Since(t), f.dataUsageCrawlMult) | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			if err == errSkipFile { | 
					
						
							|  |  |  | 				return nil | 
					
						
							|  |  |  | 			} | 
					
						
							|  |  |  | 			logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 			cache.Size += size | 
					
						
							|  |  |  | 			cache.Objects++ | 
					
						
							|  |  |  | 			cache.ObjSizes.add(size) | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 			return nil | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		}) | 
					
						
							|  |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			return nil, err | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		f.newCache.replaceHashed(thisHash, folder.parent, cache) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	return nextFolders, nil | 
					
						
							|  |  |  | } | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | // deepScanFolder will deep scan a folder and return the size if no error occurs.
 | 
					
						
							|  |  |  | func (f *folderScanner) deepScanFolder(ctx context.Context, folder string) (*dataUsageEntry, error) { | 
					
						
							|  |  |  | 	var cache dataUsageEntry | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	done := ctx.Done() | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	var addDir func(entName string, typ os.FileMode) error | 
					
						
							|  |  |  | 	var dirStack = []string{f.root, folder} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	addDir = func(entName string, typ os.FileMode) error { | 
					
						
							|  |  |  | 		select { | 
					
						
							|  |  |  | 		case <-done: | 
					
						
							|  |  |  | 			return ctx.Err() | 
					
						
							|  |  |  | 		default: | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 		} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		f.waitForLowActiveIO() | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 		if typ&os.ModeDir != 0 { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			dirStack = append(dirStack, entName) | 
					
						
							|  |  |  | 			err := readDirFn(path.Join(dirStack...), addDir) | 
					
						
							|  |  |  | 			dirStack = dirStack[:len(dirStack)-1] | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 			sleepDuration(dataUsageSleepPerFolder, f.dataUsageCrawlMult) | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			return err | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		// Dynamic time delay.
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		t := UTCNow() | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 		// Get file size, ignore errors.
 | 
					
						
							|  |  |  | 		dirStack = append(dirStack, entName) | 
					
						
							|  |  |  | 		fileName := path.Join(dirStack...) | 
					
						
							|  |  |  | 		dirStack = dirStack[:len(dirStack)-1] | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 		size, err := f.getSize(Item{Path: fileName, Typ: typ}) | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 		// Don't sleep for really small amount of time
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		sleepDuration(time.Since(t), f.dataUsageCrawlMult) | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 		if err == errSkipFile { | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 			return nil | 
					
						
							|  |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 		cache.Size += size | 
					
						
							|  |  |  | 		cache.Objects++ | 
					
						
							|  |  |  | 		cache.ObjSizes.add(size) | 
					
						
							|  |  |  | 		return nil | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	err := readDirFn(path.Join(dirStack...), addDir) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		return nil, err | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 	return &cache, nil | 
					
						
							|  |  |  | } | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | // updateUsage will crawl the basepath+cache.Info.Name and return an updated cache.
 | 
					
						
							|  |  |  | // The returned cache will always be valid, but may not be updated from the existing.
 | 
					
						
							|  |  |  | // Before each operation waitForLowActiveIO is called which can be used to temporarily halt the crawler.
 | 
					
						
							|  |  |  | // If the supplied context is canceled the function will return at the first chance.
 | 
					
						
							|  |  |  | func updateUsage(ctx context.Context, basePath string, cache dataUsageCache, waitForLowActiveIO func(), getSize getSizeFn) (dataUsageCache, error) { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	t := UTCNow() | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	dataUsageDebug := env.Get(envDataUsageCrawlDebug, config.EnableOff) == config.EnableOn | 
					
						
							|  |  |  | 	defer func() { | 
					
						
							|  |  |  | 		if dataUsageDebug { | 
					
						
							|  |  |  | 			logger.Info(color.Green("updateUsage")+" Crawl time at %s: %v", basePath, time.Since(t)) | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	}() | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	if cache.Info.Name == "" { | 
					
						
							|  |  |  | 		cache.Info.Name = dataUsageRoot | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	delayMult, err := strconv.ParseFloat(env.Get(envDataUsageCrawlDelay, "10.0"), 64) | 
					
						
							|  |  |  | 	if err != nil { | 
					
						
							|  |  |  | 		logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 		delayMult = dataUsageSleepDefMult | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	s := folderScanner{ | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		root:                basePath, | 
					
						
							|  |  |  | 		getSize:             getSize, | 
					
						
							|  |  |  | 		oldCache:            cache, | 
					
						
							|  |  |  | 		newCache:            dataUsageCache{Info: cache.Info}, | 
					
						
							|  |  |  | 		waitForLowActiveIO:  waitForLowActiveIO, | 
					
						
							|  |  |  | 		newFolders:          nil, | 
					
						
							|  |  |  | 		existingFolders:     nil, | 
					
						
							|  |  |  | 		dataUsageCrawlMult:  delayMult, | 
					
						
							|  |  |  | 		dataUsageCrawlDebug: dataUsageDebug, | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	if s.dataUsageCrawlDebug { | 
					
						
							|  |  |  | 		logger.Info(color.Green("runDataUsageInfo:") + " Starting crawler master") | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	done := ctx.Done() | 
					
						
							|  |  |  | 	var flattenLevels = 3 | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// If we are scanning inside a bucket reduce depth by 1.
 | 
					
						
							|  |  |  | 	if cache.Info.Name != dataUsageRoot { | 
					
						
							|  |  |  | 		flattenLevels-- | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							|  |  |  | 	var logPrefix, logSuffix string | 
					
						
							|  |  |  | 	if s.dataUsageCrawlDebug { | 
					
						
							|  |  |  | 		logPrefix = color.Green("data-usage: ") | 
					
						
							|  |  |  | 		logSuffix = color.Blue(" - %v + %v", basePath, cache.Info.Name) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	if s.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		logger.Info(logPrefix+"Cycle: %v"+logSuffix, cache.Info.NextCycle) | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 	// Always scan flattenLevels deep. Cache root is level 0.
 | 
					
						
							|  |  |  | 	todo := []cachedFolder{{name: cache.Info.Name}} | 
					
						
							|  |  |  | 	for i := 0; i < flattenLevels; i++ { | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 		if s.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			logger.Info(logPrefix+"Level %v, scanning %v directories."+logSuffix, i, len(todo)) | 
					
						
							| 
									
										
										
										
											2020-02-07 11:35:55 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		select { | 
					
						
							|  |  |  | 		case <-done: | 
					
						
							|  |  |  | 			return cache, ctx.Err() | 
					
						
							|  |  |  | 		default: | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		var err error | 
					
						
							|  |  |  | 		todo, err = s.scanQueuedLevels(ctx, todo, i == flattenLevels-1) | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 		if err != nil { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 			// No useful information...
 | 
					
						
							|  |  |  | 			return cache, err | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 		} | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	if s.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		logger.Info(logPrefix+"New folders: %v"+logSuffix, s.newFolders) | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	// Add new folders first
 | 
					
						
							|  |  |  | 	for _, folder := range s.newFolders { | 
					
						
							|  |  |  | 		select { | 
					
						
							|  |  |  | 		case <-done: | 
					
						
							|  |  |  | 			return s.newCache, ctx.Err() | 
					
						
							|  |  |  | 		default: | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		du, err := s.deepScanFolder(ctx, folder.name) | 
					
						
							|  |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		if du == nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, errors.New("data-usage: no disk usage provided")) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		s.newCache.replace(folder.name, "", *du) | 
					
						
							|  |  |  | 		// Add to parent manually
 | 
					
						
							|  |  |  | 		if folder.parent != nil { | 
					
						
							|  |  |  | 			parent := s.newCache.Cache[*folder.parent] | 
					
						
							|  |  |  | 			parent.addChildString(folder.name) | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 	} | 
					
						
							|  |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	if s.dataUsageCrawlDebug { | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 		logger.Info(logPrefix+"Existing folders: %v"+logSuffix, len(s.existingFolders)) | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	// Do selective scanning of existing folders.
 | 
					
						
							|  |  |  | 	for _, folder := range s.existingFolders { | 
					
						
							|  |  |  | 		select { | 
					
						
							|  |  |  | 		case <-done: | 
					
						
							|  |  |  | 			return s.newCache, ctx.Err() | 
					
						
							|  |  |  | 		default: | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		h := hashPath(folder.name) | 
					
						
							|  |  |  | 		if !h.mod(s.oldCache.Info.NextCycle, dataUsageUpdateDirCycles) { | 
					
						
							|  |  |  | 			s.newCache.replaceHashed(h, folder.parent, s.oldCache.Cache[h]) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 
 | 
					
						
							|  |  |  | 		// Update on this cycle...
 | 
					
						
							|  |  |  | 		du, err := s.deepScanFolder(ctx, folder.name) | 
					
						
							|  |  |  | 		if err != nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, err) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		if du == nil { | 
					
						
							|  |  |  | 			logger.LogIf(ctx, errors.New("data-usage: no disk usage provided")) | 
					
						
							|  |  |  | 			continue | 
					
						
							|  |  |  | 		} | 
					
						
							|  |  |  | 		s.newCache.replaceHashed(h, folder.parent, *du) | 
					
						
							|  |  |  | 	} | 
					
						
							| 
									
										
										
										
											2020-02-04 08:45:59 +08:00
										 |  |  | 
 | 
					
						
							| 
									
										
										
										
											2020-03-20 00:47:47 +08:00
										 |  |  | 	s.newCache.Info.LastUpdate = UTCNow() | 
					
						
							| 
									
										
										
										
											2020-03-19 07:19:29 +08:00
										 |  |  | 	s.newCache.Info.NextCycle++ | 
					
						
							|  |  |  | 	return s.newCache, nil | 
					
						
							| 
									
										
										
										
											2020-01-22 06:07:49 +08:00
										 |  |  | } |