Skip to content

Commit

Permalink
kvserver: metamorphically enable kv.raft.leader_fortification.fractio…
Browse files Browse the repository at this point in the history
…n_enabled

Part of cockroachdb#123847.

This commit metamorphically enables the `kv.raft.leader_fortification.fraction_enabled`
to exercise raft fortification and leader leases.

The commit also includes a few other WIP changes to try to stabilize this. It
won't be fully stable until defortification is implemented.

Release note: None
  • Loading branch information
nvanbenschoten authored and arulajmani committed Nov 29, 2024
1 parent 0837ddd commit e6a9189
Show file tree
Hide file tree
Showing 8 changed files with 33 additions and 7 deletions.
4 changes: 4 additions & 0 deletions pkg/cli/debug_recover_loss_of_quorum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
Expand Down Expand Up @@ -182,8 +183,11 @@ func TestLossOfQuorumRecovery(t *testing.T) {
// would not be able to progress, but we will apply recovery procedure and
// mark on replicas on node 1 as designated survivors. After that, starting
// single node should succeed.
st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)
tcBefore := testcluster.NewTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
// This logic is specific to the storage layer.
DefaultTestTenant: base.TestIsSpecificToStorageLayerAndNeedsASystemTenant,
},
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -341,6 +341,7 @@ func TestTransactionUnexpectedlyCommitted(t *testing.T) {

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)

// Disable closed timestamps for control over when transaction gets bumped.
closedts.TargetDuration.Override(ctx, &st.SV, 1*time.Hour)
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,7 +493,7 @@ func mergeCheckingTimestampCaches(
st := cluster.MakeTestingClusterSettings()
// This test explicitly sets up a leader/leaseholder partition, which doesn't
// work with expiration leases (the lease expires).
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)
tc := testcluster.StartTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
Expand Down Expand Up @@ -2991,13 +2991,17 @@ func TestStoreRangeMergeAbandonedFollowers(t *testing.T) {
// RHS, as it interpreted destroyReasonMergePending to mean that the RHS replica
// had already been garbage collected.
func TestStoreRangeMergeAbandonedFollowersAutomaticallyGarbageCollected(t *testing.T) {

defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)
tc := testcluster.StartTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{},
})
defer tc.Stopper().Stop(ctx)
scratch := tc.ScratchRange(t)
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5735,6 +5735,8 @@ func TestElectionAfterRestart(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)

// We use a single node to avoid rare flakes due to dueling elections.
// The code is set up to support multiple nodes, though the test will
Expand All @@ -5756,6 +5758,7 @@ func TestElectionAfterRestart(t *testing.T) {
ReplicationMode: replMode,
ParallelStart: parallel,
ServerArgs: base.TestServerArgs{
Settings: st,
RaftConfig: base.RaftConfig{
RaftElectionTimeoutTicks: electionTimeoutTicks,
RaftTickInterval: raftTickInterval,
Expand Down
12 changes: 11 additions & 1 deletion pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3308,12 +3308,17 @@ func TestChangeReplicasSwapVoterWithNonVoter(t *testing.T) {
func TestReplicaTombstone(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)

t.Run("(1) ChangeReplicasTrigger", func(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
DisableReplicaGCQueue: true,
}},
Expand Down Expand Up @@ -3351,6 +3356,7 @@ func TestReplicaTombstone(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
RaftConfig: base.RaftConfig{
// Make the tick interval short so we don't need to wait too long for
// the partitioned node to time out.
Expand Down Expand Up @@ -3418,6 +3424,7 @@ func TestReplicaTombstone(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
DisableReplicaGCQueue: true,
}},
Expand Down Expand Up @@ -3455,6 +3462,7 @@ func TestReplicaTombstone(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 4, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
DisableReplicaGCQueue: true,
}},
Expand Down Expand Up @@ -3504,6 +3512,7 @@ func TestReplicaTombstone(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
RaftConfig: base.RaftConfig{
// Make the tick interval short so we don't need to wait too long
// for a heartbeat to be sent.
Expand Down Expand Up @@ -3629,6 +3638,7 @@ func TestReplicaTombstone(t *testing.T) {
proposalFilter.Store(noopProposalFilter)
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{
DisableReplicaGCQueue: true,
TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error {
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/loqrecovery/collect_raft_log_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/loqrecovery"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/loqrecovery/loqrecoverypb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand Down Expand Up @@ -160,8 +161,11 @@ func checkRaftLog(
RaftLogTruncationThreshold: math.MaxInt64,
}

st := cluster.MakeTestingClusterSettings()
kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV)
tc := testcluster.NewTestCluster(t, 2, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
DisableGCQueue: true,
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/replica_store_liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/raft/raftstoreliveness"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
)

// RaftLeaderFortificationFractionEnabled controls the fraction of ranges for
Expand All @@ -33,9 +33,9 @@ var RaftLeaderFortificationFractionEnabled = settings.RegisterFloatSetting(
"by extension, use Leader leases for all ranges which do not require "+
"expiration-based leases. Set to a value between 0.0 and 1.0 to gradually "+
"roll out Leader leases across the ranges in a cluster.",
// TODO(nvanbenschoten): make this a metamorphic constant once raft leader
// fortification and leader leases are sufficiently stable.
envutil.EnvOrDefaultFloat64("COCKROACH_LEADER_FORTIFICATION_FRACTION_ENABLED", 0.0),
metamorphic.ConstantWithTestChoice("kv.raft.leader_fortification.fraction_enabled",
1.0, /* defaultValue */
1.0 /* otherValues */),
settings.FloatInRange(0.0, 1.0),
settings.WithPublic,
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/storeliveness/support_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ func (sm *SupportManager) startLoop(ctx context.Context) {
select {
case <-sm.storesToAdd.sig:
sm.maybeAddStores(ctx)
sm.sendHeartbeats(ctx)
continue

case <-heartbeatTicker.C:
sm.sendHeartbeats(ctx)
Expand Down

0 comments on commit e6a9189

Please sign in to comment.