Skip to content

Commit

Permalink
storage: add setting to seed up consistency checks in tests
Browse files Browse the repository at this point in the history
Previously, we'd fall back to the 3 second consistency
checker EventuallyFileOnlySnapshot (EFOS) wait in roachtests,
which was slowing all of
them down when we ran every replica through the consistency
checker in post-test assertions. This change speeds up that
consistency check in roachtest post-test assertions by
flipping a new cluster setting to speed up EFOS waits for
consistency checks after a roachtest finishes.

Epic: none
Unblocks cockroachdb#116330.

Release note: None
  • Loading branch information
itsbilal committed Dec 20, 2023
1 parent 611607c commit bfb9b74
Show file tree
Hide file tree
Showing 9 changed files with 37 additions and 14 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
<tr><td><div id="setting-kv-closed-timestamp-lead-for-global-reads-override" class="anchored"><code>kv.closed_timestamp.lead_for_global_reads_override</code></div></td><td>duration</td><td><code>0s</code></td><td>if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-closed-timestamp-side-transport-interval" class="anchored"><code>kv.closed_timestamp.side_transport_interval</code></div></td><td>duration</td><td><code>200ms</code></td><td>the interval at which the closed timestamp side-transport attempts to advance each range&#39;s closed timestamp; set to 0 to disable the side-transport</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-closed-timestamp-target-duration" class="anchored"><code>kv.closed_timestamp.target_duration</code></div></td><td>duration</td><td><code>3s</code></td><td>if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-consistency-queue-testing-fast-efos-acquisition" class="anchored"><code>kv.consistency_queue.testing_fast_efos_acquisition</code></div></td><td>boolean</td><td><code>false</code></td><td>set to true to speed up EventuallyFileOnlySnapshot acquisition/transition for tests at the expense of excessive flushes</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-log-range-and-node-events-enabled" class="anchored"><code>kv.log_range_and_node_events.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>set to true to transactionally log range events (e.g., split, merge, add/remove voter/non-voter) into system.rangelogand node join and restart events into system.eventolog</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-kv-protectedts-reconciliation-interval" class="anchored"><code>kv.protectedts.reconciliation.interval</code></div></td><td>duration</td><td><code>5m0s</code></td><td>the frequency for reconciling jobs with protected timestamp records</td><td>Serverless/Dedicated/Self-Hosted (read-only)</td></tr>
<tr><td><div id="setting-kv-range-split-by-load-enabled" class="anchored"><code>kv.range_split.by_load.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>allow automatic splits of ranges based on where load is concentrated</td><td>Dedicated/Self-Hosted</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/roachtestutil/validation_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ func CheckReplicaDivergenceOnDB(ctx context.Context, l *logger.Logger, db *gosql
// is happening or how to disable it.
started := timeutil.Now()
rows, err := db.QueryContext(ctx, `
SET CLUSTER SETTING kv.consistency_queue.testing_fast_efos_acquisition = true;
SET statement_timeout = '20m';
SELECT t.range_id, t.start_key_pretty, t.status, t.detail
FROM crdb_internal.check_consistency(false, '', '') as t;`)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/consistency_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -405,7 +405,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) {

// Compute a checksum over the content of the problematic range.
rd, err := kvserver.CalcReplicaDigest(context.Background(), *desc, cpEng,
kvpb.ChecksumMode_CHECK_FULL, quotapool.NewRateLimiter("test", quotapool.Inf(), 0))
kvpb.ChecksumMode_CHECK_FULL, quotapool.NewRateLimiter("test", quotapool.Inf(), 0), nil /* settings */)
require.NoError(t, err)
hashes[i] = rd.SHA512[:]
}
Expand Down
22 changes: 20 additions & 2 deletions pkg/kv/kvserver/replica_consistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
Expand All @@ -56,6 +58,15 @@ type replicaChecksum struct {
result chan CollectChecksumResponse
}

// TestingFastEFOSAcquisition speeds up EFOS WaitForFileOnly() to speed up
// node-wide replica consistency check calls in roachtests.
var TestingFastEFOSAcquisition = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.consistency_queue.testing_fast_efos_acquisition",
"set to true to speed up EventuallyFileOnlySnapshot acquisition/transition for tests at the expense of excessive flushes",
false, /* defaultValue */
settings.WithPublic)

// CheckConsistency runs a consistency check on the range. It first applies a
// ComputeChecksum through Raft and then issues CollectChecksum commands to the
// other replicas. These are inspected and a CheckConsistencyResponse is assembled.
Expand Down Expand Up @@ -479,6 +490,7 @@ func CalcReplicaDigest(
snap storage.Reader,
mode kvpb.ChecksumMode,
limiter *quotapool.RateLimiter,
settings *cluster.Settings,
) (*ReplicaDigest, error) {
statsOnly := mode == kvpb.ChecksumMode_CHECK_STATS

Expand All @@ -500,7 +512,13 @@ func CalcReplicaDigest(
// both requests are likely sharing the same `limiter` so if too many
// requests run concurrently, some of them could time out due to a
// combination of this wait and the limiter-induced wait.
if err := efos.WaitForFileOnly(ctx); err != nil {
efosWait := storage.MaxEFOSWait
if settings != nil && TestingFastEFOSAcquisition.Get(&settings.SV) {
if efosWait > 10*time.Millisecond {
efosWait = 10 * time.Millisecond
}
}
if err := efos.WaitForFileOnly(ctx, efosWait); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -773,7 +791,7 @@ func (r *Replica) computeChecksumPostApply(
); err != nil {
log.Errorf(ctx, "checksum collection did not join: %v", err)
} else {
result, err := CalcReplicaDigest(ctx, desc, snap, cc.Mode, r.store.consistencyLimiter)
result, err := CalcReplicaDigest(ctx, desc, snap, cc.Mode, r.store.consistencyLimiter, r.ClusterSettings())
if err != nil {
log.Errorf(ctx, "checksum computation failed: %v", err)
result = nil
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/replica_consistency_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ func TestReplicaChecksumSHA512(t *testing.T) {

// Hash the empty state.
unlim := quotapool.NewRateLimiter("test", quotapool.Inf(), 0)
rd, err := CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim)
rd, err := CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim, nil /* settings */)
require.NoError(t, err)
fmt.Fprintf(sb, "checksum0: %x\n", rd.SHA512)

Expand Down Expand Up @@ -314,7 +314,7 @@ func TestReplicaChecksumSHA512(t *testing.T) {
require.NoError(t, err)
}

rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim)
rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim, nil /* settings */)
require.NoError(t, err)
fmt.Fprintf(sb, "checksum%d: %x\n", i+1, rd.SHA512)
}
Expand All @@ -335,13 +335,13 @@ func TestReplicaChecksumSHA512(t *testing.T) {
txn := &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{ID: txnID}}
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, txn, l.str, roachpb.Key(l.key), nil, 0))

rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim)
rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim, nil /* settings */)
require.NoError(t, err)
fmt.Fprintf(sb, "checksum%d: %x\n", i+1, rd.SHA512)
}

// Run another check to obtain stats for the final state.
rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim)
rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim, nil /* settings */)
require.NoError(t, err)
jsonpb := protoutil.JSONPb{Indent: " "}
json, err := jsonpb.Marshal(&rd.RecomputedMS)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10425,7 +10425,7 @@ func TestReplicaServersideRefreshes(t *testing.T) {
snap := tc.engine.NewSnapshot()
defer snap.Close()
res, err := CalcReplicaDigest(ctx, *tc.repl.Desc(), tc.engine, kvpb.ChecksumMode_CHECK_FULL,
quotapool.NewRateLimiter("test", quotapool.Inf(), 0))
quotapool.NewRateLimiter("test", quotapool.Inf(), 0), nil /* settings */)
if err != nil {
return hlc.Timestamp{}, err
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"bytes"
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -938,6 +939,6 @@ func NewEventuallyFileOnlySnapshot(
}

// WaitForFileOnly implements the storage.EventuallyFileOnlyReader interface.
func (e *spanSetEFOS) WaitForFileOnly(ctx context.Context) error {
return e.efos.WaitForFileOnly(ctx)
func (e *spanSetEFOS) WaitForFileOnly(ctx context.Context, dur time.Duration) error {
return e.efos.WaitForFileOnly(ctx, dur)
}
6 changes: 4 additions & 2 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -632,8 +632,10 @@ type EventuallyFileOnlyReader interface {
Reader
// WaitForFileOnly blocks the calling goroutine until this reader has
// transitioned to a file-only reader that does not pin any in-memory state.
// If an error is returned, this transition did not succeed.
WaitForFileOnly(context.Context) error
// If an error is returned, this transition did not succeed. The Duration
// argument specifies how long to wait for before attempting a flush to
// force a transition to a file-only snapshot.
WaitForFileOnly(context.Context, time.Duration) error
}

// Writer is the write interface to an engine's data.
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -3072,8 +3072,8 @@ func (p *pebbleEFOS) MVCCIterate(
}

// WaitForFileOnly implements the EventuallyFileOnlyReader interface.
func (p *pebbleEFOS) WaitForFileOnly(ctx context.Context) error {
return p.efos.WaitForFileOnlySnapshot(ctx, MaxEFOSWait)
func (p *pebbleEFOS) WaitForFileOnly(ctx context.Context, dur time.Duration) error {
return p.efos.WaitForFileOnlySnapshot(ctx, dur)
}

// NewMVCCIterator implements the Reader interface.
Expand Down

0 comments on commit bfb9b74

Please sign in to comment.