/
datasource.go
721 lines (656 loc) · 21.1 KB
/
datasource.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
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You 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 exec
import (
"bytes"
"context"
"fmt"
"io"
"math"
"sort"
"sync"
"time"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/sdf"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/ioutilx"
"github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
)
// DataSource is a Root execution unit.
type DataSource struct {
UID UnitID
SID StreamID
Name string
Coder *coder.Coder
Out Node
PCol PCollection // Handles size metrics. Value instead of pointer so it's initialized by default in tests.
source DataManager
state StateReader
index int64
splitIdx int64
start time.Time
// su is non-nil if this DataSource feeds directly to a splittable unit,
// and receives that splittable unit when it is available for splitting.
// While the splittable unit is received, it is blocked from processing
// new elements, so it must be sent back through the channel once the
// DataSource is finished using it.
su chan SplittableUnit
mu sync.Mutex
// Whether the downstream transform only iterates a GBK coder once.
singleIterate bool
}
// InitSplittable initializes the SplittableUnit channel from the output unit,
// if it provides one.
func (n *DataSource) InitSplittable() {
if n.Out == nil {
return
}
if u, ok := n.Out.(*ProcessSizedElementsAndRestrictions); ok {
n.su = u.SU
}
}
// ID returns the UnitID for this node.
func (n *DataSource) ID() UnitID {
return n.UID
}
// Up initializes this datasource.
func (n *DataSource) Up(ctx context.Context) error {
// TODO(https://github.com/apache/beam/issues/23043) - Reenable single iteration or more fully rip this out.
safeToSingleIterate := false
switch n.Out.(type) {
case *Expand, *Multiplex:
// CoGBK Expands aren't safe, as they may re-iterate the GBK stream.
// Multiplexes aren't safe, since they re-iterate the GBK stream by default.
safeToSingleIterate = false
}
n.singleIterate = safeToSingleIterate
return nil
}
// StartBundle initializes this datasource for the bundle.
func (n *DataSource) StartBundle(ctx context.Context, id string, data DataContext) error {
n.mu.Lock()
n.source = data.Data
n.state = data.State
n.start = time.Now()
n.index = -1
n.splitIdx = math.MaxInt64
n.mu.Unlock()
return n.Out.StartBundle(ctx, id, data)
}
// ByteCountReader is a passthrough reader that counts all the bytes read through it.
// It trusts the nested reader to return accurate byte information.
type byteCountReader struct {
count *int
reader io.ReadCloser
}
func (r *byteCountReader) Read(p []byte) (int, error) {
n, err := r.reader.Read(p)
*r.count += n
return n, err
}
func (r *byteCountReader) Close() error {
return r.reader.Close()
}
func (r *byteCountReader) reset() int {
c := *r.count
*r.count = 0
return c
}
// Process opens the data source, reads and decodes data, kicking off element processing.
func (n *DataSource) Process(ctx context.Context) error {
r, err := n.source.OpenRead(ctx, n.SID)
if err != nil {
return err
}
defer r.Close()
n.PCol.resetSize() // initialize the size distribution for this bundle.
var byteCount int
bcr := byteCountReader{reader: r, count: &byteCount}
c := coder.SkipW(n.Coder)
wc := MakeWindowDecoder(n.Coder.Window)
var cp ElementDecoder // Decoder for the primary element or the key in CoGBKs.
var cvs []ElementDecoder // Decoders for each value stream in CoGBKs.
switch {
case coder.IsCoGBK(c):
cp = MakeElementDecoder(c.Components[0])
// TODO(https://github.com/apache/beam/issues/18032): Support multiple value streams (coder components) with
// with CoGBK.
cvs = []ElementDecoder{MakeElementDecoder(c.Components[1])}
default:
cp = MakeElementDecoder(c)
}
for {
if n.incrementIndexAndCheckSplit() {
return nil
}
// TODO(lostluck) 2020/02/22: Should we include window headers or just count the element sizes?
ws, t, pn, err := DecodeWindowedValueHeader(wc, r)
if err != nil {
if err == io.EOF {
return nil
}
return errors.Wrap(err, "source failed")
}
// Decode key or parallel element.
pe, err := cp.Decode(&bcr)
if err != nil {
return errors.Wrap(err, "source decode failed")
}
pe.Timestamp = t
pe.Windows = ws
pe.Pane = pn
var valReStreams []ReStream
for _, cv := range cvs {
values, err := n.makeReStream(ctx, cv, &bcr, len(cvs) == 1 && n.singleIterate)
if err != nil {
return err
}
valReStreams = append(valReStreams, values)
}
if err := n.Out.ProcessElement(ctx, pe, valReStreams...); err != nil {
return err
}
// Collect the actual size of the element, and reset the bytecounter reader.
n.PCol.addSize(int64(bcr.reset()))
bcr.reader = r
}
}
func (n *DataSource) makeReStream(ctx context.Context, cv ElementDecoder, bcr *byteCountReader, onlyStream bool) (ReStream, error) {
// TODO(lostluck) 2020/02/22: Do we include the chunk size, or just the element sizes?
size, err := coder.DecodeInt32(bcr.reader)
if err != nil {
return nil, errors.Wrap(err, "stream size decoding failed")
}
if onlyStream {
// If we know the stream won't be re-iterated,
// decode elements on demand instead to reduce memory usage.
switch {
case size >= 0:
return &singleUseReStream{
r: bcr,
d: cv,
size: int(size),
}, nil
case size == -1:
return &singleUseMultiChunkReStream{
r: bcr,
d: cv,
open: func(bcr *byteCountReader) (Stream, error) {
tokenLen, err := coder.DecodeVarInt(bcr.reader)
if err != nil {
return nil, err
}
token, err := ioutilx.ReadN(bcr.reader, (int)(tokenLen))
if err != nil {
return nil, err
}
r, err := n.state.OpenIterable(ctx, n.SID, token)
if err != nil {
return nil, err
}
// We can't re-use the original bcr, since we may get new iterables,
// but we can re-use the count itself.
r = &byteCountReader{reader: r, count: bcr.count}
return &elementStream{r: r, ec: cv}, nil
},
}, nil
}
}
switch {
case size >= 0:
// Single chunk streams are fully read in and buffered in memory.
buf := make([]FullValue, 0, size)
buf, err = readStreamToBuffer(cv, bcr, int64(size), buf)
if err != nil {
return nil, err
}
return &FixedReStream{Buf: buf}, nil
case size == -1:
// Multi-chunked stream.
var buf []FullValue
for {
chunk, err := coder.DecodeVarInt(bcr.reader)
if err != nil {
return nil, errors.Wrap(err, "stream chunk size decoding failed")
}
// All done, escape out.
switch {
case chunk == 0: // End of stream, return buffer.
return &FixedReStream{Buf: buf}, nil
case chunk > 0: // Non-zero chunk, read that many elements from the stream, and buffer them.
chunkBuf := make([]FullValue, 0, chunk)
chunkBuf, err = readStreamToBuffer(cv, bcr, chunk, chunkBuf)
if err != nil {
return nil, err
}
buf = append(buf, chunkBuf...)
case chunk == -1: // State backed iterable!
chunk, err := coder.DecodeVarInt(bcr.reader)
if err != nil {
return nil, err
}
token, err := ioutilx.ReadN(bcr.reader, (int)(chunk))
if err != nil {
return nil, err
}
return &concatReStream{
first: &FixedReStream{Buf: buf},
next: &proxyReStream{
open: func() (Stream, error) {
r, err := n.state.OpenIterable(ctx, n.SID, token)
if err != nil {
return nil, err
}
// We can't re-use the original bcr, since we may get new iterables,
// or multiple of them at the same time, but we can re-use the count itself.
r = &byteCountReader{reader: r, count: bcr.count}
return &elementStream{r: r, ec: cv}, nil
},
},
}, nil
default:
return nil, errors.Errorf("multi-chunk stream with invalid chunk size of %d", chunk)
}
}
default:
return nil, errors.Errorf("received stream with marker size of %d", size)
}
}
func readStreamToBuffer(cv ElementDecoder, r io.ReadCloser, size int64, buf []FullValue) ([]FullValue, error) {
for i := int64(0); i < size; i++ {
value, err := cv.Decode(r)
if err != nil {
return nil, errors.Wrap(err, "stream value decode failed")
}
buf = append(buf, *value)
}
return buf, nil
}
// FinishBundle resets the source.
func (n *DataSource) FinishBundle(ctx context.Context) error {
n.mu.Lock()
defer n.mu.Unlock()
n.source = nil
n.splitIdx = 0 // Ensure errors are returned for split requests if this plan is re-used.
return n.Out.FinishBundle(ctx)
}
// Down resets the source.
func (n *DataSource) Down(ctx context.Context) error {
n.source = nil
return nil
}
func (n *DataSource) String() string {
return fmt.Sprintf("DataSource[%v, %v] Coder:%v Out:%v", n.SID, n.Name, n.Coder, n.Out.ID())
}
// incrementIndexAndCheckSplit increments DataSource.index by one and checks if
// the caller should abort further element processing, and finish the bundle.
// Returns true if the new value of index is greater than or equal to the split
// index, and false otherwise.
func (n *DataSource) incrementIndexAndCheckSplit() bool {
b := false
n.mu.Lock()
n.index++
if n.index >= n.splitIdx {
b = true
}
n.mu.Unlock()
return b
}
// ProgressReportSnapshot captures the progress reading an input source.
type ProgressReportSnapshot struct {
ID, Name string
Count int64
pcol PCollectionSnapshot
}
// Progress returns a snapshot of the source's progress.
func (n *DataSource) Progress() ProgressReportSnapshot {
if n == nil {
return ProgressReportSnapshot{}
}
n.mu.Lock()
pcol := n.PCol.snapshot()
// The count is the number of "completely processed elements"
// which matches the index of the currently processing element.
c := n.index
n.mu.Unlock()
// Do not sent negative progress reports, index is initialized to 0.
if c < 0 {
c = 0
}
pcol.ElementCount = c
return ProgressReportSnapshot{ID: n.SID.PtransformID, Name: n.Name, Count: c, pcol: pcol}
}
// getProcessContinuation retrieves a ProcessContinuation that may be returned by
// a self-checkpointing SDF. Current support for self-checkpointing requires that the
// SDF is immediately after the DataSource.
func (n *DataSource) getProcessContinuation() sdf.ProcessContinuation {
if u, ok := n.Out.(*ProcessSizedElementsAndRestrictions); ok {
return u.continuation
}
return nil
}
func (n *DataSource) makeEncodeElms() func([]*FullValue) ([][]byte, error) {
wc := MakeWindowEncoder(n.Coder.Window)
ec := MakeElementEncoder(coder.SkipW(n.Coder))
encodeElms := func(fvs []*FullValue) ([][]byte, error) {
encElms := make([][]byte, len(fvs))
for i, fv := range fvs {
enc, err := encodeElm(fv, wc, ec)
if err != nil {
return nil, err
}
encElms[i] = enc
}
return encElms, nil
}
return encodeElms
}
// Checkpoint attempts to split an SDF that has self-checkpointed (e.g. returned a
// ProcessContinuation) and needs to be resumed later. If the underlying DoFn is not
// splittable or has not returned a resuming continuation, the function returns an empty
// SplitResult, a negative resumption time, and a false boolean to indicate that no split
// occurred.
func (n *DataSource) Checkpoint() (SplitResult, time.Duration, bool, error) {
n.mu.Lock()
defer n.mu.Unlock()
pc := n.getProcessContinuation()
if pc == nil || !pc.ShouldResume() {
return SplitResult{}, -1 * time.Minute, false, nil
}
su := SplittableUnit(n.Out.(*ProcessSizedElementsAndRestrictions))
ow := su.GetOutputWatermark()
// Checkpointing is functionally a split at fraction 0.0
rs, err := su.Checkpoint()
if err != nil {
return SplitResult{}, -1 * time.Minute, false, err
}
if len(rs) == 0 {
return SplitResult{}, -1 * time.Minute, false, nil
}
encodeElms := n.makeEncodeElms()
rsEnc, err := encodeElms(rs)
if err != nil {
return SplitResult{}, -1 * time.Minute, false, err
}
res := SplitResult{
RS: rsEnc,
TId: su.GetTransformId(),
InId: su.GetInputId(),
OW: ow,
}
return res, pc.ResumeDelay(), true, nil
}
// Split takes a sorted set of potential split indices and a fraction of the
// remainder to split at, selects and actuates a split on an appropriate split
// index, and returns the selected split index in a SplitResult if successful or
// an error when unsuccessful.
//
// If the following transform is splittable, and the split indices and fraction
// allow for splitting on the currently processing element, then a sub-element
// split is performed, and the appropriate information is returned in the
// SplitResult.
//
// The bufSize param specifies the estimated number of elements that will be
// sent to this DataSource, and is used to be able to perform accurate splits
// even if the DataSource has not yet received all its elements. A bufSize of
// 0 or less indicates that its unknown, and so uses the current known size.
func (n *DataSource) Split(splits []int64, frac float64, bufSize int64) (SplitResult, error) {
if n == nil {
return SplitResult{}, fmt.Errorf("failed to split at requested splits: {%v}, DataSource not initialized", splits)
}
if frac > 1.0 {
frac = 1.0
} else if frac < 0.0 {
frac = 0.0
}
n.mu.Lock()
defer n.mu.Unlock()
var currProg float64 // Current element progress.
var su SplittableUnit
if n.index < 0 { // Progress is at the end of the non-existant -1st element.
currProg = 1.0
} else if n.su == nil { // If this isn't sub-element splittable, estimate some progress.
currProg = 0.5
} else { // If this is sub-element splittable, get progress of the current element.
select {
case su = <-n.su:
// If an element is processing, we'll get a splittable unit.
if su == nil {
return SplitResult{}, fmt.Errorf("failed to split: splittable unit was nil")
}
defer func() {
n.su <- su
}()
currProg = su.GetProgress()
case <-time.After(500 * time.Millisecond):
// Otherwise, the current element hasn't started processing yet
// or has already finished. By adding a short timeout, we avoid
// the first possibility, and can assume progress is at max.
currProg = 1.0
}
}
// Size to split within is the minimum of bufSize or splitIdx so we avoid
// including elements we already know won't be processed.
if bufSize <= 0 || n.splitIdx < bufSize {
bufSize = n.splitIdx
}
s, fr, err := splitHelper(n.index, bufSize, currProg, splits, frac, su != nil)
if err != nil {
return SplitResult{}, err
}
// No fraction returned, perform channel split.
if fr < 0 {
n.splitIdx = s
return SplitResult{PI: s - 1, RI: s}, nil
}
// Get the output watermark before splitting to avoid accidentally overestimating
ow := su.GetOutputWatermark()
// Otherwise, perform a sub-element split.
ps, rs, err := su.Split(fr)
if err != nil {
return SplitResult{}, err
}
if len(ps) == 0 || len(rs) == 0 { // Unsuccessful split.
// Fallback to channel split, so split at next elm, not current.
n.splitIdx = s + 1
return SplitResult{PI: s, RI: s + 1}, nil
}
// TODO(https://github.com/apache/beam/issues/20343) Eventually encode elements with the splittable
// unit's input coder instead of the DataSource's coder.
encodeElms := n.makeEncodeElms()
psEnc, err := encodeElms(ps)
if err != nil {
return SplitResult{}, err
}
rsEnc, err := encodeElms(rs)
if err != nil {
return SplitResult{}, err
}
n.splitIdx = s + 1 // In a sub-element split, s is currIdx.
res := SplitResult{
PI: s - 1,
RI: s + 1,
PS: psEnc,
RS: rsEnc,
TId: su.GetTransformId(),
InId: su.GetInputId(),
OW: ow,
}
return res, nil
}
// splitHelper is a helper function that finds a split point in a range.
//
// currIdx and endIdx should match the DataSource's index and splitIdx fields,
// and represent the start and end of the splittable range respectively.
//
// currProg represents the progress through the current element (currIdx).
//
// splits is an optional slice of valid split indices, and if nil then all
// indices are considered valid split points.
//
// frac must be between [0, 1], and represents a fraction of the remaining work
// that the split point aims to be as close as possible to.
//
// splittable indicates that sub-element splitting is possible (i.e. the next
// unit is an SDF).
//
// Returns the element index to split at (first element of residual). If the
// split position qualifies for sub-element splitting, then this also returns
// the fraction of remaining work in the current element to use as a split
// fraction for a sub-element split, and otherwise returns -1.
//
// A split point is sub-element splittable iff the split point is the current
// element, the splittable param is set to true, and both the element being
// split and the following element are valid split points.
func splitHelper(
currIdx, endIdx int64,
currProg float64,
splits []int64,
frac float64,
splittable bool) (int64, float64, error) {
// Get split index from fraction. Find the closest index to the fraction of
// the remainder.
start := float64(currIdx) + currProg
safeStart := currIdx + 1 // safeStart avoids splitting at 0, or <= currIdx
if safeStart <= 0 {
safeStart = 1
}
var splitFloat = start + frac*(float64(endIdx)-start)
// Handle simpler cases where all split points are valid first.
if len(splits) == 0 {
if splittable && int64(splitFloat) == currIdx {
// Sub-element splitting is valid.
_, f := math.Modf(splitFloat)
// Convert from fraction of entire element to fraction of remainder.
fr := (f - currProg) / (1.0 - currProg)
return int64(splitFloat), fr, nil
}
// All split points are valid so just split at safe index closest to
// fraction.
splitIdx := int64(math.Round(splitFloat))
if splitIdx < safeStart {
splitIdx = safeStart
}
return splitIdx, -1.0, nil
}
// Cases where we have to find a valid split point.
sort.Slice(splits, func(i, j int) bool { return splits[i] < splits[j] })
if splittable && int64(splitFloat) == currIdx {
// Check valid split points to see if we can do a sub-element split.
// We need to find the currIdx and currIdx + 1 for it to be valid.
c, cp1 := false, false
for _, s := range splits {
if s == currIdx {
c = true
} else if s == currIdx+1 {
cp1 = true
break
} else if s > currIdx+1 {
break
}
}
if c && cp1 { // Sub-element splitting is valid.
_, f := math.Modf(splitFloat)
// Convert from fraction of entire element to fraction of remainder.
fr := (f - currProg) / (1.0 - currProg)
return int64(splitFloat), fr, nil
}
}
// For non-sub-element splitting, find the closest unprocessed split
// point to our fraction.
var prevDiff = math.MaxFloat64
var bestS int64 = -1
for _, s := range splits {
if s >= safeStart && s <= endIdx {
diff := math.Abs(splitFloat - float64(s))
if diff <= prevDiff {
prevDiff = diff
bestS = s
} else {
break // Stop early if the difference starts increasing.
}
}
}
if bestS != -1 {
return bestS, -1.0, nil
}
// Printing all splits is expensive. Instead, return the current start and
// end indices, and fraction along with the range of the indices and how
// many there are. This branch requires at least one split index, so we don't
// need to bounds check the slice.
return -1, -1.0, fmt.Errorf("failed to split DataSource (at index: %v, last index: %v) at fraction %.4f with requested splits (%v indices from %v to %v)",
currIdx, endIdx, frac, len(splits), splits[0], splits[len(splits)-1])
}
func encodeElm(elm *FullValue, wc WindowEncoder, ec ElementEncoder) ([]byte, error) {
var b bytes.Buffer
if err := EncodeWindowedValueHeader(wc, elm.Windows, elm.Timestamp, elm.Pane, &b); err != nil {
return nil, err
}
if err := ec.Encode(elm, &b); err != nil {
return nil, err
}
return b.Bytes(), nil
}
type concatReStream struct {
first, next ReStream
}
func (c *concatReStream) Open() (Stream, error) {
firstStream, err := c.first.Open()
if err != nil {
return nil, err
}
return &concatStream{first: firstStream, nextStream: c.next}, nil
}
type concatStream struct {
first Stream
nextStream ReStream
}
// Close nils the stream.
func (s *concatStream) Close() error {
if s.first == nil {
return nil
}
defer func() {
s.first = nil
s.nextStream = nil
}()
return s.first.Close()
}
func (s *concatStream) Read() (*FullValue, error) {
if s.first == nil { // When the stream is closed.
return nil, io.EOF
}
fv, err := s.first.Read()
if err == nil {
return fv, nil
}
if err == io.EOF {
if err := s.first.Close(); err != nil {
s.nextStream = nil
return nil, err
}
if s.nextStream == nil {
s.first = nil
return nil, io.EOF
}
s.first, err = s.nextStream.Open()
s.nextStream = nil
if err != nil {
return nil, err
}
fv, err := s.first.Read()
return fv, err
}
return nil, err
}