github.com/NVIDIA/aistore@v1.3.23-0.20240517131212-7df6609be51d/ext/dsort/conc_adjuster.go (about) 1 // Package dsort provides distributed massively parallel resharding for very large datasets. 2 /* 3 * Copyright (c) 2018-2023, NVIDIA CORPORATION. All rights reserved. 4 * 5 */ 6 package dsort 7 8 import ( 9 "sync" 10 "time" 11 12 "github.com/NVIDIA/aistore/cmn" 13 "github.com/NVIDIA/aistore/cmn/cos" 14 "github.com/NVIDIA/aistore/fs" 15 ) 16 17 // concAdjuster is responsible for finding optimal number of goroutine which 18 // maximizes the utilization - it is assumed that such optimal number exists and 19 // can be achieved. For each mountpath there is new 'optimizer' created called 20 // mapthAdjuster. 21 // 22 // Once in a while concAdjuster asks for current disks utilization. It saves 23 // them for future use. When it has enough information to make decision it will 24 // average previously collected disk utilization and based on new utilization 25 // it will adjust the concurrency limit. Number of information which are needed 26 // to make decision increases every time. This mechanism prevents from doing 27 // a lot of adjustments in short time but rather tries to find the best value 28 // at the very beginning (doing a lot of small steps) and then just keeps 29 // oscillating around the desired watermark. 30 // 31 // There is mechanism which resets number of information required to make 32 // decision. This mechanism is required in case utilization decreases 33 // significantly (resetRatio). 34 35 const ( 36 // Default value of initial concurrency limit for functions - defined per mountpath. 37 // This value is used when user provided `0` as concurrency limit. 38 defaultConcFuncLimit = 10 39 // Determines maximum concurrency limit per disk. 40 // 41 // Setting this too high can result in big number of goroutines in the system 42 // what can put tremendous pressure on scheduler and other goroutines in the 43 // system - eg. http handlers which are spawned in new goroutines. 44 maxConcFuncPerDiskLimit = 15 45 // Determines maximum concurrency limit per dsort per node. 46 maxConcFuncPerDsortLimit = 200 47 // TODO: add global max concurrency limit - should be preserved between 48 // all dSorts on single node. 49 50 // Batch corresponds to number of received update information per mountpath. 51 // Every time the size of the batch is reached, recalculation of limit is 52 // performed. 53 defaultBatchSize = 10 54 // Defines how much interval size will increase every time it is changed. 55 batchIncRatio = 1.2 56 // Defines maximum batch size (to prevent unlimited growth). 57 maxBatchSize = 200 58 59 lastInfoCnt = 30 // maximum number of latest adjust information 60 ) 61 62 type ( 63 mpathAdjuster struct { 64 // Determines how often much information must be processed until we 65 // adjust the concurrency limits. 66 curBatchSize float64 // float64 to not lose precision 67 // Current number updates, as it reaches the curBatchSize 68 // limit update is performed. 69 tickCnt int 70 limit int 71 maxLimit int 72 lastUtils []int64 73 // Semaphore for function calls. On update it is swapped with a new one. 74 funcCallsSema *cos.DynSemaphore 75 } 76 77 concAdjuster struct { 78 adjusters map[string]*mpathAdjuster 79 gorountinesSema *cos.DynSemaphore 80 defaultLimit int // default limit for new mpath adjusters 81 goroutineLimitCoef int // num goroutines should be allowed per one function call. 82 mu sync.RWMutex 83 stopCh cos.StopCh 84 } 85 ) 86 87 func calcMaxLimit() int { 88 availablePaths := fs.GetAvail() 89 maxLimitPerDisk := min( 90 maxConcFuncPerDiskLimit, 91 maxConcFuncPerDsortLimit/max(len(availablePaths), 1), 92 ) 93 return maxLimitPerDisk 94 } 95 96 func newMpathAdjuster(limit, maxLimit int) *mpathAdjuster { 97 return &mpathAdjuster{ 98 curBatchSize: defaultBatchSize, 99 limit: limit, 100 maxLimit: maxLimit, 101 lastUtils: make([]int64, 0, lastInfoCnt), 102 funcCallsSema: cos.NewDynSemaphore(limit), 103 } 104 } 105 106 func newConcAdjuster(maxLimit, goroutineLimitCoef int) (ca *concAdjuster) { 107 availablePaths := fs.GetAvail() 108 adjusters := make(map[string]*mpathAdjuster, len(availablePaths)) 109 if maxLimit == 0 { 110 maxLimit = calcMaxLimit() 111 } 112 limit := min(defaultConcFuncLimit, maxLimit) 113 for _, mi := range availablePaths { 114 adjusters[mi.Path] = newMpathAdjuster(limit, maxLimit) 115 } 116 ca = &concAdjuster{ 117 defaultLimit: limit, 118 adjusters: adjusters, 119 goroutineLimitCoef: goroutineLimitCoef, 120 gorountinesSema: cos.NewDynSemaphore(goroutineLimitCoef * len(availablePaths) * limit), 121 } 122 ca.stopCh.Init() 123 return 124 } 125 126 func (ca *concAdjuster) start() { 127 go ca.run() 128 } 129 130 func (ca *concAdjuster) run() { 131 config := cmn.GCO.Get() 132 ticker := time.NewTicker(config.Disk.IostatTimeShort.D()) 133 defer ticker.Stop() 134 for { 135 select { 136 case <-ticker.C: 137 ca.mu.RLock() 138 for mpath, adjuster := range ca.adjusters { 139 util := fs.GetMpathUtil(mpath) 140 141 adjuster.lastUtils = append(adjuster.lastUtils, util) 142 if len(adjuster.lastUtils) > lastInfoCnt { 143 adjuster.lastUtils = adjuster.lastUtils[1:] 144 } 145 adjuster.tickCnt++ 146 147 if float64(adjuster.tickCnt) >= adjuster.curBatchSize { 148 var totalUtil int64 149 for _, util := range adjuster.lastUtils { 150 totalUtil += util 151 } 152 newUtil := totalUtil / int64(len(adjuster.lastUtils)) 153 154 prevLimit, newLimit := adjuster.recalc(newUtil, config) 155 if prevLimit != newLimit { 156 adjuster.funcCallsSema.SetSize(newLimit) 157 diff := ca.goroutineLimitCoef * (newLimit - prevLimit) 158 ca.gorountinesSema.SetSize(ca.gorountinesSema.Size() + diff) 159 } 160 161 adjuster.tickCnt = 0 162 } 163 } 164 ca.mu.RUnlock() 165 case <-ca.stopCh.Listen(): 166 return 167 } 168 } 169 } 170 171 func (ca *concAdjuster) stop() { 172 ca.stopCh.Close() 173 } 174 175 func (ca *concAdjuster) acquireSema(mi *fs.Mountpath) { 176 ca.mu.Lock() 177 adjuster, ok := ca.adjusters[mi.Path] 178 if !ok { 179 maxLimit := calcMaxLimit() 180 adjuster = newMpathAdjuster(ca.defaultLimit, maxLimit) 181 ca.adjusters[mi.Path] = adjuster 182 183 // Also we need to update goroutine semaphore size 184 diff := ca.goroutineLimitCoef * ca.defaultLimit 185 ca.gorountinesSema.SetSize(ca.gorountinesSema.Size() + diff) 186 } 187 ca.mu.Unlock() 188 adjuster.funcCallsSema.Acquire() 189 } 190 191 func (ca *concAdjuster) releaseSema(mi *fs.Mountpath) { 192 ca.mu.RLock() 193 adjuster := ca.adjusters[mi.Path] 194 ca.mu.RUnlock() 195 adjuster.funcCallsSema.Release() 196 } 197 198 func (ca *concAdjuster) acquireGoroutineSema() { 199 ca.gorountinesSema.Acquire() 200 } 201 202 func (ca *concAdjuster) releaseGoroutineSema() { 203 ca.gorountinesSema.Release() 204 } 205 206 func (adjuster *mpathAdjuster) recalc(newUtil int64, config *cmn.Config) (prevLimit, newLimit int) { 207 prevLimit = adjuster.limit 208 switch { 209 case newUtil < config.Disk.DiskUtilLowWM: 210 adjuster.limit *= 2 211 adjuster.curBatchSize *= batchIncRatio 212 case newUtil <= config.Disk.DiskUtilHighWM: 213 adjuster.limit += 2 214 adjuster.curBatchSize *= batchIncRatio 215 case newUtil > config.Disk.DiskUtilMaxWM: 216 adjuster.limit -= 2 217 default: 218 adjuster.limit++ 219 adjuster.curBatchSize *= 2 * batchIncRatio 220 } 221 222 adjuster.curBatchSize = min(adjuster.curBatchSize, maxBatchSize) 223 if adjuster.limit < 1 { 224 adjuster.limit = 1 225 } else if adjuster.limit > adjuster.maxLimit { 226 adjuster.limit = adjuster.maxLimit 227 } 228 return prevLimit, adjuster.limit 229 }