github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/persist/fs/commitlog/commit_log.go (about) 1 // Copyright (c) 2016 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 commitlog 22 23 import ( 24 "errors" 25 "fmt" 26 "sync" 27 "sync/atomic" 28 "time" 29 30 "github.com/m3db/m3/src/dbnode/persist" 31 "github.com/m3db/m3/src/dbnode/persist/fs" 32 "github.com/m3db/m3/src/dbnode/ts" 33 "github.com/m3db/m3/src/dbnode/ts/writes" 34 "github.com/m3db/m3/src/x/clock" 35 "github.com/m3db/m3/src/x/context" 36 xerrors "github.com/m3db/m3/src/x/errors" 37 xtime "github.com/m3db/m3/src/x/time" 38 39 "github.com/uber-go/tally" 40 "go.uber.org/zap" 41 ) 42 43 var ( 44 // ErrCommitLogQueueFull is raised when trying to write to the commit log 45 // when the queue is full 46 ErrCommitLogQueueFull = errors.New("commit log queue is full") 47 48 errCommitLogClosed = errors.New("commit log is closed") 49 50 zeroFile = persist.CommitLogFile{} 51 ) 52 53 type newCommitLogWriterFn func( 54 flushFn flushFn, 55 opts Options, 56 ) commitLogWriter 57 58 type writeCommitLogFn func( 59 ctx context.Context, 60 writes writeOrWriteBatch, 61 ) error 62 63 type commitLogFailFn func(err error) 64 65 // writeOrWriteBatch is a union type of write or writeBatch so that 66 // we can handle both cases without having to allocate as slice of size 67 // 1 to handle a single write. 68 type writeOrWriteBatch struct { 69 write writes.Write 70 writeBatch writes.WriteBatch 71 } 72 73 type commitLog struct { 74 // The commitlog has two different locks that it maintains: 75 // 76 // 1) The closedState lock is acquired and held for any actions taking place that 77 // the commitlog must remain open for the duration of (or for changing the state 78 // of the commitlog to closed). 79 // 80 // 81 // 2) The flushState is only used for reading and writing the lastFlushAt variable. The scope 82 // of the flushState lock is very limited and is hidden behind helper methods for getting and 83 // setting the value of lastFlushAt. 84 closedState closedState 85 flushState flushState 86 87 writerState writerState 88 89 // Associated with the closedState, but stored separately since 90 // it does not require the closedState lock to be acquired before 91 // being accessed. 92 closeErr chan error 93 94 writes chan commitLogWrite 95 maxQueueSize int64 96 97 opts Options 98 nowFn clock.NowFn 99 log *zap.Logger 100 101 newCommitLogWriterFn newCommitLogWriterFn 102 writeFn writeCommitLogFn 103 commitLogFailFn commitLogFailFn 104 beforeAsyncWriteFn func() 105 106 metrics commitLogMetrics 107 108 numWritesInQueue int64 109 } 110 111 // Use the helper methods when interacting with this struct, the mutex 112 // should never need to be manually interacted with. 113 type flushState struct { 114 sync.RWMutex 115 lastFlushAt time.Time 116 } 117 118 func (f *flushState) setLastFlushAt(t time.Time) { 119 f.Lock() 120 f.lastFlushAt = t 121 f.Unlock() 122 } 123 124 func (f *flushState) getLastFlushAt() time.Time { 125 f.RLock() 126 lastFlush := f.lastFlushAt 127 f.RUnlock() 128 return lastFlush 129 } 130 131 type writerState struct { 132 // See "Rotating Files" section of README.md for an explanation of how the 133 // primary and secondary fields are used during commitlog rotation. 134 primary asyncResettableWriter 135 secondary asyncResettableWriter 136 // Convenience slice that is used to simplify code whenever an operation needs 137 // to be performed on both primary and secondary writers. Note that order is not 138 // maintained (I.E primary may be index 1 and secondary index 0) so the slice can 139 // only be used when the order of operations does not matter. 140 writers []commitLogWriter 141 activeFiles persist.CommitLogFiles 142 } 143 144 type asyncResettableWriter struct { 145 // Backreference to commitlog for the purpose of calling onFlush(). 146 commitlog *commitLog 147 // The commitlog writer is single-threaded, so normally the commitLogWriter can be 148 // accessed without synchronization. However, since the secondaryWriter is reset by 149 // a background goroutine, a waitgroup is used to ensure that the previous background 150 // reset has completed before attempting to access the secondary writers state and/or 151 // begin a new hot-swap. 152 *sync.WaitGroup 153 writer commitLogWriter 154 // Each writer maintains its own slice of pending flushFns because each writer will get 155 // flushed independently. This is important for maintaining correctness in code paths 156 // that care about durability, particularly during commitlog rotations. 157 // 158 // For example, imagine a call to WriteWait() occurs and the pending write is buffered 159 // in commitlog 1, but not yet flushed. Subsequently, a call to RotateLogs() occurs causing 160 // commitlog 1 to be (asynchronously) reset and commitlog 2 to become the new primary. Once 161 // the asynchronous Close and flush of commitlog 1 completes, only pending flushFns associated 162 // with commitlog 1 should be called as the writer associated with commitlog 2 may not have been 163 // flushed at all yet. 164 pendingFlushFns []callbackFn 165 } 166 167 func (w *asyncResettableWriter) onFlush(err error) { 168 w.commitlog.onFlush(w, err) 169 } 170 171 type closedState struct { 172 sync.RWMutex 173 closed bool 174 } 175 176 type commitLogMetrics struct { 177 numWritesInQueue tally.Gauge 178 queueLength tally.Gauge 179 queueCapacity tally.Gauge 180 success tally.Counter 181 errors tally.Counter 182 openErrors tally.Counter 183 closeErrors tally.Counter 184 flushErrors tally.Counter 185 flushDone tally.Counter 186 } 187 188 type eventType int 189 190 // nolint: varcheck, unused 191 const ( 192 writeEventType eventType = iota 193 flushEventType 194 activeLogsEventType 195 rotateLogsEventType 196 ) 197 198 type callbackFn func(callbackResult) 199 200 type callbackResult struct { 201 eventType eventType 202 err error 203 activeLogs activeLogsCallbackResult 204 rotateLogs rotateLogsResult 205 } 206 207 type activeLogsCallbackResult struct { 208 files persist.CommitLogFiles 209 } 210 211 type rotateLogsResult struct { 212 file persist.CommitLogFile 213 } 214 215 func (r callbackResult) activeLogsCallbackResult() (activeLogsCallbackResult, error) { 216 if r.eventType != activeLogsEventType { 217 return activeLogsCallbackResult{}, fmt.Errorf( 218 "wrong event type: expected %d but got %d", 219 activeLogsEventType, r.eventType) 220 } 221 222 if r.err != nil { 223 return activeLogsCallbackResult{}, nil 224 } 225 226 return r.activeLogs, nil 227 } 228 229 func (r callbackResult) rotateLogsResult() (rotateLogsResult, error) { 230 if r.eventType != rotateLogsEventType { 231 return rotateLogsResult{}, fmt.Errorf( 232 "wrong event type: expected %d but got %d", 233 rotateLogsEventType, r.eventType) 234 } 235 236 if r.err != nil { 237 return rotateLogsResult{}, nil 238 } 239 240 return r.rotateLogs, nil 241 } 242 243 type commitLogWrite struct { 244 eventType eventType 245 write writeOrWriteBatch 246 callbackFn callbackFn 247 } 248 249 type testOnlyOpts struct { 250 beforeAsyncWriteFn func() 251 } 252 253 // NewCommitLog creates a new commit log 254 func NewCommitLog(opts Options) (CommitLog, error) { 255 return newCommitLog(opts, testOnlyOpts{}) 256 } 257 258 func newCommitLog(opts Options, testOpts testOnlyOpts) (CommitLog, error) { 259 if err := opts.Validate(); err != nil { 260 return nil, err 261 } 262 iopts := opts.InstrumentOptions().SetMetricsScope( 263 opts.InstrumentOptions().MetricsScope().SubScope("commitlog")) 264 scope := iopts.MetricsScope() 265 266 commitLog := &commitLog{ 267 opts: opts, 268 nowFn: opts.ClockOptions().NowFn(), 269 log: iopts.Logger(), 270 newCommitLogWriterFn: newCommitLogWriter, 271 writes: make(chan commitLogWrite, opts.BacklogQueueChannelSize()), 272 writerState: writerState{ 273 primary: asyncResettableWriter{ 274 WaitGroup: &sync.WaitGroup{}, 275 }, 276 secondary: asyncResettableWriter{ 277 WaitGroup: &sync.WaitGroup{}, 278 }, 279 }, 280 maxQueueSize: int64(opts.BacklogQueueSize()), 281 closeErr: make(chan error), 282 metrics: commitLogMetrics{ 283 numWritesInQueue: scope.Gauge("writes.queued"), 284 queueLength: scope.Gauge("writes.queue-length"), 285 queueCapacity: scope.Gauge("writes.queue-capacity"), 286 success: scope.Counter("writes.success"), 287 errors: scope.Counter("writes.errors"), 288 openErrors: scope.Counter("writes.open-errors"), 289 closeErrors: scope.Counter("writes.close-errors"), 290 flushErrors: scope.Counter("writes.flush-errors"), 291 flushDone: scope.Counter("writes.flush-done"), 292 }, 293 beforeAsyncWriteFn: testOpts.beforeAsyncWriteFn, 294 } 295 // Setup backreferences for onFlush(). 296 commitLog.writerState.primary.commitlog = commitLog 297 commitLog.writerState.secondary.commitlog = commitLog 298 299 switch opts.Strategy() { 300 case StrategyWriteWait: 301 commitLog.writeFn = commitLog.writeWait 302 default: 303 commitLog.writeFn = commitLog.writeBehind 304 } 305 306 return commitLog, nil 307 } 308 309 func (l *commitLog) Open() error { 310 l.closedState.Lock() 311 defer l.closedState.Unlock() 312 313 // Open the buffered commit log writer 314 if _, _, err := l.openWriters(); err != nil { 315 return err 316 } 317 318 // Sync the info header to ensure we can write to disk and make sure that we can at least 319 // read the info about the commitlog file later. 320 for _, writer := range l.writerState.writers { 321 if err := writer.Flush(true); err != nil { 322 return err 323 } 324 } 325 326 l.commitLogFailFn = func(err error) { 327 strategy := l.opts.FailureStrategy() 328 fatal := strategy == FailureStrategyPanic 329 if strategy == FailureStrategyCallback && !l.opts.FailureCallback()(err) { 330 fatal = true 331 } 332 if fatal { 333 l.log.Fatal("fatal commit log write error", zap.Error(err)) 334 } else { 335 l.log.Error("commit log write error", zap.Error(err)) 336 } 337 } 338 339 // Asynchronously write 340 go func() { 341 // a test hook to block/release the async writer 342 if l.beforeAsyncWriteFn != nil { 343 l.beforeAsyncWriteFn() 344 } 345 l.write() 346 }() 347 348 if flushInterval := l.opts.FlushInterval(); flushInterval > 0 { 349 // Continually flush the commit log at given interval if set 350 go l.flushEvery(flushInterval) 351 } 352 353 return nil 354 } 355 356 func (l *commitLog) ActiveLogs() (persist.CommitLogFiles, error) { 357 l.closedState.RLock() 358 defer l.closedState.RUnlock() 359 360 if l.closedState.closed { 361 return nil, errCommitLogClosed 362 } 363 364 var ( 365 err error 366 files []persist.CommitLogFile 367 wg sync.WaitGroup 368 ) 369 wg.Add(1) 370 371 l.writes <- commitLogWrite{ 372 eventType: activeLogsEventType, 373 callbackFn: func(r callbackResult) { 374 defer wg.Done() 375 376 result, e := r.activeLogsCallbackResult() 377 if e != nil { 378 err = e 379 return 380 } 381 382 files = result.files 383 }, 384 } 385 386 wg.Wait() 387 388 if err != nil { 389 return nil, err 390 } 391 392 return files, nil 393 } 394 395 func (l *commitLog) RotateLogs() (persist.CommitLogFile, error) { 396 l.closedState.RLock() 397 defer l.closedState.RUnlock() 398 399 if l.closedState.closed { 400 return zeroFile, errCommitLogClosed 401 } 402 403 var ( 404 err error 405 file persist.CommitLogFile 406 wg sync.WaitGroup 407 ) 408 wg.Add(1) 409 410 l.writes <- commitLogWrite{ 411 eventType: rotateLogsEventType, 412 callbackFn: func(r callbackResult) { 413 defer wg.Done() 414 415 result, e := r.rotateLogsResult() 416 file, err = result.file, e 417 }, 418 } 419 420 wg.Wait() 421 422 if err != nil { 423 return zeroFile, err 424 } 425 426 return file, nil 427 } 428 429 func (l *commitLog) QueueLength() int64 { 430 return atomic.LoadInt64(&l.numWritesInQueue) 431 } 432 433 func (l *commitLog) flushEvery(interval time.Duration) { 434 // Periodically flush the underlying commit log writer to cover 435 // the case when writes stall for a considerable time 436 var sleepForOverride time.Duration 437 438 for { 439 // The number of actual metrics / writes in the queue. 440 l.metrics.numWritesInQueue.Update(float64(atomic.LoadInt64(&l.numWritesInQueue))) 441 // The current length of the queue, different from number of writes due to each 442 // item in the queue could (potentially) be a batch of many writes. 443 l.metrics.queueLength.Update(float64(len(l.writes))) 444 l.metrics.queueCapacity.Update(float64(cap(l.writes))) 445 446 sleepFor := interval 447 448 if sleepForOverride > 0 { 449 sleepFor = sleepForOverride 450 sleepForOverride = 0 451 } 452 453 time.Sleep(sleepFor) 454 455 lastFlushAt := l.flushState.getLastFlushAt() 456 if sinceFlush := l.nowFn().Sub(lastFlushAt); sinceFlush < interval { 457 // Flushed already recently, sleep until we would next consider flushing 458 sleepForOverride = interval - sinceFlush 459 continue 460 } 461 462 // Request a flush 463 l.closedState.RLock() 464 if l.closedState.closed { 465 l.closedState.RUnlock() 466 return 467 } 468 469 l.writes <- commitLogWrite{eventType: flushEventType} 470 l.closedState.RUnlock() 471 } 472 } 473 474 func (l *commitLog) write() { 475 // We use these to make the batch and non-batched write paths the same 476 // by turning non-batched writes into a batch of size one while avoiding 477 // any allocations. 478 singleBatch := make([]writes.BatchWrite, 1) 479 var batch []writes.BatchWrite 480 481 for write := range l.writes { 482 if write.eventType == flushEventType { 483 l.writerState.primary.writer.Flush(false) 484 continue 485 } 486 487 if write.eventType == activeLogsEventType { 488 write.callbackFn(callbackResult{ 489 eventType: write.eventType, 490 err: nil, 491 activeLogs: activeLogsCallbackResult{ 492 files: l.writerState.activeFiles, 493 }, 494 }) 495 continue 496 } 497 498 // For writes requiring acks add to pending acks 499 if write.eventType == writeEventType && write.callbackFn != nil { 500 l.writerState.primary.pendingFlushFns = append( 501 l.writerState.primary.pendingFlushFns, write.callbackFn) 502 } 503 504 isRotateLogsEvent := write.eventType == rotateLogsEventType 505 if isRotateLogsEvent { 506 primaryFile, _, err := l.openWriters() 507 if err != nil { 508 l.metrics.errors.Inc(1) 509 l.metrics.openErrors.Inc(1) 510 l.log.Error("failed to open commit log", zap.Error(err)) 511 512 if l.commitLogFailFn != nil { 513 l.commitLogFailFn(err) 514 } 515 } 516 517 write.callbackFn(callbackResult{ 518 eventType: write.eventType, 519 err: err, 520 rotateLogs: rotateLogsResult{ 521 file: primaryFile, 522 }, 523 }) 524 525 continue 526 } 527 528 var ( 529 numWritesSuccess int64 530 numDequeued int 531 ) 532 533 if write.write.writeBatch == nil { 534 singleBatch[0].Write = write.write.write 535 batch = singleBatch 536 } else { 537 batch = write.write.writeBatch.Iter() 538 } 539 numDequeued = len(batch) 540 541 for _, writeBatch := range batch { 542 if writeBatch.Err != nil { 543 // This entry was not written successfully to the in-memory datastructures so 544 // we should not persist it to the commitlog. This is important to maintain 545 // consistency and the integrity of M3DB's business logic, but also because if 546 // the write does not succeed to the in-memory datastructures then we don't have 547 // access to long-lived identifiers like the seriesID (which is pooled) so 548 // attempting to write would cause pooling / lifecycle issues as well. 549 continue 550 } 551 552 if writeBatch.SkipWrite { 553 // This entry should not be written to the commitlog as it is a duplicate 554 // datapoint. 555 continue 556 } 557 558 write := writeBatch.Write 559 err := l.writerState.primary.writer.Write(write.Series, 560 write.Datapoint, write.Unit, write.Annotation) 561 if err != nil { 562 l.handleWriteErr(err) 563 continue 564 } 565 numWritesSuccess++ 566 } 567 568 // Return the write batch to the pool. 569 if write.write.writeBatch != nil { 570 write.write.writeBatch.Finalize() 571 } 572 573 atomic.AddInt64(&l.numWritesInQueue, int64(-numDequeued)) 574 l.metrics.success.Inc(numWritesSuccess) 575 } 576 577 // Ensure that there is no active background goroutine in the middle of reseting 578 // the secondary writer / modifying its state. 579 l.waitForSecondaryWriterAsyncResetComplete() 580 581 var multiErr xerrors.MultiError 582 for i, writer := range l.writerState.writers { 583 if writer == nil { 584 // Can be nil in the case where the background goroutine spawned in openWriters 585 // encountered an error trying to re-open it. 586 continue 587 } 588 589 multiErr = multiErr.Add(writer.Close()) 590 l.writerState.writers[i] = nil 591 } 592 l.writerState.primary.writer = nil 593 l.writerState.secondary.writer = nil 594 595 l.closeErr <- multiErr.FinalError() 596 } 597 598 func (l *commitLog) onFlush(writer *asyncResettableWriter, err error) { 599 l.flushState.setLastFlushAt(l.nowFn()) 600 601 if err != nil { 602 l.metrics.errors.Inc(1) 603 l.metrics.flushErrors.Inc(1) 604 l.log.Error("failed to flush commit log", zap.Error(err)) 605 606 if l.commitLogFailFn != nil { 607 l.commitLogFailFn(err) 608 } 609 } 610 611 // onFlush will never be called concurrently. The flushFn for the primaryWriter 612 // will only ever be called synchronously by the single-threaded writer goroutine 613 // and the flushFn for the secondaryWriter will only be called by the asynchronous 614 // goroutine (created by the single-threaded writer) when it calls Close() on the 615 // secondary (previously primary due to a hot-swap) writer during the reset. 616 // 617 // Note that both the primary and secondar's flushFn may be called during calls to 618 // Open() on the commitlog, but this takes place before the single-threaded writer 619 // is spawned which precludes it from occurring concurrently with either of the 620 // scenarios described above. 621 if len(writer.pendingFlushFns) == 0 { 622 l.metrics.flushDone.Inc(1) 623 return 624 } 625 626 for i := range writer.pendingFlushFns { 627 writer.pendingFlushFns[i](callbackResult{ 628 eventType: flushEventType, 629 err: err, 630 }) 631 writer.pendingFlushFns[i] = nil 632 } 633 writer.pendingFlushFns = writer.pendingFlushFns[:0] 634 l.metrics.flushDone.Inc(1) 635 } 636 637 // writerState lock must be held for the duration of this function call. 638 func (l *commitLog) openWriters() (persist.CommitLogFile, persist.CommitLogFile, error) { 639 // Ensure that the previous asynchronous reset of the secondary writer (if any) 640 // has completed before attempting to start a new one and/or modify the writerState 641 // in any way. 642 l.waitForSecondaryWriterAsyncResetComplete() 643 644 if l.writerState.primary.writer == nil || l.writerState.secondary.writer == nil { 645 if l.writerState.primary.writer != nil { 646 // Make sure to close and flush any remaining data before creating a new writer if the 647 // primary (which contains data) is not nil. 648 if err := l.writerState.primary.writer.Close(); err != nil { 649 return zeroFile, zeroFile, err 650 } 651 } 652 653 if l.writerState.secondary.writer != nil { 654 // Ignore errors because the secondary file doesn't have any data. 655 l.writerState.secondary.writer.Close() 656 } 657 658 // If either of the commitlog writers is nil then open both of them synchronously. Under 659 // normal circumstances this will only occur when the commitlog is first opened. Although 660 // it can also happen if something goes wrong during the asynchronous reset of the secondary 661 // writer in which case this path will try again, but synchronously this time. 662 l.writerState.primary.writer = l.newCommitLogWriterFn(l.writerState.primary.onFlush, l.opts) 663 l.writerState.secondary.writer = l.newCommitLogWriterFn(l.writerState.secondary.onFlush, l.opts) 664 665 primaryFile, err := l.writerState.primary.writer.Open() 666 if err != nil { 667 return zeroFile, zeroFile, err 668 } 669 670 secondaryFile, err := l.writerState.secondary.writer.Open() 671 if err != nil { 672 return zeroFile, zeroFile, err 673 } 674 675 l.writerState.activeFiles = persist.CommitLogFiles{primaryFile, secondaryFile} 676 l.writerState.writers = []commitLogWriter{ 677 l.writerState.primary.writer, 678 l.writerState.secondary.writer, 679 } 680 681 return primaryFile, secondaryFile, nil 682 } 683 684 // Swap the primary and secondary writers so that the secondary becomes primary and vice versa. 685 // This consumes the standby secondary writer, but a new one will be prepared asynchronously by 686 // resetting the formerly primary writer. 687 l.writerState.primary, l.writerState.secondary = l.writerState.secondary, l.writerState.primary 688 689 // This is necessary because of how the above swap works. 690 // We have 2 instances of asyncResettableWriter which are writeState.primary and 691 // writeState.secondary. We never recreate these instances, and they are embedded 692 // in the struct not pointers. 693 // primary.onFlush is a method value bound with primary as a receiver, similarly for secondary. 694 // We pass these method values (mPrimary and mSecondary for primary and secondary respectively) 695 // into the writer at initialization. 696 // When we swap, we copy all the pointers in the asyncResettableWriter over each other. 697 // However, the stored copies of these method values are not updated leaving 698 // primary with a reference to mSecondary and secondary with a reference to mPrimary. 699 l.writerState.primary.writer.setOnFlush(l.writerState.primary.onFlush) 700 l.writerState.secondary.writer.setOnFlush(l.writerState.secondary.onFlush) 701 702 l.startSecondaryWriterAsyncReset() 703 704 var ( 705 // Determine the persist.CommitLogFile for the not-yet-created secondary file so that the 706 // ActiveLogs() API returns the correct values even before the asynchronous reset completes. 707 primaryFile = l.writerState.activeFiles[1] 708 fsPrefix = l.opts.FilesystemOptions().FilePathPrefix() 709 nextIndex = primaryFile.Index + 1 710 secondaryFile = persist.CommitLogFile{ 711 FilePath: fs.CommitLogFilePath(fsPrefix, int(nextIndex)), 712 Index: nextIndex, 713 } 714 ) 715 files := persist.CommitLogFiles{primaryFile, secondaryFile} 716 l.writerState.activeFiles = files 717 718 return primaryFile, secondaryFile, nil 719 } 720 721 func (l *commitLog) startSecondaryWriterAsyncReset() { 722 l.writerState.secondary.Add(1) 723 724 go func() { 725 var err error 726 defer func() { 727 if err != nil { 728 // Set to nil so that the next call to openWriters() will attempt 729 // to try and create a new writer. 730 l.writerState.secondary.writer = nil 731 732 l.metrics.errors.Inc(1) 733 l.metrics.openErrors.Inc(1) 734 } 735 736 l.writerState.secondary.Done() 737 }() 738 739 if err = l.writerState.secondary.writer.Close(); err != nil { 740 l.commitLogFailFn(err) 741 return 742 } 743 744 _, err = l.writerState.secondary.writer.Open() 745 if err != nil { 746 l.commitLogFailFn(err) 747 return 748 } 749 }() 750 } 751 752 func (l *commitLog) waitForSecondaryWriterAsyncResetComplete() { 753 l.writerState.secondary.Wait() 754 } 755 756 func (l *commitLog) Write( 757 ctx context.Context, 758 series ts.Series, 759 datapoint ts.Datapoint, 760 unit xtime.Unit, 761 annotation ts.Annotation, 762 ) error { 763 return l.writeFn(ctx, writeOrWriteBatch{ 764 write: writes.Write{ 765 Series: series, 766 Datapoint: datapoint, 767 Unit: unit, 768 Annotation: annotation, 769 }, 770 }) 771 } 772 773 func (l *commitLog) WriteBatch( 774 ctx context.Context, 775 writes writes.WriteBatch, 776 ) error { 777 return l.writeFn(ctx, writeOrWriteBatch{ 778 writeBatch: writes, 779 }) 780 } 781 782 func (l *commitLog) writeWait( 783 ctx context.Context, 784 write writeOrWriteBatch, 785 ) error { 786 l.closedState.RLock() 787 if l.closedState.closed { 788 l.closedState.RUnlock() 789 return errCommitLogClosed 790 } 791 792 var ( 793 wg sync.WaitGroup 794 result error 795 ) 796 797 wg.Add(1) 798 799 completion := func(r callbackResult) { 800 result = r.err 801 wg.Done() 802 } 803 804 writeToEnqueue := commitLogWrite{ 805 write: write, 806 callbackFn: completion, 807 } 808 809 numToEnqueue := int64(1) 810 if writeToEnqueue.write.writeBatch != nil { 811 numToEnqueue = int64(len(writeToEnqueue.write.writeBatch.Iter())) 812 } 813 814 // Optimistically increment the number of enqueued writes. 815 numEnqueued := atomic.AddInt64(&l.numWritesInQueue, numToEnqueue) 816 817 // If we exceeded the limit, decrement the number of enqueued writes and bail. 818 if numEnqueued > l.maxQueueSize { 819 atomic.AddInt64(&l.numWritesInQueue, -numToEnqueue) 820 l.closedState.RUnlock() 821 822 if write.writeBatch != nil { 823 // Make sure to finalize the write batch even though we didn't accept the writes 824 // so it can be returned to the pool. 825 write.writeBatch.Finalize() 826 } 827 828 return ErrCommitLogQueueFull 829 } 830 831 // Otherwise submit the write. 832 l.writes <- commitLogWrite{ 833 write: write, 834 callbackFn: completion, 835 } 836 837 l.closedState.RUnlock() 838 839 wg.Wait() 840 841 return result 842 } 843 844 func (l *commitLog) writeBehind( 845 ctx context.Context, 846 write writeOrWriteBatch, 847 ) error { 848 l.closedState.RLock() 849 if l.closedState.closed { 850 l.closedState.RUnlock() 851 return errCommitLogClosed 852 } 853 854 numToEnqueue := int64(1) 855 if write.writeBatch != nil { 856 numToEnqueue = int64(len(write.writeBatch.Iter())) 857 } 858 859 // Optimistically increment the number of enqueued writes. 860 numEnqueued := atomic.AddInt64(&l.numWritesInQueue, numToEnqueue) 861 862 // If we exceeded the limit, decrement the number of enqueued writes and bail. 863 if numEnqueued > l.maxQueueSize { 864 atomic.AddInt64(&l.numWritesInQueue, -numToEnqueue) 865 l.closedState.RUnlock() 866 867 if write.writeBatch != nil { 868 // Make sure to finalize the write batch even though we didn't accept the writes 869 // so it can be returned to the pool. 870 write.writeBatch.Finalize() 871 } 872 873 return ErrCommitLogQueueFull 874 } 875 876 // Otherwise submit the write. 877 l.writes <- commitLogWrite{ 878 write: write, 879 } 880 881 l.closedState.RUnlock() 882 883 return nil 884 } 885 886 func (l *commitLog) Close() error { 887 l.closedState.Lock() 888 if l.closedState.closed { 889 l.closedState.Unlock() 890 return nil 891 } 892 893 l.closedState.closed = true 894 close(l.writes) 895 l.closedState.Unlock() 896 897 // Receive the result of closing the writer from asynchronous writer 898 return <-l.closeErr 899 } 900 901 func (l *commitLog) handleWriteErr(err error) { 902 l.metrics.errors.Inc(1) 903 l.log.Error("failed to write to commit log", zap.Error(err)) 904 905 if l.commitLogFailFn != nil { 906 l.commitLogFailFn(err) 907 } 908 }