Skip to content

Commit

Permalink
storage/engine: Limit batch size in MVCC GC
Browse files Browse the repository at this point in the history
When there is a lot of garbage to collect, a single run of
MVCCGarbageCollect on the node liveness range can last long enough
that leases expire, leading to cascading unavailability.

Fixes cockroachdb#16565
  • Loading branch information
bdarnell committed Jun 21, 2017
1 parent cd22f2b commit 015e567
Show file tree
Hide file tree
Showing 5 changed files with 38 additions and 9 deletions.
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ diagnostics.reporting.report_metrics true b enable c
diagnostics.reporting.send_crash_reports true b send crash and panic reports
kv.allocator.lease_rebalancing_aggressiveness 1E+00 f set greater than 1.0 to rebalance leases toward load more aggressively, or between 0 and 1.0 to be more conservative about rebalancing leases
kv.allocator.load_based_lease_rebalancing.enabled true b set to enable rebalancing of range leases based on load and latency
kv.gc.batch_size 100000 i maximum number of keys in a batch for MVCC garbage collection
kv.raft.command.max_size 64 MiB z maximum size of a raft command
kv.raft_log.synchronize true b set to true to synchronize on Raft log writes to persistent storage
kv.snapshot_rebalance.max_rate 2.0 MiB z the rate limit (bytes/sec) to use for rebalance snapshots
Expand Down
4 changes: 3 additions & 1 deletion pkg/storage/engine/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -766,7 +766,9 @@ func BenchmarkMVCCGarbageCollect(b *testing.B) {
gcKeys := setup(engine, keySize, valSize, numKeys, numVersions, deleteVersions)

b.StartTimer()
if err := MVCCGarbageCollect(ctx, engine, &enginepb.MVCCStats{}, gcKeys, now); err != nil {
if err := MVCCGarbageCollect(
ctx, engine, &enginepb.MVCCStats{}, gcKeys, now, math.MaxInt64,
); err != nil {
b.Fatal(err)
}
}
Expand Down
20 changes: 16 additions & 4 deletions pkg/storage/engine/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -2104,23 +2104,27 @@ func MVCCResolveWriteIntentRangeUsingIter(
// MVCCGarbageCollect creates an iterator on the engine. In parallel
// it iterates through the keys listed for garbage collection by the
// keys slice. The engine iterator is seeked in turn to each listed
// key, clearing all values with timestamps <= to expiration.
// The timestamp parameter is used to compute the intent age on GC.
// key, clearing all values with timestamps <= to expiration. The
// timestamp parameter is used to compute the intent age on GC.
// Garbage collection stops after clearing maxClears values
// (to limit the size of the WriteBatch produced).
func MVCCGarbageCollect(
ctx context.Context,
engine ReadWriter,
ms *enginepb.MVCCStats,
keys []roachpb.GCRequest_GCKey,
timestamp hlc.Timestamp,
maxClears int64,
) error {
// We're allowed to use a prefix iterator because we always Seek() the
// iterator when handling a new user key.
iter := engine.NewIterator(true)
defer iter.Close()

var count int64
defer func(begin time.Time) {
log.Eventf(ctx, "done with GC evaluation for %d keys at %.2f keys/sec",
len(keys), float64(len(keys))*1E9/float64(timeutil.Since(begin)))
log.Eventf(ctx, "done with GC evaluation for %d keys at %.2f keys/sec. Deleted %d versions",
len(keys), float64(len(keys))*1E9/float64(timeutil.Since(begin)), count)
}(timeutil.Now())

// Iterate through specified GC keys.
Expand Down Expand Up @@ -2161,6 +2165,10 @@ func MVCCGarbageCollect(
if err := engine.Clear(iter.UnsafeKey()); err != nil {
return err
}
count++
if count >= maxClears {
return nil
}
}
}

Expand Down Expand Up @@ -2192,6 +2200,10 @@ func MVCCGarbageCollect(
if err := engine.Clear(unsafeIterKey); err != nil {
return err
}
count++
if count >= maxClears {
return nil
}
}
}
}
Expand Down
10 changes: 7 additions & 3 deletions pkg/storage/engine/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3704,7 +3704,9 @@ func TestMVCCGarbageCollect(t *testing.T) {
{Key: roachpb.Key("a-bad"), Timestamp: ts2},
{Key: roachpb.Key("inline-bad"), Timestamp: hlc.Timestamp{}},
}
if err := MVCCGarbageCollect(context.Background(), engine, ms, keys, ts3); err != nil {
if err := MVCCGarbageCollect(
context.Background(), engine, ms, keys, ts3, math.MaxInt64,
); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -3792,7 +3794,7 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) {
keys := []roachpb.GCRequest_GCKey{
{Key: test.key, Timestamp: ts2},
}
err := MVCCGarbageCollect(context.Background(), engine, nil, keys, ts2)
err := MVCCGarbageCollect(context.Background(), engine, nil, keys, ts2, math.MaxInt64)
if !testutils.IsError(err, test.expError) {
t.Fatalf("expected error %q when garbage collecting a non-deleted live value, found %v", test.expError, err)
}
Expand Down Expand Up @@ -3824,7 +3826,9 @@ func TestMVCCGarbageCollectIntent(t *testing.T) {
keys := []roachpb.GCRequest_GCKey{
{Key: key, Timestamp: ts2},
}
if err := MVCCGarbageCollect(context.Background(), engine, nil, keys, ts2); err == nil {
if err := MVCCGarbageCollect(
context.Background(), engine, nil, keys, ts2, math.MaxInt64,
); err == nil {
t.Fatal("expected error garbage collecting an intent")
}
}
Expand Down
12 changes: 11 additions & 1 deletion pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
Expand All @@ -62,6 +63,15 @@ const (
collectChecksumTimeout = 5 * time.Second
)

// gcBatchSize controls the amount of work done in a single pass of
// MVCC GC. Setting this too high may block the range for too long
// (especially a risk in the system ranges), while setting it too low
// may allow ranges to grow too large if we are unable to keep up with
// the amount of garbage generated.
var gcBatchSize = settings.RegisterIntSetting("kv.gc.batch_size",
"maximum number of keys in a batch for MVCC garbage collection",
100000)

// CommandArgs contains all the arguments to a command.
// TODO(bdarnell): consider merging with storagebase.FilterArgs (which
// would probably require removing the EvalCtx field due to import order
Expand Down Expand Up @@ -1423,7 +1433,7 @@ func evalGC(
}

// Garbage collect the specified keys by expiration timestamps.
err := engine.MVCCGarbageCollect(ctx, batch, cArgs.Stats, keys, h.Timestamp)
err := engine.MVCCGarbageCollect(ctx, batch, cArgs.Stats, keys, h.Timestamp, gcBatchSize.Get())
if err != nil {
return EvalResult{}, err
}
Expand Down

0 comments on commit 015e567

Please sign in to comment.