Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

deps: pebble crashes when using zstd #1706

Closed
nicktrav opened this issue May 20, 2022 · 10 comments · Fixed by #2668 or #3416
Closed

deps: pebble crashes when using zstd #1706

nicktrav opened this issue May 20, 2022 · 10 comments · Fixed by #2668 or #3416

Comments

@nicktrav
Copy link
Contributor

When upgrading the version of zstd, Pebble crashes at runtime with the following:

panic: pebble: Value has already been added to the cache: refs=0 [recovered]
        panic: pebble: Value has already been added to the cache: refs=0

goroutine 208 [running]:
testing.tRunner.func1.2({0x44b2ac0, 0xc000300100})
        /nix/store/0bh8m4vqb7c7n4sn73ka53yn45yj3zd0-go-1.18.2/share/go/src/testing/testing.go:1389 +0x24e
testing.tRunner.func1()
        /nix/store/0bh8m4vqb7c7n4sn73ka53yn45yj3zd0-go-1.18.2/share/go/src/testing/testing.go:1392 +0x39f
panic({0x44b2ac0, 0xc000300100})
        /nix/store/0bh8m4vqb7c7n4sn73ka53yn45yj3zd0-go-1.18.2/share/go/src/runtime/panic.go:838 +0x207
github.com/cockroachdb/pebble/internal/cache.(*shard).Set(0xc0000d3b68?, 0x4310c25?, 0x600001710040?, 0x1b?, 0x16?)
        /Users/nickt/Development/pebble/internal/cache/clockpro.go:136 +0x365
github.com/cockroachdb/pebble/internal/cache.(*Cache).Set(0x5772c48?, 0x600001710007?, 0x600001710060?, 0x17?, 0x1c?)
        /Users/nickt/Development/pebble/internal/cache/clockpro.go:750 +0x46
github.com/cockroachdb/pebble/sstable.(*Reader).readBlock(0xc0000dd8c0, {0x0?, 0x17?}, 0x0, 0xc00019e5f8)
        /Users/nickt/Development/pebble/sstable/reader.go:2372 +0x617
github.com/cockroachdb/pebble/sstable.(*singleLevelIterator).readBlockWithStats(0xc00019e380, {0x2?, 0x2?}, 0x430699f?)
        /Users/nickt/Development/pebble/sstable/reader.go:406 +0x2f
github.com/cockroachdb/pebble/sstable.(*singleLevelIterator).loadBlock(0xc00019e380)
        /Users/nickt/Development/pebble/sstable/reader.go:388 +0x22a
github.com/cockroachdb/pebble/sstable.(*singleLevelIterator).firstInternal(0xc00019e380)
        /Users/nickt/Development/pebble/sstable/reader.go:786 +0xe7
github.com/cockroachdb/pebble/sstable.(*singleLevelIterator).First(0x0?)
        /Users/nickt/Development/pebble/sstable/reader.go:768 +0x27
github.com/cockroachdb/pebble/sstable.(*compactionIterator).First(0xc0003c02a0)
        /Users/nickt/Development/pebble/sstable/reader.go:1127 +0x58
github.com/cockroachdb/pebble/sstable.testBytesIteratedWithCompression(0xc000511a00, 0x40551ff?, 0x1, {0xc00002c8a0, 0x5, 0x100000001?}, {0xc0000d3f38, 0x5, 0x3351bd6b84e6e?})
        /Users/nickt/Development/pebble/sstable/reader_test.go:495 +0x223
github.com/cockroachdb/pebble/sstable.TestBytesIterated.func3(0x0?)
        /Users/nickt/Development/pebble/sstable/reader_test.go:531 +0x85
testing.tRunner(0xc000511a00, 0xc000088060)
        /nix/store/0bh8m4vqb7c7n4sn73ka53yn45yj3zd0-go-1.18.2/share/go/src/testing/testing.go:1439 +0x102
created by testing.(*T).Run
        /nix/store/0bh8m4vqb7c7n4sn73ka53yn45yj3zd0-go-1.18.2/share/go/src/testing/testing.go:1486 +0x35f

This blocks upgrading the version of zstd that we use.

See #1704 for the diff and reproducer. Note that the issue may have been introduced in a version prior to v1.5.2 (Pebble at the time of writing is at version v1.4.5). I only tested against the latest.

@nicktrav
Copy link
Contributor Author

Looks like v1.5.2 is currently marked as "pre-release", so perhaps not an official release yet. But given the Git tag exists, go get is going to pull in that version.

I also tried with v1.5.0 (the version prior), and the test failure above is no longer an issue. However, there are some diffs in the zstd test fixtures that we'll need to update.

@nicktrav
Copy link
Contributor Author

nicktrav commented Jul 8, 2022

Took another look at this with v1.5.2+patch1, and it still appears there's some things we'll need to dig into before upgrading.

@nicktrav nicktrav changed the title deps: pebble crashes when using zstd v1.5.2 deps: pebble crashes when using zstd May 15, 2023
@nicktrav
Copy link
Contributor Author

The issue is still there in v1.5.5.

@jbowens

This comment was marked as abuse.

@jbowens
Copy link
Collaborator

jbowens commented Jun 15, 2023

Might be time to address this: #999 (comment)

It has a corresponding TODO:

// TODO(peter): It may be better to separate the allocation of the value and
// the buffer in order to reduce internal fragmentation in malloc. If the
// buffer is right at a power of 2, adding valueSize might push the
// allocation over into the next larger size.

@jbowens
Copy link
Collaborator

jbowens commented Jun 16, 2023

There are a whole host of issues going on in the zstd codepath.

First, in this error case we swallow err and still return decoded. This can cause a whole host of memory corruption.

if _, err := decompressInto(blockType, b, decodedBuf); err != nil {
cache.Free(decoded)
}

Second, I think the swallowing of the err is hiding this error:

if len(result) != 0 && (len(result) != len(buf) || &result[0] != &buf[0]) {
return nil, base.CorruptionErrorf("pebble/table: decompressed into unexpected buffer: %p != %p",
errors.Safe(result), errors.Safe(buf))
}

Since we're decoding into manually-managed memory, we require zstd to use our buffer. But recent versions of the zstd library will use their own buffer whenever the provided buffer is not big enough to hold the worst case decoded size:

https://github.com/DataDog/zstd/blob/5f14d6af117fa84b37f99d4cde775e6039be6d3b/zstd.go#L97-L101

We manually allocated memory for the decoded value using the exact value we wrote in a prefix to the block, so there's no chance the zstd library will use our buffer.

Additionally at encode time, we appear to clobber the varint prefix we already wrote by passing compressedBuf[:varIntLen] as the destination:

buf := bytes.NewBuffer(compressedBuf[:varIntLen])
writer := zstd.NewWriterLevel(buf, 3)
writer.Write(b)
writer.Close()
return buf.Bytes()

Edit: I don't think this last one is an issue, although it appears to be relying on undocumented behavior (NewBuffer says you should pass it a zero-length slice with the preallocated capacity. It says nothing of what happens to the contents if passed a non-empty slice.)

@jbowens
Copy link
Collaborator

jbowens commented Jun 16, 2023

Opened DataDog/zstd#130 to allow us to decompress directly into a perflectly-sized block-cache buffer.

jbowens added a commit to jbowens/pebble that referenced this issue Jun 22, 2023
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit that referenced this issue Jun 23, 2023
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close #1706.
@jbowens
Copy link
Collaborator

jbowens commented Jun 26, 2023

The commit upgrading was reverted due to issues with Bazel building zstd. Reopening.

@jbowens jbowens reopened this Jun 26, 2023
@pkieltyka
Copy link

hi all, just wondering if any updates on cockroachdb size to update the dependency? would be nice to have zstd working with pebble as we're also hitting the panic with current version

@jbowens
Copy link
Collaborator

jbowens commented Oct 16, 2023

cockroachdb/cockroach#105568 is the tracking issue, and there's been no progress yet.

jbowens added a commit to jbowens/pebble that referenced this issue Mar 15, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit to jbowens/pebble that referenced this issue Mar 19, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit to jbowens/pebble that referenced this issue Mar 19, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit to jbowens/pebble that referenced this issue Mar 19, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit to jbowens/pebble that referenced this issue Mar 19, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit to jbowens/pebble that referenced this issue Mar 19, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close cockroachdb#1706.
jbowens added a commit that referenced this issue Mar 20, 2024
Upgrade zstd to v1.5.6. Due to changes in the mechanics of zstd.Decompress
function, this required a change to using a new zstd.DecompressInto entrypoint
that guarantees it'll deserialize into the provided destination buffer, never
allocating a new buffer.

Close #1706.
@jbowens jbowens moved this to Done in [Deprecated] Storage Jun 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
Archived in project
3 participants