github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/shard_insert_queue.go (about) 1 // Copyright (c) 2017 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package storage 22 23 import ( 24 "errors" 25 "strconv" 26 "sync" 27 "time" 28 29 "github.com/m3db/m3/src/dbnode/namespace" 30 "github.com/m3db/m3/src/dbnode/runtime" 31 "github.com/m3db/m3/src/dbnode/storage/series" 32 "github.com/m3db/m3/src/dbnode/ts" 33 "github.com/m3db/m3/src/x/checked" 34 "github.com/m3db/m3/src/x/clock" 35 "github.com/m3db/m3/src/x/ident" 36 xsync "github.com/m3db/m3/src/x/sync" 37 xtime "github.com/m3db/m3/src/x/time" 38 39 "github.com/uber-go/tally" 40 "go.uber.org/atomic" 41 "go.uber.org/zap" 42 ) 43 44 const ( 45 resetShardInsertsEvery = 3 * time.Minute 46 ) 47 48 var ( 49 errShardInsertQueueNotOpen = errors.New("shard insert queue is not open") 50 errShardInsertQueueAlreadyOpenOrClosed = errors.New("shard insert queue already open or is closed") 51 errNewSeriesInsertRateLimitExceeded = errors.New("shard insert of new series exceeds rate limit") 52 ) 53 54 type dbShardInsertQueueState int 55 56 const ( 57 dbShardInsertQueueStateNotOpen dbShardInsertQueueState = iota 58 dbShardInsertQueueStateOpen 59 dbShardInsertQueueStateClosed 60 ) 61 62 type dbShardInsertQueue struct { 63 sync.RWMutex 64 65 state dbShardInsertQueueState 66 nowFn clock.NowFn 67 insertEntryBatchFn dbShardInsertEntryBatchFn 68 sleepFn func(time.Duration) 69 coreFn xsync.CoreFn 70 71 // rate limits, protected by mutex 72 insertBatchBackoff time.Duration 73 insertPerSecondLimit *atomic.Uint64 74 75 insertPerSecondLimitWindowNanos *atomic.Uint64 76 insertPerSecondLimitWindowValues *atomic.Uint64 77 78 currBatch *dbShardInsertBatch 79 notifyInsert chan struct{} 80 closeCh chan struct{} 81 82 metrics dbShardInsertQueueMetrics 83 logger *zap.Logger 84 } 85 86 type dbShardInsertQueueMetrics struct { 87 insertsNoPendingWrite tally.Counter 88 insertsPendingWrite tally.Counter 89 insertsBatchErrors tally.Counter 90 } 91 92 func newDatabaseShardInsertQueueMetrics( 93 scope tally.Scope, 94 ) dbShardInsertQueueMetrics { 95 insertName := "inserts" 96 insertPendingWriteTagName := "pending-write" 97 return dbShardInsertQueueMetrics{ 98 insertsNoPendingWrite: scope.Tagged(map[string]string{ 99 insertPendingWriteTagName: "no", 100 }).Counter(insertName), 101 insertsPendingWrite: scope.Tagged(map[string]string{ 102 insertPendingWriteTagName: "yes", 103 }).Counter(insertName), 104 insertsBatchErrors: scope.Counter("inserts-batch.errors"), 105 } 106 } 107 108 type dbShardInsertEntryBatchFn func(inserts []dbShardInsert) error 109 110 // newDatabaseShardInsertQueue creates a new shard insert queue. The shard 111 // insert queue is used to batch inserts into the shard series map without 112 // sacrificing delays to insert the series. 113 // 114 // This is important as during floods of new IDs we want to avoid acquiring 115 // the lock to insert each individual series and insert as many as possible 116 // all together acquiring the lock once. 117 // 118 // It was experimented also sleeping for a very short duration, i.e. 1ms, 119 // during the insert loop and it actually added so much latency even just 120 // 1ms that it hurt it more than just acquiring the lock for each series. 121 // 122 // The batching as it is without any sleep and just relying on a notification 123 // trigger and hot looping when being flooded improved by a factor of roughly 124 // 4x during floods of new series. 125 func newDatabaseShardInsertQueue( 126 insertEntryBatchFn dbShardInsertEntryBatchFn, 127 nowFn clock.NowFn, 128 coreFn xsync.CoreFn, 129 scope tally.Scope, 130 logger *zap.Logger, 131 ) *dbShardInsertQueue { 132 scope = scope.SubScope("insert-queue") 133 currBatch := newDbShardInsertBatch(nowFn, scope) 134 return &dbShardInsertQueue{ 135 nowFn: nowFn, 136 insertEntryBatchFn: insertEntryBatchFn, 137 sleepFn: time.Sleep, 138 coreFn: coreFn, 139 currBatch: currBatch, 140 // NB(r): Use 2 * num cores so that each CPU insert queue which 141 // is 1 per num CPU core can always enqueue a notification without 142 // it being lost. 143 notifyInsert: make(chan struct{}, 2*xsync.NumCores()), 144 closeCh: make(chan struct{}, 1), 145 insertPerSecondLimit: atomic.NewUint64(0), 146 insertPerSecondLimitWindowNanos: atomic.NewUint64(0), 147 insertPerSecondLimitWindowValues: atomic.NewUint64(0), 148 metrics: newDatabaseShardInsertQueueMetrics(scope), 149 logger: logger, 150 } 151 } 152 153 func (q *dbShardInsertQueue) SetRuntimeOptions(value runtime.Options) { 154 q.Lock() 155 q.insertBatchBackoff = value.WriteNewSeriesBackoffDuration() 156 q.Unlock() 157 158 // Use atomics so no locks outside of per CPU core lock used. 159 v := uint64(value.WriteNewSeriesLimitPerShardPerSecond()) 160 q.insertPerSecondLimit.Store(v) 161 } 162 163 func (q *dbShardInsertQueue) insertLoop() { 164 defer func() { 165 close(q.closeCh) 166 }() 167 168 var ( 169 lastInsert time.Time 170 allInserts []dbShardInsert 171 allInsertsLastReset time.Time 172 ) 173 batch := newDbShardInsertBatch(q.nowFn, tally.NoopScope) 174 for range q.notifyInsert { 175 // Check if inserting too fast 176 elapsedSinceLastInsert := q.nowFn().Sub(lastInsert) 177 178 // Rotate batches 179 var ( 180 state dbShardInsertQueueState 181 backoff time.Duration 182 ) 183 q.Lock() 184 state = q.state 185 if elapsedSinceLastInsert < q.insertBatchBackoff { 186 // Need to backoff before rotate and insert 187 backoff = q.insertBatchBackoff - elapsedSinceLastInsert 188 } 189 q.Unlock() 190 191 if backoff > 0 { 192 q.sleepFn(backoff) 193 } 194 195 batchWg := q.currBatch.Rotate(batch) 196 197 // NB(r): Either reset (to avoid spikey allocations sticking around 198 // forever) or reuse existing slice. 199 now := q.nowFn() 200 if now.Sub(allInsertsLastReset) > resetShardInsertsEvery { 201 allInserts = nil 202 allInsertsLastReset = now 203 } else { 204 allInserts = allInserts[:0] 205 } 206 // Batch together for single insertion. 207 for _, batchByCPUCore := range batch.insertsByCPUCore { 208 batchByCPUCore.Lock() 209 allInserts = append(allInserts, batchByCPUCore.inserts...) 210 batchByCPUCore.Unlock() 211 } 212 213 err := q.insertEntryBatchFn(allInserts) 214 if err != nil { 215 q.metrics.insertsBatchErrors.Inc(1) 216 q.logger.Error("shard insert queue batch insert failed", 217 zap.Error(err)) 218 } 219 220 batchWg.Done() 221 222 // Memset optimization to clear inserts holding refs to objects. 223 var insertZeroValue dbShardInsert 224 for i := range allInserts { 225 allInserts[i] = insertZeroValue 226 } 227 228 lastInsert = q.nowFn() 229 230 if state != dbShardInsertQueueStateOpen { 231 return // Break if the queue closed 232 } 233 } 234 } 235 236 func (q *dbShardInsertQueue) Start() error { 237 q.Lock() 238 defer q.Unlock() 239 240 if q.state != dbShardInsertQueueStateNotOpen { 241 return errShardInsertQueueAlreadyOpenOrClosed 242 } 243 244 q.state = dbShardInsertQueueStateOpen 245 go q.insertLoop() 246 return nil 247 } 248 249 func (q *dbShardInsertQueue) Stop() error { 250 q.Lock() 251 252 if q.state != dbShardInsertQueueStateOpen { 253 q.Unlock() 254 return errShardInsertQueueNotOpen 255 } 256 257 q.state = dbShardInsertQueueStateClosed 258 q.Unlock() 259 260 // Final flush. 261 select { 262 case q.notifyInsert <- struct{}{}: 263 default: 264 // Loop busy, already ready to consume notification. 265 } 266 267 // wait till other go routine is done 268 <-q.closeCh 269 270 return nil 271 } 272 273 func (q *dbShardInsertQueue) Insert(insert dbShardInsert) (*sync.WaitGroup, error) { 274 if !insert.opts.skipRateLimit { 275 if limit := q.insertPerSecondLimit.Load(); limit > 0 { 276 windowNanos := uint64(q.nowFn().Truncate(time.Second).UnixNano()) 277 currLimitWindowNanos := q.insertPerSecondLimitWindowNanos.Load() 278 if currLimitWindowNanos != windowNanos { 279 // Rolled into a new window. 280 if q.insertPerSecondLimitWindowNanos.CAS(currLimitWindowNanos, windowNanos) { 281 // If managed to set it to the new window, reset the counter 282 // otherwise another goroutine got to it first and 283 // will zero the counter. 284 q.insertPerSecondLimitWindowValues.Store(0) 285 } 286 } 287 if q.insertPerSecondLimitWindowValues.Inc() > uint64(limit) { 288 return nil, errNewSeriesInsertRateLimitExceeded 289 } 290 } 291 } 292 293 inserts := q.currBatch.insertsByCPUCore[q.coreFn()] 294 inserts.Lock() 295 // Track if first insert, if so then we need to notify insert loop, 296 // otherwise we already have a pending notification. 297 firstInsert := len(inserts.inserts) == 0 298 inserts.inserts = append(inserts.inserts, insert) 299 wg := inserts.wg 300 inserts.Unlock() 301 302 // Notify insert loop, only required if first to insert for this 303 // this CPU core. 304 if firstInsert { 305 select { 306 case q.notifyInsert <- struct{}{}: 307 default: 308 // Loop busy, already ready to consume notification. 309 } 310 } 311 312 if insert.opts.hasPendingWrite { 313 q.metrics.insertsPendingWrite.Inc(1) 314 } else { 315 q.metrics.insertsNoPendingWrite.Inc(1) 316 } 317 318 return wg, nil 319 } 320 321 type dbShardInsertBatch struct { 322 nowFn clock.NowFn 323 wg *sync.WaitGroup 324 // Note: since inserts by CPU core is allocated when 325 // nsIndexInsertBatch is constructed and then never modified 326 // it is safe to concurently read (but not modify obviously). 327 insertsByCPUCore []*dbShardInsertsByCPUCore 328 lastReset time.Time 329 } 330 331 type dbShardInsertsByCPUCore struct { 332 sync.Mutex 333 334 wg *sync.WaitGroup 335 inserts []dbShardInsert 336 metrics dbShardInsertsByCPUCoreMetrics 337 } 338 339 type dbShardInsert struct { 340 entry *Entry 341 opts dbShardInsertAsyncOptions 342 } 343 344 type dbShardInsertAsyncOptions struct { 345 skipRateLimit bool 346 347 pendingWrite dbShardPendingWrite 348 pendingRetrievedBlock dbShardPendingRetrievedBlock 349 pendingIndex dbShardPendingIndex 350 351 hasPendingWrite bool 352 hasPendingRetrievedBlock bool 353 hasPendingIndexing bool 354 355 // NB(prateek): `releaseEntryRef` indicates if the 356 // entry provided along with the dbShardInsertAsyncOptions 357 // already has it's ref count incremented and it will be decremented after insert. 358 // It's used to correctly manage the lifecycle of the entry across the 359 // shard -> shard Queue -> shard boundaries. 360 releaseEntryRef bool 361 } 362 363 type dbShardPendingWrite struct { 364 timestamp xtime.UnixNano 365 value float64 366 unit xtime.Unit 367 annotation checked.Bytes 368 opts series.WriteOptions 369 } 370 371 type dbShardPendingIndex struct { 372 timestamp xtime.UnixNano 373 enqueuedAt time.Time 374 } 375 376 type dbShardPendingRetrievedBlock struct { 377 id ident.ID 378 tags ident.TagIterator 379 start xtime.UnixNano 380 segment ts.Segment 381 nsCtx namespace.Context 382 } 383 384 func newDbShardInsertBatch( 385 nowFn clock.NowFn, 386 scope tally.Scope, 387 ) *dbShardInsertBatch { 388 b := &dbShardInsertBatch{ 389 nowFn: nowFn, 390 wg: &sync.WaitGroup{}, 391 } 392 numCores := xsync.NumCores() 393 for i := 0; i < numCores; i++ { 394 b.insertsByCPUCore = append(b.insertsByCPUCore, &dbShardInsertsByCPUCore{ 395 wg: b.wg, 396 metrics: newDBShardInsertsByCPUCoreMetrics(i, scope), 397 }) 398 } 399 b.Rotate(nil) 400 return b 401 } 402 403 type dbShardInsertsByCPUCoreMetrics struct { 404 rotateInserts tally.Counter 405 } 406 407 func newDBShardInsertsByCPUCoreMetrics( 408 cpuIndex int, 409 scope tally.Scope, 410 ) dbShardInsertsByCPUCoreMetrics { 411 scope = scope.Tagged(map[string]string{ 412 "cpu-index": strconv.Itoa(cpuIndex), 413 }) 414 415 return dbShardInsertsByCPUCoreMetrics{ 416 rotateInserts: scope.Counter("rotate-inserts"), 417 } 418 } 419 420 func (b *dbShardInsertBatch) Rotate(target *dbShardInsertBatch) *sync.WaitGroup { 421 prevWg := b.wg 422 423 // We always expect to be waiting for an index. 424 b.wg = &sync.WaitGroup{} 425 b.wg.Add(1) 426 427 reset := false 428 now := b.nowFn() 429 if now.Sub(b.lastReset) > resetShardInsertsEvery { 430 // NB(r): Sometimes this can grow very high, so we reset it 431 // relatively frequently. 432 reset = true 433 b.lastReset = now 434 } 435 436 // Rotate to target if we need to. 437 for idx, inserts := range b.insertsByCPUCore { 438 if target == nil { 439 // No target to rotate with. 440 inserts.Lock() 441 // Reset 442 inserts.inserts = inserts.inserts[:0] 443 // Use new wait group. 444 inserts.wg = b.wg 445 inserts.Unlock() 446 continue 447 } 448 449 // First prepare the target to take the current batch's inserts. 450 targetInserts := target.insertsByCPUCore[idx] 451 targetInserts.Lock() 452 453 // Reset the target inserts since we'll take ref to them in a second. 454 var prevTargetInserts []dbShardInsert 455 if !reset { 456 // Only reuse if not resetting the allocation. 457 // memset optimization. 458 var zeroDbShardInsert dbShardInsert 459 for i := range targetInserts.inserts { 460 targetInserts.inserts[i] = zeroDbShardInsert 461 } 462 prevTargetInserts = targetInserts.inserts[:0] 463 } 464 465 // Lock the current batch inserts now ready to rotate to the target. 466 inserts.Lock() 467 468 // Update current slice refs to take target's inserts. 469 targetInserts.inserts = inserts.inserts 470 targetInserts.wg = inserts.wg 471 472 // Reuse the target's old slices. 473 inserts.inserts = prevTargetInserts 474 475 // Use new wait group. 476 inserts.wg = b.wg 477 478 // Unlock as early as possible for writes to keep enqueuing. 479 inserts.Unlock() 480 481 numTargetInserts := len(targetInserts.inserts) 482 483 // Now can unlock target inserts too. 484 targetInserts.Unlock() 485 486 if n := numTargetInserts; n > 0 { 487 inserts.metrics.rotateInserts.Inc(int64(n)) 488 } 489 } 490 491 return prevWg 492 }