Skip to content

Commit

Permalink
roachtest: better logging of GitHub-related logic
Browse files Browse the repository at this point in the history
Whenever a roachtest fails on a release branch, we should either
create a new issue with the failure, or comment on an existing
issue. However, the logic to decide when to create an issue and the
creation process itself are relatively complex. If a roachtest fails
on a nightly build and we don't get a corresponding issue, there's no
way to understand _why_ that happened: issue posting could have been
mistakenly skipped, or maybe an error creating the issue was swallowed
in the process.

This commit adds appropriate logging to the GitHub issue poster: if
issue creation is skipped, the corresponding reason is logged; in
addition, any errors encountered in the process of creating the
issue/comment are also logged.

Epic: None

Release note: None
  • Loading branch information
renatolabs committed Jan 25, 2023
1 parent b84f0eb commit 38854a0
Show file tree
Hide file tree
Showing 9 changed files with 100 additions and 30 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/bazci/githubpost/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//pkg/cmd/internal/issues",
"//pkg/internal/codeowners",
"//pkg/internal/team",
"//pkg/roachprod/logger",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
7 changes: 6 additions & 1 deletion pkg/cmd/bazci/githubpost/githubpost.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/internal/issues"
"github.com/cockroachdb/cockroach/pkg/internal/codeowners"
"github.com/cockroachdb/cockroach/pkg/internal/team"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -95,7 +96,11 @@ func getIssueFilerForFormatter(formatterName string) func(ctx context.Context, f

return func(ctx context.Context, f failure) error {
fmter, req := reqFromFailure(ctx, f)
return issues.Post(ctx, fmter, req)
l, err := logger.RootLogger("", false)
if err != nil {
return err
}
return issues.Post(ctx, l, fmter, req)
}
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/internal/issues/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues",
visibility = ["//pkg/cmd:__subpackages__"],
deps = [
"//pkg/roachprod/logger",
"//pkg/util/version",
"@com_github_cockroachdb_errors//:errors",
"@com_github_google_go_github//github",
Expand All @@ -31,6 +32,7 @@ go_test(
data = glob(["testdata/**"]),
embed = [":issues"],
deps = [
"//pkg/roachprod/logger",
"//pkg/testutils/datapathutils",
"//pkg/testutils/skip",
"@com_github_cockroachdb_datadriven//:datadriven",
Expand Down
24 changes: 14 additions & 10 deletions pkg/cmd/internal/issues/issues.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"os/exec"
"strings"

"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/version"
"github.com/cockroachdb/errors"
"github.com/google/go-github/github"
Expand Down Expand Up @@ -112,6 +113,8 @@ func (p *poster) getProbableMilestone(ctx *postCtx) *int {
type poster struct {
*Options

l *logger.Logger

createIssue func(ctx context.Context, owner string, repo string,
issue *github.IssueRequest) (*github.Issue, *github.Response, error)
searchIssues func(ctx context.Context, query string,
Expand All @@ -126,9 +129,10 @@ type poster struct {
opt *github.ProjectCardOptions) (*github.ProjectCard, *github.Response, error)
}

func newPoster(client *github.Client, opts *Options) *poster {
func newPoster(l *logger.Logger, client *github.Client, opts *Options) *poster {
return &poster{
Options: opts,
l: l,
createIssue: client.Issues.Create,
searchIssues: client.Search.Issues,
createComment: client.Issues.CreateComment,
Expand Down Expand Up @@ -312,9 +316,7 @@ func (p *poster) post(origCtx context.Context, formatter IssueFormatter, req Pos
if err != nil {
// Tough luck, keep going even if that means we're going to add a duplicate
// issue.
//
// TODO(tbg): surface this error.
_ = err
p.l.Printf("error trying to find existing GitHub issues: %v", err)
rExisting = &github.IssuesSearchResult{}
}

Expand All @@ -325,16 +327,15 @@ func (p *poster) post(origCtx context.Context, formatter IssueFormatter, req Pos
})
if err != nil {
// This is no reason to throw the towel, keep going.
//
// TODO(tbg): surface this error.
_ = err
p.l.Printf("error trying to find related GitHub issues: %v", err)
rRelated = &github.IssuesSearchResult{}
}

var foundIssue *int
if len(rExisting.Issues) > 0 {
// We found an existing issue to post a comment into.
foundIssue = rExisting.Issues[0].Number
p.l.Printf("found existing GitHub issue: #%d", *foundIssue)
// We are not going to create an issue, so don't show
// MentionOnCreate to the formatter.Body call below.
data.MentionOnCreate = nil
Expand Down Expand Up @@ -366,6 +367,7 @@ func (p *poster) post(origCtx context.Context, formatter IssueFormatter, req Pos
github.Stringify(issueRequest))
}

p.l.Printf("created GitHub issue #%d", *issue.Number)
if req.ProjectColumnID != 0 {
_, _, err := p.createProjectCard(ctx, int64(req.ProjectColumnID), &github.ProjectCardOptions{
ContentID: *issue.ID,
Expand All @@ -376,7 +378,7 @@ func (p *poster) post(origCtx context.Context, formatter IssueFormatter, req Pos
//
// TODO(tbg): retrieve the project column ID before posting, so that if
// it can't be found we can mention that in the issue we'll file anyway.
_ = err
p.l.Printf("could not create GitHub project card: %v", err)
}
}
} else {
Expand All @@ -385,6 +387,8 @@ func (p *poster) post(origCtx context.Context, formatter IssueFormatter, req Pos
ctx, p.Org, p.Repo, *foundIssue, &comment); err != nil {
return errors.Wrapf(err, "failed to update issue #%d with %s",
*foundIssue, github.Stringify(comment))
} else {
p.l.Printf("created comment on existing GitHub issue (#%d)", *foundIssue)
}
}

Expand Down Expand Up @@ -450,7 +454,7 @@ type PostRequest struct {
// existing open issue. GITHUB_API_TOKEN must be set to a valid GitHub token
// that has permissions to search and create issues and comments or an error
// will be returned.
func Post(ctx context.Context, formatter IssueFormatter, req PostRequest) error {
func Post(ctx context.Context, l *logger.Logger, formatter IssueFormatter, req PostRequest) error {
opts := DefaultOptionsFromEnv()
if !opts.CanPost() {
return errors.Newf("GITHUB_API_TOKEN env variable is not set; cannot post issue")
Expand All @@ -459,7 +463,7 @@ func Post(ctx context.Context, formatter IssueFormatter, req PostRequest) error
client := github.NewClient(oauth2.NewClient(ctx, oauth2.StaticTokenSource(
&oauth2.Token{AccessToken: opts.Token},
)))
return newPoster(client, opts).post(ctx, formatter, req)
return newPoster(l, client, opts).post(ctx, formatter, req)
}

// ReproductionCommandFromString returns a value for the
Expand Down
11 changes: 9 additions & 2 deletions pkg/cmd/internal/issues/issues_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/datadriven"
Expand Down Expand Up @@ -230,8 +231,11 @@ test logs left over in: /go/src/github.com/cockroachdb/cockroach/artifacts/logTe
return tag, nil
}

l, err := logger.RootLogger("", false)
require.NoError(t, err)
p := &poster{
Options: &opts,
l: l,
}

createdIssue := false
Expand All @@ -246,7 +250,7 @@ test logs left over in: /go/src/github.com/cockroachdb/cockroach/artifacts/logTe
render := ghURL(t, title, body)
t.Log(render)
_, _ = fmt.Fprintf(&buf, "createIssue owner=%s repo=%s:\n%s\n\n%s\n\n%s\n\nRendered: %s", owner, repo, github.Stringify(issue), title, body, render)
return &github.Issue{ID: github.Int64(issueID)}, nil, nil
return &github.Issue{ID: github.Int64(issueID), Number: github.Int(issueNumber)}, nil, nil
}

p.searchIssues = func(_ context.Context, query string,
Expand Down Expand Up @@ -366,7 +370,10 @@ func TestPostEndToEnd(t *testing.T) {
HelpCommand: UnitTestHelpCommand(""),
}

require.NoError(t, Post(context.Background(), UnitTestFormatter, req))
l, err := logger.RootLogger("", false)
require.NoError(t, err)

require.NoError(t, Post(context.Background(), l, UnitTestFormatter, req))
}

// setEnv overrides the env variables corresponding to the input map. The
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ go_test(
args = ["-test.timeout=55s"],
embed = [":roachtest_lib"],
deps = [
"//pkg/cmd/internal/issues",
"//pkg/cmd/roachtest/cluster",
"//pkg/cmd/roachtest/option",
"//pkg/cmd/roachtest/registry",
Expand Down
63 changes: 54 additions & 9 deletions pkg/cmd/roachtest/github.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ type githubIssues struct {
l *logger.Logger
cluster *clusterImpl
vmCreateOpts *vm.CreateOpts
issuePoster func(ctx context.Context, formatter issues.IssueFormatter, req issues.PostRequest) error
issuePoster func(context.Context, *logger.Logger, issues.IssueFormatter, issues.PostRequest) error
teamLoader func() (team.Map, error)
}

Expand Down Expand Up @@ -59,13 +59,55 @@ func roachtestPrefix(p string) string {
return "ROACHTEST_" + p
}

func (g *githubIssues) shouldPost(t test.Test) bool {
opts := issues.DefaultOptionsFromEnv()
return !g.disable && opts.CanPost() &&
opts.IsReleaseBranch() &&
t.Spec().(*registry.TestSpec).Run != nil &&
// NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests.
t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0
// postIssueCondition encapsulates a condition that causes issue
// posting to be skipped. The `reason` field contains a textual
// description as to why issue posting was skipped.
type postIssueCondition struct {
cond func(g *githubIssues, t test.Test) bool
reason string
}

var defaultOpts = issues.DefaultOptionsFromEnv()

var skipConditions = []postIssueCondition{
{
cond: func(g *githubIssues, _ test.Test) bool { return g.disable },
reason: "issue posting was disabled via command line flag",
},
{
cond: func(g *githubIssues, _ test.Test) bool { return !defaultOpts.CanPost() },
reason: "GitHub API token not set",
},
{
cond: func(g *githubIssues, _ test.Test) bool { return !defaultOpts.IsReleaseBranch() },
reason: fmt.Sprintf("not a release branch: %q", defaultOpts.Branch),
},
{
cond: func(_ *githubIssues, t test.Test) bool { return t.Spec().(*registry.TestSpec).Run == nil },
reason: "TestSpec.Run is nil",
},
{
cond: func(_ *githubIssues, t test.Test) bool { return t.Spec().(*registry.TestSpec).Cluster.NodeCount == 0 },
reason: "Cluster.NodeCount is zero",
},
}

// shouldPost two values: whether GitHub posting should happen, and a
// reason for skipping (non-empty only when posting should *not*
// happen).
func (g *githubIssues) shouldPost(t test.Test) (bool, string) {
post := true
var reason string

for _, sc := range skipConditions {
if sc.cond(g, t) {
post = false
reason = sc.reason
break
}
}

return post, reason
}

func (g *githubIssues) createPostRequest(
Expand Down Expand Up @@ -160,7 +202,9 @@ func (g *githubIssues) createPostRequest(
}

func (g *githubIssues) MaybePost(t *testImpl, message string) error {
if !g.shouldPost(t) {
doPost, skipReason := g.shouldPost(t)
if !doPost {
g.l.Printf("skipping GitHub issue posting (%s)", skipReason)
return nil
}

Expand All @@ -176,6 +220,7 @@ func (g *githubIssues) MaybePost(t *testImpl, message string) error {

return g.issuePoster(
context.Background(),
g.l,
issues.UnitTestFormatter,
g.createPostRequest(t, cat, message),
)
Expand Down
19 changes: 12 additions & 7 deletions pkg/cmd/roachtest/github_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/cmd/internal/issues"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
Expand Down Expand Up @@ -60,24 +61,26 @@ func TestShouldPost(t *testing.T) {
nodeCount int
envGithubAPIToken string
envTcBuildBranch string
expected bool
expectedPost bool
expectedReason string
}{
/* Cases 1 - 4 verify that issues are not posted if any of the relevant criteria checks fail */
// disable
{true, 1, "token", "master", false},
{true, 1, "token", "master", false, "issue posting was disabled via command line flag"},
// nodeCount
{false, 0, "token", "master", false},
{false, 0, "token", "master", false, "Cluster.NodeCount is zero"},
// apiToken
{false, 1, "", "master", false},
{false, 1, "", "master", false, "GitHub API token not set"},
// branch
{false, 1, "token", "", false},
{false, 1, "token", "master", true},
{false, 1, "token", "", false, `not a release branch: "branch-not-found-in-env"`},
{false, 1, "token", "master", true, ""},
}

reg, _ := makeTestRegistry(spec.GCE, "", "", false)
for _, c := range testCases {
t.Setenv("GITHUB_API_TOKEN", c.envGithubAPIToken)
t.Setenv("TC_BUILD_BRANCH", c.envTcBuildBranch)
defaultOpts = issues.DefaultOptionsFromEnv() // recompute options from env

clusterSpec := reg.MakeClusterSpec(c.nodeCount)
testSpec := &registry.TestSpec{
Expand All @@ -97,7 +100,9 @@ func TestShouldPost(t *testing.T) {
disable: c.disableIssues,
}

require.Equal(t, c.expected, github.shouldPost(ti))
doPost, skipReason := github.shouldPost(ti)
require.Equal(t, c.expectedPost, doPost)
require.Equal(t, c.expectedReason, skipReason)
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/roachprod/logger/log.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ type Logger struct {
// If path is empty, logs will go to stdout/Stderr.
func (cfg *Config) NewLogger(path string) (*Logger, error) {
if path == "" {
// Log to os.Stdout/Stderr is no other options are passed in.
// Log to os.Stdout/Stderr if no other options are passed in.
stdout := cfg.Stdout
if stdout == nil {
stdout = os.Stdout
Expand Down

0 comments on commit 38854a0

Please sign in to comment.