mirror of
				https://github.com/minio/minio.git
				synced 2025-10-29 15:55:00 -04:00 
			
		
		
		
	Use errgroups instead of sync.WaitGroup as needed (#8354)
This commit is contained in:
		
							parent
							
								
									c33bae057f
								
							
						
					
					
						commit
						68a519a468
					
				| @ -71,8 +71,8 @@ func initFederatorBackend(objLayer ObjectLayer) { | ||||
| 	// Add buckets that are not registered with the DNS | ||||
| 	g := errgroup.WithNErrs(len(b)) | ||||
| 	for index := range b { | ||||
| 		index := index | ||||
| 		bucketSet.Add(b[index].Name) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			r, gerr := globalDNSConfig.Get(b[index].Name) | ||||
| 			if gerr != nil { | ||||
| @ -99,7 +99,6 @@ func initFederatorBackend(objLayer ObjectLayer) { | ||||
| 	// Remove buckets that are in DNS for this server, but aren't local | ||||
| 	for index := range dnsBuckets { | ||||
| 		index := index | ||||
| 
 | ||||
| 		g.Go(func() error { | ||||
| 			// This is a local bucket that exists, so we can continue | ||||
| 			if bucketSet.Contains(dnsBuckets[index].Key) { | ||||
|  | ||||
| @ -16,6 +16,7 @@ import ( | ||||
| 	"github.com/minio/minio/cmd/config/cache" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/color" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| 	"github.com/minio/minio/pkg/wildcard" | ||||
| ) | ||||
| 
 | ||||
| @ -450,36 +451,32 @@ func checkAtimeSupport(dir string) (err error) { | ||||
| func (c *cacheObjects) migrateCacheFromV1toV2(ctx context.Context) { | ||||
| 	logStartupMessage(color.Blue("Cache migration initiated ....")) | ||||
| 
 | ||||
| 	var wg sync.WaitGroup | ||||
| 	errs := make([]error, len(c.cache)) | ||||
| 	for i, dc := range c.cache { | ||||
| 	g := errgroup.WithNErrs(len(c.cache)) | ||||
| 	for index, dc := range c.cache { | ||||
| 		if dc == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// start migration from V1 to V2 | ||||
| 		go func(ctx context.Context, dc *diskCache, errs []error, idx int) { | ||||
| 			defer wg.Done() | ||||
| 			if err := migrateOldCache(ctx, dc); err != nil { | ||||
| 				errs[idx] = err | ||||
| 				logger.LogIf(ctx, err) | ||||
| 				return | ||||
| 			} | ||||
| 			// start purge routine after migration completes. | ||||
| 			go dc.purge() | ||||
| 		}(ctx, dc, errs, i) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// start migration from V1 to V2 | ||||
| 			return migrateOldCache(ctx, c.cache[index]) | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	errCnt := 0 | ||||
| 	for _, err := range errs { | ||||
| 	for index, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			errCnt++ | ||||
| 			logger.LogIf(ctx, err) | ||||
| 			continue | ||||
| 		} | ||||
| 		go c.cache[index].purge() | ||||
| 	} | ||||
| 
 | ||||
| 	if errCnt > 0 { | ||||
| 		return | ||||
| 	} | ||||
| 
 | ||||
| 	// update migration status | ||||
| 	c.migMutex.Lock() | ||||
| 	defer c.migMutex.Unlock() | ||||
|  | ||||
							
								
								
									
										119
									
								
								cmd/format-xl.go
									
									
									
									
									
								
							
							
						
						
									
										119
									
								
								cmd/format-xl.go
									
									
									
									
									
								
							| @ -23,12 +23,12 @@ import ( | ||||
| 	"fmt" | ||||
| 	"io/ioutil" | ||||
| 	"reflect" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"encoding/hex" | ||||
| 
 | ||||
| 	humanize "github.com/dustin/go-humanize" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| 	sha256 "github.com/minio/sha256-simd" | ||||
| ) | ||||
| 
 | ||||
| @ -315,40 +315,30 @@ func quorumUnformattedDisks(errs []error) bool { | ||||
| 
 | ||||
| // loadFormatXLAll - load all format config from all input disks in parallel. | ||||
| func loadFormatXLAll(storageDisks []StorageAPI) ([]*formatXLV3, []error) { | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	// Initialize list of errors. | ||||
| 	var sErrs = make([]error, len(storageDisks)) | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	// Initialize format configs. | ||||
| 	var formats = make([]*formatXLV3, len(storageDisks)) | ||||
| 
 | ||||
| 	// Load format from each disk in parallel | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			sErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Launch go-routine per disk. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 
 | ||||
| 			format, lErr := loadFormatXL(disk) | ||||
| 			if lErr != nil { | ||||
| 				sErrs[index] = lErr | ||||
| 				return | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if storageDisks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			format, err := loadFormatXL(storageDisks[index]) | ||||
| 			if err != nil { | ||||
| 				return err | ||||
| 			} | ||||
| 			formats[index] = format | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all go-routines to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	// Return all formats and nil | ||||
| 	return formats, sErrs | ||||
| 	// Return all formats and errors if any. | ||||
| 	return formats, g.Wait() | ||||
| } | ||||
| 
 | ||||
| func saveFormatXL(disk StorageAPI, format interface{}) error { | ||||
| @ -643,28 +633,22 @@ func formatXLV3Check(reference *formatXLV3, format *formatXLV3) error { | ||||
| 
 | ||||
| // saveFormatXLAll - populates `format.json` on disks in its order. | ||||
| func saveFormatXLAll(ctx context.Context, storageDisks []StorageAPI, formats []*formatXLV3) error { | ||||
| 	var errs = make([]error, len(storageDisks)) | ||||
| 
 | ||||
| 	var wg sync.WaitGroup | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	// Write `format.json` to all disks. | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if formats[index] == nil || disk == nil { | ||||
| 			errs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI, format *formatXLV3) { | ||||
| 			defer wg.Done() | ||||
| 			errs[index] = saveFormatXL(disk, format) | ||||
| 		}(index, disk, formats[index]) | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if formats[index] == nil || storageDisks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			return saveFormatXL(storageDisks[index], formats[index]) | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for the routines to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	writeQuorum := len(storageDisks)/2 + 1 | ||||
| 	return reduceWriteQuorumErrs(ctx, errs, nil, writeQuorum) | ||||
| 	// Wait for the routines to finish. | ||||
| 	return reduceWriteQuorumErrs(ctx, g.Wait(), nil, writeQuorum) | ||||
| } | ||||
| 
 | ||||
| // relinquishes the underlying connection for all storage disks. | ||||
| @ -682,17 +666,19 @@ func closeStorageDisks(storageDisks []StorageAPI) { | ||||
| func initStorageDisksWithErrors(endpoints EndpointList) ([]StorageAPI, []error) { | ||||
| 	// Bootstrap disks. | ||||
| 	storageDisks := make([]StorageAPI, len(endpoints)) | ||||
| 	errs := make([]error, len(endpoints)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for index, endpoint := range endpoints { | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, endpoint Endpoint) { | ||||
| 			defer wg.Done() | ||||
| 			storageDisks[index], errs[index] = newStorageAPI(endpoint) | ||||
| 		}(index, endpoint) | ||||
| 	g := errgroup.WithNErrs(len(endpoints)) | ||||
| 	for index := range endpoints { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			storageDisk, err := newStorageAPI(endpoints[index]) | ||||
| 			if err != nil { | ||||
| 				return err | ||||
| 			} | ||||
| 			storageDisks[index] = storageDisk | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return storageDisks, errs | ||||
| 	return storageDisks, g.Wait() | ||||
| } | ||||
| 
 | ||||
| // formatXLV3ThisEmpty - find out if '.This' field is empty | ||||
| @ -793,31 +779,24 @@ func initFormatXLMetaVolume(storageDisks []StorageAPI, formats []*formatXLV3) er | ||||
| 	// This happens for the first time, but keep this here since this | ||||
| 	// is the only place where it can be made expensive optimizing all | ||||
| 	// other calls. Create minio meta volume, if it doesn't exist yet. | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	// Initialize errs to collect errors inside go-routine. | ||||
| 	var errs = make([]error, len(storageDisks)) | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	// Initialize all disks in parallel. | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if formats[index] == nil || disk == nil { | ||||
| 			// Ignore create meta volume on disks which are not found. | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			// Indicate this wait group is done. | ||||
| 			defer wg.Done() | ||||
| 
 | ||||
| 			errs[index] = makeFormatXLMetaVolumes(disk) | ||||
| 		}(index, disk) | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if formats[index] == nil || storageDisks[index] == nil { | ||||
| 				// Ignore create meta volume on disks which are not found. | ||||
| 				return nil | ||||
| 			} | ||||
| 			return makeFormatXLMetaVolumes(storageDisks[index]) | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all cleanup to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	// Return upon first error. | ||||
| 	for _, err := range errs { | ||||
| 	for _, err := range g.Wait() { | ||||
| 		if err == nil { | ||||
| 			continue | ||||
| 		} | ||||
|  | ||||
| @ -38,6 +38,7 @@ import ( | ||||
| 	"github.com/minio/minio/pkg/madmin" | ||||
| 	xnet "github.com/minio/minio/pkg/net" | ||||
| 	"github.com/minio/minio/pkg/policy" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // NotificationSys - notification system. | ||||
| @ -72,24 +73,6 @@ type NotificationPeerErr struct { | ||||
| 	Err  error     // Error returned by the remote peer for an rpc call | ||||
| } | ||||
| 
 | ||||
| // DeleteBucket - calls DeleteBucket RPC call on all peers. | ||||
| func (sys *NotificationSys) DeleteBucket(ctx context.Context, bucketName string) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.DeleteBucket(bucketName); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // A NotificationGroup is a collection of goroutines working on subtasks that are part of | ||||
| // the same overall task. | ||||
| // | ||||
| @ -438,43 +421,44 @@ func (sys *NotificationSys) SignalService(sig serviceSignal) []NotificationPeerE | ||||
| // ServerInfo - calls ServerInfo RPC call on all peers. | ||||
| func (sys *NotificationSys) ServerInfo(ctx context.Context) []ServerInfo { | ||||
| 	serverInfo := make([]ServerInfo, len(sys.peerClients)) | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(sys.peerClients)) | ||||
| 	for index, client := range sys.peerClients { | ||||
| 		if client == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(idx int, client *peerRESTClient) { | ||||
| 			defer wg.Done() | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// Try to fetch serverInfo remotely in three attempts. | ||||
| 			for i := 0; i < 3; i++ { | ||||
| 				info, err := client.ServerInfo() | ||||
| 				if err == nil { | ||||
| 					serverInfo[idx] = ServerInfo{ | ||||
| 						Addr: client.host.String(), | ||||
| 						Data: &info, | ||||
| 					} | ||||
| 					return | ||||
| 				serverInfo[index] = ServerInfo{ | ||||
| 					Addr: sys.peerClients[index].host.String(), | ||||
| 				} | ||||
| 				serverInfo[idx] = ServerInfo{ | ||||
| 					Addr:  client.host.String(), | ||||
| 					Data:  &info, | ||||
| 					Error: err.Error(), | ||||
| 				info, err := sys.peerClients[index].ServerInfo() | ||||
| 				if err != nil { | ||||
| 					serverInfo[index].Error = err.Error() | ||||
| 				} | ||||
| 				serverInfo[index].Data = &info | ||||
| 				// Last iteration log the error. | ||||
| 				if i == 2 { | ||||
| 					reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", client.host.String()) | ||||
| 					ctx := logger.SetReqInfo(ctx, reqInfo) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 					return err | ||||
| 				} | ||||
| 				// Wait for one second and no need wait after last attempt. | ||||
| 				if i < 2 { | ||||
| 					time.Sleep(1 * time.Second) | ||||
| 				} | ||||
| 			} | ||||
| 		}(index, client) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	for index, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			addr := sys.peerClients[index].host.String() | ||||
| 			reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", addr) | ||||
| 			ctx := logger.SetReqInfo(ctx, reqInfo) | ||||
| 			logger.LogIf(ctx, err) | ||||
| 		} | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return serverInfo | ||||
| } | ||||
| 
 | ||||
| @ -482,166 +466,163 @@ func (sys *NotificationSys) ServerInfo(ctx context.Context) []ServerInfo { | ||||
| func (sys *NotificationSys) GetLocks(ctx context.Context) []*PeerLocks { | ||||
| 
 | ||||
| 	locksResp := make([]*PeerLocks, len(sys.peerClients)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	g := errgroup.WithNErrs(len(sys.peerClients)) | ||||
| 	for index, client := range sys.peerClients { | ||||
| 		if client == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(idx int, client *peerRESTClient) { | ||||
| 			defer wg.Done() | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// Try to fetch serverInfo remotely in three attempts. | ||||
| 			for i := 0; i < 3; i++ { | ||||
| 				serverLocksResp, err := client.GetLocks() | ||||
| 				serverLocksResp, err := sys.peerClients[index].GetLocks() | ||||
| 				if err == nil { | ||||
| 					locksResp[idx] = &PeerLocks{ | ||||
| 						Addr:  client.host.String(), | ||||
| 					locksResp[index] = &PeerLocks{ | ||||
| 						Addr:  sys.peerClients[index].host.String(), | ||||
| 						Locks: serverLocksResp, | ||||
| 					} | ||||
| 					return | ||||
| 					return nil | ||||
| 				} | ||||
| 
 | ||||
| 				// Last iteration log the error. | ||||
| 				if i == 2 { | ||||
| 					reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", client.host.String()) | ||||
| 					ctx := logger.SetReqInfo(ctx, reqInfo) | ||||
| 					logger.LogOnceIf(ctx, err, client.host.String()) | ||||
| 					return err | ||||
| 				} | ||||
| 				// Wait for one second and no need wait after last attempt. | ||||
| 				if i < 2 { | ||||
| 					time.Sleep(1 * time.Second) | ||||
| 				} | ||||
| 			} | ||||
| 		}(index, client) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	for index, err := range g.Wait() { | ||||
| 		reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", | ||||
| 			sys.peerClients[index].host.String()) | ||||
| 		ctx := logger.SetReqInfo(ctx, reqInfo) | ||||
| 		logger.LogOnceIf(ctx, err, sys.peerClients[index].host.String()) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return locksResp | ||||
| } | ||||
| 
 | ||||
| // SetBucketPolicy - calls SetBucketPolicy RPC call on all peers. | ||||
| func (sys *NotificationSys) SetBucketPolicy(ctx context.Context, bucketName string, bucketPolicy *policy.Policy) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.SetBucketPolicy(bucketName, bucketPolicy); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.SetBucketPolicy(bucketName, bucketPolicy) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // DeleteBucket - calls DeleteBucket RPC call on all peers. | ||||
| func (sys *NotificationSys) DeleteBucket(ctx context.Context, bucketName string) { | ||||
| 	go func() { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.DeleteBucket(bucketName) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // RemoveBucketPolicy - calls RemoveBucketPolicy RPC call on all peers. | ||||
| func (sys *NotificationSys) RemoveBucketPolicy(ctx context.Context, bucketName string) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.RemoveBucketPolicy(bucketName); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.RemoveBucketPolicy(bucketName) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // SetBucketLifecycle - calls SetBucketLifecycle on all peers. | ||||
| func (sys *NotificationSys) SetBucketLifecycle(ctx context.Context, bucketName string, bucketLifecycle *lifecycle.Lifecycle) { | ||||
| func (sys *NotificationSys) SetBucketLifecycle(ctx context.Context, bucketName string, | ||||
| 	bucketLifecycle *lifecycle.Lifecycle) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.SetBucketLifecycle(bucketName, bucketLifecycle); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.SetBucketLifecycle(bucketName, bucketLifecycle) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // RemoveBucketLifecycle - calls RemoveLifecycle on all peers. | ||||
| func (sys *NotificationSys) RemoveBucketLifecycle(ctx context.Context, bucketName string) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.RemoveBucketLifecycle(bucketName); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.RemoveBucketLifecycle(bucketName) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // PutBucketNotification - calls PutBucketNotification RPC call on all peers. | ||||
| func (sys *NotificationSys) PutBucketNotification(ctx context.Context, bucketName string, rulesMap event.RulesMap) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient, rulesMap event.RulesMap) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.PutBucketNotification(bucketName, rulesMap); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client, rulesMap.Clone()) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.PutBucketNotification(bucketName, rulesMap) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // ListenBucketNotification - calls ListenBucketNotification RPC call on all peers. | ||||
| func (sys *NotificationSys) ListenBucketNotification(ctx context.Context, bucketName string, eventNames []event.Name, pattern string, | ||||
| 	targetID event.TargetID, localPeer xnet.Host) { | ||||
| func (sys *NotificationSys) ListenBucketNotification(ctx context.Context, bucketName string, | ||||
| 	eventNames []event.Name, pattern string, targetID event.TargetID, localPeer xnet.Host) { | ||||
| 	go func() { | ||||
| 		var wg sync.WaitGroup | ||||
| 		for _, client := range sys.peerClients { | ||||
| 		ng := WithNPeers(len(sys.peerClients)) | ||||
| 		for idx, client := range sys.peerClients { | ||||
| 			if client == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(client *peerRESTClient) { | ||||
| 				defer wg.Done() | ||||
| 				if err := client.ListenBucketNotification(bucketName, eventNames, pattern, targetID, localPeer); err != nil { | ||||
| 					logger.GetReqInfo(ctx).AppendTags("remotePeer", client.host.Name) | ||||
| 					logger.LogIf(ctx, err) | ||||
| 				} | ||||
| 			}(client) | ||||
| 			client := client | ||||
| 			ng.Go(ctx, func() error { | ||||
| 				return client.ListenBucketNotification(bucketName, eventNames, pattern, targetID, localPeer) | ||||
| 			}, idx, *client.host) | ||||
| 		} | ||||
| 		wg.Wait() | ||||
| 		ng.Wait() | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| @ -981,78 +962,90 @@ func (sys *NotificationSys) CollectNetPerfInfo(size int64) map[string][]ServerNe | ||||
| // DrivePerfInfo - Drive speed (read and write) information | ||||
| func (sys *NotificationSys) DrivePerfInfo(size int64) []madmin.ServerDrivesPerfInfo { | ||||
| 	reply := make([]madmin.ServerDrivesPerfInfo, len(sys.peerClients)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i, client := range sys.peerClients { | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(sys.peerClients)) | ||||
| 	for index, client := range sys.peerClients { | ||||
| 		if client == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(client *peerRESTClient, idx int) { | ||||
| 			defer wg.Done() | ||||
| 			di, err := client.DrivePerfInfo(size) | ||||
| 			if err != nil { | ||||
| 				reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", client.host.String()) | ||||
| 				ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 				logger.LogIf(ctx, err) | ||||
| 				di.Addr = client.host.String() | ||||
| 				di.Error = err.Error() | ||||
| 			} | ||||
| 			reply[idx] = di | ||||
| 		}(client, i) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			var err error | ||||
| 			reply[index], err = sys.peerClients[index].DrivePerfInfo(size) | ||||
| 			return err | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	for index, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			addr := sys.peerClients[index].host.String() | ||||
| 			reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr) | ||||
| 			ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 			logger.LogIf(ctx, err) | ||||
| 			reply[index].Addr = addr | ||||
| 			reply[index].Error = err.Error() | ||||
| 		} | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return reply | ||||
| } | ||||
| 
 | ||||
| // MemUsageInfo - Mem utilization information | ||||
| func (sys *NotificationSys) MemUsageInfo() []ServerMemUsageInfo { | ||||
| 	reply := make([]ServerMemUsageInfo, len(sys.peerClients)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i, client := range sys.peerClients { | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(sys.peerClients)) | ||||
| 	for index, client := range sys.peerClients { | ||||
| 		if client == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(client *peerRESTClient, idx int) { | ||||
| 			defer wg.Done() | ||||
| 			memi, err := client.MemUsageInfo() | ||||
| 			if err != nil { | ||||
| 				reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", client.host.String()) | ||||
| 				ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 				logger.LogIf(ctx, err) | ||||
| 				memi.Addr = client.host.String() | ||||
| 				memi.Error = err.Error() | ||||
| 			} | ||||
| 			reply[idx] = memi | ||||
| 		}(client, i) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			var err error | ||||
| 			reply[index], err = sys.peerClients[index].MemUsageInfo() | ||||
| 			return err | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	for index, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			addr := sys.peerClients[index].host.String() | ||||
| 			reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr) | ||||
| 			ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 			logger.LogIf(ctx, err) | ||||
| 			reply[index].Addr = addr | ||||
| 			reply[index].Error = err.Error() | ||||
| 		} | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return reply | ||||
| } | ||||
| 
 | ||||
| // CPULoadInfo - CPU utilization information | ||||
| func (sys *NotificationSys) CPULoadInfo() []ServerCPULoadInfo { | ||||
| 	reply := make([]ServerCPULoadInfo, len(sys.peerClients)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i, client := range sys.peerClients { | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(sys.peerClients)) | ||||
| 	for index, client := range sys.peerClients { | ||||
| 		if client == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(client *peerRESTClient, idx int) { | ||||
| 			defer wg.Done() | ||||
| 			cpui, err := client.CPULoadInfo() | ||||
| 			if err != nil { | ||||
| 				reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", client.host.String()) | ||||
| 				ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 				logger.LogIf(ctx, err) | ||||
| 				cpui.Addr = client.host.String() | ||||
| 				cpui.Error = err.Error() | ||||
| 			} | ||||
| 			reply[idx] = cpui | ||||
| 		}(client, i) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			var err error | ||||
| 			reply[index], err = sys.peerClients[index].CPULoadInfo() | ||||
| 			return err | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	for index, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			addr := sys.peerClients[index].host.String() | ||||
| 			reqInfo := (&logger.ReqInfo{}).AppendTags("remotePeer", addr) | ||||
| 			ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 			logger.LogIf(ctx, err) | ||||
| 			reply[index].Addr = addr | ||||
| 			reply[index].Error = err.Error() | ||||
| 		} | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return reply | ||||
| } | ||||
| 
 | ||||
|  | ||||
| @ -129,8 +129,8 @@ func setupTestReadDirGeneric(t *testing.T) (testResults []result) { | ||||
| 
 | ||||
| // Test to read non-empty directory with symlinks. | ||||
| func setupTestReadDirSymlink(t *testing.T) (testResults []result) { | ||||
| 	if runtime.GOOS != "Windows" { | ||||
| 		t.Log("symlinks not available on windows") | ||||
| 	if runtime.GOOS == globalWindowsOSName { | ||||
| 		t.Skip("symlinks not available on windows") | ||||
| 		return nil | ||||
| 	} | ||||
| 	dir := mustSetupDir(t) | ||||
|  | ||||
| @ -306,19 +306,21 @@ func newXLSets(endpoints EndpointList, format *formatXLV3, setCount int, drivesP | ||||
| // StorageInfo - combines output of StorageInfo across all erasure coded object sets. | ||||
| func (s *xlSets) StorageInfo(ctx context.Context) StorageInfo { | ||||
| 	var storageInfo StorageInfo | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	storageInfos := make([]StorageInfo, len(s.sets)) | ||||
| 	storageInfo.Backend.Type = BackendErasure | ||||
| 	for index, set := range s.sets { | ||||
| 		wg.Add(1) | ||||
| 		go func(id int, set *xlObjects) { | ||||
| 			defer wg.Done() | ||||
| 			storageInfos[id] = set.StorageInfo(ctx) | ||||
| 		}(index, set) | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(s.sets)) | ||||
| 	for index := range s.sets { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			storageInfos[index] = s.sets[index].StorageInfo(ctx) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for the go routines. | ||||
| 	wg.Wait() | ||||
| 	g.Wait() | ||||
| 
 | ||||
| 	for _, lstorageInfo := range storageInfos { | ||||
| 		storageInfo.Used += lstorageInfo.Used | ||||
| @ -458,11 +460,12 @@ func undoMakeBucketSets(bucket string, sets []*xlObjects, errs []error) { | ||||
| 	// Undo previous make bucket entry on all underlying sets. | ||||
| 	for index := range sets { | ||||
| 		index := index | ||||
| 		if errs[index] == nil { | ||||
| 			g.Go(func() error { | ||||
| 		g.Go(func() error { | ||||
| 			if errs[index] == nil { | ||||
| 				return sets[index].DeleteBucket(context.Background(), bucket) | ||||
| 			}, index) | ||||
| 		} | ||||
| 			} | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all delete bucket to finish. | ||||
| @ -618,11 +621,12 @@ func undoDeleteBucketSets(bucket string, sets []*xlObjects, errs []error) { | ||||
| 	// Undo previous delete bucket on all underlying sets. | ||||
| 	for index := range sets { | ||||
| 		index := index | ||||
| 		if errs[index] == nil { | ||||
| 			g.Go(func() error { | ||||
| 		g.Go(func() error { | ||||
| 			if errs[index] == nil { | ||||
| 				return sets[index].MakeBucketWithLocation(context.Background(), bucket, "") | ||||
| 			}, index) | ||||
| 		} | ||||
| 			} | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	g.Wait() | ||||
| @ -742,19 +746,24 @@ func (s *xlSets) CopyObject(ctx context.Context, srcBucket, srcObject, destBucke | ||||
| func listDirSetsFactory(ctx context.Context, sets ...*xlObjects) ListDirFunc { | ||||
| 	listDirInternal := func(bucket, prefixDir, prefixEntry string, disks []StorageAPI) (mergedEntries []string) { | ||||
| 		var diskEntries = make([][]string, len(disks)) | ||||
| 		var wg sync.WaitGroup | ||||
| 		g := errgroup.WithNErrs(len(disks)) | ||||
| 		for index, disk := range disks { | ||||
| 			if disk == nil { | ||||
| 				continue | ||||
| 			} | ||||
| 			wg.Add(1) | ||||
| 			go func(index int, disk StorageAPI) { | ||||
| 				defer wg.Done() | ||||
| 				diskEntries[index], _ = disk.ListDir(bucket, prefixDir, -1, xlMetaJSONFile) | ||||
| 			}(index, disk) | ||||
| 			index := index | ||||
| 			g.Go(func() error { | ||||
| 				var err error | ||||
| 				diskEntries[index], err = disks[index].ListDir(bucket, prefixDir, -1, xlMetaJSONFile) | ||||
| 				return err | ||||
| 			}, index) | ||||
| 		} | ||||
| 
 | ||||
| 		wg.Wait() | ||||
| 		for _, err := range g.Wait() { | ||||
| 			if err != nil { | ||||
| 				logger.LogIf(ctx, err) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		// Find elements in entries which are not in mergedEntries | ||||
| 		for _, entries := range diskEntries { | ||||
| @ -1405,21 +1414,21 @@ func isTestSetup(infos []DiskInfo, errs []error) bool { | ||||
| 
 | ||||
| func getAllDiskInfos(storageDisks []StorageAPI) ([]DiskInfo, []error) { | ||||
| 	infos := make([]DiskInfo, len(storageDisks)) | ||||
| 	errs := make([]error, len(storageDisks)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i := range storageDisks { | ||||
| 		if storageDisks[i] == nil { | ||||
| 			errs[i] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(i int) { | ||||
| 			defer wg.Done() | ||||
| 			infos[i], errs[i] = storageDisks[i].DiskInfo() | ||||
| 		}(i) | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			var err error | ||||
| 			if storageDisks[index] != nil { | ||||
| 				infos[index], err = storageDisks[index].DiskInfo() | ||||
| 			} else { | ||||
| 				// Disk not found. | ||||
| 				err = errDiskNotFound | ||||
| 			} | ||||
| 			return err | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	return infos, errs | ||||
| 	return infos, g.Wait() | ||||
| } | ||||
| 
 | ||||
| // Mark root disks as down so as not to heal them. | ||||
|  | ||||
| @ -19,12 +19,12 @@ package cmd | ||||
| import ( | ||||
| 	"context" | ||||
| 	"sort" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"github.com/minio/minio-go/v6/pkg/s3utils" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/lifecycle" | ||||
| 	"github.com/minio/minio/pkg/policy" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // list all errors that can be ignore in a bucket operation. | ||||
| @ -42,83 +42,71 @@ func (xl xlObjects) MakeBucketWithLocation(ctx context.Context, bucket, location | ||||
| 		return BucketNameInvalid{Bucket: bucket} | ||||
| 	} | ||||
| 
 | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 	storageDisks := xl.getDisks() | ||||
| 
 | ||||
| 	// Initialize list of errors. | ||||
| 	var dErrs = make([]error, len(xl.getDisks())) | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	// Make a volume entry on all underlying storage disks. | ||||
| 	for index, disk := range xl.getDisks() { | ||||
| 		if disk == nil { | ||||
| 			dErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Make a volume inside a go-routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			err := disk.MakeVol(bucket) | ||||
| 			if err != nil { | ||||
| 				if err != errVolumeExists { | ||||
| 					logger.LogIf(ctx, err) | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if storageDisks[index] != nil { | ||||
| 				if err := storageDisks[index].MakeVol(bucket); err != nil { | ||||
| 					if err != errVolumeExists { | ||||
| 						logger.LogIf(ctx, err) | ||||
| 					} | ||||
| 					return err | ||||
| 				} | ||||
| 				dErrs[index] = err | ||||
| 				return nil | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return errDiskNotFound | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all make vol to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	writeQuorum := len(xl.getDisks())/2 + 1 | ||||
| 	err := reduceWriteQuorumErrs(ctx, dErrs, bucketOpIgnoredErrs, writeQuorum) | ||||
| 	writeQuorum := len(storageDisks)/2 + 1 | ||||
| 	err := reduceWriteQuorumErrs(ctx, g.Wait(), bucketOpIgnoredErrs, writeQuorum) | ||||
| 	if err == errXLWriteQuorum { | ||||
| 		// Purge successfully created buckets if we don't have writeQuorum. | ||||
| 		undoMakeBucket(xl.getDisks(), bucket) | ||||
| 		undoMakeBucket(storageDisks, bucket) | ||||
| 	} | ||||
| 	return toObjectErr(err, bucket) | ||||
| } | ||||
| 
 | ||||
| func (xl xlObjects) undoDeleteBucket(bucket string) { | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| func undoDeleteBucket(storageDisks []StorageAPI, bucket string) { | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 	// Undo previous make bucket entry on all underlying storage disks. | ||||
| 	for index, disk := range xl.getDisks() { | ||||
| 		if disk == nil { | ||||
| 	for index := range storageDisks { | ||||
| 		if storageDisks[index] == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Delete a bucket inside a go-routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			_ = disk.MakeVol(bucket) | ||||
| 		}(index, disk) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			_ = storageDisks[index].MakeVol(bucket) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all make vol to finish. | ||||
| 	wg.Wait() | ||||
| 	g.Wait() | ||||
| } | ||||
| 
 | ||||
| // undo make bucket operation upon quorum failure. | ||||
| func undoMakeBucket(storageDisks []StorageAPI, bucket string) { | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 	// Undo previous make bucket entry on all underlying storage disks. | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 	for index := range storageDisks { | ||||
| 		if storageDisks[index] == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Delete a bucket inside a go-routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			_ = disk.DeleteVol(bucket) | ||||
| 		}(index, disk) | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			_ = storageDisks[index].DeleteVol(bucket) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all make vol to finish. | ||||
| 	wg.Wait() | ||||
| 	g.Wait() | ||||
| } | ||||
| 
 | ||||
| // getBucketInfo - returns the BucketInfo from one of the load balanced disks. | ||||
| @ -245,42 +233,34 @@ func (xl xlObjects) DeleteBucket(ctx context.Context, bucket string) error { | ||||
| 	defer bucketLock.Unlock() | ||||
| 
 | ||||
| 	// Collect if all disks report volume not found. | ||||
| 	var wg sync.WaitGroup | ||||
| 	var dErrs = make([]error, len(xl.getDisks())) | ||||
| 
 | ||||
| 	// Remove a volume entry on all underlying storage disks. | ||||
| 	storageDisks := xl.getDisks() | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			dErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Delete volume inside a go-routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			// Attempt to delete bucket. | ||||
| 			err := disk.DeleteVol(bucket) | ||||
| 			if err != nil { | ||||
| 				dErrs[index] = err | ||||
| 				return | ||||
| 			} | ||||
| 
 | ||||
| 			// Cleanup all the previously incomplete multiparts. | ||||
| 			err = cleanupDir(ctx, disk, minioMetaMultipartBucket, bucket) | ||||
| 			if err != nil && err != errVolumeNotFound { | ||||
| 				dErrs[index] = err | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if storageDisks[index] != nil { | ||||
| 				if err := storageDisks[index].DeleteVol(bucket); err != nil { | ||||
| 					return err | ||||
| 				} | ||||
| 				err := cleanupDir(ctx, storageDisks[index], minioMetaMultipartBucket, bucket) | ||||
| 				if err != nil && err != errVolumeNotFound { | ||||
| 					return err | ||||
| 				} | ||||
| 				return nil | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return errDiskNotFound | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all the delete vols to finish. | ||||
| 	wg.Wait() | ||||
| 	dErrs := g.Wait() | ||||
| 
 | ||||
| 	writeQuorum := len(xl.getDisks())/2 + 1 | ||||
| 	writeQuorum := len(storageDisks)/2 + 1 | ||||
| 	err := reduceWriteQuorumErrs(ctx, dErrs, bucketOpIgnoredErrs, writeQuorum) | ||||
| 	if err == errXLWriteQuorum { | ||||
| 		xl.undoDeleteBucket(bucket) | ||||
| 		undoDeleteBucket(storageDisks, bucket) | ||||
| 	} | ||||
| 	if err != nil { | ||||
| 		return toObjectErr(err, bucket) | ||||
|  | ||||
| @ -19,7 +19,8 @@ package cmd | ||||
| import ( | ||||
| 	"context" | ||||
| 	"path" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // getLoadBalancedDisks - fetches load balanced (sufficiently randomized) disk slice. | ||||
| @ -53,35 +54,33 @@ func (xl xlObjects) parentDirIsObject(ctx context.Context, bucket, parent string | ||||
| // isObject - returns `true` if the prefix is an object i.e if | ||||
| // `xl.json` exists at the leaf, false otherwise. | ||||
| func (xl xlObjects) isObject(bucket, prefix string) (ok bool) { | ||||
| 	var errs = make([]error, len(xl.getDisks())) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for index, disk := range xl.getDisks() { | ||||
| 	storageDisks := xl.getDisks() | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// Check if 'prefix' is an object on this 'disk', else continue the check the next disk | ||||
| 			fi, err := disk.StatFile(bucket, path.Join(prefix, xlMetaJSONFile)) | ||||
| 			fi, err := storageDisks[index].StatFile(bucket, pathJoin(prefix, xlMetaJSONFile)) | ||||
| 			if err != nil { | ||||
| 				errs[index] = err | ||||
| 				return | ||||
| 				return err | ||||
| 			} | ||||
| 			if fi.Size == 0 { | ||||
| 				errs[index] = errCorruptedFormat | ||||
| 				return | ||||
| 				return errCorruptedFormat | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	// NOTE: Observe we are not trying to read `xl.json` and figure out the actual | ||||
| 	// quorum intentionally, but rely on the default case scenario. Actual quorum | ||||
| 	// verification will happen by top layer by using getObjectInfo() and will be | ||||
| 	// ignored if necessary. | ||||
| 	readQuorum := len(xl.getDisks()) / 2 | ||||
| 	readQuorum := len(storageDisks) / 2 | ||||
| 
 | ||||
| 	return reduceReadQuorumErrs(context.Background(), errs, objectOpIgnoredErrs, readQuorum) == nil | ||||
| 	return reduceReadQuorumErrs(context.Background(), g.Wait(), objectOpIgnoredErrs, readQuorum) == nil | ||||
| } | ||||
|  | ||||
| @ -20,11 +20,11 @@ import ( | ||||
| 	"context" | ||||
| 	"fmt" | ||||
| 	"io" | ||||
| 	"sync" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/madmin" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| func (xl xlObjects) ReloadFormat(ctx context.Context, dryRun bool) error { | ||||
| @ -57,40 +57,31 @@ func healBucket(ctx context.Context, storageDisks []StorageAPI, bucket string, w | ||||
| 	dryRun bool) (res madmin.HealResultItem, err error) { | ||||
| 
 | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	// Initialize list of errors. | ||||
| 	var dErrs = make([]error, len(storageDisks)) | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	// Disk states slices | ||||
| 	beforeState := make([]string, len(storageDisks)) | ||||
| 	afterState := make([]string, len(storageDisks)) | ||||
| 
 | ||||
| 	// Make a volume entry on all underlying storage disks. | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			dErrs[index] = errDiskNotFound | ||||
| 			beforeState[index] = madmin.DriveStateOffline | ||||
| 			afterState[index] = madmin.DriveStateOffline | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 
 | ||||
| 		// Make a volume inside a go-routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			if _, serr := disk.StatVol(bucket); serr != nil { | ||||
| 	for index := range storageDisks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if storageDisks[index] == nil { | ||||
| 				beforeState[index] = madmin.DriveStateOffline | ||||
| 				afterState[index] = madmin.DriveStateOffline | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			if _, serr := storageDisks[index].StatVol(bucket); serr != nil { | ||||
| 				if serr == errDiskNotFound { | ||||
| 					beforeState[index] = madmin.DriveStateOffline | ||||
| 					afterState[index] = madmin.DriveStateOffline | ||||
| 					dErrs[index] = serr | ||||
| 					return | ||||
| 					return serr | ||||
| 				} | ||||
| 				if serr != errVolumeNotFound { | ||||
| 					beforeState[index] = madmin.DriveStateCorrupt | ||||
| 					afterState[index] = madmin.DriveStateCorrupt | ||||
| 					dErrs[index] = serr | ||||
| 					return | ||||
| 					return serr | ||||
| 				} | ||||
| 
 | ||||
| 				beforeState[index] = madmin.DriveStateMissing | ||||
| @ -98,23 +89,22 @@ func healBucket(ctx context.Context, storageDisks []StorageAPI, bucket string, w | ||||
| 
 | ||||
| 				// mutate only if not a dry-run | ||||
| 				if dryRun { | ||||
| 					return | ||||
| 					return nil | ||||
| 				} | ||||
| 
 | ||||
| 				makeErr := disk.MakeVol(bucket) | ||||
| 				dErrs[index] = makeErr | ||||
| 				makeErr := storageDisks[index].MakeVol(bucket) | ||||
| 				if makeErr == nil { | ||||
| 					afterState[index] = madmin.DriveStateOk | ||||
| 				} | ||||
| 				return | ||||
| 				return makeErr | ||||
| 			} | ||||
| 			beforeState[index] = madmin.DriveStateOk | ||||
| 			afterState[index] = madmin.DriveStateOk | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all make vol to finish. | ||||
| 	wg.Wait() | ||||
| 	errs := g.Wait() | ||||
| 
 | ||||
| 	// Initialize heal result info | ||||
| 	res = madmin.HealResultItem{ | ||||
| @ -122,13 +112,13 @@ func healBucket(ctx context.Context, storageDisks []StorageAPI, bucket string, w | ||||
| 		Bucket:    bucket, | ||||
| 		DiskCount: len(storageDisks), | ||||
| 	} | ||||
| 	for i, before := range beforeState { | ||||
| 	for i := range beforeState { | ||||
| 		if storageDisks[i] != nil { | ||||
| 			drive := storageDisks[i].String() | ||||
| 			res.Before.Drives = append(res.Before.Drives, madmin.HealDriveInfo{ | ||||
| 				UUID:     "", | ||||
| 				Endpoint: drive, | ||||
| 				State:    before, | ||||
| 				State:    beforeState[i], | ||||
| 			}) | ||||
| 			res.After.Drives = append(res.After.Drives, madmin.HealDriveInfo{ | ||||
| 				UUID:     "", | ||||
| @ -138,7 +128,7 @@ func healBucket(ctx context.Context, storageDisks []StorageAPI, bucket string, w | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	reducedErr := reduceWriteQuorumErrs(ctx, dErrs, bucketOpIgnoredErrs, writeQuorum) | ||||
| 	reducedErr := reduceWriteQuorumErrs(ctx, errs, bucketOpIgnoredErrs, writeQuorum) | ||||
| 	if reducedErr == errXLWriteQuorum { | ||||
| 		// Purge successfully created buckets if we don't have writeQuorum. | ||||
| 		undoMakeBucket(storageDisks, bucket) | ||||
| @ -597,29 +587,25 @@ func defaultHealResult(latestXLMeta xlMetaV1, storageDisks []StorageAPI, errs [] | ||||
| 
 | ||||
| // Stat all directories. | ||||
| func statAllDirs(ctx context.Context, storageDisks []StorageAPI, bucket, prefix string) []error { | ||||
| 	var errs = make([]error, len(storageDisks)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			entries, err := disk.ListDir(bucket, prefix, 1, "") | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			entries, err := storageDisks[index].ListDir(bucket, prefix, 1, "") | ||||
| 			if err != nil { | ||||
| 				errs[index] = err | ||||
| 				return | ||||
| 				return err | ||||
| 			} | ||||
| 			if len(entries) > 0 { | ||||
| 				errs[index] = errVolumeNotEmpty | ||||
| 				return | ||||
| 				return errVolumeNotEmpty | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 	return errs | ||||
| 	return g.Wait() | ||||
| } | ||||
| 
 | ||||
| // ObjectDir is considered dangling/corrupted if any only | ||||
|  | ||||
| @ -24,11 +24,11 @@ import ( | ||||
| 	"net/http" | ||||
| 	"path" | ||||
| 	"sort" | ||||
| 	"sync" | ||||
| 	"time" | ||||
| 
 | ||||
| 	xhttp "github.com/minio/minio/cmd/http" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| 	"github.com/minio/sha256-simd" | ||||
| ) | ||||
| 
 | ||||
| @ -452,31 +452,23 @@ func renameXLMetadata(ctx context.Context, disks []StorageAPI, srcBucket, srcEnt | ||||
| 
 | ||||
| // writeUniqueXLMetadata - writes unique `xl.json` content for each disk in order. | ||||
| func writeUniqueXLMetadata(ctx context.Context, disks []StorageAPI, bucket, prefix string, xlMetas []xlMetaV1, quorum int) ([]StorageAPI, error) { | ||||
| 	var wg sync.WaitGroup | ||||
| 	var mErrs = make([]error, len(disks)) | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 
 | ||||
| 	// Start writing `xl.json` to all disks in parallel. | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			mErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 
 | ||||
| 		// Pick one xlMeta for a disk at index. | ||||
| 		xlMetas[index].Erasure.Index = index + 1 | ||||
| 
 | ||||
| 		// Write `xl.json` in a routine. | ||||
| 		go func(index int, disk StorageAPI, xlMeta xlMetaV1) { | ||||
| 			defer wg.Done() | ||||
| 
 | ||||
| 			// Write unique `xl.json` for a disk at index. | ||||
| 			mErrs[index] = writeXLMetadata(ctx, disk, bucket, prefix, xlMeta) | ||||
| 		}(index, disk, xlMetas[index]) | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if disks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			// Pick one xlMeta for a disk at index. | ||||
| 			xlMetas[index].Erasure.Index = index + 1 | ||||
| 			return writeXLMetadata(ctx, disks[index], bucket, prefix, xlMetas[index]) | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all the routines. | ||||
| 	wg.Wait() | ||||
| 	mErrs := g.Wait() | ||||
| 
 | ||||
| 	err := reduceWriteQuorumErrs(ctx, mErrs, objectOpIgnoredErrs, quorum) | ||||
| 	return evalDisks(disks, mErrs), err | ||||
|  | ||||
| @ -24,12 +24,12 @@ import ( | ||||
| 	"sort" | ||||
| 	"strconv" | ||||
| 	"strings" | ||||
| 	"sync" | ||||
| 	"time" | ||||
| 
 | ||||
| 	xhttp "github.com/minio/minio/cmd/http" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/mimedb" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| func (xl xlObjects) getUploadIDDir(bucket, object, uploadID string) string { | ||||
| @ -57,21 +57,23 @@ func (xl xlObjects) checkUploadIDExists(ctx context.Context, bucket, object, upl | ||||
| // Removes part given by partName belonging to a mulitpart upload from minioMetaBucket | ||||
| func (xl xlObjects) removeObjectPart(bucket, object, uploadID, partName string) { | ||||
| 	curpartPath := path.Join(bucket, object, uploadID, partName) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i, disk := range xl.getDisks() { | ||||
| 	storageDisks := xl.getDisks() | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// Ignoring failure to remove parts that weren't present in CompleteMultipartUpload | ||||
| 			// requests. xl.json is the authoritative source of truth on which parts constitute | ||||
| 			// the object. The presence of parts that don't belong in the object doesn't affect correctness. | ||||
| 			_ = disk.DeleteFile(minioMetaMultipartBucket, curpartPath) | ||||
| 		}(i, disk) | ||||
| 			_ = storageDisks[index].DeleteFile(minioMetaMultipartBucket, curpartPath) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	g.Wait() | ||||
| } | ||||
| 
 | ||||
| // statPart - returns fileInfo structure for a successful stat on part file. | ||||
| @ -104,31 +106,29 @@ func (xl xlObjects) statPart(ctx context.Context, bucket, object, uploadID, part | ||||
| 
 | ||||
| // commitXLMetadata - commit `xl.json` from source prefix to destination prefix in the given slice of disks. | ||||
| func commitXLMetadata(ctx context.Context, disks []StorageAPI, srcBucket, srcPrefix, dstBucket, dstPrefix string, quorum int) ([]StorageAPI, error) { | ||||
| 	var wg sync.WaitGroup | ||||
| 	var mErrs = make([]error, len(disks)) | ||||
| 
 | ||||
| 	srcJSONFile := path.Join(srcPrefix, xlMetaJSONFile) | ||||
| 	dstJSONFile := path.Join(dstPrefix, xlMetaJSONFile) | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 
 | ||||
| 	// Rename `xl.json` to all disks in parallel. | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			mErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Rename `xl.json` in a routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if disks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 
 | ||||
| 			// Delete any dangling directories. | ||||
| 			defer disk.DeleteFile(srcBucket, srcPrefix) | ||||
| 			defer disks[index].DeleteFile(srcBucket, srcPrefix) | ||||
| 
 | ||||
| 			// Renames `xl.json` from source prefix to destination prefix. | ||||
| 			mErrs[index] = disk.RenameFile(srcBucket, srcJSONFile, dstBucket, dstJSONFile) | ||||
| 		}(index, disk) | ||||
| 			return disks[index].RenameFile(srcBucket, srcJSONFile, dstBucket, dstJSONFile) | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all the routines. | ||||
| 	wg.Wait() | ||||
| 	mErrs := g.Wait() | ||||
| 
 | ||||
| 	err := reduceWriteQuorumErrs(ctx, mErrs, objectOpIgnoredErrs, quorum) | ||||
| 	return evalDisks(disks, mErrs), err | ||||
|  | ||||
| @ -22,11 +22,11 @@ import ( | ||||
| 	"io" | ||||
| 	"net/http" | ||||
| 	"path" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	xhttp "github.com/minio/minio/cmd/http" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/mimedb" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // list all errors which can be ignored in object operations. | ||||
| @ -34,25 +34,26 @@ var objectOpIgnoredErrs = append(baseIgnoredErrs, errDiskAccessDenied) | ||||
| 
 | ||||
| // putObjectDir hints the bottom layer to create a new directory. | ||||
| func (xl xlObjects) putObjectDir(ctx context.Context, bucket, object string, writeQuorum int) error { | ||||
| 	var wg sync.WaitGroup | ||||
| 	storageDisks := xl.getDisks() | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	errs := make([]error, len(xl.getDisks())) | ||||
| 	// Prepare object creation in all disks | ||||
| 	for index, disk := range xl.getDisks() { | ||||
| 		if disk == nil { | ||||
| 	for index := range storageDisks { | ||||
| 		if storageDisks[index] == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			if err := disk.MakeVol(pathJoin(bucket, object)); err != nil && err != errVolumeExists { | ||||
| 				errs[index] = err | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			err := storageDisks[index].MakeVol(pathJoin(bucket, object)) | ||||
| 			if err != nil && err != errVolumeExists { | ||||
| 				return err | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	return reduceWriteQuorumErrs(ctx, errs, objectOpIgnoredErrs, writeQuorum) | ||||
| 	return reduceWriteQuorumErrs(ctx, g.Wait(), objectOpIgnoredErrs, writeQuorum) | ||||
| } | ||||
| 
 | ||||
| /// Object Operations | ||||
| @ -335,36 +336,34 @@ func (xl xlObjects) getObject(ctx context.Context, bucket, object string, startO | ||||
| } | ||||
| 
 | ||||
| // getObjectInfoDir - This getObjectInfo is specific to object directory lookup. | ||||
| func (xl xlObjects) getObjectInfoDir(ctx context.Context, bucket, object string) (oi ObjectInfo, err error) { | ||||
| 	var wg sync.WaitGroup | ||||
| func (xl xlObjects) getObjectInfoDir(ctx context.Context, bucket, object string) (ObjectInfo, error) { | ||||
| 	storageDisks := xl.getDisks() | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(storageDisks)) | ||||
| 
 | ||||
| 	errs := make([]error, len(xl.getDisks())) | ||||
| 	// Prepare object creation in a all disks | ||||
| 	for index, disk := range xl.getDisks() { | ||||
| 	for index, disk := range storageDisks { | ||||
| 		if disk == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			// Check if 'prefix' is an object on this 'disk'. | ||||
| 			entries, err := disk.ListDir(bucket, object, 1, "") | ||||
| 			entries, err := storageDisks[index].ListDir(bucket, object, 1, "") | ||||
| 			if err != nil { | ||||
| 				errs[index] = err | ||||
| 				return | ||||
| 				return err | ||||
| 			} | ||||
| 			if len(entries) > 0 { | ||||
| 				// Not a directory if not empty. | ||||
| 				errs[index] = errFileNotFound | ||||
| 				return | ||||
| 				return errFileNotFound | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	readQuorum := len(xl.getDisks()) / 2 | ||||
| 	return dirObjectInfo(bucket, object, 0, map[string]string{}), reduceReadQuorumErrs(ctx, errs, objectOpIgnoredErrs, readQuorum) | ||||
| 	readQuorum := len(storageDisks) / 2 | ||||
| 	err := reduceReadQuorumErrs(ctx, g.Wait(), objectOpIgnoredErrs, readQuorum) | ||||
| 	return dirObjectInfo(bucket, object, 0, map[string]string{}), err | ||||
| } | ||||
| 
 | ||||
| // GetObjectInfo - reads object metadata and replies back ObjectInfo. | ||||
| @ -424,7 +423,6 @@ func (xl xlObjects) getObjectInfo(ctx context.Context, bucket, object string) (o | ||||
| } | ||||
| 
 | ||||
| func undoRename(disks []StorageAPI, srcBucket, srcEntry, dstBucket, dstEntry string, isDir bool, errs []error) { | ||||
| 	var wg sync.WaitGroup | ||||
| 	// Undo rename object on disks where RenameFile succeeded. | ||||
| 
 | ||||
| 	// If srcEntry/dstEntry are objects then add a trailing slash to copy | ||||
| @ -433,56 +431,51 @@ func undoRename(disks []StorageAPI, srcBucket, srcEntry, dstBucket, dstEntry str | ||||
| 		srcEntry = retainSlash(srcEntry) | ||||
| 		dstEntry = retainSlash(dstEntry) | ||||
| 	} | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			continue | ||||
| 		} | ||||
| 		// Undo rename object in parallel. | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			if errs[index] != nil { | ||||
| 				return | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if errs[index] == nil { | ||||
| 				_ = disks[index].RenameFile(dstBucket, dstEntry, srcBucket, srcEntry) | ||||
| 			} | ||||
| 			_ = disk.RenameFile(dstBucket, dstEntry, srcBucket, srcEntry) | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| 	g.Wait() | ||||
| } | ||||
| 
 | ||||
| // rename - common function that renamePart and renameObject use to rename | ||||
| // the respective underlying storage layer representations. | ||||
| func rename(ctx context.Context, disks []StorageAPI, srcBucket, srcEntry, dstBucket, dstEntry string, isDir bool, writeQuorum int, ignoredErr []error) ([]StorageAPI, error) { | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	// Initialize list of errors. | ||||
| 	var errs = make([]error, len(disks)) | ||||
| 
 | ||||
| 	if isDir { | ||||
| 		dstEntry = retainSlash(dstEntry) | ||||
| 		srcEntry = retainSlash(srcEntry) | ||||
| 	} | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 
 | ||||
| 	// Rename file on all underlying storage disks. | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			errs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			if err := disk.RenameFile(srcBucket, srcEntry, dstBucket, dstEntry); err != nil { | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if disks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			if err := disks[index].RenameFile(srcBucket, srcEntry, dstBucket, dstEntry); err != nil { | ||||
| 				if !IsErrIgnored(err, ignoredErr...) { | ||||
| 					errs[index] = err | ||||
| 					return err | ||||
| 				} | ||||
| 			} | ||||
| 		}(index, disk) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all renames to finish. | ||||
| 	wg.Wait() | ||||
| 	errs := g.Wait() | ||||
| 
 | ||||
| 	// We can safely allow RenameFile errors up to len(xl.getDisks()) - writeQuorum | ||||
| 	// otherwise return failure. Cleanup successful renames. | ||||
| @ -744,39 +737,31 @@ func (xl xlObjects) deleteObject(ctx context.Context, bucket, object string, wri | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	// Initialize sync waitgroup. | ||||
| 	var wg sync.WaitGroup | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 
 | ||||
| 	// Initialize list of errors. | ||||
| 	var dErrs = make([]error, len(disks)) | ||||
| 
 | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			dErrs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(index int, disk StorageAPI, isDir bool) { | ||||
| 			defer wg.Done() | ||||
| 			var e error | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if disks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			var err error | ||||
| 			if isDir { | ||||
| 				// DeleteFile() simply tries to remove a directory | ||||
| 				// and will succeed only if that directory is empty. | ||||
| 				e = disk.DeleteFile(minioMetaTmpBucket, tmpObj) | ||||
| 				err = disks[index].DeleteFile(minioMetaTmpBucket, tmpObj) | ||||
| 			} else { | ||||
| 				e = cleanupDir(ctx, disk, minioMetaTmpBucket, tmpObj) | ||||
| 				err = cleanupDir(ctx, disks[index], minioMetaTmpBucket, tmpObj) | ||||
| 			} | ||||
| 			if e != nil && e != errVolumeNotFound { | ||||
| 				dErrs[index] = e | ||||
| 			if err != nil && err != errVolumeNotFound { | ||||
| 				return err | ||||
| 			} | ||||
| 		}(index, disk, isDir) | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all routines to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	// return errors if any during deletion | ||||
| 	return reduceWriteQuorumErrs(ctx, dErrs, objectOpIgnoredErrs, writeQuorum) | ||||
| 	return reduceWriteQuorumErrs(ctx, g.Wait(), objectOpIgnoredErrs, writeQuorum) | ||||
| } | ||||
| 
 | ||||
| // deleteObject - wrapper for delete object, deletes an object from | ||||
|  | ||||
| @ -21,10 +21,10 @@ import ( | ||||
| 	"errors" | ||||
| 	"hash/crc32" | ||||
| 	"path" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	jsoniter "github.com/json-iterator/go" | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // Returns number of errors that occurred the most (incl. nil) and the | ||||
| @ -180,28 +180,23 @@ func readXLMeta(ctx context.Context, disk StorageAPI, bucket string, object stri | ||||
| // Reads all `xl.json` metadata as a xlMetaV1 slice. | ||||
| // Returns error slice indicating the failed metadata reads. | ||||
| func readAllXLMetadata(ctx context.Context, disks []StorageAPI, bucket, object string) ([]xlMetaV1, []error) { | ||||
| 	errs := make([]error, len(disks)) | ||||
| 	metadataArray := make([]xlMetaV1, len(disks)) | ||||
| 	var wg sync.WaitGroup | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 	// Read `xl.json` parallelly across disks. | ||||
| 	for index, disk := range disks { | ||||
| 		if disk == nil { | ||||
| 			errs[index] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		// Read `xl.json` in routine. | ||||
| 		go func(index int, disk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			metadataArray[index], errs[index] = readXLMeta(ctx, disk, bucket, object) | ||||
| 		}(index, disk) | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() (err error) { | ||||
| 			if disks[index] == nil { | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			metadataArray[index], err = readXLMeta(ctx, disks[index], bucket, object) | ||||
| 			return err | ||||
| 		}, index) | ||||
| 	} | ||||
| 
 | ||||
| 	// Wait for all the routines to finish. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	// Return all the metadata. | ||||
| 	return metadataArray, errs | ||||
| 	return metadataArray, g.Wait() | ||||
| } | ||||
| 
 | ||||
| // Return shuffled partsMetadata depending on distribution. | ||||
|  | ||||
							
								
								
									
										43
									
								
								cmd/xl-v1.go
									
									
									
									
									
								
							
							
						
						
									
										43
									
								
								cmd/xl-v1.go
									
									
									
									
									
								
							| @ -19,10 +19,10 @@ package cmd | ||||
| import ( | ||||
| 	"context" | ||||
| 	"sort" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"github.com/minio/minio/cmd/logger" | ||||
| 	"github.com/minio/minio/pkg/bpool" | ||||
| 	"github.com/minio/minio/pkg/sync/errgroup" | ||||
| ) | ||||
| 
 | ||||
| // XL constants. | ||||
| @ -71,34 +71,31 @@ func (d byDiskTotal) Less(i, j int) bool { | ||||
| // getDisksInfo - fetch disks info across all other storage API. | ||||
| func getDisksInfo(disks []StorageAPI) (disksInfo []DiskInfo, onlineDisks int, offlineDisks int) { | ||||
| 	disksInfo = make([]DiskInfo, len(disks)) | ||||
| 	errs := make([]error, len(disks)) | ||||
| 	var wg sync.WaitGroup | ||||
| 	for i, storageDisk := range disks { | ||||
| 		if storageDisk == nil { | ||||
| 			// Storage disk is empty, perhaps ignored disk or not available. | ||||
| 			errs[i] = errDiskNotFound | ||||
| 			continue | ||||
| 		} | ||||
| 		wg.Add(1) | ||||
| 		go func(id int, sDisk StorageAPI) { | ||||
| 			defer wg.Done() | ||||
| 			info, err := sDisk.DiskInfo() | ||||
| 
 | ||||
| 	g := errgroup.WithNErrs(len(disks)) | ||||
| 	for index := range disks { | ||||
| 		index := index | ||||
| 		g.Go(func() error { | ||||
| 			if disks[index] == nil { | ||||
| 				// Storage disk is empty, perhaps ignored disk or not available. | ||||
| 				return errDiskNotFound | ||||
| 			} | ||||
| 			info, err := disks[index].DiskInfo() | ||||
| 			if err != nil { | ||||
| 				reqInfo := (&logger.ReqInfo{}).AppendTags("disk", sDisk.String()) | ||||
| 				if IsErr(err, baseErrs...) { | ||||
| 					return err | ||||
| 				} | ||||
| 				reqInfo := (&logger.ReqInfo{}).AppendTags("disk", disks[index].String()) | ||||
| 				ctx := logger.SetReqInfo(context.Background(), reqInfo) | ||||
| 				logger.LogIf(ctx, err) | ||||
| 				if IsErr(err, baseErrs...) { | ||||
| 					errs[id] = err | ||||
| 					return | ||||
| 				} | ||||
| 			} | ||||
| 			disksInfo[id] = info | ||||
| 		}(i, storageDisk) | ||||
| 			disksInfo[index] = info | ||||
| 			return nil | ||||
| 		}, index) | ||||
| 	} | ||||
| 	// Wait for the routines. | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	for _, err := range errs { | ||||
| 	// Wait for the routines. | ||||
| 	for _, err := range g.Wait() { | ||||
| 		if err != nil { | ||||
| 			offlineDisks++ | ||||
| 			continue | ||||
|  | ||||
		Loading…
	
	
			
			x
			
			
		
	
		Reference in New Issue
	
	Block a user