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
|
// 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 avro
import (
"context"
"errors"
"fmt"
"io"
"sync/atomic"
"github.com/apache/arrow-go/v18/arrow"
"github.com/apache/arrow-go/v18/arrow/array"
"github.com/apache/arrow-go/v18/arrow/internal/debug"
"github.com/apache/arrow-go/v18/arrow/memory"
"github.com/hamba/avro/v2/ocf"
"github.com/tidwall/sjson"
avro "github.com/hamba/avro/v2"
)
var ErrMismatchFields = errors.New("arrow/avro: number of records mismatch")
// Option configures an Avro reader/writer.
type (
Option func(config)
config *OCFReader
)
type schemaEdit struct {
method string
path string
value any
}
// Reader wraps goavro/OCFReader and creates array.Records from a schema.
type OCFReader struct {
r *ocf.Decoder
avroSchema string
avroSchemaEdits []schemaEdit
schema *arrow.Schema
refs int64
bld *array.RecordBuilder
bldMap *fieldPos
ldr *dataLoader
cur arrow.Record
err error
primed bool
readerCtx context.Context
readCancel func()
maxOCF int
maxRec int
avroChan chan any
avroDatumCount int64
avroChanSize int
recChan chan arrow.Record
bldDone chan struct{}
recChanSize int
chunk int
mem memory.Allocator
}
// NewReader returns a reader that reads from an Avro OCF file and creates
// arrow.Records from the converted avro data.
func NewOCFReader(r io.Reader, opts ...Option) (*OCFReader, error) {
ocfr, err := ocf.NewDecoder(r)
if err != nil {
return nil, fmt.Errorf("%w: could not create avro ocfreader", arrow.ErrInvalid)
}
rr := &OCFReader{
r: ocfr,
refs: 1,
chunk: 1,
avroChanSize: 500,
recChanSize: 10,
}
for _, opt := range opts {
opt(rr)
}
rr.avroChan = make(chan any, rr.avroChanSize)
rr.recChan = make(chan arrow.Record, rr.recChanSize)
rr.bldDone = make(chan struct{})
schema, err := avro.Parse(string(ocfr.Metadata()["avro.schema"]))
if err != nil {
return nil, fmt.Errorf("%w: could not parse avro header", arrow.ErrInvalid)
}
rr.avroSchema = schema.String()
if len(rr.avroSchemaEdits) > 0 {
// execute schema edits
for _, e := range rr.avroSchemaEdits {
err := rr.editAvroSchema(e)
if err != nil {
return nil, fmt.Errorf("%w: could not edit avro schema", arrow.ErrInvalid)
}
}
// validate edited schema
schema, err = avro.Parse(rr.avroSchema)
if err != nil {
return nil, fmt.Errorf("%w: could not parse modified avro schema", arrow.ErrInvalid)
}
}
rr.schema, err = ArrowSchemaFromAvro(schema)
if err != nil {
return nil, fmt.Errorf("%w: could not convert avro schema", arrow.ErrInvalid)
}
if rr.mem == nil {
rr.mem = memory.DefaultAllocator
}
rr.readerCtx, rr.readCancel = context.WithCancel(context.Background())
go rr.decodeOCFToChan()
rr.bld = array.NewRecordBuilder(rr.mem, rr.schema)
rr.bldMap = newFieldPos()
rr.ldr = newDataLoader()
for idx, fb := range rr.bld.Fields() {
mapFieldBuilders(fb, rr.schema.Field(idx), rr.bldMap)
}
rr.ldr.drawTree(rr.bldMap)
go rr.recordFactory()
return rr, nil
}
// Reuse allows the OCFReader to be reused to read another Avro file provided the
// new Avro file has an identical schema.
func (rr *OCFReader) Reuse(r io.Reader, opts ...Option) error {
rr.Close()
rr.err = nil
ocfr, err := ocf.NewDecoder(r)
if err != nil {
return fmt.Errorf("%w: could not create avro ocfreader", arrow.ErrInvalid)
}
schema, err := avro.Parse(string(ocfr.Metadata()["avro.schema"]))
if err != nil {
return fmt.Errorf("%w: could not parse avro header", arrow.ErrInvalid)
}
if rr.avroSchema != schema.String() {
return fmt.Errorf("%w: avro schema mismatch", arrow.ErrInvalid)
}
rr.r = ocfr
for _, opt := range opts {
opt(rr)
}
rr.maxOCF = 0
rr.maxRec = 0
rr.avroDatumCount = 0
rr.primed = false
rr.avroChan = make(chan any, rr.avroChanSize)
rr.recChan = make(chan arrow.Record, rr.recChanSize)
rr.bldDone = make(chan struct{})
rr.readerCtx, rr.readCancel = context.WithCancel(context.Background())
go rr.decodeOCFToChan()
go rr.recordFactory()
return nil
}
// Err returns the last error encountered during the iteration over the
// underlying Avro file.
func (r *OCFReader) Err() error { return r.err }
// AvroSchema returns the Avro schema of the Avro OCF
func (r *OCFReader) AvroSchema() string { return r.avroSchema }
// Schema returns the converted Arrow schema of the Avro OCF
func (r *OCFReader) Schema() *arrow.Schema { return r.schema }
// Record returns the current record that has been extracted from the
// underlying Avro OCF file.
// It is valid until the next call to Next.
func (r *OCFReader) Record() arrow.Record { return r.cur }
// Metrics returns the maximum queue depth of the Avro record read cache and of the
// converted Arrow record cache.
func (r *OCFReader) Metrics() string {
return fmt.Sprintf("Max. OCF queue depth: %d/%d Max. record queue depth: %d/%d", r.maxOCF, r.avroChanSize, r.maxRec, r.recChanSize)
}
// OCFRecordsReadCount returns the number of Avro datum that were read from the Avro file.
func (r *OCFReader) OCFRecordsReadCount() int64 { return r.avroDatumCount }
// Close closes the OCFReader's Avro record read cache and converted Arrow record cache. OCFReader must
// be closed if the Avro OCF's records have not been read to completion.
func (r *OCFReader) Close() {
r.readCancel()
r.err = r.readerCtx.Err()
}
func (r *OCFReader) editAvroSchema(e schemaEdit) error {
var err error
switch e.method {
case "set":
r.avroSchema, err = sjson.Set(r.avroSchema, e.path, e.value)
if err != nil {
return fmt.Errorf("%w: schema edit 'set %s = %v' failure - %v", arrow.ErrInvalid, e.path, e.value, err)
}
case "delete":
r.avroSchema, err = sjson.Delete(r.avroSchema, e.path)
if err != nil {
return fmt.Errorf("%w: schema edit 'delete' failure - %v", arrow.ErrInvalid, err)
}
default:
return fmt.Errorf("%w: schema edit method must be 'set' or 'delete'", arrow.ErrInvalid)
}
return nil
}
// Next returns whether a Record can be received from the converted record queue.
// The user should check Err() after call to Next that return false to check
// if an error took place.
func (r *OCFReader) Next() bool {
if r.cur != nil {
r.cur.Release()
r.cur = nil
}
if r.maxOCF < len(r.avroChan) {
r.maxOCF = len(r.avroChan)
}
if r.maxRec < len(r.recChan) {
r.maxRec = len(r.recChan)
}
select {
case r.cur = <-r.recChan:
case <-r.bldDone:
if len(r.recChan) > 0 {
r.cur = <-r.recChan
}
}
if r.err != nil {
return false
}
return r.cur != nil
}
// WithAllocator specifies the Arrow memory allocator used while building records.
func WithAllocator(mem memory.Allocator) Option {
return func(cfg config) {
cfg.mem = mem
}
}
// WithReadCacheSize specifies the size of the OCF record decode queue, default value
// is 500.
func WithReadCacheSize(n int) Option {
return func(cfg config) {
if n < 1 {
cfg.avroChanSize = 500
} else {
cfg.avroChanSize = n
}
}
}
// WithRecordCacheSize specifies the size of the converted Arrow record queue, default
// value is 1.
func WithRecordCacheSize(n int) Option {
return func(cfg config) {
if n < 1 {
cfg.recChanSize = 1
} else {
cfg.recChanSize = n
}
}
}
// WithSchemaEdit specifies modifications to the Avro schema. Supported methods are 'set' and
// 'delete'. Set sets the value for the specified path. Delete deletes the value for the specified path.
// A path is in dot syntax, such as "fields.1" or "fields.0.type". The modified Avro schema is
// validated before conversion to Arrow schema - NewOCFReader will return an error if the modified schema
// cannot be parsed.
func WithSchemaEdit(method, path string, value any) Option {
return func(cfg config) {
var e schemaEdit
e.method = method
e.path = path
e.value = value
cfg.avroSchemaEdits = append(cfg.avroSchemaEdits, e)
}
}
// WithChunk specifies the chunk size used while reading Avro OCF files.
//
// If n is zero or 1, no chunking will take place and the reader will create
// one record per row.
// If n is greater than 1, chunks of n rows will be read.
// If n is negative, the reader will load the whole Avro OCF file into memory and
// create one big record with all the rows.
func WithChunk(n int) Option {
return func(cfg config) {
cfg.chunk = n
}
}
// Retain increases the reference count by 1.
// Retain may be called simultaneously from multiple goroutines.
func (r *OCFReader) Retain() {
atomic.AddInt64(&r.refs, 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 *OCFReader) Release() {
debug.Assert(atomic.LoadInt64(&r.refs) > 0, "too many releases")
if atomic.AddInt64(&r.refs, -1) == 0 {
if r.cur != nil {
r.cur.Release()
}
}
}
var _ array.RecordReader = (*OCFReader)(nil)
|