github.com/apache/arrow/go/v10@v10.0.1/parquet/pqarrow/file_reader.go (about) 1 // Licensed to the Apache Software Foundation (ASF) under one 2 // or more contributor license agreements. See the NOTICE file 3 // distributed with this work for additional information 4 // regarding copyright ownership. The ASF licenses this file 5 // to you under the Apache License, Version 2.0 (the 6 // "License"); you may not use this file except in compliance 7 // with the License. You may obtain a copy of the License at 8 // 9 // http://www.apache.org/licenses/LICENSE-2.0 10 // 11 // Unless required by applicable law or agreed to in writing, software 12 // distributed under the License is distributed on an "AS IS" BASIS, 13 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 // See the License for the specific language governing permissions and 15 // limitations under the License. 16 17 package pqarrow 18 19 import ( 20 "context" 21 "fmt" 22 "io" 23 "sync" 24 "sync/atomic" 25 26 "github.com/apache/arrow/go/v10/arrow" 27 "github.com/apache/arrow/go/v10/arrow/array" 28 "github.com/apache/arrow/go/v10/arrow/arrio" 29 "github.com/apache/arrow/go/v10/arrow/memory" 30 "github.com/apache/arrow/go/v10/parquet" 31 "github.com/apache/arrow/go/v10/parquet/file" 32 "github.com/apache/arrow/go/v10/parquet/schema" 33 "golang.org/x/sync/errgroup" 34 "golang.org/x/xerrors" 35 ) 36 37 type itrFactory func(int, *file.Reader) *columnIterator 38 39 type readerCtx struct { 40 rdr *file.Reader 41 mem memory.Allocator 42 colFactory itrFactory 43 filterLeaves bool 44 includedLeaves map[int]bool 45 } 46 47 func (r readerCtx) includesLeaf(idx int) bool { 48 _, ok := r.includedLeaves[idx] 49 return ok 50 } 51 52 // ReadTable is a convenience function to quickly and easily read a parquet file 53 // into an arrow table. 54 // 55 // The schema of the arrow table is generated based on the schema of the parquet file, 56 // including nested columns/lists/etc. in the same fashion as the FromParquetSchema 57 // function. This just encapsulates the logic of creating a separate file.Reader and 58 // pqarrow.FileReader to make a single easy function when you just want to construct 59 // a table from the entire parquet file rather than reading it piecemeal. 60 func ReadTable(ctx context.Context, r parquet.ReaderAtSeeker, props *parquet.ReaderProperties, arrProps ArrowReadProperties, mem memory.Allocator) (arrow.Table, error) { 61 pf, err := file.NewParquetReader(r, file.WithReadProps(props)) 62 if err != nil { 63 return nil, err 64 } 65 66 reader, err := NewFileReader(pf, arrProps, mem) 67 if err != nil { 68 return nil, err 69 } 70 71 return reader.ReadTable(ctx) 72 } 73 74 // FileReader is the base object for reading a parquet file into arrow object 75 // types. 76 // 77 // It provides utility functions for reading record batches, a table, subsets of 78 // columns / rowgroups, and so on. 79 type FileReader struct { 80 mem memory.Allocator 81 rdr *file.Reader 82 83 Props ArrowReadProperties 84 Manifest *SchemaManifest 85 } 86 87 // NewFileReader constructs a reader for converting to Arrow objects from an existing 88 // parquet file reader object. 89 // 90 // Only returns an error if there is some error constructing the schema manifest from 91 // the parquet file metadata. 92 func NewFileReader(rdr *file.Reader, props ArrowReadProperties, mem memory.Allocator) (*FileReader, error) { 93 manifest, err := NewSchemaManifest(rdr.MetaData().Schema, rdr.MetaData().KeyValueMetadata(), &props) 94 if err != nil { 95 return nil, err 96 } 97 98 return &FileReader{ 99 mem: mem, 100 rdr: rdr, 101 Props: props, 102 Manifest: manifest, 103 }, nil 104 } 105 106 // Schema returns the arrow schema representation of the underlying file's schema. 107 func (fr *FileReader) Schema() (*arrow.Schema, error) { 108 return FromParquet(fr.rdr.MetaData().Schema, &fr.Props, fr.rdr.MetaData().KeyValueMetadata()) 109 } 110 111 type colReaderImpl interface { 112 LoadBatch(nrecs int64) error 113 BuildArray(boundedLen int64) (*arrow.Chunked, error) 114 GetDefLevels() ([]int16, error) 115 GetRepLevels() ([]int16, error) 116 Field() *arrow.Field 117 IsOrHasRepeatedChild() bool 118 Retain() 119 Release() 120 } 121 122 // ColumnReader is used for reading batches of data from a specific column 123 // across multiple row groups to return a chunked arrow array. 124 type ColumnReader struct { 125 colReaderImpl 126 } 127 128 // NextBatch returns a chunked array after reading `size` values, potentially 129 // across multiple row groups. 130 func (c *ColumnReader) NextBatch(size int64) (*arrow.Chunked, error) { 131 if err := c.LoadBatch(size); err != nil { 132 return nil, err 133 } 134 return c.BuildArray(size) 135 } 136 137 type rdrCtxKey struct{} 138 139 func readerCtxFromContext(ctx context.Context) readerCtx { 140 rdc := ctx.Value(rdrCtxKey{}) 141 if rdc != nil { 142 return rdc.(readerCtx) 143 } 144 panic("no readerctx") 145 } 146 147 // ParquetReader returns the underlying parquet file reader that it was constructed with 148 func (fr *FileReader) ParquetReader() *file.Reader { return fr.rdr } 149 150 // GetColumn returns a reader for pulling the data of leaf column index i 151 // across all row groups in the file. 152 func (fr *FileReader) GetColumn(ctx context.Context, i int) (*ColumnReader, error) { 153 return fr.getColumnReader(ctx, i, fr.allRowGroupFactory()) 154 } 155 156 func rowGroupFactory(rowGroups []int) itrFactory { 157 return func(i int, rdr *file.Reader) *columnIterator { 158 return &columnIterator{ 159 index: i, 160 rdr: rdr, 161 schema: rdr.MetaData().Schema, 162 rowGroups: rowGroups, 163 } 164 } 165 } 166 167 func (fr *FileReader) allRowGroupFactory() itrFactory { 168 rowGroups := make([]int, fr.rdr.NumRowGroups()) 169 for idx := range rowGroups { 170 rowGroups[idx] = idx 171 } 172 return rowGroupFactory(rowGroups) 173 } 174 175 // GetFieldReader returns a reader for the entire Field of index i which could potentially include reading 176 // multiple columns from the underlying parquet file if that field is a nested field. 177 // 178 // IncludedLeaves and RowGroups are used to specify precisely which leaf indexes and row groups to read a subset of. 179 func (fr *FileReader) GetFieldReader(ctx context.Context, i int, includedLeaves map[int]bool, rowGroups []int) (*ColumnReader, error) { 180 ctx = context.WithValue(ctx, rdrCtxKey{}, readerCtx{ 181 rdr: fr.rdr, 182 mem: fr.mem, 183 colFactory: rowGroupFactory(rowGroups), 184 filterLeaves: true, 185 includedLeaves: includedLeaves, 186 }) 187 return fr.getReader(ctx, &fr.Manifest.Fields[i], *fr.Manifest.Fields[i].Field) 188 } 189 190 // GetFieldReaders is for retrieving readers for multiple fields at one time for only the list 191 // of column indexes and rowgroups requested. It returns a slice of the readers and the corresponding 192 // arrow.Schema for those columns. 193 func (fr *FileReader) GetFieldReaders(ctx context.Context, colIndices, rowGroups []int) ([]*ColumnReader, *arrow.Schema, error) { 194 fieldIndices, err := fr.Manifest.GetFieldIndices(colIndices) 195 if err != nil { 196 return nil, nil, err 197 } 198 199 includedLeaves := make(map[int]bool) 200 for _, col := range colIndices { 201 includedLeaves[col] = true 202 } 203 204 out := make([]*ColumnReader, len(fieldIndices)) 205 outFields := make([]arrow.Field, len(fieldIndices)) 206 207 // Load batches in parallel 208 // When reading structs with large numbers of columns, the serial load is very slow. 209 // This is especially true when reading Cloud Storage. Loading concurrently 210 // greatly improves performance. 211 // GetFieldReader causes read operations, when issued serially on large numbers of columns, 212 // this is super time consuming. Get field readers concurrently. 213 g, gctx := errgroup.WithContext(ctx) 214 if !fr.Props.Parallel { 215 g.SetLimit(1) 216 } 217 for idx, fidx := range fieldIndices { 218 idx, fidx := idx, fidx // create concurrent copy 219 g.Go(func() error { 220 rdr, err := fr.GetFieldReader(gctx, fidx, includedLeaves, rowGroups) 221 if err != nil { 222 return err 223 } 224 outFields[idx] = *rdr.Field() 225 out[idx] = rdr 226 return nil 227 }) 228 } 229 if err = g.Wait(); err != nil { 230 return nil, nil, err 231 } 232 233 return out, arrow.NewSchema(outFields, fr.Manifest.SchemaMeta), nil 234 } 235 236 // RowGroup creates a reader that will *only* read from the requested row group 237 func (fr *FileReader) RowGroup(idx int) RowGroupReader { 238 return RowGroupReader{fr, idx} 239 } 240 241 // ReadColumn reads data to create a chunked array only from the requested row groups. 242 func (fr *FileReader) ReadColumn(rowGroups []int, rdr *ColumnReader) (*arrow.Chunked, error) { 243 recs := int64(0) 244 for _, rg := range rowGroups { 245 recs += fr.rdr.MetaData().RowGroups[rg].GetNumRows() 246 } 247 return rdr.NextBatch(recs) 248 } 249 250 // ReadTable reads the entire file into an array.Table 251 func (fr *FileReader) ReadTable(ctx context.Context) (arrow.Table, error) { 252 var ( 253 cols = []int{} 254 rgs = []int{} 255 ) 256 for i := 0; i < fr.rdr.MetaData().Schema.NumColumns(); i++ { 257 cols = append(cols, i) 258 } 259 for i := 0; i < fr.rdr.NumRowGroups(); i++ { 260 rgs = append(rgs, i) 261 } 262 return fr.ReadRowGroups(ctx, cols, rgs) 263 } 264 265 func (fr *FileReader) checkCols(indices []int) (err error) { 266 for _, col := range indices { 267 if col < 0 || col >= fr.rdr.MetaData().Schema.NumColumns() { 268 err = fmt.Errorf("invalid column index specified %d out of %d", col, fr.rdr.MetaData().Schema.NumColumns()) 269 break 270 } 271 } 272 return 273 } 274 275 func (fr *FileReader) checkRowGroups(indices []int) (err error) { 276 for _, rg := range indices { 277 if rg < 0 || rg >= fr.rdr.NumRowGroups() { 278 err = fmt.Errorf("invalid row group specified: %d, file only has %d row groups", rg, fr.rdr.NumRowGroups()) 279 break 280 } 281 } 282 return 283 } 284 285 type readerInfo struct { 286 rdr *ColumnReader 287 idx int 288 } 289 290 type resultPair struct { 291 idx int 292 data *arrow.Chunked 293 err error 294 } 295 296 //! This is Super complicated. I would simpify the pattern, but it works and hesitant to change what works. 297 298 // ReadRowGroups is for generating an array.Table from the file but filtering to only read the requested 299 // columns and row groups rather than the entire file which ReadTable does. 300 func (fr *FileReader) ReadRowGroups(ctx context.Context, indices, rowGroups []int) (arrow.Table, error) { 301 if err := fr.checkRowGroups(rowGroups); err != nil { 302 return nil, err 303 } 304 if err := fr.checkCols(indices); err != nil { 305 return nil, err 306 } 307 308 // TODO(mtopol): add optimizations for pre-buffering data options 309 310 readers, sc, err := fr.GetFieldReaders(ctx, indices, rowGroups) 311 if err != nil { 312 return nil, err 313 } 314 315 // producer-consumer parallelization 316 var ( 317 np = 1 318 wg sync.WaitGroup 319 ch = make(chan readerInfo, len(readers)) 320 results = make(chan resultPair, 2) 321 ) 322 323 if fr.Props.Parallel { 324 np = len(readers) 325 } 326 327 ctx, cancel := context.WithCancel(ctx) 328 defer cancel() 329 330 wg.Add(np) // fan-out to np readers 331 for i := 0; i < np; i++ { 332 go func() { 333 defer wg.Done() 334 for { 335 select { 336 case r, ok := <-ch: 337 if !ok { 338 return 339 } 340 341 chnked, err := fr.ReadColumn(rowGroups, r.rdr) 342 // pass the result column data to the result channel 343 // for the consumer goroutine to process 344 results <- resultPair{r.idx, chnked, err} 345 case <-ctx.Done(): // check if we cancelled 346 return 347 } 348 } 349 }() 350 } 351 352 go func() { 353 wg.Wait() 354 close(results) // close the result channel when there's no more 355 }() 356 357 // pass pairs of reader and column index to the channel for the 358 // goroutines to read the data 359 for idx, r := range readers { 360 defer func(r *ColumnReader) { 361 r.Release() 362 }(r) 363 ch <- readerInfo{r, idx} 364 } 365 close(ch) 366 367 // output slice of columns 368 columns := make([]arrow.Column, len(sc.Fields())) 369 for data := range results { 370 if data.err != nil { 371 err = data.err 372 cancel() 373 break 374 } 375 //lint:ignore SA9001 defer. 376 defer data.data.Release() 377 col := arrow.NewColumn(sc.Field(data.idx), data.data) 378 columns[data.idx] = *col 379 } 380 381 if err != nil { 382 // if we encountered an error, consume any waiting data on the channel 383 // so the goroutines don't leak and so memory can get cleaned up. we already 384 // cancelled the context so we're just consuming anything that was already queued up. 385 for data := range results { 386 //lint:ignore SA9001 defer. 387 defer data.data.Release() 388 } 389 return nil, err 390 } 391 392 var nrows int 393 if len(columns) > 0 { 394 nrows = columns[0].Len() 395 } 396 397 return array.NewTable(sc, columns, int64(nrows)), nil 398 } 399 400 func (fr *FileReader) getColumnReader(ctx context.Context, i int, colFactory itrFactory) (*ColumnReader, error) { 401 if i < 0 || i >= fr.rdr.MetaData().Schema.NumColumns() { 402 return nil, fmt.Errorf("invalid column index chosen %d, there are only %d columns", i, fr.rdr.MetaData().Schema.NumColumns()) 403 } 404 405 ctx = context.WithValue(ctx, rdrCtxKey{}, readerCtx{ 406 rdr: fr.rdr, 407 mem: fr.mem, 408 colFactory: colFactory, 409 filterLeaves: false, 410 }) 411 412 return fr.getReader(ctx, &fr.Manifest.Fields[i], *fr.Manifest.Fields[i].Field) 413 } 414 415 // RecordReader is a Record Batch Reader that meets the interfaces for both 416 // array.RecordReader and arrio.Reader to allow easy progressive reading 417 // of record batches from the parquet file. Ideal for streaming. 418 type RecordReader interface { 419 array.RecordReader 420 arrio.Reader 421 } 422 423 // GetRecordReader returns a record reader that reads only the requested column indexes and row groups. 424 // 425 // For both cases, if you pass nil for column indexes or rowgroups it will default to reading all of them. 426 func (fr *FileReader) GetRecordReader(ctx context.Context, colIndices, rowGroups []int) (RecordReader, error) { 427 if err := fr.checkRowGroups(rowGroups); err != nil { 428 return nil, err 429 } 430 431 if rowGroups == nil { 432 rowGroups = make([]int, fr.rdr.NumRowGroups()) 433 for idx := range rowGroups { 434 rowGroups[idx] = idx 435 } 436 } 437 438 if err := fr.checkCols(colIndices); err != nil { 439 return nil, err 440 } 441 442 if colIndices == nil { 443 colIndices = make([]int, fr.rdr.MetaData().Schema.NumColumns()) 444 for idx := range colIndices { 445 colIndices[idx] = idx 446 } 447 } 448 449 // TODO(mtopol): add optimizations to pre-buffer data from the file 450 451 readers, sc, err := fr.GetFieldReaders(ctx, colIndices, rowGroups) 452 if err != nil { 453 return nil, err 454 } 455 456 if len(readers) == 0 { 457 return nil, xerrors.New("no leaf column readers matched col indices") 458 } 459 460 nrows := int64(0) 461 for _, rg := range rowGroups { 462 nrows += fr.rdr.MetaData().RowGroup(rg).NumRows() 463 } 464 465 return &recordReader{ 466 numRows: nrows, 467 batchSize: fr.Props.BatchSize, 468 parallel: fr.Props.Parallel, 469 sc: sc, 470 fieldReaders: readers, 471 refCount: 1, 472 }, nil 473 } 474 475 func (fr *FileReader) getReader(ctx context.Context, field *SchemaField, arrowField arrow.Field) (out *ColumnReader, err error) { 476 rctx := readerCtxFromContext(ctx) 477 if len(field.Children) == 0 { 478 if !field.IsLeaf() { 479 return nil, xerrors.New("parquet non-leaf node has no children") 480 } 481 if rctx.filterLeaves && !rctx.includesLeaf(field.ColIndex) { 482 return nil, nil 483 } 484 485 out, err = newLeafReader(&rctx, field.Field, rctx.colFactory(field.ColIndex, rctx.rdr), field.LevelInfo, fr.Props, fr.rdr.BufferPool()) 486 return 487 } 488 489 switch arrowField.Type.ID() { 490 case arrow.EXTENSION: 491 return nil, xerrors.New("extension type not implemented") 492 case arrow.STRUCT: 493 494 childReaders := make([]*ColumnReader, len(field.Children)) 495 childFields := make([]arrow.Field, len(field.Children)) 496 497 // Get child field readers concurrently 498 // 'getReader' causes a read operation. Issue the 'reads' concurrently 499 // When reading structs with large numbers of columns, the serial load is very slow. 500 // This is especially true when reading Cloud Storage. Loading concurrently 501 // greatly improves performance. 502 g, gctx := errgroup.WithContext(ctx) 503 if !fr.Props.Parallel { 504 g.SetLimit(1) 505 } 506 507 for n, child := range field.Children { 508 n, child := n, child 509 g.Go(func() error { 510 reader, err := fr.getReader(gctx, &child, *child.Field) 511 if err != nil { 512 return err 513 } 514 if reader == nil { 515 return nil 516 } 517 childFields[n] = *child.Field 518 childReaders[n] = reader 519 return nil 520 }) 521 } 522 if err = g.Wait(); err != nil { 523 return nil, err 524 } 525 526 // because we performed getReader concurrently, we need to prune out any empty readers 527 for n := len(childReaders) - 1; n >= 0; n-- { 528 if childReaders[n] == nil { 529 childReaders = append(childReaders[:n], childReaders[n+1:]...) 530 childFields = append(childFields[:n], childFields[n+1:]...) 531 } 532 } 533 if len(childFields) == 0 { 534 return nil, nil 535 } 536 filtered := arrow.Field{Name: arrowField.Name, Nullable: arrowField.Nullable, 537 Metadata: arrowField.Metadata, Type: arrow.StructOf(childFields...)} 538 out = newStructReader(&rctx, &filtered, field.LevelInfo, childReaders, fr.Props) 539 case arrow.LIST, arrow.FIXED_SIZE_LIST, arrow.MAP: 540 child := field.Children[0] 541 childReader, err := fr.getReader(ctx, &child, *child.Field) 542 if err != nil { 543 return nil, err 544 } 545 if childReader == nil { 546 return nil, nil 547 } 548 defer childReader.Release() 549 550 switch arrowField.Type.(type) { 551 case *arrow.MapType: 552 if len(child.Children) != 2 { 553 arrowField.Type = arrow.ListOf(childReader.Field().Type) 554 } 555 out = newListReader(&rctx, &arrowField, field.LevelInfo, childReader, fr.Props) 556 case *arrow.ListType: 557 out = newListReader(&rctx, &arrowField, field.LevelInfo, childReader, fr.Props) 558 case *arrow.FixedSizeListType: 559 out = newFixedSizeListReader(&rctx, &arrowField, field.LevelInfo, childReader, fr.Props) 560 default: 561 return nil, fmt.Errorf("unknown list type: %s", field.Field.String()) 562 } 563 } 564 return 565 } 566 567 // RowGroupReader is a reader for getting data only from a single row group of the file 568 // rather than having to repeatedly pass the index to functions on the reader. 569 type RowGroupReader struct { 570 impl *FileReader 571 idx int 572 } 573 574 // ReadTable provides an array.Table consisting only of the columns requested for this rowgroup 575 func (rgr RowGroupReader) ReadTable(ctx context.Context, colIndices []int) (arrow.Table, error) { 576 return rgr.impl.ReadRowGroups(ctx, colIndices, []int{rgr.idx}) 577 } 578 579 // Column creates a reader for just the requested column chunk in only this row group. 580 func (rgr RowGroupReader) Column(idx int) ColumnChunkReader { 581 return ColumnChunkReader{rgr.impl, idx, rgr.idx} 582 } 583 584 // ColumnChunkReader is a reader that reads only a single column chunk from a single 585 // column in a single row group 586 type ColumnChunkReader struct { 587 impl *FileReader 588 idx int 589 rowGroup int 590 } 591 592 func (ccr ColumnChunkReader) Read(ctx context.Context) (*arrow.Chunked, error) { 593 rdr, err := ccr.impl.getColumnReader(ctx, ccr.idx, rowGroupFactory([]int{ccr.rowGroup})) 594 if err != nil { 595 return nil, err 596 } 597 return ccr.impl.ReadColumn([]int{ccr.rowGroup}, rdr) 598 } 599 600 type columnIterator struct { 601 index int 602 rdr *file.Reader 603 schema *schema.Schema 604 rowGroups []int 605 } 606 607 func (c *columnIterator) NextChunk() (file.PageReader, error) { 608 if len(c.rowGroups) == 0 { 609 return nil, nil 610 } 611 612 rgr := c.rdr.RowGroup(c.rowGroups[0]) 613 c.rowGroups = c.rowGroups[1:] 614 return rgr.GetColumnPageReader(c.index) 615 } 616 617 func (c *columnIterator) Descr() *schema.Column { return c.schema.Column(c.index) } 618 619 // implementation of arrio.Reader for streaming record batches 620 // from the parquet data. 621 type recordReader struct { 622 numRows int64 623 batchSize int64 624 parallel bool 625 sc *arrow.Schema 626 fieldReaders []*ColumnReader 627 cur arrow.Record 628 err error 629 630 refCount int64 631 } 632 633 func (r *recordReader) Retain() { 634 atomic.AddInt64(&r.refCount, 1) 635 } 636 637 func (r *recordReader) Release() { 638 if atomic.AddInt64(&r.refCount, -1) == 0 { 639 if r.cur != nil { 640 r.cur.Release() 641 r.cur = nil 642 } 643 if r.fieldReaders == nil { 644 return 645 } 646 for _, fr := range r.fieldReaders { 647 fr.Release() 648 } 649 r.fieldReaders = nil 650 } 651 } 652 653 func (r *recordReader) Schema() *arrow.Schema { return r.sc } 654 655 func (r *recordReader) next() bool { 656 cols := make([]arrow.Array, len(r.sc.Fields())) 657 defer func() { 658 for _, c := range cols { 659 if c != nil { 660 c.Release() 661 } 662 } 663 }() 664 readField := func(idx int, rdr *ColumnReader) error { 665 data, err := rdr.NextBatch(r.batchSize) 666 if err != nil { 667 return err 668 } 669 670 if data.Len() == 0 { 671 return io.EOF 672 } 673 674 arrdata, err := chunksToSingle(data) 675 if err != nil { 676 return err 677 } 678 cols[idx] = array.MakeFromData(arrdata) 679 return nil 680 } 681 682 if !r.parallel { 683 for idx, rdr := range r.fieldReaders { 684 if err := readField(idx, rdr); err != nil { 685 r.err = err 686 return false 687 } 688 } 689 690 r.cur = array.NewRecord(r.sc, cols, -1) 691 return true 692 } 693 694 var ( 695 wg sync.WaitGroup 696 np = len(cols) 697 ch = make(chan int, np) 698 errch = make(chan error, np) 699 ) 700 701 ctx, cancel := context.WithCancel(context.Background()) 702 defer cancel() 703 704 wg.Add(np) 705 for i := 0; i < np; i++ { 706 go func() { 707 defer wg.Done() 708 for { 709 select { 710 case idx, ok := <-ch: 711 if !ok { 712 return 713 } 714 715 if err := readField(idx, r.fieldReaders[idx]); err != nil { 716 errch <- err 717 cancel() 718 return 719 } 720 721 case <-ctx.Done(): 722 return 723 } 724 } 725 }() 726 } 727 728 for idx := range r.fieldReaders { 729 ch <- idx 730 } 731 close(ch) 732 wg.Wait() 733 close(errch) 734 735 var ok bool 736 // check for any errors 737 if r.err, ok = <-errch; ok { 738 // return the first error that was reported and drain 739 // any remaining errors from the channel before returning. 740 for range errch { 741 } 742 return false 743 } 744 745 r.cur = array.NewRecord(r.sc, cols, -1) 746 return true 747 } 748 749 func (r *recordReader) Next() bool { 750 if r.cur != nil { 751 r.cur.Release() 752 r.cur = nil 753 } 754 755 if r.err != nil { 756 return false 757 } 758 759 return r.next() 760 } 761 762 func (r *recordReader) Record() arrow.Record { return r.cur } 763 764 func (r *recordReader) Read() (arrow.Record, error) { 765 if r.cur != nil { 766 r.cur.Release() 767 r.cur = nil 768 } 769 770 if !r.next() { 771 return nil, r.err 772 } 773 774 return r.cur, nil 775 }