github.com/cockroachdb/pebble@v1.1.1-0.20240513155919-3622ade60459/docs/RFCS/20220311_pebble_flushable_ingested_sstable.md (about)

     1  - Feature Name: Flushable Ingested SSTable
     2  - Status: in-progress
     3  - Start Date: 2022-03-11
     4  - Authors: Mufeez Amjad
     5  - RFC PR: [#1586](https://github.com/cockroachdb/pebble/pull/1586)
     6  - Pebble Issues: [#25](https://github.com/cockroachdb/pebble/issues/25)
     7  - Cockroach Issues:
     8  
     9  ## Summary
    10  
    11  To avoid a forced flush when ingesting SSTables that have an overlap with a
    12  memtable, we "lazily" add the SSTs to the LSM as a `*flushableEntry` to
    13  `d.mu.mem.queue`. In comparison to a regular ingest which adds the SSTs to the
    14  lowest possible level, the SSTs will get placed in the memtable queue before
    15  they are eventually flushed (to the lowest level possible). This state is only
    16  persisted in memory until a flush occurs, thus we require a WAL entry to replay
    17  the ingestion in the event of a crash.
    18  
    19  ## Motivation
    20  
    21  Currently, if any of the SSTs that need to be ingested have an overlap with a
    22  memtable, we
    23  [wait](https://github.com/cockroachdb/pebble/blob/56c5aebe151977964db7e464bb6c87ebd3451bd5/ingest.go#L671)
    24  for the memtable to be flushed before the ingestion can proceed. This is to
    25  satisfy the invariant that newer entries (those in the ingested SSTs) in the LSM
    26  have a higher sequence number than old entries (those in the memtables). This
    27  problem is also present for subsequent normal writes that are blocked behind the
    28  ingest waiting for their sequence number to be made visible.
    29  
    30  ## Technical Design
    31  
    32  The proposed design is mostly taken from Peter's suggestion in #25. The core
    33  requirements are:
    34  1. Replayable WAL entry for the ingest.
    35  2. Implementation of the `flushable` interface for a new `ingestedSSTables` struct.
    36  3. Lazily adding the ingested SSTs to the LSM.
    37  4. Flushing logic to move SSTs into L0-L6.
    38  
    39  <br>
    40  
    41  ### 1. WAL Entry
    42  
    43  We require a WAL entry to make the ingestion into the flushable queue
    44  replayable, and there is a need for a new type of WAL entry that does not get
    45  applied to the memtable. 2 approaches were considered:
    46  1. Using `seqnum=0` to differentiate this new WAL entry.
    47  2. Introduce a new `InternalKeyKind` for the new WAL entry,
    48     `InternalKeyKindIngestSST`.
    49  
    50  We believe the second approach is better because it avoids modifying batch
    51  headers which can be messy/hacky and because `seqnum=0` is already used for
    52  unapplied batches. The second approach also gives way for a simpler/cleaner
    53  implementation because it utilizes the extensibility of `InternalKeyKind` and is
    54  similar to the treatment of `InternalKeyKindLogData`. It also follows the
    55  correct seqnum semantics for SSTable ingestion in the event of a WAL replay —
    56  each SST in the ingestion batch already gets its own sequence number.
    57  
    58  This change will need to be gated on a `FormatMajorVersion` because if the store
    59  is opened with an older version of Pebble, Pebble will not understand any WAL
    60  entry that contains the new `InternalKeyKind`.
    61  
    62  <br>
    63  
    64  When performing an ingest (with overlap), we create a batch with the header:
    65  
    66  ```
    67  +-------------+------------+--- ... ---+
    68  | SeqNum (8B) | Count (4B) |  Entries  |
    69  +-------------+------------+--- ... ---+
    70  ```
    71  
    72  where`SeqNum` is the current running sequence number in the WAL, `Count` is the
    73  number of ingested SSTs, and each entry has the form:
    74  
    75  ```
    76  +-----------+-----------------+-------------------+
    77  | Kind (1B) | Key (varstring) | Value (varstring) |
    78  +-----------+-----------------+-------------------+
    79  ```
    80  
    81  where `Kind` is `InternalKeyKindIngestSST`, and `Key` is a path to the
    82  ingested SST on disk.
    83  
    84  When replaying the WAL, we check every batch's first entry and if `keykind ==
    85  InternalKeyKindIngestSSTs` then we continue reading the rest of the entries in
    86  the batch of SSTs and replay the ingestion steps - we construct a
    87  `flushableEntry` and add it to the flushable queue:
    88  
    89  ```go
    90  b = Batch{db: d}
    91  b.SetRepr(buf.Bytes())
    92  seqNum := b.SeqNum()
    93  maxSeqNum = seqNum + uint64(b.Count())
    94  br := b.Reader()
    95  if kind, _, _, _ := br.Next(); kind == InternalKeyKindIngestSST {
    96    // Continue reading the rest of the batch and construct flushable 
    97    // of sstables with correct seqnum and add to queue.
    98    buf.Reset()
    99    continue
   100  }
   101  ```
   102  
   103  
   104  ### 2. `flushable` Implementation
   105  
   106  Introduce a new flushable type: `ingestedSSTables`.
   107  
   108  ```go
   109  type ingestedSSTables struct {
   110      files []*fileMetadata
   111      size  uint64
   112  
   113      cmp      Compare
   114      newIters tableNewIters
   115  }
   116  ```
   117  which implements the following functions from the `flushable` interface:
   118  
   119  #### 1. `newIter(o *IterOptions) internalIterator`
   120  
   121  We return a `levelIter` since the ingested SSTables have no overlap, and we can
   122  treat them like a level in the LSM.
   123  
   124  ```go
   125  levelSlice := manifest.NewLevelSliceKeySorted(s.cmp, s.files)
   126  return newLevelIter(*o, s.cmp, nil, s.newIters, levelSlice.Iter(), 0, nil)
   127  ```
   128  
   129  <br>
   130  
   131  On the client-side, this iterator would have to be used like this:
   132  ```go
   133  var iter internalIteratorWithStats
   134  var rangeDelIter keyspan.FragmentIterator
   135  iter = base.WrapIterWithStats(mem.newIter(&dbi.opts))
   136  switch mem.flushable.(type) {
   137  case *ingestedSSTables:
   138      iter.(*levelIter).initRangeDel(&rangeDelIter)
   139  default:
   140      rangeDelIter = mem.newRangeDelIter(&dbi.opts)
   141  }
   142  
   143  mlevels = append(mlevels, mergingIterLevel{
   144      iter:         iter,
   145      rangeDelIter: rangeDelIter,
   146  })
   147  ```
   148  
   149  #### 2. `newFlushIter(o *IterOptions, bytesFlushed *uint64) internalIterator`
   150  
   151  #### 3. `newRangeDelIter(o *IterOptions) keyspan.FragmentIterator`
   152  
   153  The above two methods would return `nil`. By doing so, in `c.newInputIter()`:
   154  ```go
   155  if flushIter := f.newFlushIter(nil, &c.bytesIterated); flushIter != nil {
   156      iters = append(iters, flushIter)
   157  }
   158  if rangeDelIter := f.newRangeDelIter(nil); rangeDelIter != nil {
   159      iters = append(iters, rangeDelIter)
   160  }
   161  ```
   162  we ensure that no iterators on `ingestedSSTables` will be used while flushing in
   163  `c.runCompaction()`.
   164  
   165  The special-cased flush process for this flushable is described in [Section
   166  4](#4-flushing-logic-to-move-ssts-into-l0).
   167  
   168  #### 4. `newRangeKeyIter(o *IterOptions) keyspan.FragmentIterator`
   169  
   170  Will wait on range key support in `levelIter` to land before implementing.
   171  
   172  #### 5. `inuseBytes() uint64` and `totalBytes() uint64`
   173  
   174  For both functions, we return 0.
   175  
   176  Returning 0 for `inuseBytes()` means that the calculation of `c.maxOverlapBytes`
   177  is not affected by the SSTs (the ingested SSTs don't participate in the
   178  compaction).
   179  
   180  We don't want the size of the ingested SSTs to contribute to the size of the
   181  memtable when determining whether or not to stall writes
   182  (`MemTableStopWritesThreshold`); they should contribute to the L0 read-amp
   183  instead (`L0StopWritesThreshold`). Thus, we'll have to special case for ingested
   184  SSTs in `d.makeRoomForWrite()` to address this detail.
   185  
   186  `totalBytes()` represents the number of bytes allocated by the flushable, which
   187  in our case is 0. A consequence for this is that the size of the SSTs do not
   188  count towards the flush threshold calculation. However, by setting
   189  `flushableEntry.flushForced` we can achieve the same behaviour.
   190  
   191  #### 6. `readyForFlush() bool`
   192  
   193  The flushable of ingested SSTs can always be flushed because the files are
   194  already on disk, so we return true.
   195  
   196  ### 3. Lazily adding the ingested SSTs to the LSM
   197  
   198  The steps to add the ingested SSTs to the flushable queue are:
   199  1. Detect an overlap exists (existing logic).
   200  
   201  Add a check that falls back to the old ingestion logic of blocking the ingest on
   202  the flush when `len(d.mu.mem.queue) >= MemtablesStopWritesThreshold - 1`. This
   203  reduces the chance that many short, overlapping, and successive ingestions cause
   204  a memtable write stall.
   205  
   206  Additionally, to mitigate the hiccup on subsequent normal writes, we could wait
   207  before the call to `d.commit.AllocateSeqNum` until:
   208  1. the number of immutable memtables and `ingestedSSTs` in the flushable queue
   209     is below a certain threshold (to prevent building up too many sublevels)
   210  2. the number of immutable memtables is low. This could lead to starvation if
   211     there is a high rate of normal writes.
   212  
   213  2. Create a batch with the list of ingested SSTs.
   214  ```go
   215  b := newBatch()
   216  for _, path := range paths:
   217      b.IngestSSTs([]byte(path), nil)
   218  ```
   219  3. Apply the batch.
   220  
   221  In the call to `d.commit.AllocateSeqNum`, `b.count` sequence numbers are already
   222  allocated before the `prepare` step. When we identify a memtable overlap, we
   223  commit the batch to the WAL manually (through logic similar to
   224  `commitPipeline.prepare`). The `apply` step would be a no-op if we performed a
   225  WAL write in the `prepare` step. We would also need to truncate the memtable/WAL
   226  after this step.
   227  
   228  5. Create `ingestedSSTables` flushable and `flushableEntry`.
   229  
   230  We'd need to call `ingestUpdateSeqNum` on these SSTs before adding them to the
   231  flushable. This is to respect the sequence number ordering invariant while the
   232  SSTs reside in the flushable queue.
   233  
   234  6. Add to flushable queue.
   235  
   236  Pebble requires that the last entry in `d.mu.mem.queue` is the mutable memtable
   237  with value `d.mu.mem.mutable`. When adding a `flushableEntry` to the queue, we
   238  want to maintain this invariant. To do this we pass `nil` as the batch to
   239  `d.makeRoomForWrite()`. The result is
   240  
   241  ```
   242  | immutable old memtable | mutable new memtable |
   243  ```
   244  
   245  We then append our new `flushableEntry`, and swap the last two elements in
   246  `d.mu.mem.queue`:
   247  
   248  ```
   249  | immutable old memtable | ingestedSSTables | mutable new memtable |
   250  ```
   251  
   252  Because we add the ingested SSTs to the flushable queue when there is overlap,
   253  and are skipping applying the version edit through the `apply` step of the
   254  ingestion, we ensure that the SSTs are only added to the LSM once.
   255  
   256  7. Call `d.maybeScheduleFlush()`.
   257  
   258  Because we've added an immutable memtable to the flushable queue and set
   259  `flushForced` on the `flushableEntry`, this will surely result in a flush. This
   260  call can be done asynchronously.
   261  
   262  We can then return to caller without waiting for the flush to finish.
   263  
   264  ### 4. Flushing logic to move SSTs into L0-L6
   265  
   266  By returning `nil` for both `flushable.newFlushIter()` and
   267  `flushable.newRangeDelIter()`, the `ingestedSSTables` flushable will not be
   268  flushed normally.
   269  
   270  The suggestion in issue #25 is to move the SSTs from the flushable queue into
   271  L0. However, only the tables that overlap with the memtable will need to target
   272  L0 (because they will likely overlap with L0 post flush), the others can be
   273  moved to lower levels in the LSM. We can use the existing logic in
   274  `ingestTargetLevel` to determine which level to move the ingested SSTables to
   275  during `c.runCompaction()`. However, it's important to do this step after the
   276  memtable has been flushed to use the correct `version` when determining overlap.
   277  
   278  The flushable of ingested SSTs should not influence the bounds on the
   279  compaction, so we will have to skip updating `c.smallest` and `c.largest` in
   280  `d.newFlush()` for this flushable.