Skip to content

Commit

Permalink
server: add cluster setting for SQL keep alive time
Browse files Browse the repository at this point in the history
Previously, we only had an environment variable for controlling the keep
alive probe interval and keep alive idle time for pgwire connections.
Unfortunately, this defaulted to around 10 minutes on Linux since the
probe count was not configurable either. To address this, this patch
adds the settings: server.sql_tcp_keep_alive_timeout and
server.sql_tcp_keep_alive_probe_interval, which allow much better
control than previously. These settings can be used to pick an overall
connection time out for idle connections and the probe interval.

Fixes: #115422
Release note (sql change): Add configurable settings for total TCP keep
alive timeouts (server.sql_tcp_keep_alive_timeout) and TCP probe
intervals (server.sql_tcp_keep_alive_probe_interval) for SQL
connections.
  • Loading branch information
fqazi committed Dec 9, 2023
1 parent b2ef8cb commit c753ad9
Show file tree
Hide file tree
Showing 8 changed files with 250 additions and 13 deletions.
2 changes: 2 additions & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,8 @@ server.shutdown.connections.timeout duration 0s the maximum amount of time a ser
server.shutdown.initial_wait duration 0s the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting. --drain-wait is to specify the duration of the whole draining process, while server.shutdown.initial_wait is to set the wait time for health probes to notice that the node is not ready.) application
server.shutdown.jobs.timeout duration 10s the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown application
server.shutdown.transactions.timeout duration 10s the timeout for waiting for active transactions to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) application
server.sql_tcp_keep_alive.interval duration 15s time between keep alive probes and idle time before probes are sent out application
server.sql_tcp_keep_alive.timeout duration 2m0s maximum time for which an idle TCP connection to the server will be kept alive application
server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead application
server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM application
server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled boolean true if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt application
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,8 @@
<tr><td><div id="setting-server-shutdown-jobs-wait" class="anchored"><code>server.shutdown.jobs.timeout</code></div></td><td>duration</td><td><code>10s</code></td><td>the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-shutdown-lease-transfer-wait" class="anchored"><code>server.shutdown.lease_transfer_iteration.timeout</code></div></td><td>duration</td><td><code>5s</code></td><td>the timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-shutdown-query-wait" class="anchored"><code>server.shutdown.transactions.timeout</code></div></td><td>duration</td><td><code>10s</code></td><td>the timeout for waiting for active transactions to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-sql-tcp-keep-alive-interval" class="anchored"><code>server.sql_tcp_keep_alive.interval</code></div></td><td>duration</td><td><code>15s</code></td><td>time between keep alive probes and idle time before probes are sent out</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-sql-tcp-keep-alive-timeout" class="anchored"><code>server.sql_tcp_keep_alive.timeout</code></div></td><td>duration</td><td><code>2m0s</code></td><td>maximum time for which an idle TCP connection to the server will be kept alive</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-time-until-store-dead" class="anchored"><code>server.time_until_store_dead</code></div></td><td>duration</td><td><code>5m0s</code></td><td>the time after which if there is no new gossiped information about a store, it is considered dead</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-user-login-cert-password-method-auto-scram-promotion-enabled" class="anchored"><code>server.user_login.cert_password_method.auto_scram_promotion.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>whether to automatically promote cert-password authentication to use SCRAM</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-user-login-downgrade-scram-stored-passwords-to-bcrypt-enabled" class="anchored"><code>server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
95 changes: 94 additions & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -467,6 +467,7 @@ go_test(
"statements_test.go",
"status_ext_test.go",
"sticky_vfs_test.go",
"tcp_keepalive_manager_test.go",
"tenant_delayed_id_set_test.go",
"tenant_range_lookup_test.go",
"testserver_test.go",
Expand Down Expand Up @@ -582,5 +583,97 @@ go_test(
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//metadata",
"@org_golang_google_grpc//status",
],
] + select({
"@io_bazel_rules_go//go/platform:android_386": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:android_amd64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:android_arm": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:android_arm64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:darwin_arm64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:ios_arm64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_386": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_amd64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_arm": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_arm64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_mips": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_mips64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_mips64le": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_mipsle": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_ppc64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_ppc64le": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_riscv64": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"@io_bazel_rules_go//go/platform:linux_s390x": [
"//pkg/util/ctxgroup",
"//pkg/util/sysutil",
"@com_github_cockroachdb_cmux//:cmux",
],
"//conditions:default": [],
}),
)
1 change: 1 addition & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -2243,6 +2243,7 @@ func (s *topLevelServer) AcceptClients(ctx context.Context) error {
s.pgPreServer,
s.serverController.sqlMux,
s.pgL,
s.ClusterSettings(),
&s.cfg.SocketFile,
); err != nil {
return err
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1895,12 +1895,13 @@ func startServeSQL(
pgPreServer *pgwire.PreServeConnHandler,
serveConn func(ctx context.Context, conn net.Conn, preServeStatus pgwire.PreServeStatus) error,
pgL net.Listener,
st *cluster.Settings,
socketFileCfg *string,
) error {
log.Ops.Info(ctx, "serving sql connections")
// Start servicing SQL connections.

tcpKeepAlive := makeTCPKeepAliveManager()
tcpKeepAlive := makeTCPKeepAliveManager(st)

// The connManager is responsible for tearing down the net.Conn
// objects when the stopper tells us to shut down.
Expand Down
54 changes: 43 additions & 11 deletions pkg/server/tcp_keepalive_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,31 +17,44 @@ import (
"time"

"github.com/cockroachdb/cmux"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/sysutil"
)

var KeepAliveTimeout = settings.RegisterDurationSetting(
settings.ApplicationLevel,
"server.sql_tcp_keep_alive.timeout",
"maximum time for which an idle TCP connection to the server will be kept alive",
time.Minute*2,
settings.WithPublic,
)

var KeepAliveProbeFrequency = settings.RegisterDurationSetting(
settings.ApplicationLevel,
"server.sql_tcp_keep_alive.interval",
"time between keep alive probes and idle time before probes are sent out",
time.Second*15,
settings.WithPublic,
)

type tcpKeepAliveManager struct {
// The keepalive duration.
tcpKeepAlive time.Duration
// loggedKeepAliveStatus ensures that errors about setting the TCP
// keepalive status are only reported once.
loggedKeepAliveStatus int32
settings *cluster.Settings
}

func makeTCPKeepAliveManager() tcpKeepAliveManager {
func makeTCPKeepAliveManager(settings *cluster.Settings) tcpKeepAliveManager {
return tcpKeepAliveManager{
tcpKeepAlive: envutil.EnvOrDefaultDuration("COCKROACH_SQL_TCP_KEEP_ALIVE", time.Minute),
settings: settings,
}
}

// configure attempts to set TCP keep-alive on
// connection. Does not fail on errors.
func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) {
if k.tcpKeepAlive == 0 {
return
}

muxConn, ok := conn.(*cmux.MuxConn)
if !ok {
return
Expand All @@ -60,14 +73,33 @@ func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) {
return

}
if err := tcpConn.SetKeepAlivePeriod(k.tcpKeepAlive); err != nil {
// Based on the maximum connection life span and probe interval, pick a maximum
// probe count.
maximumConnectionTime := KeepAliveTimeout.Get(&k.settings.SV)
probeFrequency := KeepAliveProbeFrequency.Get(&k.settings.SV)
// We are going to try to hit our maximum connection time by
// allowing enough probes of our select frequency. Additionally,
// subtract one, since probing starts once the connection has been
// idle that long.
probeCount := (maximumConnectionTime / (probeFrequency)) - 1
if probeCount <= 0 {
probeCount = 1
}

if err := sysutil.SetKeepAliveCount(tcpConn, int(probeCount)); err != nil {
if doLog {
log.Ops.Warningf(ctx, "failed to set TCP keep-alive probe count for pgwire: %v", err)
}
}

if err := tcpConn.SetKeepAlivePeriod(probeFrequency); err != nil {
if doLog {
log.Ops.Warningf(ctx, "failed to set TCP keep-alive duration for pgwire: %v", err)
}
return
}

if doLog {
log.VEventf(ctx, 2, "setting TCP keep-alive to %s for pgwire", k.tcpKeepAlive)
log.VEventf(ctx, 2, "setting TCP keep-alive interval %d and probe count to %d for pgwire", probeFrequency, probeCount)
}
}
105 changes: 105 additions & 0 deletions pkg/server/tcp_keepalive_manager_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
// Copyright 2023 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.

//go:build linux || (arm64 && darwin)

package server

import (
"context"
"net"
"testing"
"time"

"github.com/cockroachdb/cmux"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"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/sysutil"
"github.com/stretchr/testify/require"
)

func TestKeepAliveManager(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

grp := ctxgroup.WithContext(ctx)
clusterSettings := cluster.MakeTestingClusterSettings()
KeepAliveProbeFrequency.Override(ctx, &clusterSettings.SV, time.Second*5)
KeepAliveTimeout.Override(ctx, &clusterSettings.SV, time.Minute*5)
keepAliveMgr := makeTCPKeepAliveManager(clusterSettings)

l, err := net.Listen("tcp", ":0")
require.NoError(t, err)
mux := cmux.New(l)
mux.HandleError(func(err error) bool {
return false
})

listener := mux.Match(cmux.Any())
grp.Go(func() error {
netutil.FatalIfUnexpected(mux.Serve())
return nil
})
connStr := listener.Addr()

grp.GoCtx(func(ctx context.Context) error {
conn, err := net.Dial(connStr.Network(), connStr.String())
if err != nil {
return err
}
reply := make([]byte, 1)
_, err = conn.Read(reply)
return err
})

conn, err := listener.Accept()
require.NoError(t, err)

// Configure this new connection with keep alive settings.
keepAliveMgr.configure(ctx, conn)
_, err = conn.Write([]byte("1"))
require.NoError(t, err)
// Confirm the settings are set on any TCP connection that we
// process.
muxConn, ok := conn.(*cmux.MuxConn)
if !ok {
return
}
tcpConn, ok := muxConn.Conn.(*net.TCPConn)
if !ok {
return
}
idleTime, probeInterval, probeCount, err := sysutil.GetKeepAliveSettings(tcpConn)
require.NoError(t, err)

require.Equal(t,
idleTime,
KeepAliveProbeFrequency.Get(&clusterSettings.SV),
"keep alive probe frequency not set")
require.Equal(t,
probeInterval,
KeepAliveProbeFrequency.Get(&clusterSettings.SV),
"keep alive probe frequency not set")

require.Equal(t,
KeepAliveTimeout.Get(&clusterSettings.SV),
time.Duration(probeCount+1)*probeInterval,
"Computed wait time doesn't match our target timeout")

// Validate we didn't hit any errors using the sockets.
require.NoError(t, err)
require.NoError(t, listener.Close())
require.NoError(t, grp.Wait())
require.NoError(t, conn.Close())
}
1 change: 1 addition & 0 deletions pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -966,6 +966,7 @@ func (s *SQLServerWrapper) AcceptClients(ctx context.Context) error {
s.pgPreServer,
s.serveConn,
s.pgL,
s.ClusterSettings(),
&s.sqlServer.cfg.SocketFile,
); err != nil {
return err
Expand Down

0 comments on commit c753ad9

Please sign in to comment.