Skip to content

Commit

Permalink
Merge #122281
Browse files Browse the repository at this point in the history
122281: roachtest: VM hostError should be flagged as an infra flake r=vidit-bhat a=vidit-bhat

Previously, we were seeing roachtest failures due to `compute.instances.hostError`. These were being wrongly directed to different engineering teams while in reality the VM hostError is an infra flake.

This PR flags the `compute.instances.hostError` as an infra flake and avoids the unnecessary noise.

Epic: none
Fixes: #101223
Release note: None

Co-authored-by: Vidit Bhat <[email protected]>
  • Loading branch information
craig[bot] and vidit-bhat committed Apr 18, 2024
2 parents d9e7a6a + 0273873 commit 859b6b1
Show file tree
Hide file tree
Showing 12 changed files with 239 additions and 34 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ go_library(
"//pkg/cmd/roachtest/tests",
"//pkg/internal/team",
"//pkg/roachprod",
"//pkg/roachprod/cloud",
"//pkg/roachprod/config",
"//pkg/roachprod/errors",
"//pkg/roachprod/install",
Expand Down
65 changes: 53 additions & 12 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests"
"github.com/cockroachdb/cockroach/pkg/roachprod"
"github.com/cockroachdb/cockroach/pkg/roachprod/cloud"
"github.com/cockroachdb/cockroach/pkg/roachprod/config"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
Expand Down Expand Up @@ -3013,31 +3014,46 @@ func archForTest(ctx context.Context, l *logger.Logger, testSpec registry.TestSp
return arch
}

// GetPreemptedVMs gets any VMs that were part of the cluster but preempted by cloud vendor.
func (c *clusterImpl) GetPreemptedVMs(
ctx context.Context, l *logger.Logger,
) ([]vm.PreemptedVM, error) {
if c.IsLocal() || !c.spec.UseSpotVMs {
return nil, nil
// bucketVMsByProvider buckets cachedCluster.VMs by provider.
func bucketVMsByProvider(cachedCluster *cloud.Cluster) map[string][]vm.VM {
providerToVMs := make(map[string][]vm.VM)
for _, vm := range cachedCluster.VMs {
providerToVMs[vm.Provider] = append(providerToVMs[vm.Provider], vm)
}
return providerToVMs
}

cachedCluster, ok := roachprod.CachedCluster(c.name)
// getCachedCluster checks if the passed cluster name is present in cached clusters
// and returns an error if not found.
func getCachedCluster(clusterName string) (*cloud.Cluster, error) {
cachedCluster, ok := roachprod.CachedCluster(clusterName)
if !ok {
var availableClusters []string
roachprod.CachedClusters(func(name string, _ int) {
availableClusters = append(availableClusters, name)
})

err := errors.Wrapf(errClusterNotFound, "%q", c.name)
err := errors.Wrapf(errClusterNotFound, "%q", clusterName)
return nil, errors.WithHintf(err, "\nAvailable clusters:\n%s", strings.Join(availableClusters, "\n"))
}

// Bucket cachedCluster.VMs by provider.
providerToVMs := make(map[string][]vm.VM)
for _, vm := range cachedCluster.VMs {
providerToVMs[vm.Provider] = append(providerToVMs[vm.Provider], vm)
return cachedCluster, nil
}

// GetPreemptedVMs gets any VMs that were part of the cluster but preempted by cloud vendor.
func (c *clusterImpl) GetPreemptedVMs(
ctx context.Context, l *logger.Logger,
) ([]vm.PreemptedVM, error) {
if c.IsLocal() || !c.spec.UseSpotVMs {
return nil, nil
}

cachedCluster, err := getCachedCluster(c.name)
if err != nil {
return nil, err
}
providerToVMs := bucketVMsByProvider(cachedCluster)

var allPreemptedVMs []vm.PreemptedVM
for provider, vms := range providerToVMs {
p := vm.Providers[provider]
Expand All @@ -3052,3 +3068,28 @@ func (c *clusterImpl) GetPreemptedVMs(
}
return allPreemptedVMs, nil
}

// GetHostErrorVMs gets any VMs that were part of the cluster but has a host error.
func (c *clusterImpl) GetHostErrorVMs(ctx context.Context, l *logger.Logger) ([]string, error) {
if c.IsLocal() {
return nil, nil
}

cachedCluster, err := getCachedCluster(c.name)
if err != nil {
return nil, err
}
providerToVMs := bucketVMsByProvider(cachedCluster)

var allHostErrorVMs []string
for provider, vms := range providerToVMs {
p := vm.Providers[provider]
hostErrorVMS, err := p.GetHostErrorVMs(l, vms, cachedCluster.CreatedAt)
if err != nil {
l.Errorf("failed to get hostError VMs for provider %s: %s", provider, err)
continue
}
allHostErrorVMs = append(allHostErrorVMs, hostErrorVMS...)
}
return allHostErrorVMs, nil
}
24 changes: 24 additions & 0 deletions pkg/cmd/roachtest/github_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,30 @@ func TestCreatePostRequest(t *testing.T) {
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
// 13. Verify hostError failure are routed to test-eng and marked as infra-flake, when the
// first failure is a non-handled error.
{
nonReleaseBlocker: true,
failures: []failure{createFailure(errors.New("random")), createFailure(vmHostError("my_VM"))},
expectedPost: true,
expectedTeam: "@cockroachdb/test-eng",
expectedName: "vm_host_error",
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
// 14. Verify hostError failure are routed to test-eng and marked as infra-flake, when the only error is
// hostError failure
{
nonReleaseBlocker: true,
failures: []failure{
{errors: []error{vmHostError("my_VM")}},
},
expectedPost: true,
expectedTeam: "@cockroachdb/test-eng",
expectedName: "vm_host_error",
expectedMessagePrefix: testName + " failed",
expectedLabels: []string{"T-testeng", "X-infra-flake"},
},
}

reg := makeTestRegistry()
Expand Down
70 changes: 55 additions & 15 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,17 @@ var (
)
}

// vmHostError is the error that indicates that a test failed
// a result of VM host error. These errors are directed to Test Eng
// instead of owning teams.
vmHostError = func(hostErrorVMs string) error {
return registry.ErrorWithOwner(
registry.OwnerTestEng, fmt.Errorf("hostError VMs: %s", hostErrorVMs),
registry.WithTitleOverride("vm_host_error"),
registry.InfraFlake,
)
}

prng, _ = randutil.NewLockedPseudoRand()

runID string
Expand Down Expand Up @@ -1045,6 +1056,13 @@ func (r *testRunner) runTest(
t.resetFailures()
t.Error(vmPreemptionError(preemptedVMNames))
}
hostErrorVMNames := getHostErrorVMNames(ctx, c, l)
if hostErrorVMNames != "" {
failureMsg = fmt.Sprintf("VMs received host error during the test run: %s\n\n**Other Failures:**\n%s", hostErrorVMNames, failureMsg)
t.resetFailures()
t.Error(vmHostError(hostErrorVMNames))
}

output := fmt.Sprintf("%s\ntest artifacts and logs in: %s", failureMsg, t.ArtifactsDir())

issue, err := github.MaybePost(t, l, output)
Expand Down Expand Up @@ -1238,33 +1256,55 @@ func (r *testRunner) runTest(
}
}

// getPreemptedVMNames returns a comma separated list of preempted VM
// names, or an empty string if no VM was preempted or an error was found.
func getPreemptedVMNames(ctx context.Context, c *clusterImpl, l *logger.Logger) string {
preemptedVMs, err := c.GetPreemptedVMs(ctx, l)
if err != nil {
l.Printf("failed to check preempted VMs:\n%+v", err)
return ""
}

var preemptedVMNames []string
for _, item := range preemptedVMs {
// getVMNames returns a comma separated list of VM names.
func getVMNames(fullVMNames []string) string {
var vmNames []string
for _, name := range fullVMNames {
// Expected format: projects/{project}/zones/{zone}/instances/{name}
parts := strings.Split(item.Name, "/")
parts := strings.Split(name, "/")

// If the instance name is in the expected format, only include
// the VM name and the zone, to make it easier to for a human
// reading the output.
if len(parts) == 6 {
instanceName := parts[5]
zone := parts[3]
preemptedVMNames = append(preemptedVMNames, fmt.Sprintf("%s (%s)", instanceName, zone))
vmNames = append(vmNames, fmt.Sprintf("%s (%s)", instanceName, zone))
} else {
preemptedVMNames = append(preemptedVMNames, item.Name)
vmNames = append(vmNames, name)
}
}

return strings.Join(preemptedVMNames, ", ")
return strings.Join(vmNames, ", ")
}

// getPreemptedVMNames returns a comma separated list of preempted VM
// names, or an empty string if no VM was preempted or an error was found.
func getPreemptedVMNames(ctx context.Context, c *clusterImpl, l *logger.Logger) string {
preemptedVMs, err := c.GetPreemptedVMs(ctx, l)
if err != nil {
l.Printf("failed to check preempted VMs:\n%+v", err)
return ""
}

preemptedVMNames := make([]string, len(preemptedVMs))
for _, preemptedVM := range preemptedVMs {
preemptedVMNames = append(preemptedVMNames, preemptedVM.Name)
}

return getVMNames(preemptedVMNames)
}

// getHostErrorVMNames returns a comma separated list of host error VM
// names, or an empty string if no VM had a host error.
func getHostErrorVMNames(ctx context.Context, c *clusterImpl, l *logger.Logger) string {
hostErrorVMs, err := c.GetHostErrorVMs(ctx, l)
if err != nil {
l.Printf("failed to check hostError VMs:\n%+v", err)
return ""
}

return getVMNames(hostErrorVMs)
}

// The assertions here are executed after each test, and may result in a test failure. Test authors
Expand Down
17 changes: 17 additions & 0 deletions pkg/cmd/roachtest/testdata/help_command_createpost_13.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
echo
----
----


See: [roachtest README](https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md)



See: [How To Investigate \(internal\)](https://cockroachlabs.atlassian.net/l/c/SSSBr8c7)



See: [Grafana](https://go.crdb.dev/roachtest-grafana//github-test/1689957243000/1689957853000)

----
----
17 changes: 17 additions & 0 deletions pkg/cmd/roachtest/testdata/help_command_createpost_14.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
echo
----
----


See: [roachtest README](https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md)



See: [How To Investigate \(internal\)](https://cockroachlabs.atlassian.net/l/c/SSSBr8c7)



See: [Grafana](https://go.crdb.dev/roachtest-grafana//github-test/1689957243000/1689957853000)

----
----
6 changes: 6 additions & 0 deletions pkg/roachprod/vm/aws/aws.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,6 +272,12 @@ func (p *Provider) GetPreemptedSpotVMs(
return nil, nil
}

func (p *Provider) GetHostErrorVMs(
l *logger.Logger, vms vm.List, since time.Time,
) ([]string, error) {
return nil, nil
}

const (
defaultSSDMachineType = "m6id.xlarge"
defaultMachineType = "m6i.xlarge"
Expand Down
6 changes: 6 additions & 0 deletions pkg/roachprod/vm/azure/azure.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,12 @@ func (p *Provider) GetPreemptedSpotVMs(
return nil, nil
}

func (p *Provider) GetHostErrorVMs(
l *logger.Logger, vms vm.List, since time.Time,
) ([]string, error) {
return nil, nil
}

func (p *Provider) CreateVolumeSnapshot(
l *logger.Logger, volume vm.Volume, vsco vm.VolumeSnapshotCreateOpts,
) (vm.VolumeSnapshot, error) {
Expand Down
6 changes: 6 additions & 0 deletions pkg/roachprod/vm/flagstub/flagstub.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@ func (p *provider) GetPreemptedSpotVMs(
return nil, nil
}

func (p *provider) GetHostErrorVMs(
l *logger.Logger, vms vm.List, since time.Time,
) ([]string, error) {
return nil, nil
}

func (p *provider) CreateVolumeSnapshot(
l *logger.Logger, volume vm.Volume, vsco vm.VolumeSnapshotCreateOpts,
) (vm.VolumeSnapshot, error) {
Expand Down
53 changes: 46 additions & 7 deletions pkg/roachprod/vm/gce/gcloud.go
Original file line number Diff line number Diff line change
Expand Up @@ -379,11 +379,32 @@ func (p *Provider) GetPreemptedSpotVMs(
return preemptedVMs, nil
}

// buildFilterPreemptionCliArgs returns the arguments to be passed to gcloud cli to query the logs for preemption events.
func buildFilterPreemptionCliArgs(
vms vm.List, projectName string, since time.Time,
// GetHostErrorVMs checks the host error status of the given VMs, by querying the GCP logging service.
func (p *Provider) GetHostErrorVMs(
l *logger.Logger, vms vm.List, since time.Time,
) ([]string, error) {
args, err := buildFilterHostErrorCliArgs(vms, since, p.GetProject())
if err != nil {
l.Printf("Error building gcloud cli command: %v\n", err)
return nil, err
}
l.Printf("gcloud cli for host error : " + strings.Join(append([]string{"gcloud"}, args...), " "))
var logEntries []LogEntry
if err := runJSONCommand(args, &logEntries); err != nil {
l.Printf("Error running gcloud cli command: %v\n", err)
return nil, err
}
// Extract the name of the VM with host error from logs.
var hostErrorVMs []string
for _, logEntry := range logEntries {
hostErrorVMs = append(hostErrorVMs, logEntry.ProtoPayload.ResourceName)
}
return hostErrorVMs, nil
}

func buildFilterCliArgs(
vms vm.List, projectName string, since time.Time, filter string,
) ([]string, error) {
vmFullResourceNames := make([]string, len(vms))
if projectName == "" {
return nil, errors.New("project name cannot be empty")
}
Expand All @@ -394,6 +415,7 @@ func buildFilterPreemptionCliArgs(
return nil, errors.New("vms cannot be nil")
}
// construct full resource names
vmFullResourceNames := make([]string, len(vms))
for i, vmNode := range vms {
// example format : projects/cockroach-ephemeral/zones/us-east1-b/instances/test-name
vmFullResourceNames[i] = "projects/" + projectName + "/zones/" + vmNode.Zone + "/instances/" + vmNode.Name
Expand All @@ -403,9 +425,7 @@ func buildFilterPreemptionCliArgs(
for i, vmID := range vmFullResourceNames {
vmIDFilter[i] = fmt.Sprintf("protoPayload.resourceName=%s", vmID)
}
// Create a filter to match preemption events for the specified VM IDs
filter := fmt.Sprintf(`resource.type=gce_instance AND (protoPayload.methodName=compute.instances.preempted) AND (%s)`,
strings.Join(vmIDFilter, " OR "))
filter += fmt.Sprintf(` AND (%s)`, strings.Join(vmIDFilter, " OR "))
args := []string{
"logging",
"read",
Expand All @@ -417,6 +437,25 @@ func buildFilterPreemptionCliArgs(
return args, nil
}

// buildFilterPreemptionCliArgs returns the arguments to be passed to gcloud cli to query the logs for preemption events.
func buildFilterPreemptionCliArgs(
vms vm.List, projectName string, since time.Time,
) ([]string, error) {
// Create a filter to match preemption events
filter := `resource.type=gce_instance AND (protoPayload.methodName=compute.instances.preempted)`
return buildFilterCliArgs(vms, projectName, since, filter)
}

// buildFilterHostErrorCliArgs returns the arguments to be passed to gcloud cli to query the logs for host error events.
func buildFilterHostErrorCliArgs(
vms vm.List, since time.Time, projectName string,
) ([]string, error) {
// Create a filter to match hostError events for the specified projectName
filter := fmt.Sprintf(`resource.type=gce_instance AND protoPayload.methodName=compute.instances.hostError
AND logName=projects/%s/logs/cloudaudit.googleapis.com%%2Fsystem_event`, projectName)
return buildFilterCliArgs(vms, projectName, since, filter)
}

type snapshotJson struct {
CreationSizeBytes string `json:"creationSizeBytes"`
CreationTimestamp time.Time `json:"creationTimestamp"`
Expand Down
Loading

0 comments on commit 859b6b1

Please sign in to comment.