Skip to content

Commit d0cd2cf

Browse files
craig[bot]yuzefovichsouravcrlrharding6373sumeerbhola
committed
145482: util/mon: complete the revert of "nil as unlimited account" r=yuzefovich a=yuzefovich **util/mon: remove temporary nil checks** This commit finalizes the process of reverting of "nil as unlimited account" behavior that was mostly done in b8d259d. The main leftover piece was that we left the previous behavior in productions builds out of caution. It's been a year now without any panics in our tests, so it should be safe to remove this. **util/mon: assume non-nil ConcurrentBoundAccount** This commit completes the process of reverting of "nil as unlimited account" behavior. The only remaining case was with ConcurrentBoundAccount used only in a handful of places, so they were audited to ensure that non-nil account was used. Epic: None Release note: None 146177: security: update TestTLSCipherRestrict test setup r=souravcrl a=souravcrl Current test setup starts the test server only once and runs various tls cipher configurations but this seems to fail the test as final cipher configuration set by previous run of the test may not be overridden by current run, and we have stale ciphers configured for the subtest. Another issue is the timout being set is 2s which may fail and increasing it is not possible as the test may timeout before that leading to leaky goroutines. Hence, handling the http client response timeouts as a flake. Waitgroups are added for all client calls to ensure there are no open connections during test breakdown. fixes #145812 fixes #145527 fixes #145459 fixes #145313 Release Note: None 146248: changefeedccl: add timeout to nemeses test inserts r=aerfrei a=rharding6373 Some mutation queries take a long time to plan and execute even if they're safe. This PR adds a short timeout to each mutation, like tlp does for mutations, in order to avoid hanging. Fixes: #146159 Epic: none Release note: none 146288: admission: log the ingest model r=kvoli a=sumeerbhola We were previously only logging the L0 ingest model, which is useful for modeling the growth of L0. For disk bandwidth accounting, the ingest model for the whole LSM is used, and with this change it is also logged. Logging these models helps understand the behavior of the ioLoadListener. Epic: none Release note: None Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: souravcrl <[email protected]> Co-authored-by: rharding6373 <[email protected]> Co-authored-by: sumeerbhola <[email protected]>
5 parents 1c796d1 + 67511f4 + 6d56aaf + fddd1d1 + 950585a commit d0cd2cf

File tree

11 files changed

+187
-226
lines changed

11 files changed

+187
-226
lines changed

pkg/ccl/changefeedccl/cdctest/nemeses.go

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"fmt"
1414
"math/rand"
1515
"strings"
16+
"time"
1617

1718
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
1819
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
@@ -418,11 +419,16 @@ func RunNemesis(
418419

419420
defer queryGen.Close()
420421
const numInserts = 100
422+
const insertTimeout = 5 * time.Second
421423
time := timeutil.Now()
422424
for i := 0; i < numInserts; i++ {
423425
query := queryGen.Generate()
424426
log.Infof(ctx, "Executing query: %s", query)
425-
_, err := db.Exec(query)
427+
err := timeutil.RunWithTimeout(ctx, "nemeses populate table",
428+
insertTimeout, func(ctx context.Context) error {
429+
_, err := db.ExecContext(ctx, query)
430+
return err
431+
})
426432
log.Infof(ctx, "Time taken to execute last query: %s", timeutil.Since(time))
427433
time = timeutil.Now()
428434
if err != nil {

pkg/kv/kvclient/rangefeed/db_adapter.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -118,6 +118,8 @@ func (dbc *dbAdapter) Scan(
118118
if cfg.mon != nil {
119119
acc = cfg.mon.MakeConcurrentBoundAccount()
120120
defer acc.Close(ctx)
121+
} else {
122+
acc = mon.NewStandaloneUnlimitedConcurrentAccount()
121123
}
122124

123125
// If we don't have parallelism configured, just scan each span in turn.

pkg/security/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,6 @@ go_test(
9393
"//pkg/server",
9494
"//pkg/testutils",
9595
"//pkg/testutils/serverutils",
96-
"//pkg/testutils/skip",
9796
"//pkg/util/envutil",
9897
"//pkg/util/leaktest",
9998
"//pkg/util/log",

pkg/security/tls_test.go

Lines changed: 70 additions & 71 deletions
Original file line numberDiff line numberDiff line change
@@ -13,16 +13,15 @@ import (
1313
"net"
1414
"net/http"
1515
"strings"
16+
"sync"
1617
"testing"
17-
"time"
1818

1919
"github.com/cockroachdb/cockroach/pkg/base"
2020
"github.com/cockroachdb/cockroach/pkg/security"
2121
"github.com/cockroachdb/cockroach/pkg/security/certnames"
2222
"github.com/cockroachdb/cockroach/pkg/security/username"
2323
"github.com/cockroachdb/cockroach/pkg/testutils"
2424
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
25-
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
2625
"github.com/cockroachdb/cockroach/pkg/util/envutil"
2726
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2827
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -124,50 +123,6 @@ func verifyX509Cert(cert *x509.Certificate, dnsName string, roots *x509.CertPool
124123
func TestTLSCipherRestrict(t *testing.T) {
125124
defer leaktest.AfterTest(t)()
126125
defer log.Scope(t).Close(t)
127-
skip.UnderStress(t, "http server accessing previous test's restriction fn")
128-
skip.UnderRace(t)
129-
130-
// since the listener does not return rpc/sql/http connection errors, we
131-
// need to have a separate hook to obtain and validate it.
132-
type cipherErrContainer struct {
133-
syncutil.Mutex
134-
err net.Error
135-
}
136-
137-
cipherErrC := &cipherErrContainer{}
138-
cipherRestrictFn := security.TLSCipherRestrict
139-
defer testutils.TestingHook(&security.TLSCipherRestrict, func(conn net.Conn) (err net.Error) {
140-
err = cipherRestrictFn(conn)
141-
cipherErrC.Lock()
142-
cipherErrC.err = err
143-
cipherErrC.Unlock()
144-
return err
145-
})()
146-
147-
ctx := context.Background()
148-
149-
// Start with a clean cipher configuration state
150-
err := security.SetTLSCipherSuitesConfigured([]string{})
151-
require.NoError(t, err)
152-
153-
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
154-
defer s.Stopper().Stop(ctx)
155-
defer require.NoError(t, security.SetTLSCipherSuitesConfigured([]string{}))
156-
157-
// setup for db console tests
158-
httpClient, err := s.GetUnauthenticatedHTTPClient()
159-
require.NoError(t, err)
160-
httpClient.Timeout = 2 * time.Second
161-
defer httpClient.CloseIdleConnections()
162-
163-
secureClient, err := s.GetAuthenticatedHTTPClient(false, serverutils.SingleTenantSession)
164-
require.NoError(t, err)
165-
secureClient.Timeout = 2 * time.Second
166-
defer secureClient.CloseIdleConnections()
167-
168-
urlsToTest := []string{"/_status/vars", "/index.html", "/"}
169-
adminURLHTTPS := s.AdminURL().String()
170-
adminURLHTTP := strings.Replace(adminURLHTTPS, "https", "http", 1)
171126

172127
tests := []struct {
173128
name string
@@ -179,63 +134,106 @@ func TestTLSCipherRestrict(t *testing.T) {
179134
cipherErr string
180135
}{
181136
{name: "no cipher set", ciphers: []string{}, wantErr: false},
182-
{name: "valid ciphers", ciphers: []string{"TLS_AES_256_GCM_SHA384", "TLS_AES_128_GCM_SHA256"},
137+
{name: "valid ciphers", ciphers: []string{"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", "TLS_AES_256_GCM_SHA384", "TLS_AES_128_GCM_SHA256"},
183138
wantErr: false},
184-
{name: "invalid ciphers", ciphers: []string{"TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256"}, wantErr: true,
139+
{name: "invalid ciphers", ciphers: []string{"TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256"}, wantErr: true,
185140
httpsErr: []string{"\": EOF", "connect: connection refused", "read: connection reset by peer", "http: server closed idle connection"},
186141
sqlErr: "failed to connect to `host=127.0.0.1 user=root database=`: failed to receive message (unexpected EOF)",
187142
rpcErr: "initial connection heartbeat failed: grpc:",
188143
cipherErr: "^presented cipher [^ ]+ not in allowed cipher suite list$"},
189144
}
190145
for _, tt := range tests {
191146
t.Run(tt.name, func(t *testing.T) {
192-
// First ensure we're starting with no restrictions
193-
err := security.SetTLSCipherSuitesConfigured([]string{})
147+
// since the listener does not return rpc/sql/http connection errors, we
148+
// need to have a separate hook to obtain and validate it.
149+
type cipherErrContainer struct {
150+
syncutil.Mutex
151+
err net.Error
152+
}
153+
154+
cipherErrC := &cipherErrContainer{}
155+
cipherRestrictFn := security.TLSCipherRestrict
156+
defer testutils.TestingHook(&security.TLSCipherRestrict, func(conn net.Conn) (err net.Error) {
157+
err = cipherRestrictFn(conn)
158+
cipherErrC.Lock()
159+
cipherErrC.err = err
160+
cipherErrC.Unlock()
161+
return err
162+
})()
163+
ctx := context.Background()
164+
165+
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
166+
defer s.Stopper().Stop(ctx)
167+
168+
// set the custom test ciphers
169+
err := security.SetTLSCipherSuitesConfigured(tt.ciphers)
194170
require.NoError(t, err)
195171

172+
// setup for db console tests
173+
httpClient, transport, err := s.ApplicationLayer().GetUnauthenticatedHTTPClientWithTransport()
174+
require.NoError(t, err)
175+
const httpTimeout = "Client.Timeout exceeded while awaiting headers"
176+
transport.TLSClientConfig.MinVersion = tls.VersionTLS13
177+
defer httpClient.CloseIdleConnections()
178+
179+
urlsToTest := []string{"/_status/vars", "/index.html", "/"}
180+
adminURLHTTPS := s.AdminURL().String()
181+
adminURLHTTP := strings.Replace(adminURLHTTPS, "https", "http", 1)
182+
196183
// Reset the error container before each test
197184
cipherErrC.Lock()
198185
cipherErrC.err = nil
199186
cipherErrC.Unlock()
200187

201-
// Now set the custom test ciphers
202-
err = security.SetTLSCipherSuitesConfigured(tt.ciphers)
203-
require.NoError(t, err)
204-
// unset the ciphers after test
205-
defer func() { _ = security.SetTLSCipherSuitesConfigured([]string{}) }()
206-
207188
// test db console tls access for cipher restriction.
208189
for _, u := range urlsToTest {
209-
for _, client := range []http.Client{httpClient, secureClient} {
210-
resp, err := client.Get(adminURLHTTP + u)
211-
if (err == nil) == tt.wantErr {
212-
var body []byte
190+
for _, client := range []http.Client{httpClient} {
191+
var wg sync.WaitGroup
192+
wg.Add(1)
193+
var body []byte
194+
go func() {
195+
defer wg.Done()
196+
var resp *http.Response
197+
resp, err = client.Get(adminURLHTTP + u)
213198
if resp != nil && resp.Body != nil {
214199
defer resp.Body.Close()
215200
body, err = io.ReadAll(resp.Body)
216201
}
217-
t.Fatalf("expected wantError=%t, got err=%v, resp=%v", tt.wantErr, err, string(body))
202+
}()
203+
wg.Wait()
204+
if (err == nil) == tt.wantErr {
205+
if !(err != nil && strings.Contains(err.Error(), httpTimeout)) {
206+
t.Fatalf("expected wantError=%t, got err=%v, resp=%v", tt.wantErr, err, string(body))
207+
}
218208
}
219209
if tt.wantErr {
220-
cipherErrC.Lock()
221-
errVal := cipherErrC.err
222-
cipherErrC.Unlock()
223-
require.Regexp(t, tt.cipherErr, errVal.Error())
224210
var errMatch bool
225211
for idx := range tt.httpsErr {
226212
errMatch = errMatch || strings.Contains(err.Error(), tt.httpsErr[idx])
227213
}
228214
if !errMatch {
229215
t.Fatalf("the provided error %s does not match any of the expected errors: %v", err.Error(), strings.Join(tt.httpsErr, ", "))
230216
}
217+
cipherErrC.Lock()
218+
errVal := cipherErrC.err
219+
cipherErrC.Unlock()
220+
require.NotNil(t, errVal)
221+
require.Regexp(t, tt.cipherErr, errVal.Error())
231222
}
232223
}
233224
}
234225

235226
// test pgx connection for root user with cert auth
236227
pgURL, cleanup := s.PGUrl(t, serverutils.User(username.RootUser), serverutils.ClientCerts(true))
237228
defer cleanup()
238-
rootConn, err := pgx.Connect(ctx, pgURL.String())
229+
var wg sync.WaitGroup
230+
wg.Add(1)
231+
go func() {
232+
defer wg.Done()
233+
_, err = pgx.Connect(ctx, pgURL.String())
234+
}()
235+
wg.Wait()
236+
239237
if (err == nil) == tt.wantErr {
240238
t.Fatalf("expected wantError=%t, got err=%v", tt.wantErr, err)
241239
}
@@ -245,12 +243,15 @@ func TestTLSCipherRestrict(t *testing.T) {
245243
cipherErrC.Unlock()
246244
require.Regexp(t, tt.cipherErr, errVal.Error())
247245
require.Equal(t, tt.sqlErr, err.Error())
248-
} else {
249-
require.NoError(t, rootConn.Close(ctx))
250246
}
251247

252248
// test rpc connection for root user.
253-
conn, err := s.RPCClientConnE(username.RootUserName())
249+
wg.Add(1)
250+
go func() {
251+
defer wg.Done()
252+
_, err = s.RPCClientConnE(username.RootUserName())
253+
}()
254+
wg.Wait()
254255
if (err == nil) == tt.wantErr {
255256
t.Fatalf("expected wantError=%t, got err=%v", tt.wantErr, err)
256257
}
@@ -260,8 +261,6 @@ func TestTLSCipherRestrict(t *testing.T) {
260261
cipherErrC.Unlock()
261262
require.Regexp(t, tt.cipherErr, errVal.Error())
262263
require.Contains(t, err.Error(), tt.rpcErr)
263-
} else {
264-
require.NoError(t, conn.Close()) // nolint:grpcconnclose
265264
}
266265
})
267266
}

pkg/server/testserver_http.go

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,28 @@ func (ts *httpTestServer) GetUnauthenticatedHTTPClient() (http.Client, error) {
9191
return client, nil
9292
}
9393

94+
// GetUnauthenticatedHTTPClientWithTransport implements TestServerInterface.
95+
func (ts *httpTestServer) GetUnauthenticatedHTTPClientWithTransport() (
96+
http.Client,
97+
*http.Transport,
98+
error,
99+
) {
100+
client, err := ts.t.sqlServer.execCfg.RPCContext.GetHTTPClient()
101+
if err != nil {
102+
return client, nil, err
103+
}
104+
transport := client.Transport.(*http.Transport)
105+
client.Transport = &tenantHeaderDecorator{
106+
RoundTripper: client.Transport,
107+
tenantName: ts.t.tenantName,
108+
}
109+
client.Timeout = 2 * time.Second
110+
if util.RaceEnabled {
111+
client.Timeout = 30 * time.Second
112+
}
113+
return client, transport, nil
114+
}
115+
94116
// GetAdminHTTPClient implements the TestServerInterface.
95117
func (ts *httpTestServer) GetAdminHTTPClient() (http.Client, error) {
96118
httpClient, _, err := ts.GetAuthenticatedHTTPClientAndCookie(

pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -92,20 +92,22 @@ type Container struct {
9292
func New(
9393
st *cluster.Settings,
9494
uniqueServerCount *SQLStatsAtomicCounters,
95-
mon *mon.BytesMonitor,
95+
monitor *mon.BytesMonitor,
9696
appName string,
9797
knobs *sqlstats.TestingKnobs,
9898
) *Container {
9999
s := &Container{
100100
st: st,
101101
appName: appName,
102-
mon: mon,
102+
mon: monitor,
103103
knobs: knobs,
104104
uniqueServerCount: uniqueServerCount,
105105
}
106106

107-
if mon != nil {
108-
s.acc = mon.MakeConcurrentBoundAccount()
107+
if monitor != nil {
108+
s.acc = monitor.MakeConcurrentBoundAccount()
109+
} else {
110+
s.acc = mon.NewStandaloneUnlimitedConcurrentAccount()
109111
}
110112

111113
s.mu.stmts = make(map[stmtKey]*stmtStats)

pkg/testutils/serverutils/api.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -359,6 +359,12 @@ type ApplicationLayerInterface interface {
359359
// with verbose method name.
360360
GetUnauthenticatedHTTPClient() (http.Client, error)
361361

362+
// GetUnauthenticatedHTTPClientWithTransport returns an http client and its
363+
// corresponding transport configured with the client TLS config required by
364+
// the TestServer's configuration. Discourages implementer from using
365+
// unauthenticated http connections with verbose method name.
366+
GetUnauthenticatedHTTPClientWithTransport() (http.Client, *http.Transport, error)
367+
362368
// GetAdminHTTPClient returns an http client which has been
363369
// authenticated to access Admin API methods (via a cookie).
364370
// The user has admin privileges.

pkg/util/admission/io_load_listener.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1341,10 +1341,14 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) {
13411341
res.aux.perWorkTokensAux.intL0WriteLinearModel.multiplier,
13421342
ib(res.aux.perWorkTokensAux.intL0WriteLinearModel.constant),
13431343
res.l0WriteLM.multiplier, ib(res.l0WriteLM.constant))
1344-
p.Printf("ingested-model %.2fx+%s (smoothed %.2fx+%s) + ",
1344+
p.Printf("l0-ingest-model %.2fx+%s (smoothed %.2fx+%s) + ",
13451345
res.aux.perWorkTokensAux.intL0IngestedLinearModel.multiplier,
13461346
ib(res.aux.perWorkTokensAux.intL0IngestedLinearModel.constant),
13471347
res.l0IngestLM.multiplier, ib(res.l0IngestLM.constant))
1348+
p.Printf("ingest-model %.2fx+%s (smoothed %.2fx+%s) + ",
1349+
res.aux.perWorkTokensAux.intIngestedLinearModel.multiplier,
1350+
ib(res.aux.perWorkTokensAux.intIngestedLinearModel.constant),
1351+
res.ingestLM.multiplier, ib(res.ingestLM.constant))
13481352
p.Printf("write-amp-model %.2fx+%s (smoothed %.2fx+%s) + ",
13491353
res.aux.perWorkTokensAux.intWriteAmpLinearModel.multiplier,
13501354
ib(res.aux.perWorkTokensAux.intWriteAmpLinearModel.constant),
Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
echo
22
----
33
zero:
4-
compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.00); admitting all (used total: 0 B elastic 0 B); write stalls 12
4+
compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + l0-ingest-model 0.00x+0 B (smoothed 0.00x+0 B) + ingest-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.00); admitting all (used total: 0 B elastic 0 B); write stalls 12
55
real-numbers:
6-
compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B] (mult 1.00); admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 2
6+
compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 0.00x+0 B) + l0-ingest-model 0.00x+0 B (smoothed 0.00x+0 B) + ingest-model 0.00x+0 B (smoothed 0.00x+0 B) + write-amp-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B] (mult 1.00); admitting 116 MiB (rate 7.7 MiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 2

0 commit comments

Comments
 (0)