diff --git a/pkg/base/config.go b/pkg/base/config.go index dd7a759d0c1b..8d22ccd1b782 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -231,18 +231,6 @@ type Config struct { // The flag exists mostly for the benefit of tests, and for // `cockroach start-single-node`. AutoInitializeCluster bool - - // IdleExistAfter, If nonzero, will cause the server to run normally for the - // indicated amount of time, wait for all SQL connections to terminate, - // start a `defaultCountdownDuration` countdown and exit upon countdown - // reaching zero if no new connections occur. New connections will be - // accepted at all times and will effectively delay the exit (indefinitely - // if there is always at least one connection or there are no connection - // for less than `defaultCountdownDuration`. A new `defaultCountdownDuration` - // countdown will start when no more SQL connections exist. - // The interval is specified with a suffix of 's' for seconds, 'm' for - // minutes, and 'h' for hours. - IdleExitAfter time.Duration } // HistogramWindowInterval is used to determine the approximate length of time diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 42d38faba186..f9b8fd0a911d 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -233,9 +233,6 @@ type TestTenantArgs struct { // to be created by StartTenant. Existing bool - // IdleExitAfter, if set will cause the tenant process to exit if idle. - IdleExitAfter time.Duration - // Settings allows the caller to control the settings object used for the // tenant cluster. Settings *cluster.Settings diff --git a/pkg/ccl/serverccl/BUILD.bazel b/pkg/ccl/serverccl/BUILD.bazel index 13c64de79d69..d85efcca6b72 100644 --- a/pkg/ccl/serverccl/BUILD.bazel +++ b/pkg/ccl/serverccl/BUILD.bazel @@ -28,7 +28,6 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/server/serverpb", - "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog/catconstants", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/ccl/serverccl/server_sql_test.go b/pkg/ccl/serverccl/server_sql_test.go index 1a80a497b0ec..d7ae0c21b414 100644 --- a/pkg/ccl/serverccl/server_sql_test.go +++ b/pkg/ccl/serverccl/server_sql_test.go @@ -14,12 +14,10 @@ import ( "io/ioutil" "net/http" "testing" - "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -143,71 +141,6 @@ func TestTenantHTTP(t *testing.T) { } -func TestIdleExit(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - - tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) - defer tc.Stopper().Stop(ctx) - - warmupDuration := 500 * time.Millisecond - countdownDuration := 4000 * time.Millisecond - tenant, err := tc.Server(0).StartTenant(ctx, - base.TestTenantArgs{ - TenantID: serverutils.TestTenantID(), - IdleExitAfter: warmupDuration, - TestingKnobs: base.TestingKnobs{ - TenantTestingKnobs: &sql.TenantTestingKnobs{ - ClusterSettingsUpdater: cluster.MakeTestingClusterSettings().MakeUpdater(), - IdleExitCountdownDuration: countdownDuration, - }, - }, - Stopper: tc.Stopper(), - }) - - require.NoError(t, err) - - time.Sleep(warmupDuration / 2) - log.Infof(context.Background(), "Opening first con") - db := serverutils.OpenDBConn( - t, tenant.SQLAddr(), "", false, tc.Stopper(), - ) - r := sqlutils.MakeSQLRunner(db) - r.QueryStr(t, `SELECT 1`) - require.NoError(t, db.Close()) - - time.Sleep(warmupDuration/2 + countdownDuration/2) - - // Opening a connection in the middle of the countdown should stop the - // countdown timer. Closing the connection will restart the countdown. - log.Infof(context.Background(), "Opening second con") - db = serverutils.OpenDBConn( - t, tenant.SQLAddr(), "", false, tc.Stopper(), - ) - r = sqlutils.MakeSQLRunner(db) - r.QueryStr(t, `SELECT 1`) - require.NoError(t, db.Close()) - - time.Sleep(countdownDuration / 2) - - // If the tenant is stopped, that most likely means that the second connection - // didn't stop the countdown - select { - case <-tc.Stopper().IsStopped(): - t.Error("stop on idle triggered too early") - default: - } - - time.Sleep(countdownDuration * 3 / 2) - - select { - case <-tc.Stopper().IsStopped(): - default: - t.Error("stop on idle didn't trigger") - } -} - func TestNonExistentTenant(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index bd766283ca31..e9f53d45299b 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -1532,21 +1532,6 @@ without any other details. `, } - IdleExitAfter = FlagInfo{ - Name: "idle-exit-after", - Description: ` -If nonzero, will cause the server to run normally for the -indicated amount of time, wait for all SQL connections to terminate, -start a 30s countdown and exit upon countdown reaching zero if no new -connections occur. New connections will be accepted at all times and -will effectively delay the exit (indefinitely if there is always at least -one connection or there are no connection for less than 30 sec. -A new 30s countdown will start when no more SQL connections -exist. The interval is specified with a suffix of 's' for seconds, -'m' for minutes, and 'h' for hours. -`, - } - ExportTableTarget = FlagInfo{ Name: "table", Description: `Select the table to export data from.`, diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index 60f243d9bae9..528fc8385d00 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -971,8 +971,6 @@ func init() { stringSliceFlag(f, &serverCfg.SQLConfig.TenantKVAddrs, cliflags.KVAddrs) - durationFlag(f, &serverCfg.IdleExitAfter, cliflags.IdleExitAfter) - boolFlag(f, &serverCfg.ExternalIODirConfig.DisableHTTP, cliflags.ExternalIODisableHTTP) boolFlag(f, &serverCfg.ExternalIODirConfig.DisableOutbound, cliflags.ExternalIODisabled) boolFlag(f, &serverCfg.ExternalIODirConfig.DisableImplicitCredentials, cliflags.ExternalIODisableImplicitCredentials) diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 2fd1df44af46..1cdd15cee732 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "doc.go", "drain.go", "grpc_server.go", - "idle_monitor.go", "index_usage_stats.go", "init.go", "init_handshake.go", @@ -267,7 +266,6 @@ go_test( "connectivity_test.go", "drain_test.go", "graphite_test.go", - "idle_monitor_test.go", "index_usage_stats_test.go", "init_handshake_test.go", "intent_test.go", @@ -358,7 +356,6 @@ go_test( "//pkg/util/log", "//pkg/util/log/logpb", "//pkg/util/metric", - "//pkg/util/netutil", "//pkg/util/netutil/addr", "//pkg/util/protoutil", "//pkg/util/randutil", diff --git a/pkg/server/idle_monitor.go b/pkg/server/idle_monitor.go deleted file mode 100644 index 1029c883ff63..000000000000 --- a/pkg/server/idle_monitor.go +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package server - -import ( - "context" - "time" - - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -// IdleMonitor monitors the active connections and calls onIdle() when -// there are no more active connections. -// It will stays dormant initially for the warmup duration interval. -// Once it detects that there are no more active connections, it will -// start a countdown timer and call onIdle() once the countdown timer expires. -// Any new connections before the times expires will stop the timer and start -// the active connection monitoring again. -type IdleMonitor struct { - mu syncutil.Mutex - onIdle func() - activated bool - activeConnectionCount uint32 - totalConnectionCount uint32 - countdownTimer *time.Timer - shutdownInitiated bool - countdownDuration time.Duration -} - -// defaultCountdownDuration specifies the time, the monitor will wait, after -// the number of active connections drops to zero, before the onIdle is -// called. -// If this is changed, cli/cliflags/flags.go should be updated as -// well to reflect the new value. -const defaultCountdownDuration = 30 * time.Second - -// MakeIdleMonitor creates a new IdleMonitor. -func MakeIdleMonitor( - ctx context.Context, - warmupDuration time.Duration, - onIdle func(), - countdownDuration ...time.Duration, -) *IdleMonitor { - monitor := &IdleMonitor{onIdle: onIdle} - - if len(countdownDuration) > 0 { - monitor.countdownDuration = countdownDuration[0] - } else { - // Default if there isn't an overwrite - monitor.countdownDuration = defaultCountdownDuration - } - log.VEventf(ctx, 2, - "create with warmup %s and countdown %s durations", warmupDuration, - monitor.countdownDuration, - ) - // Activate the monitor after the warmup duration is over and trigger the - // countdown timer if the number of active connections is zero. - time.AfterFunc(warmupDuration, func() { - log.VEventf(ctx, 2, "warmup duration is over") - monitor.mu.Lock() - defer monitor.mu.Unlock() - monitor.activated = true - if monitor.activeConnectionCount == 0 { - monitor.countdownTimer = time.AfterFunc(monitor.countdownDuration, func() { - log.VEventf(ctx, 2, "firing because no "+ - "connection ever occurred and it has been %s after warmup", - monitor.countdownDuration, - ) - onIdle() - }) - } - }) - - return monitor -} - -// NewConnection registers a new connection and if there is a shutdown -// timer running - stops the time. It returns false to indicate that there is -// no shutdown in progress and true when the shutdown already started. -func (i *IdleMonitor) NewConnection(ctx context.Context) bool { - log.VEventf(ctx, 3, "new connection") - i.mu.Lock() - defer i.mu.Unlock() - - // If there is countdown timer - stop it. - if i.countdownTimer != nil { - log.VEventf(ctx, 3, "countdown timer found - stopping") - i.shutdownInitiated = !i.countdownTimer.Stop() - if i.shutdownInitiated { - log.VEventf(ctx, 2, "shutdown already initiated") - return true - } - log.VEventf(ctx, 3, "countdown timer stopped successfully") - i.countdownTimer = nil - } - - // Update connection counts - i.activeConnectionCount++ - i.totalConnectionCount++ - log.VEventf(ctx, 3, "active connections %d and total connections %d", - i.activeConnectionCount, i.totalConnectionCount) - return false -} - -// CloseConnection is called when a connection terminates. It will update the -// connection counters and start a new shutdown timer if the number of -// active connections reaches zero and the warmup period is over. -func (i *IdleMonitor) CloseConnection(ctx context.Context) { - log.VEventf(ctx, 3, "closed connection") - i.mu.Lock() - defer i.mu.Unlock() - if i.shutdownInitiated { - log.VEventf(ctx, 2, "shutdown already initiated") - return - } - i.activeConnectionCount-- - log.VEventf(ctx, 3, "post close connection active connection count %d", - i.activeConnectionCount) - if i.activeConnectionCount == 0 && i.activated { - log.VEventf(ctx, 3, - "zero active connections and warmup done - starting countdown timer") - i.countdownTimer = time.AfterFunc(i.countdownDuration, func() { - log.VEventf(ctx, 2, "firing because warmed up and no "+ - "active connections for %s", i.countdownDuration) - i.onIdle() - }) - } -} diff --git a/pkg/server/idle_monitor_test.go b/pkg/server/idle_monitor_test.go deleted file mode 100644 index 4418a61e5601..000000000000 --- a/pkg/server/idle_monitor_test.go +++ /dev/null @@ -1,262 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package server - -import ( - "context" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/stretchr/testify/require" -) - -var ( - warmupDuration = 600 * time.Millisecond - countdownDuration = 1200 * time.Millisecond - delta = 150 * time.Millisecond -) - -func TestMakeIdleMonitor(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.UnderShort(t) - - ctx := context.Background() - - start := timeutil.Now() - - var handlerCalled syncutil.AtomicBool - monitor := MakeIdleMonitor(ctx, warmupDuration, func() { - // The handler is called after the warmup and countdown durations - require.WithinDuration(t, - start.Add(warmupDuration+countdownDuration+delta), timeutil.Now(), delta, - ) - handlerCalled.Set(true) - }, countdownDuration) - - require.EqualValues(t, countdownDuration, monitor.countdownDuration) - - // activated is set after the warmup duration is over - time.Sleep(warmupDuration - delta) - monitor.mu.Lock() - require.False(t, monitor.activated) - require.Nil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(2 * delta) - - monitor.mu.Lock() - require.True(t, monitor.activated) - // Countdown timer triggers as there is no connection. - require.NotNil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(countdownDuration + 2*delta) - - monitor.mu.Lock() - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 0, monitor.totalConnectionCount) - monitor.mu.Unlock() - - require.True(t, handlerCalled.Get()) - - require.True(t, monitor.NewConnection(ctx)) - - monitor.mu.Lock() - require.True(t, monitor.shutdownInitiated) - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 0, monitor.totalConnectionCount) - monitor.mu.Unlock() -} - -func TestMakeIdleMonitor_WithConnectionDuringWarmup(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.UnderShort(t) - - ctx := context.Background() - - start := timeutil.Now() - - var handlerCalled syncutil.AtomicBool - monitor := MakeIdleMonitor(ctx, warmupDuration, func() { - // The handler is called after the warmup and countdown durations - require.WithinDuration(t, - start.Add(warmupDuration+countdownDuration+delta), timeutil.Now(), delta, - ) - handlerCalled.Set(true) - }, countdownDuration) - - time.AfterFunc(warmupDuration/2, func() { - require.False(t, monitor.NewConnection(ctx)) - }) - - require.EqualValues(t, countdownDuration, monitor.countdownDuration) - - // activated is set after the warmup duration is over - time.Sleep(warmupDuration - delta) - - monitor.mu.Lock() - require.False(t, monitor.activated) - require.Nil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(2 * delta) - - monitor.mu.Lock() - require.True(t, monitor.activated) - // Countdown timer does not trigger as there is a connection. - require.Nil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(countdownDuration + 2*delta) - - monitor.mu.Lock() - require.EqualValues(t, 1, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() - - require.False(t, handlerCalled.Get()) - - require.False(t, monitor.NewConnection(ctx)) - - monitor.mu.Lock() - require.False(t, monitor.shutdownInitiated) - require.EqualValues(t, 2, monitor.activeConnectionCount) - require.EqualValues(t, 2, monitor.totalConnectionCount) - monitor.mu.Unlock() -} - -func TestMakeIdleMonitor_WithBriefConnectionDuringWarmup(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.UnderShort(t) - - ctx := context.Background() - - start := timeutil.Now() - - var handlerCalled syncutil.AtomicBool - monitor := MakeIdleMonitor(ctx, warmupDuration, func() { - // The handler is called after the warmup and countdown durations - require.WithinDuration(t, - start.Add(warmupDuration+countdownDuration+delta), timeutil.Now(), delta, - ) - handlerCalled.Set(true) - }, countdownDuration) - - require.EqualValues(t, countdownDuration, monitor.countdownDuration) - - time.AfterFunc(warmupDuration/3, func() { - require.False(t, monitor.NewConnection(ctx)) - }) - time.AfterFunc(warmupDuration*2/3, func() { - monitor.CloseConnection(ctx) - }) - - // activated is set after the warmup duration is over - time.Sleep(warmupDuration - delta) - - monitor.mu.Lock() - require.False(t, monitor.activated) - require.Nil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(2 * delta) - - monitor.mu.Lock() - require.True(t, monitor.activated) - // Countdown timer triggers as there is no connection. - require.NotNil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(countdownDuration + 2*delta) - - monitor.mu.Lock() - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() - - require.True(t, handlerCalled.Get()) - - require.True(t, monitor.NewConnection(ctx)) - - monitor.mu.Lock() - require.True(t, monitor.shutdownInitiated) - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() -} - -func TestMakeIdleMonitor_WithBriefConnectionDuringCountdown(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.UnderShort(t) - - ctx := context.Background() - - start := timeutil.Now() - - var handlerCalled syncutil.AtomicBool - monitor := MakeIdleMonitor(ctx, warmupDuration, func() { - // The handler is called after the warmup and countdown durations - require.WithinDuration(t, - start.Add(warmupDuration+countdownDuration*5/3+delta), timeutil.Now(), delta, - ) - handlerCalled.Set(true) - }, countdownDuration) - - require.EqualValues(t, countdownDuration, monitor.countdownDuration) - - time.AfterFunc(warmupDuration+countdownDuration/3, func() { - require.False(t, monitor.NewConnection(ctx)) - }) - time.AfterFunc(warmupDuration+countdownDuration*2/3, func() { - monitor.CloseConnection(ctx) - }) - - // activated is set after the warmup duration is over - time.Sleep(warmupDuration - delta) - - monitor.mu.Lock() - require.False(t, monitor.activated) - require.Nil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(2 * delta) - - monitor.mu.Lock() - require.True(t, monitor.activated) - // Countdown timer triggers as there is no connection. - require.NotNil(t, monitor.countdownTimer) - monitor.mu.Unlock() - - time.Sleep(countdownDuration + 2*delta) - - monitor.mu.Lock() - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() - - require.False(t, handlerCalled.Get()) - - time.Sleep(warmupDuration + countdownDuration*5/3 + 3*delta) - - require.True(t, handlerCalled.Get()) - - require.True(t, monitor.NewConnection(ctx)) - - monitor.mu.Lock() - require.True(t, monitor.shutdownInitiated) - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() -} diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index a9a35f3a0f1f..4fa76f510067 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -77,12 +77,6 @@ func StartTenant( nil, // tlsConfig nil, // handler ) - knobs := baseCfg.TestingKnobs.TenantTestingKnobs - if tenantKnobs, ok := knobs.(*sql.TenantTestingKnobs); ok && tenantKnobs.IdleExitCountdownDuration != 0 { - SetupIdleMonitor(ctx, args.stopper, baseCfg.IdleExitAfter, connManager, tenantKnobs.IdleExitCountdownDuration) - } else { - SetupIdleMonitor(ctx, args.stopper, baseCfg.IdleExitAfter, connManager) - } pgL, err := ListenAndUpdateAddrs(ctx, &args.Config.SQLAddr, &args.Config.SQLAdvertiseAddr, "sql") if err != nil { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 9063a3b05c12..3d776de11390 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -16,7 +16,6 @@ import ( "encoding/base64" "fmt" "math/rand" - "net" "net/http" "net/http/cookiejar" "net/url" @@ -59,7 +58,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -534,42 +532,6 @@ func (t *TestTenant) TestingKnobs() *base.TestingKnobs { return &t.Cfg.TestingKnobs } -// SetupIdleMonitor will monitor the active connections and if there are none, -// will activate a `defaultCountdownDuration` countdown timer and terminate -// the application. The monitoring will start after a warmup period -// specified by warmupDuration. If the warmupDuration is zero, the idle -// detection will be turned off. -func SetupIdleMonitor( - ctx context.Context, - stopper *stop.Stopper, - warmupDuration time.Duration, - server netutil.Server, - countdownDuration ...time.Duration, -) *IdleMonitor { - if warmupDuration != 0 { - log.VEventf(ctx, 2, "idle exit will activate after warmup duration of %s", warmupDuration) - oldConnStateHandler := server.ConnState - idleMonitor := MakeIdleMonitor(ctx, warmupDuration, - func() { - log.VEventf(ctx, 2, "idle exiting") - stopper.Stop(ctx) - }, - countdownDuration..., - ) - server.ConnState = func(conn net.Conn, state http.ConnState) { - if state == http.StateNew { - defer oldConnStateHandler(conn, state) - idleMonitor.NewConnection(ctx) - } else if state == http.StateClosed { - defer idleMonitor.CloseConnection(ctx) - oldConnStateHandler(conn, state) - } - } - return idleMonitor - } - return nil -} - // StartTenant starts a SQL tenant communicating with this TestServer. func (ts *TestServer) StartTenant( ctx context.Context, params base.TestTenantArgs, @@ -613,7 +575,6 @@ func (ts *TestServer) StartTenant( } baseCfg := makeTestBaseConfig(st) baseCfg.TestingKnobs = params.TestingKnobs - baseCfg.IdleExitAfter = params.IdleExitAfter baseCfg.Insecure = params.ForceInsecure if params.AllowSettingClusterSettings { tenantKnobs, ok := baseCfg.TestingKnobs.TenantTestingKnobs.(*sql.TenantTestingKnobs) diff --git a/pkg/server/testserver_test.go b/pkg/server/testserver_test.go index 56df09ff58b7..b03cc6af3d9a 100644 --- a/pkg/server/testserver_test.go +++ b/pkg/server/testserver_test.go @@ -12,20 +12,12 @@ package server import ( "context" - "net" - "reflect" "testing" - "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/netutil" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/stretchr/testify/require" ) func TestServerTest(t *testing.T) { @@ -34,104 +26,3 @@ func TestServerTest(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) } - -func TestSetupIdleMonitor_WithNoWarmupProvided(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - testServer := netutil.MakeServer(stopper, nil, nil) - oldConnStateHandler := testServer.ConnState - require.Nil(t, SetupIdleMonitor(ctx, stopper, 0, testServer)) - // The conn state handler doesn't change - require.Equal(t, - reflect.ValueOf(oldConnStateHandler), - reflect.ValueOf(testServer.ConnState), - ) -} - -func TestSetupIdleMonitor_WithWarmupProvided(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, 66767, "flaky test") - defer log.Scope(t).Close(t) - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - ts := netutil.MakeServer(stopper, nil, nil) - oldConnStateHandler := ts.ConnState - warmupDuration := 200 * time.Millisecond - countdownDuration := 200 * time.Millisecond - monitor := SetupIdleMonitor(ctx, stopper, warmupDuration, ts, countdownDuration) - require.NotNil(t, monitor) - // The conn state handler changes - require.NotEqual(t, - reflect.ValueOf(oldConnStateHandler), - reflect.ValueOf(ts.ConnState), - ) - - ln, err := net.Listen("tcp", "") - require.NoError(t, err) - require.NoError(t, stopper.RunAsyncTask(ctx, "ln-close", func(ctx context.Context) { - <-stopper.ShouldQuiesce() - require.NoError(t, ln.Close()) - })) - - var connectionEstablished syncutil.AtomicBool - var connectionOver syncutil.AtomicBool - - require.NoError(t, stopper.RunAsyncTask(ctx, "serve-conn", func(ctx context.Context) { - _ = ts.ServeWith(ctx, stopper, ln, func(conn net.Conn) { - defer conn.Close() - connectionEstablished.Set(true) - time.Sleep(100 * time.Millisecond) - connectionOver.Set(true) - }) - })) - - monitor.mu.Lock() - require.EqualValues(t, 0, monitor.activeConnectionCount) - monitor.mu.Unlock() - - conn, err := net.Dial("tcp", ln.Addr().String()) - require.NotNil(t, conn) - require.NoError(t, err) - defer conn.Close() - - time.Sleep(50 * time.Millisecond) - - require.True(t, connectionEstablished.Get()) - - monitor.mu.Lock() - require.False(t, monitor.activated) - require.EqualValues(t, 1, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() - - time.Sleep(100 * time.Millisecond) - - require.True(t, connectionOver.Get()) - monitor.mu.Lock() - require.False(t, monitor.activated) - require.EqualValues(t, 0, monitor.activeConnectionCount) - require.EqualValues(t, 1, monitor.totalConnectionCount) - monitor.mu.Unlock() - - time.Sleep(100 * time.Millisecond) - - monitor.mu.Lock() - require.True(t, monitor.activated) - monitor.mu.Unlock() - - time.Sleep(200 * time.Millisecond) - - select { - case <-stopper.IsStopped(): - default: - t.Error("stop on idle didn't trigger") - } -} diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index c2682cd5981e..ba3c2383a5e4 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1185,11 +1185,6 @@ type TenantTestingKnobs struct { // server's codec, but nothing else (e.g. its certs). TenantIDCodecOverride roachpb.TenantID - // IdleExitCountdownDuration is a filed that if set, will overwrite the - // default countdown duration of the countdown timer that leads to shutdown in - // case of no SQL connections. - IdleExitCountdownDuration time.Duration - // OverrideTokenBucketProvider allows a test-only TokenBucketProvider (which // can optionally forward requests to the real provider). OverrideTokenBucketProvider func(origProvider kvtenant.TokenBucketProvider) kvtenant.TokenBucketProvider