github.com/petermattis/pebble@v0.0.0-20190905164901-ab51a2166067/internal/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 "encoding/binary" 9 "errors" 10 "io" 11 "sync" 12 "sync/atomic" 13 14 "github.com/petermattis/pebble/internal/crc" 15 ) 16 17 type block struct { 18 // buf[:written] has already been filled with fragments. Updated atomically. 19 written int32 20 // buf[:flushed] has already been flushed to w. 21 flushed int32 22 buf [blockSize]byte 23 } 24 25 type flusher interface { 26 Flush() error 27 } 28 29 type syncer interface { 30 Sync() error 31 } 32 33 const ( 34 syncConcurrencyBits = 9 35 36 // SyncConcurrency is the maximum number of concurrent sync operations that 37 // can be performed. Exported as this value also limits the commit 38 // concurrency in commitPipeline. 39 SyncConcurrency = 1 << syncConcurrencyBits 40 ) 41 42 // syncQueue is a lock-free fixed-size single-producer, single-consumer 43 // queue. The single-producer can push to the head, and the single-consumer can 44 // pop multiple values from the tail. Popping calls Done() on each of the 45 // available *sync.WaitGroup elements. 46 type syncQueue struct { 47 // headTail packs together a 32-bit head index and a 32-bit tail index. Both 48 // are indexes into slots modulo len(slots)-1. 49 // 50 // tail = index of oldest data in queue 51 // head = index of next slot to fill 52 // 53 // Slots in the range [tail, head) are owned by consumers. A consumer 54 // continues to own a slot outside this range until it nils the slot, at 55 // which point ownership passes to the producer. 56 // 57 // The head index is stored in the most-significant bits so that we can 58 // atomically add to it and the overflow is harmless. 59 headTail uint64 60 61 // slots is a ring buffer of values stored in this queue. The size must be a 62 // power of 2. A slot is in use until the tail index has moved beyond it. 63 slots [SyncConcurrency]*sync.WaitGroup 64 } 65 66 const dequeueBits = 32 67 68 func (q *syncQueue) unpack(ptrs uint64) (head, tail uint32) { 69 const mask = 1<<dequeueBits - 1 70 head = uint32((ptrs >> dequeueBits) & mask) 71 tail = uint32(ptrs & mask) 72 return 73 } 74 75 func (q *syncQueue) push(wg *sync.WaitGroup) { 76 ptrs := atomic.LoadUint64(&q.headTail) 77 head, tail := q.unpack(ptrs) 78 if (tail+uint32(len(q.slots)))&(1<<dequeueBits-1) == head { 79 panic("queue is full") 80 } 81 82 slot := &q.slots[head&uint32(len(q.slots)-1)] 83 *slot = wg 84 85 // Increment head. This passes ownership of slot to dequeue and acts as a 86 // store barrier for writing the slot. 87 atomic.AddUint64(&q.headTail, 1<<dequeueBits) 88 } 89 90 func (q *syncQueue) empty() bool { 91 head, tail := q.load() 92 return head == tail 93 } 94 95 func (q *syncQueue) load() (head, tail uint32) { 96 ptrs := atomic.LoadUint64(&q.headTail) 97 head, tail = q.unpack(ptrs) 98 return head, tail 99 } 100 101 func (q *syncQueue) pop(head, tail uint32) { 102 if tail == head { 103 // Queue is empty. 104 return 105 } 106 107 for ; tail != head; tail++ { 108 slot := &q.slots[tail&uint32(len(q.slots)-1)] 109 wg := *slot 110 if wg == nil { 111 panic("nil waiter") 112 } 113 *slot = nil 114 // We need to bump the tail count before signalling the wait group as 115 // signalling the wait group can trigger release a blocked goroutine which 116 // will try to enqueue before we've "freed" space in the queue. 117 atomic.AddUint64(&q.headTail, 1) 118 wg.Done() 119 } 120 } 121 122 // flusherCond is a specialized condition variable that is safe to signal for 123 // readiness without holding the associated mutex in some circumstances. In 124 // particular, when waiter is added to syncQueue, this condition variable can 125 // be signalled without holding flusher.Mutex. 126 type flusherCond struct { 127 mu *sync.Mutex 128 q *syncQueue 129 cond sync.Cond 130 } 131 132 func (c *flusherCond) init(mu *sync.Mutex, q *syncQueue) { 133 c.mu = mu 134 c.q = q 135 // Yes, this is a bit circular, but that is intentional. flusherCond.cond.L 136 // points flusherCond so that when cond.L.Unlock is called flusherCond.Unlock 137 // will be called and we can check the !syncQueue.empty() condition. 138 c.cond.L = c 139 } 140 141 func (c *flusherCond) Signal() { 142 // Pass-through to the cond var. 143 c.cond.Signal() 144 } 145 146 func (c *flusherCond) Wait() { 147 // Pass-through to the cond var. Note that internally the cond var implements 148 // Wait as: 149 // 150 // t := notifyListAdd() 151 // L.Unlock() 152 // notifyListWait(t) 153 // L.Lock() 154 // 155 // We've configured the cond var to call flusherReady.Unlock() which allows 156 // us to check the !syncQueue.empty() condition without a danger of missing a 157 // notification. Any call to flusherReady.Signal() after notifyListAdd() is 158 // called will cause the subsequent notifyListWait() to return immediately. 159 c.cond.Wait() 160 } 161 162 func (c *flusherCond) Lock() { 163 c.mu.Lock() 164 } 165 166 func (c *flusherCond) Unlock() { 167 c.mu.Unlock() 168 if !c.q.empty() { 169 // If the current goroutine is about to block on sync.Cond.Wait, this call 170 // to Signal will prevent that. The comment in Wait above explains a bit 171 // about what is going on here, but it is worth reiterating: 172 // 173 // flusherCond.Wait() 174 // sync.Cond.Wait() 175 // t := notifyListAdd() 176 // flusherCond.Unlock() <-- we are here 177 // notifyListWait(t) 178 // flusherCond.Lock() 179 // 180 // The call to Signal here results in: 181 // 182 // sync.Cond.Signal() 183 // notifyListNotifyOne() 184 // 185 // The call to notifyListNotifyOne() will prevent the call to 186 // notifyListWait(t) from blocking. 187 c.cond.Signal() 188 } 189 } 190 191 // LogWriter writes records to an underlying io.Writer. In order to support WAL 192 // file reuse, a LogWriter's records are tagged with the WAL's file 193 // number. When reading a log file a record from a previous incarnation of the 194 // file will return the error ErrInvalidLogNum. 195 type LogWriter struct { 196 // w is the underlying writer. 197 w io.Writer 198 // c is w as a closer. 199 c io.Closer 200 // f is w as a flusher. 201 f flusher 202 // s is w as a syncer. 203 s syncer 204 // logNum is the low 32-bits of the log's file number. 205 logNum uint32 206 // blockNum is the zero based block number for the current block. 207 blockNum int64 208 // err is any accumulated error. TODO(peter): This needs to be protected in 209 // some fashion. Perhaps using atomic.Value. 210 err error 211 // block is the current block being written. Protected by flusher.Mutex. 212 block *block 213 free chan *block 214 215 flusher struct { 216 sync.Mutex 217 // Flusher ready is a condition variable that is signalled when there are 218 // blocks to flush, syncing has been requested, or the LogWriter has been 219 // closed. For signalling of a sync, it is safe to call without holding 220 // flusher.Mutex. 221 ready flusherCond 222 // Has the writer been closed? 223 closed bool 224 // Accumulated flush error. 225 err error 226 pending []*block 227 syncQ syncQueue 228 } 229 } 230 231 // NewLogWriter returns a new LogWriter. 232 func NewLogWriter(w io.Writer, logNum uint64) *LogWriter { 233 c, _ := w.(io.Closer) 234 f, _ := w.(flusher) 235 s, _ := w.(syncer) 236 r := &LogWriter{ 237 w: w, 238 c: c, 239 f: f, 240 s: s, 241 // NB: we truncate the 64-bit log number to 32-bits. This is ok because a) 242 // we are very unlikely to reach a file number of 4 billion and b) the log 243 // number is used as a validation check and using only the low 32-bits is 244 // sufficient for that purpose. 245 logNum: uint32(logNum), 246 free: make(chan *block, 4), 247 } 248 for i := 0; i < cap(r.free); i++ { 249 r.free <- &block{} 250 } 251 r.block = <-r.free 252 r.flusher.ready.init(&r.flusher.Mutex, &r.flusher.syncQ) 253 go r.flushLoop() 254 return r 255 } 256 257 func (w *LogWriter) flushLoop() { 258 f := &w.flusher 259 f.Lock() 260 defer f.Unlock() 261 262 for { 263 var data []byte 264 for { 265 if f.closed { 266 return 267 } 268 // Grab the portion of the current block that requires flushing. Note that 269 // the current block can be added to the pending blocks list after we release 270 // the flusher lock, but it won't be part of pending. 271 written := atomic.LoadInt32(&w.block.written) 272 data = w.block.buf[w.block.flushed:written] 273 w.block.flushed = written 274 if len(f.pending) > 0 || len(data) > 0 || !f.syncQ.empty() { 275 break 276 } 277 f.ready.Wait() 278 continue 279 } 280 281 pending := f.pending 282 f.pending = f.pending[len(f.pending):] 283 head, tail := f.syncQ.load() 284 285 f.Unlock() 286 287 var err error 288 for _, b := range pending { 289 if err = w.flushBlock(b); err != nil { 290 break 291 } 292 } 293 if err == nil && len(data) > 0 { 294 _, err = w.w.Write(data) 295 } 296 if err == nil && head != tail { 297 if w.s != nil { 298 err = w.s.Sync() 299 } 300 if err == nil { 301 f.syncQ.pop(head, tail) 302 } 303 } 304 305 f.Lock() 306 f.err = err 307 if f.err != nil { 308 return 309 } 310 } 311 } 312 313 func (w *LogWriter) flushBlock(b *block) error { 314 if _, err := w.w.Write(b.buf[b.flushed:]); err != nil { 315 return err 316 } 317 b.written = 0 318 b.flushed = 0 319 w.free <- b 320 return nil 321 } 322 323 // queueBlock queues the current block for writing to the underlying writer, 324 // allocates a new block and reserves space for the next header. 325 func (w *LogWriter) queueBlock() { 326 // Allocate a new block, blocking until one is available. We do this first 327 // because w.block is protected by w.flusher.Mutex. 328 nextBlock := <-w.free 329 330 f := &w.flusher 331 f.Lock() 332 f.pending = append(f.pending, w.block) 333 w.block = nextBlock 334 f.ready.Signal() 335 w.err = w.flusher.err 336 f.Unlock() 337 338 w.blockNum++ 339 } 340 341 // Close flushes and syncs any unwritten data and closes the writer. 342 func (w *LogWriter) Close() error { 343 f := &w.flusher 344 345 // Force a sync of any unwritten data. 346 wg := &sync.WaitGroup{} 347 wg.Add(1) 348 f.syncQ.push(wg) 349 f.ready.Signal() 350 wg.Wait() 351 352 f.Lock() 353 f.closed = true 354 f.ready.Signal() 355 f.Unlock() 356 357 if w.c != nil { 358 if err := w.c.Close(); err != nil { 359 return err 360 } 361 } 362 w.err = errors.New("pebble/record: closed LogWriter") 363 return nil 364 } 365 366 // WriteRecord writes a complete record. Returns the offset just past the end 367 // of the record. 368 func (w *LogWriter) WriteRecord(p []byte) (int64, error) { 369 return w.SyncRecord(p, nil) 370 } 371 372 // SyncRecord writes a complete record. If wg!= nil the record will be 373 // asynchronously persisted to the underlying writer and done will be called on 374 // the wait group upon completion. Returns the offset just past the end of the 375 // record. 376 func (w *LogWriter) SyncRecord(p []byte, wg *sync.WaitGroup) (int64, error) { 377 if w.err != nil { 378 return -1, w.err 379 } 380 381 for i := 0; i == 0 || len(p) > 0; i++ { 382 p = w.emitFragment(i, p) 383 } 384 385 if wg != nil { 386 f := &w.flusher 387 f.syncQ.push(wg) 388 f.ready.Signal() 389 } 390 391 offset := w.blockNum*blockSize + int64(w.block.written) 392 return offset, w.err 393 } 394 395 // Size returns the current size of the file. 396 func (w *LogWriter) Size() int64 { 397 return w.blockNum*blockSize + int64(w.block.written) 398 } 399 400 func (w *LogWriter) emitFragment(n int, p []byte) []byte { 401 b := w.block 402 i := b.written 403 first := n == 0 404 last := blockSize-i-recyclableHeaderSize >= int32(len(p)) 405 406 if last { 407 if first { 408 b.buf[i+6] = recyclableFullChunkType 409 } else { 410 b.buf[i+6] = recyclableLastChunkType 411 } 412 } else { 413 if first { 414 b.buf[i+6] = recyclableFirstChunkType 415 } else { 416 b.buf[i+6] = recyclableMiddleChunkType 417 } 418 } 419 420 binary.LittleEndian.PutUint32(b.buf[i+7:i+11], w.logNum) 421 422 r := copy(b.buf[i+recyclableHeaderSize:], p) 423 j := i + int32(recyclableHeaderSize+r) 424 binary.LittleEndian.PutUint32(b.buf[i+0:i+4], crc.New(b.buf[i+6:j]).Value()) 425 binary.LittleEndian.PutUint16(b.buf[i+4:i+6], uint16(r)) 426 atomic.StoreInt32(&b.written, j) 427 428 if blockSize-b.written < recyclableHeaderSize { 429 // There is no room for another fragment in the block, so fill the 430 // remaining bytes with zeros and queue the block for flushing. 431 for i := b.written; i < blockSize; i++ { 432 b.buf[i] = 0 433 } 434 atomic.StoreInt32(&b.written, j) 435 w.queueBlock() 436 } 437 return p[r:] 438 }