Skip to content

Commit cc9fe7f

Browse files
craig[bot]iskettaneh
andcommitted
143583: storage: test that batch commit doesn't touch SSTs r=iskettaneh a=iskettaneh This commit adds simple test where it injects errors on any operation touching any SSTable file before attempting to commit a batch, and asserts that the batch can commit successfully without needing any SSTable. References: #143135 Release note: None 143599: batcheval: prevent excising non-user keys r=iskettaneh a=iskettaneh This commit performs some checks in the Excise command evaluation. It prevents the command to run against non-user keys. Excising non-global keys is not allowed as it would leave the replica in a bad state. For example, we don't want to allow excising a range descriptor. References: #143135 Release note: None Co-authored-by: Ibrahim Kettaneh <[email protected]>
3 parents d2acf29 + 5f86dbe + 8524b83 commit cc9fe7f

File tree

5 files changed

+185
-3
lines changed

5 files changed

+185
-3
lines changed

pkg/kv/kvserver/batcheval/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -112,6 +112,7 @@ go_test(
112112
"cmd_delete_range_gchint_test.go",
113113
"cmd_delete_range_test.go",
114114
"cmd_end_transaction_test.go",
115+
"cmd_excise_test.go",
115116
"cmd_export_test.go",
116117
"cmd_get_test.go",
117118
"cmd_is_span_empty_test.go",

pkg/kv/kvserver/batcheval/cmd_excise.go

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ import (
1414
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
1515
"github.com/cockroachdb/cockroach/pkg/roachpb"
1616
"github.com/cockroachdb/cockroach/pkg/storage"
17+
"github.com/cockroachdb/errors"
1718
)
1819

1920
func init() {
@@ -22,10 +23,33 @@ func init() {
2223

2324
// EvalExcise evaluates a Excise command.
2425
func EvalExcise(
25-
_ context.Context, _ storage.ReadWriter, cArgs CommandArgs, _ kvpb.Response,
26+
_ context.Context, _ storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response,
2627
) (result.Result, error) {
2728
args := cArgs.Args.(*kvpb.ExciseRequest)
28-
start, end := storage.MVCCKey{Key: args.Key}, storage.MVCCKey{Key: args.EndKey}
29+
start, end := args.Key, args.EndKey
30+
31+
// Verify that the start and end keys are for global key space. Excising
32+
// non-global keys is not allowed as it would leave the replica in a bad
33+
// state. For example, we don't want to allow excising a range descriptor.
34+
rStart, err := keys.Addr(start)
35+
if err != nil {
36+
return result.Result{}, err
37+
}
38+
39+
rEnd, err := keys.Addr(end)
40+
if err != nil {
41+
return result.Result{}, err
42+
}
43+
44+
if !start.Equal(rStart.AsRawKey()) {
45+
return result.Result{},
46+
errors.Errorf("excise can only be run against global keys, but found start key: %s", start)
47+
}
48+
49+
if !end.Equal(rEnd.AsRawKey()) {
50+
return result.Result{},
51+
errors.Errorf("excise can only be run against global keys, but found end key: %s", end)
52+
}
2953

3054
// Since we can't know the exact range stats, mark it as an estimate.
3155
cArgs.Stats.ContainsEstimates++
@@ -36,7 +60,7 @@ func EvalExcise(
3660
return result.Result{
3761
Replicated: kvserverpb.ReplicatedEvalResult{
3862
Excise: &kvserverpb.ReplicatedEvalResult_Excise{
39-
Span: roachpb.Span{Key: start.Key, EndKey: end.Key},
63+
Span: roachpb.Span{Key: start, EndKey: end},
4064
LockTableSpan: roachpb.Span{Key: ltStart, EndKey: ltEnd},
4165
},
4266
},
Lines changed: 95 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,95 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package batcheval_test
7+
8+
import (
9+
"context"
10+
"testing"
11+
12+
"github.com/cockroachdb/cockroach/pkg/keys"
13+
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
14+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
15+
"github.com/cockroachdb/cockroach/pkg/roachpb"
16+
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
17+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
18+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
19+
"github.com/cockroachdb/cockroach/pkg/util/log"
20+
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
21+
"github.com/stretchr/testify/require"
22+
)
23+
24+
// TestExciseEval tests basic Excise evaluation.
25+
func TestExciseEval(t *testing.T) {
26+
defer leaktest.AfterTest(t)()
27+
defer log.Scope(t).Close(t)
28+
29+
ctx := context.Background()
30+
clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Now()))
31+
evalCtx := (&batcheval.MockEvalCtx{Clock: clock}).EvalContext()
32+
33+
testCases := []struct {
34+
name string
35+
startKey roachpb.Key
36+
endKey roachpb.Key
37+
expectErr string
38+
}{
39+
{
40+
// Valid range.
41+
startKey: roachpb.Key("a"),
42+
endKey: roachpb.Key("z"),
43+
expectErr: "",
44+
},
45+
{
46+
// Only the start key is a non-user key.
47+
startKey: keys.RangeDescriptorKey(roachpb.RKey("a")),
48+
endKey: roachpb.Key("z"),
49+
expectErr: "excise can only be run against global keys",
50+
},
51+
{
52+
// Only the end key is a non-user key.
53+
startKey: roachpb.Key("a"),
54+
endKey: keys.RangeDescriptorKey(roachpb.RKey("z")),
55+
expectErr: "excise can only be run against global keys",
56+
},
57+
{
58+
// Both keys are non-user keys.
59+
startKey: keys.RangeDescriptorKey(roachpb.RKey("a")),
60+
endKey: keys.RangeDescriptorKey(roachpb.RKey("z")),
61+
expectErr: "excise can only be run against global keys",
62+
},
63+
}
64+
65+
for _, tc := range testCases {
66+
resp := &kvpb.ExciseResponse{}
67+
res, err := batcheval.EvalExcise(ctx, nil, batcheval.CommandArgs{
68+
EvalCtx: evalCtx,
69+
Stats: &enginepb.MVCCStats{},
70+
Args: &kvpb.ExciseRequest{
71+
RequestHeader: kvpb.RequestHeader{
72+
Key: tc.startKey,
73+
EndKey: tc.endKey,
74+
},
75+
},
76+
}, resp)
77+
78+
// If there are no errors, we expect the result to be populated.
79+
if tc.expectErr == "" {
80+
require.NoError(t, err)
81+
82+
userSpan := roachpb.Span{Key: tc.startKey, EndKey: tc.endKey}
83+
ltStart, _ := keys.LockTableSingleKey(tc.startKey, nil)
84+
ltEnd, _ := keys.LockTableSingleKey(tc.endKey, nil)
85+
LockTableSpan := roachpb.Span{Key: ltStart, EndKey: ltEnd}
86+
87+
require.NotNil(t, res.Replicated.Excise)
88+
require.Equal(t, userSpan, res.Replicated.Excise.Span)
89+
require.Equal(t, LockTableSpan, res.Replicated.Excise.LockTableSpan)
90+
} else {
91+
require.Regexp(t, tc.expectErr, err)
92+
require.Nil(t, res.Replicated.Excise)
93+
}
94+
}
95+
}

pkg/storage/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -217,6 +217,7 @@ go_test(
217217
"@com_github_cockroachdb_pebble//sstable",
218218
"@com_github_cockroachdb_pebble//sstable/block",
219219
"@com_github_cockroachdb_pebble//vfs",
220+
"@com_github_cockroachdb_pebble//vfs/errorfs",
220221
"@com_github_cockroachdb_redact//:redact",
221222
"@com_github_kr_pretty//:pretty",
222223
"@com_github_stretchr_testify//assert",

pkg/storage/batch_test.go

Lines changed: 61 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"fmt"
1313
"reflect"
1414
"strconv"
15+
"strings"
1516
"sync/atomic"
1617
"testing"
1718

@@ -27,6 +28,8 @@ import (
2728
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
2829
"github.com/cockroachdb/errors"
2930
"github.com/cockroachdb/pebble"
31+
"github.com/cockroachdb/pebble/vfs"
32+
"github.com/cockroachdb/pebble/vfs/errorfs"
3033
"github.com/stretchr/testify/assert"
3134
"github.com/stretchr/testify/require"
3235
)
@@ -1010,3 +1013,61 @@ func TestBatchReader(t *testing.T) {
10101013
require.False(t, r.Next())
10111014
require.NoError(t, r.Error())
10121015
}
1016+
1017+
// TestBatchCommitDoesntTouchSST tests that committing a writeBatch doesn't
1018+
// touch SST files.
1019+
func TestBatchCommitDoesntTouchSST(t *testing.T) {
1020+
defer leaktest.AfterTest(t)()
1021+
defer log.Scope(t).Close(t)
1022+
1023+
// Create an atomic variable that will cause an error when SST operations are
1024+
// performed.
1025+
var failSSTOps atomic.Bool
1026+
1027+
// Create a custom injector that blocks SST operations when failSSTOps is
1028+
// true.
1029+
injector := errorfs.InjectorFunc(func(op errorfs.Op) error {
1030+
if strings.Contains(op.Path, ".sst") && failSSTOps.Load() {
1031+
return errors.Newf("blocking SST operation: %+v", op)
1032+
}
1033+
return nil
1034+
})
1035+
1036+
// Create the wrapped filesystem, and create a db.
1037+
memFS := vfs.NewMem()
1038+
wrappedFS := errorfs.Wrap(memFS, injector)
1039+
env := mustInitTestEnv(t, wrappedFS, "")
1040+
db, err := Open(context.Background(), env, cluster.MakeClusterSettings())
1041+
require.NoError(t, err)
1042+
defer db.Close()
1043+
1044+
// Initialize the db with some data.
1045+
initBatch := db.NewBatch()
1046+
defer initBatch.Close()
1047+
1048+
// Perform some operations.
1049+
require.NoError(t, initBatch.PutUnversioned(mvccKey("key1").Key, []byte("val1")))
1050+
require.NoError(t, initBatch.PutUnversioned(mvccKey("key2").Key, []byte("val2")))
1051+
require.NoError(t, initBatch.PutUnversioned(mvccKey("key3").Key, []byte("val3")))
1052+
require.NoError(t, initBatch.PutUnversioned(mvccKey("key4").Key, []byte("val4")))
1053+
require.NoError(t, initBatch.Commit(true /* sync */))
1054+
1055+
// Force a flush to create an SST file.
1056+
require.NoError(t, db.Flush())
1057+
1058+
// Create a new batch for testing.
1059+
testingBatch := db.NewBatch()
1060+
defer testingBatch.Close()
1061+
1062+
// Perform some operations.
1063+
require.Equal(t, []byte("val1"), mvccGetRaw(t, testingBatch, mvccKey("key1")))
1064+
require.Equal(t, []byte(nil), mvccGetRaw(t, testingBatch, mvccKey("non-existent-key")))
1065+
_, err = Scan(context.Background(), testingBatch, localMax, roachpb.KeyMax, 0)
1066+
require.NoError(t, err)
1067+
require.NoError(t, testingBatch.ClearUnversioned(mvccKey("key4").Key, ClearOptions{}))
1068+
1069+
// Before committing, enable SST operation errors and make sure the commit
1070+
// succeeds.
1071+
failSSTOps.Store(true)
1072+
require.NoError(t, testingBatch.Commit(true /* sync */))
1073+
}

0 commit comments

Comments
 (0)