Skip to content

Commit 6f3d557

Browse files
craig[bot]tbg
craig[bot]
andcommitted
Merge #37868
37868: storage: fix checksum version check r=ajwerner a=tbg The check was sitting at evaluation time, where it is useless. It needs to sit in the below-Raft code that actually computes the checksums. This flew under the radar for quite some time, but was found in: #37737 (comment) thanks to the aggressive consistency checks we run in version/mixed/nodes=3. Release note: None Co-authored-by: Tobias Schottdorf <[email protected]>
2 parents 8f42e0d + e13298a commit 6f3d557

10 files changed

+240
-152
lines changed

pkg/cmd/roachtest/version.go

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020
"fmt"
2121
"runtime"
2222
"strconv"
23+
"strings"
2324
"time"
2425

2526
"github.com/cockroachdb/cockroach/pkg/util/binfetcher"
@@ -105,9 +106,16 @@ func registerVersion(r *registry) {
105106
if err := rows.Close(); err != nil {
106107
return err
107108
}
108-
// Regression test for #37425.
109-
if err := c.CheckReplicaDivergenceOnDB(ctx, db); err != nil {
110-
return errors.Wrapf(err, "node %d", i)
109+
// Regression test for #37425. We can't run this in 2.1 because
110+
// 19.1 changed downstream-of-raft semantics for consistency
111+
// checks but unfortunately our versioning story for these
112+
// checks had been broken for a long time. See:
113+
//
114+
// https://github.com/cockroachdb/cockroach/issues/37737#issuecomment-496026918
115+
if !strings.HasPrefix(version, "2.") {
116+
if err := c.CheckReplicaDivergenceOnDB(ctx, db); err != nil {
117+
return errors.Wrapf(err, "node %d", i)
118+
}
111119
}
112120
}
113121
return nil

pkg/storage/api.pb.go

Lines changed: 11 additions & 9 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/storage/api.proto

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,8 @@ message CollectChecksumRequest {
4444
}
4545

4646
message CollectChecksumResponse {
47+
// The checksum is the sha512 hash of the requested computation. It is empty
48+
// if the computation failed.
4749
bytes checksum = 1;
4850
// snapshot is set if the roachpb.ComputeChecksumRequest had snapshot = true
4951
// and the response checksum is different from the request checksum.

pkg/storage/batcheval/cmd_compute_checksum.go

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/storage/engine"
2424
"github.com/cockroachdb/cockroach/pkg/storage/spanset"
2525
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
26-
"github.com/cockroachdb/cockroach/pkg/util/log"
2726
"github.com/cockroachdb/cockroach/pkg/util/uuid"
2827
)
2928

@@ -54,17 +53,12 @@ func ComputeChecksum(
5453
) (result.Result, error) {
5554
args := cArgs.Args.(*roachpb.ComputeChecksumRequest)
5655

57-
if args.Version != ReplicaChecksumVersion {
58-
log.Infof(ctx, "incompatible ComputeChecksum versions (server: %d, requested: %d)",
59-
ReplicaChecksumVersion, args.Version)
60-
return result.Result{}, nil
61-
}
62-
6356
reply := resp.(*roachpb.ComputeChecksumResponse)
6457
reply.ChecksumID = uuid.MakeV4()
6558

6659
var pd result.Result
6760
pd.Replicated.ComputeChecksum = &storagepb.ComputeChecksum{
61+
Version: args.Version,
6862
ChecksumID: reply.ChecksumID,
6963
SaveSnapshot: args.Snapshot,
7064
Mode: args.Mode,

pkg/storage/replica_consistency.go

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -401,12 +401,11 @@ func (r *Replica) getChecksum(ctx context.Context, id uuid.UUID) (ReplicaChecksu
401401
r.mu.RLock()
402402
c, ok = r.mu.checksums[id]
403403
r.mu.RUnlock()
404-
if !ok {
405-
return ReplicaChecksum{}, errors.Errorf("no map entry for checksum (ID = %s)", id)
406-
}
407-
if c.Checksum == nil {
408-
return ReplicaChecksum{}, errors.Errorf(
409-
"checksum is nil, most likely because the async computation could not be run (ID = %s)", id)
404+
// If the checksum wasn't found or the checksum could not be computed, error out.
405+
// The latter case can occur when there's a version mismatch or, more generally,
406+
// when the (async) checksum computation fails.
407+
if !ok || c.Checksum == nil {
408+
return ReplicaChecksum{}, errors.Errorf("no checksum found (ID = %s)", id)
410409
}
411410
return c, nil
412411
}
Lines changed: 62 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,62 @@
1+
// Copyright 2019 The Cockroach Authors.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
12+
// implied. See the License for the specific language governing
13+
// permissions and limitations under the License.
14+
15+
package storage
16+
17+
import (
18+
"context"
19+
"testing"
20+
21+
"github.com/cockroachdb/cockroach/pkg/roachpb"
22+
"github.com/cockroachdb/cockroach/pkg/storage/batcheval"
23+
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
24+
"github.com/cockroachdb/cockroach/pkg/testutils"
25+
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
26+
"github.com/cockroachdb/cockroach/pkg/util/stop"
27+
"github.com/cockroachdb/cockroach/pkg/util/uuid"
28+
"github.com/stretchr/testify/require"
29+
)
30+
31+
func TestReplicaChecksumVersion(t *testing.T) {
32+
defer leaktest.AfterTest(t)()
33+
34+
ctx := context.TODO()
35+
tc := testContext{}
36+
stopper := stop.NewStopper()
37+
defer stopper.Stop(ctx)
38+
tc.Start(t, stopper)
39+
40+
testutils.RunTrueAndFalse(t, "matchingVersion", func(t *testing.T, matchingVersion bool) {
41+
cc := storagepb.ComputeChecksum{
42+
ChecksumID: uuid.FastMakeV4(),
43+
Mode: roachpb.ChecksumMode_CHECK_FULL,
44+
}
45+
if matchingVersion {
46+
cc.Version = batcheval.ReplicaChecksumVersion
47+
} else {
48+
cc.Version = 1
49+
}
50+
tc.repl.computeChecksumPostApply(ctx, cc)
51+
rc, err := tc.repl.getChecksum(ctx, cc.ChecksumID)
52+
if !matchingVersion {
53+
if !testutils.IsError(err, "no checksum found") {
54+
t.Fatal(err)
55+
}
56+
require.Nil(t, rc.Checksum)
57+
} else {
58+
require.NoError(t, err)
59+
require.NotNil(t, rc.Checksum)
60+
}
61+
})
62+
}

pkg/storage/replica_proposal.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/keys"
2727
"github.com/cockroachdb/cockroach/pkg/roachpb"
2828
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
29+
"github.com/cockroachdb/cockroach/pkg/storage/batcheval"
2930
"github.com/cockroachdb/cockroach/pkg/storage/batcheval/result"
3031
"github.com/cockroachdb/cockroach/pkg/storage/engine"
3132
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
@@ -169,6 +170,14 @@ func (r *Replica) computeChecksumPostApply(ctx context.Context, cc storagepb.Com
169170
r.mu.checksums[cc.ChecksumID] = ReplicaChecksum{started: true, notify: notify}
170171
desc := *r.mu.state.Desc
171172
r.mu.Unlock()
173+
174+
if cc.Version != batcheval.ReplicaChecksumVersion {
175+
r.computeChecksumDone(ctx, cc.ChecksumID, nil, nil)
176+
log.Infof(ctx, "incompatible ComputeChecksum versions (requested: %d, have: %d)",
177+
cc.Version, batcheval.ReplicaChecksumVersion)
178+
return
179+
}
180+
172181
// Caller is holding raftMu, so an engine snapshot is automatically
173182
// Raft-consistent (i.e. not in the middle of an AddSSTable).
174183
snap := r.store.engine.NewSnapshot()

pkg/storage/replica_test.go

Lines changed: 0 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -7012,32 +7012,6 @@ func TestReplicaTryAbandon(t *testing.T) {
70127012
})
70137013
}
70147014

7015-
// TestComputeChecksumVersioning checks that the ComputeChecksum post-commit
7016-
// trigger is called if and only if the checksum version is right.
7017-
func TestComputeChecksumVersioning(t *testing.T) {
7018-
defer leaktest.AfterTest(t)()
7019-
tc := testContext{}
7020-
stopper := stop.NewStopper()
7021-
defer stopper.Stop(context.TODO())
7022-
tc.Start(t, stopper)
7023-
7024-
if pct, _ := batcheval.ComputeChecksum(context.TODO(), nil,
7025-
batcheval.CommandArgs{Args: &roachpb.ComputeChecksumRequest{
7026-
Version: batcheval.ReplicaChecksumVersion,
7027-
}}, &roachpb.ComputeChecksumResponse{},
7028-
); pct.Replicated.ComputeChecksum == nil {
7029-
t.Error("right checksum version: expected post-commit trigger")
7030-
}
7031-
7032-
if pct, _ := batcheval.ComputeChecksum(context.TODO(), nil,
7033-
batcheval.CommandArgs{Args: &roachpb.ComputeChecksumRequest{
7034-
Version: batcheval.ReplicaChecksumVersion + 1,
7035-
}}, &roachpb.ComputeChecksumResponse{},
7036-
); pct.Replicated.ComputeChecksum != nil {
7037-
t.Errorf("wrong checksum version: expected no post-commit trigger: %s", pct.Replicated.ComputeChecksum)
7038-
}
7039-
}
7040-
70417015
func TestNewReplicaCorruptionError(t *testing.T) {
70427016
defer leaktest.AfterTest(t)()
70437017
for i, tc := range []struct {

0 commit comments

Comments
 (0)