forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathexternal_sst_reader.go
363 lines (325 loc) · 9.85 KB
/
external_sst_reader.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
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
// Copyright 2021 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package storageccl
import (
"context"
"io"
"os"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/sstable"
)
// RemoteSSTs lets external SSTables get iterated directly in some cases,
// rather than being downloaded entirely first.
var remoteSSTs = settings.RegisterBoolSetting(
settings.TenantWritable,
"kv.bulk_ingest.stream_external_ssts.enabled",
"if enabled, external SSTables are iterated directly in some cases, rather than being downloaded entirely first",
true,
)
var remoteSSTSuffixCacheSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"kv.bulk_ingest.stream_external_ssts.suffix_cache_size",
"size of suffix of remote SSTs to download and cache before reading from remote stream",
64<<10,
)
func getFileWithRetry(
ctx context.Context, basename string, e cloud.ExternalStorage,
) (ioctx.ReadCloserCtx, int64, error) {
// Do an initial read of the file, from the beginning, to get the file size as
// this is used e.g. to read the trailer.
var f ioctx.ReadCloserCtx
var sz int64
const maxAttempts = 3
if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), maxAttempts, func() error {
var err error
f, sz, err = e.ReadFileAt(ctx, basename, 0)
return err
}); err != nil {
return nil, 0, err
}
return f, sz, nil
}
// newMemPebbleSSTReader returns a PebbleSSTIterator for in-memory SSTs from
// external storage, optionally decrypting with the supplied parameters.
//
// ctx is captured and used throughout the life of the returned iterator, until
// the iterator's Close() method is called.
func newMemPebbleSSTReader(
ctx context.Context,
e []cloud.ExternalStorage,
basenames []string,
encryption *roachpb.FileEncryptionOptions,
iterOps storage.IterOptions,
) (storage.SimpleMVCCIterator, error) {
inMemorySSTs := make([][]byte, 0, len(basenames))
for i, basename := range basenames {
f, _, err := getFileWithRetry(ctx, basename, e[i])
if err != nil {
return nil, err
}
content, err := ioctx.ReadAll(ctx, f)
f.Close(ctx)
if err != nil {
return nil, err
}
if encryption != nil {
content, err = DecryptFile(ctx, content, encryption.Key, nil /* mm */)
if err != nil {
return nil, err
}
}
inMemorySSTs = append(inMemorySSTs, content)
}
return storage.NewPebbleMultiMemSSTIterator(inMemorySSTs, false, iterOps)
}
// ExternalSSTReader returns a PebbleSSTIterator for the SSTs in external storage,
// optionally decrypting with the supplied parameters.
//
// ctx is captured and used throughout the life of the returned iterator, until
// the iterator's Close() method is called.
func ExternalSSTReader(
ctx context.Context,
e []cloud.ExternalStorage,
basenames []string,
encryption *roachpb.FileEncryptionOptions,
iterOps storage.IterOptions,
) (storage.SimpleMVCCIterator, error) {
if !remoteSSTs.Get(&e[0].Settings().SV) {
return newMemPebbleSSTReader(ctx, e, basenames, encryption, iterOps)
}
remoteCacheSize := remoteSSTSuffixCacheSize.Get(&e[0].Settings().SV)
readers := make([]sstable.ReadableFile, 0, len(basenames))
for i, basename := range basenames {
f, sz, err := getFileWithRetry(ctx, basename, e[i])
if err != nil {
return nil, err
}
raw := &sstReader{
ctx: ctx,
sz: sizeStat(sz),
body: f,
openAt: func(offset int64) (ioctx.ReadCloserCtx, error) {
reader, _, err := e[i].ReadFileAt(ctx, basename, offset)
return reader, err
},
}
var reader sstable.ReadableFile
if encryption != nil {
r, err := decryptingReader(raw, encryption.Key)
if err != nil {
f.Close(ctx)
return nil, err
}
reader = r
} else {
// We only explicitly buffer the suffix of the file when not decrypting as
// the decrypting reader has its own internal block buffer.
if err := raw.readAndCacheSuffix(remoteCacheSize); err != nil {
f.Close(ctx)
return nil, err
}
reader = raw
}
readers = append(readers, reader)
}
return storage.NewPebbleSSTIterator(readers, iterOps)
}
// DeprecatingExternalSSTReader returns opens an SST in external storage, optionally
// decrypting with the supplied parameters, and returns iterator over it.
//
// ctx is captured and used throughout the life of the returned iterator, until
// the iterator's Close() method is called.
//
// TODO (msbutler): replace all current calls with new ExternalSSTReader,
// as it does not handle range keys
func DeprecatingExternalSSTReader(
ctx context.Context,
e cloud.ExternalStorage,
basename string,
encryption *roachpb.FileEncryptionOptions,
) (storage.SimpleMVCCIterator, error) {
// Do an initial read of the file, from the beginning, to get the file size as
// this is used e.g. to read the trailer.
var f ioctx.ReadCloserCtx
var sz int64
const maxAttempts = 3
if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), maxAttempts, func() error {
var err error
f, sz, err = e.ReadFileAt(ctx, basename, 0)
return err
}); err != nil {
return nil, err
}
if !remoteSSTs.Get(&e.Settings().SV) {
content, err := ioctx.ReadAll(ctx, f)
f.Close(ctx)
if err != nil {
return nil, err
}
if encryption != nil {
content, err = DecryptFile(ctx, content, encryption.Key, nil /* mm */)
if err != nil {
return nil, err
}
}
return storage.NewMemSSTIterator(content, false)
}
raw := &sstReader{
ctx: ctx,
sz: sizeStat(sz),
body: f,
openAt: func(offset int64) (ioctx.ReadCloserCtx, error) {
reader, _, err := e.ReadFileAt(ctx, basename, offset)
return reader, err
},
}
var reader sstable.ReadableFile = raw
if encryption != nil {
r, err := decryptingReader(raw, encryption.Key)
if err != nil {
f.Close(ctx)
return nil, err
}
reader = r
} else {
// We only explicitly buffer the suffix of the file when not decrypting as
// the decrypting reader has its own internal block buffer.
if err := raw.readAndCacheSuffix(remoteSSTSuffixCacheSize.Get(&e.Settings().SV)); err != nil {
f.Close(ctx)
return nil, err
}
}
iter, err := storage.NewSSTIterator(reader)
if err != nil {
reader.Close()
return nil, err
}
return iter, nil
}
type sstReader struct {
// ctx is captured at construction time and used for I/O operations.
ctx context.Context
sz sizeStat
openAt func(int64) (ioctx.ReadCloserCtx, error)
// body and pos are mutated by calls to ReadAt and Close.
body ioctx.ReadCloserCtx
pos int64
readPos int64 // readPos is used to transform Read() to ReadAt(readPos).
// This wrapper's primary purpose is reading SSTs which often perform many
// tiny reads in a cluster of offsets near the end of the file. If we can read
// the whole region once and fullfil those from a cache, we can avoid repeated
// RPCs.
cache struct {
pos int64
buf []byte
}
}
func (r *sstReader) reset() error {
r.pos = 0
var err error
if r.body != nil {
err = r.body.Close(r.ctx)
r.body = nil
}
return err
}
// Close implements io.Closer.
func (r *sstReader) Close() error {
err := r.reset()
r.ctx = nil
return err
}
// Stat returns the size of the file.
func (r *sstReader) Stat() (os.FileInfo, error) {
return r.sz, nil
}
func (r *sstReader) Read(p []byte) (int, error) {
n, err := r.ReadAt(p, r.readPos)
r.readPos += int64(n)
return n, err
}
// readAndCacheSuffix caches the `size` suffix of the file (which could the
// whole file) for use by later ReadAt calls to avoid making additional RPCs.
func (r *sstReader) readAndCacheSuffix(size int64) error {
if size == 0 {
return nil
}
r.cache.buf = nil
r.cache.pos = int64(r.sz) - size
if r.cache.pos <= 0 {
r.cache.pos = 0
}
reader, err := r.openAt(r.cache.pos)
if err != nil {
return err
}
defer reader.Close(r.ctx)
read, err := ioctx.ReadAll(r.ctx, reader)
if err != nil {
return err
}
r.cache.buf = read
return nil
}
// ReadAt implements io.ReaderAt by opening a Reader at an offset before reading
// from it. Note: contrary to io.ReaderAt, ReadAt does *not* support parallel
// calls.
func (r *sstReader) ReadAt(p []byte, offset int64) (int, error) {
var read int
if offset >= r.cache.pos && offset < r.cache.pos+int64(len(r.cache.buf)) {
read += copy(p, r.cache.buf[offset-r.cache.pos:])
if read == len(p) {
return read, nil
}
// Advance offset to end of what cache read.
offset += int64(read)
}
if offset == int64(r.sz) {
return read, io.EOF
}
// Position the underlying reader at offset if needed.
if r.pos != offset {
if err := r.reset(); err != nil {
return 0, err
}
b, err := r.openAt(offset)
if err != nil {
return 0, err
}
r.pos = offset
r.body = b
}
var err error
for n := 0; read < len(p); n, err = r.body.Read(r.ctx, p[read:]) {
read += n
if err != nil {
break
}
}
r.pos += int64(read)
// If we got an EOF after we had read enough, ignore it.
if read == len(p) && err == io.EOF {
return read, nil
}
return read, err
}
type sizeStat int64
func (s sizeStat) Size() int64 { return int64(s) }
func (sizeStat) IsDir() bool { panic(errors.AssertionFailedf("unimplemented")) }
func (sizeStat) ModTime() time.Time { panic(errors.AssertionFailedf("unimplemented")) }
func (sizeStat) Mode() os.FileMode { panic(errors.AssertionFailedf("unimplemented")) }
func (sizeStat) Name() string { panic(errors.AssertionFailedf("unimplemented")) }
func (sizeStat) Sys() interface{} { panic(errors.AssertionFailedf("unimplemented")) }