Skip to content

Commit

Permalink
backupccl: avoid splitting if the split point might be unsafe
Browse files Browse the repository at this point in the history
Restore may use unsafe keys as split points, which may cause unsafe splits
between column families, which may cause SQL to fail when reading the row, or
worse, return wrong resutls.

This commit avoids splitting on keys that might be unsafe.

See the issue for more info.

Epic: none
Informs: #109483

Release note: None.
  • Loading branch information
lidorcarmel committed Aug 30, 2023
1 parent 95dcf06 commit b30c7b0
Show file tree
Hide file tree
Showing 4 changed files with 102 additions and 20 deletions.
68 changes: 68 additions & 0 deletions pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -265,3 +267,69 @@ func makeTestingGenerativeSplitAndScatterSpec(
UseSimpleImportSpans: false,
}
}

type mockAdminSplitAndScatterer struct {
t *testing.T
lastSplitKey string
}

func (s *mockAdminSplitAndScatterer) AdminSplit(
_ context.Context, splitKey interface{}, _ hlc.Timestamp, _ ...roachpb.Key,
) error {
k := splitKey.(roachpb.Key)
strKey := k.String()
require.NotEqual(s.t, "NULL", strKey[len(strKey)-4:], strKey)
s.lastSplitKey = strKey
return nil
}

func (s *mockAdminSplitAndScatterer) AdminScatter(
ctx context.Context, key roachpb.Key, maxSize int64,
) (*kvpb.AdminScatterResponse, error) {
panic("unimplemented")
}

func (s *mockAdminSplitAndScatterer) Clock() *hlc.Clock {
return hlc.NewClockForTesting(nil)
}

func (s *mockAdminSplitAndScatterer) NonTransactionalSender() kv.Sender {
panic("unimplemented")
}

// TestUnsafeSplit verifies dbSplitAndScatterer.split() doesn't call
// AdminSplit() for a key that was generated by Key.Next(). See issue #109483.
func TestUnsafeSplit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

rekeys := []execinfrapb.TableRekey{
{
OldID: uint32(42),
NewDesc: mustMarshalDesc(t, &descpb.TableDescriptor{ID: 43}),
},
}
kr, err := MakeKeyRewriterFromRekeys(keys.SystemSQLCodec, rekeys, nil, false)
require.NoError(t, err)
mockSplitter := &mockAdminSplitAndScatterer{t: t}
splitter := makeSplitAndScatterer(mockSplitter, kr)
k := rowenc.MakeIndexKeyPrefix(keys.SystemSQLCodec, 42, 1)

key := roachpb.Key(k)
nextKey := key.Next()

// Splitting on key should work.
mockSplitter.lastSplitKey = ""
require.Equal(t, "/Table/42/1", key.String())
require.NoError(t, splitter.split(ctx, keys.SystemSQLCodec, key))

// 42 was rekeyed as 43 and the index id was dropped.
require.Equal(t, "/Table/43", mockSplitter.lastSplitKey)

// Splitting on nextKey should be a noop.
mockSplitter.lastSplitKey = ""
require.Equal(t, "/Table/42/1/NULL", nextKey.String())
require.NoError(t, splitter.split(ctx, keys.SystemSQLCodec, nextKey))
require.Equal(t, "", mockSplitter.lastSplitKey)
}
34 changes: 24 additions & 10 deletions pkg/ccl/backupccl/split_and_scatter_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -61,18 +62,27 @@ func (n noopSplitAndScatterer) scatter(
return n.scatterNode, nil
}

// adminSplitAndScatterer is used by dbSplitAndScatterer to run the actual KV
// commands to split and scatter.
type adminSplitAndScatterer interface {
AdminSplit(ctx context.Context, splitKey interface{}, expirationTime hlc.Timestamp, predicateKeys ...roachpb.Key) error
AdminScatter(ctx context.Context, key roachpb.Key, maxSize int64) (*kvpb.AdminScatterResponse, error)
Clock() *hlc.Clock
NonTransactionalSender() kv.Sender
}

// dbSplitAndScatter is the production implementation of this processor's
// scatterer. It actually issues the split and scatter requests against the KV
// layer.
type dbSplitAndScatterer struct {
db *kv.DB
kr *KeyRewriter
sas adminSplitAndScatterer
kr *KeyRewriter
}

var _ splitAndScatterer = dbSplitAndScatterer{}

func makeSplitAndScatterer(db *kv.DB, kr *KeyRewriter) splitAndScatterer {
return dbSplitAndScatterer{db: db, kr: kr}
func makeSplitAndScatterer(sas adminSplitAndScatterer, kr *KeyRewriter) splitAndScatterer {
return dbSplitAndScatterer{sas: sas, kr: kr}
}

// split implements splitAndScatterer.
Expand All @@ -82,22 +92,26 @@ func (s dbSplitAndScatterer) split(
if s.kr == nil {
return errors.AssertionFailedf("KeyRewriter was not set when expected to be")
}
if s.db == nil {
if s.sas == nil {
return errors.AssertionFailedf("split and scatterer's database was not set when expected")
}

expirationTime := s.db.Clock().Now().Add(time.Hour.Nanoseconds(), 0)
expirationTime := s.sas.Clock().Now().Add(time.Hour.Nanoseconds(), 0)
newSplitKey, err := rewriteBackupSpanKey(codec, s.kr, splitKey)
if err != nil {
return err
}
if splitAt, err := keys.EnsureSafeSplitKey(newSplitKey); err != nil {
// Ignore the error, not all keys are table keys.
// The key might be corrupt, and therefore we cannot guarantee it is a valid
// split key. The restore can still continue without this split. This error
// is not expected after #109483 is fixed.
log.Errorf(ctx, "failed splitting at key %s err: %v", newSplitKey, err)
return nil
} else if len(splitAt) != 0 {
newSplitKey = splitAt
}
log.VEventf(ctx, 1, "presplitting new key %+v", newSplitKey)
if err := s.db.AdminSplit(ctx, newSplitKey, expirationTime); err != nil {
if err := s.sas.AdminSplit(ctx, newSplitKey, expirationTime); err != nil {
return errors.Wrapf(err, "splitting key %s", newSplitKey)
}

Expand All @@ -111,7 +125,7 @@ func (s dbSplitAndScatterer) scatter(
if s.kr == nil {
return 0, errors.AssertionFailedf("KeyRewriter was not set when expected to be")
}
if s.db == nil {
if s.sas == nil {
return 0, errors.AssertionFailedf("split and scatterer's database was not set when expected")
}

Expand Down Expand Up @@ -142,7 +156,7 @@ func (s dbSplitAndScatterer) scatter(
MaxSize: 1, // don't scatter non-empty ranges on resume.
}

res, pErr := kv.SendWrapped(ctx, s.db.NonTransactionalSender(), req)
res, pErr := kv.SendWrapped(ctx, s.sas.NonTransactionalSender(), req)
if pErr != nil {
// TODO(dt): typed error.
if !strings.Contains(pErr.String(), "existing range size") {
Expand Down
12 changes: 6 additions & 6 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -918,7 +918,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
// Strip tenant ID prefix to get a "SQL key" starting with a table ID.
sqlKey, _, err := DecodeTenantPrefix(key)
if err != nil {
return 0, errors.Errorf("%s: not a valid table key", key)
return 0, errors.Wrapf(err, "%s: not a valid table key, bad tenant prefix", key)
}
sqlN := len(sqlKey)

Expand All @@ -929,7 +929,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
}
tableIDLen, err := encoding.GetUvarintLen(sqlKey)
if err != nil {
return 0, err
return 0, errors.Wrapf(err, "%s: not a valid table key, bad table id", key)
}

// Check whether the prefix contains a valid IndexID after the TableID. Not
Expand All @@ -939,7 +939,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
}
indexIDLen, err := encoding.GetUvarintLen(sqlKey[tableIDLen:])
if err != nil {
return 0, err
return 0, errors.Wrapf(err, "%s: not a valid table key, bad index id", key)
}
// If the IndexID is the last part of the key, the entire key is the prefix.
if tableIDLen+indexIDLen == sqlN {
Expand All @@ -953,7 +953,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
colFamIDLenByte := sqlKey[sqlN-1:]
if encoding.PeekType(colFamIDLenByte) != encoding.Int {
// The last byte is not a valid column family ID suffix.
return 0, errors.Errorf("%s: not a valid table key", key)
return 0, errors.Errorf("%s: not a valid table key, bad column family id type", key)
}

// Strip off the column family ID suffix from the buf. The last byte of the
Expand All @@ -962,7 +962,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
// 0 (see the optimization in MakeFamilyKey).
_, colFamIDLen, err := encoding.DecodeUvarintAscending(colFamIDLenByte)
if err != nil {
return 0, err
return 0, errors.Wrapf(err, "%s: not a valid table key, bad column family id", key)
}
// Note how this next comparison (and by extension the code after it) is
// overflow-safe. There are more intuitive ways of writing this that aren't
Expand All @@ -976,7 +976,7 @@ func GetRowPrefixLength(key roachpb.Key) (int, error) {
// because EnsureSafeSplitKey can be called on keys that look like table
// keys but which do not have a column family ID length suffix (e.g by
// SystemConfig.ComputeSplitKey).
return 0, errors.Errorf("%s: malformed table key", key)
return 0, errors.Errorf("%s: not a valid table key, bad column family id len", key)
}
return n - int(colFamIDLen) - 1, nil
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/keys/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -680,21 +680,21 @@ func TestEnsureSafeSplitKey(t *testing.T) {
err string
}{
// Column ID suffix size is too large.
{es(1, 2, 5), "malformed table key"},
{es(1, 2, 5), "not a valid table key, bad column family id len"},
// The table ID is invalid.
{es(200)[:1], "insufficient bytes to decode uvarint value"},
// The column ID suffix is invalid.
{es(1, 200)[:2], "insufficient bytes to decode uvarint value"},
{es(1, 2, 200)[:3], "insufficient bytes to decode uvarint value"},
// Exercises a former overflow bug. We decode a uint(18446744073709551610) which, if cast
// to int carelessly, results in -6 for the column family length.
{encoding.EncodeVarintAscending(es(999, 2), 322434), "malformed table key"},
{encoding.EncodeVarintAscending(es(999, 2), 322434), "not a valid table key, bad column family id len"},
// Same test cases, but for tenant 5.
{e5(1, 2, 5), "malformed table key"},
{e5(1, 2, 5), "not a valid table key, bad column family id len"},
{e5(200)[:3], "insufficient bytes to decode uvarint value"},
{e5(1, 200)[:4], "insufficient bytes to decode uvarint value"},
{e5(1, 2, 200)[:5], "insufficient bytes to decode uvarint value"},
{encoding.EncodeVarintAscending(e5(999, 2), 322434), "malformed table key"},
{encoding.EncodeVarintAscending(e5(999, 2), 322434), "not a valid table key, bad column family id len"},
}
for i, d := range errorData {
_, err := EnsureSafeSplitKey(d.in)
Expand Down

0 comments on commit b30c7b0

Please sign in to comment.