/
record_batch_reader.go
128 lines (107 loc) · 2.82 KB
/
record_batch_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
package ipc
import (
"bytes"
"github.com/aliyun/aliyun-odps-go-sdk/arrow"
"github.com/aliyun/aliyun-odps-go-sdk/arrow/array"
"github.com/aliyun/aliyun-odps-go-sdk/arrow/internal/debug"
"github.com/aliyun/aliyun-odps-go-sdk/arrow/memory"
"golang.org/x/xerrors"
"io"
"sync/atomic"
)
type RecordBatchReader struct {
msgReader MessageReader
schema *arrow.Schema
refCount int64
rec array.Record
err error
types dictTypeMap
memo dictMemo
mem memory.Allocator
done bool
}
// NewRecordBatchReader returns a reader that reads records from an input stream.
func NewRecordBatchReader(r io.Reader, schema *arrow.Schema, opts ...Option) *RecordBatchReader {
cfg := newConfig()
for _, opt := range opts {
opt(cfg)
}
return &RecordBatchReader{
msgReader: NewMessageReader(r, opts...),
schema: schema,
refCount: 1,
types: make(dictTypeMap),
memo: newMemo(),
mem: cfg.alloc,
}
}
// Retain increases the reference count by 1.
// Retain may be called simultaneously from multiple goroutines.
func (r *RecordBatchReader) Retain() {
atomic.AddInt64(&r.refCount, 1)
}
// Release decreases the reference count by 1.
// When the reference count goes to zero, the memory is freed.
// Release may be called simultaneously from multiple goroutines.
func (r *RecordBatchReader) Release() {
debug.Assert(atomic.LoadInt64(&r.refCount) > 0, "too many releases")
if atomic.AddInt64(&r.refCount, -1) == 0 {
if r.rec != nil {
r.rec.Release()
r.rec = nil
}
if r.msgReader != nil {
r.msgReader.Release()
r.msgReader = nil
}
}
}
// Next returns whether a Record could be extracted from the underlying stream.
func (r *RecordBatchReader) Next() bool {
if r.rec != nil {
r.rec.Release()
r.rec = nil
}
if r.err != nil || r.done {
return false
}
return r.next()
}
func (r *RecordBatchReader) next() bool {
var msg *Message
msg, r.err = r.msgReader.Message()
if r.err != nil {
r.done = true
if r.err == io.EOF {
r.err = nil
}
return false
}
if got, want := msg.Type(), MessageRecordBatch; got != want {
r.err = xerrors.Errorf("arrow/ipc: invalid message type (got=%v, want=%v", got, want)
return false
}
r.rec = newRecord(r.schema, msg.meta, bytes.NewReader(msg.body.Bytes()), r.mem)
return true
}
// Record returns the current record that has been extracted from the
// underlying stream.
// It is valid until the next call to Next.
func (r *RecordBatchReader) Record() array.Record {
return r.rec
}
// Read reads the current record from the underlying stream and an error, if any.
// When the Reader reaches the end of the underlying stream, it returns (nil, io.EOF).
func (r *RecordBatchReader) Read() (array.Record, error) {
if r.rec != nil {
r.rec.Release()
r.rec = nil
}
if !r.next() {
if r.done {
return nil, io.EOF
}
return nil, r.err
}
return r.rec, nil
}