Skip to content

Commit a9fe917

Browse files
craig[bot]DarrylWongdavidwdingspilchenmsbutler
committed
154164: roachtest: add github issue posting dry run mode r=williamchoe3,herkolategan a=DarrylWong We already have the functionality in datadriven tests to render github issue post requests as clickable links. This change extends that functionality to roachtests in the form of a --dry-run-issue-posting flag. This allows for viewing what the output of a failed roachtest would look like even if github posting is disabled. Fixes: none Release note: none Epic: none 154266: sqlproxyccl: pass in ctx when transferring connections r=jeffswenson a=davidwding Previously, the span used when transferring connections was reused from the forwarder. However, transferring connections has async components, so it's possible that the forwarder calls `Finish` on the span before the connection transfer is done using the span. This leads to a panic: `panic: use of Span after Finish`. To address this, this patch passes a context in when transferring a connection, instead of using the forwarder's context. Fixes: #153569 Release note: None 154363: sql/types: reintroduce downgradeType for NAME compatibility in mixed clusters r=spilchen a=spilchen This patch restores the `downgradeType` hook to maintain serialization compatibility for the NAME type in mixed-version clusters (v25.3 and 25.4). In previous releases, a NAME type could be misinterpreted by older nodes. Specifically, a v25.3 node deserializing a NAME encoded by a v25.4 node would mistakenly treat it as text (`oid.T_text`). This was due to the old `upgradeType` logic overwriting the OID when no `VisibleType` was set, causing `T_name` to degrade to `T_text`. PR #148694 addressed this issue for v25.4 by changing `upgradeType` (to avoid clobbering a provided OID) and removing the `downgradeType` function. However, the follow-up fix for mixed-version serialization (PR #152633) relied on the `InternalType.VisibleType` field to tag string types for older nodes. Because the NAME type doesn’t use a `VisibleType` (it’s represented as a STRING family with a special OID), that change didn’t cover NAME. As a result, NAME remained broken in a 25.3/25.4 mixed cluster. This commit explicitly handles NAME during serialization. When sending a NAME type to the wire, we downgrade it to the legacy encoding: we set its type family to `Family=name` (legacy enum value 11) so that v25.3 nodes recognize it as a NAME. Additionally, if the locale is an empty string, we set it to `nil` to ensure older versions don’t misclassify the NAME as a collated string. Together, these steps preserve the correct type interpretation on older nodes. Once all nodes are on v25.4, this compatibility shim can be removed. At that point, the improved upgrade logic in v25.4 (which no longer overrides a valid OID) means we won’t need to downgrade types for backward compatibility. We may also perform a one time descriptor migration to update any NAME descriptors to the modern format after the cluster upgrade. In short, `downgradeType` for NAME is a temporary measure to bridge the v25.3 gap. Fixes #153322 Release note: None 154369: roachest: ensure reader tenant accepts connections before restart r=jeffswenson a=msbutler Previously, the roachtest would restart the tenant before the tenant could accept connections, which means the tenant could still be bootstrapping. Restarting the tenant service during bootstrap could lead to sadness, as outlined in #154356. Informs #154356 Informs #154311 Informs #153131 Release note: none Co-authored-by: DarrylWong <[email protected]> Co-authored-by: David Ding <[email protected]> Co-authored-by: Matt Spilchen <[email protected]> Co-authored-by: Michael Butler <[email protected]>
5 parents 8b550fa + 90f80fa + 65d0b22 + 036b796 + 20ea8e1 commit a9fe917

File tree

17 files changed

+260
-77
lines changed

17 files changed

+260
-77
lines changed

pkg/ccl/sqlproxyccl/balancer/balancer.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -371,7 +371,7 @@ func (b *Balancer) processQueue(ctx context.Context) {
371371

372372
// Each request is retried up to maxTransferAttempts.
373373
for i := 0; i < maxTransferAttempts && ctx.Err() == nil; i++ {
374-
err := req.conn.TransferConnection()
374+
err := req.conn.TransferConnection(ctx)
375375
if err == nil || errors.Is(err, context.Canceled) {
376376
break
377377
}

pkg/ccl/sqlproxyccl/balancer/conn.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ type ConnectionHandle interface {
1919
// TransferConnection performs a connection migration on the connection
2020
// handle. Invoking this blocks until the connection migration process has
2121
// been completed.
22-
TransferConnection() error
22+
TransferConnection(context.Context) error
2323

2424
// IsIdle returns true if the connection is idle, and false otherwise.
2525
IsIdle() bool

pkg/ccl/sqlproxyccl/balancer/conn_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ func (h *testConnHandle) Close() {
4444
}
4545

4646
// TransferConnection implements the ConnectionHandle interface.
47-
func (h *testConnHandle) TransferConnection() error {
47+
func (h *testConnHandle) TransferConnection(ctx context.Context) error {
4848
h.mu.Lock()
4949
defer h.mu.Unlock()
5050
h.mu.onTransferConnectionCount++

pkg/ccl/sqlproxyccl/conn_migration.go

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -127,7 +127,7 @@ var errTransferCannotStart = errors.New("transfer cannot be started")
127127
// error).
128128
//
129129
// TransferConnection implements the balancer.ConnectionHandle interface.
130-
func (f *forwarder) TransferConnection() (retErr error) {
130+
func (f *forwarder) TransferConnection(ctx context.Context) (retErr error) {
131131
// A previous non-recoverable transfer would have closed the forwarder, so
132132
// return right away.
133133
if f.ctx.Err() != nil {
@@ -146,18 +146,18 @@ func (f *forwarder) TransferConnection() (retErr error) {
146146
// whenever the context expires. We have to close the forwarder because
147147
// the transfer may be blocked on I/O, and the only way for now is to close
148148
// the connections. This then allow TransferConnection to return and cleanup.
149-
ctx, cancel := newTransferContext(f.ctx)
149+
transferCtx, cancel := newTransferContext(ctx)
150150
defer cancel()
151151

152152
// Use a separate handler for timeouts. This is the only way to handle
153153
// blocked I/Os as described above.
154154
go func() {
155-
<-ctx.Done()
155+
<-transferCtx.Done()
156156
// This Close call here in addition to the one in the defer callback
157157
// below is on purpose. This would help unblock situations where we're
158158
// blocked on sending/reading messages from connections that couldn't
159159
// be handled with context.Context.
160-
if !ctx.isRecoverable() {
160+
if !transferCtx.isRecoverable() {
161161
f.Close()
162162
}
163163
}()
@@ -179,7 +179,7 @@ func (f *forwarder) TransferConnection() (retErr error) {
179179

180180
// When TransferConnection returns, it's either the forwarder has been
181181
// closed, or the procesors have been resumed.
182-
if !ctx.isRecoverable() {
182+
if !transferCtx.isRecoverable() {
183183
log.Dev.Infof(logCtx, "transfer failed: connection closed, latency=%v, err=%v", latencyDur, retErr)
184184
f.metrics.ConnMigrationErrorFatalCount.Inc(1)
185185
f.Close()
@@ -201,16 +201,16 @@ func (f *forwarder) TransferConnection() (retErr error) {
201201

202202
// Suspend both processors before starting the transfer.
203203
request, response := f.getProcessors()
204-
if err := request.suspend(ctx); err != nil {
204+
if err := request.suspend(transferCtx); err != nil {
205205
return errors.Wrap(err, "suspending request processor")
206206
}
207-
if err := response.suspend(ctx); err != nil {
207+
if err := response.suspend(transferCtx); err != nil {
208208
return errors.Wrap(err, "suspending response processor")
209209
}
210210

211211
// Transfer the connection.
212212
clientConn, serverConn := f.getConns()
213-
newServerConn, err := transferConnection(ctx, f, f.connector, f.metrics, clientConn, serverConn)
213+
newServerConn, err := transferConnection(transferCtx, f, f.connector, f.metrics, clientConn, serverConn)
214214
if err != nil {
215215
return errors.Wrap(err, "transferring connection")
216216
}

pkg/ccl/sqlproxyccl/proxy_handler_test.go

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -2018,12 +2018,12 @@ func TestConnectionMigration(t *testing.T) {
20182018
require.Equal(t, totalAttempts, count)
20192019
}
20202020

2021-
transferConnWithRetries := func(t *testing.T, f *forwarder) error {
2021+
transferConnWithRetries := func(t *testing.T, ctx context.Context, f *forwarder) error {
20222022
t.Helper()
20232023

20242024
var nonRetriableErrSeen bool
20252025
err := testutils.SucceedsSoonError(func() error {
2026-
err := f.TransferConnection()
2026+
err := f.TransferConnection(ctx)
20272027
if err == nil {
20282028
return nil
20292029
}
@@ -2091,7 +2091,7 @@ func TestConnectionMigration(t *testing.T) {
20912091
require.NoError(t, err)
20922092

20932093
// Show that we get alternating SQL pods when we transfer.
2094-
require.NoError(t, transferConnWithRetries(t, f))
2094+
require.NoError(t, transferConnWithRetries(t, tCtx, f))
20952095
require.Equal(t, int64(1), f.metrics.ConnMigrationSuccessCount.Count())
20962096
require.Equal(t, tenant2.SQLAddr(), queryAddr(tCtx, t, db))
20972097

@@ -2102,7 +2102,7 @@ func TestConnectionMigration(t *testing.T) {
21022102
_, err = db.Exec("SET application_name = 'bar'")
21032103
require.NoError(t, err)
21042104

2105-
require.NoError(t, transferConnWithRetries(t, f))
2105+
require.NoError(t, transferConnWithRetries(t, tCtx, f))
21062106
require.Equal(t, int64(2), f.metrics.ConnMigrationSuccessCount.Count())
21072107
require.Equal(t, tenant1.SQLAddr(), queryAddr(tCtx, t, db))
21082108

@@ -2120,14 +2120,14 @@ func TestConnectionMigration(t *testing.T) {
21202120
go func() {
21212121
defer wg.Done()
21222122
for subCtx.Err() == nil {
2123-
_ = f.TransferConnection()
2123+
_ = f.TransferConnection(tCtx)
21242124
time.Sleep(100 * time.Millisecond)
21252125
}
21262126
}()
21272127

21282128
// This loop will run approximately 5 seconds.
21292129
var tenant1Addr, tenant2Addr int
2130-
for i := 0; i < 100; i++ {
2130+
for range 100 {
21312131
addr := queryAddr(tCtx, t, db)
21322132
if addr == tenant1.SQLAddr() {
21332133
tenant1Addr++
@@ -2167,7 +2167,7 @@ func TestConnectionMigration(t *testing.T) {
21672167
err = crdb.ExecuteTx(tCtx, db, nil /* txopts */, func(tx *gosql.Tx) error {
21682168
// Run multiple times to ensure that connection isn't closed.
21692169
for i := 0; i < 5; {
2170-
err := f.TransferConnection()
2170+
err := f.TransferConnection(tCtx)
21712171
if err == nil {
21722172
return errors.New("no error")
21732173
}
@@ -2201,7 +2201,7 @@ func TestConnectionMigration(t *testing.T) {
22012201
require.Equal(t, int64(0), f.metrics.ConnMigrationErrorFatalCount.Count())
22022202

22032203
// Once the transaction is closed, transfers should work.
2204-
require.NoError(t, transferConnWithRetries(t, f))
2204+
require.NoError(t, transferConnWithRetries(t, tCtx, f))
22052205
require.NotEqual(t, initAddr, queryAddr(tCtx, t, db))
22062206
require.Nil(t, f.ctx.Err())
22072207
require.Equal(t, initSuccessCount+1, f.metrics.ConnMigrationSuccessCount.Count())
@@ -2223,7 +2223,7 @@ func TestConnectionMigration(t *testing.T) {
22232223
lookupAddrDelayDuration = 10 * time.Second
22242224
defer testutils.TestingHook(&defaultTransferTimeout, 3*time.Second)()
22252225

2226-
err := f.TransferConnection()
2226+
err := f.TransferConnection(tCtx)
22272227
require.Error(t, err)
22282228
require.Regexp(t, "injected delays", err.Error())
22292229
require.Equal(t, initAddr, queryAddr(tCtx, t, db))
@@ -2319,7 +2319,7 @@ func TestConnectionMigration(t *testing.T) {
23192319
time.Sleep(2 * time.Second)
23202320
// This should be an error because the transfer timed out. Connection
23212321
// should automatically be closed.
2322-
require.Error(t, f.TransferConnection())
2322+
require.Error(t, f.TransferConnection(tCtx))
23232323

23242324
select {
23252325
case <-time.After(10 * time.Second):

pkg/cmd/roachtest/github.go

Lines changed: 79 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,9 @@ package main
88
import (
99
"context"
1010
"fmt"
11+
"net/url"
1112
"os"
13+
"strings"
1214
"time"
1315

1416
"github.com/cockroachdb/cockroach/pkg/cmd/bazci/githubpost/issues"
@@ -35,6 +37,7 @@ type GithubPoster interface {
3537
// githubIssues struct implements GithubPoster
3638
type githubIssues struct {
3739
disable bool
40+
dryRun bool
3841
issuePoster func(context.Context, issues.Logger, issues.IssueFormatter, issues.PostRequest,
3942
*issues.Options) (*issues.TestFailureIssue, error)
4043
teamLoader func() (team.Map, error)
@@ -327,6 +330,10 @@ func (g *githubIssues) MaybePost(
327330
message string,
328331
params map[string]string,
329332
) (*issues.TestFailureIssue, error) {
333+
if g.dryRun {
334+
return nil, g.dryRunPost(t, issueInfo, l, message, params)
335+
}
336+
330337
skipReason := g.shouldPost(t)
331338
if skipReason != "" {
332339
l.Printf("skipping GitHub issue posting (%s)", skipReason)
@@ -352,3 +359,75 @@ func (g *githubIssues) MaybePost(
352359
opts,
353360
)
354361
}
362+
363+
// dryRunPost simulates posting an issue to GitHub by rendering
364+
// the issue and logging a clickable link.
365+
func (g *githubIssues) dryRunPost(
366+
t *testImpl,
367+
issueInfo *githubIssueInfo,
368+
l *logger.Logger,
369+
message string,
370+
params map[string]string,
371+
) error {
372+
postRequest, err := g.createPostRequest(
373+
t.Name(), t.start, t.end, t.spec, t.failures(),
374+
message,
375+
roachtestutil.UsingRuntimeAssertions(t), t.goCoverEnabled, params, issueInfo,
376+
)
377+
if err != nil {
378+
return err
379+
}
380+
_, url, err := formatPostRequest(postRequest)
381+
if err != nil {
382+
return err
383+
}
384+
l.Printf("GitHub issue posting in dry-run mode:\n%s", url)
385+
return nil
386+
}
387+
388+
// formatPostRequest returns a string representation of the rendered PostRequest
389+
// as well as a link that can be followed to open the issue in Github. The rendered
390+
// PostRequest also includes the labels that would be applied to the issue as part
391+
// of the body.
392+
func formatPostRequest(req issues.PostRequest) (string, string, error) {
393+
data := issues.TemplateData{
394+
PostRequest: req,
395+
Parameters: req.ExtraParams,
396+
CondensedMessage: issues.CondensedMessage(req.Message),
397+
Branch: "test_branch",
398+
Commit: "test_SHA",
399+
PackageNameShort: strings.TrimPrefix(req.PackageName, issues.CockroachPkgPrefix),
400+
}
401+
402+
formatter := issues.UnitTestFormatter
403+
r := &issues.Renderer{}
404+
if err := formatter.Body(r, data); err != nil {
405+
return "", "", err
406+
}
407+
408+
var post strings.Builder
409+
post.WriteString(r.String())
410+
411+
// Github labels are normally not part of the rendered issue body, but we want to
412+
// still test that they are correctly set so append them here.
413+
post.WriteString("\n------\nLabels:\n")
414+
for _, label := range req.Labels {
415+
post.WriteString(fmt.Sprintf("- <code>%s</code>\n", label))
416+
}
417+
418+
u, err := url.Parse("https://github.com/cockroachdb/cockroach/issues/new")
419+
if err != nil {
420+
return "", "", err
421+
}
422+
q := u.Query()
423+
q.Add("title", formatter.Title(data))
424+
q.Add("body", post.String())
425+
// Adding a template parameter is required to be able to view the rendered
426+
// template on GitHub, otherwise it just takes you to the template selection
427+
// page.
428+
q.Add("template", "none")
429+
u.RawQuery = q.Encode()
430+
post.WriteString(fmt.Sprintf("Rendered:\n%s", u.String()))
431+
432+
return post.String(), u.String(), nil
433+
}

pkg/cmd/roachtest/github_test.go

Lines changed: 2 additions & 49 deletions
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,6 @@ package main
77
import (
88
"context"
99
"fmt"
10-
"net/url"
1110
"path/filepath"
1211
"strings"
1312
"testing"
@@ -90,7 +89,7 @@ func TestShouldPost(t *testing.T) {
9089

9190
ti := &testImpl{spec: testSpec}
9291
ti.mu.failures = c.failures
93-
github := &githubIssues{disable: c.disableIssues}
92+
github := &githubIssues{disable: c.disableIssues, dryRun: false}
9493

9594
skipReason := github.shouldPost(ti)
9695
require.Equal(t, c.expectedReason, skipReason)
@@ -184,7 +183,7 @@ func TestCreatePostRequest(t *testing.T) {
184183
}
185184
require.NoError(t, err)
186185

187-
post, err := formatPostRequest(req)
186+
post, _, err := formatPostRequest(req)
188187
require.NoError(t, err)
189188

190189
return post
@@ -258,49 +257,3 @@ F250826 19:49:07.194443 3106 sql/sem/builtins/builtins.go:6063 ⋮ [T1,Vsystem,n
258257
})
259258
})
260259
}
261-
262-
// formatPostRequest returns a string representation of the rendered PostRequest.
263-
// Additionally, it also includes labels, as well as a link that can be followed
264-
// to open the issue in Github.
265-
func formatPostRequest(req issues.PostRequest) (string, error) {
266-
data := issues.TemplateData{
267-
PostRequest: req,
268-
Parameters: req.ExtraParams,
269-
CondensedMessage: issues.CondensedMessage(req.Message),
270-
Branch: "test_branch",
271-
Commit: "test_SHA",
272-
PackageNameShort: strings.TrimPrefix(req.PackageName, issues.CockroachPkgPrefix),
273-
}
274-
275-
formatter := issues.UnitTestFormatter
276-
r := &issues.Renderer{}
277-
if err := formatter.Body(r, data); err != nil {
278-
return "", err
279-
}
280-
281-
var post strings.Builder
282-
post.WriteString(r.String())
283-
284-
// Github labels are normally not part of the rendered issue body, but we want to
285-
// still test that they are correctly set so append them here.
286-
post.WriteString("\n------\nLabels:\n")
287-
for _, label := range req.Labels {
288-
post.WriteString(fmt.Sprintf("- <code>%s</code>\n", label))
289-
}
290-
291-
u, err := url.Parse("https://github.com/cockroachdb/cockroach/issues/new")
292-
if err != nil {
293-
return "", err
294-
}
295-
q := u.Query()
296-
q.Add("title", formatter.Title(data))
297-
q.Add("body", post.String())
298-
// Adding a template parameter is required to be able to view the rendered
299-
// template on GitHub, otherwise it just takes you to the template selection
300-
// page.
301-
q.Add("template", "none")
302-
u.RawQuery = q.Encode()
303-
post.WriteString(fmt.Sprintf("Rendered:\n%s", u.String()))
304-
305-
return post.String(), nil
306-
}

pkg/cmd/roachtest/roachtestflags/flags.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -445,6 +445,12 @@ var (
445445
Usage: `Disable posting GitHub issue for failures`,
446446
})
447447

448+
DryRunIssuePosting bool
449+
_ = registerRunFlag(&DryRunIssuePosting, FlagInfo{
450+
Name: "dry-run-issue-posting",
451+
Usage: `Enable dry-run mode for GitHub issue posting (formats issues but doesn't post them)`,
452+
})
453+
448454
PromPort int = 2113
449455
_ = registerRunFlag(&PromPort, FlagInfo{
450456
Name: "prom-port",

pkg/cmd/roachtest/run.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -153,6 +153,7 @@ func runTests(register func(registry.Registry), filter *registry.TestFilter) err
153153

154154
github := &githubIssues{
155155
disable: runner.config.disableIssue,
156+
dryRun: runner.config.dryRunIssuePosting,
156157
issuePoster: issues.Post,
157158
teamLoader: team.DefaultLoadTeams,
158159
}

pkg/cmd/roachtest/test_runner.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,8 @@ type testRunner struct {
148148
skipClusterWipeOnAttach bool
149149
// disableIssue disables posting GitHub issues for test failures.
150150
disableIssue bool
151+
// dryRunIssuePosting enables dry-run mode for GitHub issue posting.
152+
dryRunIssuePosting bool
151153
// overrideShutdownPromScrapeInterval overrides the default time a test runner waits to
152154
// shut down, normally used to ensure a remote prometheus server has scraped the roachtest
153155
// endpoint.
@@ -215,6 +217,7 @@ func newTestRunner(cr *clusterRegistry, stopper *stop.Stopper) *testRunner {
215217
}
216218
r.config.skipClusterWipeOnAttach = !roachtestflags.ClusterWipe
217219
r.config.disableIssue = roachtestflags.DisableIssue
220+
r.config.dryRunIssuePosting = roachtestflags.DryRunIssuePosting
218221
r.workersMu.workers = make(map[string]*workerStatus)
219222
return r
220223
}

0 commit comments

Comments
 (0)