-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
cmd_clear_range.go
174 lines (157 loc) · 6.68 KB
/
cmd_clear_range.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
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
// Copyright 2017 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 batcheval
import (
"errors"
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/spanset"
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/kr/pretty"
)
// clearRangeBytesThreshold is the threshold over which the ClearRange
// command will use engine.ClearRange to efficiently perform a range
// deletion. Otherwise, will revert to iterating through the values
// and clearing them individually with engine.Clear.
const clearRangeBytesThreshold = 512 << 10 // 512KiB
func init() {
RegisterCommand(roachpb.ClearRange, declareKeysClearRange, ClearRange)
}
func declareKeysClearRange(
desc roachpb.RangeDescriptor, header roachpb.Header, req roachpb.Request, spans *spanset.SpanSet,
) {
DefaultDeclareKeys(desc, header, req, spans)
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
// We may look up the current range stats in order to efficiently
// negate them in the case of being able to clear the entire user-
// space span of keys in the range.
spans.Add(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeStatsKey(header.RangeID)})
// Add the GC threshold key, as this is updated as part of clear a
// range of data.
spans.Add(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLastGCKey(header.RangeID)})
}
// ClearRange wipes all MVCC versions of keys covered by the specified
// span, adjusting the MVCC stats accordingly.
//
// Note that "correct" use of this command is only possible for key
// spans consisting of user data that we know is not being written to
// or queried any more, such as after a DROP or TRUNCATE table.
func ClearRange(
ctx context.Context, batch engine.ReadWriter, cArgs CommandArgs, resp roachpb.Response,
) (result.Result, error) {
if cArgs.Header.Txn != nil {
return result.Result{}, errors.New("cannot execute ClearRange within a transaction")
}
log.VEventf(ctx, 2, "ClearRange %+v", cArgs.Args)
// Encode MVCCKey values for start and end of clear span.
args := cArgs.Args.(*roachpb.ClearRangeRequest)
from := engine.MVCCKey{Key: args.Key}
to := engine.MVCCKey{Key: args.EndKey}
// Before clearing, compute the delta in MVCCStats.
statsDelta, err := computeStatsDelta(ctx, batch, cArgs, from, to)
if err != nil {
return result.Result{}, err
}
cArgs.Stats.Subtract(statsDelta)
// Forward the range's GC threshold to the wall clock's now() in order
// to be newer than any previous write, and to disallow reads at earlier
// timestamps which will be invalid after deleting all existing keys
// in the span.
var pd result.Result
gcThreshold := cArgs.EvalCtx.GetGCThreshold()
gcThreshold.Forward(cArgs.EvalCtx.Clock().Now())
pd.Replicated.State = &storagebase.ReplicaState{
GCThreshold: &gcThreshold,
}
stateLoader := MakeStateLoader(cArgs.EvalCtx)
if err := stateLoader.SetGCThreshold(ctx, batch, cArgs.Stats, &gcThreshold); err != nil {
return result.Result{}, err
}
// If the total size of data to be cleared is less than
// clearRangeBytesThreshold, clear the individual values manually,
// instead of using a range tombstone (inefficient for small ranges).
if delta := statsDelta.KeyBytes + statsDelta.ValBytes; delta < clearRangeBytesThreshold {
log.VEventf(ctx, 2, "delta=%d < threshold=%d; using non-range clear", delta, clearRangeBytesThreshold)
return pd, batch.Iterate(
from, to,
func(kv engine.MVCCKeyValue) (bool, error) {
return false, batch.Clear(kv.Key)
},
)
}
// Otherwise, suggest a compaction for the cleared range and clear
// the key span using engine.ClearRange.
pd.Replicated.SuggestedCompactions = []storagebase.SuggestedCompaction{
{
StartKey: roachpb.RKey(from.Key),
EndKey: roachpb.RKey(to.Key),
Compaction: enginepb.Compaction{
Cleared: true,
Bytes: statsDelta.KeyBytes + statsDelta.ValBytes,
},
},
}
return pd, batch.ClearRange(from, to)
}
// computeStatsDelta determines the change in stats caused by the
// ClearRange command. If the cleared span is the entire range,
// computing MVCCStats is easy. We just negate all fields except sys
// bytes and count. Note that if a race build is enabled, we use
// the expectation of running in a CI environment to compute stats
// by iterating over the span to provide extra verification that the
// fast path of simply subtracting the non-system values is accurate.
func computeStatsDelta(
ctx context.Context, batch engine.ReadWriter, cArgs CommandArgs, from, to engine.MVCCKey,
) (enginepb.MVCCStats, error) {
desc := cArgs.EvalCtx.Desc()
var delta enginepb.MVCCStats
// We can avoid manually computing the stats delta if we're clearing
// the entire range.
fast := desc.StartKey.Equal(from.Key) && desc.EndKey.Equal(to.Key)
if fast {
// Note this it is valid to use the full range MVCC stats, as
// opposed to the usual method of computing only a localizied
// stats delta, because a full-range clear prevents any concurrent
// access to the stats.
delta = cArgs.EvalCtx.GetMVCCStats()
delta.SysCount, delta.SysBytes = 0, 0 // no change to system stats
}
// If we can't use the fast stats path, or race test is enabled,
// compute stats across the key span to be cleared.
if !fast || util.RaceEnabled {
iter := batch.NewIterator(false)
computed, err := iter.ComputeStats(from, to, delta.LastUpdateNanos)
iter.Close()
if err != nil {
return enginepb.MVCCStats{}, err
}
// If we took the fast path but race is enabled, assert stats were correctly computed.
if fast {
if !delta.Equal(computed) {
log.Fatalf(ctx, "fast-path MVCCStats copmutation gave wrong result: diff(fast, computed) = %s",
pretty.Diff(delta, computed))
}
}
delta = computed
}
return delta, nil
}