github.com/m3db/m3@v1.5.0/src/msg/producer/writer/message_writer.go (about) 1 // Copyright (c) 2018 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 writer 22 23 import ( 24 "container/list" 25 "errors" 26 "math" 27 "sync" 28 "time" 29 30 "github.com/uber-go/tally" 31 32 "github.com/m3db/m3/src/msg/producer" 33 "github.com/m3db/m3/src/msg/protocol/proto" 34 "github.com/m3db/m3/src/x/clock" 35 "github.com/m3db/m3/src/x/instrument" 36 "github.com/m3db/m3/src/x/retry" 37 "github.com/m3db/m3/src/x/unsafe" 38 ) 39 40 // MessageRetryNanosFn returns the message backoff time for retry in nanoseconds. 41 type MessageRetryNanosFn func(writeTimes int) int64 42 43 var ( 44 errInvalidBackoffDuration = errors.New("invalid backoff duration") 45 errFailAllConsumers = errors.New("could not write to any consumer") 46 errNoWriters = errors.New("no writers") 47 ) 48 49 const _recordMessageDelayEvery = 4 // keep it a power of two value to keep modulo fast 50 51 type messageWriter interface { 52 // Write writes a message, messages not acknowledged in time will be retried. 53 // New messages will be written in order, but retries could be out of order. 54 Write(rm *producer.RefCountedMessage) 55 56 // Ack acknowledges the metadata. 57 Ack(meta metadata) bool 58 59 // Init initialize the message writer. 60 Init() 61 62 // Close closes the writer. 63 // It should block until all buffered messages have been acknowledged. 64 Close() 65 66 // AddConsumerWriter adds a consumer writer. 67 AddConsumerWriter(cw consumerWriter) 68 69 // RemoveConsumerWriter removes the consumer writer for the given address. 70 RemoveConsumerWriter(addr string) 71 72 // Metrics returns the metrics 73 Metrics() messageWriterMetrics 74 75 // SetMetrics sets the metrics 76 // 77 // This allows changing the labels of the metrics when the downstream consumer instance changes. 78 SetMetrics(m messageWriterMetrics) 79 80 // ReplicatedShardID returns the replicated shard id. 81 ReplicatedShardID() uint64 82 83 // CutoverNanos returns the cutover nanoseconds. 84 CutoverNanos() int64 85 86 // SetCutoverNanos sets the cutover nanoseconds. 87 SetCutoverNanos(nanos int64) 88 89 // CutoffNanos returns the cutoff nanoseconds. 90 CutoffNanos() int64 91 92 // SetCutoffNanos sets the cutoff nanoseconds. 93 SetCutoffNanos(nanos int64) 94 95 // MessageTTLNanos returns the message ttl nanoseconds. 96 MessageTTLNanos() int64 97 98 // SetMessageTTLNanos sets the message ttl nanoseconds. 99 SetMessageTTLNanos(nanos int64) 100 101 // QueueSize returns the number of messages queued in the writer. 102 QueueSize() int 103 } 104 105 type messageWriterMetrics struct { 106 withoutConsumerScope bool 107 scope tally.Scope 108 opts instrument.TimerOptions 109 writeSuccess tally.Counter 110 oneConsumerWriteError tally.Counter 111 allConsumersWriteError tally.Counter 112 noWritersError tally.Counter 113 writeAfterCutoff tally.Counter 114 writeBeforeCutover tally.Counter 115 messageAcked tally.Counter 116 messageClosed tally.Counter 117 messageDroppedBufferFull tally.Counter 118 messageDroppedTTLExpire tally.Counter 119 messageRetry tally.Counter 120 messageConsumeLatency tally.Timer 121 messageWriteDelay tally.Timer 122 scanBatchLatency tally.Timer 123 scanTotalLatency tally.Timer 124 enqueuedMessages tally.Counter 125 dequeuedMessages tally.Counter 126 processedWrite tally.Counter 127 processedClosed tally.Counter 128 processedNotReady tally.Counter 129 processedTTL tally.Counter 130 processedAck tally.Counter 131 processedDrop tally.Counter 132 } 133 134 func (m messageWriterMetrics) withConsumer(consumer string) messageWriterMetrics { 135 if m.withoutConsumerScope { 136 return m 137 } 138 return newMessageWriterMetricsWithConsumer(m.scope, m.opts, consumer, false) 139 } 140 141 func newMessageWriterMetrics( 142 scope tally.Scope, 143 opts instrument.TimerOptions, 144 withoutConsumerScope bool, 145 ) messageWriterMetrics { 146 return newMessageWriterMetricsWithConsumer(scope, opts, "unknown", withoutConsumerScope) 147 } 148 149 func newMessageWriterMetricsWithConsumer( 150 scope tally.Scope, 151 opts instrument.TimerOptions, 152 consumer string, 153 withoutConsumerScope bool) messageWriterMetrics { 154 consumerScope := scope 155 if !withoutConsumerScope { 156 consumerScope = scope.Tagged(map[string]string{"consumer": consumer}) 157 } 158 return messageWriterMetrics{ 159 withoutConsumerScope: withoutConsumerScope, 160 scope: scope, 161 opts: opts, 162 writeSuccess: consumerScope.Counter("write-success"), 163 oneConsumerWriteError: scope.Counter("write-error-one-consumer"), 164 allConsumersWriteError: consumerScope. 165 Tagged(map[string]string{"error-type": "all-consumers"}). 166 Counter("write-error"), 167 noWritersError: consumerScope. 168 Tagged(map[string]string{"error-type": "no-writers"}). 169 Counter("write-error"), 170 writeAfterCutoff: consumerScope. 171 Tagged(map[string]string{"reason": "after-cutoff"}). 172 Counter("invalid-write"), 173 writeBeforeCutover: consumerScope. 174 Tagged(map[string]string{"reason": "before-cutover"}). 175 Counter("invalid-write"), 176 messageAcked: consumerScope.Counter("message-acked"), 177 messageClosed: consumerScope.Counter("message-closed"), 178 messageDroppedBufferFull: consumerScope.Tagged( 179 map[string]string{"reason": "buffer-full"}, 180 ).Counter("message-dropped"), 181 messageDroppedTTLExpire: consumerScope.Tagged( 182 map[string]string{"reason": "ttl-expire"}, 183 ).Counter("message-dropped"), 184 messageRetry: consumerScope.Counter("message-retry"), 185 messageConsumeLatency: instrument.NewTimer(consumerScope, "message-consume-latency", opts), 186 messageWriteDelay: instrument.NewTimer(consumerScope, "message-write-delay", opts), 187 scanBatchLatency: instrument.NewTimer(consumerScope, "scan-batch-latency", opts), 188 scanTotalLatency: instrument.NewTimer(consumerScope, "scan-total-latency", opts), 189 enqueuedMessages: consumerScope.Counter("message-enqueue"), 190 dequeuedMessages: consumerScope.Counter("message-dequeue"), 191 processedWrite: consumerScope. 192 Tagged(map[string]string{"result": "write"}). 193 Counter("message-processed"), 194 processedClosed: consumerScope. 195 Tagged(map[string]string{"result": "closed"}). 196 Counter("message-processed"), 197 processedNotReady: consumerScope. 198 Tagged(map[string]string{"result": "not-ready"}). 199 Counter("message-processed"), 200 processedTTL: consumerScope. 201 Tagged(map[string]string{"result": "ttl"}). 202 Counter("message-processed"), 203 processedAck: consumerScope. 204 Tagged(map[string]string{"result": "ack"}). 205 Counter("message-processed"), 206 processedDrop: consumerScope. 207 Tagged(map[string]string{"result": "drop"}). 208 Counter("message-processed"), 209 } 210 } 211 212 type messageWriterImpl struct { 213 sync.RWMutex 214 215 replicatedShardID uint64 216 mPool messagePool 217 opts Options 218 nextRetryAfterNanos MessageRetryNanosFn 219 encoder proto.Encoder 220 numConnections int 221 222 msgID uint64 223 queue *list.List 224 consumerWriters []consumerWriter 225 iterationIndexes []int 226 acks *acks 227 cutOffNanos int64 228 cutOverNanos int64 229 messageTTLNanos int64 230 msgsToWrite []*message 231 isClosed bool 232 doneCh chan struct{} 233 wg sync.WaitGroup 234 // metrics can be updated when a consumer instance changes, so must be guarded with RLock 235 m *messageWriterMetrics 236 nextFullScan time.Time 237 lastNewWrite *list.Element 238 239 nowFn clock.NowFn 240 } 241 242 func newMessageWriter( 243 replicatedShardID uint64, 244 mPool messagePool, 245 opts Options, 246 m messageWriterMetrics, 247 ) messageWriter { 248 if opts == nil { 249 opts = NewOptions() 250 } 251 nowFn := time.Now 252 return &messageWriterImpl{ 253 replicatedShardID: replicatedShardID, 254 mPool: mPool, 255 opts: opts, 256 nextRetryAfterNanos: opts.MessageRetryNanosFn(), 257 encoder: proto.NewEncoder(opts.EncoderOptions()), 258 numConnections: opts.ConnectionOptions().NumConnections(), 259 msgID: 0, 260 queue: list.New(), 261 acks: newAckHelper(opts.InitialAckMapSize()), 262 cutOffNanos: 0, 263 cutOverNanos: 0, 264 msgsToWrite: make([]*message, 0, opts.MessageQueueScanBatchSize()), 265 isClosed: false, 266 doneCh: make(chan struct{}), 267 m: &m, 268 nowFn: nowFn, 269 } 270 } 271 272 func (w *messageWriterImpl) Write(rm *producer.RefCountedMessage) { 273 var ( 274 nowNanos = w.nowFn().UnixNano() 275 msg = w.newMessage() 276 ) 277 w.Lock() 278 if !w.isValidWriteWithLock(nowNanos) { 279 w.Unlock() 280 w.close(msg) 281 return 282 } 283 rm.IncRef() 284 w.msgID++ 285 meta := metadata{ 286 metadataKey: metadataKey{ 287 shard: w.replicatedShardID, 288 id: w.msgID, 289 }, 290 } 291 msg.Set(meta, rm, nowNanos) 292 w.acks.add(meta, msg) 293 // Make sure all the new writes are ordered in queue. 294 w.m.enqueuedMessages.Inc(1) 295 if w.lastNewWrite != nil { 296 w.lastNewWrite = w.queue.InsertAfter(msg, w.lastNewWrite) 297 } else { 298 w.lastNewWrite = w.queue.PushFront(msg) 299 } 300 w.Unlock() 301 } 302 303 func (w *messageWriterImpl) isValidWriteWithLock(nowNanos int64) bool { 304 if w.opts.IgnoreCutoffCutover() { 305 return true 306 } 307 308 if w.cutOffNanos > 0 && nowNanos >= w.cutOffNanos { 309 w.m.writeAfterCutoff.Inc(1) 310 return false 311 } 312 if w.cutOverNanos > 0 && nowNanos < w.cutOverNanos { 313 w.m.writeBeforeCutover.Inc(1) 314 return false 315 } 316 317 return true 318 } 319 320 func (w *messageWriterImpl) write( 321 iterationIndexes []int, 322 consumerWriters []consumerWriter, 323 metrics *messageWriterMetrics, 324 m *message, 325 ) error { 326 m.IncReads() 327 m.SetSentAt(w.nowFn().UnixNano()) 328 msg, isValid := m.Marshaler() 329 if !isValid { 330 m.DecReads() 331 return nil 332 } 333 // The write function is accessed through only one thread, 334 // so no lock is required for encoding. 335 err := w.encoder.Encode(msg) 336 m.DecReads() 337 if err != nil { 338 return err 339 } 340 var ( 341 // NB(r): Always select the same connection index per shard. 342 connIndex = int(w.replicatedShardID % uint64(w.numConnections)) 343 writes int64 344 writeErrors int64 345 ) 346 347 for i := len(iterationIndexes) - 1; i >= 0; i-- { 348 consumerWriter := consumerWriters[randIndex(iterationIndexes, i)] 349 if err := consumerWriter.Write(connIndex, w.encoder.Bytes()); err != nil { 350 writeErrors++ 351 continue 352 } 353 writes++ 354 break 355 } 356 357 if writeErrors > 0 { 358 metrics.oneConsumerWriteError.Inc(writeErrors) 359 } 360 361 if writes > 0 { 362 metrics.writeSuccess.Inc(writes) 363 return nil 364 } 365 // Could not be written to any consumer, will retry later. 366 metrics.allConsumersWriteError.Inc(1) 367 return errFailAllConsumers 368 } 369 370 func randIndex(iterationIndexes []int, i int) int { 371 j := int(unsafe.Fastrandn(uint32(i + 1))) 372 // NB: we should only mutate the order in the iteration indexes and 373 // keep the order of consumer writers unchanged to prevent data race. 374 iterationIndexes[i], iterationIndexes[j] = iterationIndexes[j], iterationIndexes[i] 375 return iterationIndexes[i] 376 } 377 378 func (w *messageWriterImpl) Ack(meta metadata) bool { 379 acked, expectedProcessNanos := w.acks.ack(meta) 380 if acked { 381 w.RLock() 382 defer w.RUnlock() 383 w.m.messageConsumeLatency.Record(time.Duration(w.nowFn().UnixNano() - expectedProcessNanos)) 384 w.m.messageAcked.Inc(1) 385 return true 386 } 387 return false 388 } 389 390 func (w *messageWriterImpl) Init() { 391 w.wg.Add(1) 392 go func() { 393 w.scanMessageQueueUntilClose() 394 w.wg.Done() 395 }() 396 } 397 398 func (w *messageWriterImpl) scanMessageQueueUntilClose() { 399 var ( 400 interval = w.opts.MessageQueueNewWritesScanInterval() 401 jitter = time.Duration( 402 // approx ~40 days of jitter at millisecond precision - more than enough 403 unsafe.Fastrandn(uint32(interval.Milliseconds())), 404 ) * time.Millisecond 405 ) 406 // NB(cw): Add some jitter before the tick starts to reduce 407 // some contention between all the message writers. 408 time.Sleep(jitter) 409 ticker := time.NewTicker(interval) 410 defer ticker.Stop() 411 412 for { 413 select { 414 case <-ticker.C: 415 w.scanMessageQueue() 416 case <-w.doneCh: 417 return 418 } 419 } 420 } 421 422 func (w *messageWriterImpl) scanMessageQueue() { 423 w.RLock() 424 e := w.queue.Front() 425 w.lastNewWrite = nil 426 isClosed := w.isClosed 427 m := w.m 428 w.RUnlock() 429 var ( 430 nowFn = w.nowFn 431 msgsToWrite []*message 432 beforeScan = nowFn() 433 beforeBatchNanos = beforeScan.UnixNano() 434 batchSize = w.opts.MessageQueueScanBatchSize() 435 consumerWriters []consumerWriter 436 iterationIndexes []int 437 fullScan = isClosed || beforeScan.After(w.nextFullScan) 438 scanMetrics scanBatchMetrics 439 skipWrites bool 440 ) 441 defer scanMetrics.record(m) 442 for e != nil { 443 w.Lock() 444 e, msgsToWrite = w.scanBatchWithLock(e, beforeBatchNanos, batchSize, fullScan, &scanMetrics) 445 consumerWriters = w.consumerWriters 446 iterationIndexes = w.iterationIndexes 447 w.Unlock() 448 if !fullScan && len(msgsToWrite) == 0 { 449 m.scanBatchLatency.Record(time.Duration(nowFn().UnixNano() - beforeBatchNanos)) 450 // If this is not a full scan, abort after the iteration batch 451 // that no new messages were found. 452 break 453 } 454 if skipWrites { 455 m.scanBatchLatency.Record(time.Duration(nowFn().UnixNano() - beforeBatchNanos)) 456 continue 457 } 458 if err := w.writeBatch(iterationIndexes, consumerWriters, m, msgsToWrite); err != nil { 459 // When we can't write to any consumer writer, skip the writes in this scan 460 // to avoid meaningless attempts but continue to clean up the queue. 461 skipWrites = true 462 } 463 nowNanos := nowFn().UnixNano() 464 m.scanBatchLatency.Record(time.Duration(nowNanos - beforeBatchNanos)) 465 beforeBatchNanos = nowNanos 466 } 467 afterScan := nowFn() 468 m.scanTotalLatency.Record(afterScan.Sub(beforeScan)) 469 if fullScan { 470 w.nextFullScan = afterScan.Add(w.opts.MessageQueueFullScanInterval()) 471 } 472 } 473 474 func (w *messageWriterImpl) writeBatch( 475 iterationIndexes []int, 476 consumerWriters []consumerWriter, 477 metrics *messageWriterMetrics, 478 messages []*message, 479 ) error { 480 if len(consumerWriters) == 0 { 481 // Not expected in a healthy/valid placement. 482 metrics.noWritersError.Inc(int64(len(messages))) 483 return errNoWriters 484 } 485 delay := metrics.messageWriteDelay 486 nowFn := w.nowFn 487 for i := range messages { 488 if err := w.write(iterationIndexes, consumerWriters, metrics, messages[i]); err != nil { 489 return err 490 } 491 if i%_recordMessageDelayEvery == 0 { 492 delay.Record(time.Duration(nowFn().UnixNano() - messages[i].ExpectedProcessAtNanos())) 493 } 494 } 495 return nil 496 } 497 498 // scanBatchWithLock iterates the message queue with a lock. It returns after 499 // visited enough elements. So it holds the lock for less time and allows new 500 // writes to be unblocked. 501 func (w *messageWriterImpl) scanBatchWithLock( 502 start *list.Element, 503 nowNanos int64, 504 batchSize int, 505 fullScan bool, 506 scanMetrics *scanBatchMetrics, 507 ) (*list.Element, []*message) { 508 var ( 509 iterated int 510 next *list.Element 511 ) 512 w.msgsToWrite = w.msgsToWrite[:0] 513 for e := start; e != nil; e = next { 514 iterated++ 515 if iterated > batchSize { 516 break 517 } 518 next = e.Next() 519 m := e.Value.(*message) 520 if w.isClosed { 521 scanMetrics[_processedClosed]++ 522 // Simply ack the messages here to mark them as consumed for this 523 // message writer, this is useful when user removes a consumer service 524 // during runtime that may be unhealthy to consume the messages. 525 // So that the unacked messages for the unhealthy consumer services 526 // do not stay in memory forever. 527 // NB: The message must be added to the ack map to be acked here. 528 w.acks.ack(m.Metadata()) 529 w.removeFromQueueWithLock(e, m) 530 scanMetrics[_messageClosed]++ 531 continue 532 } 533 if m.RetryAtNanos() >= nowNanos { 534 scanMetrics[_processedNotReady]++ 535 if !fullScan { 536 // If this is not a full scan, bail after the first element that 537 // is not a new write. 538 break 539 } 540 continue 541 } 542 // If the message exceeded its allowed ttl of the consumer service, 543 // remove it from the buffer. 544 if w.messageTTLNanos > 0 && m.InitNanos()+w.messageTTLNanos <= nowNanos { 545 scanMetrics[_processedTTL]++ 546 // There is a chance the message was acked right before the ack is 547 // called, in which case just remove it from the queue. 548 if acked, _ := w.acks.ack(m.Metadata()); acked { 549 scanMetrics[_messageDroppedTTLExpire]++ 550 } 551 w.removeFromQueueWithLock(e, m) 552 continue 553 } 554 if m.IsAcked() { 555 scanMetrics[_processedAck]++ 556 w.removeFromQueueWithLock(e, m) 557 continue 558 } 559 if m.IsDroppedOrConsumed() { 560 scanMetrics[_processedDrop]++ 561 // There is a chance the message could be acked between m.Acked() 562 // and m.IsDroppedOrConsumed() check, in which case we should not 563 // mark it as dropped, just continue and next tick will remove it 564 // as acked. 565 if m.IsAcked() { 566 continue 567 } 568 w.acks.remove(m.Metadata()) 569 w.removeFromQueueWithLock(e, m) 570 scanMetrics[_messageDroppedBufferFull]++ 571 continue 572 } 573 m.IncWriteTimes() 574 writeTimes := m.WriteTimes() 575 m.SetRetryAtNanos(w.nextRetryAfterNanos(writeTimes) + nowNanos) 576 if writeTimes > 1 { 577 scanMetrics[_messageRetry]++ 578 } 579 scanMetrics[_processedWrite]++ 580 w.msgsToWrite = append(w.msgsToWrite, m) 581 } 582 return next, w.msgsToWrite 583 } 584 585 func (w *messageWriterImpl) Close() { 586 w.Lock() 587 if w.isClosed { 588 w.Unlock() 589 return 590 } 591 w.isClosed = true 592 w.Unlock() 593 // NB: Wait until all messages cleaned up then close. 594 w.waitUntilAllMessageRemoved() 595 close(w.doneCh) 596 w.wg.Wait() 597 } 598 599 func (w *messageWriterImpl) waitUntilAllMessageRemoved() { 600 // The message writers are being closed sequentially, checking isEmpty() 601 // before always waiting for the first tick can speed up Close() a lot. 602 if w.isEmpty() { 603 return 604 } 605 ticker := time.NewTicker(w.opts.CloseCheckInterval()) 606 defer ticker.Stop() 607 608 for range ticker.C { 609 if w.isEmpty() { 610 return 611 } 612 } 613 } 614 615 func (w *messageWriterImpl) isEmpty() bool { 616 w.RLock() 617 l := w.queue.Len() 618 w.RUnlock() 619 return l == 0 620 } 621 622 func (w *messageWriterImpl) ReplicatedShardID() uint64 { 623 return w.replicatedShardID 624 } 625 626 func (w *messageWriterImpl) CutoffNanos() int64 { 627 w.RLock() 628 res := w.cutOffNanos 629 w.RUnlock() 630 return res 631 } 632 633 func (w *messageWriterImpl) SetCutoffNanos(nanos int64) { 634 w.Lock() 635 w.cutOffNanos = nanos 636 w.Unlock() 637 } 638 639 func (w *messageWriterImpl) CutoverNanos() int64 { 640 w.RLock() 641 res := w.cutOverNanos 642 w.RUnlock() 643 return res 644 } 645 646 func (w *messageWriterImpl) SetCutoverNanos(nanos int64) { 647 w.Lock() 648 w.cutOverNanos = nanos 649 w.Unlock() 650 } 651 652 func (w *messageWriterImpl) MessageTTLNanos() int64 { 653 w.RLock() 654 res := w.messageTTLNanos 655 w.RUnlock() 656 return res 657 } 658 659 func (w *messageWriterImpl) SetMessageTTLNanos(nanos int64) { 660 w.Lock() 661 w.messageTTLNanos = nanos 662 w.Unlock() 663 } 664 665 func (w *messageWriterImpl) AddConsumerWriter(cw consumerWriter) { 666 w.Lock() 667 newConsumerWriters := make([]consumerWriter, 0, len(w.consumerWriters)+1) 668 newConsumerWriters = append(newConsumerWriters, w.consumerWriters...) 669 newConsumerWriters = append(newConsumerWriters, cw) 670 671 w.iterationIndexes = make([]int, len(newConsumerWriters)) 672 for i := range w.iterationIndexes { 673 w.iterationIndexes[i] = i 674 } 675 w.consumerWriters = newConsumerWriters 676 w.Unlock() 677 } 678 679 func (w *messageWriterImpl) RemoveConsumerWriter(addr string) { 680 w.Lock() 681 newConsumerWriters := make([]consumerWriter, 0, len(w.consumerWriters)-1) 682 for _, cw := range w.consumerWriters { 683 if cw.Address() == addr { 684 continue 685 } 686 newConsumerWriters = append(newConsumerWriters, cw) 687 } 688 689 w.iterationIndexes = make([]int, len(newConsumerWriters)) 690 for i := range w.iterationIndexes { 691 w.iterationIndexes[i] = i 692 } 693 w.consumerWriters = newConsumerWriters 694 w.Unlock() 695 } 696 697 func (w *messageWriterImpl) Metrics() messageWriterMetrics { 698 w.RLock() 699 defer w.RUnlock() 700 return *w.m 701 } 702 703 func (w *messageWriterImpl) SetMetrics(m messageWriterMetrics) { 704 w.Lock() 705 w.m = &m 706 w.Unlock() 707 } 708 709 func (w *messageWriterImpl) QueueSize() int { 710 return w.acks.size() 711 } 712 713 func (w *messageWriterImpl) newMessage() *message { 714 if w.mPool != nil { 715 return w.mPool.Get() 716 } 717 return newMessage() 718 } 719 720 func (w *messageWriterImpl) removeFromQueueWithLock(e *list.Element, m *message) { 721 w.queue.Remove(e) 722 w.m.dequeuedMessages.Inc(1) 723 w.close(m) 724 } 725 726 func (w *messageWriterImpl) close(m *message) { 727 if w.mPool != nil { 728 m.Close() 729 w.mPool.Put(m) 730 } 731 } 732 733 type acks struct { 734 sync.Mutex 735 736 ackMap map[metadataKey]*message 737 } 738 739 // nolint: unparam 740 func newAckHelper(size int) *acks { 741 return &acks{ 742 ackMap: make(map[metadataKey]*message, size), 743 } 744 } 745 746 func (a *acks) add(meta metadata, m *message) { 747 a.Lock() 748 a.ackMap[meta.metadataKey] = m 749 a.Unlock() 750 } 751 752 func (a *acks) remove(meta metadata) { 753 a.Lock() 754 delete(a.ackMap, meta.metadataKey) 755 a.Unlock() 756 } 757 758 // ack processes the ack. returns true if the message was not already acked. additionally returns the expected 759 // processing time for lag calculations. 760 func (a *acks) ack(meta metadata) (bool, int64) { 761 a.Lock() 762 m, ok := a.ackMap[meta.metadataKey] 763 if !ok { 764 a.Unlock() 765 // Acking a message that is already acked, which is ok. 766 return false, 0 767 } 768 delete(a.ackMap, meta.metadataKey) 769 a.Unlock() 770 expectedProcessAtNanos := m.ExpectedProcessAtNanos() 771 m.Ack() 772 return true, expectedProcessAtNanos 773 } 774 775 func (a *acks) size() int { 776 a.Lock() 777 l := len(a.ackMap) 778 a.Unlock() 779 return l 780 } 781 782 type metricIdx byte 783 784 const ( 785 _messageClosed metricIdx = iota 786 _messageDroppedBufferFull 787 _messageDroppedTTLExpire 788 _messageRetry 789 _processedAck 790 _processedClosed 791 _processedDrop 792 _processedNotReady 793 _processedTTL 794 _processedWrite 795 _lastMetricIdx 796 ) 797 798 type scanBatchMetrics [_lastMetricIdx]int32 799 800 func (m *scanBatchMetrics) record(metrics *messageWriterMetrics) { 801 m.recordNonzeroCounter(_messageClosed, metrics.messageClosed) 802 m.recordNonzeroCounter(_messageDroppedBufferFull, metrics.messageDroppedBufferFull) 803 m.recordNonzeroCounter(_messageDroppedTTLExpire, metrics.messageDroppedTTLExpire) 804 m.recordNonzeroCounter(_messageRetry, metrics.messageRetry) 805 m.recordNonzeroCounter(_processedAck, metrics.processedAck) 806 m.recordNonzeroCounter(_processedClosed, metrics.processedClosed) 807 m.recordNonzeroCounter(_processedDrop, metrics.processedDrop) 808 m.recordNonzeroCounter(_processedNotReady, metrics.processedNotReady) 809 m.recordNonzeroCounter(_processedTTL, metrics.processedTTL) 810 m.recordNonzeroCounter(_processedWrite, metrics.processedWrite) 811 } 812 813 func (m *scanBatchMetrics) recordNonzeroCounter(idx metricIdx, c tally.Counter) { 814 if m[idx] > 0 { 815 c.Inc(int64(m[idx])) 816 } 817 } 818 819 // NextRetryNanosFn creates a MessageRetryNanosFn based on the retry options. 820 func NextRetryNanosFn(retryOpts retry.Options) func(int) int64 { 821 var ( 822 jitter = retryOpts.Jitter() 823 backoffFactor = retryOpts.BackoffFactor() 824 initialBackoff = retryOpts.InitialBackoff() 825 maxBackoff = retryOpts.MaxBackoff() 826 initialBackoffFloat = float64(initialBackoff) 827 ) 828 829 // inlined and specialized retry function that does not have any state that needs to be kept 830 // between tries 831 return func(writeTimes int) int64 { 832 backoff := initialBackoff.Nanoseconds() 833 if writeTimes >= 1 { 834 backoffFloat64 := initialBackoffFloat * math.Pow(backoffFactor, float64(writeTimes-1)) 835 backoff = int64(backoffFloat64) 836 } 837 // Validate the value of backoff to make sure Fastrandn() does not panic and 838 // check for overflow from the exponentiation op - unlikely, but prevents weird side effects. 839 halfInMicros := (backoff / 2) / int64(time.Microsecond) 840 if jitter && backoff >= 2 && halfInMicros < math.MaxUint32 { 841 // Jitter can be only up to ~1 hour in microseconds, but it's not a limitation here 842 jitterInMicros := unsafe.Fastrandn(uint32(halfInMicros)) 843 jitterInNanos := time.Duration(jitterInMicros) * time.Microsecond 844 halfInNanos := time.Duration(halfInMicros) * time.Microsecond 845 backoff = int64(halfInNanos + jitterInNanos) 846 } 847 // Clamp backoff to maxBackoff 848 if maxBackoff := maxBackoff.Nanoseconds(); backoff > maxBackoff { 849 backoff = maxBackoff 850 } 851 return backoff 852 } 853 } 854 855 // StaticRetryNanosFn creates a MessageRetryNanosFn based on static config. 856 func StaticRetryNanosFn(backoffDurations []time.Duration) (MessageRetryNanosFn, error) { 857 if len(backoffDurations) == 0 { 858 return nil, errInvalidBackoffDuration 859 } 860 backoffInt64s := make([]int64, 0, len(backoffDurations)) 861 for _, b := range backoffDurations { 862 backoffInt64s = append(backoffInt64s, int64(b)) 863 } 864 return func(writeTimes int) int64 { 865 retry := writeTimes - 1 866 l := len(backoffInt64s) 867 if retry < l { 868 return backoffInt64s[retry] 869 } 870 return backoffInt64s[l-1] 871 }, nil 872 }