Skip to content

Commit 80a5615

Browse files
committed
sstable: use block cache / readhandle for downloads
Previously, if we had some blocks in the block cache at the time of starting a download, we would still read all blocks from object storage. This was pretty wasteful, especially for index/footer/etc blocks that are more likely to be in cache. This change threads through the appropriate Reader through sstable.CopySpan, so that the cached blocks can be appropriately used. Data blocks are also read through the cache if they exist in the cache; if not, we fall back to the old sequence of reading and writing bytes blindly from object storage to local disk as fast as IO allows us to. Fixes #3758.
1 parent 9f56153 commit 80a5615

File tree

4 files changed

+84
-37
lines changed

4 files changed

+84
-37
lines changed

compaction.go

Lines changed: 12 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -2287,11 +2287,18 @@ func (d *DB) runCopyCompaction(
22872287
}
22882288
}
22892289

2290-
wrote, err := sstable.CopySpan(ctx,
2291-
src, d.opts.MakeReaderOptions(),
2292-
w, d.opts.MakeWriterOptions(c.outputLevel.level, d.FormatMajorVersion().MaxTableFormat()),
2293-
start, end,
2294-
)
2290+
// NB: external files are always virtual.
2291+
var wrote uint64
2292+
err = d.tableCache.withVirtualReader(inputMeta.VirtualMeta(), func(r sstable.VirtualReader) error {
2293+
var err error
2294+
wrote, err = sstable.CopySpan(ctx,
2295+
src, r.UnsafeReader(), d.opts.MakeReaderOptions(),
2296+
w, d.opts.MakeWriterOptions(c.outputLevel.level, d.FormatMajorVersion().MaxTableFormat()),
2297+
start, end,
2298+
)
2299+
return err
2300+
})
2301+
22952302
src = nil // We passed src to CopySpan; it's responsible for closing it.
22962303
if err != nil {
22972304
if errors.Is(err, sstable.ErrEmptySpan) {

objstorage/objstorage.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -375,9 +375,7 @@ type RemoteObjectToAttach struct {
375375
}
376376

377377
// Copy copies the specified range from the input to the output.
378-
func Copy(ctx context.Context, in Readable, out Writable, offset, length uint64) error {
379-
r := in.NewReadHandle(NoReadBefore)
380-
r.SetupForCompaction()
378+
func Copy(ctx context.Context, r ReadHandle, out Writable, offset, length uint64) error {
381379
buf := make([]byte, 256<<10)
382380
end := offset + length
383381
for offset < end {

sstable/copier.go

Lines changed: 66 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -44,18 +44,13 @@ import (
4444
func CopySpan(
4545
ctx context.Context,
4646
input objstorage.Readable,
47+
r *Reader,
4748
rOpts ReaderOptions,
4849
output objstorage.Writable,
4950
o WriterOptions,
5051
start, end InternalKey,
5152
) (size uint64, _ error) {
52-
r, err := NewReader(ctx, input, rOpts)
53-
if err != nil {
54-
input.Close()
55-
output.Abort()
56-
return 0, err
57-
}
58-
defer r.Close() // r.Close now owns calling input.Close().
53+
defer input.Close()
5954

6055
if r.Properties.NumValueBlocks > 0 || r.Properties.NumRangeKeys() > 0 || r.Properties.NumRangeDeletions > 0 {
6156
return copyWholeFileBecauseOfUnsupportedFeature(ctx, input, output) // Finishes/Aborts output.
@@ -97,6 +92,7 @@ func CopySpan(
9792
rh := objstorageprovider.UsePreallocatedReadHandle(
9893
r.readable, objstorage.ReadBeforeForIndexAndFilter, &preallocRH)
9994
defer rh.Close()
95+
rh.SetupForCompaction()
10096
indexH, err := r.readIndex(ctx, rh, nil, nil)
10197
if err != nil {
10298
return 0, err
@@ -150,34 +146,73 @@ func CopySpan(
150146
return 0, ErrEmptySpan
151147
}
152148

153-
// Find the span of the input file that contains all our blocks, and then copy
154-
// it byte-for-byte without doing any per-key processing.
155-
offset := blocks[0].bh.Offset
156-
157-
// The block lengths don't include their trailers, which just sit after the
158-
// block length, before the next offset; We get the ones between the blocks
159-
// we copy implicitly but need to explicitly add the last trailer to length.
160-
length := blocks[len(blocks)-1].bh.Offset + blocks[len(blocks)-1].bh.Length + block.TrailerLen - offset
149+
// Copy all blocks byte-for-byte without doing any per-key processing.
150+
var blocksNotInCache []indexEntry
161151

162-
if spanEnd := length + offset; spanEnd < offset {
163-
return 0, base.AssertionFailedf("invalid intersecting span for CopySpan [%d, %d)", offset, spanEnd)
164-
}
165-
166-
if err := objstorage.Copy(ctx, r.readable, w.layout.writable, offset, length); err != nil {
167-
return 0, err
152+
copyBlocksToFile := func(blocks []indexEntry) error {
153+
blockOffset := blocks[0].bh.Offset
154+
// The block lengths don't include their trailers, which just sit after the
155+
// block length, before the next offset; We get the ones between the blocks
156+
// we copy implicitly but need to explicitly add the last trailer to length.
157+
length := blocks[len(blocks)-1].bh.Offset + blocks[len(blocks)-1].bh.Length + block.TrailerLen - blockOffset
158+
if spanEnd := length + blockOffset; spanEnd < blockOffset {
159+
return base.AssertionFailedf("invalid intersecting span for CopySpan [%d, %d)", blockOffset, spanEnd)
160+
}
161+
if err := objstorage.Copy(ctx, rh, w.layout.writable, blockOffset, length); err != nil {
162+
return err
163+
}
164+
// Update w.meta.Size so subsequently flushed metadata has correct offsets.
165+
w.meta.Size += length
166+
for i := range blocks {
167+
blocks[i].bh.Offset = w.layout.offset
168+
// blocks[i].bh.Length remains unmodified.
169+
if err := w.addIndexEntrySep(blocks[i].sep, blocks[i].bh, w.dataBlockBuf.tmp[:]); err != nil {
170+
return err
171+
}
172+
w.layout.offset += uint64(blocks[i].bh.Length) + block.TrailerLen
173+
}
174+
return nil
168175
}
169-
w.layout.offset += length
176+
for i := range blocks {
177+
h := r.cacheOpts.Cache.Get(r.cacheOpts.CacheID, r.cacheOpts.FileNum, blocks[i].bh.Offset)
178+
if h.Get() == nil {
179+
// Cache miss. Add this block to the list of blocks that are not in cache.
180+
blocksNotInCache = blocks[i-len(blocksNotInCache) : i+1]
181+
continue
182+
}
170183

171-
// Update w.meta.Size so subsequently flushed metadata has correct offsets.
172-
w.meta.Size += length
184+
// Cache hit.
185+
rh.RecordCacheHit(ctx, int64(blocks[i].bh.Offset), int64(blocks[i].bh.Length+block.TrailerLen))
186+
if len(blocksNotInCache) > 0 {
187+
// We have some blocks that were not in cache preceding this block.
188+
// Copy them using objstorage.Copy.
189+
if err := copyBlocksToFile(blocksNotInCache); err != nil {
190+
h.Release()
191+
return 0, err
192+
}
193+
blocksNotInCache = nil
194+
}
173195

174-
// Now we can setup index entries for all the blocks we just copied, pointing
175-
// into the copied span.
176-
for i := range blocks {
177-
blocks[i].bh.Offset -= offset
196+
// layout.WriteDataBlock keeps layout.offset up-to-date for us.
197+
bh, err := w.layout.WriteDataBlock(h.Get(), &w.dataBlockBuf.blockBuf)
198+
h.Release()
199+
if err != nil {
200+
return 0, err
201+
}
202+
blocks[i].bh.Handle = bh
178203
if err := w.addIndexEntrySep(blocks[i].sep, blocks[i].bh, w.dataBlockBuf.tmp[:]); err != nil {
179204
return 0, err
180205
}
206+
w.meta.Size += uint64(bh.Length) + block.TrailerLen
207+
}
208+
209+
if len(blocksNotInCache) > 0 {
210+
// We have some remaining blocks that were not in cache. Copy them
211+
// using objstorage.Copy.
212+
if err := copyBlocksToFile(blocksNotInCache); err != nil {
213+
return 0, err
214+
}
215+
blocksNotInCache = nil
181216
}
182217

183218
// TODO(dt): Copy range keys (the fact there are none is checked above).
@@ -282,7 +317,9 @@ func copyWholeFileBecauseOfUnsupportedFeature(
282317
ctx context.Context, input objstorage.Readable, output objstorage.Writable,
283318
) (size uint64, _ error) {
284319
length := uint64(input.Size())
285-
if err := objstorage.Copy(ctx, input, output, 0, length); err != nil {
320+
rh := input.NewReadHandle(objstorage.NoReadBefore)
321+
rh.SetupForCompaction()
322+
if err := objstorage.Copy(ctx, rh, output, 0, length); err != nil {
286323
output.Abort()
287324
return 0, err
288325
}

sstable/reader_virtual.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -209,6 +209,11 @@ func (v *VirtualReader) NewRawRangeKeyIter(
209209
), nil
210210
}
211211

212+
// UnsafeReader returns the underlying *sstable.Reader behind a VirtualReader.
213+
func (v *VirtualReader) UnsafeReader() *Reader {
214+
return v.reader
215+
}
216+
212217
// Constrain bounds will narrow the start, end bounds if they do not fit within
213218
// the virtual sstable. The function will return if the new end key is
214219
// inclusive.

0 commit comments

Comments
 (0)