github.com/cockroachdb/pebble@v1.1.2/flushable.go (about) 1 // Copyright 2020 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 pebble 6 7 import ( 8 "context" 9 "fmt" 10 "sync/atomic" 11 "time" 12 13 "github.com/cockroachdb/pebble/internal/keyspan" 14 "github.com/cockroachdb/pebble/internal/manifest" 15 ) 16 17 // flushable defines the interface for immutable memtables. 18 type flushable interface { 19 newIter(o *IterOptions) internalIterator 20 newFlushIter(o *IterOptions, bytesFlushed *uint64) internalIterator 21 newRangeDelIter(o *IterOptions) keyspan.FragmentIterator 22 newRangeKeyIter(o *IterOptions) keyspan.FragmentIterator 23 containsRangeKeys() bool 24 // inuseBytes returns the number of inuse bytes by the flushable. 25 inuseBytes() uint64 26 // totalBytes returns the total number of bytes allocated by the flushable. 27 totalBytes() uint64 28 // readyForFlush returns true when the flushable is ready for flushing. See 29 // memTable.readyForFlush for one implementation which needs to check whether 30 // there are any outstanding write references. 31 readyForFlush() bool 32 } 33 34 // flushableEntry wraps a flushable and adds additional metadata and 35 // functionality that is common to all flushables. 36 type flushableEntry struct { 37 flushable 38 // Channel which is closed when the flushable has been flushed. 39 flushed chan struct{} 40 // flushForced indicates whether a flush was forced on this memtable (either 41 // manual, or due to ingestion). Protected by DB.mu. 42 flushForced bool 43 // delayedFlushForcedAt indicates whether a timer has been set to force a 44 // flush on this memtable at some point in the future. Protected by DB.mu. 45 // Holds the timestamp of when the flush will be issued. 46 delayedFlushForcedAt time.Time 47 // logNum corresponds to the WAL that contains the records present in the 48 // receiver. 49 logNum FileNum 50 // logSize is the size in bytes of the associated WAL. Protected by DB.mu. 51 logSize uint64 52 // The current logSeqNum at the time the memtable was created. This is 53 // guaranteed to be less than or equal to any seqnum stored in the memtable. 54 logSeqNum uint64 55 // readerRefs tracks the read references on the flushable. The two sources of 56 // reader references are DB.mu.mem.queue and readState.memtables. The memory 57 // reserved by the flushable in the cache is released when the reader refs 58 // drop to zero. If the flushable is referencing sstables, then the file 59 // refount is also decreased once the reader refs drops to 0. If the 60 // flushable is a memTable, when the reader refs drops to zero, the writer 61 // refs will already be zero because the memtable will have been flushed and 62 // that only occurs once the writer refs drops to zero. 63 readerRefs atomic.Int32 64 // Closure to invoke to release memory accounting. 65 releaseMemAccounting func() 66 // unrefFiles, if not nil, should be invoked to decrease the ref count of 67 // files which are backing the flushable. 68 unrefFiles func() []*fileBacking 69 // deleteFnLocked should be called if the caller is holding DB.mu. 70 deleteFnLocked func(obsolete []*fileBacking) 71 // deleteFn should be called if the caller is not holding DB.mu. 72 deleteFn func(obsolete []*fileBacking) 73 } 74 75 func (e *flushableEntry) readerRef() { 76 switch v := e.readerRefs.Add(1); { 77 case v <= 1: 78 panic(fmt.Sprintf("pebble: inconsistent reference count: %d", v)) 79 } 80 } 81 82 // db.mu must not be held when this is called. 83 func (e *flushableEntry) readerUnref(deleteFiles bool) { 84 e.readerUnrefHelper(deleteFiles, e.deleteFn) 85 } 86 87 // db.mu must be held when this is called. 88 func (e *flushableEntry) readerUnrefLocked(deleteFiles bool) { 89 e.readerUnrefHelper(deleteFiles, e.deleteFnLocked) 90 } 91 92 func (e *flushableEntry) readerUnrefHelper( 93 deleteFiles bool, deleteFn func(obsolete []*fileBacking), 94 ) { 95 switch v := e.readerRefs.Add(-1); { 96 case v < 0: 97 panic(fmt.Sprintf("pebble: inconsistent reference count: %d", v)) 98 case v == 0: 99 if e.releaseMemAccounting == nil { 100 panic("pebble: memtable reservation already released") 101 } 102 e.releaseMemAccounting() 103 e.releaseMemAccounting = nil 104 if e.unrefFiles != nil { 105 obsolete := e.unrefFiles() 106 e.unrefFiles = nil 107 if deleteFiles { 108 deleteFn(obsolete) 109 } 110 } 111 } 112 } 113 114 type flushableList []*flushableEntry 115 116 // ingestedFlushable is the implementation of the flushable interface for the 117 // ingesting sstables which are added to the flushable list. 118 type ingestedFlushable struct { 119 files []physicalMeta 120 comparer *Comparer 121 newIters tableNewIters 122 newRangeKeyIters keyspan.TableNewSpanIter 123 124 // Since the level slice is immutable, we construct and set it once. It 125 // should be safe to read from slice in future reads. 126 slice manifest.LevelSlice 127 // hasRangeKeys is set on ingestedFlushable construction. 128 hasRangeKeys bool 129 } 130 131 func newIngestedFlushable( 132 files []*fileMetadata, 133 comparer *Comparer, 134 newIters tableNewIters, 135 newRangeKeyIters keyspan.TableNewSpanIter, 136 ) *ingestedFlushable { 137 var physicalFiles []physicalMeta 138 var hasRangeKeys bool 139 for _, f := range files { 140 if f.HasRangeKeys { 141 hasRangeKeys = true 142 } 143 physicalFiles = append(physicalFiles, f.PhysicalMeta()) 144 } 145 146 ret := &ingestedFlushable{ 147 files: physicalFiles, 148 comparer: comparer, 149 newIters: newIters, 150 newRangeKeyIters: newRangeKeyIters, 151 // slice is immutable and can be set once and used many times. 152 slice: manifest.NewLevelSliceKeySorted(comparer.Compare, files), 153 hasRangeKeys: hasRangeKeys, 154 } 155 156 return ret 157 } 158 159 // TODO(sumeer): ingestedFlushable iters also need to plumb context for 160 // tracing. 161 162 // newIter is part of the flushable interface. 163 func (s *ingestedFlushable) newIter(o *IterOptions) internalIterator { 164 var opts IterOptions 165 if o != nil { 166 opts = *o 167 } 168 // TODO(bananabrick): The manifest.Level in newLevelIter is only used for 169 // logging. Update the manifest.Level encoding to account for levels which 170 // aren't truly levels in the lsm. Right now, the encoding only supports 171 // L0 sublevels, and the rest of the levels in the lsm. 172 return newLevelIter( 173 opts, s.comparer, s.newIters, s.slice.Iter(), manifest.Level(0), internalIterOpts{}, 174 ) 175 } 176 177 // newFlushIter is part of the flushable interface. 178 func (s *ingestedFlushable) newFlushIter(o *IterOptions, bytesFlushed *uint64) internalIterator { 179 // newFlushIter is only used for writing memtables to disk as sstables. 180 // Since ingested sstables are already present on disk, they don't need to 181 // make use of a flush iter. 182 panic("pebble: not implemented") 183 } 184 185 func (s *ingestedFlushable) constructRangeDelIter( 186 file *manifest.FileMetadata, _ keyspan.SpanIterOptions, 187 ) (keyspan.FragmentIterator, error) { 188 // Note that the keyspan level iter expects a non-nil iterator to be 189 // returned even if there is an error. So, we return the emptyKeyspanIter. 190 iter, rangeDelIter, err := s.newIters(context.Background(), file, nil, internalIterOpts{}) 191 if err != nil { 192 return emptyKeyspanIter, err 193 } 194 iter.Close() 195 if rangeDelIter == nil { 196 return emptyKeyspanIter, nil 197 } 198 return rangeDelIter, nil 199 } 200 201 // newRangeDelIter is part of the flushable interface. 202 // TODO(bananabrick): Using a level iter instead of a keyspan level iter to 203 // surface range deletes is more efficient. 204 func (s *ingestedFlushable) newRangeDelIter(_ *IterOptions) keyspan.FragmentIterator { 205 return keyspan.NewLevelIter( 206 keyspan.SpanIterOptions{}, s.comparer.Compare, 207 s.constructRangeDelIter, s.slice.Iter(), manifest.Level(0), 208 manifest.KeyTypePoint, 209 ) 210 } 211 212 // newRangeKeyIter is part of the flushable interface. 213 func (s *ingestedFlushable) newRangeKeyIter(o *IterOptions) keyspan.FragmentIterator { 214 if !s.containsRangeKeys() { 215 return nil 216 } 217 218 return keyspan.NewLevelIter( 219 keyspan.SpanIterOptions{}, s.comparer.Compare, s.newRangeKeyIters, 220 s.slice.Iter(), manifest.Level(0), manifest.KeyTypeRange, 221 ) 222 } 223 224 // containsRangeKeys is part of the flushable interface. 225 func (s *ingestedFlushable) containsRangeKeys() bool { 226 return s.hasRangeKeys 227 } 228 229 // inuseBytes is part of the flushable interface. 230 func (s *ingestedFlushable) inuseBytes() uint64 { 231 // inuseBytes is only used when memtables are flushed to disk as sstables. 232 panic("pebble: not implemented") 233 } 234 235 // totalBytes is part of the flushable interface. 236 func (s *ingestedFlushable) totalBytes() uint64 { 237 // We don't allocate additional bytes for the ingestedFlushable. 238 return 0 239 } 240 241 // readyForFlush is part of the flushable interface. 242 func (s *ingestedFlushable) readyForFlush() bool { 243 // ingestedFlushable should always be ready to flush. However, note that 244 // memtables before the ingested sstables in the memtable queue must be 245 // flushed before an ingestedFlushable can be flushed. This is because the 246 // ingested sstables need an updated view of the Version to 247 // determine where to place the files in the lsm. 248 return true 249 }