Skip to content

Commit

Permalink
kvserver: metamorphically enable kv.expiration_leases_only.enabled
Browse files Browse the repository at this point in the history
Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Mar 18, 2023
1 parent 85c6e38 commit ecc931b
Show file tree
Hide file tree
Showing 11 changed files with 109 additions and 46 deletions.
78 changes: 39 additions & 39 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand All @@ -57,10 +56,15 @@ func TestStoreRangeLease(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
DisableMergeQueue: true,
Expand All @@ -69,7 +73,7 @@ func TestStoreRangeLease(t *testing.T) {
},
},
)
defer tc.Stopper().Stop(context.Background())
defer tc.Stopper().Stop(ctx)

store := tc.GetFirstStoreFromServer(t, 0)
// NodeLivenessKeyMax is a static split point, so this is always
Expand Down Expand Up @@ -839,7 +843,6 @@ func TestLeaseholderRelocate(t *testing.T) {

// Make sure the lease is on 3 and is fully upgraded.
tc.TransferRangeLeaseOrFatal(t, rhsDesc, tc.Target(2))
tc.WaitForLeaseUpgrade(ctx, t, rhsDesc)

// Check that the lease moved to 3.
leaseHolder, err := tc.FindRangeLeaseHolder(rhsDesc, nil)
Expand Down Expand Up @@ -872,17 +875,24 @@ func TestLeaseholderRelocate(t *testing.T) {
require.NoError(t, err)
require.Equal(t, tc.Target(3), leaseHolder)

// Double check that lease moved directly.
// Double check that lease moved directly. The tail of the lease history
// should all be on leaseHolder.NodeID. We may metamorphically enable
// kv.expiration_leases_only.enabled, in which case there will be a single
// expiration lease, but otherwise we'll have transferred an expiration lease
// and then upgraded to an epoch lease.
repl := tc.GetFirstStoreFromServer(t, 3).
LookupReplica(roachpb.RKey(rhsDesc.StartKey.AsRawKey()))
history := repl.GetLeaseHistory()

require.Equal(t, leaseHolder.NodeID,
history[len(history)-1].Replica.NodeID)
require.Equal(t, leaseHolder.NodeID,
history[len(history)-2].Replica.NodeID) // account for the lease upgrade
require.Equal(t, tc.Target(2).NodeID,
history[len(history)-3].Replica.NodeID)
require.Equal(t, leaseHolder.NodeID, history[len(history)-1].Replica.NodeID)
var prevLeaseHolder roachpb.NodeID
for i := len(history) - 1; i >= 0; i-- {
if id := history[i].Replica.NodeID; id != leaseHolder.NodeID {
prevLeaseHolder = id
break
}
}
require.Equal(t, tc.Target(2).NodeID, prevLeaseHolder)
}

func gossipLiveness(t *testing.T, tc *testcluster.TestCluster) {
Expand Down Expand Up @@ -1100,15 +1110,20 @@ func TestLeasesDontThrashWhenNodeBecomesSuspect(t *testing.T) {
locality("us-west"),
locality("us-west"),
}

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

// Speed up lease transfers.
stickyRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyRegistry.CloseAllStickyInMemEngines()
ctx := context.Background()
manualClock := hlc.NewHybridManualClock()
serverArgs := make(map[int]base.TestServerArgs)
numNodes := 4
for i := 0; i < numNodes; i++ {
serverArgs[i] = base.TestServerArgs{
Settings: st,
Locality: localities[i],
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
Expand Down Expand Up @@ -1346,21 +1361,18 @@ func TestAcquireLeaseTimeout(t *testing.T) {
return nil
}

// The lease request timeout depends on the Raft election timeout, so we set
// it low to get faster timeouts (800 ms) and speed up the test.
var raftCfg base.RaftConfig
raftCfg.SetDefaults()
raftCfg.RaftHeartbeatIntervalTicks = 1
raftCfg.RaftElectionTimeoutTicks = 2

manualClock := hlc.NewHybridManualClock()

// Start a two-node cluster.
const numNodes = 2
tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
RaftConfig: raftCfg,
RaftConfig: base.RaftConfig{
// Lease request timeout depends on Raft election timeout, speed it up.
RaftHeartbeatIntervalTicks: 1,
RaftElectionTimeoutTicks: 2,
},
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
WallClock: manualClock,
Expand All @@ -1383,27 +1395,10 @@ func TestAcquireLeaseTimeout(t *testing.T) {
repl, err := tc.GetFirstStoreFromServer(t, 0).GetReplica(desc.RangeID)
require.NoError(t, err)

tc.IncrClockForLeaseUpgrade(t, manualClock)
tc.WaitForLeaseUpgrade(ctx, t, desc)

// Stop n2 and increment its epoch to invalidate the lease.
// Stop n2 and invalidate its leases by forwarding the clock.
tc.StopServer(1)
n2ID := tc.Server(1).NodeID()
lv, ok := tc.Server(0).NodeLiveness().(*liveness.NodeLiveness)
require.True(t, ok)
lvNode2, ok := lv.GetLiveness(n2ID)
require.True(t, ok)
manualClock.Forward(lvNode2.Expiration.WallTime)

testutils.SucceedsSoon(t, func() error {
lvNode2, ok = lv.GetLiveness(n2ID)
require.True(t, ok)
err := lv.IncrementEpoch(context.Background(), lvNode2.Liveness)
if errors.Is(err, liveness.ErrEpochAlreadyIncremented) {
return nil
}
return err
})
leaseDuration := tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().RangeLeaseDuration
manualClock.Increment(leaseDuration.Nanoseconds())
require.False(t, repl.CurrentLeaseStatus(ctx).IsValid())

// Trying to acquire the lease should error with an empty NLHE, since the
Expand Down Expand Up @@ -1456,11 +1451,14 @@ func TestLeaseTransfersUseExpirationLeasesAndBumpToEpochBasedOnes(t *testing.T)
}{}

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

manualClock := hlc.NewHybridManualClock()
tci := serverutils.StartNewTestCluster(t, 2, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
// Never ticked -- demonstrating that we're not relying on
Expand Down Expand Up @@ -1527,6 +1525,8 @@ func TestLeaseUpgradeVersionGate(t *testing.T) {
clusterversion.ByKey(clusterversion.TODODelete_V22_2EnableLeaseUpgrade-1),
false, /* initializeVersion */
)
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

tci := serverutils.StartNewTestCluster(t, 2, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Expand Down
8 changes: 5 additions & 3 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -486,10 +486,15 @@ func mergeCheckingTimestampCaches(

manualClock := hlc.NewHybridManualClock()
ctx := context.Background()
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
tc := testcluster.StartTestCluster(t, 3,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
WallClock: manualClock,
Expand Down Expand Up @@ -1012,9 +1017,6 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) {
if !lhsRepl1.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) {
return errors.New("s2 does not own valid lease for lhs range")
}
if lhsRepl1.CurrentLeaseStatus(ctx).Lease.Type() != roachpb.LeaseEpoch {
return errors.Errorf("lease still an expiration based lease")
}
return nil
})

Expand Down
13 changes: 12 additions & 1 deletion pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1315,9 +1315,13 @@ func TestRequestsOnFollowerWithNonLiveLeaseholder(t *testing.T) {
return nil
}

st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

clusterArgs := base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
// Reduce the election timeout some to speed up the test.
RaftConfig: base.RaftConfig{RaftElectionTimeoutTicks: 10},
Knobs: base.TestingKnobs{
Expand Down Expand Up @@ -1749,7 +1753,7 @@ func TestLogGrowthWhenRefreshingPendingCommands(t *testing.T) {
}
propNode := tc.GetFirstStoreFromServer(t, propIdx).TestSender()
tc.TransferRangeLeaseOrFatal(t, *leaderRepl.Desc(), tc.Target(propIdx))
tc.WaitForLeaseUpgrade(ctx, t, *leaderRepl.Desc())
tc.MaybeWaitForLeaseUpgrade(ctx, t, *leaderRepl.Desc())
testutils.SucceedsSoon(t, func() error {
// Lease transfers may not be immediately observed by the new
// leaseholder. Wait until the new leaseholder is aware.
Expand Down Expand Up @@ -4872,10 +4876,17 @@ func TestDefaultConnectionDisruptionDoesNotInterfereWithSystemTraffic(t *testing
},
}

// This test relies on epoch leases being invalidated when a node restart,
// which isn't true for expiration leases, so we disable expiration lease
// metamorphism.
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

const numServers int = 3
stickyServerArgs := make(map[int]base.TestServerArgs)
for i := 0; i < numServers; i++ {
stickyServerArgs[i] = base.TestServerArgs{
Settings: st,
StoreSpecs: []base.StoreSpec{
{
InMemory: true,
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/client_replica_circuit_breaker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -700,6 +700,10 @@ func setupCircuitBreakerTest(t *testing.T) *circuitBreakerTest {
var rangeID int64 // atomic
slowThresh := &atomic.Value{} // supports .SetSlowThreshold(x)
slowThresh.Store(time.Duration(0))
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
// TODO(erikgrinaker): We may not need this for all circuit breaker tests.
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism
storeKnobs := &kvserver.StoreTestingKnobs{
SlowReplicationThresholdOverride: func(ba *kvpb.BatchRequest) time.Duration {
t.Helper()
Expand Down Expand Up @@ -748,6 +752,7 @@ func setupCircuitBreakerTest(t *testing.T) *circuitBreakerTest {
args := base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
RaftConfig: raftCfg,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
Expand Down
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2060,11 +2060,14 @@ func TestLeaseMetricsOnSplitAndTransfer(t *testing.T) {
return nil
}
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism
manualClock := hlc.NewHybridManualClock()
tc := testcluster.StartTestCluster(t, 2,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
EvalKnobs: kvserverbase.BatchEvalTestingKnobs{
Expand Down Expand Up @@ -4358,6 +4361,11 @@ func TestStrictGCEnforcement(t *testing.T) {
protectedts.PollInterval.Override(ctx, &tc.Server(0).ClusterSettings().SV, 500*time.Hour)
defer protectedts.PollInterval.Override(ctx, &tc.Server(0).ClusterSettings().SV, 2*time.Minute)

// Disable follower reads. When metamorphically enabling expiration-based
// leases, an expired lease will cause a follower read which bypasses the
// strict GC enforcement.
sqlDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = false")

sqlDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '10 ms'")
defer sqlDB.Exec(t, `SET CLUSTER SETTING kv.gc_ttl.strict_enforcement.enabled = DEFAULT`)
setStrictGC(t, true)
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
Expand Down Expand Up @@ -2784,11 +2785,14 @@ func TestStoreCapacityAfterSplit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism
manualClock := hlc.NewHybridManualClock()
tc := testcluster.StartTestCluster(t, 2,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
WallClock: manualClock,
Expand Down
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/closed_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -671,8 +672,13 @@ func TestClosedTimestampFrozenAfterSubsumption(t *testing.T) {
st := mergeFilter{}
manual := hlc.NewHybridManualClock()
pinnedLeases := kvserver.NewPinnedLeases()

cs := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &cs.SV, false) // override metamorphism

clusterArgs := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: cs,
RaftConfig: base.RaftConfig{
// We set the raft election timeout to a small duration. This should
// result in the node liveness duration being ~3.6 seconds. Note that
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/replica_range_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand All @@ -75,11 +76,11 @@ var transferExpirationLeasesFirstEnabled = settings.RegisterBoolSetting(
true,
)

var expirationLeasesOnly = settings.RegisterBoolSetting(
var ExpirationLeasesOnly = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.expiration_leases_only.enabled",
"only use expiration-based leases, never epoch-based ones (experimental, affects performance)",
false,
util.ConstantWithMetamorphicTestBool("kv.expiration_leases_only.enabled", false),
)

var leaseStatusLogLimiter = func() *log.EveryN {
Expand Down Expand Up @@ -789,7 +790,7 @@ func (r *Replica) requiresExpirationLeaseRLocked() bool {
// expiration-based lease, either because it requires one or because
// kv.expiration_leases_only.enabled is enabled.
func (r *Replica) shouldUseExpirationLeaseRLocked() bool {
return expirationLeasesOnly.Get(&r.ClusterSettings().SV) || r.requiresExpirationLeaseRLocked()
return ExpirationLeasesOnly.Get(&r.ClusterSettings().SV) || r.requiresExpirationLeaseRLocked()
}

// requestLeaseLocked executes a request to obtain or extend a lease
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/replica_rangefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1242,10 +1242,14 @@ func TestRangefeedCheckpointsRecoverFromLeaseExpiration(t *testing.T) {
// evaluating on the scratch range.
var rejectExtraneousRequests int64 // accessed atomically

st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism

cargs := aggressiveResolvedTimestampClusterArgs
cargs.ReplicationMode = base.ReplicationManual
manualClock := hlc.NewHybridManualClock()
cargs.ServerArgs = base.TestServerArgs{
Settings: st,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
WallClock: manualClock,
Expand Down
Loading

0 comments on commit ecc931b

Please sign in to comment.