github.com/zuoyebang/bitalostable@v1.0.1-0.20240229032404-e3b99a834294/commit.go (about) 1 // Copyright 2018 The LevelDB-Go and Pebble and Bitalostored 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 bitalostable 6 7 import ( 8 "runtime" 9 "sync" 10 "sync/atomic" 11 "unsafe" 12 13 "github.com/zuoyebang/bitalostable/record" 14 ) 15 16 // commitQueue is a lock-free fixed-size single-producer, multi-consumer 17 // queue. The single producer can enqueue (push) to the head, and consumers can 18 // dequeue (pop) from the tail. 19 // 20 // It has the added feature that it nils out unused slots to avoid unnecessary 21 // retention of objects. 22 type commitQueue struct { 23 // headTail packs together a 32-bit head index and a 32-bit tail index. Both 24 // are indexes into slots modulo len(slots)-1. 25 // 26 // tail = index of oldest data in queue 27 // head = index of next slot to fill 28 // 29 // Slots in the range [tail, head) are owned by consumers. A consumer 30 // continues to own a slot outside this range until it nils the slot, at 31 // which point ownership passes to the producer. 32 // 33 // The head index is stored in the most-significant bits so that we can 34 // atomically add to it and the overflow is harmless. 35 headTail uint64 36 37 // slots is a ring buffer of values stored in this queue. The size must be a 38 // power of 2. A slot is in use until *both* the tail index has moved beyond 39 // it and the slot value has been set to nil. The slot value is set to nil 40 // atomically by the consumer and read atomically by the producer. 41 slots [record.SyncConcurrency]unsafe.Pointer 42 } 43 44 const dequeueBits = 32 45 46 func (q *commitQueue) unpack(ptrs uint64) (head, tail uint32) { 47 const mask = 1<<dequeueBits - 1 48 head = uint32((ptrs >> dequeueBits) & mask) 49 tail = uint32(ptrs & mask) 50 return 51 } 52 53 func (q *commitQueue) pack(head, tail uint32) uint64 { 54 const mask = 1<<dequeueBits - 1 55 return (uint64(head) << dequeueBits) | 56 uint64(tail&mask) 57 } 58 59 func (q *commitQueue) enqueue(b *Batch) { 60 ptrs := atomic.LoadUint64(&q.headTail) 61 head, tail := q.unpack(ptrs) 62 if (tail+uint32(len(q.slots)))&(1<<dequeueBits-1) == head { 63 // Queue is full. This should never be reached because commitPipeline.sem 64 // limits the number of concurrent operations. 65 panic("bitalostable: not reached") 66 } 67 slot := &q.slots[head&uint32(len(q.slots)-1)] 68 69 // Check if the head slot has been released by dequeue. 70 for atomic.LoadPointer(slot) != nil { 71 // Another goroutine is still cleaning up the tail, so the queue is 72 // actually still full. We spin because this should resolve itself 73 // momentarily. 74 runtime.Gosched() 75 } 76 77 // The head slot is free, so we own it. 78 atomic.StorePointer(slot, unsafe.Pointer(b)) 79 80 // Increment head. This passes ownership of slot to dequeue and acts as a 81 // store barrier for writing the slot. 82 atomic.AddUint64(&q.headTail, 1<<dequeueBits) 83 } 84 85 func (q *commitQueue) dequeue() *Batch { 86 for { 87 ptrs := atomic.LoadUint64(&q.headTail) 88 head, tail := q.unpack(ptrs) 89 if tail == head { 90 // Queue is empty. 91 return nil 92 } 93 94 slot := &q.slots[tail&uint32(len(q.slots)-1)] 95 b := (*Batch)(atomic.LoadPointer(slot)) 96 if b == nil || atomic.LoadUint32(&b.applied) == 0 { 97 // The batch is not ready to be dequeued, or another goroutine has 98 // already dequeued it. 99 return nil 100 } 101 102 // Confirm head and tail (for our speculative check above) and increment 103 // tail. If this succeeds, then we own the slot at tail. 104 ptrs2 := q.pack(head, tail+1) 105 if atomic.CompareAndSwapUint64(&q.headTail, ptrs, ptrs2) { 106 // We now own slot. 107 // 108 // Tell enqueue that we're done with this slot. Zeroing the slot is also 109 // important so we don't leave behind references that could keep this object 110 // live longer than necessary. 111 atomic.StorePointer(slot, nil) 112 // At this point enqueue owns the slot. 113 return b 114 } 115 } 116 } 117 118 // commitEnv contains the environment that a commitPipeline interacts 119 // with. This allows fine-grained testing of commitPipeline behavior without 120 // construction of an entire DB. 121 type commitEnv struct { 122 // The next sequence number to give to a batch. Protected by 123 // commitPipeline.mu. 124 logSeqNum *uint64 125 // The visible sequence number at which reads should be performed. Ratcheted 126 // upwards atomically as batches are applied to the memtable. 127 visibleSeqNum *uint64 128 129 // Apply the batch to the specified memtable. Called concurrently. 130 apply func(b *Batch, mem *memTable) error 131 // Write the batch to the WAL. If wg != nil, the data will be persisted 132 // asynchronously and done will be called on wg upon completion. If wg != nil 133 // and err != nil, a failure to persist the WAL will populate *err. Returns 134 // the memtable the batch should be applied to. Serial execution enforced by 135 // commitPipeline.mu. 136 write func(b *Batch, wg *sync.WaitGroup, err *error) (*memTable, error) 137 } 138 139 // A commitPipeline manages the stages of committing a set of mutations 140 // (contained in a single Batch) atomically to the DB. The steps are 141 // conceptually: 142 // 143 // 1. Write the batch to the WAL and optionally sync the WAL 144 // 2. Apply the mutations in the batch to the memtable 145 // 146 // These two simple steps are made complicated by the desire for high 147 // performance. In the absence of concurrency, performance is limited by how 148 // fast a batch can be written (and synced) to the WAL and then added to the 149 // memtable, both of which are outside the purview of the commit 150 // pipeline. Performance under concurrency is the primary concern of the commit 151 // pipeline, though it also needs to maintain two invariants: 152 // 153 // 1. Batches need to be written to the WAL in sequence number order. 154 // 2. Batches need to be made visible for reads in sequence number order. This 155 // invariant arises from the use of a single sequence number which 156 // indicates which mutations are visible. 157 // 158 // Taking these invariants into account, let's revisit the work the commit 159 // pipeline needs to perform. Writing the batch to the WAL is necessarily 160 // serialized as there is a single WAL object. The order of the entries in the 161 // WAL defines the sequence number order. Note that writing to the WAL is 162 // extremely fast, usually just a memory copy. Applying the mutations in a 163 // batch to the memtable can occur concurrently as the underlying skiplist 164 // supports concurrent insertions. Publishing the visible sequence number is 165 // another serialization point, but one with a twist: the visible sequence 166 // number cannot be bumped until the mutations for earlier batches have 167 // finished applying to the memtable (the visible sequence number only ratchets 168 // up). Lastly, if requested, the commit waits for the WAL to sync. Note that 169 // waiting for the WAL sync after ratcheting the visible sequence number allows 170 // another goroutine to read committed data before the WAL has synced. This is 171 // similar behavior to RocksDB's manual WAL flush functionality. Application 172 // code needs to protect against this if necessary. 173 // 174 // The full outline of the commit pipeline operation is as follows: 175 // 176 // with commitPipeline mutex locked: 177 // assign batch sequence number 178 // write batch to WAL 179 // (optionally) add batch to WAL sync list 180 // apply batch to memtable (concurrently) 181 // wait for earlier batches to apply 182 // ratchet read sequence number 183 // (optionally) wait for the WAL to sync 184 // 185 // As soon as a batch has been written to the WAL, the commitPipeline mutex is 186 // released allowing another batch to write to the WAL. Each commit operation 187 // individually applies its batch to the memtable providing concurrency. The 188 // WAL sync happens concurrently with applying to the memtable (see 189 // commitPipeline.syncLoop). 190 // 191 // The "waits for earlier batches to apply" work is more complicated than might 192 // be expected. The obvious approach would be to keep a queue of pending 193 // batches and for each batch to wait for the previous batch to finish 194 // committing. This approach was tried initially and turned out to be too 195 // slow. The problem is that it causes excessive goroutine activity as each 196 // committing goroutine needs to wake up in order for the next goroutine to be 197 // unblocked. The approach taken in the current code is conceptually similar, 198 // though it avoids waking a goroutine to perform work that another goroutine 199 // can perform. A commitQueue (a single-producer, multiple-consumer queue) 200 // holds the ordered list of committing batches. Addition to the queue is done 201 // while holding commitPipeline.mutex ensuring the same ordering of batches in 202 // the queue as the ordering in the WAL. When a batch finishes applying to the 203 // memtable, it atomically updates its Batch.applied field. Ratcheting of the 204 // visible sequence number is done by commitPipeline.publish which loops 205 // dequeueing "applied" batches and ratcheting the visible sequence number. If 206 // we hit an unapplied batch at the head of the queue we can block as we know 207 // that committing of that unapplied batch will eventually find our (applied) 208 // batch in the queue. See commitPipeline.publish for additional commentary. 209 type commitPipeline struct { 210 // WARNING: The following struct `commitQueue` contains fields which will 211 // be accessed atomically. 212 // 213 // Go allocations are guaranteed to be 64-bit aligned which we take advantage 214 // of by placing the 64-bit fields which we access atomically at the beginning 215 // of the commitPipeline struct. 216 // For more information, see https://golang.org/pkg/sync/atomic/#pkg-note-BUG. 217 // Queue of pending batches to commit. 218 pending commitQueue 219 env commitEnv 220 sem chan struct{} 221 // The mutex to use for synchronizing access to logSeqNum and serializing 222 // calls to commitEnv.write(). 223 mu sync.Mutex 224 } 225 226 func newCommitPipeline(env commitEnv) *commitPipeline { 227 p := &commitPipeline{ 228 env: env, 229 // NB: the commit concurrency is one less than SyncConcurrency because we 230 // have to allow one "slot" for a concurrent WAL rotation which will close 231 // and sync the WAL. 232 sem: make(chan struct{}, record.SyncConcurrency-1), 233 } 234 return p 235 } 236 237 // Commit the specified batch, writing it to the WAL, optionally syncing the 238 // WAL, and applying the batch to the memtable. Upon successful return the 239 // batch's mutations will be visible for reading. 240 func (p *commitPipeline) Commit(b *Batch, syncWAL bool) error { 241 if b.Empty() { 242 return nil 243 } 244 245 p.sem <- struct{}{} 246 247 // Prepare the batch for committing: enqueuing the batch in the pending 248 // queue, determining the batch sequence number and writing the data to the 249 // WAL. 250 // 251 // NB: We set Batch.commitErr on error so that the batch won't be a candidate 252 // for reuse. See Batch.release(). 253 mem, err := p.prepare(b, syncWAL) 254 if err != nil { 255 b.db = nil 256 return err 257 } 258 259 // Apply the batch to the memtable. 260 if err := p.env.apply(b, mem); err != nil { 261 b.db = nil 262 return err 263 } 264 265 // Publish the batch sequence number. 266 p.publish(b) 267 268 <-p.sem 269 270 if b.commitErr != nil { 271 b.db = nil 272 } 273 return b.commitErr 274 } 275 276 // AllocateSeqNum allocates count sequence numbers, invokes the prepare 277 // callback, then the apply callback, and then publishes the sequence 278 // numbers. AllocateSeqNum does not write to the WAL or add entries to the 279 // memtable. AllocateSeqNum can be used to sequence an operation such as 280 // sstable ingestion within the commit pipeline. The prepare callback is 281 // invoked with commitPipeline.mu held, but note that DB.mu is not held and 282 // must be locked if necessary. 283 func (p *commitPipeline) AllocateSeqNum(count int, prepare func(), apply func(seqNum uint64)) { 284 // This method is similar to Commit and prepare. Be careful about trying to 285 // share additional code with those methods because Commit and prepare are 286 // performance critical code paths. 287 288 b := newBatch(nil) 289 defer b.release() 290 291 // Give the batch a count of 1 so that the log and visible sequence number 292 // are incremented correctly. 293 b.data = make([]byte, batchHeaderLen) 294 b.setCount(uint32(count)) 295 b.commit.Add(1) 296 297 p.sem <- struct{}{} 298 299 p.mu.Lock() 300 301 // Enqueue the batch in the pending queue. Note that while the pending queue 302 // is lock-free, we want the order of batches to be the same as the sequence 303 // number order. 304 p.pending.enqueue(b) 305 306 // Assign the batch a sequence number. Note that we use atomic operations 307 // here to handle concurrent reads of logSeqNum. commitPipeline.mu provides 308 // mutual exclusion for other goroutines writing to logSeqNum. 309 logSeqNum := atomic.AddUint64(p.env.logSeqNum, uint64(count)) - uint64(count) 310 seqNum := logSeqNum 311 if seqNum == 0 { 312 // We can't use the value 0 for the global seqnum during ingestion, because 313 // 0 indicates no global seqnum. So allocate one more seqnum. 314 atomic.AddUint64(p.env.logSeqNum, 1) 315 seqNum++ 316 } 317 b.setSeqNum(seqNum) 318 319 // Wait for any outstanding writes to the memtable to complete. This is 320 // necessary for ingestion so that the check for memtable overlap can see any 321 // writes that were sequenced before the ingestion. The spin loop is 322 // unfortunate, but obviates the need for additional synchronization. 323 for { 324 visibleSeqNum := atomic.LoadUint64(p.env.visibleSeqNum) 325 if visibleSeqNum == logSeqNum { 326 break 327 } 328 runtime.Gosched() 329 } 330 331 // Invoke the prepare callback. Note the lack of error reporting. Even if the 332 // callback internally fails, the sequence number needs to be published in 333 // order to allow the commit pipeline to proceed. 334 prepare() 335 336 p.mu.Unlock() 337 338 // Invoke the apply callback. 339 apply(b.SeqNum()) 340 341 // Publish the sequence number. 342 p.publish(b) 343 344 <-p.sem 345 } 346 347 func (p *commitPipeline) prepare(b *Batch, syncWAL bool) (*memTable, error) { 348 n := uint64(b.Count()) 349 if n == invalidBatchCount { 350 return nil, ErrInvalidBatch 351 } 352 count := 1 353 if syncWAL { 354 count++ 355 } 356 // count represents the waiting needed for publish, and optionally the 357 // waiting needed for the WAL sync. 358 b.commit.Add(count) 359 360 var syncWG *sync.WaitGroup 361 var syncErr *error 362 if syncWAL { 363 syncWG, syncErr = &b.commit, &b.commitErr 364 } 365 366 p.mu.Lock() 367 368 // Enqueue the batch in the pending queue. Note that while the pending queue 369 // is lock-free, we want the order of batches to be the same as the sequence 370 // number order. 371 p.pending.enqueue(b) 372 373 // Assign the batch a sequence number. Note that we use atomic operations 374 // here to handle concurrent reads of logSeqNum. commitPipeline.mu provides 375 // mutual exclusion for other goroutines writing to logSeqNum. 376 b.setSeqNum(atomic.AddUint64(p.env.logSeqNum, n) - n) 377 378 // Write the data to the WAL. 379 mem, err := p.env.write(b, syncWG, syncErr) 380 381 p.mu.Unlock() 382 383 return mem, err 384 } 385 386 func (p *commitPipeline) publish(b *Batch) { 387 // Mark the batch as applied. 388 atomic.StoreUint32(&b.applied, 1) 389 390 // Loop dequeuing applied batches from the pending queue. If our batch was 391 // the head of the pending queue we are guaranteed that either we'll publish 392 // it or someone else will dequeue and publish it. If our batch is not the 393 // head of the queue then either we'll dequeue applied batches and reach our 394 // batch or there is an unapplied batch blocking us. When that unapplied 395 // batch applies it will go through the same process and publish our batch 396 // for us. 397 for { 398 t := p.pending.dequeue() 399 if t == nil { 400 // Wait for another goroutine to publish us. We might also be waiting for 401 // the WAL sync to finish. 402 b.commit.Wait() 403 break 404 } 405 if atomic.LoadUint32(&t.applied) != 1 { 406 panic("not reached") 407 } 408 409 // We're responsible for publishing the sequence number for batch t, but 410 // another concurrent goroutine might sneak in and publish the sequence 411 // number for a subsequent batch. That's ok as all we're guaranteeing is 412 // that the sequence number ratchets up. 413 for { 414 curSeqNum := atomic.LoadUint64(p.env.visibleSeqNum) 415 newSeqNum := t.SeqNum() + uint64(t.Count()) 416 if newSeqNum <= curSeqNum { 417 // t's sequence number has already been published. 418 break 419 } 420 if atomic.CompareAndSwapUint64(p.env.visibleSeqNum, curSeqNum, newSeqNum) { 421 // We successfully published t's sequence number. 422 break 423 } 424 } 425 426 t.commit.Done() 427 } 428 } 429 430 // ratchetSeqNum allocates and marks visible all sequence numbers less than 431 // but excluding `nextSeqNum`. 432 func (p *commitPipeline) ratchetSeqNum(nextSeqNum uint64) { 433 p.mu.Lock() 434 defer p.mu.Unlock() 435 436 logSeqNum := atomic.LoadUint64(p.env.logSeqNum) 437 if logSeqNum >= nextSeqNum { 438 return 439 } 440 count := nextSeqNum - logSeqNum 441 _ = atomic.AddUint64(p.env.logSeqNum, uint64(count)) - uint64(count) 442 atomic.StoreUint64(p.env.visibleSeqNum, nextSeqNum) 443 }