Skip to content

Commit 273e266

Browse files
jbowensRaduBerinde
andcommitted
sstable: prevent block metadata pointers to garbage
When allocating memory for a new block, we allocate an additional block.MetadataSize bytes for holding a Go structure containing decoded data describing the block and maintaining pointers into the block's contents. This allows us to decode the block structure once when the block is loaded into the block cache. Readers that find the block in the cache simply cast this 'block metadata' preamble into the appropriate type. When a block is not found within the block cache, we: load the block, decode the block's header and initialize the block's unique metadata. We initialize the metadata by first casting the allocation's pointer to the appropriate Go pointer type. Then we decode the block's data, initializing the metadata struct accordingly. In the interim between the pointer cast and the initialization, the contents of the metadata are undefined. The per-block metadata types typically contain pointer fields. Once initialized, these pointers point into the block's data, all part of the same CGo manual allocation. If the Go garbage collector examines this Go struct before it's been fully initialized, it may observe arbitrary garbage within the pointer fields. If one of these values looks like a pointer to a Go-allocated part of the address space, the Go garbage collector crashes the process. It's a little unclear the circumstances in which the Go garbage collector will examine these metadata structs that exist within CGo memory, but experimentally it appears to be the case. This commit fixes the issue by always zeroing block metadata memory before casting it into the appropriate metadata struct. Additionally, it solves an analogous problem during de-allocation by nil-ing metadata pointers into blocks before we release the blocks back to the cache or buffer pool. Together, these changes ensure that whenever we have a pointer to a block metadata Go structure, the structure's backing memory is either zeroed or the result of initialization based on decoding the block. Informs cockroachdb/cockroach#149955. Informs cockroachdb/cockroach#150216. Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
1 parent 136248c commit 273e266

File tree

5 files changed

+57
-30
lines changed

5 files changed

+57
-30
lines changed

sstable/blob/blocks.go

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -236,10 +236,7 @@ const _ uint = block.MetadataSize - uint(unsafe.Sizeof(indexBlockDecoder{}))
236236

237237
// initIndexBlockMetadata initializes the index block metadata.
238238
func initIndexBlockMetadata(md *block.Metadata, data []byte) (err error) {
239-
if uintptr(unsafe.Pointer(md))%8 != 0 {
240-
return errors.AssertionFailedf("metadata is not 8-byte aligned")
241-
}
242-
d := (*indexBlockDecoder)(unsafe.Pointer(md))
239+
d := block.CastMetadataZero[indexBlockDecoder](md)
243240
// Initialization can panic; convert panics to corruption errors (so higher
244241
// layers can add file number and offset information).
245242
defer func() {
@@ -355,10 +352,7 @@ const _ uint = block.MetadataSize - uint(unsafe.Sizeof(blobValueBlockDecoder{}))
355352

356353
// initBlobValueBlockMetadata initializes the blob value block metadata.
357354
func initBlobValueBlockMetadata(md *block.Metadata, data []byte) (err error) {
358-
if uintptr(unsafe.Pointer(md))%8 != 0 {
359-
return errors.AssertionFailedf("metadata is not 8-byte aligned")
360-
}
361-
d := (*blobValueBlockDecoder)(unsafe.Pointer(md))
355+
d := block.CastMetadataZero[blobValueBlockDecoder](md)
362356
// Initialization can panic; convert panics to corruption errors (so higher
363357
// layers can add file number and offset information).
364358
defer func() {

sstable/blob/fetcher.go

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@ package blob
66

77
import (
88
"context"
9-
"unsafe"
109

1110
"github.com/cockroachdb/errors"
1211
"github.com/cockroachdb/pebble/internal/base"
@@ -250,7 +249,7 @@ func (cr *cachedReader) GetUnsafeValue(
250249
if err != nil {
251250
return nil, err
252251
}
253-
cr.indexBlock.dec = (*indexBlockDecoder)(unsafe.Pointer(cr.indexBlock.buf.BlockMetadata()))
252+
cr.indexBlock.dec = block.CastMetadata[indexBlockDecoder](cr.indexBlock.buf.BlockMetadata())
254253
cr.indexBlock.loaded = true
255254
}
256255

@@ -281,14 +280,17 @@ func (cr *cachedReader) GetUnsafeValue(
281280
// this case to be rare, and this is a hot path for the more common case
282281
// of non-rewritten blob files, so we defer optimizing for now.
283282
h := cr.indexBlock.dec.BlockHandle(physicalBlockIndex)
283+
// Nil out the decoder before releasing the buffers to ensure the Go GC
284+
// doesn't misinterpret the freed memory backing the decoders.
285+
cr.currentValueBlock.dec = nil
284286
cr.currentValueBlock.buf.Release()
285287
cr.currentValueBlock.loaded = false
286288
var err error
287289
cr.currentValueBlock.buf, err = cr.r.ReadValueBlock(ctx, env, cr.rh, h)
288290
if err != nil {
289291
return nil, err
290292
}
291-
cr.currentValueBlock.dec = (*blobValueBlockDecoder)(unsafe.Pointer(cr.currentValueBlock.buf.BlockMetadata()))
293+
cr.currentValueBlock.dec = block.CastMetadata[blobValueBlockDecoder](cr.currentValueBlock.buf.BlockMetadata())
292294
cr.currentValueBlock.physicalIndex = physicalBlockIndex
293295
cr.currentValueBlock.virtualID = vh.BlockID
294296
cr.currentValueBlock.valueIDOffset = valueIDOffset
@@ -311,6 +313,10 @@ func (cfr *cachedReader) Close() (err error) {
311313
if cfr.rh != nil {
312314
err = cfr.rh.Close()
313315
}
316+
// Nil out the decoders before releasing the buffers to ensure the Go GC
317+
// doesn't misinterpret the freed memory backing the decoders.
318+
cfr.indexBlock.dec = nil
319+
cfr.currentValueBlock.dec = nil
314320
cfr.indexBlock.buf.Release()
315321
cfr.currentValueBlock.buf.Release()
316322
// Release the cfg.Reader. closeFunc is provided by the file cache and

sstable/block/block.go

Lines changed: 34 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"runtime"
1313
"slices"
1414
"time"
15+
"unsafe"
1516

1617
"github.com/cespare/xxhash/v2"
1718
"github.com/cockroachdb/crlib/crtime"
@@ -219,12 +220,41 @@ func ValidateChecksum(checksumType ChecksumType, b []byte, bh Handle) error {
219220
// when the block is read from disk.
220221
//
221222
// Portions of this buffer can be cast to the structures we need (through
222-
// unsafe.Pointer), but note that any pointers in these structures will be
223-
// invisible to the GC. Pointers to the block's data buffer are ok, since the
224-
// metadata and the data have the same lifetime (sharing the underlying
225-
// allocation).
223+
// CastMetadata[Zero]), but note that any pointers in these structures should be
224+
// considered invisible to the GC for the purpose of preserving lifetime.
225+
// Pointers to the block's data buffer are ok, since the metadata and the data
226+
// have the same lifetime (sharing the underlying allocation).
226227
type Metadata [MetadataSize]byte
227228

229+
// CastMetadataZero casts the provided metadata to the type parameter T, zeroing
230+
// the memory backing the metadata first. This zeroing is necessary when first
231+
// initializing the data structure to ensure that the Go garbage collector
232+
// doesn't misinterpret any of T's pointer fields, falsely detecting them as
233+
// invalid pointers.
234+
func CastMetadataZero[T any](md *Metadata) *T {
235+
var z T
236+
if invariants.Enabled {
237+
if uintptr(unsafe.Pointer(md))%unsafe.Alignof(z) != 0 {
238+
panic(errors.AssertionFailedf("incorrect alignment for %T (%p)", z, unsafe.Pointer(md)))
239+
}
240+
}
241+
clear((*md)[:unsafe.Sizeof(z)])
242+
return (*T)(unsafe.Pointer(md))
243+
}
244+
245+
// CastMetadata casts the provided metadata to the type parameter T. If the
246+
// Metadata has not already been initialized, callers should use
247+
// CastMetadataZero.
248+
func CastMetadata[T any](md *Metadata) *T {
249+
var z T
250+
if invariants.Enabled {
251+
if uintptr(unsafe.Pointer(md))%unsafe.Alignof(z) != 0 {
252+
panic(fmt.Sprintf("incorrect alignment for %T (%p)", z, unsafe.Pointer(md)))
253+
}
254+
}
255+
return (*T)(unsafe.Pointer(md))
256+
}
257+
228258
// MetadataSize is the size of the metadata. The value is chosen to fit a
229259
// colblk.DataBlockDecoder and a CockroachDB colblk.KeySeeker.
230260
const MetadataSize = 336

sstable/block/buffer_pool.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,8 +177,9 @@ func (p *BufferPool) Release() {
177177
if p.pool[i].b != nil {
178178
panic(errors.AssertionFailedf("Release called on a BufferPool with in-use buffers"))
179179
}
180-
cache.Free(p.pool[i].v)
180+
v := p.pool[i].v
181181
p.pool[i].v = nil
182+
cache.Free(v)
182183
}
183184
p.pool = p.pool[:0]
184185
}

sstable/colblk/data_block.go

Lines changed: 10 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -802,20 +802,22 @@ const _ uint = block.MetadataSize - uint(dataBlockDecoderSize) - KeySeekerMetada
802802

803803
// InitDataBlockMetadata initializes the metadata for a data block.
804804
func InitDataBlockMetadata(schema *KeySchema, md *block.Metadata, data []byte) (err error) {
805-
if uintptr(unsafe.Pointer(md))%8 != 0 {
806-
return errors.AssertionFailedf("metadata is not 8-byte aligned")
805+
type blockDecoderAndKeySeekerMetadata struct {
806+
d DataBlockDecoder
807+
// Pad to ensure KeySeekerMetadata is 8-byte aligned.
808+
_ [dataBlockDecoderSize - unsafe.Sizeof(DataBlockDecoder{})]byte
809+
keySchemaMeta KeySeekerMetadata
807810
}
808-
d := (*DataBlockDecoder)(unsafe.Pointer(md))
811+
metadatas := block.CastMetadataZero[blockDecoderAndKeySeekerMetadata](md)
809812
// Initialization can panic; convert panics to corruption errors (so higher
810813
// layers can add file number and offset information).
811814
defer func() {
812815
if r := recover(); r != nil {
813816
err = base.CorruptionErrorf("error initializing data block metadata: %v", r)
814817
}
815818
}()
816-
d.Init(schema, data)
817-
keySchemaMeta := (*KeySeekerMetadata)(unsafe.Pointer(&md[dataBlockDecoderSize]))
818-
schema.InitKeySeekerMetadata(keySchemaMeta, d)
819+
metadatas.d.Init(schema, data)
820+
schema.InitKeySeekerMetadata(&metadatas.keySchemaMeta, &metadatas.d)
819821
return nil
820822
}
821823

@@ -824,10 +826,7 @@ const _ uint = block.MetadataSize - uint(unsafe.Sizeof(IndexBlockDecoder{}))
824826

825827
// InitIndexBlockMetadata initializes the metadata for an index block.
826828
func InitIndexBlockMetadata(md *block.Metadata, data []byte) (err error) {
827-
if uintptr(unsafe.Pointer(md))%8 != 0 {
828-
return errors.AssertionFailedf("metadata is not 8-byte aligned")
829-
}
830-
d := (*IndexBlockDecoder)(unsafe.Pointer(md))
829+
d := block.CastMetadataZero[IndexBlockDecoder](md)
831830
// Initialization can panic; convert panics to corruption errors (so higher
832831
// layers can add file number and offset information).
833832
defer func() {
@@ -844,10 +843,7 @@ const _ uint = block.MetadataSize - uint(unsafe.Sizeof(KeyspanDecoder{}))
844843

845844
// InitKeyspanBlockMetadata initializes the metadata for a rangedel or range key block.
846845
func InitKeyspanBlockMetadata(md *block.Metadata, data []byte) (err error) {
847-
if uintptr(unsafe.Pointer(md))%8 != 0 {
848-
return errors.AssertionFailedf("metadata is not 8-byte aligned")
849-
}
850-
d := (*KeyspanDecoder)(unsafe.Pointer(md))
846+
d := block.CastMetadataZero[KeyspanDecoder](md)
851847
// Initialization can panic; convert panics to corruption errors (so higher
852848
// layers can add file number and offset information).
853849
defer func() {

0 commit comments

Comments
 (0)