github.com/parquet-go/parquet-go@v0.20.0/reader.go (about)

     1  package parquet
     2  
     3  import (
     4  	"errors"
     5  	"fmt"
     6  	"io"
     7  	"reflect"
     8  )
     9  
    10  // GenericReader is similar to a Reader but uses a type parameter to define the
    11  // Go type representing the schema of rows being read.
    12  //
    13  // See GenericWriter for details about the benefits over the classic Reader API.
    14  type GenericReader[T any] struct {
    15  	base Reader
    16  	read readFunc[T]
    17  }
    18  
    19  // NewGenericReader is like NewReader but returns GenericReader[T] suited to write
    20  // rows of Go type T.
    21  //
    22  // The type parameter T should be a map, struct, or any. Any other types will
    23  // cause a panic at runtime. Type checking is a lot more effective when the
    24  // generic parameter is a struct type, using map and interface types is somewhat
    25  // similar to using a Writer.
    26  //
    27  // If the option list may explicitly declare a schema, it must be compatible
    28  // with the schema generated from T.
    29  func NewGenericReader[T any](input io.ReaderAt, options ...ReaderOption) *GenericReader[T] {
    30  	c, err := NewReaderConfig(options...)
    31  	if err != nil {
    32  		panic(err)
    33  	}
    34  
    35  	f, err := openFile(input)
    36  	if err != nil {
    37  		panic(err)
    38  	}
    39  
    40  	rowGroup := fileRowGroupOf(f)
    41  
    42  	t := typeOf[T]()
    43  	if c.Schema == nil {
    44  		if t == nil {
    45  			c.Schema = rowGroup.Schema()
    46  		} else {
    47  			c.Schema = schemaOf(dereference(t))
    48  		}
    49  	}
    50  
    51  	r := &GenericReader[T]{
    52  		base: Reader{
    53  			file: reader{
    54  				schema:   c.Schema,
    55  				rowGroup: rowGroup,
    56  			},
    57  		},
    58  	}
    59  
    60  	if !nodesAreEqual(c.Schema, f.schema) {
    61  		r.base.file.rowGroup = convertRowGroupTo(r.base.file.rowGroup, c.Schema)
    62  	}
    63  
    64  	r.base.read.init(r.base.file.schema, r.base.file.rowGroup)
    65  	r.read = readFuncOf[T](t, r.base.file.schema)
    66  	return r
    67  }
    68  
    69  func NewGenericRowGroupReader[T any](rowGroup RowGroup, options ...ReaderOption) *GenericReader[T] {
    70  	c, err := NewReaderConfig(options...)
    71  	if err != nil {
    72  		panic(err)
    73  	}
    74  
    75  	t := typeOf[T]()
    76  	if c.Schema == nil {
    77  		if t == nil {
    78  			c.Schema = rowGroup.Schema()
    79  		} else {
    80  			c.Schema = schemaOf(dereference(t))
    81  		}
    82  	}
    83  
    84  	r := &GenericReader[T]{
    85  		base: Reader{
    86  			file: reader{
    87  				schema:   c.Schema,
    88  				rowGroup: rowGroup,
    89  			},
    90  		},
    91  	}
    92  
    93  	if !nodesAreEqual(c.Schema, rowGroup.Schema()) {
    94  		r.base.file.rowGroup = convertRowGroupTo(r.base.file.rowGroup, c.Schema)
    95  	}
    96  
    97  	r.base.read.init(r.base.file.schema, r.base.file.rowGroup)
    98  	r.read = readFuncOf[T](t, r.base.file.schema)
    99  	return r
   100  }
   101  
   102  func (r *GenericReader[T]) Reset() {
   103  	r.base.Reset()
   104  }
   105  
   106  // Read reads the next rows from the reader into the given rows slice up to len(rows).
   107  //
   108  // The returned values are safe to reuse across Read calls and do not share
   109  // memory with the reader's underlying page buffers.
   110  //
   111  // The method returns the number of rows read and io.EOF when no more rows
   112  // can be read from the reader.
   113  func (r *GenericReader[T]) Read(rows []T) (int, error) {
   114  	return r.read(r, rows)
   115  }
   116  
   117  func (r *GenericReader[T]) ReadRows(rows []Row) (int, error) {
   118  	return r.base.ReadRows(rows)
   119  }
   120  
   121  func (r *GenericReader[T]) Schema() *Schema {
   122  	return r.base.Schema()
   123  }
   124  
   125  func (r *GenericReader[T]) NumRows() int64 {
   126  	return r.base.NumRows()
   127  }
   128  
   129  func (r *GenericReader[T]) SeekToRow(rowIndex int64) error {
   130  	return r.base.SeekToRow(rowIndex)
   131  }
   132  
   133  func (r *GenericReader[T]) Close() error {
   134  	return r.base.Close()
   135  }
   136  
   137  // readRows reads the next rows from the reader into the given rows slice up to len(rows).
   138  //
   139  // The returned values are safe to reuse across readRows calls and do not share
   140  // memory with the reader's underlying page buffers.
   141  //
   142  // The method returns the number of rows read and io.EOF when no more rows
   143  // can be read from the reader.
   144  func (r *GenericReader[T]) readRows(rows []T) (int, error) {
   145  	nRequest := len(rows)
   146  	if cap(r.base.rowbuf) < nRequest {
   147  		r.base.rowbuf = make([]Row, nRequest)
   148  	} else {
   149  		r.base.rowbuf = r.base.rowbuf[:nRequest]
   150  	}
   151  
   152  	var n, nTotal int
   153  	var err error
   154  	for {
   155  		// ReadRows reads the minimum remaining rows in a column page across all columns
   156  		// of the underlying reader, unless the length of the slice passed to it is smaller.
   157  		// In that case, ReadRows will read the number of rows equal to the length of the
   158  		// given slice argument. We limit that length to never be more than requested
   159  		// because sequential reads can cross page boundaries.
   160  		n, err = r.base.ReadRows(r.base.rowbuf[:nRequest-nTotal])
   161  		if n > 0 {
   162  			schema := r.base.Schema()
   163  
   164  			for i, row := range r.base.rowbuf[:n] {
   165  				if err2 := schema.Reconstruct(&rows[nTotal+i], row); err2 != nil {
   166  					return nTotal + i, err2
   167  				}
   168  			}
   169  		}
   170  		nTotal += n
   171  		if n == 0 || nTotal == nRequest || err != nil {
   172  			break
   173  		}
   174  	}
   175  
   176  	return nTotal, err
   177  }
   178  
   179  var (
   180  	_ Rows                = (*GenericReader[any])(nil)
   181  	_ RowReaderWithSchema = (*Reader)(nil)
   182  
   183  	_ Rows                = (*GenericReader[struct{}])(nil)
   184  	_ RowReaderWithSchema = (*GenericReader[struct{}])(nil)
   185  
   186  	_ Rows                = (*GenericReader[map[struct{}]struct{}])(nil)
   187  	_ RowReaderWithSchema = (*GenericReader[map[struct{}]struct{}])(nil)
   188  )
   189  
   190  type readFunc[T any] func(*GenericReader[T], []T) (int, error)
   191  
   192  func readFuncOf[T any](t reflect.Type, schema *Schema) readFunc[T] {
   193  	if t == nil {
   194  		return (*GenericReader[T]).readRows
   195  	}
   196  	switch t.Kind() {
   197  	case reflect.Interface, reflect.Map:
   198  		return (*GenericReader[T]).readRows
   199  
   200  	case reflect.Struct:
   201  		return (*GenericReader[T]).readRows
   202  
   203  	case reflect.Pointer:
   204  		if e := t.Elem(); e.Kind() == reflect.Struct {
   205  			return (*GenericReader[T]).readRows
   206  		}
   207  	}
   208  	panic("cannot create reader for values of type " + t.String())
   209  }
   210  
   211  // Deprecated: A Reader reads Go values from parquet files.
   212  //
   213  // This example showcases a typical use of parquet readers:
   214  //
   215  //	reader := parquet.NewReader(file)
   216  //	rows := []RowType{}
   217  //	for {
   218  //		row := RowType{}
   219  //		err := reader.Read(&row)
   220  //		if err != nil {
   221  //			if err == io.EOF {
   222  //				break
   223  //			}
   224  //			...
   225  //		}
   226  //		rows = append(rows, row)
   227  //	}
   228  //	if err := reader.Close(); err != nil {
   229  //		...
   230  //	}
   231  //
   232  // For programs building with Go 1.18 or later, the GenericReader[T] type
   233  // supersedes this one.
   234  type Reader struct {
   235  	seen     reflect.Type
   236  	file     reader
   237  	read     reader
   238  	rowIndex int64
   239  	rowbuf   []Row
   240  }
   241  
   242  // NewReader constructs a parquet reader reading rows from the given
   243  // io.ReaderAt.
   244  //
   245  // In order to read parquet rows, the io.ReaderAt must be converted to a
   246  // parquet.File. If r is already a parquet.File it is used directly; otherwise,
   247  // the io.ReaderAt value is expected to either have a `Size() int64` method or
   248  // implement io.Seeker in order to determine its size.
   249  //
   250  // The function panics if the reader configuration is invalid. Programs that
   251  // cannot guarantee the validity of the options passed to NewReader should
   252  // construct the reader configuration independently prior to calling this
   253  // function:
   254  //
   255  //	config, err := parquet.NewReaderConfig(options...)
   256  //	if err != nil {
   257  //		// handle the configuration error
   258  //		...
   259  //	} else {
   260  //		// this call to create a reader is guaranteed not to panic
   261  //		reader := parquet.NewReader(input, config)
   262  //		...
   263  //	}
   264  func NewReader(input io.ReaderAt, options ...ReaderOption) *Reader {
   265  	c, err := NewReaderConfig(options...)
   266  	if err != nil {
   267  		panic(err)
   268  	}
   269  
   270  	f, err := openFile(input)
   271  	if err != nil {
   272  		panic(err)
   273  	}
   274  
   275  	r := &Reader{
   276  		file: reader{
   277  			schema:   f.schema,
   278  			rowGroup: fileRowGroupOf(f),
   279  		},
   280  	}
   281  
   282  	if c.Schema != nil {
   283  		r.file.schema = c.Schema
   284  		r.file.rowGroup = convertRowGroupTo(r.file.rowGroup, c.Schema)
   285  	}
   286  
   287  	r.read.init(r.file.schema, r.file.rowGroup)
   288  	return r
   289  }
   290  
   291  func openFile(input io.ReaderAt) (*File, error) {
   292  	f, _ := input.(*File)
   293  	if f != nil {
   294  		return f, nil
   295  	}
   296  	n, err := sizeOf(input)
   297  	if err != nil {
   298  		return nil, err
   299  	}
   300  	return OpenFile(input, n)
   301  }
   302  
   303  func fileRowGroupOf(f *File) RowGroup {
   304  	switch rowGroups := f.RowGroups(); len(rowGroups) {
   305  	case 0:
   306  		return newEmptyRowGroup(f.Schema())
   307  	case 1:
   308  		return rowGroups[0]
   309  	default:
   310  		// TODO: should we attempt to merge the row groups via MergeRowGroups
   311  		// to preserve the global order of sorting columns within the file?
   312  		return newMultiRowGroup(f.config.ReadMode, rowGroups...)
   313  	}
   314  }
   315  
   316  // NewRowGroupReader constructs a new Reader which reads rows from the RowGroup
   317  // passed as argument.
   318  func NewRowGroupReader(rowGroup RowGroup, options ...ReaderOption) *Reader {
   319  	c, err := NewReaderConfig(options...)
   320  	if err != nil {
   321  		panic(err)
   322  	}
   323  
   324  	if c.Schema != nil {
   325  		rowGroup = convertRowGroupTo(rowGroup, c.Schema)
   326  	}
   327  
   328  	r := &Reader{
   329  		file: reader{
   330  			schema:   rowGroup.Schema(),
   331  			rowGroup: rowGroup,
   332  		},
   333  	}
   334  
   335  	r.read.init(r.file.schema, r.file.rowGroup)
   336  	return r
   337  }
   338  
   339  func convertRowGroupTo(rowGroup RowGroup, schema *Schema) RowGroup {
   340  	if rowGroupSchema := rowGroup.Schema(); !nodesAreEqual(schema, rowGroupSchema) {
   341  		conv, err := Convert(schema, rowGroupSchema)
   342  		if err != nil {
   343  			// TODO: this looks like something we should not be panicking on,
   344  			// but the current NewReader API does not offer a mechanism to
   345  			// report errors.
   346  			panic(err)
   347  		}
   348  		rowGroup = ConvertRowGroup(rowGroup, conv)
   349  	}
   350  	return rowGroup
   351  }
   352  
   353  func sizeOf(r io.ReaderAt) (int64, error) {
   354  	switch f := r.(type) {
   355  	case interface{ Size() int64 }:
   356  		return f.Size(), nil
   357  	case io.Seeker:
   358  		off, err := f.Seek(0, io.SeekCurrent)
   359  		if err != nil {
   360  			return 0, err
   361  		}
   362  		end, err := f.Seek(0, io.SeekEnd)
   363  		if err != nil {
   364  			return 0, err
   365  		}
   366  		_, err = f.Seek(off, io.SeekStart)
   367  		return end, err
   368  	default:
   369  		return 0, fmt.Errorf("cannot determine length of %T", r)
   370  	}
   371  }
   372  
   373  // Reset repositions the reader at the beginning of the underlying parquet file.
   374  func (r *Reader) Reset() {
   375  	r.file.Reset()
   376  	r.read.Reset()
   377  	r.rowIndex = 0
   378  	clearRows(r.rowbuf)
   379  }
   380  
   381  // Read reads the next row from r. The type of the row must match the schema
   382  // of the underlying parquet file or an error will be returned.
   383  //
   384  // The method returns io.EOF when no more rows can be read from r.
   385  func (r *Reader) Read(row interface{}) error {
   386  	if rowType := dereference(reflect.TypeOf(row)); rowType.Kind() == reflect.Struct {
   387  		if r.seen != rowType {
   388  			if err := r.updateReadSchema(rowType); err != nil {
   389  				return fmt.Errorf("cannot read parquet row into go value of type %T: %w", row, err)
   390  			}
   391  		}
   392  	}
   393  
   394  	if err := r.read.SeekToRow(r.rowIndex); err != nil {
   395  		if errors.Is(err, io.ErrClosedPipe) {
   396  			return io.EOF
   397  		}
   398  		return fmt.Errorf("seeking reader to row %d: %w", r.rowIndex, err)
   399  	}
   400  
   401  	if cap(r.rowbuf) == 0 {
   402  		r.rowbuf = make([]Row, 1)
   403  	} else {
   404  		r.rowbuf = r.rowbuf[:1]
   405  	}
   406  
   407  	n, err := r.read.ReadRows(r.rowbuf[:])
   408  	if n == 0 {
   409  		return err
   410  	}
   411  
   412  	r.rowIndex++
   413  	return r.read.schema.Reconstruct(row, r.rowbuf[0])
   414  }
   415  
   416  func (r *Reader) updateReadSchema(rowType reflect.Type) error {
   417  	schema := schemaOf(rowType)
   418  
   419  	if nodesAreEqual(schema, r.file.schema) {
   420  		r.read.init(schema, r.file.rowGroup)
   421  	} else {
   422  		conv, err := Convert(schema, r.file.schema)
   423  		if err != nil {
   424  			return err
   425  		}
   426  		r.read.init(schema, ConvertRowGroup(r.file.rowGroup, conv))
   427  	}
   428  
   429  	r.seen = rowType
   430  	return nil
   431  }
   432  
   433  // ReadRows reads the next rows from r into the given Row buffer.
   434  //
   435  // The returned values are laid out in the order expected by the
   436  // parquet.(*Schema).Reconstruct method.
   437  //
   438  // The method returns io.EOF when no more rows can be read from r.
   439  func (r *Reader) ReadRows(rows []Row) (int, error) {
   440  	if err := r.file.SeekToRow(r.rowIndex); err != nil {
   441  		return 0, err
   442  	}
   443  	n, err := r.file.ReadRows(rows)
   444  	r.rowIndex += int64(n)
   445  	return n, err
   446  }
   447  
   448  // Schema returns the schema of rows read by r.
   449  func (r *Reader) Schema() *Schema { return r.file.schema }
   450  
   451  // NumRows returns the number of rows that can be read from r.
   452  func (r *Reader) NumRows() int64 { return r.file.rowGroup.NumRows() }
   453  
   454  // SeekToRow positions r at the given row index.
   455  func (r *Reader) SeekToRow(rowIndex int64) error {
   456  	if err := r.file.SeekToRow(rowIndex); err != nil {
   457  		return err
   458  	}
   459  	r.rowIndex = rowIndex
   460  	return nil
   461  }
   462  
   463  // Close closes the reader, preventing more rows from being read.
   464  func (r *Reader) Close() error {
   465  	if err := r.read.Close(); err != nil {
   466  		return err
   467  	}
   468  	if err := r.file.Close(); err != nil {
   469  		return err
   470  	}
   471  	return nil
   472  }
   473  
   474  // reader is a subtype used in the implementation of Reader to support the two
   475  // use cases of either reading rows calling the ReadRow method (where full rows
   476  // are read from the underlying parquet file), or calling the Read method to
   477  // read rows into Go values, potentially doing partial reads on a subset of the
   478  // columns due to using a converted row group view.
   479  type reader struct {
   480  	schema   *Schema
   481  	rowGroup RowGroup
   482  	rows     Rows
   483  	rowIndex int64
   484  }
   485  
   486  func (r *reader) init(schema *Schema, rowGroup RowGroup) {
   487  	r.schema = schema
   488  	r.rowGroup = rowGroup
   489  	r.Reset()
   490  }
   491  
   492  func (r *reader) Reset() {
   493  	r.rowIndex = 0
   494  
   495  	if rows, ok := r.rows.(interface{ Reset() }); ok {
   496  		// This optimization works for the common case where the underlying type
   497  		// of the Rows instance is rowGroupRows, which should be true in most
   498  		// cases since even external implementations of the RowGroup interface
   499  		// can construct values of this type via the NewRowGroupRowReader
   500  		// function.
   501  		//
   502  		// Foreign implementations of the Rows interface may also define a Reset
   503  		// method in order to participate in this optimization.
   504  		rows.Reset()
   505  		return
   506  	}
   507  
   508  	if r.rows != nil {
   509  		r.rows.Close()
   510  		r.rows = nil
   511  	}
   512  }
   513  
   514  func (r *reader) ReadRows(rows []Row) (int, error) {
   515  	if r.rowGroup == nil {
   516  		return 0, io.EOF
   517  	}
   518  	if r.rows == nil {
   519  		r.rows = r.rowGroup.Rows()
   520  		if r.rowIndex > 0 {
   521  			if err := r.rows.SeekToRow(r.rowIndex); err != nil {
   522  				return 0, err
   523  			}
   524  		}
   525  	}
   526  	n, err := r.rows.ReadRows(rows)
   527  	r.rowIndex += int64(n)
   528  	return n, err
   529  }
   530  
   531  func (r *reader) SeekToRow(rowIndex int64) error {
   532  	if r.rowGroup == nil {
   533  		return io.ErrClosedPipe
   534  	}
   535  	if rowIndex != r.rowIndex {
   536  		if r.rows != nil {
   537  			if err := r.rows.SeekToRow(rowIndex); err != nil {
   538  				return err
   539  			}
   540  		}
   541  		r.rowIndex = rowIndex
   542  	}
   543  	return nil
   544  }
   545  
   546  func (r *reader) Close() (err error) {
   547  	r.rowGroup = nil
   548  	if r.rows != nil {
   549  		err = r.rows.Close()
   550  	}
   551  	return err
   552  }
   553  
   554  var (
   555  	_ Rows                = (*Reader)(nil)
   556  	_ RowReaderWithSchema = (*Reader)(nil)
   557  
   558  	_ RowReader = (*reader)(nil)
   559  	_ RowSeeker = (*reader)(nil)
   560  )