Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
39149: roachtest: skip kv/contention/nodes=4 for release-19.1 r=tbg a=nvanbenschoten

Fixes #39116.

release-19.1 is susceptible to the issues described in #36089, so it won't reliably pass this test.

39160: storage: add DisableRaftLogQueue to StoreTestingKnobs r=tbg a=nvanbenschoten

Pulled from #38954, which I want to keep focused, especially with the PR's new secondary focus on refactoring entry application to be easier to mock and test.

Release note: None

39161: storage: address TODO in TestPushTxnHeartbeatTimeout r=tbg a=nvanbenschoten

Pulled from #38954, which I want to keep focused, especially with the PR's new secondary focus on refactoring entry application to be easier to mock and test.

Release note: None

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
  • Loading branch information
craig[bot] and nvanbenschoten committed Jul 30, 2019
4 parents aaa0356 + 5082052 + 80abd32 + aa97806 commit 634dc24
Show file tree
Hide file tree
Showing 16 changed files with 43 additions and 37 deletions.
8 changes: 4 additions & 4 deletions pkg/cmd/roachtest/cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ func runCDCSchemaRegistry(ctx context.Context, t *test, c *cluster) {

func registerCDC(r *testRegistry) {
useRangeFeed := true
if r.buildVersion.Compare(version.MustParse(`v2.2.0-0`)) < 0 {
if r.buildVersion.Compare(version.MustParse(`v19.1.0-0`)) < 0 {
// RangeFeed is not production ready in 2.1, so run the tests with the
// poller.
useRangeFeed = false
Expand Down Expand Up @@ -504,7 +504,7 @@ func registerCDC(r *testRegistry) {
// When testing a 2.1 binary, we use the poller for all the other tests
// and this is close enough to cdc/tpcc-1000 test to be redundant, so
// skip it.
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(4, cpu(16)),
Run: func(ctx context.Context, t *test, c *cluster) {
cdcBasicTest(ctx, t, c, cdcTestArgs{
Expand Down Expand Up @@ -576,7 +576,7 @@ func registerCDC(r *testRegistry) {
})
r.Add(testSpec{
Name: "cdc/cloud-sink-gcs/rangefeed=true",
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(4, cpu(16)),
Run: func(ctx context.Context, t *test, c *cluster) {
cdcBasicTest(ctx, t, c, cdcTestArgs{
Expand Down Expand Up @@ -609,7 +609,7 @@ func registerCDC(r *testRegistry) {
})
r.Add(testSpec{
Name: "cdc/schemareg",
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(1),
Run: func(ctx context.Context, t *test, c *cluster) {
runCDCSchemaRegistry(ctx, t, c)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/clearrange.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func registerClearRange(r *testRegistry) {
// 5h for import, 90 for the test. The import should take closer
// to <3:30h but it varies.
Timeout: 5*time.Hour + 90*time.Minute,
MinVersion: `v2.2.0`,
MinVersion: "v19.1.0",
// This test reformats a drive to ZFS, so we don't want it reused.
// TODO(andrei): Can the test itself reuse the cluster (under --count=2)?
// In other words, would a OnlyTagged("clearrange") policy be good?
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func registerDiskStalledDetection(r *testRegistry) {
"disk-stalled/log=%t,data=%t",
affectsLogDir, affectsDataDir,
),
MinVersion: `v2.2.0`,
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(1),
Run: func(ctx context.Context, t *test, c *cluster) {
runDiskStalledDetection(ctx, t, c, affectsLogDir, affectsDataDir)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/follower_reads.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func registerFollowerReads(r *testRegistry) {
r.Add(testSpec{
Name: "follower-reads/nodes=3",
Cluster: makeClusterSpec(3 /* nodeCount */, cpu(2), geo()),
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Run: runFollowerReadsTest,
})
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/cmd/roachtest/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,8 +154,9 @@ func registerKV(r *testRegistry) {
func registerKVContention(r *testRegistry) {
const nodes = 4
r.Add(testSpec{
Name: fmt.Sprintf("kv/contention/nodes=%d", nodes),
Cluster: makeClusterSpec(nodes + 1),
Name: fmt.Sprintf("kv/contention/nodes=%d", nodes),
MinVersion: "v19.2.0",
Cluster: makeClusterSpec(nodes + 1),
Run: func(ctx context.Context, t *test, c *cluster) {
c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes))
c.Put(ctx, workload, "./workload", c.Node(nodes+1))
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/psycopg.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ func registerPsycopg(r *testRegistry) {
r.Add(testSpec{
Name: "psycopg",
Cluster: makeClusterSpec(1),
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Run: func(ctx context.Context, t *test, c *cluster) {
runPsycopg(ctx, t, c)
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/schemachange.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,7 +310,7 @@ func makeIndexAddTpccTest(spec clusterSpec, warehouses int, length time.Duration
Duration: length,
})
},
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/scrub.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,6 @@ func makeScrubTPCCTest(
Duration: length,
})
},
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
}
}
6 changes: 3 additions & 3 deletions pkg/cmd/roachtest/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func registerLoadSplits(r *testRegistry) {

r.Add(testSpec{
Name: fmt.Sprintf("splits/load/uniform/nodes=%d", numNodes),
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(numNodes),
Run: func(ctx context.Context, t *test, c *cluster) {
// This number was determined experimentally. Often, but not always,
Expand Down Expand Up @@ -84,7 +84,7 @@ func registerLoadSplits(r *testRegistry) {
})
r.Add(testSpec{
Name: fmt.Sprintf("splits/load/sequential/nodes=%d", numNodes),
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(numNodes),
Run: func(ctx context.Context, t *test, c *cluster) {
runLoadSplits(ctx, t, c, splitParams{
Expand All @@ -104,7 +104,7 @@ func registerLoadSplits(r *testRegistry) {
})
r.Add(testSpec{
Name: fmt.Sprintf("splits/load/spanning/nodes=%d", numNodes),
MinVersion: "v2.2.0",
MinVersion: "v19.1.0",
Cluster: makeClusterSpec(numNodes),
Run: func(ctx context.Context, t *test, c *cluster) {
runLoadSplits(ctx, t, c, splitParams{
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/synctest.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ fi

r.Add(testSpec{
Name: "synctest",
MinVersion: `v2.2.0`,
MinVersion: "v19.1.0",
// This test sets up a custom file system; we don't want the cluster reused.
Cluster: makeClusterSpec(1, reuseNone()),
Run: func(ctx context.Context, t *test, c *cluster) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tpcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -404,7 +404,7 @@ func gceOrAws(cloud string, gce, aws int) int {
}

func maybeMinVersionForFixturesImport(cloud string) string {
const minVersionForFixturesImport = "v2.2.0"
const minVersionForFixturesImport = "v19.1.0"
if cloud == "aws" {
return minVersionForFixturesImport
}
Expand Down
10 changes: 6 additions & 4 deletions pkg/storage/raft_log_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -722,10 +722,11 @@ func TestSnapshotLogTruncationConstraints(t *testing.T) {
func TestTruncateLog(t *testing.T) {
defer leaktest.AfterTest(t)()
tc := testContext{}
cfg := TestStoreConfig(nil)
cfg.TestingKnobs.DisableRaftLogQueue = true
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
tc.Start(t, stopper)
tc.repl.store.SetRaftLogQueueActive(false)
tc.StartWithStoreConfig(t, stopper, cfg)

// Populate the log with 10 entries. Save the LastIndex after each write.
var indexes []uint64
Expand Down Expand Up @@ -887,10 +888,11 @@ func TestTruncateLogRecompute(t *testing.T) {
tc := testContext{
engine: eng,
}
cfg := TestStoreConfig(nil)
cfg.TestingKnobs.DisableRaftLogQueue = true
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
tc.Start(t, stopper)
tc.repl.store.SetRaftLogQueueActive(false)
tc.StartWithStoreConfig(t, stopper, cfg)

key := roachpb.Key("a")
repl := tc.store.LookupReplica(keys.MustAddr(key))
Expand Down
26 changes: 12 additions & 14 deletions pkg/storage/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1285,10 +1285,10 @@ func TestReplicaTSCacheLowWaterOnLease(t *testing.T) {
tc := testContext{manualClock: hlc.NewManualClock(123)}
cfg := TestStoreConfig(hlc.NewClock(tc.manualClock.UnixNano, time.Nanosecond))
cfg.TestingKnobs.DisableAutomaticLeaseRenewal = true
// Disable raft log truncation which confuses this test.
cfg.TestingKnobs.DisableRaftLogQueue = true
tc.StartWithStoreConfig(t, stopper, cfg)

// Disable raft log truncation which confuses this test.
tc.store.SetRaftLogQueueActive(false)
secondReplica, err := tc.addBogusReplicaToRangeDesc(context.TODO())
if err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -5159,15 +5159,12 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) {
t.Fatalf("%d: %s", i, pErr)
}
case roachpb.STAGING:
// TODO(nvanbenschoten): Avoid writing directly to the engine once
// there's a way to create a STAGING transaction record.
txnKey := keys.TransactionKey(pushee.Key, pushee.ID)
txnRecord := pushee.AsRecord()
txnRecord.Status = roachpb.STAGING
if err := engine.MVCCPutProto(
context.Background(), tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &txnRecord,
); err != nil {
t.Fatal(err)
et, etH := endTxnArgs(pushee, true)
et.InFlightWrites = []roachpb.SequencedWrite{
{Key: key, Sequence: 1},
}
if _, pErr := client.SendWrappedWith(context.Background(), tc.Sender(), etH, &et); pErr != nil {
t.Fatalf("%d: %s", i, pErr)
}
default:
t.Fatalf("unexpected status: %v", test.status)
Expand Down Expand Up @@ -6624,10 +6621,10 @@ func TestEntries(t *testing.T) {
// Disable ticks to avoid quiescence, which can result in empty
// entries being proposed and causing the test to flake.
cfg.RaftTickInterval = math.MaxInt32
cfg.TestingKnobs.DisableRaftLogQueue = true
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
tc.StartWithStoreConfig(t, stopper, cfg)
tc.repl.store.SetRaftLogQueueActive(false)

repl := tc.repl
rangeID := repl.RangeID
Expand Down Expand Up @@ -6774,10 +6771,11 @@ func TestEntries(t *testing.T) {
func TestTerm(t *testing.T) {
defer leaktest.AfterTest(t)()
tc := testContext{}
tsc := TestStoreConfig(nil)
tsc.TestingKnobs.DisableRaftLogQueue = true
stopper := stop.NewStopper()
defer stopper.Stop(context.TODO())
tc.Start(t, stopper)
tc.repl.store.SetRaftLogQueueActive(false)
tc.StartWithStoreConfig(t, stopper, tsc)

repl := tc.repl
rangeID := repl.RangeID
Expand Down
3 changes: 3 additions & 0 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -937,6 +937,9 @@ func NewStore(
if cfg.TestingKnobs.DisableMergeQueue {
s.setMergeQueueActive(false)
}
if cfg.TestingKnobs.DisableRaftLogQueue {
s.setRaftLogQueueActive(false)
}
if cfg.TestingKnobs.DisableReplicaGCQueue {
s.setReplicaGCQueueActive(false)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,8 @@ type StoreTestingKnobs struct {
DisableGCQueue bool
// DisableMergeQueue disables the merge queue.
DisableMergeQueue bool
// DisableReplicateQueue disables the raft log queue.
DisableRaftLogQueue bool
// DisableReplicaGCQueue disables the replica GC queue.
DisableReplicaGCQueue bool
// DisableReplicateQueue disables the replication queue.
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/txn_recovery_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,8 +71,8 @@ func TestTxnRecoveryFromStaging(t *testing.T) {
// state. Include both writes as the EndTransaction's in-flight writes.
et, etH := endTxnArgs(txn, true)
et.InFlightWrites = []roachpb.SequencedWrite{
{Key: keyA, Sequence: 0},
{Key: keyB, Sequence: 1},
{Key: keyA, Sequence: 1},
{Key: keyB, Sequence: 2},
}
etReply, pErr := client.SendWrappedWith(ctx, store.TestSender(), etH, &et)
if pErr != nil {
Expand Down

0 comments on commit 634dc24

Please sign in to comment.