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