@@ -9,12 +9,14 @@ import (
9
9
"context"
10
10
"time"
11
11
12
+ "github.com/cockroachdb/cockroach/pkg/kv"
12
13
"github.com/cockroachdb/cockroach/pkg/kv/bulk/bulkpb"
13
14
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
14
15
"github.com/cockroachdb/cockroach/pkg/roachpb"
15
16
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
16
17
"github.com/cockroachdb/cockroach/pkg/storage"
17
18
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
19
+ "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
18
20
"github.com/cockroachdb/cockroach/pkg/util/hlc"
19
21
"github.com/cockroachdb/cockroach/pkg/util/log"
20
22
"github.com/cockroachdb/cockroach/pkg/util/retry"
@@ -23,30 +25,71 @@ import (
23
25
"github.com/cockroachdb/errors"
24
26
)
25
27
28
+ type sstAdder struct {
29
+ settings * cluster.Settings
30
+ db * kv.DB
31
+
32
+ // disallowShadowingBelow is described on kvpb.AddSSTableRequest.
33
+ disallowShadowingBelow hlc.Timestamp
34
+
35
+ // priority is the admission priority used for AddSSTable
36
+ // requests.
37
+ priority admissionpb.WorkPriority
38
+
39
+ // writeAtBatchTS is true if the SST should be written at the
40
+ // batch timestamp. If this is set to true, then all the keys in
41
+ //the sst must have the same timestamp and they must be equal to
42
+ // the batch timestamp.
43
+ // TODO(jeffswenson): remove this from `sstAdder` in a stand alone PR.
44
+ writeAtBatchTS bool
45
+ }
46
+
47
+ func newSSTAdder (
48
+ db * kv.DB ,
49
+ settings * cluster.Settings ,
50
+ writeAtBatchTS bool ,
51
+ disallowShadowingBelow hlc.Timestamp ,
52
+ priority admissionpb.WorkPriority ,
53
+ ) * sstAdder {
54
+ return & sstAdder {
55
+ db : db ,
56
+ disallowShadowingBelow : disallowShadowingBelow ,
57
+ priority : priority ,
58
+ settings : settings ,
59
+ writeAtBatchTS : writeAtBatchTS ,
60
+ }
61
+ }
62
+
26
63
type sstSpan struct {
27
64
start , end roachpb.Key // [inclusive, exclusive)
28
65
sstBytes []byte
29
66
stats enginepb.MVCCStats
30
67
}
31
68
69
+ type addSSTResult struct {
70
+ timestamp hlc.Timestamp
71
+ availableBytes int64
72
+ followingLikelyNonEmptySpanStart roachpb.Key
73
+ rangeSpan roachpb.Span
74
+ }
75
+
32
76
// addSSTable retries db.AddSSTable if retryable errors occur, including if the
33
77
// SST spans a split, in which case it is iterated and split into two SSTs, one
34
78
// for each side of the split in the error, and each are retried.
35
- func (b * SSTBatcher ) addSSTable (
79
+ func (a * sstAdder ) AddSSTable (
36
80
ctx context.Context ,
37
81
batchTS hlc.Timestamp ,
38
82
start , end roachpb.Key ,
39
83
sstBytes []byte ,
40
84
stats enginepb.MVCCStats ,
41
- updatesLastRange bool ,
42
85
ingestionPerformanceStats * bulkpb.IngestionPerformanceStats ,
43
- ) error {
86
+ ) ([] addSSTResult , error ) {
44
87
ctx , sp := tracing .ChildSpan (ctx , "*SSTBatcher.addSSTable" )
45
88
defer sp .Finish ()
46
89
47
90
sendStart := timeutil .Now ()
48
91
if ingestionPerformanceStats == nil {
49
- return errors .AssertionFailedf ("ingestionPerformanceStats should not be nil" )
92
+ return nil , errors .AssertionFailedf ("ingestionPerformanceStats should not be nil" )
50
93
}
51
94
52
95
// Currently, the SSTBatcher cannot ingest range keys, so it is safe to
@@ -58,21 +101,25 @@ func (b *SSTBatcher) addSSTable(
58
101
}
59
102
iter , err := storage .NewMemSSTIterator (sstBytes , true , iterOpts )
60
103
if err != nil {
61
- return err
104
+ return nil , err
62
105
}
63
106
defer iter .Close ()
64
107
65
- if (stats == enginepb.MVCCStats {}) {
108
+ if stats == (enginepb.MVCCStats {}) {
109
+ // TODO(jeffswenson): Audit AddSST callers to see if they generate
110
+ // server side stats now. Accurately computing stats in the face of replays
111
+ // requires the server to do it.
66
112
iter .SeekGE (storage.MVCCKey {Key : start })
67
113
// NB: even though this ComputeStatsForIter call exhausts the iterator, we
68
114
// can reuse/re-seek on the iterator, as part of the MVCCIterator contract.
69
115
stats , err = storage .ComputeStatsForIter (iter , sendStart .UnixNano ())
70
116
if err != nil {
71
- return errors .Wrapf (err , "computing stats for SST [%s, %s)" , start , end )
117
+ return nil , errors .Wrapf (err , "computing stats for SST [%s, %s)" , start , end )
72
118
}
73
119
}
74
120
75
121
work := []* sstSpan {{start : start , end : end , sstBytes : sstBytes , stats : stats }}
122
+ var results []addSSTResult
76
123
var files int
77
124
for len (work ) > 0 {
78
125
item := work [0 ]
@@ -97,7 +144,7 @@ func (b *SSTBatcher) addSSTable(
97
144
// detection - making it is simpler to just always use the same API
98
145
// and just switch how it writes its result.
99
146
ingestAsWriteBatch := false
100
- if b .settings != nil && int64 (len (item .sstBytes )) < tooSmallSSTSize .Get (& b .settings .SV ) {
147
+ if a .settings != nil && int64 (len (item .sstBytes )) < tooSmallSSTSize .Get (& a .settings .SV ) {
101
148
log .VEventf (ctx , 3 , "ingest data is too small (%d keys/%d bytes) for SSTable, adding via regular batch" , item .stats .KeyCount , len (item .sstBytes ))
102
149
ingestAsWriteBatch = true
103
150
ingestionPerformanceStats .AsWrites ++
@@ -106,29 +153,30 @@ func (b *SSTBatcher) addSSTable(
106
153
req := & kvpb.AddSSTableRequest {
107
154
RequestHeader : kvpb.RequestHeader {Key : item .start , EndKey : item .end },
108
155
Data : item .sstBytes ,
109
- DisallowShadowingBelow : b .disallowShadowingBelow ,
156
+ DisallowShadowingBelow : a .disallowShadowingBelow ,
110
157
MVCCStats : & item .stats ,
111
158
IngestAsWrites : ingestAsWriteBatch ,
112
159
ReturnFollowingLikelyNonEmptySpanStart : true ,
113
160
}
114
- if b .writeAtBatchTS {
161
+ if a .writeAtBatchTS {
115
162
req .SSTTimestampToRequestTimestamp = batchTS
116
163
}
117
164
118
165
ba := & kvpb.BatchRequest {
119
166
Header : kvpb.Header {Timestamp : batchTS , ClientRangeInfo : roachpb.ClientRangeInfo {ExplicitlyRequested : true }},
120
167
AdmissionHeader : kvpb.AdmissionHeader {
121
- Priority : int32 (b .priority ),
168
+ Priority : int32 (a .priority ),
122
169
CreateTime : timeutil .Now ().UnixNano (),
123
170
Source : kvpb .AdmissionHeader_FROM_SQL ,
124
171
NoMemoryReservedAtSource : true ,
125
172
},
126
173
}
174
+
127
175
ba .Add (req )
128
176
beforeSend := timeutil .Now ()
129
177
130
178
sendCtx , sendSp := tracing .ChildSpan (ctx , "*SSTBatcher.addSSTable/Send" )
131
- br , pErr := b .db .NonTransactionalSender ().Send (sendCtx , ba )
179
+ br , pErr := a .db .NonTransactionalSender ().Send (sendCtx , ba )
132
180
sendSp .Finish ()
133
181
134
182
sendTime := timeutil .Since (beforeSend )
@@ -147,23 +195,12 @@ func (b *SSTBatcher) addSSTable(
147
195
148
196
if pErr == nil {
149
197
resp := br .Responses [0 ].GetInner ().(* kvpb.AddSSTableResponse )
150
- b .mu .Lock ()
151
- if b .writeAtBatchTS {
152
- b .mu .maxWriteTS .Forward (br .Timestamp )
153
- }
154
- b .mu .Unlock ()
155
- // If this was sent async then, by the time the reply gets back, it
156
- // might not be the last range anymore. We can just discard the last
157
- // range reply in this case though because async sends are only used
158
- // for SSTs sent due to range boundaries, i.e. when we are done with
159
- // with that range anyway.
160
- if updatesLastRange {
161
- b .lastRange .span = resp .RangeSpan
162
- if resp .RangeSpan .Valid () {
163
- b .lastRange .remaining = sz (resp .AvailableBytes )
164
- b .lastRange .nextExistingKey = resp .FollowingLikelyNonEmptySpanStart
165
- }
166
- }
198
+ results = append (results , addSSTResult {
199
+ timestamp : br .Timestamp ,
200
+ availableBytes : resp .AvailableBytes ,
201
+ followingLikelyNonEmptySpanStart : resp .FollowingLikelyNonEmptySpanStart ,
202
+ rangeSpan : resp .RangeSpan ,
203
+ })
167
204
files ++
168
205
log .VEventf (ctx , 3 , "adding %s AddSSTable [%s,%s) took %v" , sz (len (item .sstBytes )), item .start , item .end , sendTime )
169
206
return nil
@@ -185,7 +222,7 @@ func (b *SSTBatcher) addSSTable(
185
222
}
186
223
split := mr .Desc .EndKey .AsRawKey ()
187
224
log .Infof (ctx , "SSTable cannot be added spanning range bounds %v, retrying..." , split )
188
- left , right , err := createSplitSSTable (ctx , item .start , split , iter , b .settings )
225
+ left , right , err := createSplitSSTable (ctx , item .start , split , iter , a .settings )
189
226
if err != nil {
190
227
return err
191
228
}
@@ -199,7 +236,7 @@ func (b *SSTBatcher) addSSTable(
199
236
}
200
237
return err
201
238
}(); err != nil {
202
- return errors .Wrapf (err , "addsstable [%s,%s)" , item .start , item .end )
239
+ return nil , errors .Wrapf (err , "addsstable [%s,%s)" , item .start , item .end )
203
240
}
204
241
// explicitly deallocate SST. This will not deallocate the
205
242
// top level SST which is kept around to iterate over.
@@ -208,7 +245,7 @@ func (b *SSTBatcher) addSSTable(
208
245
ingestionPerformanceStats .SplitRetries += int64 (files - 1 )
209
246
210
247
log .VEventf (ctx , 3 , "AddSSTable [%v, %v) added %d files and took %v" , start , end , files , timeutil .Since (sendStart ))
211
- return nil
248
+ return results , nil
212
249
}
213
250
214
251
// createSplitSSTable is a helper for splitting up SSTs. The iterator
0 commit comments