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