Skip to content

Commit

Permalink
kvserver: prioritize splitting ranges which are currently backpressuring
Browse files Browse the repository at this point in the history
This commit is a further mitigation to the decrease of range size. It is
important to split range which are too big. It is especially important to
split ranges which would block writes.

Release note: None
  • Loading branch information
ajwerner committed Apr 2, 2020
1 parent 0eb9035 commit 8f5b0d1
Show file tree
Hide file tree
Showing 4 changed files with 32 additions and 9 deletions.
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -245,7 +245,9 @@ func (mq *mergeQueue) process(
// in a situation where we keep merging ranges that would be split soon after
// by a small increase in load.
conservativeLoadBasedSplitThreshold := 0.5 * lhsRepl.SplitByLoadQPSThreshold()
if ok, _ := shouldSplitRange(mergedDesc, mergedStats, lhsRepl.GetMaxBytes(), sysCfg); ok || mergedQPS >= conservativeLoadBasedSplitThreshold {
shouldSplit, _ := shouldSplitRange(mergedDesc, mergedStats,
lhsRepl.GetMaxBytes(), lhsRepl.shouldBackpressureWrites(), sysCfg)
if shouldSplit || mergedQPS >= conservativeLoadBasedSplitThreshold {
log.VEventf(ctx, 2,
"skipping merge to avoid thrashing: merged range %s may split "+
"(estimated size, estimated QPS: %d, %v)",
Expand Down
4 changes: 1 addition & 3 deletions pkg/kv/kvserver/replica_backpressure.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,16 +50,14 @@ var backpressureRangeSizeMultiplier = settings.RegisterValidatedFloatSetting(
// is reduced by roughly exactly the multiplier then we'd potentially have
// lots of ranges in this state.
//
// We additionally mitigate this situation further by:
// We additionally mitigate this situation further by doing the following:
//
// 1) We store in-memory on each replica the largest zone configuration range
// size (largestPreviousMaxRangeBytes) we've seen and we do not backpressure
// if the current range size is less than that. That value is cleared when
// a range splits or runs GC such that the range size becomes smaller than
// the current max range size.
//
// TODO(ajwerner): We could mitigate this even further by:
//
// 2) We assign a higher priority in the snapshot queue to ranges which are
// currently backpressuring than ranges which are larger but are not
// applying backpressure.
Expand Down
24 changes: 22 additions & 2 deletions pkg/kv/kvserver/split_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,11 @@ func newSplitQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *splitQueue {
}

func shouldSplitRange(
desc *roachpb.RangeDescriptor, ms enginepb.MVCCStats, maxBytes int64, sysCfg *config.SystemConfig,
desc *roachpb.RangeDescriptor,
ms enginepb.MVCCStats,
maxBytes int64,
shouldBackpressureWrites bool,
sysCfg *config.SystemConfig,
) (shouldQ bool, priority float64) {
if sysCfg.NeedsSplit(desc.StartKey, desc.EndKey) {
// Set priority to 1 in the event the range is split by zone configs.
Expand All @@ -105,6 +109,22 @@ func shouldSplitRange(
shouldQ = true
}

// additionalPriorityDueToBackpressure is a mechanism to prioritize splitting
// ranges which will actively backpressuring writes.
//
// NB: This additional weight is totally arbitrary. The priority in the split
// queue is usually 1 plus the ratio of the current size over the max size.
// When a range is much larger than it is allowed to be given the
// backpressureRangeSizeMultiplier and the zone config, backpressure is
// not going to be applied because of the backpressureByteTolerance (see the
// comment there for more details). However, when the range size is close to
// the limit, we will backpressure. We strongly prefer to split over
// backpressure.
const addiitonalPriorityDueToBackpressure = 50
if shouldQ && shouldBackpressureWrites {
priority += addiitonalPriorityDueToBackpressure
}

return shouldQ, priority
}

Expand All @@ -116,7 +136,7 @@ func (sq *splitQueue) shouldQueue(
ctx context.Context, now hlc.Timestamp, repl *Replica, sysCfg *config.SystemConfig,
) (shouldQ bool, priority float64) {
shouldQ, priority = shouldSplitRange(repl.Desc(), repl.GetMVCCStats(),
repl.GetMaxBytes(), sysCfg)
repl.GetMaxBytes(), repl.shouldBackpressureWrites(), sysCfg)

if !shouldQ && repl.SplitByLoadEnabled() {
if splitKey := repl.loadBasedSplitter.MaybeSplitKey(timeutil.Now()); splitKey != nil {
Expand Down
9 changes: 6 additions & 3 deletions pkg/kv/kvserver/split_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,10 @@ func TestSplitQueueShouldQueue(t *testing.T) {
{roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64 << 20, 64 << 20, false, 0},
// No intersection, max bytes+1, no load.
{roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64<<20 + 1, 64 << 20, true, 1},
// No intersection, max bytes * 2, no load.
{roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64 << 21, 64 << 20, true, 2},
// No intersection, max bytes * 2 + 2, no load, should backpressure
{roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64<<21 + 2, 64 << 20, true, 52},
// No intersection, max bytes * 4, no load, should not backpressure
{roachpb.RKeyMin, roachpb.RKey(keys.MetaMax), 64 << 22, 64 << 20, true, 4},
// Intersection, max bytes +1, no load.
{keys.MakeTablePrefix(2000), roachpb.RKeyMax, 32<<20 + 1, 32 << 20, true, 2},
// Split needed at table boundary, but no zone config, no load.
Expand Down Expand Up @@ -92,7 +94,8 @@ func TestSplitQueueShouldQueue(t *testing.T) {
// Testing using shouldSplitRange instead of shouldQueue to avoid using the splitFinder
// This tests the merge queue behavior too as a result. For splitFinder tests,
// see split/split_test.go.
shouldQ, priority := shouldSplitRange(repl.Desc(), repl.GetMVCCStats(), repl.GetMaxBytes(), cfg)
shouldQ, priority := shouldSplitRange(repl.Desc(), repl.GetMVCCStats(),
repl.GetMaxBytes(), repl.ShouldBackpressureWrites(), cfg)
if shouldQ != test.shouldQ {
t.Errorf("%d: should queue expected %t; got %t", i, test.shouldQ, shouldQ)
}
Expand Down

0 comments on commit 8f5b0d1

Please sign in to comment.