github.com/zuoyebang/bitalostable@v1.0.1-0.20240229032404-e3b99a834294/record/log_writer.go (about) 1 // Copyright 2018 The LevelDB-Go and Pebble Authors. All rights reserved. Use 2 // of this source code is governed by a BSD-style license that can be found in 3 // the LICENSE file. 4 5 package record 6 7 import ( 8 "context" 9 "encoding/binary" 10 "io" 11 "runtime/pprof" 12 "sync" 13 "sync/atomic" 14 "time" 15 16 "github.com/HdrHistogram/hdrhistogram-go" 17 "github.com/cockroachdb/errors" 18 "github.com/zuoyebang/bitalostable/internal/base" 19 "github.com/zuoyebang/bitalostable/internal/crc" 20 ) 21 22 var walSyncLabels = pprof.Labels("bitalostable", "wal-sync") 23 24 type block struct { 25 // buf[:written] has already been filled with fragments. Updated atomically. 26 written int32 27 // buf[:flushed] has already been flushed to w. 28 flushed int32 29 buf [blockSize]byte 30 } 31 32 type flusher interface { 33 Flush() error 34 } 35 36 type syncer interface { 37 Sync() error 38 } 39 40 const ( 41 syncConcurrencyBits = 9 42 43 // SyncConcurrency is the maximum number of concurrent sync operations that 44 // can be performed. Note that a sync operation is initiated either by a call 45 // to SyncRecord or by a call to Close. Exported as this value also limits 46 // the commit concurrency in commitPipeline. 47 SyncConcurrency = 1 << syncConcurrencyBits 48 ) 49 50 type syncSlot struct { 51 wg *sync.WaitGroup 52 err *error 53 } 54 55 // syncQueue is a lock-free fixed-size single-producer, single-consumer 56 // queue. The single-producer can push to the head, and the single-consumer can 57 // pop multiple values from the tail. Popping calls Done() on each of the 58 // available *sync.WaitGroup elements. 59 type syncQueue struct { 60 // headTail packs together a 32-bit head index and a 32-bit tail index. Both 61 // are indexes into slots modulo len(slots)-1. 62 // 63 // tail = index of oldest data in queue 64 // head = index of next slot to fill 65 // 66 // Slots in the range [tail, head) are owned by consumers. A consumer 67 // continues to own a slot outside this range until it nils the slot, at 68 // which point ownership passes to the producer. 69 // 70 // The head index is stored in the most-significant bits so that we can 71 // atomically add to it and the overflow is harmless. 72 headTail uint64 73 74 // slots is a ring buffer of values stored in this queue. The size must be a 75 // power of 2. A slot is in use until the tail index has moved beyond it. 76 slots [SyncConcurrency]syncSlot 77 78 // blocked is an atomic boolean which indicates whether syncing is currently 79 // blocked or can proceed. It is used by the implementation of 80 // min-sync-interval to block syncing until the min interval has passed. 81 blocked uint32 82 } 83 84 const dequeueBits = 32 85 86 func (q *syncQueue) unpack(ptrs uint64) (head, tail uint32) { 87 const mask = 1<<dequeueBits - 1 88 head = uint32((ptrs >> dequeueBits) & mask) 89 tail = uint32(ptrs & mask) 90 return 91 } 92 93 func (q *syncQueue) push(wg *sync.WaitGroup, err *error) { 94 ptrs := atomic.LoadUint64(&q.headTail) 95 head, tail := q.unpack(ptrs) 96 if (tail+uint32(len(q.slots)))&(1<<dequeueBits-1) == head { 97 panic("bitalostable: queue is full") 98 } 99 100 slot := &q.slots[head&uint32(len(q.slots)-1)] 101 slot.wg = wg 102 slot.err = err 103 104 // Increment head. This passes ownership of slot to dequeue and acts as a 105 // store barrier for writing the slot. 106 atomic.AddUint64(&q.headTail, 1<<dequeueBits) 107 } 108 109 func (q *syncQueue) setBlocked() { 110 atomic.StoreUint32(&q.blocked, 1) 111 } 112 113 func (q *syncQueue) clearBlocked() { 114 atomic.StoreUint32(&q.blocked, 0) 115 } 116 117 func (q *syncQueue) empty() bool { 118 head, tail, _ := q.load() 119 return head == tail 120 } 121 122 // load returns the head, tail of the queue for what should be synced to the 123 // caller. It can return a head, tail of zero if syncing is blocked due to 124 // min-sync-interval. It additionally returns the real length of this queue, 125 // regardless of whether syncing is blocked. 126 func (q *syncQueue) load() (head, tail, realLength uint32) { 127 ptrs := atomic.LoadUint64(&q.headTail) 128 head, tail = q.unpack(ptrs) 129 realLength = head - tail 130 if atomic.LoadUint32(&q.blocked) == 1 { 131 return 0, 0, realLength 132 } 133 return head, tail, realLength 134 } 135 136 func (q *syncQueue) pop(head, tail uint32, err error) error { 137 if tail == head { 138 // Queue is empty. 139 return nil 140 } 141 142 for ; tail != head; tail++ { 143 slot := &q.slots[tail&uint32(len(q.slots)-1)] 144 wg := slot.wg 145 if wg == nil { 146 return errors.Errorf("nil waiter at %d", errors.Safe(tail&uint32(len(q.slots)-1))) 147 } 148 *slot.err = err 149 slot.wg = nil 150 slot.err = nil 151 // We need to bump the tail count before signalling the wait group as 152 // signalling the wait group can trigger release a blocked goroutine which 153 // will try to enqueue before we've "freed" space in the queue. 154 atomic.AddUint64(&q.headTail, 1) 155 wg.Done() 156 } 157 158 return nil 159 } 160 161 // flusherCond is a specialized condition variable that allows its condition to 162 // change and readiness be signalled without holding its associated mutex. In 163 // particular, when a waiter is added to syncQueue atomically, this condition 164 // variable can be signalled without holding flusher.Mutex. 165 type flusherCond struct { 166 mu *sync.Mutex 167 q *syncQueue 168 cond sync.Cond 169 } 170 171 func (c *flusherCond) init(mu *sync.Mutex, q *syncQueue) { 172 c.mu = mu 173 c.q = q 174 // Yes, this is a bit circular, but that is intentional. flusherCond.cond.L 175 // points flusherCond so that when cond.L.Unlock is called flusherCond.Unlock 176 // will be called and we can check the !syncQueue.empty() condition. 177 c.cond.L = c 178 } 179 180 func (c *flusherCond) Signal() { 181 // Pass-through to the cond var. 182 c.cond.Signal() 183 } 184 185 func (c *flusherCond) Wait() { 186 // Pass-through to the cond var. Note that internally the cond var implements 187 // Wait as: 188 // 189 // t := notifyListAdd() 190 // L.Unlock() 191 // notifyListWait(t) 192 // L.Lock() 193 // 194 // We've configured the cond var to call flusherReady.Unlock() which allows 195 // us to check the !syncQueue.empty() condition without a danger of missing a 196 // notification. Any call to flusherReady.Signal() after notifyListAdd() is 197 // called will cause the subsequent notifyListWait() to return immediately. 198 c.cond.Wait() 199 } 200 201 func (c *flusherCond) Lock() { 202 c.mu.Lock() 203 } 204 205 func (c *flusherCond) Unlock() { 206 c.mu.Unlock() 207 if !c.q.empty() { 208 // If the current goroutine is about to block on sync.Cond.Wait, this call 209 // to Signal will prevent that. The comment in Wait above explains a bit 210 // about what is going on here, but it is worth reiterating: 211 // 212 // flusherCond.Wait() 213 // sync.Cond.Wait() 214 // t := notifyListAdd() 215 // flusherCond.Unlock() <-- we are here 216 // notifyListWait(t) 217 // flusherCond.Lock() 218 // 219 // The call to Signal here results in: 220 // 221 // sync.Cond.Signal() 222 // notifyListNotifyOne() 223 // 224 // The call to notifyListNotifyOne() will prevent the call to 225 // notifyListWait(t) from blocking. 226 c.cond.Signal() 227 } 228 } 229 230 type durationFunc func() time.Duration 231 232 // syncTimer is an interface for timers, modeled on the closure callback mode 233 // of time.Timer. See time.AfterFunc and LogWriter.afterFunc. syncTimer is used 234 // by tests to mock out the timer functionality used to implement 235 // min-sync-interval. 236 type syncTimer interface { 237 Reset(time.Duration) bool 238 Stop() bool 239 } 240 241 // LogWriter writes records to an underlying io.Writer. In order to support WAL 242 // file reuse, a LogWriter's records are tagged with the WAL's file 243 // number. When reading a log file a record from a previous incarnation of the 244 // file will return the error ErrInvalidLogNum. 245 type LogWriter struct { 246 // w is the underlying writer. 247 w io.Writer 248 // c is w as a closer. 249 c io.Closer 250 // s is w as a syncer. 251 s syncer 252 // logNum is the low 32-bits of the log's file number. 253 logNum uint32 254 // blockNum is the zero based block number for the current block. 255 blockNum int64 256 // err is any accumulated error. TODO(peter): This needs to be protected in 257 // some fashion. Perhaps using atomic.Value. 258 err error 259 // block is the current block being written. Protected by flusher.Mutex. 260 block *block 261 free struct { 262 sync.Mutex 263 // Condition variable used to signal a block is freed. 264 cond sync.Cond 265 blocks []*block 266 allocated int 267 } 268 269 flusher struct { 270 sync.Mutex 271 // Flusher ready is a condition variable that is signalled when there are 272 // blocks to flush, syncing has been requested, or the LogWriter has been 273 // closed. For signalling of a sync, it is safe to call without holding 274 // flusher.Mutex. 275 ready flusherCond 276 // Set to true when the flush loop should be closed. 277 close bool 278 // Closed when the flush loop has terminated. 279 closed chan struct{} 280 // Accumulated flush error. 281 err error 282 // minSyncInterval is the minimum duration between syncs. 283 minSyncInterval durationFunc 284 pending []*block 285 syncQ syncQueue 286 metrics *LogWriterMetrics 287 } 288 289 // afterFunc is a hook to allow tests to mock out the timer functionality 290 // used for min-sync-interval. In normal operation this points to 291 // time.AfterFunc. 292 afterFunc func(d time.Duration, f func()) syncTimer 293 } 294 295 // CapAllocatedBlocks is the maximum number of blocks allocated by the 296 // LogWriter. 297 const CapAllocatedBlocks = 16 298 299 // NewLogWriter returns a new LogWriter. 300 func NewLogWriter(w io.Writer, logNum base.FileNum) *LogWriter { 301 c, _ := w.(io.Closer) 302 s, _ := w.(syncer) 303 r := &LogWriter{ 304 w: w, 305 c: c, 306 s: s, 307 // NB: we truncate the 64-bit log number to 32-bits. This is ok because a) 308 // we are very unlikely to reach a file number of 4 billion and b) the log 309 // number is used as a validation check and using only the low 32-bits is 310 // sufficient for that purpose. 311 logNum: uint32(logNum), 312 afterFunc: func(d time.Duration, f func()) syncTimer { 313 return time.AfterFunc(d, f) 314 }, 315 } 316 r.free.cond.L = &r.free.Mutex 317 r.free.blocks = make([]*block, 0, CapAllocatedBlocks) 318 r.free.allocated = 1 319 r.block = &block{} 320 r.flusher.ready.init(&r.flusher.Mutex, &r.flusher.syncQ) 321 r.flusher.closed = make(chan struct{}) 322 r.flusher.pending = make([]*block, 0, cap(r.free.blocks)) 323 r.flusher.metrics = &LogWriterMetrics{} 324 // Histogram with max value of 30s. We are not trying to detect anomalies 325 // with this, and normally latencies range from 0.5ms to 25ms. 326 r.flusher.metrics.SyncLatencyMicros = hdrhistogram.New( 327 0, (time.Second * 30).Microseconds(), 2) 328 go func() { 329 pprof.Do(context.Background(), walSyncLabels, r.flushLoop) 330 }() 331 return r 332 } 333 334 // SetMinSyncInterval sets the closure to invoke for retrieving the minimum 335 // sync duration between syncs. 336 func (w *LogWriter) SetMinSyncInterval(minSyncInterval durationFunc) { 337 f := &w.flusher 338 f.Lock() 339 f.minSyncInterval = minSyncInterval 340 f.Unlock() 341 } 342 343 func (w *LogWriter) flushLoop(context.Context) { 344 f := &w.flusher 345 f.Lock() 346 347 // Initialize idleStartTime to when the loop starts. 348 idleStartTime := time.Now() 349 var syncTimer syncTimer 350 defer func() { 351 // Capture the idle duration between the last piece of work and when the 352 // loop terminated. 353 f.metrics.WriteThroughput.IdleDuration += time.Since(idleStartTime) 354 if syncTimer != nil { 355 syncTimer.Stop() 356 } 357 close(f.closed) 358 f.Unlock() 359 }() 360 361 // The flush loop performs flushing of full and partial data blocks to the 362 // underlying writer (LogWriter.w), syncing of the writer, and notification 363 // to sync requests that they have completed. 364 // 365 // - flusher.ready is a condition variable that is signalled when there is 366 // work to do. Full blocks are contained in flusher.pending. The current 367 // partial block is in LogWriter.block. And sync operations are held in 368 // flusher.syncQ. 369 // 370 // - The decision to sync is determined by whether there are any sync 371 // requests present in flusher.syncQ and whether enough time has elapsed 372 // since the last sync. If not enough time has elapsed since the last sync, 373 // flusher.syncQ.blocked will be set to 1. If syncing is blocked, 374 // syncQueue.empty() will return true and syncQueue.load() will return 0,0 375 // (i.e. an empty list). 376 // 377 // - flusher.syncQ.blocked is cleared by a timer that is initialized when 378 // blocked is set to 1. When blocked is 1, no syncing will take place, but 379 // flushing will continue to be performed. The on/off toggle for syncing 380 // does not need to be carefully synchronized with the rest of processing 381 // -- all we need to ensure is that after any transition to blocked=1 there 382 // is eventually a transition to blocked=0. syncTimer performs this 383 // transition. Note that any change to min-sync-interval will not take 384 // effect until the previous timer elapses. 385 // 386 // - Picking up the syncing work to perform requires coordination with 387 // picking up the flushing work. Specifically, flushing work is queued 388 // before syncing work. The guarantee of this code is that when a sync is 389 // requested, any previously queued flush work will be synced. This 390 // motivates reading the syncing work (f.syncQ.load()) before picking up 391 // the flush work (atomic.LoadInt32(&w.block.written)). 392 393 // The list of full blocks that need to be written. This is copied from 394 // f.pending on every loop iteration, though the number of elements is small 395 // (usually 1, max 16). 396 pending := make([]*block, 0, cap(f.pending)) 397 for { 398 for { 399 // Grab the portion of the current block that requires flushing. Note that 400 // the current block can be added to the pending blocks list after we release 401 // the flusher lock, but it won't be part of pending. 402 written := atomic.LoadInt32(&w.block.written) 403 if len(f.pending) > 0 || written > w.block.flushed || !f.syncQ.empty() { 404 break 405 } 406 if f.close { 407 // If the writer is closed, pretend the sync timer fired immediately so 408 // that we can process any queued sync requests. 409 f.syncQ.clearBlocked() 410 if !f.syncQ.empty() { 411 break 412 } 413 return 414 } 415 f.ready.Wait() 416 continue 417 } 418 // Found work to do, so no longer idle. 419 workStartTime := time.Now() 420 idleDuration := workStartTime.Sub(idleStartTime) 421 pending = pending[:len(f.pending)] 422 copy(pending, f.pending) 423 f.pending = f.pending[:0] 424 f.metrics.PendingBufferLen.AddSample(int64(len(pending))) 425 426 // Grab the list of sync waiters. Note that syncQueue.load() will return 427 // 0,0 while we're waiting for the min-sync-interval to expire. This 428 // allows flushing to proceed even if we're not ready to sync. 429 head, tail, realSyncQLen := f.syncQ.load() 430 f.metrics.SyncQueueLen.AddSample(int64(realSyncQLen)) 431 432 // Grab the portion of the current block that requires flushing. Note that 433 // the current block can be added to the pending blocks list after we 434 // release the flusher lock, but it won't be part of pending. This has to 435 // be ordered after we get the list of sync waiters from syncQ in order to 436 // prevent a race where a waiter adds itself to syncQ, but this thread 437 // picks up the entry in syncQ and not the buffered data. 438 written := atomic.LoadInt32(&w.block.written) 439 data := w.block.buf[w.block.flushed:written] 440 w.block.flushed = written 441 442 // If flusher has an error, we propagate it to waiters. Note in spite of 443 // error we consume the pending list above to free blocks for writers. 444 if f.err != nil { 445 f.syncQ.pop(head, tail, f.err) 446 // Update the idleStartTime if work could not be done, so that we don't 447 // include the duration we tried to do work as idle. We don't bother 448 // with the rest of the accounting, which means we will undercount. 449 idleStartTime = time.Now() 450 continue 451 } 452 f.Unlock() 453 synced, syncLatency, bytesWritten, err := w.flushPending(data, pending, head, tail) 454 f.Lock() 455 if synced { 456 f.metrics.SyncLatencyMicros.RecordValue(syncLatency.Microseconds()) 457 } 458 f.err = err 459 if f.err != nil { 460 f.syncQ.clearBlocked() 461 // Update the idleStartTime if work could not be done, so that we don't 462 // include the duration we tried to do work as idle. We don't bother 463 // with the rest of the accounting, which means we will undercount. 464 idleStartTime = time.Now() 465 continue 466 } 467 468 if synced && f.minSyncInterval != nil { 469 // A sync was performed. Make sure we've waited for the min sync 470 // interval before syncing again. 471 if min := f.minSyncInterval(); min > 0 { 472 f.syncQ.setBlocked() 473 if syncTimer == nil { 474 syncTimer = w.afterFunc(min, func() { 475 f.syncQ.clearBlocked() 476 f.ready.Signal() 477 }) 478 } else { 479 syncTimer.Reset(min) 480 } 481 } 482 } 483 // Finished work, and started idling. 484 idleStartTime = time.Now() 485 workDuration := idleStartTime.Sub(workStartTime) 486 f.metrics.WriteThroughput.Bytes += bytesWritten 487 f.metrics.WriteThroughput.WorkDuration += workDuration 488 f.metrics.WriteThroughput.IdleDuration += idleDuration 489 } 490 } 491 492 func (w *LogWriter) flushPending( 493 data []byte, pending []*block, head, tail uint32, 494 ) (synced bool, syncLatency time.Duration, bytesWritten int64, err error) { 495 defer func() { 496 // Translate panics into errors. The errors will cause flushLoop to shut 497 // down, but allows us to do so in a controlled way and avoid swallowing 498 // the stack that created the panic if panic'ing itself hits a panic 499 // (e.g. unlock of unlocked mutex). 500 if r := recover(); r != nil { 501 err = errors.Newf("%v", r) 502 } 503 }() 504 505 for _, b := range pending { 506 bytesWritten += blockSize - int64(b.flushed) 507 if err = w.flushBlock(b); err != nil { 508 break 509 } 510 } 511 if n := len(data); err == nil && n > 0 { 512 bytesWritten += int64(n) 513 _, err = w.w.Write(data) 514 } 515 516 synced = head != tail 517 if synced { 518 if err == nil && w.s != nil { 519 syncLatency, err = w.syncWithLatency() 520 } 521 f := &w.flusher 522 if popErr := f.syncQ.pop(head, tail, err); popErr != nil { 523 return synced, syncLatency, bytesWritten, popErr 524 } 525 } 526 527 return synced, syncLatency, bytesWritten, err 528 } 529 530 func (w *LogWriter) syncWithLatency() (time.Duration, error) { 531 start := time.Now() 532 err := w.s.Sync() 533 syncLatency := time.Since(start) 534 return syncLatency, err 535 } 536 537 func (w *LogWriter) flushBlock(b *block) error { 538 if _, err := w.w.Write(b.buf[b.flushed:]); err != nil { 539 return err 540 } 541 b.written = 0 542 b.flushed = 0 543 w.free.Lock() 544 w.free.blocks = append(w.free.blocks, b) 545 w.free.cond.Signal() 546 w.free.Unlock() 547 return nil 548 } 549 550 // queueBlock queues the current block for writing to the underlying writer, 551 // allocates a new block and reserves space for the next header. 552 func (w *LogWriter) queueBlock() { 553 // Allocate a new block, blocking until one is available. We do this first 554 // because w.block is protected by w.flusher.Mutex. 555 w.free.Lock() 556 if len(w.free.blocks) == 0 { 557 if w.free.allocated < cap(w.free.blocks) { 558 w.free.allocated++ 559 w.free.blocks = append(w.free.blocks, &block{}) 560 } else { 561 for len(w.free.blocks) == 0 { 562 w.free.cond.Wait() 563 } 564 } 565 } 566 nextBlock := w.free.blocks[len(w.free.blocks)-1] 567 w.free.blocks = w.free.blocks[:len(w.free.blocks)-1] 568 w.free.Unlock() 569 570 f := &w.flusher 571 f.Lock() 572 f.pending = append(f.pending, w.block) 573 w.block = nextBlock 574 f.ready.Signal() 575 w.err = w.flusher.err 576 f.Unlock() 577 578 w.blockNum++ 579 } 580 581 // Close flushes and syncs any unwritten data and closes the writer. 582 // Where required, external synchronisation is provided by commitPipeline.mu. 583 func (w *LogWriter) Close() error { 584 f := &w.flusher 585 586 // Emit an EOF trailer signifying the end of this log. This helps readers 587 // differentiate between a corrupted entry in the middle of a log from 588 // garbage at the tail from a recycled log file. 589 w.emitEOFTrailer() 590 591 // Signal the flush loop to close. 592 f.Lock() 593 f.close = true 594 f.ready.Signal() 595 f.Unlock() 596 597 // Wait for the flush loop to close. The flush loop will not close until all 598 // pending data has been written or an error occurs. 599 <-f.closed 600 601 // Sync any flushed data to disk. NB: flushLoop will sync after flushing the 602 // last buffered data only if it was requested via syncQ, so we need to sync 603 // here to ensure that all the data is synced. 604 err := w.flusher.err 605 var syncLatency time.Duration 606 if err == nil && w.s != nil { 607 syncLatency, err = w.syncWithLatency() 608 } 609 f.Lock() 610 f.metrics.SyncLatencyMicros.RecordValue(syncLatency.Microseconds()) 611 f.Unlock() 612 613 if w.c != nil { 614 cerr := w.c.Close() 615 w.c = nil 616 if cerr != nil { 617 return cerr 618 } 619 } 620 w.err = errors.New("bitalostable/record: closed LogWriter") 621 return err 622 } 623 624 // WriteRecord writes a complete record. Returns the offset just past the end 625 // of the record. 626 // External synchronisation provided by commitPipeline.mu. 627 func (w *LogWriter) WriteRecord(p []byte) (int64, error) { 628 return w.SyncRecord(p, nil, nil) 629 } 630 631 // SyncRecord writes a complete record. If wg!= nil the record will be 632 // asynchronously persisted to the underlying writer and done will be called on 633 // the wait group upon completion. Returns the offset just past the end of the 634 // record. 635 // External synchronisation provided by commitPipeline.mu. 636 func (w *LogWriter) SyncRecord(p []byte, wg *sync.WaitGroup, err *error) (int64, error) { 637 if w.err != nil { 638 return -1, w.err 639 } 640 641 // The `i == 0` condition ensures we handle empty records. Such records can 642 // possibly be generated for VersionEdits stored in the MANIFEST. While the 643 // MANIFEST is currently written using Writer, it is good to support the same 644 // semantics with LogWriter. 645 for i := 0; i == 0 || len(p) > 0; i++ { 646 p = w.emitFragment(i, p) 647 } 648 649 if wg != nil { 650 // If we've been asked to persist the record, add the WaitGroup to the sync 651 // queue and signal the flushLoop. Note that flushLoop will write partial 652 // blocks to the file if syncing has been requested. The contract is that 653 // any record written to the LogWriter to this point will be flushed to the 654 // OS and synced to disk. 655 f := &w.flusher 656 f.syncQ.push(wg, err) 657 f.ready.Signal() 658 } 659 660 offset := w.blockNum*blockSize + int64(w.block.written) 661 // Note that we don't return w.err here as a concurrent call to Close would 662 // race with our read. That's ok because the only error we could be seeing is 663 // one to syncing for which the caller can receive notification of by passing 664 // in a non-nil err argument. 665 return offset, nil 666 } 667 668 // Size returns the current size of the file. 669 // External synchronisation provided by commitPipeline.mu. 670 func (w *LogWriter) Size() int64 { 671 return w.blockNum*blockSize + int64(w.block.written) 672 } 673 674 func (w *LogWriter) emitEOFTrailer() { 675 // Write a recyclable chunk header with a different log number. Readers 676 // will treat the header as EOF when the log number does not match. 677 b := w.block 678 i := b.written 679 binary.LittleEndian.PutUint32(b.buf[i+0:i+4], 0) // CRC 680 binary.LittleEndian.PutUint16(b.buf[i+4:i+6], 0) // Size 681 b.buf[i+6] = recyclableFullChunkType 682 binary.LittleEndian.PutUint32(b.buf[i+7:i+11], w.logNum+1) // Log number 683 atomic.StoreInt32(&b.written, i+int32(recyclableHeaderSize)) 684 } 685 686 func (w *LogWriter) emitFragment(n int, p []byte) []byte { 687 b := w.block 688 i := b.written 689 first := n == 0 690 last := blockSize-i-recyclableHeaderSize >= int32(len(p)) 691 692 if last { 693 if first { 694 b.buf[i+6] = recyclableFullChunkType 695 } else { 696 b.buf[i+6] = recyclableLastChunkType 697 } 698 } else { 699 if first { 700 b.buf[i+6] = recyclableFirstChunkType 701 } else { 702 b.buf[i+6] = recyclableMiddleChunkType 703 } 704 } 705 706 binary.LittleEndian.PutUint32(b.buf[i+7:i+11], w.logNum) 707 708 r := copy(b.buf[i+recyclableHeaderSize:], p) 709 j := i + int32(recyclableHeaderSize+r) 710 binary.LittleEndian.PutUint32(b.buf[i+0:i+4], crc.New(b.buf[i+6:j]).Value()) 711 binary.LittleEndian.PutUint16(b.buf[i+4:i+6], uint16(r)) 712 atomic.StoreInt32(&b.written, j) 713 714 if blockSize-b.written < recyclableHeaderSize { 715 // There is no room for another fragment in the block, so fill the 716 // remaining bytes with zeros and queue the block for flushing. 717 for i := b.written; i < blockSize; i++ { 718 b.buf[i] = 0 719 } 720 w.queueBlock() 721 } 722 return p[r:] 723 } 724 725 // Metrics must be called after Close. The callee will no longer modify the 726 // returned LogWriterMetrics. 727 func (w *LogWriter) Metrics() *LogWriterMetrics { 728 return w.flusher.metrics 729 } 730 731 // LogWriterMetrics contains misc metrics for the log writer. 732 type LogWriterMetrics struct { 733 WriteThroughput base.ThroughputMetric 734 PendingBufferLen base.GaugeSampleMetric 735 SyncQueueLen base.GaugeSampleMetric 736 SyncLatencyMicros *hdrhistogram.Histogram 737 } 738 739 // Merge merges metrics from x. Requires that x is non-nil. 740 func (m *LogWriterMetrics) Merge(x *LogWriterMetrics) error { 741 m.WriteThroughput.Merge(x.WriteThroughput) 742 m.PendingBufferLen.Merge(x.PendingBufferLen) 743 m.SyncQueueLen.Merge(x.SyncQueueLen) 744 dropped := m.SyncLatencyMicros.Merge(x.SyncLatencyMicros) 745 if dropped > 0 { 746 // This should never happen since we use a consistent min, max when 747 // creating these histograms, and out-of-range is the only reason for the 748 // merge to drop samples. 749 return errors.Errorf("sync latency histogram merge dropped %d samples", dropped) 750 } 751 return nil 752 }