Line data Source code
1 : package sstable
2 :
3 : import (
4 : "bytes"
5 : "context"
6 : "math"
7 : "sync"
8 :
9 : "github.com/cockroachdb/errors"
10 : "github.com/cockroachdb/pebble/internal/base"
11 : "github.com/cockroachdb/pebble/internal/bytealloc"
12 : "github.com/cockroachdb/pebble/internal/invariants"
13 : "github.com/cockroachdb/pebble/internal/rangekey"
14 : "github.com/cockroachdb/pebble/objstorage"
15 : "github.com/cockroachdb/pebble/sstable/block"
16 : "github.com/cockroachdb/pebble/sstable/rowblk"
17 : )
18 :
19 : // RewriteKeySuffixesAndReturnFormat copies the content of the passed SSTable
20 : // bytes to a new sstable, written to `out`, in which the suffix `from` has is
21 : // replaced with `to` in every key. The input sstable must consist of only
22 : // Sets or RangeKeySets and every key must have `from` as its suffix as
23 : // determined by the Split function of the Comparer in the passed
24 : // WriterOptions. Range deletes must not exist in this sstable, as they will
25 : // be ignored.
26 : //
27 : // Data blocks are rewritten in parallel by `concurrency` workers and then
28 : // assembled into a final SST. Filters are copied from the original SST without
29 : // modification as they are not affected by the suffix, while block and table
30 : // properties are only minimally recomputed.
31 : //
32 : // TODO(sumeer): document limitations, if any, due to this limited
33 : // re-computation of properties (is there any loss of fidelity?).
34 : //
35 : // Any block property collectors configured in the WriterOptions must implement
36 : // AddCollectedWithSuffixChange.
37 : //
38 : // The WriterOptions.TableFormat is ignored, and the output sstable has the
39 : // same TableFormat as the input, which is returned in case the caller wants
40 : // to do some error checking. Suffix rewriting is meant to be efficient, and
41 : // allowing changes in the TableFormat detracts from that efficiency.
42 : //
43 : // Any obsolete bits that key-value pairs may be annotated with are ignored
44 : // and lost during the rewrite. Additionally, the output sstable has the
45 : // pebble.obsolete.is_strict property set to false. These limitations could be
46 : // removed if needed. The current use case for
47 : // RewriteKeySuffixesAndReturnFormat in CockroachDB is for MVCC-compliant file
48 : // ingestion, where these files do not contain RANGEDELs and have one
49 : // key-value pair per userkey -- so they trivially satisfy the strict
50 : // criteria, and we don't need the obsolete bit as a performance optimization.
51 : // For disaggregated storage, strict obsolete sstables are needed for L5 and
52 : // L6, but at the time of writing, we expect such MVCC-compliant file
53 : // ingestion to only ingest into levels L4 and higher. If this changes, we can
54 : // do one of two things to get rid of this limitation:
55 : // - Validate that there are no duplicate userkeys and no RANGEDELs/MERGEs
56 : // in the sstable to be rewritten. Validating no duplicate userkeys is
57 : // non-trivial when rewriting blocks in parallel, so we could encode the
58 : // pre-existing condition in the (existing) SnapshotPinnedKeys property --
59 : // we need to update the external sst writer to calculate and encode this
60 : // property.
61 : // - Preserve the obsolete bit (with changes to the blockIter).
62 : func RewriteKeySuffixesAndReturnFormat(
63 : sst []byte,
64 : rOpts ReaderOptions,
65 : out objstorage.Writable,
66 : o WriterOptions,
67 : from, to []byte,
68 : concurrency int,
69 0 : ) (*WriterMetadata, TableFormat, error) {
70 0 : r, err := NewMemReader(sst, rOpts)
71 0 : if err != nil {
72 0 : return nil, TableFormatUnspecified, err
73 0 : }
74 0 : defer r.Close()
75 0 : return rewriteKeySuffixesInBlocks(r, out, o, from, to, concurrency)
76 : }
77 :
78 : func rewriteKeySuffixesInBlocks(
79 : r *Reader, out objstorage.Writable, o WriterOptions, from, to []byte, concurrency int,
80 1 : ) (*WriterMetadata, TableFormat, error) {
81 1 : if o.Comparer == nil || o.Comparer.Split == nil {
82 1 : return nil, TableFormatUnspecified,
83 1 : errors.New("a valid splitter is required to rewrite suffixes")
84 1 : }
85 1 : if concurrency < 1 {
86 0 : return nil, TableFormatUnspecified, errors.New("concurrency must be >= 1")
87 0 : }
88 : // Even though NumValueBlocks = 0 => NumValuesInValueBlocks = 0, check both
89 : // as a defensive measure.
90 1 : if r.Properties.NumValueBlocks > 0 || r.Properties.NumValuesInValueBlocks > 0 {
91 0 : return nil, TableFormatUnspecified,
92 0 : errors.New("sstable with a single suffix should not have value blocks")
93 0 : }
94 :
95 1 : tableFormat := r.tableFormat
96 1 : o.TableFormat = tableFormat
97 1 : w := NewWriter(out, o)
98 1 : defer func() {
99 1 : if w != nil {
100 1 : w.Close()
101 1 : }
102 : }()
103 :
104 1 : for _, c := range w.blockPropCollectors {
105 1 : if !c.SupportsSuffixReplacement() {
106 0 : return nil, TableFormatUnspecified,
107 0 : errors.Errorf("block property collector %s does not support suffix replacement", c.Name())
108 0 : }
109 : }
110 :
111 1 : l, err := r.Layout()
112 1 : if err != nil {
113 0 : return nil, TableFormatUnspecified, errors.Wrap(err, "reading layout")
114 0 : }
115 :
116 1 : if err := rewriteDataBlocksToWriter(r, w, l.Data, from, to, w.split, concurrency); err != nil {
117 1 : return nil, TableFormatUnspecified, errors.Wrap(err, "rewriting data blocks")
118 1 : }
119 :
120 : // Copy over the range key block and replace suffixes in it if it exists.
121 1 : if err := rewriteRangeKeyBlockToWriter(r, w, from, to); err != nil {
122 0 : return nil, TableFormatUnspecified, errors.Wrap(err, "rewriting range key blocks")
123 0 : }
124 :
125 : // Copy over the filter block if it exists (rewriteDataBlocksToWriter will
126 : // already have ensured this is valid if it exists).
127 1 : if w.filter != nil && l.Filter.Length > 0 {
128 1 : filterBlock, _, err := readBlockBuf(r, l.Filter, nil)
129 1 : if err != nil {
130 0 : return nil, TableFormatUnspecified, errors.Wrap(err, "reading filter")
131 0 : }
132 1 : w.filter = copyFilterWriter{
133 1 : origPolicyName: w.filter.policyName(), origMetaName: w.filter.metaName(), data: filterBlock,
134 1 : }
135 : }
136 :
137 1 : if err := w.Close(); err != nil {
138 0 : w = nil
139 0 : return nil, TableFormatUnspecified, err
140 0 : }
141 1 : writerMeta, err := w.Metadata()
142 1 : w = nil
143 1 : return writerMeta, tableFormat, err
144 : }
145 :
146 : var errBadKind = errors.New("key does not have expected kind (set)")
147 :
148 : type blockWithSpan struct {
149 : start, end InternalKey
150 : data []byte
151 : }
152 :
153 : func rewriteBlocks(
154 : r *Reader,
155 : restartInterval int,
156 : checksumType block.ChecksumType,
157 : compression Compression,
158 : input []BlockHandleWithProperties,
159 : output []blockWithSpan,
160 : totalWorkers, worker int,
161 : from, to []byte,
162 : split Split,
163 1 : ) error {
164 1 : bw := rowblk.Writer{RestartInterval: restartInterval}
165 1 : buf := blockBuf{checksummer: block.Checksummer{Type: checksumType}}
166 1 : var blockAlloc bytealloc.A
167 1 : var keyAlloc bytealloc.A
168 1 : var scratch InternalKey
169 1 :
170 1 : var inputBlock, inputBlockBuf []byte
171 1 :
172 1 : iter := &rowblk.Iter{}
173 1 :
174 1 : // We'll assume all blocks are _roughly_ equal so round-robin static partition
175 1 : // of each worker doing every ith block is probably enough.
176 1 : for i := worker; i < len(input); i += totalWorkers {
177 1 : bh := input[i]
178 1 :
179 1 : var err error
180 1 : inputBlock, inputBlockBuf, err = readBlockBuf(r, bh.Handle, inputBlockBuf)
181 1 : if err != nil {
182 0 : return err
183 0 : }
184 1 : if err := iter.Init(r.Compare, r.Split, inputBlock, NoTransforms); err != nil {
185 0 : return err
186 0 : }
187 :
188 : /*
189 : TODO(jackson): Move rewriteBlocks into rowblk.
190 :
191 : if cap(bw.restarts) < int(iter.restarts) {
192 : bw.restarts = make([]uint32, 0, iter.restarts)
193 : }
194 : if cap(bw.buf) == 0 {
195 : bw.buf = make([]byte, 0, len(inputBlock))
196 : }
197 : if cap(bw.restarts) < int(iter.numRestarts) {
198 : bw.restarts = make([]uint32, 0, iter.numRestarts)
199 : }
200 : */
201 :
202 1 : for kv := iter.First(); kv != nil; kv = iter.Next() {
203 1 : if kv.Kind() != InternalKeyKindSet {
204 0 : return errBadKind
205 0 : }
206 1 : si := split(kv.K.UserKey)
207 1 : oldSuffix := kv.K.UserKey[si:]
208 1 : if !bytes.Equal(oldSuffix, from) {
209 1 : err := errors.Errorf("key has suffix %q, expected %q", oldSuffix, from)
210 1 : return err
211 1 : }
212 1 : newLen := si + len(to)
213 1 : if cap(scratch.UserKey) < newLen {
214 1 : scratch.UserKey = make([]byte, 0, len(kv.K.UserKey)*2+len(to)-len(from))
215 1 : }
216 :
217 1 : scratch.Trailer = kv.K.Trailer
218 1 : scratch.UserKey = scratch.UserKey[:newLen]
219 1 : copy(scratch.UserKey, kv.K.UserKey[:si])
220 1 : copy(scratch.UserKey[si:], to)
221 1 :
222 1 : // NB: for TableFormatPebblev3 and higher, since
223 1 : // !iter.lazyValueHandling.hasValuePrefix, it will return the raw value
224 1 : // in the block, which includes the 1-byte prefix. This is fine since bw
225 1 : // also does not know about the prefix and will preserve it in bw.add.
226 1 : v := kv.InPlaceValue()
227 1 : if invariants.Enabled && r.tableFormat >= TableFormatPebblev3 &&
228 1 : kv.Kind() == InternalKeyKindSet {
229 1 : if len(v) < 1 {
230 0 : return errors.Errorf("value has no prefix")
231 0 : }
232 1 : prefix := block.ValuePrefix(v[0])
233 1 : if prefix.IsValueHandle() {
234 0 : return errors.Errorf("value prefix is incorrect")
235 0 : }
236 1 : if prefix.SetHasSamePrefix() {
237 0 : return errors.Errorf("multiple keys with same key prefix")
238 0 : }
239 : }
240 1 : bw.Add(scratch, v)
241 1 : if output[i].start.UserKey == nil {
242 1 : keyAlloc, output[i].start = cloneKeyWithBuf(scratch, keyAlloc)
243 1 : }
244 : }
245 1 : *iter = iter.ResetForReuse()
246 1 :
247 1 : keyAlloc, output[i].end = cloneKeyWithBuf(scratch, keyAlloc)
248 1 :
249 1 : finished, trailer := compressAndChecksum(bw.Finish(), compression, &buf)
250 1 :
251 1 : // copy our finished block into the output buffer.
252 1 : blockAlloc, output[i].data = blockAlloc.Alloc(len(finished) + block.TrailerLen)
253 1 : copy(output[i].data, finished)
254 1 : copy(output[i].data[len(finished):], trailer[:])
255 : }
256 1 : return nil
257 : }
258 :
259 2 : func checkWriterFilterMatchesReader(r *Reader, w *Writer) error {
260 2 : if r.Properties.FilterPolicyName != w.filter.policyName() {
261 0 : return errors.New("mismatched filters")
262 0 : }
263 2 : if was, is := r.Properties.ComparerName, w.props.ComparerName; was != is {
264 1 : return errors.Errorf("mismatched Comparer %s vs %s, replacement requires same splitter to copy filters", was, is)
265 1 : }
266 2 : return nil
267 : }
268 :
269 : func rewriteDataBlocksToWriter(
270 : r *Reader,
271 : w *Writer,
272 : data []BlockHandleWithProperties,
273 : from, to []byte,
274 : split Split,
275 : concurrency int,
276 1 : ) error {
277 1 : if r.Properties.NumEntries == 0 {
278 1 : // No point keys.
279 1 : return nil
280 1 : }
281 1 : blocks := make([]blockWithSpan, len(data))
282 1 :
283 1 : if w.filter != nil {
284 1 : if err := checkWriterFilterMatchesReader(r, w); err != nil {
285 1 : return err
286 1 : }
287 : }
288 :
289 1 : g := &sync.WaitGroup{}
290 1 : g.Add(concurrency)
291 1 : errCh := make(chan error, concurrency)
292 1 : for i := 0; i < concurrency; i++ {
293 1 : worker := i
294 1 : go func() {
295 1 : defer g.Done()
296 1 : err := rewriteBlocks(
297 1 : r,
298 1 : w.dataBlockBuf.dataBlock.RestartInterval,
299 1 : w.blockBuf.checksummer.Type,
300 1 : w.compression,
301 1 : data,
302 1 : blocks,
303 1 : concurrency,
304 1 : worker,
305 1 : from, to,
306 1 : split,
307 1 : )
308 1 : if err != nil {
309 1 : errCh <- err
310 1 : }
311 : }()
312 : }
313 1 : g.Wait()
314 1 : close(errCh)
315 1 : if err, ok := <-errCh; ok {
316 1 : return err
317 1 : }
318 :
319 : // oldShortIDs maps the shortID for the block property collector in the old
320 : // blocks to the shortID in the new blocks. Initialized once for the sstable.
321 1 : var oldShortIDs []shortID
322 1 : // oldProps is the property value in an old block, indexed by the shortID in
323 1 : // the new block. Slice is reused for each block.
324 1 : var oldProps [][]byte
325 1 : if len(w.blockPropCollectors) > 0 {
326 1 : oldProps = make([][]byte, len(w.blockPropCollectors))
327 1 : oldShortIDs = make([]shortID, math.MaxUint8)
328 1 : for i := range oldShortIDs {
329 1 : oldShortIDs[i] = invalidShortID
330 1 : }
331 1 : for i, p := range w.blockPropCollectors {
332 1 : if prop, ok := r.Properties.UserProperties[p.Name()]; ok {
333 1 : was, is := shortID(prop[0]), shortID(i)
334 1 : oldShortIDs[was] = is
335 1 : } else {
336 0 : return errors.Errorf("sstable does not contain property %s", p.Name())
337 0 : }
338 : }
339 : }
340 :
341 1 : for i := range blocks {
342 1 : // Write the rewritten block to the file.
343 1 : if err := w.writable.Write(blocks[i].data); err != nil {
344 0 : return err
345 0 : }
346 :
347 1 : n := len(blocks[i].data)
348 1 : bh := block.Handle{Offset: w.meta.Size, Length: uint64(n) - block.TrailerLen}
349 1 : // Update the overall size.
350 1 : w.meta.Size += uint64(n)
351 1 :
352 1 : // Load any previous values for our prop collectors into oldProps.
353 1 : for i := range oldProps {
354 1 : oldProps[i] = nil
355 1 : }
356 1 : decoder := makeBlockPropertiesDecoder(len(oldProps), data[i].Props)
357 1 : for !decoder.Done() {
358 1 : id, val, err := decoder.Next()
359 1 : if err != nil {
360 0 : return err
361 0 : }
362 1 : if oldShortIDs[id].IsValid() {
363 1 : oldProps[oldShortIDs[id]] = val
364 1 : }
365 : }
366 :
367 1 : for i, p := range w.blockPropCollectors {
368 1 : if err := p.AddCollectedWithSuffixReplacement(oldProps[i], from, to); err != nil {
369 0 : return err
370 0 : }
371 : }
372 :
373 1 : bhp, err := w.maybeAddBlockPropertiesToBlockHandle(bh)
374 1 : if err != nil {
375 0 : return err
376 0 : }
377 1 : var nextKey InternalKey
378 1 : if i+1 < len(blocks) {
379 1 : nextKey = blocks[i+1].start
380 1 : }
381 1 : if err = w.addIndexEntrySync(blocks[i].end, nextKey, bhp, w.dataBlockBuf.tmp[:]); err != nil {
382 0 : return err
383 0 : }
384 : }
385 :
386 1 : w.meta.updateSeqNum(blocks[0].start.SeqNum())
387 1 : w.props.NumEntries = r.Properties.NumEntries
388 1 : w.props.RawKeySize = r.Properties.RawKeySize
389 1 : w.props.RawValueSize = r.Properties.RawValueSize
390 1 : w.meta.SetSmallestPointKey(blocks[0].start)
391 1 : w.meta.SetLargestPointKey(blocks[len(blocks)-1].end)
392 1 : return nil
393 : }
394 :
395 1 : func rewriteRangeKeyBlockToWriter(r *Reader, w *Writer, from, to []byte) error {
396 1 : iter, err := r.NewRawRangeKeyIter(NoTransforms)
397 1 : if err != nil {
398 0 : return err
399 0 : }
400 1 : if iter == nil {
401 1 : // No range keys.
402 1 : return nil
403 1 : }
404 1 : defer iter.Close()
405 1 :
406 1 : s, err := iter.First()
407 1 : for ; s != nil; s, err = iter.Next() {
408 1 : if !s.Valid() {
409 0 : break
410 : }
411 1 : for i := range s.Keys {
412 1 : if s.Keys[i].Kind() != base.InternalKeyKindRangeKeySet {
413 0 : return errBadKind
414 0 : }
415 1 : if !bytes.Equal(s.Keys[i].Suffix, from) {
416 0 : return errors.Errorf("key has suffix %q, expected %q", s.Keys[i].Suffix, from)
417 0 : }
418 1 : s.Keys[i].Suffix = to
419 : }
420 :
421 1 : err = rangekey.Encode(s, func(k base.InternalKey, v []byte) error {
422 1 : // Calling AddRangeKey instead of addRangeKeySpan bypasses the fragmenter.
423 1 : // This is okay because the raw fragments off of `iter` are already
424 1 : // fragmented, and suffix replacement should not affect fragmentation.
425 1 : return w.AddRangeKey(k, v)
426 1 : })
427 1 : if err != nil {
428 0 : return err
429 0 : }
430 : }
431 1 : return err
432 : }
433 :
434 : type copyFilterWriter struct {
435 : origMetaName string
436 : origPolicyName string
437 : data []byte
438 : }
439 :
440 0 : func (copyFilterWriter) addKey(key []byte) { panic("unimplemented") }
441 2 : func (c copyFilterWriter) finish() ([]byte, error) { return c.data, nil }
442 2 : func (c copyFilterWriter) metaName() string { return c.origMetaName }
443 2 : func (c copyFilterWriter) policyName() string { return c.origPolicyName }
444 :
445 : // RewriteKeySuffixesViaWriter is similar to RewriteKeySuffixes but uses just a
446 : // single loop over the Reader that writes each key to the Writer with the new
447 : // suffix. The is significantly slower than the parallelized rewriter, and does
448 : // more work to rederive filters, props, etc.
449 : //
450 : // Any obsolete bits that key-value pairs may be annotated with are ignored
451 : // and lost during the rewrite. Some of the obsolete bits may be recreated --
452 : // specifically when there are multiple keys with the same user key.
453 : // Additionally, the output sstable has the pebble.obsolete.is_strict property
454 : // set to false. See the longer comment at RewriteKeySuffixesAndReturnFormat.
455 : func RewriteKeySuffixesViaWriter(
456 : r *Reader, out objstorage.Writable, o WriterOptions, from, to []byte,
457 1 : ) (*WriterMetadata, error) {
458 1 : if o.Comparer == nil || o.Comparer.Split == nil {
459 0 : return nil, errors.New("a valid splitter is required to rewrite suffixes")
460 0 : }
461 :
462 1 : o.IsStrictObsolete = false
463 1 : w := NewWriter(out, o)
464 1 : defer func() {
465 1 : if w != nil {
466 0 : w.Close()
467 0 : }
468 : }()
469 1 : i, err := r.NewIter(NoTransforms, nil, nil)
470 1 : if err != nil {
471 0 : return nil, err
472 0 : }
473 1 : defer i.Close()
474 1 :
475 1 : kv := i.First()
476 1 : var scratch InternalKey
477 1 : for kv != nil {
478 1 : if kv.Kind() != InternalKeyKindSet {
479 0 : return nil, errors.New("invalid key type")
480 0 : }
481 1 : oldSuffix := kv.K.UserKey[r.Split(kv.K.UserKey):]
482 1 : if !bytes.Equal(oldSuffix, from) {
483 0 : return nil, errors.Errorf("key has suffix %q, expected %q", oldSuffix, from)
484 0 : }
485 1 : scratch.UserKey = append(scratch.UserKey[:0], kv.K.UserKey[:len(kv.K.UserKey)-len(from)]...)
486 1 : scratch.UserKey = append(scratch.UserKey, to...)
487 1 : scratch.Trailer = kv.K.Trailer
488 1 :
489 1 : val, _, err := kv.Value(nil)
490 1 : if err != nil {
491 0 : return nil, err
492 0 : }
493 1 : if w.addPoint(scratch, val, false); err != nil {
494 0 : return nil, err
495 0 : }
496 1 : kv = i.Next()
497 : }
498 1 : if err := rewriteRangeKeyBlockToWriter(r, w, from, to); err != nil {
499 0 : return nil, err
500 0 : }
501 1 : if err := w.Close(); err != nil {
502 0 : w = nil
503 0 : return nil, err
504 0 : }
505 1 : writerMeta, err := w.Metadata()
506 1 : w = nil
507 1 : return writerMeta, err
508 : }
509 :
510 : // NewMemReader opens a reader over the SST stored in the passed []byte.
511 1 : func NewMemReader(sst []byte, o ReaderOptions) (*Reader, error) {
512 1 : return NewReader(newMemReader(sst), o)
513 1 : }
514 :
515 1 : func readBlockBuf(r *Reader, bh block.Handle, buf []byte) ([]byte, []byte, error) {
516 1 : raw := r.readable.(*memReader).b[bh.Offset : bh.Offset+bh.Length+block.TrailerLen]
517 1 : if err := checkChecksum(r.checksumType, raw, bh, 0); err != nil {
518 0 : return nil, buf, err
519 0 : }
520 1 : typ := blockType(raw[bh.Length])
521 1 : raw = raw[:bh.Length]
522 1 : if typ == noCompressionBlockType {
523 1 : return raw, buf, nil
524 1 : }
525 1 : decompressedLen, prefix, err := decompressedLen(typ, raw)
526 1 : if err != nil {
527 0 : return nil, buf, err
528 0 : }
529 1 : if cap(buf) < decompressedLen {
530 1 : buf = make([]byte, decompressedLen)
531 1 : }
532 1 : dst := buf[:decompressedLen]
533 1 : err = decompressInto(typ, raw[prefix:], dst)
534 1 : return dst, buf, err
535 : }
536 :
537 : // memReader is a thin wrapper around a []byte such that it can be passed to
538 : // sstable.Reader. It supports concurrent use, and does so without locking in
539 : // contrast to the heavier read/write vfs.MemFile.
540 : type memReader struct {
541 : b []byte
542 : r *bytes.Reader
543 : rh objstorage.NoopReadHandle
544 : }
545 :
546 : var _ objstorage.Readable = (*memReader)(nil)
547 :
548 1 : func newMemReader(b []byte) *memReader {
549 1 : r := &memReader{
550 1 : b: b,
551 1 : r: bytes.NewReader(b),
552 1 : }
553 1 : r.rh = objstorage.MakeNoopReadHandle(r)
554 1 : return r
555 1 : }
556 :
557 : // ReadAt is part of objstorage.Readable.
558 1 : func (m *memReader) ReadAt(_ context.Context, p []byte, off int64) error {
559 1 : n, err := m.r.ReadAt(p, off)
560 1 : if invariants.Enabled && err == nil && n != len(p) {
561 0 : panic("short read")
562 : }
563 1 : return err
564 : }
565 :
566 : // Close is part of objstorage.Readable.
567 1 : func (*memReader) Close() error {
568 1 : return nil
569 1 : }
570 :
571 : // Stat is part of objstorage.Readable.
572 1 : func (m *memReader) Size() int64 {
573 1 : return int64(len(m.b))
574 1 : }
575 :
576 : // NewReadHandle is part of objstorage.Readable.
577 : func (m *memReader) NewReadHandle(
578 : ctx context.Context, readBeforeSize objstorage.ReadBeforeSize,
579 1 : ) objstorage.ReadHandle {
580 1 : return &m.rh
581 1 : }
|