Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
107965: roachtest: roachprod: add test name and run id vm labels for metrics r=herkolategan a=smg260

These 2 commits add labels to clusters running roachtests, so that metrics can be better filtered in various dashboards. 

1. Adds `test_name` label to each cluster, and removes the label at the end of the test. Thus, each cluster would have this label updated for each test that it runs during a particular roachtest invocation. The test name will be simplified to conform to cloud labelling rules `[a-zA-Z-]`

2. Adds `test_run_id` label to each VM, *once*, for the duration of the run. Thus, each cluster would have this label added once at the beginning of a roachtest run (which would include multiple tests), and removed only after deregistration at the end.
\
In TeamCity this would take the form `<TC_USER>-<TC_BUILD_ID>`, and run locally `<USER>-<UNIX_TS>`

These 2 labels combined will allow it easy for a user to find metrics for a particular run of roachtest. (e.g. a specific GCE nightly)

Here is a [copy of an existing dashboard](https://grafana.testeng.crdb.io/d/qdkBruq4k/crdb-console-runtime-by-test?orgId=1&from=now-3h&to=now), modified to utilise the new labels.

Epic: None
Fixes: #98658
Release note: None

108037: server: return authoritative span statistics for db details endpoint r=THardy98 a=THardy98

Resolves: #96163

This change makes the admin API endpoint getting database statistics scan KV for span statistics instead of using the range descriptor cache. This provides authoritative output, helping deflake `TestMultiRegionDatabaseStats`.

Release note (sql change): admin API database details endpoint now returns authoritative range statistics.

108711: upgrades: deflake TestRoleMembersIDMigration1500Users r=rafiss a=rafiss

TeamCity has a new machine type where this test has started to time out more, so this change will make it take less time.

fixes #108539
Release note: None

Co-authored-by: Miral Gadani <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
4 people committed Aug 14, 2023
4 parents b1db60c + 45b20bc + b2faa2b + 08b2eb7 commit dc2c52d
Show file tree
Hide file tree
Showing 13 changed files with 284 additions and 103 deletions.
6 changes: 6 additions & 0 deletions pkg/ccl/serverccl/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,12 @@ func TestTableAndDatabaseDetailsAndStats(t *testing.T) {

require.Equal(t, dbResp.TableNames[0], "public.test")

var dbDetailsResp serverpb.DatabaseDetailsResponse
err = getAdminJSONProto(st, "databases/defaultdb?include_stats=true", &dbDetailsResp)
require.NoError(t, err)

require.Greater(t, dbDetailsResp.Stats.RangeCount, int64(0))

// TableStats
tableStatsResp := &serverpb.TableStatsResponse{}
err = getAdminJSONProto(st, "databases/defaultdb/tables/public.test/stats", tableStatsResp)
Expand Down
16 changes: 16 additions & 0 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -615,6 +615,11 @@ func (r *clusterRegistry) registerCluster(c *clusterImpl) error {
return fmt.Errorf("cluster named %q already exists in registry", c.name)
}
r.mu.clusters[c.name] = c
if err := c.addLabels(map[string]string{
VmLabelTestRunID: runID,
}); err != nil && c.l != nil {
c.l.Printf("failed to add %s label to cluster: %s", VmLabelTestRunID, err)
}
return nil
}

Expand All @@ -626,6 +631,9 @@ func (r *clusterRegistry) unregisterCluster(c *clusterImpl) bool {
// method to be called defensively.
return false
}
if err := c.removeLabels([]string{VmLabelTestRunID}); err != nil && c.l != nil {
c.l.Printf("failed to remove %s label from cluster: %s", VmLabelTestRunID, err)
}
delete(r.mu.clusters, c.name)
if c.tag != "" {
if _, ok := r.mu.tagCount[c.tag]; !ok {
Expand Down Expand Up @@ -1870,6 +1878,14 @@ func (c *clusterImpl) doDestroy(ctx context.Context, l *logger.Logger) <-chan st
return ch
}

func (c *clusterImpl) addLabels(labels map[string]string) error {
return roachprod.AddLabels(c.l, c.name, labels)
}

func (c *clusterImpl) removeLabels(labels []string) error {
return roachprod.RemoveLabels(c.l, c.name, labels)
}

func (c *clusterImpl) ListSnapshots(
ctx context.Context, vslo vm.VolumeSnapshotListOpts,
) ([]vm.VolumeSnapshot, error) {
Expand Down
72 changes: 30 additions & 42 deletions pkg/cmd/roachtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,29 @@ Examples:
listCmd.Flags().StringVar(
&cloud, "cloud", cloud, "cloud provider to use (aws, azure, or gce)")

runFn := func(args []string, benchOnly bool) error {
if literalArtifacts == "" {
literalArtifacts = artifacts
}
return runTests(tests.RegisterTests, cliCfg{
args: args,
count: count,
cpuQuota: cpuQuota,
runSkipped: runSkipped,
debugMode: debugModeFromOpts(),
skipInit: skipInit,
httpPort: httpPort,
promPort: promPort,
parallelism: parallelism,
artifactsDir: artifacts,
literalArtifactsDir: literalArtifacts,
user: getUser(username),
clusterID: clusterID,
versionsBinaryOverride: versionsBinaryOverride,
selectProbability: selectProbability,
}, benchOnly)
}

var runCmd = &cobra.Command{
// Don't display usage when tests fail.
SilenceUsage: true,
Expand All @@ -298,26 +321,7 @@ COCKROACH_ environment variables in the local environment are passed through to
the cluster nodes on start.
`,
RunE: func(_ *cobra.Command, args []string) error {
if literalArtifacts == "" {
literalArtifacts = artifacts
}
return runTests(tests.RegisterTests, cliCfg{
args: args,
count: count,
cpuQuota: cpuQuota,
runSkipped: runSkipped,
debugMode: debugModeFromOpts(),
skipInit: skipInit,
httpPort: httpPort,
promPort: promPort,
parallelism: parallelism,
artifactsDir: artifacts,
literalArtifactsDir: literalArtifacts,
user: username,
clusterID: clusterID,
versionsBinaryOverride: versionsBinaryOverride,
selectProbability: selectProbability,
}, false /* benchOnly */)
return runFn(args, false /* benchOnly */)
},
}

Expand All @@ -341,24 +345,7 @@ the cluster nodes on start.
Short: "run automated benchmarks on cockroach cluster",
Long: `Run automated benchmarks on existing or ephemeral cockroach clusters.`,
RunE: func(_ *cobra.Command, args []string) error {
if literalArtifacts == "" {
literalArtifacts = artifacts
}
return runTests(tests.RegisterTests, cliCfg{
args: args,
count: count,
cpuQuota: cpuQuota,
runSkipped: runSkipped,
debugMode: debugModeFromOpts(),
skipInit: skipInit,
httpPort: httpPort,
parallelism: parallelism,
artifactsDir: artifacts,
user: username,
clusterID: clusterID,
versionsBinaryOverride: versionsBinaryOverride,
selectProbability: selectProbability,
}, true /* benchOnly */)
return runFn(args, true /* benchOnly */)
},
}

Expand Down Expand Up @@ -498,10 +485,11 @@ func runTests(register func(registry.Registry), cfg cliCfg, benchOnly bool) erro
opt := clustersOpt{
typ: clusterType,
clusterName: clusterName,
user: getUser(cfg.user),
cpuQuota: cfg.cpuQuota,
debugMode: cfg.debugMode,
clusterID: cfg.clusterID,
// Precedence for resolving the user: cli arg, env.ROACHPROD_USER, current user.
user: cfg.user,
cpuQuota: cfg.cpuQuota,
debugMode: cfg.debugMode,
clusterID: cfg.clusterID,
}
if err := runner.runHTTPServer(cfg.httpPort, os.Stdout, bindTo); err != nil {
return err
Expand Down
24 changes: 24 additions & 0 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,16 @@ var (
prometheusScrapeInterval = time.Second * 15

prng, _ = randutil.NewLockedPseudoRand()

runID string
)

// VmLabelTestName is the label used to identify the test name in the VM metadata
const VmLabelTestName string = "test_name"

// VmLabelTestRunID is the label used to identify the test run id in the VM metadata
const VmLabelTestRunID string = "test_run_id"

// testRunner runs tests.
type testRunner struct {
stopper *stop.Stopper
Expand Down Expand Up @@ -296,6 +304,8 @@ func (r *testRunner) Run(

qp := quotapool.NewIntPool("cloud cpu", uint64(clustersOpt.cpuQuota))
l := lopt.l
runID = generateRunID(clustersOpt.user)
shout(ctx, l, lopt.stdout, "%s: %s", VmLabelTestRunID, runID)
var wg sync.WaitGroup

for i := 0; i < parallelism; i++ {
Expand Down Expand Up @@ -386,6 +396,16 @@ func numConcurrentClusterCreations() int {
return res
}

// This will be added as a label to all cluster nodes when the
// cluster is registered.
func generateRunID(user string) string {
uniqueId := os.Getenv("TC_BUILD_ID")
if uniqueId == "" {
uniqueId = fmt.Sprintf("%d", timeutil.Now().Unix())
}
return fmt.Sprintf("%s-%s", user, uniqueId)
}

// defaultClusterAllocator is used by workers to create new clusters (or to attach
// to an existing one).
//
Expand Down Expand Up @@ -911,7 +931,11 @@ func (r *testRunner) runTest(
t.runnerID = goid.Get()

s := t.Spec().(*registry.TestSpec)
_ = c.addLabels(map[string]string{
VmLabelTestName: s.Name,
})
defer func() {
_ = c.removeLabels([]string{VmLabelTestName})
t.end = timeutil.Now()

// We only have to record panics if the panic'd value is not the sentinel
Expand Down
28 changes: 28 additions & 0 deletions pkg/roachprod/roachprod.go
Original file line number Diff line number Diff line change
Expand Up @@ -1290,6 +1290,34 @@ func cleanupFailedCreate(l *logger.Logger, clusterName string) error {
return cloud.DestroyCluster(l, c)
}

func AddLabels(l *logger.Logger, clusterName string, labels map[string]string) error {
if err := LoadClusters(); err != nil {
return err
}
c, err := newCluster(l, clusterName)
if err != nil {
return err
}

return vm.FanOut(c.VMs, func(p vm.Provider, vms vm.List) error {
return p.AddLabels(l, vms, labels)
})
}

func RemoveLabels(l *logger.Logger, clusterName string, labels []string) error {
if err := LoadClusters(); err != nil {
return err
}
c, err := newCluster(l, clusterName)
if err != nil {
return err
}

return vm.FanOut(c.VMs, func(p vm.Provider, vms vm.List) error {
return p.RemoveLabels(l, vms, labels)
})
}

// Create TODO
func Create(
ctx context.Context,
Expand Down
82 changes: 61 additions & 21 deletions pkg/roachprod/vm/aws/aws.go
Original file line number Diff line number Diff line change
Expand Up @@ -421,6 +421,64 @@ func (p *Provider) ConfigSSH(l *logger.Logger, zones []string) error {
return g.Wait()
}

// editLabels is a helper that adds or removes labels from the given VMs.
func (p *Provider) editLabels(
l *logger.Logger, vms vm.List, labels map[string]string, remove bool,
) error {
args := []string{"ec2"}
if remove {
args = append(args, "delete-tags")
} else {
args = append(args, "create-tags")
}

args = append(args, "--tags")
tagArgs := make([]string, 0, len(labels))
for key, value := range labels {
if remove {
tagArgs = append(tagArgs, fmt.Sprintf("Key=%s", key))
} else {
tagArgs = append(tagArgs, fmt.Sprintf("Key=%s,Value=%s", key, vm.SanitizeLabel(value)))
}
}
args = append(args, tagArgs...)

byRegion, err := regionMap(vms)
if err != nil {
return err
}
g := errgroup.Group{}
for region, list := range byRegion {
// Capture loop vars here
regionArgs := make([]string, len(args))
copy(regionArgs, args)

regionArgs = append(regionArgs, "--region", region)
regionArgs = append(regionArgs, "--resources")
regionArgs = append(regionArgs, list.ProviderIDs()...)

g.Go(func() error {
_, err := p.runCommand(l, regionArgs)
return err
})
}
return g.Wait()
}

// AddLabels adds the given labels to the given VMs.
func (p *Provider) AddLabels(l *logger.Logger, vms vm.List, labels map[string]string) error {
return p.editLabels(l, vms, labels, false)
}

// RemoveLabels removes the given labels from the given VMs.
func (p *Provider) RemoveLabels(l *logger.Logger, vms vm.List, labels []string) error {
labelMap := make(map[string]string, len(labels))
for _, label := range labels {
labelMap[label] = ""
}
return p.editLabels(l, vms, labelMap, true)
}

// Create is part of the vm.Provider interface.
func (p *Provider) Create(
l *logger.Logger, names []string, opts vm.CreateOpts, vmProviderOpts vm.ProviderOpts,
Expand Down Expand Up @@ -594,27 +652,9 @@ func (p *Provider) Reset(l *logger.Logger, vms vm.List) error {
// Extend is part of the vm.Provider interface.
// This will update the Lifetime tag on the instances.
func (p *Provider) Extend(l *logger.Logger, vms vm.List, lifetime time.Duration) error {
byRegion, err := regionMap(vms)
if err != nil {
return err
}
g := errgroup.Group{}
for region, list := range byRegion {
// Capture loop vars here
args := []string{
"ec2", "create-tags",
"--region", region,
"--tags", "Key=Lifetime,Value=" + lifetime.String(),
"--resources",
}
args = append(args, list.ProviderIDs()...)

g.Go(func() error {
_, err := p.runCommand(l, args)
return err
})
}
return g.Wait()
return p.AddLabels(l, vms, map[string]string{
"Lifetime": lifetime.String(),
})
}

// cachedActiveAccount memoizes the return value from FindActiveAccount
Expand Down
10 changes: 10 additions & 0 deletions pkg/roachprod/vm/azure/azure.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,16 @@ func getAzureDefaultLabelMap(opts vm.CreateOpts) map[string]string {
return m
}

func (p *Provider) AddLabels(l *logger.Logger, vms vm.List, labels map[string]string) error {
l.Printf("adding labels to Azure VMs not yet supported")
return nil
}

func (p *Provider) RemoveLabels(l *logger.Logger, vms vm.List, labels []string) error {
l.Printf("removing labels from Azure VMs not yet supported")
return nil
}

// Create implements vm.Provider.
func (p *Provider) Create(
l *logger.Logger, names []string, opts vm.CreateOpts, vmProviderOpts vm.ProviderOpts,
Expand Down
8 changes: 8 additions & 0 deletions pkg/roachprod/vm/flagstub/flagstub.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,14 @@ func (p *provider) ConfigSSH(l *logger.Logger, zones []string) error {
return nil
}

func (p *provider) AddLabels(l *logger.Logger, vms vm.List, labels map[string]string) error {
return nil
}

func (p *provider) RemoveLabels(l *logger.Logger, vms vm.List, labels []string) error {
return nil
}

// Create implements vm.Provider and returns Unimplemented.
func (p *provider) Create(
l *logger.Logger, names []string, opts vm.CreateOpts, providerOpts vm.ProviderOpts,
Expand Down
Loading

0 comments on commit dc2c52d

Please sign in to comment.