github.com/apache/arrow/go/v16@v16.1.0/arrow/ipc/metadata.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 ipc
    18  
    19  import (
    20  	"encoding/binary"
    21  	"errors"
    22  	"fmt"
    23  	"io"
    24  	"sort"
    25  
    26  	"github.com/apache/arrow/go/v16/arrow"
    27  	"github.com/apache/arrow/go/v16/arrow/endian"
    28  	"github.com/apache/arrow/go/v16/arrow/internal/dictutils"
    29  	"github.com/apache/arrow/go/v16/arrow/internal/flatbuf"
    30  	"github.com/apache/arrow/go/v16/arrow/memory"
    31  	flatbuffers "github.com/google/flatbuffers/go"
    32  )
    33  
    34  // Magic string identifying an Apache Arrow file.
    35  var Magic = []byte("ARROW1")
    36  
    37  const (
    38  	currentMetadataVersion = MetadataV5
    39  	minMetadataVersion     = MetadataV4
    40  
    41  	// constants for the extension type metadata keys for the type name and
    42  	// any extension metadata to be passed to deserialize.
    43  	ExtensionTypeKeyName     = "ARROW:extension:name"
    44  	ExtensionMetadataKeyName = "ARROW:extension:metadata"
    45  
    46  	// ARROW-109: We set this number arbitrarily to help catch user mistakes. For
    47  	// deeply nested schemas, it is expected the user will indicate explicitly the
    48  	// maximum allowed recursion depth
    49  	kMaxNestingDepth = 64
    50  )
    51  
    52  type startVecFunc func(b *flatbuffers.Builder, n int) flatbuffers.UOffsetT
    53  
    54  type fieldMetadata struct {
    55  	Len    int64
    56  	Nulls  int64
    57  	Offset int64
    58  }
    59  
    60  type bufferMetadata struct {
    61  	Offset int64 // relative offset into the memory page to the starting byte of the buffer
    62  	Len    int64 // absolute length in bytes of the buffer
    63  }
    64  
    65  type fileBlock struct {
    66  	Offset int64
    67  	Meta   int32
    68  	Body   int64
    69  
    70  	r   io.ReaderAt
    71  	mem memory.Allocator
    72  }
    73  
    74  func fileBlocksToFB(b *flatbuffers.Builder, blocks []fileBlock, start startVecFunc) flatbuffers.UOffsetT {
    75  	start(b, len(blocks))
    76  	for i := len(blocks) - 1; i >= 0; i-- {
    77  		blk := blocks[i]
    78  		flatbuf.CreateBlock(b, blk.Offset, blk.Meta, blk.Body)
    79  	}
    80  
    81  	return b.EndVector(len(blocks))
    82  }
    83  
    84  func (blk fileBlock) NewMessage() (*Message, error) {
    85  	var (
    86  		err  error
    87  		buf  []byte
    88  		body *memory.Buffer
    89  		meta *memory.Buffer
    90  		r    = blk.section()
    91  	)
    92  
    93  	meta = memory.NewResizableBuffer(blk.mem)
    94  	meta.Resize(int(blk.Meta))
    95  	defer meta.Release()
    96  
    97  	buf = meta.Bytes()
    98  	_, err = io.ReadFull(r, buf)
    99  	if err != nil {
   100  		return nil, fmt.Errorf("arrow/ipc: could not read message metadata: %w", err)
   101  	}
   102  
   103  	prefix := 0
   104  	switch binary.LittleEndian.Uint32(buf) {
   105  	case 0:
   106  	case kIPCContToken:
   107  		prefix = 8
   108  	default:
   109  		// ARROW-6314: backwards compatibility for reading old IPC
   110  		// messages produced prior to version 0.15.0
   111  		prefix = 4
   112  	}
   113  
   114  	// drop buf-size already known from blk.Meta
   115  	meta = memory.SliceBuffer(meta, prefix, int(blk.Meta)-prefix)
   116  	defer meta.Release()
   117  
   118  	body = memory.NewResizableBuffer(blk.mem)
   119  	defer body.Release()
   120  	body.Resize(int(blk.Body))
   121  	buf = body.Bytes()
   122  	_, err = io.ReadFull(r, buf)
   123  	if err != nil {
   124  		return nil, fmt.Errorf("arrow/ipc: could not read message body: %w", err)
   125  	}
   126  
   127  	return NewMessage(meta, body), nil
   128  }
   129  
   130  func (blk fileBlock) section() io.Reader {
   131  	return io.NewSectionReader(blk.r, blk.Offset, int64(blk.Meta)+blk.Body)
   132  }
   133  
   134  func unitFromFB(unit flatbuf.TimeUnit) arrow.TimeUnit {
   135  	switch unit {
   136  	case flatbuf.TimeUnitSECOND:
   137  		return arrow.Second
   138  	case flatbuf.TimeUnitMILLISECOND:
   139  		return arrow.Millisecond
   140  	case flatbuf.TimeUnitMICROSECOND:
   141  		return arrow.Microsecond
   142  	case flatbuf.TimeUnitNANOSECOND:
   143  		return arrow.Nanosecond
   144  	default:
   145  		panic(fmt.Errorf("arrow/ipc: invalid flatbuf.TimeUnit(%d) value", unit))
   146  	}
   147  }
   148  
   149  func unitToFB(unit arrow.TimeUnit) flatbuf.TimeUnit {
   150  	switch unit {
   151  	case arrow.Second:
   152  		return flatbuf.TimeUnitSECOND
   153  	case arrow.Millisecond:
   154  		return flatbuf.TimeUnitMILLISECOND
   155  	case arrow.Microsecond:
   156  		return flatbuf.TimeUnitMICROSECOND
   157  	case arrow.Nanosecond:
   158  		return flatbuf.TimeUnitNANOSECOND
   159  	default:
   160  		panic(fmt.Errorf("arrow/ipc: invalid arrow.TimeUnit(%d) value", unit))
   161  	}
   162  }
   163  
   164  // initFB is a helper function to handle flatbuffers' polymorphism.
   165  func initFB(t interface {
   166  	Table() flatbuffers.Table
   167  	Init([]byte, flatbuffers.UOffsetT)
   168  }, f func(tbl *flatbuffers.Table) bool) {
   169  	tbl := t.Table()
   170  	if !f(&tbl) {
   171  		panic(fmt.Errorf("arrow/ipc: could not initialize %T from flatbuffer", t))
   172  	}
   173  	t.Init(tbl.Bytes, tbl.Pos)
   174  }
   175  
   176  func fieldFromFB(field *flatbuf.Field, pos dictutils.FieldPos, memo *dictutils.Memo) (arrow.Field, error) {
   177  	var (
   178  		err error
   179  		o   arrow.Field
   180  	)
   181  
   182  	o.Name = string(field.Name())
   183  	o.Nullable = field.Nullable()
   184  	o.Metadata, err = metadataFromFB(field)
   185  	if err != nil {
   186  		return o, err
   187  	}
   188  
   189  	n := field.ChildrenLength()
   190  	children := make([]arrow.Field, n)
   191  	for i := range children {
   192  		var childFB flatbuf.Field
   193  		if !field.Children(&childFB, i) {
   194  			return o, fmt.Errorf("arrow/ipc: could not load field child %d", i)
   195  
   196  		}
   197  		child, err := fieldFromFB(&childFB, pos.Child(int32(i)), memo)
   198  		if err != nil {
   199  			return o, fmt.Errorf("arrow/ipc: could not convert field child %d: %w", i, err)
   200  		}
   201  		children[i] = child
   202  	}
   203  
   204  	o.Type, err = typeFromFB(field, pos, children, &o.Metadata, memo)
   205  	if err != nil {
   206  		return o, fmt.Errorf("arrow/ipc: could not convert field type: %w", err)
   207  	}
   208  
   209  	return o, nil
   210  }
   211  
   212  func fieldToFB(b *flatbuffers.Builder, pos dictutils.FieldPos, field arrow.Field, memo *dictutils.Mapper) flatbuffers.UOffsetT {
   213  	var visitor = fieldVisitor{b: b, memo: memo, pos: pos, meta: make(map[string]string)}
   214  	return visitor.result(field)
   215  }
   216  
   217  type fieldVisitor struct {
   218  	b      *flatbuffers.Builder
   219  	memo   *dictutils.Mapper
   220  	pos    dictutils.FieldPos
   221  	dtype  flatbuf.Type
   222  	offset flatbuffers.UOffsetT
   223  	kids   []flatbuffers.UOffsetT
   224  	meta   map[string]string
   225  }
   226  
   227  func (fv *fieldVisitor) visit(field arrow.Field) {
   228  	dt := field.Type
   229  	switch dt := dt.(type) {
   230  	case *arrow.NullType:
   231  		fv.dtype = flatbuf.TypeNull
   232  		flatbuf.NullStart(fv.b)
   233  		fv.offset = flatbuf.NullEnd(fv.b)
   234  
   235  	case *arrow.BooleanType:
   236  		fv.dtype = flatbuf.TypeBool
   237  		flatbuf.BoolStart(fv.b)
   238  		fv.offset = flatbuf.BoolEnd(fv.b)
   239  
   240  	case *arrow.Uint8Type:
   241  		fv.dtype = flatbuf.TypeInt
   242  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), false)
   243  
   244  	case *arrow.Uint16Type:
   245  		fv.dtype = flatbuf.TypeInt
   246  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), false)
   247  
   248  	case *arrow.Uint32Type:
   249  		fv.dtype = flatbuf.TypeInt
   250  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), false)
   251  
   252  	case *arrow.Uint64Type:
   253  		fv.dtype = flatbuf.TypeInt
   254  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), false)
   255  
   256  	case *arrow.Int8Type:
   257  		fv.dtype = flatbuf.TypeInt
   258  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), true)
   259  
   260  	case *arrow.Int16Type:
   261  		fv.dtype = flatbuf.TypeInt
   262  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), true)
   263  
   264  	case *arrow.Int32Type:
   265  		fv.dtype = flatbuf.TypeInt
   266  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), true)
   267  
   268  	case *arrow.Int64Type:
   269  		fv.dtype = flatbuf.TypeInt
   270  		fv.offset = intToFB(fv.b, int32(dt.BitWidth()), true)
   271  
   272  	case *arrow.Float16Type:
   273  		fv.dtype = flatbuf.TypeFloatingPoint
   274  		fv.offset = floatToFB(fv.b, int32(dt.BitWidth()))
   275  
   276  	case *arrow.Float32Type:
   277  		fv.dtype = flatbuf.TypeFloatingPoint
   278  		fv.offset = floatToFB(fv.b, int32(dt.BitWidth()))
   279  
   280  	case *arrow.Float64Type:
   281  		fv.dtype = flatbuf.TypeFloatingPoint
   282  		fv.offset = floatToFB(fv.b, int32(dt.BitWidth()))
   283  
   284  	case *arrow.Decimal128Type:
   285  		fv.dtype = flatbuf.TypeDecimal
   286  		flatbuf.DecimalStart(fv.b)
   287  		flatbuf.DecimalAddPrecision(fv.b, dt.Precision)
   288  		flatbuf.DecimalAddScale(fv.b, dt.Scale)
   289  		flatbuf.DecimalAddBitWidth(fv.b, 128)
   290  		fv.offset = flatbuf.DecimalEnd(fv.b)
   291  
   292  	case *arrow.Decimal256Type:
   293  		fv.dtype = flatbuf.TypeDecimal
   294  		flatbuf.DecimalStart(fv.b)
   295  		flatbuf.DecimalAddPrecision(fv.b, dt.Precision)
   296  		flatbuf.DecimalAddScale(fv.b, dt.Scale)
   297  		flatbuf.DecimalAddBitWidth(fv.b, 256)
   298  		fv.offset = flatbuf.DecimalEnd(fv.b)
   299  
   300  	case *arrow.FixedSizeBinaryType:
   301  		fv.dtype = flatbuf.TypeFixedSizeBinary
   302  		flatbuf.FixedSizeBinaryStart(fv.b)
   303  		flatbuf.FixedSizeBinaryAddByteWidth(fv.b, int32(dt.ByteWidth))
   304  		fv.offset = flatbuf.FixedSizeBinaryEnd(fv.b)
   305  
   306  	case *arrow.BinaryType:
   307  		fv.dtype = flatbuf.TypeBinary
   308  		flatbuf.BinaryStart(fv.b)
   309  		fv.offset = flatbuf.BinaryEnd(fv.b)
   310  
   311  	case *arrow.LargeBinaryType:
   312  		fv.dtype = flatbuf.TypeLargeBinary
   313  		flatbuf.LargeBinaryStart(fv.b)
   314  		fv.offset = flatbuf.LargeBinaryEnd(fv.b)
   315  
   316  	case *arrow.StringType:
   317  		fv.dtype = flatbuf.TypeUtf8
   318  		flatbuf.Utf8Start(fv.b)
   319  		fv.offset = flatbuf.Utf8End(fv.b)
   320  
   321  	case *arrow.LargeStringType:
   322  		fv.dtype = flatbuf.TypeLargeUtf8
   323  		flatbuf.LargeUtf8Start(fv.b)
   324  		fv.offset = flatbuf.LargeUtf8End(fv.b)
   325  
   326  	case *arrow.BinaryViewType:
   327  		fv.dtype = flatbuf.TypeBinaryView
   328  		flatbuf.BinaryViewStart(fv.b)
   329  		fv.offset = flatbuf.BinaryViewEnd(fv.b)
   330  
   331  	case *arrow.StringViewType:
   332  		fv.dtype = flatbuf.TypeUtf8View
   333  		flatbuf.Utf8ViewStart(fv.b)
   334  		fv.offset = flatbuf.Utf8ViewEnd(fv.b)
   335  
   336  	case *arrow.Date32Type:
   337  		fv.dtype = flatbuf.TypeDate
   338  		flatbuf.DateStart(fv.b)
   339  		flatbuf.DateAddUnit(fv.b, flatbuf.DateUnitDAY)
   340  		fv.offset = flatbuf.DateEnd(fv.b)
   341  
   342  	case *arrow.Date64Type:
   343  		fv.dtype = flatbuf.TypeDate
   344  		flatbuf.DateStart(fv.b)
   345  		flatbuf.DateAddUnit(fv.b, flatbuf.DateUnitMILLISECOND)
   346  		fv.offset = flatbuf.DateEnd(fv.b)
   347  
   348  	case *arrow.Time32Type:
   349  		fv.dtype = flatbuf.TypeTime
   350  		flatbuf.TimeStart(fv.b)
   351  		flatbuf.TimeAddUnit(fv.b, unitToFB(dt.Unit))
   352  		flatbuf.TimeAddBitWidth(fv.b, 32)
   353  		fv.offset = flatbuf.TimeEnd(fv.b)
   354  
   355  	case *arrow.Time64Type:
   356  		fv.dtype = flatbuf.TypeTime
   357  		flatbuf.TimeStart(fv.b)
   358  		flatbuf.TimeAddUnit(fv.b, unitToFB(dt.Unit))
   359  		flatbuf.TimeAddBitWidth(fv.b, 64)
   360  		fv.offset = flatbuf.TimeEnd(fv.b)
   361  
   362  	case *arrow.TimestampType:
   363  		fv.dtype = flatbuf.TypeTimestamp
   364  		unit := unitToFB(dt.Unit)
   365  		var tz flatbuffers.UOffsetT
   366  		if dt.TimeZone != "" {
   367  			tz = fv.b.CreateString(dt.TimeZone)
   368  		}
   369  		flatbuf.TimestampStart(fv.b)
   370  		flatbuf.TimestampAddUnit(fv.b, unit)
   371  		flatbuf.TimestampAddTimezone(fv.b, tz)
   372  		fv.offset = flatbuf.TimestampEnd(fv.b)
   373  
   374  	case *arrow.StructType:
   375  		fv.dtype = flatbuf.TypeStruct_
   376  		offsets := make([]flatbuffers.UOffsetT, dt.NumFields())
   377  		for i, field := range dt.Fields() {
   378  			offsets[i] = fieldToFB(fv.b, fv.pos.Child(int32(i)), field, fv.memo)
   379  		}
   380  		flatbuf.Struct_Start(fv.b)
   381  		for i := len(offsets) - 1; i >= 0; i-- {
   382  			fv.b.PrependUOffsetT(offsets[i])
   383  		}
   384  		fv.offset = flatbuf.Struct_End(fv.b)
   385  		fv.kids = append(fv.kids, offsets...)
   386  
   387  	case *arrow.ListType:
   388  		fv.dtype = flatbuf.TypeList
   389  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   390  		flatbuf.ListStart(fv.b)
   391  		fv.offset = flatbuf.ListEnd(fv.b)
   392  
   393  	case *arrow.LargeListType:
   394  		fv.dtype = flatbuf.TypeLargeList
   395  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   396  		flatbuf.LargeListStart(fv.b)
   397  		fv.offset = flatbuf.LargeListEnd(fv.b)
   398  
   399  	case *arrow.ListViewType:
   400  		fv.dtype = flatbuf.TypeListView
   401  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   402  		flatbuf.ListViewStart(fv.b)
   403  		fv.offset = flatbuf.ListViewEnd(fv.b)
   404  
   405  	case *arrow.LargeListViewType:
   406  		fv.dtype = flatbuf.TypeLargeListView
   407  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   408  		flatbuf.LargeListViewStart(fv.b)
   409  		fv.offset = flatbuf.LargeListViewEnd(fv.b)
   410  
   411  	case *arrow.FixedSizeListType:
   412  		fv.dtype = flatbuf.TypeFixedSizeList
   413  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   414  		flatbuf.FixedSizeListStart(fv.b)
   415  		flatbuf.FixedSizeListAddListSize(fv.b, dt.Len())
   416  		fv.offset = flatbuf.FixedSizeListEnd(fv.b)
   417  
   418  	case *arrow.MonthIntervalType:
   419  		fv.dtype = flatbuf.TypeInterval
   420  		flatbuf.IntervalStart(fv.b)
   421  		flatbuf.IntervalAddUnit(fv.b, flatbuf.IntervalUnitYEAR_MONTH)
   422  		fv.offset = flatbuf.IntervalEnd(fv.b)
   423  
   424  	case *arrow.DayTimeIntervalType:
   425  		fv.dtype = flatbuf.TypeInterval
   426  		flatbuf.IntervalStart(fv.b)
   427  		flatbuf.IntervalAddUnit(fv.b, flatbuf.IntervalUnitDAY_TIME)
   428  		fv.offset = flatbuf.IntervalEnd(fv.b)
   429  
   430  	case *arrow.MonthDayNanoIntervalType:
   431  		fv.dtype = flatbuf.TypeInterval
   432  		flatbuf.IntervalStart(fv.b)
   433  		flatbuf.IntervalAddUnit(fv.b, flatbuf.IntervalUnitMONTH_DAY_NANO)
   434  		fv.offset = flatbuf.IntervalEnd(fv.b)
   435  
   436  	case *arrow.DurationType:
   437  		fv.dtype = flatbuf.TypeDuration
   438  		unit := unitToFB(dt.Unit)
   439  		flatbuf.DurationStart(fv.b)
   440  		flatbuf.DurationAddUnit(fv.b, unit)
   441  		fv.offset = flatbuf.DurationEnd(fv.b)
   442  
   443  	case *arrow.MapType:
   444  		fv.dtype = flatbuf.TypeMap
   445  		fv.kids = append(fv.kids, fieldToFB(fv.b, fv.pos.Child(0), dt.ElemField(), fv.memo))
   446  		flatbuf.MapStart(fv.b)
   447  		flatbuf.MapAddKeysSorted(fv.b, dt.KeysSorted)
   448  		fv.offset = flatbuf.MapEnd(fv.b)
   449  
   450  	case *arrow.RunEndEncodedType:
   451  		fv.dtype = flatbuf.TypeRunEndEncoded
   452  		var offsets [2]flatbuffers.UOffsetT
   453  		offsets[0] = fieldToFB(fv.b, fv.pos.Child(0),
   454  			arrow.Field{Name: "run_ends", Type: dt.RunEnds()}, fv.memo)
   455  		offsets[1] = fieldToFB(fv.b, fv.pos.Child(1),
   456  			arrow.Field{Name: "values", Type: dt.Encoded(), Nullable: true}, fv.memo)
   457  		flatbuf.RunEndEncodedStart(fv.b)
   458  		fv.b.PrependUOffsetT(offsets[1])
   459  		fv.b.PrependUOffsetT(offsets[0])
   460  		fv.offset = flatbuf.RunEndEncodedEnd(fv.b)
   461  		fv.kids = append(fv.kids, offsets[0], offsets[1])
   462  
   463  	case arrow.ExtensionType:
   464  		field.Type = dt.StorageType()
   465  		fv.visit(field)
   466  		fv.meta[ExtensionTypeKeyName] = dt.ExtensionName()
   467  		fv.meta[ExtensionMetadataKeyName] = string(dt.Serialize())
   468  
   469  	case *arrow.DictionaryType:
   470  		field.Type = dt.ValueType
   471  		fv.visit(field)
   472  
   473  	case arrow.UnionType:
   474  		fv.dtype = flatbuf.TypeUnion
   475  		offsets := make([]flatbuffers.UOffsetT, dt.NumFields())
   476  		for i, field := range dt.Fields() {
   477  			offsets[i] = fieldToFB(fv.b, fv.pos.Child(int32(i)), field, fv.memo)
   478  		}
   479  
   480  		codes := dt.TypeCodes()
   481  		flatbuf.UnionStartTypeIdsVector(fv.b, len(codes))
   482  
   483  		for i := len(codes) - 1; i >= 0; i-- {
   484  			fv.b.PlaceInt32(int32(codes[i]))
   485  		}
   486  		fbTypeIDs := fv.b.EndVector(len(dt.TypeCodes()))
   487  		flatbuf.UnionStart(fv.b)
   488  		switch dt.Mode() {
   489  		case arrow.SparseMode:
   490  			flatbuf.UnionAddMode(fv.b, flatbuf.UnionModeSparse)
   491  		case arrow.DenseMode:
   492  			flatbuf.UnionAddMode(fv.b, flatbuf.UnionModeDense)
   493  		default:
   494  			panic("invalid union mode")
   495  		}
   496  		flatbuf.UnionAddTypeIds(fv.b, fbTypeIDs)
   497  		fv.offset = flatbuf.UnionEnd(fv.b)
   498  		fv.kids = append(fv.kids, offsets...)
   499  
   500  	default:
   501  		err := fmt.Errorf("arrow/ipc: invalid data type %v", dt)
   502  		panic(err) // FIXME(sbinet): implement all data-types.
   503  	}
   504  }
   505  
   506  func (fv *fieldVisitor) result(field arrow.Field) flatbuffers.UOffsetT {
   507  	nameFB := fv.b.CreateString(field.Name)
   508  
   509  	fv.visit(field)
   510  
   511  	flatbuf.FieldStartChildrenVector(fv.b, len(fv.kids))
   512  	for i := len(fv.kids) - 1; i >= 0; i-- {
   513  		fv.b.PrependUOffsetT(fv.kids[i])
   514  	}
   515  	kidsFB := fv.b.EndVector(len(fv.kids))
   516  
   517  	storageType := field.Type
   518  	if storageType.ID() == arrow.EXTENSION {
   519  		storageType = storageType.(arrow.ExtensionType).StorageType()
   520  	}
   521  
   522  	var dictFB flatbuffers.UOffsetT
   523  	if storageType.ID() == arrow.DICTIONARY {
   524  		idxType := field.Type.(*arrow.DictionaryType).IndexType.(arrow.FixedWidthDataType)
   525  
   526  		dictID, err := fv.memo.GetFieldID(fv.pos.Path())
   527  		if err != nil {
   528  			panic(err)
   529  		}
   530  		var signed bool
   531  		switch idxType.ID() {
   532  		case arrow.UINT8, arrow.UINT16, arrow.UINT32, arrow.UINT64:
   533  			signed = false
   534  		case arrow.INT8, arrow.INT16, arrow.INT32, arrow.INT64:
   535  			signed = true
   536  		}
   537  		indexTypeOffset := intToFB(fv.b, int32(idxType.BitWidth()), signed)
   538  		flatbuf.DictionaryEncodingStart(fv.b)
   539  		flatbuf.DictionaryEncodingAddId(fv.b, dictID)
   540  		flatbuf.DictionaryEncodingAddIndexType(fv.b, indexTypeOffset)
   541  		flatbuf.DictionaryEncodingAddIsOrdered(fv.b, field.Type.(*arrow.DictionaryType).Ordered)
   542  		dictFB = flatbuf.DictionaryEncodingEnd(fv.b)
   543  	}
   544  
   545  	var (
   546  		metaFB flatbuffers.UOffsetT
   547  		kvs    []flatbuffers.UOffsetT
   548  	)
   549  	for i, k := range field.Metadata.Keys() {
   550  		v := field.Metadata.Values()[i]
   551  		kk := fv.b.CreateString(k)
   552  		vv := fv.b.CreateString(v)
   553  		flatbuf.KeyValueStart(fv.b)
   554  		flatbuf.KeyValueAddKey(fv.b, kk)
   555  		flatbuf.KeyValueAddValue(fv.b, vv)
   556  		kvs = append(kvs, flatbuf.KeyValueEnd(fv.b))
   557  	}
   558  	{
   559  		keys := make([]string, 0, len(fv.meta))
   560  		for k := range fv.meta {
   561  			keys = append(keys, k)
   562  		}
   563  		sort.Strings(keys)
   564  		for _, k := range keys {
   565  			v := fv.meta[k]
   566  			kk := fv.b.CreateString(k)
   567  			vv := fv.b.CreateString(v)
   568  			flatbuf.KeyValueStart(fv.b)
   569  			flatbuf.KeyValueAddKey(fv.b, kk)
   570  			flatbuf.KeyValueAddValue(fv.b, vv)
   571  			kvs = append(kvs, flatbuf.KeyValueEnd(fv.b))
   572  		}
   573  	}
   574  	if len(kvs) > 0 {
   575  		flatbuf.FieldStartCustomMetadataVector(fv.b, len(kvs))
   576  		for i := len(kvs) - 1; i >= 0; i-- {
   577  			fv.b.PrependUOffsetT(kvs[i])
   578  		}
   579  		metaFB = fv.b.EndVector(len(kvs))
   580  	}
   581  
   582  	flatbuf.FieldStart(fv.b)
   583  	flatbuf.FieldAddName(fv.b, nameFB)
   584  	flatbuf.FieldAddNullable(fv.b, field.Nullable)
   585  	flatbuf.FieldAddTypeType(fv.b, fv.dtype)
   586  	flatbuf.FieldAddType(fv.b, fv.offset)
   587  	flatbuf.FieldAddDictionary(fv.b, dictFB)
   588  	flatbuf.FieldAddChildren(fv.b, kidsFB)
   589  	flatbuf.FieldAddCustomMetadata(fv.b, metaFB)
   590  
   591  	offset := flatbuf.FieldEnd(fv.b)
   592  
   593  	return offset
   594  }
   595  
   596  func typeFromFB(field *flatbuf.Field, pos dictutils.FieldPos, children []arrow.Field, md *arrow.Metadata, memo *dictutils.Memo) (arrow.DataType, error) {
   597  	var data flatbuffers.Table
   598  	if !field.Type(&data) {
   599  		return nil, fmt.Errorf("arrow/ipc: could not load field type data")
   600  	}
   601  
   602  	dt, err := concreteTypeFromFB(field.TypeType(), data, children)
   603  	if err != nil {
   604  		return dt, err
   605  	}
   606  
   607  	var (
   608  		dictID        = int64(-1)
   609  		dictValueType arrow.DataType
   610  		encoding      = field.Dictionary(nil)
   611  	)
   612  	if encoding != nil {
   613  		var idt flatbuf.Int
   614  		encoding.IndexType(&idt)
   615  		idxType, err := intFromFB(idt)
   616  		if err != nil {
   617  			return nil, err
   618  		}
   619  
   620  		dictValueType = dt
   621  		dt = &arrow.DictionaryType{IndexType: idxType, ValueType: dictValueType, Ordered: encoding.IsOrdered()}
   622  		dictID = encoding.Id()
   623  
   624  		if err = memo.Mapper.AddField(dictID, pos.Path()); err != nil {
   625  			return dt, err
   626  		}
   627  		if err = memo.AddType(dictID, dictValueType); err != nil {
   628  			return dt, err
   629  		}
   630  
   631  	}
   632  
   633  	// look for extension metadata in custom metadata field.
   634  	if md.Len() > 0 {
   635  		i := md.FindKey(ExtensionTypeKeyName)
   636  		if i < 0 {
   637  			return dt, err
   638  		}
   639  
   640  		extType := arrow.GetExtensionType(md.Values()[i])
   641  		if extType == nil {
   642  			// if the extension type is unknown, we do not error here.
   643  			// simply return the storage type.
   644  			return dt, err
   645  		}
   646  
   647  		var (
   648  			data    string
   649  			dataIdx int
   650  		)
   651  
   652  		if dataIdx = md.FindKey(ExtensionMetadataKeyName); dataIdx >= 0 {
   653  			data = md.Values()[dataIdx]
   654  		}
   655  
   656  		dt, err = extType.Deserialize(dt, data)
   657  		if err != nil {
   658  			return dt, err
   659  		}
   660  
   661  		mdkeys := md.Keys()
   662  		mdvals := md.Values()
   663  		if dataIdx < 0 {
   664  			// if there was no extension metadata, just the name, we only have to
   665  			// remove the extension name metadata key/value to ensure roundtrip
   666  			// metadata consistency
   667  			*md = arrow.NewMetadata(append(mdkeys[:i], mdkeys[i+1:]...), append(mdvals[:i], mdvals[i+1:]...))
   668  		} else {
   669  			// if there was extension metadata, we need to remove both the type name
   670  			// and the extension metadata keys and values.
   671  			newkeys := make([]string, 0, md.Len()-2)
   672  			newvals := make([]string, 0, md.Len()-2)
   673  			for j := range mdkeys {
   674  				if j != i && j != dataIdx { // copy everything except the extension metadata keys/values
   675  					newkeys = append(newkeys, mdkeys[j])
   676  					newvals = append(newvals, mdvals[j])
   677  				}
   678  			}
   679  			*md = arrow.NewMetadata(newkeys, newvals)
   680  		}
   681  	}
   682  
   683  	return dt, err
   684  }
   685  
   686  func concreteTypeFromFB(typ flatbuf.Type, data flatbuffers.Table, children []arrow.Field) (arrow.DataType, error) {
   687  	switch typ {
   688  	case flatbuf.TypeNONE:
   689  		return nil, fmt.Errorf("arrow/ipc: Type metadata cannot be none")
   690  
   691  	case flatbuf.TypeNull:
   692  		return arrow.Null, nil
   693  
   694  	case flatbuf.TypeInt:
   695  		var dt flatbuf.Int
   696  		dt.Init(data.Bytes, data.Pos)
   697  		return intFromFB(dt)
   698  
   699  	case flatbuf.TypeFloatingPoint:
   700  		var dt flatbuf.FloatingPoint
   701  		dt.Init(data.Bytes, data.Pos)
   702  		return floatFromFB(dt)
   703  
   704  	case flatbuf.TypeDecimal:
   705  		var dt flatbuf.Decimal
   706  		dt.Init(data.Bytes, data.Pos)
   707  		return decimalFromFB(dt)
   708  
   709  	case flatbuf.TypeBinary:
   710  		return arrow.BinaryTypes.Binary, nil
   711  
   712  	case flatbuf.TypeFixedSizeBinary:
   713  		var dt flatbuf.FixedSizeBinary
   714  		dt.Init(data.Bytes, data.Pos)
   715  		return &arrow.FixedSizeBinaryType{ByteWidth: int(dt.ByteWidth())}, nil
   716  
   717  	case flatbuf.TypeUtf8:
   718  		return arrow.BinaryTypes.String, nil
   719  
   720  	case flatbuf.TypeLargeBinary:
   721  		return arrow.BinaryTypes.LargeBinary, nil
   722  
   723  	case flatbuf.TypeLargeUtf8:
   724  		return arrow.BinaryTypes.LargeString, nil
   725  
   726  	case flatbuf.TypeUtf8View:
   727  		return arrow.BinaryTypes.StringView, nil
   728  
   729  	case flatbuf.TypeBinaryView:
   730  		return arrow.BinaryTypes.BinaryView, nil
   731  
   732  	case flatbuf.TypeBool:
   733  		return arrow.FixedWidthTypes.Boolean, nil
   734  
   735  	case flatbuf.TypeList:
   736  		if len(children) != 1 {
   737  			return nil, fmt.Errorf("arrow/ipc: List must have exactly 1 child field (got=%d)", len(children))
   738  		}
   739  		dt := arrow.ListOfField(children[0])
   740  		return dt, nil
   741  
   742  	case flatbuf.TypeLargeList:
   743  		if len(children) != 1 {
   744  			return nil, fmt.Errorf("arrow/ipc: LargeList must have exactly 1 child field (got=%d)", len(children))
   745  		}
   746  		dt := arrow.LargeListOfField(children[0])
   747  		return dt, nil
   748  
   749  	case flatbuf.TypeListView:
   750  		if len(children) != 1 {
   751  			return nil, fmt.Errorf("arrow/ipc: ListView must have exactly 1 child field (got=%d)", len(children))
   752  		}
   753  		dt := arrow.ListViewOfField(children[0])
   754  		return dt, nil
   755  
   756  	case flatbuf.TypeLargeListView:
   757  		if len(children) != 1 {
   758  			return nil, fmt.Errorf("arrow/ipc: LargeListView must have exactly 1 child field (got=%d)", len(children))
   759  		}
   760  		dt := arrow.LargeListViewOfField(children[0])
   761  		return dt, nil
   762  
   763  	case flatbuf.TypeFixedSizeList:
   764  		var dt flatbuf.FixedSizeList
   765  		dt.Init(data.Bytes, data.Pos)
   766  		if len(children) != 1 {
   767  			return nil, fmt.Errorf("arrow/ipc: FixedSizeList must have exactly 1 child field (got=%d)", len(children))
   768  		}
   769  		ret := arrow.FixedSizeListOfField(dt.ListSize(), children[0])
   770  		return ret, nil
   771  
   772  	case flatbuf.TypeStruct_:
   773  		return arrow.StructOf(children...), nil
   774  
   775  	case flatbuf.TypeUnion:
   776  		var dt flatbuf.Union
   777  		dt.Init(data.Bytes, data.Pos)
   778  		var (
   779  			mode    arrow.UnionMode
   780  			typeIDs []arrow.UnionTypeCode
   781  		)
   782  
   783  		switch dt.Mode() {
   784  		case flatbuf.UnionModeSparse:
   785  			mode = arrow.SparseMode
   786  		case flatbuf.UnionModeDense:
   787  			mode = arrow.DenseMode
   788  		}
   789  
   790  		typeIDLen := dt.TypeIdsLength()
   791  
   792  		if typeIDLen == 0 {
   793  			for i := range children {
   794  				typeIDs = append(typeIDs, int8(i))
   795  			}
   796  		} else {
   797  			for i := 0; i < typeIDLen; i++ {
   798  				id := dt.TypeIds(i)
   799  				code := arrow.UnionTypeCode(id)
   800  				if int32(code) != id {
   801  					return nil, errors.New("union type id out of bounds")
   802  				}
   803  				typeIDs = append(typeIDs, code)
   804  			}
   805  		}
   806  
   807  		return arrow.UnionOf(mode, children, typeIDs), nil
   808  
   809  	case flatbuf.TypeTime:
   810  		var dt flatbuf.Time
   811  		dt.Init(data.Bytes, data.Pos)
   812  		return timeFromFB(dt)
   813  
   814  	case flatbuf.TypeTimestamp:
   815  		var dt flatbuf.Timestamp
   816  		dt.Init(data.Bytes, data.Pos)
   817  		return timestampFromFB(dt)
   818  
   819  	case flatbuf.TypeDate:
   820  		var dt flatbuf.Date
   821  		dt.Init(data.Bytes, data.Pos)
   822  		return dateFromFB(dt)
   823  
   824  	case flatbuf.TypeInterval:
   825  		var dt flatbuf.Interval
   826  		dt.Init(data.Bytes, data.Pos)
   827  		return intervalFromFB(dt)
   828  
   829  	case flatbuf.TypeDuration:
   830  		var dt flatbuf.Duration
   831  		dt.Init(data.Bytes, data.Pos)
   832  		return durationFromFB(dt)
   833  
   834  	case flatbuf.TypeMap:
   835  		if len(children) != 1 {
   836  			return nil, fmt.Errorf("arrow/ipc: Map must have exactly 1 child field")
   837  		}
   838  
   839  		if children[0].Nullable || children[0].Type.ID() != arrow.STRUCT || len(children[0].Type.(*arrow.StructType).Fields()) != 2 {
   840  			return nil, fmt.Errorf("arrow/ipc: Map's key-item pairs must be non-nullable structs")
   841  		}
   842  
   843  		pairType := children[0].Type.(*arrow.StructType)
   844  		if pairType.Field(0).Nullable {
   845  			return nil, fmt.Errorf("arrow/ipc: Map's keys must be non-nullable")
   846  		}
   847  
   848  		var dt flatbuf.Map
   849  		dt.Init(data.Bytes, data.Pos)
   850  		ret := arrow.MapOf(pairType.Field(0).Type, pairType.Field(1).Type)
   851  		ret.SetItemNullable(pairType.Field(1).Nullable)
   852  		ret.KeysSorted = dt.KeysSorted()
   853  		return ret, nil
   854  
   855  	case flatbuf.TypeRunEndEncoded:
   856  		if len(children) != 2 {
   857  			return nil, fmt.Errorf("%w: arrow/ipc: RunEndEncoded must have exactly 2 child fields", arrow.ErrInvalid)
   858  		}
   859  		switch children[0].Type.ID() {
   860  		case arrow.INT16, arrow.INT32, arrow.INT64:
   861  		default:
   862  			return nil, fmt.Errorf("%w: arrow/ipc: run-end encoded run_ends field must be one of int16, int32, or int64 type", arrow.ErrInvalid)
   863  		}
   864  		return arrow.RunEndEncodedOf(children[0].Type, children[1].Type), nil
   865  
   866  	default:
   867  		panic(fmt.Errorf("arrow/ipc: type %v not implemented", flatbuf.EnumNamesType[typ]))
   868  	}
   869  }
   870  
   871  func intFromFB(data flatbuf.Int) (arrow.DataType, error) {
   872  	bw := data.BitWidth()
   873  	if bw > 64 {
   874  		return nil, fmt.Errorf("arrow/ipc: integers with more than 64 bits not implemented (bits=%d)", bw)
   875  	}
   876  	if bw < 8 {
   877  		return nil, fmt.Errorf("arrow/ipc: integers with less than 8 bits not implemented (bits=%d)", bw)
   878  	}
   879  
   880  	switch bw {
   881  	case 8:
   882  		if !data.IsSigned() {
   883  			return arrow.PrimitiveTypes.Uint8, nil
   884  		}
   885  		return arrow.PrimitiveTypes.Int8, nil
   886  
   887  	case 16:
   888  		if !data.IsSigned() {
   889  			return arrow.PrimitiveTypes.Uint16, nil
   890  		}
   891  		return arrow.PrimitiveTypes.Int16, nil
   892  
   893  	case 32:
   894  		if !data.IsSigned() {
   895  			return arrow.PrimitiveTypes.Uint32, nil
   896  		}
   897  		return arrow.PrimitiveTypes.Int32, nil
   898  
   899  	case 64:
   900  		if !data.IsSigned() {
   901  			return arrow.PrimitiveTypes.Uint64, nil
   902  		}
   903  		return arrow.PrimitiveTypes.Int64, nil
   904  	default:
   905  		return nil, fmt.Errorf("arrow/ipc: integers not in cstdint are not implemented")
   906  	}
   907  }
   908  
   909  func intToFB(b *flatbuffers.Builder, bw int32, isSigned bool) flatbuffers.UOffsetT {
   910  	flatbuf.IntStart(b)
   911  	flatbuf.IntAddBitWidth(b, bw)
   912  	flatbuf.IntAddIsSigned(b, isSigned)
   913  	return flatbuf.IntEnd(b)
   914  }
   915  
   916  func floatFromFB(data flatbuf.FloatingPoint) (arrow.DataType, error) {
   917  	switch p := data.Precision(); p {
   918  	case flatbuf.PrecisionHALF:
   919  		return arrow.FixedWidthTypes.Float16, nil
   920  	case flatbuf.PrecisionSINGLE:
   921  		return arrow.PrimitiveTypes.Float32, nil
   922  	case flatbuf.PrecisionDOUBLE:
   923  		return arrow.PrimitiveTypes.Float64, nil
   924  	default:
   925  		return nil, fmt.Errorf("arrow/ipc: floating point type with %d precision not implemented", p)
   926  	}
   927  }
   928  
   929  func floatToFB(b *flatbuffers.Builder, bw int32) flatbuffers.UOffsetT {
   930  	switch bw {
   931  	case 16:
   932  		flatbuf.FloatingPointStart(b)
   933  		flatbuf.FloatingPointAddPrecision(b, flatbuf.PrecisionHALF)
   934  		return flatbuf.FloatingPointEnd(b)
   935  	case 32:
   936  		flatbuf.FloatingPointStart(b)
   937  		flatbuf.FloatingPointAddPrecision(b, flatbuf.PrecisionSINGLE)
   938  		return flatbuf.FloatingPointEnd(b)
   939  	case 64:
   940  		flatbuf.FloatingPointStart(b)
   941  		flatbuf.FloatingPointAddPrecision(b, flatbuf.PrecisionDOUBLE)
   942  		return flatbuf.FloatingPointEnd(b)
   943  	default:
   944  		panic(fmt.Errorf("arrow/ipc: invalid floating point precision %d-bits", bw))
   945  	}
   946  }
   947  
   948  func decimalFromFB(data flatbuf.Decimal) (arrow.DataType, error) {
   949  	switch data.BitWidth() {
   950  	case 128:
   951  		return &arrow.Decimal128Type{Precision: data.Precision(), Scale: data.Scale()}, nil
   952  	case 256:
   953  		return &arrow.Decimal256Type{Precision: data.Precision(), Scale: data.Scale()}, nil
   954  	default:
   955  		return nil, fmt.Errorf("arrow/ipc: invalid decimal bitwidth: %d", data.BitWidth())
   956  	}
   957  }
   958  
   959  func timeFromFB(data flatbuf.Time) (arrow.DataType, error) {
   960  	bw := data.BitWidth()
   961  	unit := unitFromFB(data.Unit())
   962  
   963  	switch bw {
   964  	case 32:
   965  		switch unit {
   966  		case arrow.Millisecond:
   967  			return arrow.FixedWidthTypes.Time32ms, nil
   968  		case arrow.Second:
   969  			return arrow.FixedWidthTypes.Time32s, nil
   970  		default:
   971  			return nil, fmt.Errorf("arrow/ipc: Time32 type with %v unit not implemented", unit)
   972  		}
   973  	case 64:
   974  		switch unit {
   975  		case arrow.Nanosecond:
   976  			return arrow.FixedWidthTypes.Time64ns, nil
   977  		case arrow.Microsecond:
   978  			return arrow.FixedWidthTypes.Time64us, nil
   979  		default:
   980  			return nil, fmt.Errorf("arrow/ipc: Time64 type with %v unit not implemented", unit)
   981  		}
   982  	default:
   983  		return nil, fmt.Errorf("arrow/ipc: Time type with %d bitwidth not implemented", bw)
   984  	}
   985  }
   986  
   987  func timestampFromFB(data flatbuf.Timestamp) (arrow.DataType, error) {
   988  	unit := unitFromFB(data.Unit())
   989  	tz := string(data.Timezone())
   990  	return &arrow.TimestampType{Unit: unit, TimeZone: tz}, nil
   991  }
   992  
   993  func dateFromFB(data flatbuf.Date) (arrow.DataType, error) {
   994  	switch data.Unit() {
   995  	case flatbuf.DateUnitDAY:
   996  		return arrow.FixedWidthTypes.Date32, nil
   997  	case flatbuf.DateUnitMILLISECOND:
   998  		return arrow.FixedWidthTypes.Date64, nil
   999  	}
  1000  	return nil, fmt.Errorf("arrow/ipc: Date type with %d unit not implemented", data.Unit())
  1001  }
  1002  
  1003  func intervalFromFB(data flatbuf.Interval) (arrow.DataType, error) {
  1004  	switch data.Unit() {
  1005  	case flatbuf.IntervalUnitYEAR_MONTH:
  1006  		return arrow.FixedWidthTypes.MonthInterval, nil
  1007  	case flatbuf.IntervalUnitDAY_TIME:
  1008  		return arrow.FixedWidthTypes.DayTimeInterval, nil
  1009  	case flatbuf.IntervalUnitMONTH_DAY_NANO:
  1010  		return arrow.FixedWidthTypes.MonthDayNanoInterval, nil
  1011  	}
  1012  	return nil, fmt.Errorf("arrow/ipc: Interval type with %d unit not implemented", data.Unit())
  1013  }
  1014  
  1015  func durationFromFB(data flatbuf.Duration) (arrow.DataType, error) {
  1016  	switch data.Unit() {
  1017  	case flatbuf.TimeUnitSECOND:
  1018  		return arrow.FixedWidthTypes.Duration_s, nil
  1019  	case flatbuf.TimeUnitMILLISECOND:
  1020  		return arrow.FixedWidthTypes.Duration_ms, nil
  1021  	case flatbuf.TimeUnitMICROSECOND:
  1022  		return arrow.FixedWidthTypes.Duration_us, nil
  1023  	case flatbuf.TimeUnitNANOSECOND:
  1024  		return arrow.FixedWidthTypes.Duration_ns, nil
  1025  	}
  1026  	return nil, fmt.Errorf("arrow/ipc: Duration type with %d unit not implemented", data.Unit())
  1027  }
  1028  
  1029  type customMetadataer interface {
  1030  	CustomMetadataLength() int
  1031  	CustomMetadata(*flatbuf.KeyValue, int) bool
  1032  }
  1033  
  1034  func metadataFromFB(md customMetadataer) (arrow.Metadata, error) {
  1035  	var (
  1036  		keys = make([]string, md.CustomMetadataLength())
  1037  		vals = make([]string, md.CustomMetadataLength())
  1038  	)
  1039  
  1040  	for i := range keys {
  1041  		var kv flatbuf.KeyValue
  1042  		if !md.CustomMetadata(&kv, i) {
  1043  			return arrow.Metadata{}, fmt.Errorf("arrow/ipc: could not read key-value %d from flatbuffer", i)
  1044  		}
  1045  		keys[i] = string(kv.Key())
  1046  		vals[i] = string(kv.Value())
  1047  	}
  1048  
  1049  	return arrow.NewMetadata(keys, vals), nil
  1050  }
  1051  
  1052  func metadataToFB(b *flatbuffers.Builder, meta arrow.Metadata, start startVecFunc) flatbuffers.UOffsetT {
  1053  	if meta.Len() == 0 {
  1054  		return 0
  1055  	}
  1056  
  1057  	n := meta.Len()
  1058  	kvs := make([]flatbuffers.UOffsetT, n)
  1059  	for i := range kvs {
  1060  		k := b.CreateString(meta.Keys()[i])
  1061  		v := b.CreateString(meta.Values()[i])
  1062  		flatbuf.KeyValueStart(b)
  1063  		flatbuf.KeyValueAddKey(b, k)
  1064  		flatbuf.KeyValueAddValue(b, v)
  1065  		kvs[i] = flatbuf.KeyValueEnd(b)
  1066  	}
  1067  
  1068  	start(b, n)
  1069  	for i := n - 1; i >= 0; i-- {
  1070  		b.PrependUOffsetT(kvs[i])
  1071  	}
  1072  	return b.EndVector(n)
  1073  }
  1074  
  1075  func schemaFromFB(schema *flatbuf.Schema, memo *dictutils.Memo) (*arrow.Schema, error) {
  1076  	var (
  1077  		err    error
  1078  		fields = make([]arrow.Field, schema.FieldsLength())
  1079  		pos    = dictutils.NewFieldPos()
  1080  	)
  1081  
  1082  	for i := range fields {
  1083  		var field flatbuf.Field
  1084  		if !schema.Fields(&field, i) {
  1085  			return nil, fmt.Errorf("arrow/ipc: could not read field %d from schema", i)
  1086  		}
  1087  
  1088  		fields[i], err = fieldFromFB(&field, pos.Child(int32(i)), memo)
  1089  		if err != nil {
  1090  			return nil, fmt.Errorf("arrow/ipc: could not convert field %d from flatbuf: %w", i, err)
  1091  		}
  1092  	}
  1093  
  1094  	md, err := metadataFromFB(schema)
  1095  	if err != nil {
  1096  		return nil, fmt.Errorf("arrow/ipc: could not convert schema metadata from flatbuf: %w", err)
  1097  	}
  1098  
  1099  	return arrow.NewSchemaWithEndian(fields, &md, endian.Endianness(schema.Endianness())), nil
  1100  }
  1101  
  1102  func schemaToFB(b *flatbuffers.Builder, schema *arrow.Schema, memo *dictutils.Mapper) flatbuffers.UOffsetT {
  1103  	fields := make([]flatbuffers.UOffsetT, schema.NumFields())
  1104  	pos := dictutils.NewFieldPos()
  1105  	for i := 0; i < schema.NumFields(); i++ {
  1106  		fields[i] = fieldToFB(b, pos.Child(int32(i)), schema.Field(i), memo)
  1107  	}
  1108  
  1109  	flatbuf.SchemaStartFieldsVector(b, len(fields))
  1110  	for i := len(fields) - 1; i >= 0; i-- {
  1111  		b.PrependUOffsetT(fields[i])
  1112  	}
  1113  	fieldsFB := b.EndVector(len(fields))
  1114  
  1115  	metaFB := metadataToFB(b, schema.Metadata(), flatbuf.SchemaStartCustomMetadataVector)
  1116  
  1117  	flatbuf.SchemaStart(b)
  1118  	flatbuf.SchemaAddEndianness(b, flatbuf.Endianness(schema.Endianness()))
  1119  	flatbuf.SchemaAddFields(b, fieldsFB)
  1120  	flatbuf.SchemaAddCustomMetadata(b, metaFB)
  1121  	offset := flatbuf.SchemaEnd(b)
  1122  
  1123  	return offset
  1124  }
  1125  
  1126  // payloadFromSchema returns a slice of payloads corresponding to the given schema.
  1127  // Callers of payloadFromSchema will need to call Release after use.
  1128  func payloadFromSchema(schema *arrow.Schema, mem memory.Allocator, memo *dictutils.Mapper) payloads {
  1129  	ps := make(payloads, 1)
  1130  	ps[0].msg = MessageSchema
  1131  	ps[0].meta = writeSchemaMessage(schema, mem, memo)
  1132  
  1133  	return ps
  1134  }
  1135  
  1136  func writeFBBuilder(b *flatbuffers.Builder, mem memory.Allocator) *memory.Buffer {
  1137  	raw := b.FinishedBytes()
  1138  	buf := memory.NewResizableBuffer(mem)
  1139  	buf.Resize(len(raw))
  1140  	copy(buf.Bytes(), raw)
  1141  	return buf
  1142  }
  1143  
  1144  func writeMessageFB(b *flatbuffers.Builder, mem memory.Allocator, hdrType flatbuf.MessageHeader, hdr flatbuffers.UOffsetT, bodyLen int64) *memory.Buffer {
  1145  
  1146  	flatbuf.MessageStart(b)
  1147  	flatbuf.MessageAddVersion(b, flatbuf.MetadataVersion(currentMetadataVersion))
  1148  	flatbuf.MessageAddHeaderType(b, hdrType)
  1149  	flatbuf.MessageAddHeader(b, hdr)
  1150  	flatbuf.MessageAddBodyLength(b, bodyLen)
  1151  	msg := flatbuf.MessageEnd(b)
  1152  	b.Finish(msg)
  1153  
  1154  	return writeFBBuilder(b, mem)
  1155  }
  1156  
  1157  func writeSchemaMessage(schema *arrow.Schema, mem memory.Allocator, dict *dictutils.Mapper) *memory.Buffer {
  1158  	b := flatbuffers.NewBuilder(1024)
  1159  	schemaFB := schemaToFB(b, schema, dict)
  1160  	return writeMessageFB(b, mem, flatbuf.MessageHeaderSchema, schemaFB, 0)
  1161  }
  1162  
  1163  func writeFileFooter(schema *arrow.Schema, dicts, recs []fileBlock, w io.Writer) error {
  1164  	var (
  1165  		b    = flatbuffers.NewBuilder(1024)
  1166  		memo dictutils.Mapper
  1167  	)
  1168  	memo.ImportSchema(schema)
  1169  
  1170  	schemaFB := schemaToFB(b, schema, &memo)
  1171  	dictsFB := fileBlocksToFB(b, dicts, flatbuf.FooterStartDictionariesVector)
  1172  	recsFB := fileBlocksToFB(b, recs, flatbuf.FooterStartRecordBatchesVector)
  1173  
  1174  	flatbuf.FooterStart(b)
  1175  	flatbuf.FooterAddVersion(b, flatbuf.MetadataVersion(currentMetadataVersion))
  1176  	flatbuf.FooterAddSchema(b, schemaFB)
  1177  	flatbuf.FooterAddDictionaries(b, dictsFB)
  1178  	flatbuf.FooterAddRecordBatches(b, recsFB)
  1179  	footer := flatbuf.FooterEnd(b)
  1180  
  1181  	b.Finish(footer)
  1182  
  1183  	_, err := w.Write(b.FinishedBytes())
  1184  	return err
  1185  }
  1186  
  1187  func writeRecordMessage(mem memory.Allocator, size, bodyLength int64, fields []fieldMetadata, meta []bufferMetadata, codec flatbuf.CompressionType, variadicCounts []int64) *memory.Buffer {
  1188  	b := flatbuffers.NewBuilder(0)
  1189  	recFB := recordToFB(b, size, bodyLength, fields, meta, codec, variadicCounts)
  1190  	return writeMessageFB(b, mem, flatbuf.MessageHeaderRecordBatch, recFB, bodyLength)
  1191  }
  1192  
  1193  func writeDictionaryMessage(mem memory.Allocator, id int64, isDelta bool, size, bodyLength int64, fields []fieldMetadata, meta []bufferMetadata, codec flatbuf.CompressionType, variadicCounts []int64) *memory.Buffer {
  1194  	b := flatbuffers.NewBuilder(0)
  1195  	recFB := recordToFB(b, size, bodyLength, fields, meta, codec, variadicCounts)
  1196  
  1197  	flatbuf.DictionaryBatchStart(b)
  1198  	flatbuf.DictionaryBatchAddId(b, id)
  1199  	flatbuf.DictionaryBatchAddData(b, recFB)
  1200  	flatbuf.DictionaryBatchAddIsDelta(b, isDelta)
  1201  	dictFB := flatbuf.DictionaryBatchEnd(b)
  1202  	return writeMessageFB(b, mem, flatbuf.MessageHeaderDictionaryBatch, dictFB, bodyLength)
  1203  }
  1204  
  1205  func recordToFB(b *flatbuffers.Builder, size, bodyLength int64, fields []fieldMetadata, meta []bufferMetadata, codec flatbuf.CompressionType, variadicCounts []int64) flatbuffers.UOffsetT {
  1206  	fieldsFB := writeFieldNodes(b, fields, flatbuf.RecordBatchStartNodesVector)
  1207  	metaFB := writeBuffers(b, meta, flatbuf.RecordBatchStartBuffersVector)
  1208  	var bodyCompressFB flatbuffers.UOffsetT
  1209  	if codec != -1 {
  1210  		bodyCompressFB = writeBodyCompression(b, codec)
  1211  	}
  1212  
  1213  	var vcFB *flatbuffers.UOffsetT
  1214  	if len(variadicCounts) > 0 {
  1215  		flatbuf.RecordBatchStartVariadicBufferCountsVector(b, len(variadicCounts))
  1216  		for i := len(variadicCounts) - 1; i >= 0; i-- {
  1217  			b.PrependInt64(variadicCounts[i])
  1218  		}
  1219  		vcFBVal := b.EndVector(len(variadicCounts))
  1220  		vcFB = &vcFBVal
  1221  	}
  1222  
  1223  	flatbuf.RecordBatchStart(b)
  1224  	flatbuf.RecordBatchAddLength(b, size)
  1225  	flatbuf.RecordBatchAddNodes(b, fieldsFB)
  1226  	flatbuf.RecordBatchAddBuffers(b, metaFB)
  1227  	if vcFB != nil {
  1228  		flatbuf.RecordBatchAddVariadicBufferCounts(b, *vcFB)
  1229  	}
  1230  
  1231  	if codec != -1 {
  1232  		flatbuf.RecordBatchAddCompression(b, bodyCompressFB)
  1233  	}
  1234  
  1235  	return flatbuf.RecordBatchEnd(b)
  1236  }
  1237  
  1238  func writeFieldNodes(b *flatbuffers.Builder, fields []fieldMetadata, start startVecFunc) flatbuffers.UOffsetT {
  1239  
  1240  	start(b, len(fields))
  1241  	for i := len(fields) - 1; i >= 0; i-- {
  1242  		field := fields[i]
  1243  		if field.Offset != 0 {
  1244  			panic(fmt.Errorf("arrow/ipc: field metadata for IPC must have offset 0"))
  1245  		}
  1246  		flatbuf.CreateFieldNode(b, field.Len, field.Nulls)
  1247  	}
  1248  
  1249  	return b.EndVector(len(fields))
  1250  }
  1251  
  1252  func writeBuffers(b *flatbuffers.Builder, buffers []bufferMetadata, start startVecFunc) flatbuffers.UOffsetT {
  1253  	start(b, len(buffers))
  1254  	for i := len(buffers) - 1; i >= 0; i-- {
  1255  		buffer := buffers[i]
  1256  		flatbuf.CreateBuffer(b, buffer.Offset, buffer.Len)
  1257  	}
  1258  	return b.EndVector(len(buffers))
  1259  }
  1260  
  1261  func writeBodyCompression(b *flatbuffers.Builder, codec flatbuf.CompressionType) flatbuffers.UOffsetT {
  1262  	flatbuf.BodyCompressionStart(b)
  1263  	flatbuf.BodyCompressionAddCodec(b, codec)
  1264  	flatbuf.BodyCompressionAddMethod(b, flatbuf.BodyCompressionMethodBUFFER)
  1265  	return flatbuf.BodyCompressionEnd(b)
  1266  }
  1267  
  1268  func writeMessage(msg *memory.Buffer, alignment int32, w io.Writer) (int, error) {
  1269  	var (
  1270  		n   int
  1271  		err error
  1272  	)
  1273  
  1274  	// ARROW-3212: we do not make any assumption on whether the output stream is aligned or not.
  1275  	paddedMsgLen := int32(msg.Len()) + 8
  1276  	remainder := paddedMsgLen % alignment
  1277  	if remainder != 0 {
  1278  		paddedMsgLen += alignment - remainder
  1279  	}
  1280  
  1281  	tmp := make([]byte, 4)
  1282  
  1283  	// write continuation indicator, to address 8-byte alignment requirement from FlatBuffers.
  1284  	binary.LittleEndian.PutUint32(tmp, kIPCContToken)
  1285  	_, err = w.Write(tmp)
  1286  	if err != nil {
  1287  		return 0, fmt.Errorf("arrow/ipc: could not write continuation bit indicator: %w", err)
  1288  	}
  1289  
  1290  	// the returned message size includes the length prefix, the flatbuffer, + padding
  1291  	n = int(paddedMsgLen)
  1292  
  1293  	// write the flatbuffer size prefix, including padding
  1294  	sizeFB := paddedMsgLen - 8
  1295  	binary.LittleEndian.PutUint32(tmp, uint32(sizeFB))
  1296  	_, err = w.Write(tmp)
  1297  	if err != nil {
  1298  		return n, fmt.Errorf("arrow/ipc: could not write message flatbuffer size prefix: %w", err)
  1299  	}
  1300  
  1301  	// write the flatbuffer
  1302  	_, err = w.Write(msg.Bytes())
  1303  	if err != nil {
  1304  		return n, fmt.Errorf("arrow/ipc: could not write message flatbuffer: %w", err)
  1305  	}
  1306  
  1307  	// write any padding
  1308  	padding := paddedMsgLen - int32(msg.Len()) - 8
  1309  	if padding > 0 {
  1310  		_, err = w.Write(paddingBytes[:padding])
  1311  		if err != nil {
  1312  			return n, fmt.Errorf("arrow/ipc: could not write message padding bytes: %w", err)
  1313  		}
  1314  	}
  1315  
  1316  	return n, err
  1317  }