github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/col/colserde/file.go (about)

     1  // Copyright 2019 The Cockroach Authors.
     2  //
     3  // Use of this software is governed by the Business Source License
     4  // included in the file licenses/BSL.txt.
     5  //
     6  // As of the Change Date specified in that file, in accordance with
     7  // the Business Source License, use of this software will be governed
     8  // by the Apache License, Version 2.0, included in the file
     9  // licenses/APL.txt.
    10  
    11  package colserde
    12  
    13  import (
    14  	"bytes"
    15  	"encoding/binary"
    16  	"io"
    17  	"os"
    18  
    19  	"github.com/apache/arrow/go/arrow/array"
    20  	"github.com/cockroachdb/cockroach/pkg/col/coldata"
    21  	"github.com/cockroachdb/cockroach/pkg/col/colserde/arrowserde"
    22  	"github.com/cockroachdb/cockroach/pkg/col/typeconv"
    23  	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
    24  	"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
    25  	"github.com/cockroachdb/cockroach/pkg/sql/types"
    26  	"github.com/cockroachdb/errors"
    27  	mmap "github.com/edsrzf/mmap-go"
    28  	flatbuffers "github.com/google/flatbuffers/go"
    29  )
    30  
    31  const fileMagic = `ARROW1`
    32  
    33  var fileMagicPadding [8 - len(fileMagic)]byte
    34  
    35  type fileBlock struct {
    36  	offset      int64
    37  	metadataLen int32
    38  	bodyLen     int64
    39  }
    40  
    41  // FileSerializer converts our in-mem columnar batch representation into the
    42  // arrow specification's file format. All batches serialized to a file must have
    43  // the same schema.
    44  type FileSerializer struct {
    45  	scratch [4]byte
    46  
    47  	w    *countingWriter
    48  	typs []*types.T
    49  	fb   *flatbuffers.Builder
    50  	a    *ArrowBatchConverter
    51  	rb   *RecordBatchSerializer
    52  
    53  	recordBatches []fileBlock
    54  }
    55  
    56  // NewFileSerializer creates a FileSerializer for the given types. The caller is
    57  // responsible for closing the given writer.
    58  func NewFileSerializer(w io.Writer, typs []*types.T) (*FileSerializer, error) {
    59  	a, err := NewArrowBatchConverter(typs)
    60  	if err != nil {
    61  		return nil, err
    62  	}
    63  	rb, err := NewRecordBatchSerializer(typs)
    64  	if err != nil {
    65  		return nil, err
    66  	}
    67  	s := &FileSerializer{
    68  		typs: typs,
    69  		fb:   flatbuffers.NewBuilder(flatbufferBuilderInitialCapacity),
    70  		a:    a,
    71  		rb:   rb,
    72  	}
    73  	return s, s.Reset(w)
    74  }
    75  
    76  // Reset can be called to reuse this FileSerializer with a new io.Writer after
    77  // calling Finish. The types will remain the ones passed to the constructor. The
    78  // caller is responsible for closing the given writer.
    79  func (s *FileSerializer) Reset(w io.Writer) error {
    80  	if s.w != nil {
    81  		return errors.New(`Finish must be called before Reset`)
    82  	}
    83  	s.w = &countingWriter{wrapped: w}
    84  	s.recordBatches = s.recordBatches[:0]
    85  	if _, err := io.WriteString(s.w, fileMagic); err != nil {
    86  		return err
    87  	}
    88  	// Pad to 8 byte boundary.
    89  	if _, err := s.w.Write(fileMagicPadding[:]); err != nil {
    90  		return err
    91  	}
    92  
    93  	// The file format is a wrapper around the streaming format and the streaming
    94  	// format starts with a Schema message.
    95  	s.fb.Reset()
    96  	messageOffset := schemaMessage(s.fb, s.typs)
    97  	s.fb.Finish(messageOffset)
    98  	schemaBytes := s.fb.FinishedBytes()
    99  	if _, err := s.w.Write(schemaBytes); err != nil {
   100  		return err
   101  	}
   102  	_, err := s.w.Write(make([]byte, calculatePadding(len(schemaBytes))))
   103  	return err
   104  }
   105  
   106  // AppendBatch adds one batch of columnar data to the file.
   107  func (s *FileSerializer) AppendBatch(batch coldata.Batch) error {
   108  	offset := int64(s.w.written)
   109  
   110  	arrow, err := s.a.BatchToArrow(batch)
   111  	if err != nil {
   112  		return err
   113  	}
   114  	metadataLen, bodyLen, err := s.rb.Serialize(s.w, arrow)
   115  	if err != nil {
   116  		return err
   117  	}
   118  
   119  	s.recordBatches = append(s.recordBatches, fileBlock{
   120  		offset:      offset,
   121  		metadataLen: int32(metadataLen),
   122  		bodyLen:     int64(bodyLen),
   123  	})
   124  	return nil
   125  }
   126  
   127  // Finish writes the footer metadata described by the arrow spec. Nothing can be
   128  // called after Finish except Reset.
   129  func (s *FileSerializer) Finish() error {
   130  	defer func() {
   131  		s.w = nil
   132  	}()
   133  
   134  	// Write the footer flatbuffer, which has byte offsets of all the record
   135  	// batch messages in the file.
   136  	s.fb.Reset()
   137  	footerOffset := fileFooter(s.fb, s.typs, s.recordBatches)
   138  	s.fb.Finish(footerOffset)
   139  	footerBytes := s.fb.FinishedBytes()
   140  	if _, err := s.w.Write(footerBytes); err != nil {
   141  		return err
   142  	}
   143  	// For the footer, and only the footer, the spec requires the length _after_
   144  	// the footer so that it can be read by starting at the back of the file and
   145  	// working forward.
   146  	binary.LittleEndian.PutUint32(s.scratch[:], uint32(len(footerBytes)))
   147  	if _, err := s.w.Write(s.scratch[:]); err != nil {
   148  		return err
   149  	}
   150  	// Spec wants the magic again here.
   151  	_, err := io.WriteString(s.w, fileMagic)
   152  	return err
   153  }
   154  
   155  // FileDeserializer decodes columnar data batches from files encoded according
   156  // to the arrow spec.
   157  type FileDeserializer struct {
   158  	buf        []byte
   159  	bufCloseFn func() error
   160  
   161  	recordBatches []fileBlock
   162  
   163  	idx  int
   164  	end  int
   165  	typs []*types.T
   166  	a    *ArrowBatchConverter
   167  	rb   *RecordBatchSerializer
   168  
   169  	arrowScratch []*array.Data
   170  }
   171  
   172  // NewFileDeserializerFromBytes constructs a FileDeserializer for an in-memory
   173  // buffer.
   174  func NewFileDeserializerFromBytes(typs []*types.T, buf []byte) (*FileDeserializer, error) {
   175  	return newFileDeserializer(typs, buf, func() error { return nil })
   176  }
   177  
   178  // NewFileDeserializerFromPath constructs a FileDeserializer by reading it from
   179  // a file.
   180  func NewFileDeserializerFromPath(typs []*types.T, path string) (*FileDeserializer, error) {
   181  	f, err := os.Open(path)
   182  	if err != nil {
   183  		return nil, pgerror.Wrapf(err, pgcode.Io, `opening %s`, path)
   184  	}
   185  	// TODO(dan): This is currently using copy on write semantics because we store
   186  	// the nulls differently in-mem than arrow does and there's an in-place
   187  	// conversion. If we used the same format that arrow does, this could be
   188  	// switched to mmap.RDONLY (it's easy to check, the test fails with a SIGBUS
   189  	// right now with mmap.RDONLY).
   190  	buf, err := mmap.Map(f, mmap.COPY, 0 /* flags */)
   191  	if err != nil {
   192  		return nil, pgerror.Wrapf(err, pgcode.Io, `mmaping %s`, path)
   193  	}
   194  	return newFileDeserializer(typs, buf, buf.Unmap)
   195  }
   196  
   197  func newFileDeserializer(
   198  	typs []*types.T, buf []byte, bufCloseFn func() error,
   199  ) (*FileDeserializer, error) {
   200  	d := &FileDeserializer{
   201  		buf:        buf,
   202  		bufCloseFn: bufCloseFn,
   203  		end:        len(buf),
   204  	}
   205  	var err error
   206  	if err = d.init(); err != nil {
   207  		return nil, err
   208  	}
   209  	d.typs = typs
   210  
   211  	if d.a, err = NewArrowBatchConverter(typs); err != nil {
   212  		return nil, err
   213  	}
   214  	if d.rb, err = NewRecordBatchSerializer(typs); err != nil {
   215  		return nil, err
   216  	}
   217  	d.arrowScratch = make([]*array.Data, 0, len(typs))
   218  
   219  	return d, nil
   220  }
   221  
   222  // Close releases any resources held by this deserializer.
   223  func (d *FileDeserializer) Close() error {
   224  	return d.bufCloseFn()
   225  }
   226  
   227  // Typs returns the in-memory types for the data stored in this file.
   228  func (d *FileDeserializer) Typs() []*types.T {
   229  	return d.typs
   230  }
   231  
   232  // NumBatches returns the number of record batches stored in this file.
   233  func (d *FileDeserializer) NumBatches() int {
   234  	return len(d.recordBatches)
   235  }
   236  
   237  // GetBatch fills in the given in-mem batch with the requested on-disk data.
   238  func (d *FileDeserializer) GetBatch(batchIdx int, b coldata.Batch) error {
   239  	rb := d.recordBatches[batchIdx]
   240  	d.idx = int(rb.offset)
   241  	buf, err := d.read(metadataLengthNumBytes + int(rb.metadataLen) + int(rb.bodyLen))
   242  	if err != nil {
   243  		return err
   244  	}
   245  	d.arrowScratch = d.arrowScratch[:0]
   246  	if err := d.rb.Deserialize(&d.arrowScratch, buf); err != nil {
   247  		return err
   248  	}
   249  	return d.a.ArrowToBatch(d.arrowScratch, b)
   250  }
   251  
   252  // read gets the next `n` bytes from the start of the buffer, consuming them.
   253  func (d *FileDeserializer) read(n int) ([]byte, error) {
   254  	if d.idx+n > d.end {
   255  		return nil, io.EOF
   256  	}
   257  	start := d.idx
   258  	d.idx += n
   259  	return d.buf[start:d.idx], nil
   260  }
   261  
   262  // readBackward gets the `n` bytes from the end of the buffer, consuming them.
   263  func (d *FileDeserializer) readBackward(n int) ([]byte, error) {
   264  	if d.idx+n > d.end {
   265  		return nil, io.EOF
   266  	}
   267  	end := d.end
   268  	d.end -= n
   269  	return d.buf[d.end:end], nil
   270  }
   271  
   272  // init verifies the file magic and headers. After init, the `idx` and `end`
   273  // fields are set to the range of record batches and dictionary batches
   274  // described by the arrow spec's streaming format.
   275  func (d *FileDeserializer) init() error {
   276  	// Check the header magic
   277  	if magic, err := d.read(8); err != nil {
   278  		return pgerror.Wrap(err, pgcode.DataException, `verifying arrow file header magic`)
   279  	} else if !bytes.Equal([]byte(fileMagic), magic[:len(fileMagic)]) {
   280  		return errors.New(`arrow file header magic mismatch`)
   281  	}
   282  	if magic, err := d.readBackward(len(fileMagic)); err != nil {
   283  		return pgerror.Wrap(err, pgcode.DataException, `verifying arrow file footer magic`)
   284  	} else if !bytes.Equal([]byte(fileMagic), magic) {
   285  		return errors.New(`arrow file magic footer mismatch`)
   286  	}
   287  
   288  	footerSize, err := d.readBackward(4)
   289  	if err != nil {
   290  		return pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`)
   291  	}
   292  	footerBytes, err := d.readBackward(int(binary.LittleEndian.Uint32(footerSize)))
   293  	if err != nil {
   294  		return pgerror.Wrap(err, pgcode.DataException, `reading arrow file footer`)
   295  	}
   296  	footer := arrowserde.GetRootAsFooter(footerBytes, 0)
   297  	if footer.Version() != arrowserde.MetadataVersionV1 {
   298  		return errors.Errorf(`only arrow V1 is supported got %d`, footer.Version())
   299  	}
   300  
   301  	var block arrowserde.Block
   302  	d.recordBatches = d.recordBatches[:0]
   303  	for blockIdx := 0; blockIdx < footer.RecordBatchesLength(); blockIdx++ {
   304  		footer.RecordBatches(&block, blockIdx)
   305  		d.recordBatches = append(d.recordBatches, fileBlock{
   306  			offset:      block.Offset(),
   307  			metadataLen: block.MetaDataLength(),
   308  			bodyLen:     block.BodyLength(),
   309  		})
   310  	}
   311  
   312  	return nil
   313  }
   314  
   315  type countingWriter struct {
   316  	wrapped io.Writer
   317  	written int
   318  }
   319  
   320  func (w *countingWriter) Write(buf []byte) (int, error) {
   321  	n, err := w.wrapped.Write(buf)
   322  	w.written += n
   323  	return n, err
   324  }
   325  
   326  func schema(fb *flatbuffers.Builder, typs []*types.T) flatbuffers.UOffsetT {
   327  	fieldOffsets := make([]flatbuffers.UOffsetT, len(typs))
   328  	for idx, typ := range typs {
   329  		var fbTyp byte
   330  		var fbTypOffset flatbuffers.UOffsetT
   331  		switch typeconv.TypeFamilyToCanonicalTypeFamily(typ.Family()) {
   332  		case types.BoolFamily:
   333  			arrowserde.BoolStart(fb)
   334  			fbTypOffset = arrowserde.BoolEnd(fb)
   335  			fbTyp = arrowserde.TypeBool
   336  		case types.BytesFamily:
   337  			arrowserde.BinaryStart(fb)
   338  			fbTypOffset = arrowserde.BinaryEnd(fb)
   339  			fbTyp = arrowserde.TypeBinary
   340  		case types.IntFamily:
   341  			switch typ.Width() {
   342  			case 16:
   343  				arrowserde.IntStart(fb)
   344  				arrowserde.IntAddBitWidth(fb, 16)
   345  				arrowserde.IntAddIsSigned(fb, 1)
   346  				fbTypOffset = arrowserde.IntEnd(fb)
   347  				fbTyp = arrowserde.TypeInt
   348  			case 32:
   349  				arrowserde.IntStart(fb)
   350  				arrowserde.IntAddBitWidth(fb, 32)
   351  				arrowserde.IntAddIsSigned(fb, 1)
   352  				fbTypOffset = arrowserde.IntEnd(fb)
   353  				fbTyp = arrowserde.TypeInt
   354  			case 0, 64:
   355  				arrowserde.IntStart(fb)
   356  				arrowserde.IntAddBitWidth(fb, 64)
   357  				arrowserde.IntAddIsSigned(fb, 1)
   358  				fbTypOffset = arrowserde.IntEnd(fb)
   359  				fbTyp = arrowserde.TypeInt
   360  			default:
   361  				panic(errors.Errorf(`unexpected int width %d`, typ.Width()))
   362  			}
   363  		case types.FloatFamily:
   364  			arrowserde.FloatingPointStart(fb)
   365  			arrowserde.FloatingPointAddPrecision(fb, arrowserde.PrecisionDOUBLE)
   366  			fbTypOffset = arrowserde.FloatingPointEnd(fb)
   367  			fbTyp = arrowserde.TypeFloatingPoint
   368  		case types.DecimalFamily:
   369  			// Decimals are marshaled into bytes, so we use binary headers.
   370  			arrowserde.BinaryStart(fb)
   371  			fbTypOffset = arrowserde.BinaryEnd(fb)
   372  			fbTyp = arrowserde.TypeDecimal
   373  		case types.TimestampTZFamily:
   374  			// Timestamps are marshaled into bytes, so we use binary headers.
   375  			arrowserde.BinaryStart(fb)
   376  			fbTypOffset = arrowserde.BinaryEnd(fb)
   377  			fbTyp = arrowserde.TypeTimestamp
   378  		case types.IntervalFamily:
   379  			// Intervals are marshaled into bytes, so we use binary headers.
   380  			arrowserde.BinaryStart(fb)
   381  			fbTypOffset = arrowserde.BinaryEnd(fb)
   382  			fbTyp = arrowserde.TypeInterval
   383  		case typeconv.DatumVecCanonicalTypeFamily:
   384  			// Datums are marshaled into bytes, so we use binary headers.
   385  			arrowserde.BinaryStart(fb)
   386  			fbTypOffset = arrowserde.BinaryEnd(fb)
   387  			fbTyp = arrowserde.TypeUtf8
   388  		default:
   389  			panic(errors.Errorf(`don't know how to map %s`, typ))
   390  		}
   391  		arrowserde.FieldStart(fb)
   392  		arrowserde.FieldAddTypeType(fb, fbTyp)
   393  		arrowserde.FieldAddType(fb, fbTypOffset)
   394  		fieldOffsets[idx] = arrowserde.FieldEnd(fb)
   395  	}
   396  
   397  	arrowserde.SchemaStartFieldsVector(fb, len(typs))
   398  	// flatbuffers adds everything back to front. Reverse iterate so they're in
   399  	// the right order when they come out.
   400  	for i := len(fieldOffsets) - 1; i >= 0; i-- {
   401  		fb.PrependUOffsetT(fieldOffsets[i])
   402  	}
   403  	fields := fb.EndVector(len(typs))
   404  
   405  	arrowserde.SchemaStart(fb)
   406  	arrowserde.SchemaAddFields(fb, fields)
   407  	return arrowserde.SchemaEnd(fb)
   408  }
   409  
   410  func schemaMessage(fb *flatbuffers.Builder, typs []*types.T) flatbuffers.UOffsetT {
   411  	schemaOffset := schema(fb, typs)
   412  	arrowserde.MessageStart(fb)
   413  	arrowserde.MessageAddVersion(fb, arrowserde.MetadataVersionV1)
   414  	arrowserde.MessageAddHeaderType(fb, arrowserde.MessageHeaderSchema)
   415  	arrowserde.MessageAddHeader(fb, schemaOffset)
   416  	return arrowserde.MessageEnd(fb)
   417  }
   418  
   419  func fileFooter(
   420  	fb *flatbuffers.Builder, typs []*types.T, recordBatches []fileBlock,
   421  ) flatbuffers.UOffsetT {
   422  	schemaOffset := schema(fb, typs)
   423  	arrowserde.FooterStartRecordBatchesVector(fb, len(recordBatches))
   424  	// flatbuffers adds everything back to front. Reverse iterate so they're in
   425  	// the right order when they come out.
   426  	for i := len(recordBatches) - 1; i >= 0; i-- {
   427  		rb := recordBatches[i]
   428  		arrowserde.CreateBlock(fb, rb.offset, rb.metadataLen, rb.bodyLen)
   429  	}
   430  	recordBatchesOffset := fb.EndVector(len(recordBatches))
   431  	arrowserde.FooterStart(fb)
   432  	arrowserde.FooterAddVersion(fb, arrowserde.MetadataVersionV1)
   433  	arrowserde.FooterAddSchema(fb, schemaOffset)
   434  	arrowserde.FooterAddRecordBatches(fb, recordBatchesOffset)
   435  	return arrowserde.FooterEnd(fb)
   436  }