Skip to content

Commit

Permalink
storage: exit process when disks are stalled
Browse files Browse the repository at this point in the history
We've had two recent incidents in which we saw clusters with disks
stalled  on a subset of nodes in the cluster. This is a fairly
treacherous failure mode since

- the symptoms are nondescript: from the UI it often looks like a Raft
problem, logspy will freeze up as well, and so you waste some time until
you end up looking at the goroutine dump and notice the writes stuck in
syscall
- the node is in some semi-live state that borders the byzantine and
can cause further trouble for the part of the cluster that isn't
affected (we have some mitigations against this in place but not
enough, and need to improve our defense mechanisms).
- it's sudden and often can't be gleaned from the logs (since everything
is fine and then nothing ever completes so no "alertable" metrics are
emitted).

This commit introduces a simple mechanism that periodically checks for
these conditions (both on the engines and logging) and invokes a fatal
error if necessary.

The accompanying roachtest exercises both a data and a logging disk
stall.

Fixes #7882.
Fixes #32736.

Touches #7646.

Release note (bug fix): CockroachDB will error with a fatal exit when
data or logging partitions become unresponsive. Previously, the process
would remain running, though in an unresponsive state.
  • Loading branch information
tbg committed Dec 10, 2018
1 parent 8f30db0 commit 09f848a
Show file tree
Hide file tree
Showing 6 changed files with 226 additions and 15 deletions.
117 changes: 117 additions & 0 deletions pkg/cmd/roachtest/disk_stall.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package main

import (
"context"
"fmt"
"io/ioutil"
"math/rand"
"os"
"strings"
"time"
)

func registerDiskStalledDetection(r *registry) {
for _, affectsLogDir := range []bool{false, true} {
for _, affectsDataDir := range []bool{false, true} {
if !affectsLogDir && !affectsDataDir {
continue
}
// Grab copies of the args because we'll pass them into a closure.
// Everyone's favorite bug to write in Go.
affectsLogDir := affectsLogDir
affectsDataDir := affectsDataDir
r.Add(testSpec{
Name: fmt.Sprintf(
"disk-stalled/log=%t,data=%t",
affectsLogDir, affectsDataDir,
),
MinVersion: `v2.2.0`,
Nodes: nodes(1),
Run: func(ctx context.Context, t *test, c *cluster) {
runDiskStalledDetection(ctx, t, c, affectsLogDir, affectsDataDir)
},
})
}
}
}

func runDiskStalledDetection(
ctx context.Context, t *test, c *cluster, affectslogDir bool, affectsDataDir bool,
) {
n := c.Node(1)
tmpDir, err := ioutil.TempDir("", "stalled")
if err != nil {
t.Fatal(err)
}
defer func() {
_ = os.RemoveAll(tmpDir)
}()

c.Put(ctx, cockroach, "./cockroach")
c.Run(ctx, n, "sudo umount {store-dir}/faulty || true")
c.Run(ctx, n, "mkdir -p {store-dir}/{real,faulty} || true")
t.Status("setting up charybdefs")

if err := execCmd(ctx, t.l, roachprod, "install", c.makeNodes(n), "charybdefs"); err != nil {
t.Fatal(err)
}
c.Run(ctx, n, "sudo charybdefs {store-dir}/faulty -oallow_other,modules=subdir,subdir={store-dir}/real && chmod 777 {store-dir}/{real,faulty}")
l, err := t.l.ChildLogger("cockroach")
if err != nil {
t.Fatal(err)
}
type result struct {
err error
out string
}
errCh := make(chan result)

logDir := "real/logs"
if affectslogDir {
logDir = "faulty/logs"
}
dataDir := "real"
if affectsDataDir {
dataDir = "faulty"
}

go func() {
t.WorkerStatus("running server")

out, err := c.RunWithBuffer(ctx, l, n,
"timeout --signal 9 10m env COCKROACH_ENGINE_HEALTH_CHECK_INTERVAL=250ms "+
"./cockroach start --insecure --store {store-dir}/"+dataDir+" --log-dir {store-dir}/"+logDir,
)
errCh <- result{err, string(out)}
}()

time.Sleep(time.Duration(rand.Intn(5)) * time.Second)

t.Status("blocking storage")
c.Run(ctx, n, "charybdefs-nemesis --delay")

res := <-errCh
if res.err == nil {
t.Fatalf("expected an error: %s", res.out)
}
if !strings.Contains(res.out, "disk stall detected") {
t.Fatalf("unexpected output: %v %s", res.err, res.out)
}

c.Run(ctx, n, "charybdefs-nemesis --clear")
c.Run(ctx, n, "sudo umount {store-dir}/faulty")
}
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ func registerTests(r *registry) {
registerDecommission(r)
registerDiskUsage(r)
registerDiskFull(r)
registerDiskStalledDetection(r)
registerDrop(r)
registerElectionAfterRestart(r)
registerEncryption(r)
Expand Down
2 changes: 2 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1306,6 +1306,8 @@ func (s *Server) Start(ctx context.Context) error {
}
s.stopper.AddCloser(&s.engines)

startAssertEngineHealth(ctx, s.stopper, s.engines)

// Write listener info files early in the startup sequence. `listenerInfo` has a comment.
listenerFiles := listenerInfo{
advertise: s.cfg.AdvertiseAddr,
Expand Down
83 changes: 83 additions & 0 deletions pkg/server/server_engine_health.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package server

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

var engineHealthCheckInterval = envutil.EnvOrDefaultDuration("COCKROACH_ENGINE_HEALTH_CHECK_INTERVAL", 5*time.Second)

func startAssertEngineHealth(ctx context.Context, stopper *stop.Stopper, engines []engine.Engine) {
stopper.RunWorker(ctx, func(ctx context.Context) {
t := timeutil.NewTimer()
t.Reset(0)

for {
select {
case <-t.C:
t.Read = true
t.Reset(engineHealthCheckInterval)
assertLoggingHealth(ctx, engineHealthCheckInterval)
assertEngineHealth(ctx, engines, engineHealthCheckInterval)
case <-stopper.ShouldQuiesce():
return
}
}
})
}

// Don't change this string without also changing the disk-stall-detection
// roachtest which greps for it.
const diskStallDetected = "disk stall detected"

func assertEngineHealth(ctx context.Context, engines []engine.Engine, maxDuration time.Duration) {
for _, eng := range engines {
func() {
t := time.AfterFunc(maxDuration, func() {
log.Shout(ctx, log.Severity_FATAL, fmt.Sprintf(
"%s: unable to write to %s within %s",
diskStallDetected, eng, engineHealthCheckInterval,
))
})
defer t.Stop()
if err := engine.WriteSyncNoop(ctx, eng); err != nil {
log.Fatal(ctx, err)
}
}()
}
}

// assertLoggingHealth flushes all the log streams and fatally exits the process
// if this does not succeed within a reasonable amount of time.
func assertLoggingHealth(ctx context.Context, maxDuration time.Duration) {
timer := time.AfterFunc(maxDuration, func() {
log.Shout(ctx, log.Severity_FATAL, fmt.Sprintf(
"%s: unable to sync log files within %s",
diskStallDetected, engineHealthCheckInterval,
))
})
defer timer.Stop()

log.Flush()
}
19 changes: 16 additions & 3 deletions pkg/storage/engine/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,9 +235,7 @@ type Writer interface {
// Put sets the given key to the value provided.
Put(key MVCCKey, value []byte) error
// LogData adds the specified data to the RocksDB WAL. The data is
// uninterpreted by RocksDB (i.e. not added to the memtable or
// sstables). Currently only used for performance testing of appending to the
// RocksDB WAL.
// uninterpreted by RocksDB (i.e. not added to the memtable or sstables).
LogData(data []byte) error
// LogLogicalOp logs the specified logical mvcc operation with the provided
// details to the writer, if it has logical op logging enabled. For most
Expand Down Expand Up @@ -448,3 +446,18 @@ func Scan(engine Reader, start, end MVCCKey, max int64) ([]MVCCKeyValue, error)
})
return kvs, err
}

// WriteSyncNoop carries out a synchronous no-op write to the engine.
func WriteSyncNoop(ctx context.Context, eng Engine) error {
batch := eng.NewBatch()
defer batch.Close()

if err := batch.LogData(nil); err != nil {
return err
}

if err := batch.Commit(true /* sync */); err != nil {
return err
}
return nil
}
19 changes: 7 additions & 12 deletions pkg/storage/node_liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -768,18 +768,13 @@ func (nl *NodeLiveness) updateLiveness(
return err
}
for _, eng := range nl.engines {
// Synchronously writing to all disks before updating node liveness because
// we don't want any excessively slow disks to prevent the lease from
// shifting to other nodes. If the disk is slow, batch.Commit() will block.
batch := eng.NewBatch()
defer batch.Close()

if err := batch.LogData(nil); err != nil {
return errors.Wrapf(err, "couldn't update node liveness because LogData to disk fails")
}

if err := batch.Commit(true /* sync */); err != nil {
return errors.Wrapf(err, "couldn't update node liveness because Commit to disk fails")
// Synchronously write to all disks before updating node liveness.
// If this node's disk is stalled for whatever reason, we don't want
// it to be able to heartbeat its liveness record (which may not
// require participation of the local storage in order to succeed).
// Instead, leases should shift away from this node.
if err := engine.WriteSyncNoop(ctx, eng); err != nil {
errors.Wrapf(err, "unable to update node liveness")
}
}
if err := nl.updateLivenessAttempt(ctx, update, oldLiveness, handleCondFailed); err != nil {
Expand Down

0 comments on commit 09f848a

Please sign in to comment.