Skip to content

Commit

Permalink
backupccl: avoid over-shrinking memory monitor
Browse files Browse the repository at this point in the history
This change updates DecryptFile to optionally use a memory monitor and
adjusts the relevant calling functions.

Previously, the readManifest function that used DecryptFile could
result in shrinking too many bytes from the bound account:

```
ERROR: a panic has occurred!
root: no bytes in account to release, current 29571, free 30851
(1) attached stack trace
  -- stack trace:
  | runtime.gopanic
  | 	GOROOT/src/runtime/panic.go:1038
  | [...repeated from below...]
Wraps: (2) attached stack trace
  -- stack trace:
  | github.com/cockroachdb/cockroach/pkg/util/log/logcrash.ReportOrPanic
  | 	github.com/cockroachdb/cockroach/pkg/util/log/logcrash/crash_reporting.go:374
  | github.com/cockroachdb/cockroach/pkg/util/mon.(*BoundAccount).Shrink
  | 	github.com/cockroachdb/cockroach/pkg/util/mon/bytes_usage.go:709
  | github.com/cockroachdb/cockroach/pkg/ccl/backupccl.(*restoreResumer).doResume.func1
  | 	github.com/cockroachdb/cockroach/pkg/ccl/backupccl/pkg/ccl/backupccl/restore_job.go:1244
```

This was the result of us freeing `cap(descBytes)` despite the fact
that we never accounted for the fact that `descBytes` had been
re-assigned after the decryption without accounting for changes in the
capacity used by the old vs new buffer. That is, we called mem.Grow
with the capcity of the old buffer, but mem.Shrink with the capacity
of the new buffer.

We generally expect that the size of encrypted data to be larger than
the size of the plaintext data, so in most cases, the existing code
would work without error because it was freeing an amount smaller than
the original allocation.

However, while the plaintext data is smaller than the encrypted data,
that doesn't mean that the _capacity of the slice holding the
plaintext data_ is smaller than the _capacity of the slice holding the
encrypted data_.

The slice holding the encrypted data was created with mon.ReadAll
which has slice growth strategy of doubling the size until it reaches
8MB. The slice holding the plaintext data was created by
ioutil.ReadAll that defers to appends slice growth strategy, which
differs from that used in mon.ReadAll.

In the current implementations, for byte sizes around 30k, the
capacity of the buffer create by append's strategy is larger than that
created by mon.ReadAll despite the len of the buffer being smaller:

    before decrypt: len(descBytes) = 27898, cap(descBytes) = 32768
     after decrypt: len(descBytes) = 27862, cap(descBytes) = 40960

We could have fixed this by simply adjusting the memory account by the
difference. Instead, I've opted to thread the memory monitor into
DecryptFile to better account for the fact that we technically do hold
2 copies of the data during this decryption.

Fixes #79488

Release note: None
  • Loading branch information
stevendanna committed Apr 6, 2022
1 parent 6aff980 commit 30419a8
Show file tree
Hide file tree
Showing 6 changed files with 57 additions and 10 deletions.
37 changes: 37 additions & 0 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7992,6 +7992,43 @@ func TestCleanupDoesNotDeleteParentsWithChildObjects(t *testing.T) {
})
}

func TestReadBackupManifestMemoryMonitoring(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

dir, dirCleanupFn := testutils.TempDir(t)
defer dirCleanupFn()

st := cluster.MakeTestingClusterSettings()
storage, err := cloud.ExternalStorageFromURI(ctx,
"nodelocal://0/test",
base.ExternalIODirConfig{},
st,
blobs.TestBlobServiceClient(dir),
security.RootUserName(), nil, nil)
require.NoError(t, err)

m := mon.NewMonitor("test-monitor", mon.MemoryResource, nil, nil, 0, 0, st)
m.Start(ctx, nil, mon.MakeStandaloneBudget(128<<20))
mem := m.MakeBoundAccount()
encOpts := &jobspb.BackupEncryptionOptions{
Mode: jobspb.EncryptionMode_Passphrase,
Key: storageccl.GenerateKey([]byte("passphrase"), []byte("sodium")),
}
desc := &BackupManifest{}
magic := 5500
for i := 0; i < magic; i++ {
desc.Files = append(desc.Files, BackupManifest_File{Path: fmt.Sprintf("%d-file-%d", i, i)})
}
require.NoError(t, writeBackupManifest(ctx, st, storage, "testmanifest", encOpts, desc))
_, sz, err := readBackupManifest(ctx, &mem, storage, "testmanifest", encOpts)
require.NoError(t, err)
mem.Shrink(ctx, sz)
mem.Close(ctx)
}

func TestManifestTooNew(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
11 changes: 7 additions & 4 deletions pkg/ccl/backupccl/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,10 +324,12 @@ func readManifest(
if err != nil {
return BackupManifest{}, 0, err
}
descBytes, err = storageccl.DecryptFile(descBytes, encryptionKey)
plaintextBytes, err := storageccl.DecryptFile(ctx, descBytes, encryptionKey, mem)
if err != nil {
return BackupManifest{}, 0, err
}
mem.Shrink(ctx, int64(cap(descBytes)))
descBytes = plaintextBytes
}

if isGZipped(descBytes) {
Expand Down Expand Up @@ -377,7 +379,6 @@ func readManifest(
t.ModificationTime = hlc.Timestamp{WallTime: 1}
}
}

return backupManifest, approxMemSize, nil
}

Expand Down Expand Up @@ -407,10 +408,12 @@ func readBackupPartitionDescriptor(
if err != nil {
return BackupPartitionDescriptor{}, 0, err
}
descBytes, err = storageccl.DecryptFile(descBytes, encryptionKey)
plaintextData, err := storageccl.DecryptFile(ctx, descBytes, encryptionKey, mem)
if err != nil {
return BackupPartitionDescriptor{}, 0, err
}
mem.Shrink(ctx, int64(cap(descBytes)))
descBytes = plaintextData
}

if isGZipped(descBytes) {
Expand Down Expand Up @@ -461,7 +464,7 @@ func readTableStatistics(
if err != nil {
return nil, err
}
statsBytes, err = storageccl.DecryptFile(statsBytes, encryptionKey)
statsBytes, err = storageccl.DecryptFile(ctx, statsBytes, encryptionKey, nil /* mm */)
if err != nil {
return nil, err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/storageccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/settings",
"//pkg/storage",
"//pkg/util/ioctx",
"//pkg/util/mon",
"//pkg/util/retry",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//sstable",
Expand Down
9 changes: 7 additions & 2 deletions pkg/ccl/storageccl/encryption.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ package storageccl

import (
"bytes"
"context"
"crypto/aes"
"crypto/cipher"
crypto_rand "crypto/rand"
Expand All @@ -19,6 +20,8 @@ import (
"io/ioutil"
"os"

"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
Expand Down Expand Up @@ -185,12 +188,14 @@ func (e *encWriter) flush() error {
// DecryptFile decrypts a file encrypted by EncryptFile, using the supplied key
// and reading the IV from a prefix of the file. See comments on EncryptFile
// for intended usage, and see DecryptFile
func DecryptFile(ciphertext, key []byte) ([]byte, error) {
func DecryptFile(
ctx context.Context, ciphertext, key []byte, mm *mon.BoundAccount,
) ([]byte, error) {
r, err := decryptingReader(bytes.NewReader(ciphertext), key)
if err != nil {
return nil, err
}
return ioutil.ReadAll(r.(io.Reader))
return mon.ReadAll(ctx, ioctx.ReaderAdapter(r.(io.Reader)), mm)
}

type decryptReader struct {
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/storageccl/encryption_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ package storageccl

import (
"bytes"
"context"
"fmt"
"io"
"io/ioutil"
Expand Down Expand Up @@ -43,7 +44,7 @@ func TestEncryptDecrypt(t *testing.T) {
require.NoError(t, err)
require.True(t, AppearsEncrypted(ciphertext), "cipher text should appear encrypted")

decrypted, err := DecryptFile(ciphertext, key)
decrypted, err := DecryptFile(context.Background(), ciphertext, key, nil /* mm */)
require.NoError(t, err)
require.Equal(t, plaintext, decrypted)
})
Expand All @@ -53,7 +54,7 @@ func TestEncryptDecrypt(t *testing.T) {
})

t.Run("helpful error on bad input", func(t *testing.T) {
_, err := DecryptFile([]byte("a"), key)
_, err := DecryptFile(context.Background(), []byte("a"), key, nil /* mm */)
require.EqualError(t, err, "file does not appear to be encrypted")
})

Expand Down Expand Up @@ -162,7 +163,7 @@ func TestEncryptDecrypt(t *testing.T) {
plaintext := randutil.RandBytes(rng, rng.Intn(1024*32))
ciphertext, err := EncryptFile(plaintext, key)
require.NoError(t, err)
decrypted, err := DecryptFile(ciphertext, key)
decrypted, err := DecryptFile(context.Background(), ciphertext, key, nil /* mm */)
require.NoError(t, err)
if len(plaintext) == 0 {
require.Equal(t, len(plaintext), len(decrypted))
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/storageccl/external_sst_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func ExternalSSTReader(
return nil, err
}
if encryption != nil {
content, err = DecryptFile(content, encryption.Key)
content, err = DecryptFile(ctx, content, encryption.Key, nil /* mm */)
if err != nil {
return nil, err
}
Expand Down

0 comments on commit 30419a8

Please sign in to comment.