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.count and
server.sql_tcp_keep_alive.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 probes (server.sql_tcp_keep_alive.count) and TCP
probe intervals (server.sql_tcp_keep_alive.interval) for SQL
connections. Removes the COCKROACH_SQL_TCP_KEEP_ALIVE environment
variable subsuming it.
  • Loading branch information
fqazi committed Jan 10, 2024
1 parent b2ef8cb commit e233d7f
Show file tree
Hide file tree
Showing 9 changed files with 352 additions and 14 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.count integer 4 maximum number of probes that will be sent out before a connection dropped because its unresponsive (Linux and Darwin only) application
server.sql_tcp_keep_alive.interval duration 15s time between keep alive probes and idle time before probes are sent out 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-count" class="anchored"><code>server.sql_tcp_keep_alive.count</code></div></td><td>integer</td><td><code>4</code></td><td>maximum number of probes that will be sent out before a connection dropped because its unresponsive (Linux and Darwin only)</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-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
110 changes: 109 additions & 1 deletion pkg/cmd/roachtest/tests/network.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package tests

import (
"context"
"errors"
"fmt"
"os"
"path/filepath"
Expand All @@ -25,7 +24,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
errors "github.com/cockroachdb/errors"
_ "github.com/lib/pq" // register postgres driver
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -293,6 +294,103 @@ sudo iptables-save
m.Wait()
}

// runClientNetworkConnectionTimeout simulates a scenario where the client and
// server loose connectivity with a connection that is idle. The purpose of this
// test is to confirm that the keep alive settings are enforced.
func runClientNetworkConnectionTimeout(ctx context.Context, t test.Test, c cluster.Cluster) {
n := c.Spec().NodeCount
serverNodes, clientNode := c.Range(1, n-1), c.Nodes(n)
settings := install.MakeClusterSettings(install.SecureOption(true))
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, serverNodes)
certsDir := "/home/ubuntu/certs"
t.L().Printf("connecting to cluster from roachtest...")
db, err := c.ConnE(ctx, t.L(), 1)
require.NoError(t, err)
defer db.Close()

grp := ctxgroup.WithContext(ctx)
// Startup a connection on the client server, which will be running a
// long transaction (i.e. just the sleep builtin).
var runOutput install.RunResultDetails
grp.GoCtx(func(ctx context.Context) error {
ips, err := c.ExternalIP(ctx, t.L(), c.Node(1))
if err != nil {
return err
}
commandThatWillDisconnect := fmt.Sprintf(`./cockroach sql --certs-dir %s --url "postgres://root@%s:26257" -e "SELECT pg_sleep(600)"`, certsDir, ips[0])
t.L().Printf("Executing long running query: %s", commandThatWillDisconnect)
output, err := c.RunWithDetails(ctx, t.L(), clientNode, commandThatWillDisconnect)
runOutput = output[0]
return err
})
// Confirm that the connection was started.
testutils.SucceedsSoon(t, func() error {
row := db.QueryRow("SELECT count(*) FROM [SHOW CLUSTER SESSIONS] WHERE active_queries='SELECT pg_sleep(600)'")
var count int
if err := row.Scan(&count); err != nil {
return err
}
// We expect 2 since the query will itself
if count != 1 {
return errors.AssertionFailedf("unexepcted count :%v", count)
}
return nil
})

const netConfigCmd = `
# ensure any failure fails the entire script.
set -e;
# Setting default filter policy
sudo iptables -P INPUT ACCEPT;
sudo iptables -P OUTPUT ACCEPT;
# Drop any client traffic to CRDB.
sudo iptables -A INPUT -p tcp --dport 26257 -j DROP;
sudo iptables -A OUTPUT -p tcp --dport 26257 -j DROP;
sudo iptables-save
`
t.L().Printf("blocking networking on client; config cmd:\n%s", netConfigCmd)
blockStartTime := timeutil.Now()
require.NoError(t, c.RunE(ctx, clientNode, netConfigCmd))

// (attempt to) restore iptables when test end, so that the client
// can be investigated afterward.
defer func() {
const restoreNet = `
set -e;
sudo iptables -D INPUT -p tcp --dport 26257 -j DROP;
sudo iptables -D OUTPUT -p tcp --dport 26257 -j DROP;
sudo iptables-save
`
t.L().Printf("restoring iptables; config cmd:\n%s", restoreNet)
require.NoError(t, c.RunE(ctx, clientNode, restoreNet))
}()

// We expect the connection to timeout within 1 minute based on
// the default settings. We will wait for up to 2 minutes for the
// connection to drop.
testutils.SucceedsWithin(t, func() error {
row := db.QueryRow("SELECT count(*) FROM [SHOW CLUSTER SESSIONS] WHERE active_queries='SELECT pg_sleep(600)'")
var count int
if err := row.Scan(&count); err != nil {
return err
}
if count != 0 {
return errors.AssertionFailedf("unexepcted count :%d", count)
}
return nil
},
time.Minute*2)
// Confirm it took at least a minute for the connection to clear out.
require.Greaterf(t, timeutil.Since(blockStartTime), time.Minute, "connection dropped earlier than expected")
// We expect the connection to be dropped with the lower keep alive settings.
require.NoError(t, grp.Wait())
require.Contains(t, runOutput.Stderr, "If the server is running, check --host client-side and --advertise server-side",
"Did not detect connection failure %s %d", runOutput.Stderr, runOutput.RemoteExitStatus)
}

func registerNetwork(r registry.Registry) {
const numNodes = 4
r.Add(registry.TestSpec{
Expand All @@ -306,4 +404,14 @@ func registerNetwork(r registry.Registry) {
runNetworkAuthentication(ctx, t, c)
},
})

r.Add(registry.TestSpec{
Name: "network/client-connection-timeout",
Owner: registry.OwnerSQLFoundations,
Cluster: r.MakeClusterSpec(2), // One server and client
CompatibleClouds: registry.AllExceptAWS,
Suites: registry.Suites(registry.Nightly),
Leases: registry.MetamorphicLeases,
Run: runClientNetworkConnectionTimeout,
})
}
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
47 changes: 36 additions & 11 deletions pkg/server/tcp_keepalive_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,31 +17,45 @@ 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 KeepAliveProbeCount = settings.RegisterIntSetting(
settings.ApplicationLevel,
"server.sql_tcp_keep_alive.count",
"maximum number of probes that will be sent out before a connection dropped because "+
"its unresponsive (Linux and Darwin only)",
4,
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 +74,25 @@ 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.
probeCount := KeepAliveProbeCount.Get(&k.settings.SV)
probeFrequency := KeepAliveProbeFrequency.Get(&k.settings.SV)

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)
}
}

0 comments on commit e233d7f

Please sign in to comment.