Skip to content

Commit bc50ec9

Browse files
craig[bot]DarrylWongspilchen
committed
144411: failure-injection: add node kill failure r=herkolategan a=DarrylWong This change adds a node kill failure mode which can either gracefully or ungracefully shutdown a cockroach process. Informs: #138970 Release note: none 147908: workload/schemachange: allow dep error as potential error for ALTER PK r=spilchen a=spilchen ALTER PRIMARY KEY has been observed to fail with the following error in the RSW (#147514 (comment)): ``` ERROR: cannot drop column "rowid" because trigger "trigger_w0_190" on table "table_w1_104" depends on it (SQLSTATE 2BP01) ``` This change adds that dependency error to the set of potential errors for ALTER PRIMARY KEY operations. Informs #147514 Epic: none Release note: none Co-authored-by: DarrylWong <[email protected]> Co-authored-by: Matt Spilchen <[email protected]>
3 parents 47672d2 + ecfe6f2 + 7c3c4c8 commit bc50ec9

File tree

12 files changed

+384
-42
lines changed

12 files changed

+384
-42
lines changed

pkg/cmd/roachtest/tests/failure_injection.go

Lines changed: 147 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,8 @@ import (
99
"context"
1010
"fmt"
1111
"math/rand"
12+
"os"
13+
"regexp"
1214
"strings"
1315
"time"
1416

@@ -23,6 +25,7 @@ import (
2325
"github.com/cockroachdb/cockroach/pkg/roachprod/failureinjection/failures"
2426
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2527
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
28+
"github.com/cockroachdb/cockroach/pkg/testutils"
2629
"github.com/cockroachdb/cockroach/pkg/util/randutil"
2730
"github.com/cockroachdb/errors"
2831
)
@@ -52,6 +55,8 @@ type failureSmokeTest struct {
5255
validateRecover func(ctx context.Context, l *logger.Logger, c cluster.Cluster, f *failures.Failer) error
5356
// The workload to be run during the failureSmokeTest, if nil, defaultSmokeTestWorkload is used.
5457
workload func(ctx context.Context, c cluster.Cluster, args ...string) error
58+
// The duration to run the workload for before injecting the failure.
59+
workloadRamp time.Duration
5560
}
5661

5762
func (t *failureSmokeTest) run(
@@ -82,6 +87,15 @@ func (t *failureSmokeTest) run(
8287
return err
8388
}
8489

90+
if t.workloadRamp > 0 {
91+
l.Printf("sleeping for %s before injecting failure", t.workloadRamp)
92+
select {
93+
case <-ctx.Done():
94+
return ctx.Err()
95+
case <-time.After(t.workloadRamp):
96+
}
97+
}
98+
8599
quietLogger, file, err = roachtestutil.LoggerForCmd(l, c.CRDBNodes(), t.testName, "inject")
86100
if err != nil {
87101
return err
@@ -596,6 +610,122 @@ var dmsetupDiskStallTest = func(c cluster.Cluster) failureSmokeTest {
596610
}
597611
}
598612

613+
var processKillTests = func(c cluster.Cluster) []failureSmokeTest {
614+
rng, _ := randutil.NewPseudoRand()
615+
var tests []failureSmokeTest
616+
for _, gracefulShutdown := range []bool{true, false} {
617+
groups, _ := c.CRDBNodes().SeededRandGroups(rng, 2 /* numGroups */)
618+
killedNodeGroup := groups[0]
619+
unaffectedNodeGroup := groups[1]
620+
621+
// These are the nodes that we will run validation on.
622+
killedNode := killedNodeGroup.SeededRandNode(rng)
623+
unaffectedNode := unaffectedNodeGroup.SeededRandNode(rng)
624+
625+
tests = append(tests, failureSmokeTest{
626+
testName: fmt.Sprintf("%s/GracefulShutdown=%t", failures.ProcessKillFailureName, gracefulShutdown),
627+
failureName: failures.ProcessKillFailureName,
628+
args: failures.ProcessKillArgs{
629+
Nodes: killedNodeGroup.InstallNodes(),
630+
GracefulShutdown: gracefulShutdown,
631+
GracePeriod: time.Minute,
632+
},
633+
validateFailure: func(ctx context.Context, l *logger.Logger, c cluster.Cluster, f *failures.Failer) error {
634+
// If we initiate a graceful shutdown, the cockroach process should
635+
// intercept it and start draining the node.
636+
if gracefulShutdown {
637+
err := testutils.SucceedsSoonError(func() error {
638+
if ctx.Err() != nil {
639+
return ctx.Err()
640+
}
641+
res, err := c.RunWithDetailsSingleNode(ctx, l, option.WithNodes(unaffectedNode), fmt.Sprintf("./cockroach node status %d --decommission --certs-dir=%s | sed -n '2p' | awk '{print $NF}'", killedNode[0], install.CockroachNodeCertsDir))
642+
if err != nil {
643+
return err
644+
}
645+
isDraining := strings.TrimSpace(res.Stdout)
646+
if isDraining != "true" {
647+
return errors.Errorf("expected node %d to be draining", killedNode[0])
648+
}
649+
return nil
650+
})
651+
if err != nil {
652+
return err
653+
}
654+
}
655+
656+
// Check that we aren't able to establish a SQL connection to the killed node.
657+
// waitForFailureToPropagate already checks system death for us, which is a
658+
// stronger assertion than checking SQL connections are unavailable. We
659+
// are mostly doing this to satisfy the smoke test framework since this is
660+
// a fairly simple failure mode with less to validate.
661+
err := testutils.SucceedsSoonError(func() error {
662+
if ctx.Err() != nil {
663+
return ctx.Err()
664+
}
665+
666+
killedDB, err := c.ConnE(ctx, l, killedNode[0])
667+
if err == nil {
668+
defer killedDB.Close()
669+
if err := killedDB.Ping(); err == nil {
670+
return errors.Errorf("expected node %d to be dead, but it is alive", killedNode)
671+
} else {
672+
l.Printf("failed to connect to node %d: %v", killedNode, err)
673+
}
674+
} else {
675+
l.Printf("unable to establish SQL connection to node %d", killedNode)
676+
}
677+
return nil
678+
})
679+
680+
return err
681+
},
682+
// Similar to validateFailure, there is not much to validate here that isn't
683+
// covered by WaitForFailureToRecover, so just skip it.
684+
validateRecover: func(ctx context.Context, l *logger.Logger, c cluster.Cluster, f *failures.Failer) error {
685+
return nil
686+
},
687+
workload: func(ctx context.Context, c cluster.Cluster, args ...string) error {
688+
return defaultFailureSmokeTestWorkload(ctx, c, "--tolerate-errors")
689+
},
690+
// Shutting down the server right after it's started can cause draining to be skipped.
691+
workloadRamp: 30 * time.Second,
692+
})
693+
}
694+
695+
groups, _ := c.CRDBNodes().SeededRandGroups(rng, 2 /* numGroups */)
696+
killedNodeGroup := groups[0]
697+
// This is the node that we will run validation on.
698+
killedNode := killedNodeGroup.SeededRandNode(rng)
699+
noopSignal := 0
700+
701+
// Test that the GracePeriod logic will kick in if the SIGTERM hangs.
702+
tests = append(tests, failureSmokeTest{
703+
testName: fmt.Sprintf("%s/hanging-drain", failures.ProcessKillFailureName),
704+
failureName: failures.ProcessKillFailureName,
705+
args: failures.ProcessKillArgs{
706+
Nodes: killedNode.InstallNodes(),
707+
Signal: &noopSignal,
708+
GracePeriod: 30 * time.Second,
709+
},
710+
// There isn't anything to validate here because our failure is effectively
711+
// a noop at first. Only after the GracePeriod will we see anything happen.
712+
// We could block for 30 seconds and then check that the node is dead, but
713+
// this is the same thing WaitForFailureToPropagate does for us.
714+
validateFailure: func(ctx context.Context, l *logger.Logger, c cluster.Cluster, f *failures.Failer) error {
715+
return nil
716+
},
717+
validateRecover: func(ctx context.Context, l *logger.Logger, c cluster.Cluster, f *failures.Failer) error {
718+
return nil
719+
},
720+
workload: func(ctx context.Context, c cluster.Cluster, args ...string) error {
721+
return defaultFailureSmokeTestWorkload(ctx, c, "--tolerate-errors")
722+
},
723+
// Shutting down the server right after it's started can cause draining to be skipped.
724+
workloadRamp: 30 * time.Second,
725+
})
726+
return tests
727+
}
728+
599729
func defaultFailureSmokeTestWorkload(ctx context.Context, c cluster.Cluster, args ...string) error {
600730
workloadArgs := strings.Join(args, " ")
601731
cmd := roachtestutil.NewCommand("./cockroach workload run kv %s", workloadArgs).
@@ -644,6 +774,7 @@ func runFailureSmokeTest(ctx context.Context, t test.Test, c cluster.Cluster, no
644774
dmsetupDiskStallTest(c),
645775
}
646776
failureSmokeTests = append(failureSmokeTests, cgroupsDiskStallTests(c)...)
777+
failureSmokeTests = append(failureSmokeTests, processKillTests(c)...)
647778

648779
// Randomize the order of the tests in case any of the failures have unexpected side
649780
// effects that may mask failures, e.g. a cgroups disk stall isn't properly recovered
@@ -652,6 +783,22 @@ func runFailureSmokeTest(ctx context.Context, t test.Test, c cluster.Cluster, no
652783
failureSmokeTests[i], failureSmokeTests[j] = failureSmokeTests[j], failureSmokeTests[i]
653784
})
654785

786+
// For testing new failure modes, it may be useful to run only a subset of
787+
// tests to increase iteration speed.
788+
if regex := os.Getenv("FAILURE_INJECTION_SMOKE_TEST_FILTER"); regex != "" {
789+
filter, err := regexp.Compile(regex)
790+
if err != nil {
791+
t.Fatal(err)
792+
}
793+
var filteredTests []failureSmokeTest
794+
for _, test := range failureSmokeTests {
795+
if filter.MatchString(test.testName) {
796+
filteredTests = append(filteredTests, test)
797+
}
798+
}
799+
failureSmokeTests = filteredTests
800+
}
801+
655802
for _, test := range failureSmokeTests {
656803
t.L().Printf("\n=====running %s test=====", test.testName)
657804
if noopFailer {

pkg/roachprod/failureinjection/failures/BUILD.bazel

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@ go_library(
99
"latency.go",
1010
"network_partition.go",
1111
"noop.go",
12+
"process_kill.go",
1213
"registry.go",
1314
],
1415
importpath = "github.com/cockroachdb/cockroach/pkg/roachprod/failureinjection/failures",
@@ -17,10 +18,13 @@ go_library(
1718
"//pkg/roachprod",
1819
"//pkg/roachprod/install",
1920
"//pkg/roachprod/logger",
21+
"//pkg/roachprod/roachprodutil",
2022
"//pkg/util/retry",
2123
"//pkg/util/syncutil",
24+
"//pkg/util/sysutil",
2225
"//pkg/util/timeutil",
2326
"@com_github_cockroachdb_errors//:errors",
27+
"@org_golang_x_sys//unix",
2428
],
2529
)
2630

pkg/roachprod/failureinjection/failures/failure.go

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/roachprod"
1616
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
1717
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
18+
"github.com/cockroachdb/cockroach/pkg/roachprod/roachprodutil"
1819
"github.com/cockroachdb/cockroach/pkg/util/retry"
1920
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2021
"github.com/cockroachdb/errors"
@@ -217,6 +218,28 @@ func (f *GenericFailure) WaitForSQLUnavailable(
217218
return errors.Wrapf(err, "connections to node %d never unavailable after %s", node, timeout)
218219
}
219220

221+
// WaitForProcessDeath checks systemd until the cockroach process is no longer running
222+
// or the timeout is reached.
223+
func (f *GenericFailure) WaitForProcessDeath(
224+
ctx context.Context, l *logger.Logger, node install.Nodes, timeout time.Duration,
225+
) error {
226+
start := timeutil.Now()
227+
err := retryForDuration(ctx, timeout, func() error {
228+
res, err := f.RunWithDetails(ctx, l, node, "systemctl is-active cockroach-system.service")
229+
if err != nil {
230+
return err
231+
}
232+
status := strings.TrimSpace(res.Stdout)
233+
if status != "active" {
234+
l.Printf("n%d cockroach process exited after %s: %s", node, timeutil.Since(start), status)
235+
return nil
236+
}
237+
return errors.Newf("systemd reported n%d cockroach process as %s", node, status)
238+
})
239+
240+
return errors.Wrapf(err, "n%d process never exited after %s", node, timeout)
241+
}
242+
220243
func (f *GenericFailure) StopCluster(
221244
ctx context.Context, l *logger.Logger, stopOpts roachprod.StopOpts,
222245
) error {
@@ -261,3 +284,15 @@ func forEachNode(nodes install.Nodes, fn func(install.Nodes) error) error {
261284
}
262285
return nil
263286
}
287+
288+
func runAsync(ctx context.Context, l *logger.Logger, f func(context.Context) error) <-chan error {
289+
errCh := make(chan error, 1)
290+
go func() {
291+
err := roachprodutil.PanicAsError(ctx, l, func(context.Context) error {
292+
return f(ctx)
293+
})
294+
errCh <- err
295+
close(errCh)
296+
}()
297+
return errCh
298+
}

0 commit comments

Comments
 (0)