Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
65158: kvserver: fix TestRejectedLeaseDoesntDictateClosedTimestamp  r=andreimatei a=andreimatei

This test was flaky (and skipped) in the case where cluster startup is
taking sufficiently many seconds such that, by the time it finishes,
the original lease on r1 is either expired or at least almost expired
such that the test inadvertendly kicks off a preemptive lease refresh.
The test tries to control exactly what lease requests get proposed, so
it doesn't like these refreshes happening.
The fix is to disable the preemptive refreshes, and also to make lease
duration longer.

Fixes #65109

cc @cockroachdb/kv 

65380: kvserver: deflake TestRollbackSyncRangedIntentResolution r=andreimatei a=erikgrinaker

Release note: None

65390: roachtest: update version map and fixtures r=j-low a=j-low

This commit adds the recently released 20.2.10
to the version map in PredecessorVersion.

Release note: None (testing change)

65394: jobs: don't check the number of deleted rows r=postamar a=postamar

When attempting to delete expired job records it's entirely possible to
legitimately delete less records than expected. This can happen when
another node is concurrently attempting cleanup. Previously, we returned
an assertion failure in these cases.

Fixes #65048.

Release note: None

65417: authors add Stephen Mooney to authors r=rail a=stephenmooney1976

Release notes: None

Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Joseph Lowinske <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: stephenmooney1976 <[email protected]>
  • Loading branch information
6 people committed May 18, 2021
6 parents 64dad5f + 1bf8d6f + 0fd0a7c + 6b75385 + a6f442e + 50f7a63 commit cd2988f
Show file tree
Hide file tree
Showing 11 changed files with 42 additions and 15 deletions.
1 change: 1 addition & 0 deletions AUTHORS
Original file line number Diff line number Diff line change
Expand Up @@ -323,6 +323,7 @@ Solon Gordon <[email protected]> solongordon <[email protected]>
Song Hao <[email protected]> songhao <[email protected]>
Spas Bojanov <[email protected]> <[email protected]>
Spencer Kimball <[email protected]> <[email protected]>
Stephen Mooney <[email protected]> <[email protected]>
Steven Danna <[email protected]>
Steven Hand <@cockroachlabs.com> hand <@cockroachlabs.com> hand-crdb <@cockroachlabs.com>
Sumeer Bhola <[email protected]> sumeerbhola <[email protected]>
Expand Down
26 changes: 23 additions & 3 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,11 +71,11 @@ const (
// used by the rpc context.
defaultRPCHeartbeatInterval = 3 * time.Second

// rangeLeaseRenewalFraction specifies what fraction the range lease
// defaultRangeLeaseRenewalFraction specifies what fraction the range lease
// renewal duration should be of the range lease active time. For example,
// with a value of 0.2 and a lease duration of 10 seconds, leases would be
// eagerly renewed 8 seconds into each lease.
rangeLeaseRenewalFraction = 0.5
defaultRangeLeaseRenewalFraction = 0.5

// livenessRenewalFraction specifies what fraction the node liveness
// renewal duration should be of the node liveness duration. For example,
Expand Down Expand Up @@ -317,6 +317,13 @@ type RaftConfig struct {
// RangeLeaseRaftElectionTimeoutMultiplier specifies what multiple the leader
// lease active duration should be of the raft election timeout.
RangeLeaseRaftElectionTimeoutMultiplier float64
// RangeLeaseRenewalFraction specifies what fraction the range lease renewal
// duration should be of the range lease active time. For example, with a
// value of 0.2 and a lease duration of 10 seconds, leases would be eagerly
// renewed 8 seconds into each lease. A value of zero means use the default
// and a value of -1 means never pre-emptively renew the lease. A value of 1
// means always renew.
RangeLeaseRenewalFraction float64

// RaftLogTruncationThreshold controls how large a single Range's Raft log
// can grow. When a Range's Raft log grows above this size, the Range will
Expand Down Expand Up @@ -384,6 +391,15 @@ func (cfg *RaftConfig) SetDefaults() {
if cfg.RangeLeaseRaftElectionTimeoutMultiplier == 0 {
cfg.RangeLeaseRaftElectionTimeoutMultiplier = defaultRangeLeaseRaftElectionTimeoutMultiplier
}
if cfg.RangeLeaseRenewalFraction == 0 {
cfg.RangeLeaseRenewalFraction = defaultRangeLeaseRenewalFraction
}
// TODO(andrei): -1 is a special value for RangeLeaseRenewalFraction which
// really means "0" (never renew), except that the zero value means "use
// default". We can't turn the -1 into 0 here because, unfortunately,
// SetDefaults is called multiple times (see NewStore()). So, we leave -1
// alone and ask all the users to handle it specially.

if cfg.RaftLogTruncationThreshold == 0 {
cfg.RaftLogTruncationThreshold = defaultRaftLogTruncationThreshold
}
Expand Down Expand Up @@ -438,7 +454,11 @@ func (cfg RaftConfig) RaftElectionTimeout() time.Duration {
func (cfg RaftConfig) RangeLeaseDurations() (rangeLeaseActive, rangeLeaseRenewal time.Duration) {
rangeLeaseActive = time.Duration(cfg.RangeLeaseRaftElectionTimeoutMultiplier *
float64(cfg.RaftElectionTimeout()))
rangeLeaseRenewal = time.Duration(float64(rangeLeaseActive) * rangeLeaseRenewalFraction)
renewalFraction := cfg.RangeLeaseRenewalFraction
if renewalFraction == -1 {
renewalFraction = 0
}
rangeLeaseRenewal = time.Duration(float64(rangeLeaseActive) * renewalFraction)
return
}

Expand Down
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1217,7 +1217,7 @@ func PredecessorVersion(buildVersion version.Version) (string, error) {
// map.
verMap := map[string]string{
"21.2": "21.1.0-beta.5",
"21.1": "20.2.9",
"21.1": "20.2.10",
"20.2": "20.1.16",
"20.1": "19.2.11",
"19.2": "19.1.11",
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ func runVersionUpgrade(ctx context.Context, t *test, c *cluster, buildVersion ve
// The version to create/update the fixture for. Must be released (i.e.
// can download it from the homepage); if that is not the case use the
// empty string which uses the local cockroach binary.
newV := "20.1.16"
newV := "20.2.10"
predV, err := PredecessorVersion(*version.MustParse("v" + newV))
if err != nil {
t.Fatal(err)
Expand Down
7 changes: 2 additions & 5 deletions pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -923,18 +923,15 @@ func (r *Registry) cleanupOldJobsPage(

log.VEventf(ctx, 2, "read potentially expired jobs: %d", numRows)
if len(toDelete.Array) > 0 {
log.Infof(ctx, "cleaning up expired job records: %d", len(toDelete.Array))
log.Infof(ctx, "attempting to clean up %d expired job records", len(toDelete.Array))
const stmt = `DELETE FROM system.jobs WHERE id = ANY($1)`
var nDeleted int
if nDeleted, err = r.ex.Exec(
ctx, "gc-jobs", nil /* txn */, stmt, toDelete,
); err != nil {
return false, 0, errors.Wrap(err, "deleting old jobs")
}
if nDeleted != len(toDelete.Array) {
return false, 0, errors.AssertionFailedf("asked to delete %d rows but %d were actually deleted",
len(toDelete.Array), nDeleted)
}
log.Infof(ctx, "cleaned up %d expired job records", nDeleted)
}
// If we got as many rows as we asked for, there might be more.
morePages := numRows == pageSize
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/intent_resolver_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,8 +186,8 @@ func TestContendedIntentWithDependencyCycle(t *testing.T) {
func TestRollbackSyncRangedIntentResolution(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.UnderRace(t)
skip.UnderStress(t)
skip.UnderRace(t, "timing-sensitive test")
skip.UnderBazelWithIssue(t, 65407, "times out due to high concurrency")

ctx := context.Background()
srv, _, _ := serverutils.StartServer(t, base.TestServerArgs{
Expand All @@ -208,7 +208,7 @@ func TestRollbackSyncRangedIntentResolution(t *testing.T) {
batch.Put([]byte(fmt.Sprintf("key%v", i)), []byte("value"))
}
require.NoError(t, txn.Run(ctx, batch))
ctx, cancel := context.WithTimeout(ctx, 20*time.Second)
ctx, cancel := context.WithTimeout(ctx, 30*time.Second)
defer cancel()
require.NoError(t, txn.Rollback(ctx))
require.NoError(t, ctx.Err())
Expand Down
13 changes: 11 additions & 2 deletions pkg/kv/kvserver/replica_closedts_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -450,7 +449,6 @@ func TestBumpSideTransportClosed(t *testing.T) {
// 6, the lease proposal doesn't bump the assignedClosedTimestamp.
func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 65109, "flaky test")
defer log.Scope(t).Close(t)
ctx := context.Background()

Expand Down Expand Up @@ -517,6 +515,16 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) {
tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
RaftConfig: base.RaftConfig{
// Disable preemptive lease extensions because, if the server startup
// takes too long before we pause the clock, such an extension can
// happen on the range of interest, and messes up the test that expects
// the lease to expire.
RangeLeaseRenewalFraction: -1,
// Also make expiration-based leases last for a long time, as the test
// wants a valid lease after cluster start.
RaftElectionTimeoutTicks: 1000,
},
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
ClockSource: manual.UnixNano,
Expand Down Expand Up @@ -594,6 +602,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) {
select {
case <-leaseAcqCh:
case err := <-leaseAcqErrCh:
close(leaseTransferCh)
t.Fatalf("lease request unexpectedly finished. err: %v", err)
}
// Let the previously blocked transfer succeed. n2's lease acquisition remains
Expand Down

0 comments on commit cd2988f

Please sign in to comment.