-
Notifications
You must be signed in to change notification settings - Fork 3.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
failure-injection: add disk stall failure modes #143104
base: master
Are you sure you want to change the base?
Conversation
Your pull request contains more than 1000 changes. It is strongly encouraged to split big PRs into smaller chunks. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
6696122
to
af95c8d
Compare
// 22.04+ addition, as older distributions and the upstream cgroup implementation do not | ||
// have this check. | ||
// | ||
// This additional check appears to protect against the io hanging when allowing bursts |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Mystery solved although I can't say that was worth the time spent reading the cgroups source code 🤣 At least we can now say roachprod documents cgroups better than cgroups does.
0e9ad9c
to
d9375bd
Compare
return err | ||
} | ||
|
||
// When we unmounted the disk in setup, the cgroups controllers may have been removed, re-add them. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This feels iffy to me to have to manually add back the controllers, but no matter how hard I tried I couldn't get them to be added back short of restarting the roachprod cluster.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I tried I couldn't get them to be added back short of restarting the roachprod cluster.
By "restarting", you mean VMs (not systemd)? It does look error-prone. What if we miss something; what's the chance that the cluster remains fully reusable (by independent roachtest)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
By "restarting", you mean VMs
Yeah, neither restarting the daemon or restarting the cockroach service (although I don't think this would do anything since we restart the node anyway) would re-add the controllers. Looking at the roachprod start script, we don't touch cgroups so no idea what the difference would be.
// If true, allow the failure mode to restart nodes as needed. E.g. dmsetup requires | ||
// the cockroach process to not be running to properly setup. If RestartNodes is true, | ||
// then the failure mode will restart the cluster for the user. | ||
RestartNodes bool |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The current disk stall roachtests are okay with the node dying since they don't try to do anything else in the test after the fact. However, I still think we should restart the node at the end so artifacts can be collected - you can see in all of the current roachtests there's no cockroach logs for the stalled node.
var err error | ||
|
||
// Disabling journaling requires the cockroach process to not have been started yet. | ||
if diskStallArgs.RestartNodes { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note to self: having to restart the cluster to setup/cleanup dmsetup seems really disruptive - makes me lean towards using cgroups in the failure injection framework
1b98045
to
172479a
Compare
var tests []failureSmokeTest | ||
for _, stallWrites := range []bool{true, false} { | ||
for _, stallReads := range []bool{true, false} { | ||
if !stallWrites && !stallReads { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if it might be still useful as a smoke test to verify that the no-op versions of validateFailure
and validateRestore
actually work as expected.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, do you mean check that they should fail? If so then failure-injection/smoke-test/noop
should cover that, but lemme know if i'm misunderstanding.
25c1f6e
to
917453a
Compare
return errors.CombineErrors(err, res[0].Err) | ||
} | ||
|
||
func (f *GenericFailure) WaitForSQLLiveness( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: sqlliveness is for session management [1], this is more like "is sql ready". I also wonder if we could merge it with WaitForSQLReady
? (The "pinging" logic could be passed as a func
, but the retry logic is reused.)
[1] https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20200615_sql_liveness.md
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 Changed the name to WaitForSqlReady
. I also added a retryForDuration
helper that should extract the retry logic out and be reusable.
I also wonder if we could merge it with WaitForSQLReady?
Do you mean the roachtestutil
WaitForSQLReady
? Not aware of something similar on the roachprod layer.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yep. We can refactor on subsequent passes.
e960d18
to
dcd297b
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice work, left a few comments. Also curious if the cgroup test would work with the dmsetup staller as both seem to wait for and recover from node death?
if err = failureMode.Restore(ctx, quietLogger, t.args); err != nil { | ||
l.Printf("%s: Running Recover(); details in %s.log", t.failureName, file) | ||
if err = c.AddGrafanaAnnotation(ctx, l, grafana.AddAnnotationRequest{ | ||
Text: fmt.Sprintf("%s recovred", t.testName), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Small typo: "recovred" -> "recovered"
@@ -70,7 +83,8 @@ func (f *GenericFailure) Run( | |||
l.Printf("Local cluster detected, logging command instead of running:\n%s", cmd) | |||
return nil | |||
} | |||
return f.c.Run(ctx, l, l.Stdout, l.Stderr, install.WithNodes(node), f.runTitle, cmd) | |||
l.Printf(cmd) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Was this for debugging? If it's meant to stay in, could add something to the print to indicate it's a command.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah it was intentional, before I left it up to the caller if they wanted to log the command, but I found I always wanted to log the command.
could add something to the print to indicate it's a command.
Done, also reworded the local cluster log to make it less confusing.
// something went wrong in Recover. | ||
err := s.setThroughput(ctx, l, stallType, throughput{limited: false}, nodes, cockroachIOController) | ||
if err != nil { | ||
l.PrintfCtx(ctx, "error unstalling the disk; stumbling on: %v", err) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Curiosity question: When do we consider Cleanup to return an error, or just warn as is happening here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The user may or may not have successfully called Restore
so Cleanup
just best effort retries in case. I added this because a disk stall is so disruptive, it seemed like a bad idea to just leave the cluster in a bad state.
Maybe this needs to be clarified in the failure injection library API contract, i.e. if Cleanup
can be called without calling Recover
first. I think it makes sense in the context of how users would want to use Cleanup
, e.g. something went wrong and we just want to restore everything in one shot.
type DiskStallArgs struct { | ||
StallLogs bool | ||
StallReads bool | ||
StallWrites bool |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we allow setting more than one stall type, is there a scenario where we would want the throughput to be different for the read vs. the write stall?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was thinking if you wanted to set two different outputs you could just inject two different failures. I originally had a misleading comment that said concurrent disk stall failures weren't allowed - I removed that.
// While this is exactly what we want, it's not the intended use case and is invalid. | ||
bytesPerSecond := 4 | ||
if diskStallArgs.Throughput > 0 { | ||
bytesPerSecond = diskStallArgs.Throughput |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does this still not allow the FI framework to accidentally set it to between (>0, <4)? Might be worthwhile to error out earlier with a more descriptive error than from the cli call.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good call, I also changed the default to be 2. From what I can tell there's no reason to not set it to 2, I was just porting over the existing behavior.
return nil | ||
} | ||
|
||
func getStallType(diskStallArgs DiskStallArgs) ([]bandwidthType, error) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: Feels like this should be plural getStallTypes
, and in usage references stallTypes, err = getStallTypes(...)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
}) | ||
} | ||
|
||
func (s *CGroupDiskStaller) WaitForFailureToPropagate( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A general question on how the disk staller will be used. Since we have the throughput
setting I'm assuming it will always be set to a value that will cause the node to die?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm assuming it will always be set to a value will cause the node to die?
Good question, I don't think that's true so maybe always waiting for the node to die is too strong. The registerDiskBandwidthOverload
for example, doesn't want the node to die it just slows writes to 128 MiB to test AC.
I have two trains of though for fixing this. The first is that we can just add a ExpectNodeDeath
flag. The second is that we don't need to do anything, tests that don't expect the node to die can just skip calling WaitForFailureToPropagate
.
My thinking is that WaitForFailureToPropagate
would be used mostly a suggestion anyway (i.e. log a warning but don't fail the test). Waiting for a disk stall to be detected (when the throughput isn't 0) or waiting for ranges to rebalance is workload dependent/not bounded anyway. e.g. if we slow writes to 5 MiB, it's not obvious if the node will die or not, so wait up to N minutes and log a warning if it doesn't.
Unfortunately the cgroups test works by parsing Switching to |
fc0225d
to
a497ab1
Compare
This change adds cgroup and dmsetup disk stalls to the failure injection library. A majority of this logic is a port of the existing disk stall implementations found in roachtestutil, however several additions were added to make cleanup and restore of said failures restore the system back to it's original state.
…just one This will exercise the failure injection library's ability to inject failures on multiple nodes at once. To support this, a SeededRandGroups helper was added to NodeListOption.
While the two names have similar meanings in the context of failure injection, we often use restore to refer to restoring a backup. Lets remove any ambiguity by renaming it to recover.
} | ||
|
||
func (f *GenericFailure) Run( | ||
ctx context.Context, l *logger.Logger, node install.Nodes, args ...string, | ||
) error { | ||
cmd := strings.Join(args, " ") | ||
l.Printf("running cmd: %s", cmd) | ||
// In general, most failures shouldn't be run locally out of caution. | ||
if f.c.IsLocal() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe this should be enforced as a precondition, i.e., panic instead of logging?
return errors.Wrapf(err, "never connected to node %d after %s", node, timeout) | ||
} | ||
|
||
func (f *GenericFailure) WaitForSQLNodeDeath( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: WaitForSQLUnavailable
. "Death" implies the process is down, which PingNode
can't establish; so, the current name is somewhat misleading.
start := timeutil.Now() | ||
err := retryForDuration(ctx, timeout, func() error { | ||
if err := f.PingNode(ctx, l, node); err != nil { | ||
l.Printf("Connections to node %d dropped after %s", node, timeutil.Since(start)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: "dropped" -> "unavailable" for the same reason, as above. We can't really tell if it's unavailable because of dropped network packets or some other reason.
return &CGroupDiskStaller{GenericFailure: genericFailure}, nil | ||
} | ||
|
||
const CgroupsDiskStallName = "cgroup-disk-stall" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: move above the func
s.t. all var/const/type declarations precede the definitions.
return CgroupsDiskStallName | ||
} | ||
|
||
type DiskStallArgs struct { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: move above the func
s s.t. all var/const/type declarations precede the definitions.
if err := s.Run(ctx, l, s.c.Nodes, "mkdir -p {store-dir}/logs"); err != nil { | ||
return err | ||
} | ||
if err := s.Run(ctx, l, s.c.Nodes, "rm -f logs && ln -s {store-dir}/logs logs || true"); err != nil { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This would end up wiping node logs, every time this failure is injected, for the duration of some arbitrary test.
if diskStallArgs.StallWrites { | ||
stallTypes = []bandwidthType{writeBandwidth} | ||
} | ||
if diskStallArgs.StallReads { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we further validate if StallLogs
is set without StallWrites
? In that case, we don't expect the process to panic.
`sudo dmsetup create data1`); err != nil { | ||
return err | ||
} | ||
// This has occasionally been seen to fail with "Device or resource busy", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: missing TODO?
// snapd will run "snapd auto-import /dev/dm-0" via udev triggers when | ||
// /dev/dm-0 is created. This possibly interferes with the dmsetup create | ||
// reload, so uninstall snapd. | ||
if err = s.Run(ctx, l, s.c.Nodes, `sudo apt-get purge -y snapd`); err != nil { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This makes me wonder if we should disable cluster reuse for anything that touches system dependencies. apt
tends to flake occasionally and adds a source of non-determinism.
return nil | ||
} | ||
|
||
func (s *DmsetupDiskStaller) WaitForFailureToPropagate( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Both of these helpers seem generic; could be moved to GenericFailure
.
stalledNode := stalledNodeGroup.SeededRandNode(rng) | ||
unaffectedNode := unaffectedNodeGroup.SeededRandNode(rng) | ||
|
||
ableToCreateFile := func(ctx context.Context, l *logger.Logger, c cluster.Cluster, node option.NodeListOption) bool { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: ableToCreateFile
-> touchFile
. Since we can't tell exactly why this func failed, e.g., ssh timeout, we should use a "weaker", i.e., less precise, name. Otherwise, the name could mislead the author/reader into thinking that the false
result is caused by something else entirely, e.g., file permissions.
} | ||
return nil | ||
}, | ||
workload: func(ctx context.Context, c cluster.Cluster) error { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same workload is duplicated above, albeit with different CLI options.
if err := test.run(ctx, t.L(), c, fr); err != nil { | ||
t.Fatal(errors.Wrapf(err, "%s failed", test.testName)) | ||
} | ||
cancel() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For clean shutdown, we want this inside defer
since Fatal
escapes.
// Check the number of bytes read and written to disk. | ||
res, err := s.RunWithDetails( | ||
ctx, l, node, | ||
fmt.Sprintf(`grep -E '%d:%d' /sys/fs/cgroup/system.slice/io.stat |`, maj, min), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems there are some lightweight metrics collectors, e.g., [1], which we could easily add as a scrape target. This could make it easier to debug; rather than sampling over discrete intervals ourselves, we'd have a more or less continuous view in Grafana.
[1] https://github.com/arianvp/cgroup-exporter/tree/main/collector
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice work! There are a number of things we could improve in subsequent PR(s). It might be instructive to document some of them by way of GH issues. Otherwise, feel free to merge after a final sanity pass. 🚢
This change adds cgroup and dmsetup disk stalls to the failure injection library. A majority of this logic is a port of the existing disk stall implementations found in roachtestutil, however several additions were added to make cleanup and restore of said failures restore the system back to it's original state.
Informs: #138970
Release note: none