Skip to content

Commit

Permalink
Merge #106145
Browse files Browse the repository at this point in the history
106145: kvserver: deflake all tests that restart test cluster servers r=erikgrinaker a=tbg

This PR reworks how TestCluster's reusable listeners work. They are now enforced in tests that restart servers under the same listening address, since restarting without one opens the door to a race where the port is now assigned to another process.

The 23.1 backport will address #105940.

Epic: None
Release note: None

Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
  • Loading branch information
craig[bot] and tbg committed Jul 13, 2023
2 parents a1d096a + 4e8a998 commit 5a6c91a
Show file tree
Hide file tree
Showing 24 changed files with 318 additions and 106 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,7 @@ ALL_TESTS = [
"//pkg/testutils/lint/passes/timer:timer_test",
"//pkg/testutils/lint/passes/unconvert:unconvert_test",
"//pkg/testutils/lint:lint_test",
"//pkg/testutils/listenerutil:listenerutil_test",
"//pkg/testutils/release:release_test",
"//pkg/testutils/sqlutils:sqlutils_test",
"//pkg/testutils/testcluster:testcluster_test",
Expand Down Expand Up @@ -2152,6 +2153,8 @@ GO_TARGETS = [
"//pkg/testutils/lint/passes/unconvert:unconvert_test",
"//pkg/testutils/lint:lint",
"//pkg/testutils/lint:lint_test",
"//pkg/testutils/listenerutil:listenerutil",
"//pkg/testutils/listenerutil:listenerutil_test",
"//pkg/testutils/localtestcluster:localtestcluster",
"//pkg/testutils/metrictestutils:metrictestutils",
"//pkg/testutils/pgtest:pgtest",
Expand Down
1 change: 1 addition & 0 deletions pkg/base/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ go_library(
"//pkg/security/username",
"//pkg/server/autoconfig/acprovider",
"//pkg/settings/cluster",
"//pkg/testutils/listenerutil",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/humanizeutil",
Expand Down
15 changes: 9 additions & 6 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -204,12 +205,14 @@ type TestClusterArgs struct {
// and potentially adjusted according to ReplicationMode.
ServerArgsPerNode map[int]TestServerArgs

// If reusable listeners is true, then restart should keep listeners untouched
// so that servers are kept on the same ports. It is up to the test to set
// proxy listeners to TestServerArgs.Listener that would survive
// net.Listener.Close() and then allow restarted server to use them again.
// See testutils.ListenerRegistry.
ReusableListeners bool
// If set, listeners will be created from the below registry and they will be
// retained across restarts (i.e. servers are kept on the same ports, but
// avoiding races where another process grabs the port while the server is
// down). It's also possible not to set this field but set a *ReusableListener
// directly in TestServerArgs.Listener. If a non-reusable listener is set in
// that field, RestartServer will return an error to guide the developer
// towards a non-flaky pattern.
ReusableListenerReg *listenerutil.ListenerRegistry
}

// DefaultTestTenantOptions specifies the conditions under which the default
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -394,6 +394,7 @@ go_test(
"//pkg/storage",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/listenerutil",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
Expand Down
9 changes: 4 additions & 5 deletions pkg/cli/debug_recover_loss_of_quorum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
Expand Down Expand Up @@ -437,7 +438,7 @@ func TestHalfOnlineLossOfQuorumRecovery(t *testing.T) {
})
defer c.Cleanup()

listenerReg := testutils.NewListenerRegistry()
listenerReg := listenerutil.NewListenerRegistry()
defer listenerReg.Close()

storeReg := server.NewStickyInMemEnginesRegistry()
Expand All @@ -461,7 +462,6 @@ func TestHalfOnlineLossOfQuorumRecovery(t *testing.T) {
StickyEngineRegistry: storeReg,
},
},
Listener: listenerReg.GetOrFail(t, i),
StoreSpecs: []base.StoreSpec{
{
InMemory: true,
Expand All @@ -470,8 +470,8 @@ func TestHalfOnlineLossOfQuorumRecovery(t *testing.T) {
}
}
tc := testcluster.NewTestCluster(t, 3, base.TestClusterArgs{
ReusableListeners: true,
ServerArgsPerNode: sa,
ReusableListenerReg: listenerReg,
ServerArgsPerNode: sa,
})
tc.Start(t)
s := sqlutils.MakeSQLRunner(tc.Conns[0])
Expand Down Expand Up @@ -554,7 +554,6 @@ func TestHalfOnlineLossOfQuorumRecovery(t *testing.T) {
// NB: If recovery is not performed, server will just hang on startup.
// This is caused by liveness range becoming unavailable and preventing any
// progress. So it is likely that test will timeout if basic workflow fails.
listenerReg.ReopenOrFail(t, 0)
require.NoError(t, tc.RestartServer(0), "restart failed")
s = sqlutils.MakeSQLRunner(tc.Conns[0])

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -457,6 +457,7 @@ go_test(
"//pkg/testutils/echotest",
"//pkg/testutils/gossiputil",
"//pkg/testutils/kvclientutils",
"//pkg/testutils/listenerutil",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
Expand Down
46 changes: 34 additions & 12 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/storageutils"
Expand Down Expand Up @@ -92,11 +93,14 @@ func TestStoreRecoverFromEngine(t *testing.T) {

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgs: base.TestServerArgs{
StoreSpecs: []base.StoreSpec{
{
Expand Down Expand Up @@ -198,14 +202,17 @@ func TestStoreRecoverWithErrors(t *testing.T) {

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

numIncrements := 0
keyA := roachpb.Key("a")

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
Expand Down Expand Up @@ -341,6 +348,8 @@ func TestRestoreReplicas(t *testing.T) {

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

const numServers int = 2
stickyServerArgs := make(map[int]base.TestServerArgs)
Expand All @@ -363,8 +372,9 @@ func TestRestoreReplicas(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 2,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: stickyServerArgs,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgsPerNode: stickyServerArgs,
})
defer tc.Stopper().Stop(ctx)
store := tc.GetFirstStoreFromServer(t, 0)
Expand Down Expand Up @@ -661,6 +671,8 @@ func TestSnapshotAfterTruncation(t *testing.T) {
t.Run(name, func(t *testing.T) {
stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

const numServers int = 3
stickyServerArgs := make(map[int]base.TestServerArgs)
Expand All @@ -683,8 +695,9 @@ func TestSnapshotAfterTruncation(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, numServers,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: stickyServerArgs,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgsPerNode: stickyServerArgs,
})
defer tc.Stopper().Stop(ctx)
store := tc.GetFirstStoreFromServer(t, 0)
Expand Down Expand Up @@ -4192,6 +4205,8 @@ func TestInitRaftGroupOnRequest(t *testing.T) {

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

const numServers int = 2
stickyServerArgs := make(map[int]base.TestServerArgs)
Expand All @@ -4214,8 +4229,9 @@ func TestInitRaftGroupOnRequest(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, numServers,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: stickyServerArgs,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgsPerNode: stickyServerArgs,
})
defer tc.Stopper().Stop(ctx)

Expand Down Expand Up @@ -4703,6 +4719,8 @@ func TestDefaultConnectionDisruptionDoesNotInterfereWithSystemTraffic(t *testing

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

stopper := stop.NewStopper()
ctx := context.Background()
Expand Down Expand Up @@ -4764,8 +4782,9 @@ func TestDefaultConnectionDisruptionDoesNotInterfereWithSystemTraffic(t *testing

tc := testcluster.StartTestCluster(t, numServers,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: stickyServerArgs,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgsPerNode: stickyServerArgs,
})
defer tc.Stopper().Stop(ctx)
// Make a key that's in the user data space.
Expand Down Expand Up @@ -5097,6 +5116,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) {
stickyEngineRegistry server.StickyInMemEnginesRegistry,
) {
stickyEngineRegistry = server.NewStickyInMemEnginesRegistry()
lisReg := listenerutil.NewListenerRegistry()
const numServers int = 3
stickyServerArgs := make(map[int]base.TestServerArgs)
for i := 0; i < numServers; i++ {
Expand Down Expand Up @@ -5128,10 +5148,12 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) {

tc = testcluster.StartTestCluster(t, numServers,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgsPerNode: stickyServerArgs,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgsPerNode: stickyServerArgs,
})

tc.Stopper().AddCloser(stop.CloserFn(lisReg.Close))
db = tc.GetFirstStoreFromServer(t, 1).DB()

// Split off a non-system range so we don't have to account for node liveness
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/kvclientutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
Expand Down Expand Up @@ -2158,13 +2159,16 @@ func TestLeaseNotUsedAfterRestart(t *testing.T) {

stickyEngineRegistry := server.NewStickyInMemEnginesRegistry()
defer stickyEngineRegistry.CloseAllStickyInMemEngines()
lisReg := listenerutil.NewListenerRegistry()
defer lisReg.Close()

var leaseAcquisitionTrap atomic.Value
ctx := context.Background()
manual := hlc.NewHybridManualClock()
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: lisReg,
ServerArgs: base.TestServerArgs{
StoreSpecs: []base.StoreSpec{
{
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/client_store_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/listenerutil"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -120,19 +121,18 @@ func TestStoreLoadReplicaQuiescent(t *testing.T) {
testutils.RunTrueAndFalse(t, "kv.expiration_leases_only.enabled", func(t *testing.T, expOnly bool) {
storeReg := server.NewStickyInMemEnginesRegistry()
defer storeReg.CloseAllStickyInMemEngines()
listenerReg := testutils.NewListenerRegistry()
listenerReg := listenerutil.NewListenerRegistry()
defer listenerReg.Close()

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, expOnly)

tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ReusableListeners: true,
ReplicationMode: base.ReplicationManual,
ReusableListenerReg: listenerReg,
ServerArgs: base.TestServerArgs{
Settings: st,
Listener: listenerReg.GetOrFail(t, 0),
RaftConfig: base.RaftConfig{
RaftTickInterval: 100 * time.Millisecond,
},
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/loqrecovery/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ go_test(
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/listenerutil",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/testcluster",
Expand All @@ -105,6 +106,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/randutil",
"//pkg/util/stop",
"//pkg/util/strutil",
"//pkg/util/timeutil",
"//pkg/util/uuid",
Expand Down
Loading

0 comments on commit 5a6c91a

Please sign in to comment.