diff --git a/AUTHORS b/AUTHORS index 579fe0c1b866..405c5bc27498 100644 --- a/AUTHORS +++ b/AUTHORS @@ -323,6 +323,7 @@ Solon Gordon solongordon Song Hao songhao Spas Bojanov Spencer Kimball +Stephen Mooney Steven Danna Steven Hand <@cockroachlabs.com> hand <@cockroachlabs.com> hand-crdb <@cockroachlabs.com> Sumeer Bhola sumeerbhola diff --git a/pkg/base/config.go b/pkg/base/config.go index ee2c618fbc4f..dd7a759d0c1b 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -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, @@ -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 @@ -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 } @@ -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 } diff --git a/pkg/cmd/roachtest/fixtures/1/checkpoint-v20.2-48.tgz b/pkg/cmd/roachtest/fixtures/1/checkpoint-v20.2-48.tgz new file mode 100644 index 000000000000..21f11e3512ef Binary files /dev/null and b/pkg/cmd/roachtest/fixtures/1/checkpoint-v20.2-48.tgz differ diff --git a/pkg/cmd/roachtest/fixtures/2/checkpoint-v20.2-48.tgz b/pkg/cmd/roachtest/fixtures/2/checkpoint-v20.2-48.tgz new file mode 100644 index 000000000000..f20160a2e80c Binary files /dev/null and b/pkg/cmd/roachtest/fixtures/2/checkpoint-v20.2-48.tgz differ diff --git a/pkg/cmd/roachtest/fixtures/3/checkpoint-v20.2-48.tgz b/pkg/cmd/roachtest/fixtures/3/checkpoint-v20.2-48.tgz new file mode 100644 index 000000000000..b3e1bb4ebf0f Binary files /dev/null and b/pkg/cmd/roachtest/fixtures/3/checkpoint-v20.2-48.tgz differ diff --git a/pkg/cmd/roachtest/fixtures/4/checkpoint-v20.2-48.tgz b/pkg/cmd/roachtest/fixtures/4/checkpoint-v20.2-48.tgz new file mode 100644 index 000000000000..9ee7eed0fb6e Binary files /dev/null and b/pkg/cmd/roachtest/fixtures/4/checkpoint-v20.2-48.tgz differ diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 3dafd304eddf..7028f8c95b1b 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -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", diff --git a/pkg/cmd/roachtest/versionupgrade.go b/pkg/cmd/roachtest/versionupgrade.go index 3a8f85d93870..0f465a353894 100644 --- a/pkg/cmd/roachtest/versionupgrade.go +++ b/pkg/cmd/roachtest/versionupgrade.go @@ -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) diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 8f269435b795..145bfa51acfb 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -923,7 +923,7 @@ 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( @@ -931,10 +931,7 @@ func (r *Registry) cleanupOldJobsPage( ); 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 diff --git a/pkg/kv/kvserver/intent_resolver_integration_test.go b/pkg/kv/kvserver/intent_resolver_integration_test.go index a6f712e62114..f5f00c5a15b9 100644 --- a/pkg/kv/kvserver/intent_resolver_integration_test.go +++ b/pkg/kv/kvserver/intent_resolver_integration_test.go @@ -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{ @@ -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()) diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 600386e6abf1..cc0c39c6c624 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -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" @@ -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() @@ -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, @@ -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