github.com/apache/arrow/go/v10@v10.0.1/parquet/pqarrow/column_readers.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  	"encoding/binary"
    21  	"fmt"
    22  	"reflect"
    23  	"sync"
    24  	"sync/atomic"
    25  	"time"
    26  	"unsafe"
    27  
    28  	"github.com/apache/arrow/go/v10/arrow"
    29  	"github.com/apache/arrow/go/v10/arrow/array"
    30  	"github.com/apache/arrow/go/v10/arrow/bitutil"
    31  	"github.com/apache/arrow/go/v10/arrow/decimal128"
    32  	"github.com/apache/arrow/go/v10/arrow/memory"
    33  	"github.com/apache/arrow/go/v10/internal/utils"
    34  	"github.com/apache/arrow/go/v10/parquet"
    35  	"github.com/apache/arrow/go/v10/parquet/file"
    36  	"github.com/apache/arrow/go/v10/parquet/schema"
    37  	"golang.org/x/sync/errgroup"
    38  	"golang.org/x/xerrors"
    39  )
    40  
    41  // column reader for leaf columns (non-nested)
    42  type leafReader struct {
    43  	out       *arrow.Chunked
    44  	rctx      *readerCtx
    45  	field     *arrow.Field
    46  	input     *columnIterator
    47  	descr     *schema.Column
    48  	recordRdr file.RecordReader
    49  	props     ArrowReadProperties
    50  
    51  	refCount int64
    52  }
    53  
    54  func newLeafReader(rctx *readerCtx, field *arrow.Field, input *columnIterator, leafInfo file.LevelInfo, props ArrowReadProperties, bufferPool *sync.Pool) (*ColumnReader, error) {
    55  	ret := &leafReader{
    56  		rctx:      rctx,
    57  		field:     field,
    58  		input:     input,
    59  		descr:     input.Descr(),
    60  		recordRdr: file.NewRecordReader(input.Descr(), leafInfo, field.Type.ID() == arrow.DICTIONARY, rctx.mem, bufferPool),
    61  		props:     props,
    62  		refCount:  1,
    63  	}
    64  	err := ret.nextRowGroup()
    65  	return &ColumnReader{ret}, err
    66  }
    67  
    68  func (lr *leafReader) Retain() {
    69  	atomic.AddInt64(&lr.refCount, 1)
    70  }
    71  
    72  func (lr *leafReader) Release() {
    73  	if atomic.AddInt64(&lr.refCount, -1) == 0 {
    74  		if lr.out != nil {
    75  			lr.out.Release()
    76  			lr.out = nil
    77  		}
    78  		if lr.recordRdr != nil {
    79  			lr.recordRdr.Release()
    80  			lr.recordRdr = nil
    81  		}
    82  	}
    83  }
    84  
    85  func (lr *leafReader) GetDefLevels() ([]int16, error) {
    86  	return lr.recordRdr.DefLevels()[:int(lr.recordRdr.LevelsPos())], nil
    87  }
    88  
    89  func (lr *leafReader) GetRepLevels() ([]int16, error) {
    90  	return lr.recordRdr.RepLevels()[:int(lr.recordRdr.LevelsPos())], nil
    91  }
    92  
    93  func (lr *leafReader) IsOrHasRepeatedChild() bool { return false }
    94  
    95  func (lr *leafReader) LoadBatch(nrecords int64) (err error) {
    96  	if lr.out != nil {
    97  		lr.out.Release()
    98  		lr.out = nil
    99  	}
   100  	lr.recordRdr.Reset()
   101  
   102  	if err := lr.recordRdr.Reserve(nrecords); err != nil {
   103  		return err
   104  	}
   105  	for nrecords > 0 {
   106  		if !lr.recordRdr.HasMore() {
   107  			break
   108  		}
   109  		numRead, err := lr.recordRdr.ReadRecords(nrecords)
   110  		if err != nil {
   111  			return err
   112  		}
   113  		nrecords -= numRead
   114  		if numRead == 0 {
   115  			if err = lr.nextRowGroup(); err != nil {
   116  				return err
   117  			}
   118  		}
   119  	}
   120  	lr.out, err = transferColumnData(lr.recordRdr, lr.field.Type, lr.descr, lr.rctx.mem)
   121  	return
   122  }
   123  
   124  func (lr *leafReader) BuildArray(_ int64) (*arrow.Chunked, error) {
   125  	return lr.out, nil
   126  }
   127  
   128  func (lr *leafReader) Field() *arrow.Field { return lr.field }
   129  
   130  func (lr *leafReader) nextRowGroup() error {
   131  	pr, err := lr.input.NextChunk()
   132  	if err != nil {
   133  		return err
   134  	}
   135  	lr.recordRdr.SetPageReader(pr)
   136  	return nil
   137  }
   138  
   139  // column reader for struct arrays, has readers for each child which could
   140  // themselves be nested or leaf columns.
   141  type structReader struct {
   142  	rctx             *readerCtx
   143  	filtered         *arrow.Field
   144  	levelInfo        file.LevelInfo
   145  	children         []*ColumnReader
   146  	defRepLevelChild *ColumnReader
   147  	hasRepeatedChild bool
   148  	props            ArrowReadProperties
   149  
   150  	refCount int64
   151  }
   152  
   153  func (sr *structReader) Retain() {
   154  	atomic.AddInt64(&sr.refCount, 1)
   155  }
   156  
   157  func (sr *structReader) Release() {
   158  	if atomic.AddInt64(&sr.refCount, -1) == 0 {
   159  		if sr.defRepLevelChild != nil {
   160  			sr.defRepLevelChild.Release()
   161  			sr.defRepLevelChild = nil
   162  		}
   163  		for _, c := range sr.children {
   164  			c.Release()
   165  		}
   166  		sr.children = nil
   167  	}
   168  }
   169  
   170  func newStructReader(rctx *readerCtx, filtered *arrow.Field, levelInfo file.LevelInfo, children []*ColumnReader, props ArrowReadProperties) *ColumnReader {
   171  	// there could be a mix of children some might be repeated and some might not be
   172  	// if possible use one that isn't since that will be guaranteed to have the least
   173  	// number of levels to reconstruct a nullable bitmap
   174  	var result *ColumnReader
   175  	for _, child := range children {
   176  		if !child.IsOrHasRepeatedChild() {
   177  			result = child
   178  		}
   179  	}
   180  
   181  	ret := &structReader{
   182  		rctx:      rctx,
   183  		filtered:  filtered,
   184  		levelInfo: levelInfo,
   185  		children:  children,
   186  		props:     props,
   187  		refCount:  1,
   188  	}
   189  	if result != nil {
   190  		ret.defRepLevelChild = result
   191  		ret.hasRepeatedChild = false
   192  	} else {
   193  		ret.defRepLevelChild = children[0]
   194  		ret.hasRepeatedChild = true
   195  	}
   196  	ret.defRepLevelChild.Retain()
   197  	return &ColumnReader{ret}
   198  }
   199  
   200  func (sr *structReader) IsOrHasRepeatedChild() bool { return sr.hasRepeatedChild }
   201  
   202  func (sr *structReader) GetDefLevels() ([]int16, error) {
   203  	if len(sr.children) == 0 {
   204  		return nil, xerrors.New("struct raeder has no children")
   205  	}
   206  
   207  	// this method should only be called when this struct or one of its parents
   208  	// are optional/repeated or has a repeated child
   209  	// meaning all children must have rep/def levels associated with them
   210  	return sr.defRepLevelChild.GetDefLevels()
   211  }
   212  
   213  func (sr *structReader) GetRepLevels() ([]int16, error) {
   214  	if len(sr.children) == 0 {
   215  		return nil, xerrors.New("struct raeder has no children")
   216  	}
   217  
   218  	// this method should only be called when this struct or one of its parents
   219  	// are optional/repeated or has a repeated child
   220  	// meaning all children must have rep/def levels associated with them
   221  	return sr.defRepLevelChild.GetRepLevels()
   222  }
   223  
   224  func (sr *structReader) LoadBatch(nrecords int64) error {
   225  	// Load batches in parallel
   226  	// When reading structs with large numbers of columns, the serial load is very slow.
   227  	// This is especially true when reading Cloud Storage. Loading concurrently
   228  	// greatly improves performance.
   229  	g := new(errgroup.Group)
   230  	if !sr.props.Parallel {
   231  		g.SetLimit(1)
   232  	}
   233  	for _, rdr := range sr.children {
   234  		rdr := rdr
   235  		g.Go(func() error {
   236  			return rdr.LoadBatch(nrecords)
   237  		})
   238  	}
   239  
   240  	return g.Wait()
   241  }
   242  
   243  func (sr *structReader) Field() *arrow.Field { return sr.filtered }
   244  
   245  func (sr *structReader) BuildArray(lenBound int64) (*arrow.Chunked, error) {
   246  	validityIO := file.ValidityBitmapInputOutput{
   247  		ReadUpperBound: lenBound,
   248  		Read:           lenBound,
   249  	}
   250  
   251  	var nullBitmap *memory.Buffer
   252  
   253  	if sr.hasRepeatedChild {
   254  		nullBitmap = memory.NewResizableBuffer(sr.rctx.mem)
   255  		nullBitmap.Resize(int(bitutil.BytesForBits(lenBound)))
   256  		validityIO.ValidBits = nullBitmap.Bytes()
   257  		defLevels, err := sr.GetDefLevels()
   258  		if err != nil {
   259  			return nil, err
   260  		}
   261  		repLevels, err := sr.GetRepLevels()
   262  		if err != nil {
   263  			return nil, err
   264  		}
   265  
   266  		if err := file.DefRepLevelsToBitmap(defLevels, repLevels, sr.levelInfo, &validityIO); err != nil {
   267  			return nil, err
   268  		}
   269  
   270  	} else if sr.filtered.Nullable {
   271  		nullBitmap = memory.NewResizableBuffer(sr.rctx.mem)
   272  		nullBitmap.Resize(int(bitutil.BytesForBits(lenBound)))
   273  		validityIO.ValidBits = nullBitmap.Bytes()
   274  		defLevels, err := sr.GetDefLevels()
   275  		if err != nil {
   276  			return nil, err
   277  		}
   278  
   279  		file.DefLevelsToBitmap(defLevels, sr.levelInfo, &validityIO)
   280  	}
   281  
   282  	if nullBitmap != nil {
   283  		nullBitmap.Resize(int(bitutil.BytesForBits(validityIO.Read)))
   284  	}
   285  
   286  	childArrData := make([]arrow.ArrayData, 0)
   287  	// gather children arrays and def levels
   288  	for _, child := range sr.children {
   289  		field, err := child.BuildArray(validityIO.Read)
   290  		if err != nil {
   291  			return nil, err
   292  		}
   293  		arrdata, err := chunksToSingle(field)
   294  		if err != nil {
   295  			return nil, err
   296  		}
   297  		childArrData = append(childArrData, arrdata)
   298  	}
   299  
   300  	if !sr.filtered.Nullable && !sr.hasRepeatedChild {
   301  		validityIO.Read = int64(childArrData[0].Len())
   302  	}
   303  
   304  	buffers := make([]*memory.Buffer, 1)
   305  	if validityIO.NullCount > 0 {
   306  		buffers[0] = nullBitmap
   307  	}
   308  
   309  	data := array.NewData(sr.filtered.Type, int(validityIO.Read), buffers, childArrData, int(validityIO.NullCount), 0)
   310  	defer data.Release()
   311  	arr := array.MakeFromData(data)
   312  	defer arr.Release()
   313  	return arrow.NewChunked(sr.filtered.Type, []arrow.Array{arr}), nil
   314  }
   315  
   316  // column reader for repeated columns specifically for list arrays
   317  type listReader struct {
   318  	rctx     *readerCtx
   319  	field    *arrow.Field
   320  	info     file.LevelInfo
   321  	itemRdr  *ColumnReader
   322  	props    ArrowReadProperties
   323  	refCount int64
   324  }
   325  
   326  func newListReader(rctx *readerCtx, field *arrow.Field, info file.LevelInfo, childRdr *ColumnReader, props ArrowReadProperties) *ColumnReader {
   327  	childRdr.Retain()
   328  	return &ColumnReader{&listReader{rctx, field, info, childRdr, props, 1}}
   329  }
   330  
   331  func (lr *listReader) Retain() {
   332  	atomic.AddInt64(&lr.refCount, 1)
   333  }
   334  
   335  func (lr *listReader) Release() {
   336  	if atomic.AddInt64(&lr.refCount, -1) == 0 {
   337  		if lr.itemRdr != nil {
   338  			lr.itemRdr.Release()
   339  			lr.itemRdr = nil
   340  		}
   341  	}
   342  }
   343  
   344  func (lr *listReader) GetDefLevels() ([]int16, error) {
   345  	return lr.itemRdr.GetDefLevels()
   346  }
   347  
   348  func (lr *listReader) GetRepLevels() ([]int16, error) {
   349  	return lr.itemRdr.GetRepLevels()
   350  }
   351  
   352  func (lr *listReader) Field() *arrow.Field { return lr.field }
   353  
   354  func (lr *listReader) IsOrHasRepeatedChild() bool { return true }
   355  
   356  func (lr *listReader) LoadBatch(nrecords int64) error {
   357  	return lr.itemRdr.LoadBatch(nrecords)
   358  }
   359  
   360  func (lr *listReader) BuildArray(lenBound int64) (*arrow.Chunked, error) {
   361  	var (
   362  		defLevels      []int16
   363  		repLevels      []int16
   364  		err            error
   365  		validityBuffer *memory.Buffer
   366  	)
   367  
   368  	if defLevels, err = lr.itemRdr.GetDefLevels(); err != nil {
   369  		return nil, err
   370  	}
   371  	if repLevels, err = lr.itemRdr.GetRepLevels(); err != nil {
   372  		return nil, err
   373  	}
   374  
   375  	validityIO := file.ValidityBitmapInputOutput{ReadUpperBound: lenBound}
   376  	if lr.field.Nullable {
   377  		validityBuffer = memory.NewResizableBuffer(lr.rctx.mem)
   378  		validityBuffer.Resize(int(bitutil.BytesForBits(lenBound)))
   379  		defer validityBuffer.Release()
   380  		validityIO.ValidBits = validityBuffer.Bytes()
   381  	}
   382  	offsetsBuffer := memory.NewResizableBuffer(lr.rctx.mem)
   383  	offsetsBuffer.Resize(arrow.Int32Traits.BytesRequired(int(lenBound) + 1))
   384  	defer offsetsBuffer.Release()
   385  
   386  	offsetData := arrow.Int32Traits.CastFromBytes(offsetsBuffer.Bytes())
   387  	if err = file.DefRepLevelsToListInfo(defLevels, repLevels, lr.info, &validityIO, offsetData); err != nil {
   388  		return nil, err
   389  	}
   390  
   391  	// if the parent (itemRdr) has nulls and is a nested type like list
   392  	// then we need BuildArray to account for that with the number of
   393  	// definition levels when building out the bitmap. So the upper bound
   394  	// to make sure we have the space for is the worst case scenario,
   395  	// the upper bound is the value of the last offset + the nullcount
   396  	arr, err := lr.itemRdr.BuildArray(int64(offsetData[int(validityIO.Read)]) + validityIO.NullCount)
   397  	if err != nil {
   398  		return nil, err
   399  	}
   400  
   401  	// resize to actual number of elems returned
   402  	offsetsBuffer.Resize(arrow.Int32Traits.BytesRequired(int(validityIO.Read) + 1))
   403  	if validityBuffer != nil {
   404  		validityBuffer.Resize(int(bitutil.BytesForBits(validityIO.Read)))
   405  	}
   406  
   407  	item, err := chunksToSingle(arr)
   408  	if err != nil {
   409  		return nil, err
   410  	}
   411  	defer item.Release()
   412  
   413  	buffers := []*memory.Buffer{nil, offsetsBuffer}
   414  	if validityIO.NullCount > 0 {
   415  		buffers[0] = validityBuffer
   416  	}
   417  
   418  	data := array.NewData(lr.field.Type, int(validityIO.Read), buffers, []arrow.ArrayData{item}, int(validityIO.NullCount), 0)
   419  	defer data.Release()
   420  	if lr.field.Type.ID() == arrow.FIXED_SIZE_LIST {
   421  		defer data.Buffers()[1].Release()
   422  		listSize := lr.field.Type.(*arrow.FixedSizeListType).Len()
   423  		for x := 1; x < data.Len(); x++ {
   424  			size := offsetData[x] - offsetData[x-1]
   425  			if size != listSize {
   426  				return nil, fmt.Errorf("expected all lists to be of size=%d, but index %d had size=%d", listSize, x, size)
   427  			}
   428  		}
   429  		data.Buffers()[1] = nil
   430  	}
   431  	out := array.MakeFromData(data)
   432  	defer out.Release()
   433  	return arrow.NewChunked(lr.field.Type, []arrow.Array{out}), nil
   434  }
   435  
   436  // column reader logic for fixed size lists instead of variable length ones.
   437  type fixedSizeListReader struct {
   438  	listReader
   439  }
   440  
   441  func newFixedSizeListReader(rctx *readerCtx, field *arrow.Field, info file.LevelInfo, childRdr *ColumnReader, props ArrowReadProperties) *ColumnReader {
   442  	childRdr.Retain()
   443  	return &ColumnReader{&fixedSizeListReader{listReader{rctx, field, info, childRdr, props, 1}}}
   444  }
   445  
   446  // helper function to combine chunks into a single array.
   447  //
   448  // nested data conversion for chunked array outputs not yet implemented
   449  func chunksToSingle(chunked *arrow.Chunked) (arrow.ArrayData, error) {
   450  	switch len(chunked.Chunks()) {
   451  	case 0:
   452  		return array.NewData(chunked.DataType(), 0, []*memory.Buffer{nil, nil}, nil, 0, 0), nil
   453  	case 1:
   454  		return chunked.Chunk(0).Data(), nil
   455  	default: // if an item reader yields a chunked array, this is not yet implemented
   456  		return nil, xerrors.New("not implemented")
   457  	}
   458  }
   459  
   460  // create a chunked arrow array from the raw record data
   461  func transferColumnData(rdr file.RecordReader, valueType arrow.DataType, descr *schema.Column, mem memory.Allocator) (*arrow.Chunked, error) {
   462  	var data arrow.ArrayData
   463  	switch valueType.ID() {
   464  	// case arrow.DICTIONARY:
   465  	case arrow.NULL:
   466  		return arrow.NewChunked(arrow.Null, []arrow.Array{array.NewNull(rdr.ValuesWritten())}), nil
   467  	case arrow.INT32, arrow.INT64, arrow.FLOAT32, arrow.FLOAT64:
   468  		data = transferZeroCopy(rdr, valueType) // can just reference the raw data without copying
   469  	case arrow.BOOL:
   470  		data = transferBool(rdr)
   471  	case arrow.UINT8,
   472  		arrow.UINT16,
   473  		arrow.UINT32,
   474  		arrow.UINT64,
   475  		arrow.INT8,
   476  		arrow.INT16,
   477  		arrow.DATE32,
   478  		arrow.TIME32,
   479  		arrow.TIME64:
   480  		data = transferInt(rdr, valueType)
   481  	case arrow.DATE64:
   482  		data = transferDate64(rdr, valueType)
   483  	case arrow.FIXED_SIZE_BINARY, arrow.BINARY, arrow.STRING:
   484  		return transferBinary(rdr, valueType), nil
   485  	case arrow.DECIMAL:
   486  		switch descr.PhysicalType() {
   487  		case parquet.Types.Int32, parquet.Types.Int64:
   488  			data = transferDecimalInteger(rdr, valueType)
   489  		case parquet.Types.ByteArray, parquet.Types.FixedLenByteArray:
   490  			return transferDecimalBytes(rdr.(file.BinaryRecordReader), valueType)
   491  		default:
   492  			return nil, xerrors.New("physical type for decimal128 must be int32, int64, bytearray or fixed len byte array")
   493  		}
   494  	case arrow.TIMESTAMP:
   495  		tstype := valueType.(*arrow.TimestampType)
   496  		switch tstype.Unit {
   497  		case arrow.Millisecond, arrow.Microsecond:
   498  			data = transferZeroCopy(rdr, valueType)
   499  		case arrow.Nanosecond:
   500  			if descr.PhysicalType() == parquet.Types.Int96 {
   501  				data = transferInt96(rdr, valueType)
   502  			} else {
   503  				data = transferZeroCopy(rdr, valueType)
   504  			}
   505  		default:
   506  			return nil, xerrors.New("time unit not supported")
   507  		}
   508  	default:
   509  		return nil, fmt.Errorf("no support for reading columns of type: %s", valueType.Name())
   510  	}
   511  
   512  	defer data.Release()
   513  	arr := array.MakeFromData(data)
   514  	defer arr.Release()
   515  	return arrow.NewChunked(valueType, []arrow.Array{arr}), nil
   516  }
   517  
   518  func transferZeroCopy(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData {
   519  	bitmap := rdr.ReleaseValidBits()
   520  	values := rdr.ReleaseValues()
   521  	defer func() {
   522  		if bitmap != nil {
   523  			bitmap.Release()
   524  		}
   525  		if values != nil {
   526  			values.Release()
   527  		}
   528  	}()
   529  
   530  	return array.NewData(dt, rdr.ValuesWritten(), []*memory.Buffer{
   531  		bitmap, values}, nil, int(rdr.NullCount()), 0)
   532  }
   533  
   534  func transferBinary(rdr file.RecordReader, dt arrow.DataType) *arrow.Chunked {
   535  	brdr := rdr.(file.BinaryRecordReader)
   536  	chunks := brdr.GetBuilderChunks()
   537  	if dt == arrow.BinaryTypes.String {
   538  		// convert chunks from binary to string without copying data,
   539  		// just changing the interpretation of the metadata
   540  		for idx := range chunks {
   541  			chunks[idx] = array.MakeFromData(chunks[idx].Data())
   542  			defer chunks[idx].Data().Release()
   543  			defer chunks[idx].Release()
   544  		}
   545  	}
   546  	return arrow.NewChunked(dt, chunks)
   547  }
   548  
   549  func transferInt(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData {
   550  	var (
   551  		output reflect.Value
   552  	)
   553  
   554  	signed := true
   555  	// create buffer for proper type since parquet only has int32 and int64
   556  	// physical representations, but we want the correct type representation
   557  	// for Arrow's in memory buffer.
   558  	data := make([]byte, rdr.ValuesWritten()*int(bitutil.BytesForBits(int64(dt.(arrow.FixedWidthDataType).BitWidth()))))
   559  	switch dt.ID() {
   560  	case arrow.INT8:
   561  		output = reflect.ValueOf(arrow.Int8Traits.CastFromBytes(data))
   562  	case arrow.UINT8:
   563  		signed = false
   564  		output = reflect.ValueOf(arrow.Uint8Traits.CastFromBytes(data))
   565  	case arrow.INT16:
   566  		output = reflect.ValueOf(arrow.Int16Traits.CastFromBytes(data))
   567  	case arrow.UINT16:
   568  		signed = false
   569  		output = reflect.ValueOf(arrow.Uint16Traits.CastFromBytes(data))
   570  	case arrow.UINT32:
   571  		signed = false
   572  		output = reflect.ValueOf(arrow.Uint32Traits.CastFromBytes(data))
   573  	case arrow.UINT64:
   574  		signed = false
   575  		output = reflect.ValueOf(arrow.Uint64Traits.CastFromBytes(data))
   576  	case arrow.DATE32:
   577  		output = reflect.ValueOf(arrow.Date32Traits.CastFromBytes(data))
   578  	case arrow.TIME32:
   579  		output = reflect.ValueOf(arrow.Time32Traits.CastFromBytes(data))
   580  	case arrow.TIME64:
   581  		output = reflect.ValueOf(arrow.Time64Traits.CastFromBytes(data))
   582  	}
   583  
   584  	length := rdr.ValuesWritten()
   585  	// copy the values semantically with the correct types
   586  	switch rdr.Type() {
   587  	case parquet.Types.Int32:
   588  		values := arrow.Int32Traits.CastFromBytes(rdr.Values())
   589  		if signed {
   590  			for idx, v := range values[:length] {
   591  				output.Index(idx).SetInt(int64(v))
   592  			}
   593  		} else {
   594  			for idx, v := range values[:length] {
   595  				output.Index(idx).SetUint(uint64(v))
   596  			}
   597  		}
   598  	case parquet.Types.Int64:
   599  		values := arrow.Int64Traits.CastFromBytes(rdr.Values())
   600  		if signed {
   601  			for idx, v := range values[:length] {
   602  				output.Index(idx).SetInt(v)
   603  			}
   604  		} else {
   605  			for idx, v := range values[:length] {
   606  				output.Index(idx).SetUint(uint64(v))
   607  			}
   608  		}
   609  	}
   610  
   611  	bitmap := rdr.ReleaseValidBits()
   612  	if bitmap != nil {
   613  		defer bitmap.Release()
   614  	}
   615  
   616  	return array.NewData(dt, rdr.ValuesWritten(), []*memory.Buffer{
   617  		bitmap, memory.NewBufferBytes(data),
   618  	}, nil, int(rdr.NullCount()), 0)
   619  }
   620  
   621  func transferBool(rdr file.RecordReader) arrow.ArrayData {
   622  	// TODO(mtopol): optimize this so we don't convert bitmap to []bool back to bitmap
   623  	length := rdr.ValuesWritten()
   624  	data := make([]byte, int(bitutil.BytesForBits(int64(length))))
   625  	bytedata := rdr.Values()
   626  	values := *(*[]bool)(unsafe.Pointer(&bytedata))
   627  
   628  	for idx, v := range values[:length] {
   629  		if v {
   630  			bitutil.SetBit(data, idx)
   631  		}
   632  	}
   633  
   634  	bitmap := rdr.ReleaseValidBits()
   635  	if bitmap != nil {
   636  		defer bitmap.Release()
   637  	}
   638  	return array.NewData(&arrow.BooleanType{}, length, []*memory.Buffer{
   639  		bitmap, memory.NewBufferBytes(data),
   640  	}, nil, int(rdr.NullCount()), 0)
   641  }
   642  
   643  var milliPerDay = time.Duration(24 * time.Hour).Milliseconds()
   644  
   645  // parquet equivalent for date64 is a 32-bit integer of the number of days
   646  // since the epoch. Convert each value to milliseconds for date64
   647  func transferDate64(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData {
   648  	length := rdr.ValuesWritten()
   649  	values := arrow.Int32Traits.CastFromBytes(rdr.Values())
   650  
   651  	data := make([]byte, arrow.Int64Traits.BytesRequired(length))
   652  	out := arrow.Int64Traits.CastFromBytes(data)
   653  	for idx, val := range values[:length] {
   654  		out[idx] = int64(val) * milliPerDay
   655  	}
   656  
   657  	bitmap := rdr.ReleaseValidBits()
   658  	if bitmap != nil {
   659  		defer bitmap.Release()
   660  	}
   661  	return array.NewData(dt, length, []*memory.Buffer{
   662  		bitmap, memory.NewBufferBytes(data),
   663  	}, nil, int(rdr.NullCount()), 0)
   664  }
   665  
   666  // coerce int96 to nanosecond timestamp
   667  func transferInt96(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData {
   668  	length := rdr.ValuesWritten()
   669  	values := parquet.Int96Traits.CastFromBytes(rdr.Values())
   670  
   671  	data := make([]byte, arrow.Int64SizeBytes*length)
   672  	out := arrow.Int64Traits.CastFromBytes(data)
   673  
   674  	for idx, val := range values[:length] {
   675  		if binary.LittleEndian.Uint32(val[8:]) == 0 {
   676  			out[idx] = 0
   677  		} else {
   678  			out[idx] = val.ToTime().UnixNano()
   679  		}
   680  	}
   681  
   682  	bitmap := rdr.ReleaseValidBits()
   683  	if bitmap != nil {
   684  		defer bitmap.Release()
   685  	}
   686  	return array.NewData(dt, length, []*memory.Buffer{
   687  		bitmap, memory.NewBufferBytes(data),
   688  	}, nil, int(rdr.NullCount()), 0)
   689  }
   690  
   691  // convert physical integer storage of a decimal logical type to a decimal128 typed array
   692  func transferDecimalInteger(rdr file.RecordReader, dt arrow.DataType) arrow.ArrayData {
   693  	length := rdr.ValuesWritten()
   694  
   695  	var values reflect.Value
   696  	switch rdr.Type() {
   697  	case parquet.Types.Int32:
   698  		values = reflect.ValueOf(arrow.Int32Traits.CastFromBytes(rdr.Values())[:length])
   699  	case parquet.Types.Int64:
   700  		values = reflect.ValueOf(arrow.Int64Traits.CastFromBytes(rdr.Values())[:length])
   701  	}
   702  
   703  	data := make([]byte, arrow.Decimal128Traits.BytesRequired(length))
   704  	out := arrow.Decimal128Traits.CastFromBytes(data)
   705  	for i := 0; i < values.Len(); i++ {
   706  		out[i] = decimal128.FromI64(values.Index(i).Int())
   707  	}
   708  
   709  	var nullmap *memory.Buffer
   710  	if rdr.NullCount() > 0 {
   711  		nullmap = rdr.ReleaseValidBits()
   712  		defer nullmap.Release()
   713  	}
   714  	return array.NewData(dt, length, []*memory.Buffer{
   715  		nullmap, memory.NewBufferBytes(data),
   716  	}, nil, int(rdr.NullCount()), 0)
   717  }
   718  
   719  func uint64FromBigEndianShifted(buf []byte) uint64 {
   720  	var (
   721  		bytes [8]byte
   722  	)
   723  	copy(bytes[8-len(buf):], buf)
   724  	return binary.BigEndian.Uint64(bytes[:])
   725  }
   726  
   727  // parquet's defined encoding for decimal data is for it to be written as big
   728  // endian bytes, so convert a bit endian byte order to a decimal128
   729  func bigEndianToDecimal128(buf []byte) (decimal128.Num, error) {
   730  	const (
   731  		minDecimalBytes = 1
   732  		maxDecimalBytes = 16
   733  	)
   734  
   735  	if len(buf) < minDecimalBytes || len(buf) > maxDecimalBytes {
   736  		return decimal128.Num{}, fmt.Errorf("length of byte array passed to bigEndianToDecimal128 was %d but must be between %d and %d",
   737  			len(buf), minDecimalBytes, maxDecimalBytes)
   738  	}
   739  
   740  	// bytes are big endian so first byte is MSB and holds the sign bit
   741  	isNeg := int8(buf[0]) < 0
   742  
   743  	// 1. extract high bits
   744  	highBitsOffset := utils.MaxInt(0, len(buf)-8)
   745  	var (
   746  		highBits uint64
   747  		lowBits  uint64
   748  		hi       int64
   749  		lo       int64
   750  	)
   751  	highBits = uint64FromBigEndianShifted(buf[:highBitsOffset])
   752  
   753  	if highBitsOffset == 8 {
   754  		hi = int64(highBits)
   755  	} else {
   756  		if isNeg && len(buf) < maxDecimalBytes {
   757  			hi = -1
   758  		}
   759  
   760  		hi = int64(uint64(hi) << (uint64(highBitsOffset) * 8))
   761  		hi |= int64(highBits)
   762  	}
   763  
   764  	// 2. extract lower bits
   765  	lowBitsOffset := utils.MinInt(len(buf), 8)
   766  	lowBits = uint64FromBigEndianShifted(buf[highBitsOffset:])
   767  
   768  	if lowBitsOffset == 8 {
   769  		lo = int64(lowBits)
   770  	} else {
   771  		if isNeg && len(buf) < 8 {
   772  			lo = -1
   773  		}
   774  
   775  		lo = int64(uint64(lo) << (uint64(lowBitsOffset) * 8))
   776  		lo |= int64(lowBits)
   777  	}
   778  
   779  	return decimal128.New(hi, uint64(lo)), nil
   780  }
   781  
   782  type varOrFixedBin interface {
   783  	arrow.Array
   784  	Value(i int) []byte
   785  }
   786  
   787  // convert physical byte storage, instead of integers, to decimal128
   788  func transferDecimalBytes(rdr file.BinaryRecordReader, dt arrow.DataType) (*arrow.Chunked, error) {
   789  	convert := func(arr arrow.Array) (arrow.Array, error) {
   790  		length := arr.Len()
   791  		data := make([]byte, arrow.Decimal128Traits.BytesRequired(length))
   792  		out := arrow.Decimal128Traits.CastFromBytes(data)
   793  
   794  		input := arr.(varOrFixedBin)
   795  		nullCount := input.NullN()
   796  
   797  		var err error
   798  		for i := 0; i < length; i++ {
   799  			if nullCount > 0 && input.IsNull(i) {
   800  				continue
   801  			}
   802  
   803  			rec := input.Value(i)
   804  			if len(rec) <= 0 {
   805  				return nil, fmt.Errorf("invalud BYTEARRAY length for type: %s", dt)
   806  			}
   807  			out[i], err = bigEndianToDecimal128(rec)
   808  			if err != nil {
   809  				return nil, err
   810  			}
   811  		}
   812  
   813  		ret := array.NewData(dt, length, []*memory.Buffer{
   814  			input.Data().Buffers()[0], memory.NewBufferBytes(data),
   815  		}, nil, nullCount, 0)
   816  		defer ret.Release()
   817  		return array.MakeFromData(ret), nil
   818  	}
   819  
   820  	chunks := rdr.GetBuilderChunks()
   821  	var err error
   822  	for idx, chunk := range chunks {
   823  		defer chunk.Release()
   824  		if chunks[idx], err = convert(chunk); err != nil {
   825  			return nil, err
   826  		}
   827  		defer chunks[idx].Release()
   828  	}
   829  	return arrow.NewChunked(dt, chunks), nil
   830  }