Skip to content

Commit 094779f

Browse files
craig[bot]kev-cao
andcommitted
Merge #144724
144724: backup: fix compaction issue with attempting to write keys outside of span r=msbutler a=kev-cao After the fix applied by #144652, the test failures outlined in #144216 resurfaced as job failures due to attempts by the `SSTSinkKeyWriter` to write keys outside of the `BackupManifest_File` range. The behavior was unexpected, because for some key `/Table/1/1/5`, we would see an attempt to write it to a span starting with `/Table/1/1/5/1`, despite the the last `SSTSinkKeyWriter.Reset` call being made on a span starting with `/Table/1/1/1`. This was ultimately determined to be caused by the fact that in the compaction processor's `compactSpanEntry`, we re-use the same underlying memory for the key that is being passed to `SSTSinkKeyWriter.WriteKey` to save on memory allocations. However, if the sink performed a flush due to size constraints in `maybeDoSizeFlush`, the start key of the span used to reset the sink referenced the same memory location as key that was passed in. So for subsequent keys in that span that were written, the re-use of that underlying memory in `compactSpanEntry` would consistently mutate the span referenced by that `BackupManifest_File`, causing corruption. This would usually result in job failures as eventually a key may be written outside of the span, but occasionally result in insidious job successes and the resulting `BackupManifest_File` would report a far smaller span than it actually covered. One solution was to perform a clone of the key in `maybeDoSizeFlush` when creating the span to reset the sink with, but ultimately decided to instead ensure that anytime `SSTSinkKeyWriter.Reset` is called, we pass a clone of the span to the `BackupManifest_File`. This ensures that once `Reset` is called, the caller is free to reuse the underlying memory of the span however they wish. The same holds true for the key passed to `WriteKey`, as in any instances in which the passed in key is actually persisted, we always persist a copy. Fixes: #144216, #144339 Release note: None Co-authored-by: Kevin Cao <[email protected]>
2 parents 77ff853 + e035358 commit 094779f

File tree

3 files changed

+60
-5
lines changed

3 files changed

+60
-5
lines changed

pkg/backup/backupsink/sst_sink_key_writer.go

Lines changed: 15 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -45,10 +45,12 @@ func MakeSSTSinkKeyWriter(conf SSTSinkConf, dest cloud.ExternalStorage) (*SSTSin
4545
// including the prefix. Reset needs to be called prior to WriteKey whenever
4646
// writing keys from a new span. Keys must also be written in order.
4747
//
48-
// After writing the last key for a span, AssumeNotMidRow must be called to
49-
// enforce the invariant that BackupManifest_File spans do not end mid-row.
48+
// Once a key has been written, the caller may safely reuse the underlying
49+
// memory for the passed in key.
5050
//
51-
// Flush should be called before the sink is closed to ensure the SST
51+
// NOTE: After writing the last key for a span, AssumeNotMidRow must be called
52+
// to enforce the invariant that BackupManifest_File spans do not end mid-row.
53+
// Flush should also be called before the sink is closed to ensure the SST
5254
// is written to the destination.
5355
func (s *SSTSinkKeyWriter) WriteKey(ctx context.Context, key storage.MVCCKey, value []byte) error {
5456
if len(s.flushedFiles) == 0 {
@@ -117,6 +119,9 @@ func (s *SSTSinkKeyWriter) AssumeNotMidRow() {
117119
// and calling AssumeNotMidRow before resetting to enforce this invariant.
118120
// Any time a new span is being written, Reset MUST be called prior to any
119121
// WriteKey calls.
122+
//
123+
// Once Reset has been called, the caller may safely reuse the underlying memory
124+
// of the passed in span.
120125
func (s *SSTSinkKeyWriter) Reset(ctx context.Context, newSpan roachpb.Span) error {
121126
log.VEventf(ctx, 2, "resetting sink to span %s", newSpan)
122127
if s.midRow {
@@ -143,7 +148,8 @@ func (s *SSTSinkKeyWriter) Reset(ctx context.Context, newSpan roachpb.Span) erro
143148
lastFile.EntryCounts.DataSize < fileSpanByteLimit {
144149
log.VEventf(ctx, 2, "extending span %s to %s", lastFile.Span, newSpan)
145150
s.stats.spanGrows++
146-
lastFile.Span.EndKey = newSpan.EndKey
151+
// See reason for Clone() below.
152+
lastFile.Span.EndKey = newSpan.EndKey.Clone()
147153
return nil
148154
}
149155
}
@@ -156,7 +162,11 @@ func (s *SSTSinkKeyWriter) Reset(ctx context.Context, newSpan roachpb.Span) erro
156162
s.flushedFiles = append(
157163
s.flushedFiles,
158164
backuppb.BackupManifest_File{
159-
Span: newSpan,
165+
// Because there are situations where the underlying memory for keys of
166+
// the span is reused to optimize memory usage, we need to clone the keys
167+
// to ensure that the BackupManifest_File's span is not unintentionally
168+
// mutated outside of the SSTSinkKeyWriter.
169+
Span: newSpan.Clone(),
160170
Path: s.outName,
161171
},
162172
)

pkg/backup/backupsink/sst_sink_key_writer_test.go

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -481,6 +481,48 @@ func TestEnforceFileSSTSinkAssumeNotMidRow(t *testing.T) {
481481
})
482482
}
483483

484+
func TestSSTSinkWriterSafeAgainstKeyMutation(t *testing.T) {
485+
defer leaktest.AfterTest(t)()
486+
defer log.Scope(t).Close(t)
487+
488+
ctx := context.Background()
489+
st := cluster.MakeTestingClusterSettings()
490+
sink, _ := sstSinkKeyWriterTestSetup(t, st, execinfrapb.ElidePrefix_TenantAndTable)
491+
defer func() {
492+
require.NoError(t, sink.Close())
493+
}()
494+
495+
mutateSpan := func(span *roachpb.Span) {
496+
span.Key = append(span.Key[:0], make([]byte, len(span.Key))...)
497+
span.EndKey = append(span.EndKey[:0], make([]byte, len(span.EndKey))...)
498+
}
499+
500+
t.Run("safe on reset on new span", func(t *testing.T) {
501+
keySet := newMVCCKeySet("a", "c").withKVs([]kvAndTS{{key: "a", timestamp: 10}})
502+
require.NoError(t, sink.Reset(ctx, keySet.span))
503+
require.Len(t, sink.flushedFiles, 1)
504+
originalSpan := sink.flushedFiles[0].Span.Clone()
505+
mutateSpan(&keySet.span)
506+
require.Equal(t, originalSpan, sink.flushedFiles[0].Span)
507+
require.NoError(t, sink.Flush(ctx))
508+
})
509+
510+
t.Run("safe on reset on extending span", func(t *testing.T) {
511+
keySet := newMVCCKeySet("a", "c").withKVs([]kvAndTS{{key: "a", timestamp: 10}})
512+
extendingSet := newMVCCKeySet("c", "e").withKVs([]kvAndTS{{key: "c", timestamp: 10}})
513+
require.NoError(t, sink.Reset(ctx, keySet.span))
514+
require.NoError(t, sink.WriteKey(ctx, keySet.kvs[0].key, keySet.kvs[0].value))
515+
sink.AssumeNotMidRow()
516+
517+
require.NoError(t, sink.Reset(ctx, extendingSet.span))
518+
require.Len(t, sink.flushedFiles, 1)
519+
originalSpan := sink.flushedFiles[0].Span.Clone()
520+
mutateSpan(&extendingSet.span)
521+
require.Equal(t, originalSpan, sink.flushedFiles[0].Span)
522+
require.NoError(t, sink.Flush(ctx))
523+
})
524+
}
525+
484526
func sstSinkKeyWriterTestSetup(
485527
t *testing.T, settings *cluster.Settings, elideMode execinfrapb.ElidePrefix,
486528
) (*SSTSinkKeyWriter, cloud.ExternalStorage) {

pkg/backup/compaction_processor.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -395,6 +395,9 @@ func compactSpanEntry(
395395
) error {
396396
defer sstIter.cleanup()
397397
entry := sstIter.entry
398+
if err := assertCommonPrefix(entry.Span, entry.ElidedPrefix); err != nil {
399+
return err
400+
}
398401
prefix, err := backupsink.ElidedPrefix(entry.Span.Key, entry.ElidedPrefix)
399402
if err != nil {
400403
return err

0 commit comments

Comments
 (0)