-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathsplit_queue.go
155 lines (141 loc) · 5.07 KB
/
split_queue.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
// Copyright 2015 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.
//
// Author: Spencer Kimball ([email protected])
package storage
import (
"time"
"github.com/pkg/errors"
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
)
const (
// splitQueueMaxSize is the max size of the split queue.
splitQueueMaxSize = 100
// splitQueueTimerDuration is the duration between splits of queued ranges.
splitQueueTimerDuration = 0 // zero duration to process splits greedily.
)
// splitQueue manages a queue of ranges slated to be split due to size
// or along intersecting zone config boundaries.
type splitQueue struct {
*baseQueue
db *client.DB
}
// newSplitQueue returns a new instance of splitQueue.
func newSplitQueue(store *Store, db *client.DB, gossip *gossip.Gossip) *splitQueue {
sq := &splitQueue{
db: db,
}
sq.baseQueue = newBaseQueue(
"split", sq, store, gossip,
queueConfig{
maxSize: splitQueueMaxSize,
needsLease: true,
acceptsUnsplitRanges: true,
successes: store.metrics.SplitQueueSuccesses,
failures: store.metrics.SplitQueueFailures,
pending: store.metrics.SplitQueuePending,
processingNanos: store.metrics.SplitQueueProcessingNanos,
},
)
return sq
}
// shouldQueue determines whether a range should be queued for
// splitting. This is true if the range is intersected by a zone config
// prefix or if the range's size in bytes exceeds the limit for the zone.
func (sq *splitQueue) shouldQueue(
ctx context.Context, now hlc.Timestamp, repl *Replica, sysCfg config.SystemConfig,
) (shouldQ bool, priority float64) {
desc := repl.Desc()
if len(sysCfg.ComputeSplitKeys(desc.StartKey, desc.EndKey)) > 0 {
// Set priority to 1 in the event the range is split by zone configs.
priority = 1
shouldQ = true
}
// Add priority based on the size of range compared to the max
// size for the zone it's in.
zone, err := sysCfg.GetZoneConfigForKey(desc.StartKey)
if err != nil {
log.Error(ctx, err)
return
}
if ratio := float64(repl.GetMVCCStats().Total()) / float64(zone.RangeMaxBytes); ratio > 1 {
priority += ratio
shouldQ = true
}
return
}
// process synchronously invokes admin split for each proposed split key.
func (sq *splitQueue) process(
ctx context.Context, now hlc.Timestamp, r *Replica, sysCfg config.SystemConfig,
) error {
// First handle case of splitting due to zone config maps.
desc := r.Desc()
splitKeys := sysCfg.ComputeSplitKeys(desc.StartKey, desc.EndKey)
if len(splitKeys) > 0 {
log.Infof(ctx, "splitting at keys %v", splitKeys)
for _, splitKey := range splitKeys {
// Use db.AdminSplit() here because the zone config stipulates
// splitting at a specific key independent of which range it lies
// on (the specific replica desc passed in can change before the
// command is applied). If any range was already split at this
// key, it returns an error.
if err := sq.db.AdminSplit(ctx, splitKey.AsRawKey()); err != nil {
return errors.Errorf("unable to split %s at key %q: %s", r, splitKey, err)
}
}
return nil
}
// Next handle case of splitting due to size.
zone, err := sysCfg.GetZoneConfigForKey(desc.StartKey)
if err != nil {
return err
}
size := r.GetMVCCStats().Total()
if float64(size)/float64(zone.RangeMaxBytes) > 1 {
log.Infof(ctx, "splitting size=%d max=%d", size, zone.RangeMaxBytes)
// We cannot call db.AdminSplit() here because it can only be used to
// split the range holding a specific key without specifying the range
// itself. We are attempting to split a specific range somewhere along
// its midpoint without specifying a key and use Replica.AdminSplit()
// instead. An alternative is to compute the midpoint key here and
// pass it into AdminSplit().
//
// Note: another split can occur between the time the range size is
// computed above and the following request is processed.
if _, pErr := r.AdminSplit(
ctx,
roachpb.AdminSplitRequest{
Span: roachpb.Span{Key: desc.StartKey.AsRawKey()},
},
desc,
); pErr != nil {
return pErr.GoError()
}
}
return nil
}
// timer returns interval between processing successive queued splits.
func (*splitQueue) timer() time.Duration {
return splitQueueTimerDuration
}
// purgatoryChan returns nil.
func (*splitQueue) purgatoryChan() <-chan struct{} {
return nil
}