/
reader.go
284 lines (244 loc) · 7.51 KB
/
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
// 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 ipc
import (
"bytes"
"errors"
"fmt"
"io"
"sync/atomic"
"github.com/apache/arrow/go/v12/arrow"
"github.com/apache/arrow/go/v12/arrow/array"
"github.com/apache/arrow/go/v12/arrow/endian"
"github.com/apache/arrow/go/v12/arrow/internal/debug"
"github.com/apache/arrow/go/v12/arrow/internal/dictutils"
"github.com/apache/arrow/go/v12/arrow/internal/flatbuf"
"github.com/apache/arrow/go/v12/arrow/memory"
)
// Reader reads records from an io.Reader.
// Reader expects a schema (plus any dictionaries) as the first messages
// in the stream, followed by records.
type Reader struct {
r MessageReader
schema *arrow.Schema
refCount int64
rec arrow.Record
err error
// types dictTypeMap
memo dictutils.Memo
readInitialDicts bool
done bool
swapEndianness bool
ensureNativeEndian bool
expectedSchema *arrow.Schema
mem memory.Allocator
}
// NewReaderFromMessageReader allows constructing a new reader object with the
// provided MessageReader allowing injection of reading messages other than
// by simple streaming bytes such as Arrow Flight which receives a protobuf message
func NewReaderFromMessageReader(r MessageReader, opts ...Option) (reader *Reader, err error) {
defer func() {
if pErr := recover(); pErr != nil {
err = fmt.Errorf("arrow/ipc: unknown error while reading: %v", pErr)
}
}()
cfg := newConfig()
for _, opt := range opts {
opt(cfg)
}
rr := &Reader{
r: r,
refCount: 1,
// types: make(dictTypeMap),
memo: dictutils.NewMemo(),
mem: cfg.alloc,
ensureNativeEndian: cfg.ensureNativeEndian,
expectedSchema: cfg.schema,
}
if !cfg.noAutoSchema {
if err := rr.readSchema(cfg.schema); err != nil {
return nil, err
}
}
return rr, nil
}
// NewReader returns a reader that reads records from an input stream.
func NewReader(r io.Reader, opts ...Option) (*Reader, error) {
return NewReaderFromMessageReader(NewMessageReader(r, opts...), opts...)
}
// Err returns the last error encountered during the iteration over the
// underlying stream.
func (r *Reader) Err() error { return r.err }
func (r *Reader) Schema() *arrow.Schema {
if r.schema == nil {
if err := r.readSchema(r.expectedSchema); err != nil {
r.err = fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err)
r.done = true
}
}
return r.schema
}
func (r *Reader) readSchema(schema *arrow.Schema) error {
msg, err := r.r.Message()
if err != nil {
return fmt.Errorf("arrow/ipc: could not read message schema: %w", err)
}
if msg.Type() != MessageSchema {
return fmt.Errorf("arrow/ipc: invalid message type (got=%v, want=%v)", msg.Type(), MessageSchema)
}
// FIXME(sbinet) refactor msg-header handling.
var schemaFB flatbuf.Schema
initFB(&schemaFB, msg.msg.Header)
r.schema, err = schemaFromFB(&schemaFB, &r.memo)
if err != nil {
return fmt.Errorf("arrow/ipc: could not decode schema from message schema: %w", err)
}
// check the provided schema match the one read from stream.
if schema != nil && !schema.Equal(r.schema) {
return errInconsistentSchema
}
if r.ensureNativeEndian && !r.schema.IsNativeEndian() {
r.swapEndianness = true
r.schema = r.schema.WithEndianness(endian.NativeEndian)
}
return nil
}
// Retain increases the reference count by 1.
// Retain may be called simultaneously from multiple goroutines.
func (r *Reader) 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 *Reader) 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.r != nil {
r.r.Release()
r.r = nil
}
}
}
// Next returns whether a Record could be extracted from the underlying stream.
func (r *Reader) 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 *Reader) getInitialDicts() bool {
var msg *Message
// we have to get all dictionaries before reconstructing the first
// record. subsequent deltas and replacements modify the memo
numDicts := r.memo.Mapper.NumDicts()
// there should be numDicts dictionary messages
for i := 0; i < numDicts; i++ {
msg, r.err = r.r.Message()
if r.err != nil {
r.done = true
if r.err == io.EOF {
if i == 0 {
r.err = nil
} else {
r.err = fmt.Errorf("arrow/ipc: IPC stream ended without reading the expected (%d) dictionaries", numDicts)
}
}
return false
}
if msg.Type() != MessageDictionaryBatch {
r.err = fmt.Errorf("arrow/ipc: IPC stream did not have the expected (%d) dictionaries at the start of the stream", numDicts)
}
if _, err := readDictionary(&r.memo, msg.meta, bytes.NewReader(msg.body.Bytes()), r.swapEndianness, r.mem); err != nil {
r.done = true
r.err = err
return false
}
}
r.readInitialDicts = true
return true
}
func (r *Reader) next() bool {
defer func() {
if pErr := recover(); pErr != nil {
r.err = fmt.Errorf("arrow/ipc: unknown error while reading: %v", pErr)
}
}()
if r.schema == nil {
if err := r.readSchema(r.expectedSchema); err != nil {
r.err = fmt.Errorf("arrow/ipc: could not read schema from stream: %w", err)
r.done = true
return false
}
}
if !r.readInitialDicts && !r.getInitialDicts() {
return false
}
var msg *Message
msg, r.err = r.r.Message()
for msg != nil && msg.Type() == MessageDictionaryBatch {
if _, r.err = readDictionary(&r.memo, msg.meta, bytes.NewReader(msg.body.Bytes()), r.swapEndianness, r.mem); r.err != nil {
r.done = true
return false
}
msg, r.err = r.r.Message()
}
if r.err != nil {
r.done = true
if errors.Is(r.err, io.EOF) {
r.err = nil
}
return false
}
if got, want := msg.Type(), MessageRecordBatch; got != want {
r.err = fmt.Errorf("arrow/ipc: invalid message type (got=%v, want=%v", got, want)
return false
}
r.rec = newRecord(r.schema, &r.memo, msg.meta, bytes.NewReader(msg.body.Bytes()), r.swapEndianness, 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 *Reader) Record() arrow.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 *Reader) Read() (arrow.Record, error) {
if r.rec != nil {
r.rec.Release()
r.rec = nil
}
if !r.next() {
if r.done && r.err == nil {
return nil, io.EOF
}
return nil, r.err
}
return r.rec, nil
}
var (
_ array.RecordReader = (*Reader)(nil)
)