Skip to content

Commit dbc05bb

Browse files
committed
util/sysutil: add functions for getting TCP round-trip time and variance
Previously, we relied on timing heartbeats in the RPC layer to estimate network latency (rpc.connection.avg_round_trip_latency). However, because these measurements are computed within cockroach, they can be confounded by CPU-heavy workloads. Luckily, Linux maintains the smoothed round-trip time (SRTT) and RTT variance for each TCP socket it opens. As kernel-computed metrics, these are less sensitive to CPU overload. This patch adds a sysutil function for obtaining SRTT and RTT variance given a a net.TCPConn pointer. Part of: #149959 Release note: None
1 parent b8c2405 commit dbc05bb

File tree

5 files changed

+197
-0
lines changed

5 files changed

+197
-0
lines changed

pkg/util/sysutil/BUILD.bazel

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,9 @@ go_library(
1616
"sysutil.go",
1717
"sysutil_unix.go",
1818
"sysutil_windows.go",
19+
"tcpinfo.go",
20+
"tcpinfo_linux.go",
21+
"tcpinfo_stub.go",
1922
],
2023
importpath = "github.com/cockroachdb/cockroach/pkg/util/sysutil",
2124
visibility = ["//visibility:public"],
@@ -94,6 +97,7 @@ go_test(
9497
"large_file_test.go",
9598
"sysutil_test.go",
9699
"sysutil_unix_test.go",
100+
"tcpinfo_linux_test.go",
97101
],
98102
embed = [":sysutil"],
99103
deps = [
@@ -105,6 +109,7 @@ go_test(
105109
"@org_golang_x_sys//unix",
106110
],
107111
"@io_bazel_rules_go//go/platform:android": [
112+
"//pkg/testutils/skip",
108113
"@com_github_stretchr_testify//assert",
109114
"@org_golang_x_sys//unix",
110115
],
@@ -133,6 +138,7 @@ go_test(
133138
"@org_golang_x_sys//unix",
134139
],
135140
"@io_bazel_rules_go//go/platform:linux": [
141+
"//pkg/testutils/skip",
136142
"@com_github_stretchr_testify//assert",
137143
"@org_golang_x_sys//unix",
138144
],

pkg/util/sysutil/tcpinfo.go

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,14 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package sysutil
7+
8+
import "time"
9+
10+
// RTTInfo holds the round-trip time information for a TCP connection.
11+
type RTTInfo struct {
12+
RTT time.Duration
13+
RTTVar time.Duration
14+
}

pkg/util/sysutil/tcpinfo_linux.go

Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
//go:build linux
7+
8+
package sysutil
9+
10+
import (
11+
"net"
12+
"syscall"
13+
"time"
14+
15+
"golang.org/x/sys/unix"
16+
)
17+
18+
func getTCPInfo(conn syscall.RawConn) (*unix.TCPInfo, error) {
19+
var (
20+
tcpInfo *unix.TCPInfo
21+
syscallErr error
22+
)
23+
err := conn.Control(func(fd uintptr) {
24+
tcpInfo, syscallErr = unix.GetsockoptTCPInfo(int(fd), unix.IPPROTO_TCP, unix.TCP_INFO)
25+
})
26+
if syscallErr != nil {
27+
return nil, syscallErr
28+
}
29+
if err != nil {
30+
return nil, err
31+
}
32+
return tcpInfo, nil
33+
}
34+
35+
// GetRTTInfo retrieves round-trip time information from a TCP connection.
36+
func GetRTTInfo(conn *net.TCPConn) (value *RTTInfo, ok bool) {
37+
if conn == nil {
38+
return nil, false
39+
}
40+
rawConn, err := conn.SyscallConn()
41+
if err != nil {
42+
return nil, false
43+
}
44+
tcpInfo, err := getTCPInfo(rawConn)
45+
if err != nil {
46+
return nil, false
47+
}
48+
return &RTTInfo{
49+
RTT: time.Duration(tcpInfo.Rtt) * time.Microsecond,
50+
RTTVar: time.Duration(tcpInfo.Rttvar) * time.Microsecond,
51+
}, true
52+
}
Lines changed: 110 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,110 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
//go:build linux
7+
8+
package sysutil
9+
10+
import (
11+
"context"
12+
"io"
13+
"net"
14+
"runtime"
15+
"testing"
16+
"time"
17+
18+
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
19+
"github.com/stretchr/testify/require"
20+
)
21+
22+
// GetRTTInfo doesn't panic when it's called with a nil connection.
23+
func TestGetTCPInfoNilConn(t *testing.T) {
24+
var conn *net.TCPConn
25+
info, ok := GetRTTInfo(conn)
26+
require.Nil(t, info)
27+
require.False(t, ok)
28+
}
29+
30+
func TestGetTCPInfoLinux(t *testing.T) {
31+
if runtime.GOOS != "linux" {
32+
skip.IgnoreLint(t, "skipping test; RTT inspection is only supported on Linux")
33+
}
34+
35+
ctx, cancel := context.WithCancel(context.Background())
36+
defer cancel()
37+
38+
// Start a TCP echo server.
39+
listener, _, err := createEchoServer(ctx, "127.0.0.1:0")
40+
require.NoError(t, err)
41+
defer func() {
42+
require.NoError(t, listener.Close())
43+
}()
44+
45+
// Connect a client.
46+
conn, err := net.Dial("tcp", listener.Addr().String())
47+
require.NoError(t, err)
48+
defer func() {
49+
require.NoError(t, conn.Close())
50+
}()
51+
52+
// Give the connection a moment to be established.
53+
time.Sleep(10 * time.Millisecond)
54+
55+
pingPayload := []byte("ping")
56+
pongBuffer := make([]byte, len(pingPayload))
57+
58+
// Write a "ping".
59+
_, err = conn.Write(pingPayload)
60+
require.NoError(t, err)
61+
62+
// Read the "pong" echo.
63+
_, err = io.ReadFull(conn, pongBuffer)
64+
require.NoError(t, err)
65+
66+
// RTT info is only available on TCP connections. GetRTTInfo takes a
67+
// *net.TCPConn, forcing the caller to perform a type assertion.
68+
tcpConn, ok := conn.(*net.TCPConn)
69+
require.True(t, ok)
70+
71+
info, ok := GetRTTInfo(tcpConn)
72+
73+
require.True(t, ok)
74+
require.NotNil(t, info)
75+
}
76+
77+
// createEchoServer creates a TCP echo server that echoes back all received data.
78+
// It returns the listener and a channel that will receive any server errors.
79+
func createEchoServer(ctx context.Context, addr string) (net.Listener, <-chan error, error) {
80+
listener, err := net.Listen("tcp", addr)
81+
if err != nil {
82+
return nil, nil, err
83+
}
84+
85+
serverErrChan := make(chan error, 1)
86+
go func() {
87+
conn, err := listener.Accept()
88+
if err != nil {
89+
// If the context is canceled, the listener may be closed,
90+
// resulting in an error. This is expected during cleanup.
91+
select {
92+
case <-ctx.Done():
93+
serverErrChan <- nil
94+
default:
95+
serverErrChan <- err
96+
}
97+
return
98+
}
99+
defer conn.Close()
100+
// Echo all data received back to the client. This will run until
101+
// the client closes the connection, resulting in an io.EOF.
102+
if _, err := io.Copy(conn, conn); err != nil && err != io.EOF {
103+
serverErrChan <- err
104+
} else {
105+
serverErrChan <- nil
106+
}
107+
}()
108+
109+
return listener, serverErrChan, nil
110+
}

pkg/util/sysutil/tcpinfo_stub.go

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,15 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
//go:build !linux
7+
8+
package sysutil
9+
10+
import "syscall"
11+
12+
// GetRTTInfo is a stub implementation for non-Linux platforms returning (nil, false).
13+
func GetRTTInfo(conn syscall.RawConn) (value *RTTInfo, ok bool) {
14+
return nil, false
15+
}

0 commit comments

Comments
 (0)