github.com/cockroachdb/cockroachdb-parser@v0.23.3-0.20240213214944-911057d40c9a/pkg/col/coldata/bytes.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 coldata
    12  
    13  import (
    14  	"encoding/binary"
    15  	"fmt"
    16  	"reflect"
    17  	"strings"
    18  	"unsafe"
    19  
    20  	"github.com/cockroachdb/cockroachdb-parser/pkg/sql/colexecerror"
    21  	"github.com/cockroachdb/cockroachdb-parser/pkg/sql/types"
    22  	"github.com/cockroachdb/cockroachdb-parser/pkg/util/buildutil"
    23  	"github.com/cockroachdb/errors"
    24  )
    25  
    26  // element describes a single []byte value. If a value doesn't exceed
    27  // BytesMaxInlineLength (30 bytes on 64 bit systems), then the whole value is
    28  // stored within the element; otherwise, the element "points" at a region in
    29  // Bytes.buffer where the non-inlined value is stored.
    30  //
    31  // If the value is inlined, then the layout of element is used as follows:
    32  //
    33  //	         24-byte header | 6-byte padding
    34  //	element: .............................. | length | true
    35  //	Bytes.buffer: N/A
    36  //
    37  // where 30 dots describe the inlinable space followed by a single byte for the
    38  // length followed by a boolean 'true' indicating an inlined value.
    39  //
    40  // If the value is non-inlined, then the layout of element is used as follows:
    41  //
    42  //	                                          padding
    43  //	element: .offset. | ..len... | ..cap... | xxxxxx | x | false
    44  //	Bytes.buffer: xxxxxxxx | offset .... | xxxxxxxx
    45  //
    46  // where first 24 bytes contain our custom "header" of a byte slice that is
    47  // backed by Bytes.buffer. The following 7 bytes (the padding and the
    48  // inlinedLength byte) are not used, and then we have a boolean 'false'
    49  // indicating a non-inlined value.
    50  type element struct {
    51  	header sliceHeader
    52  	_      [6]byte
    53  	// inlinedLength contains the length of the []byte value if it is inlined.
    54  	// Since the length of that value is at most BytesMaxInlineLength (30), it
    55  	// can definitely fit into a single byte.
    56  	inlinedLength byte
    57  	// inlined indicates whether the byte value is inlined in the element.
    58  	inlined bool
    59  }
    60  
    61  // sliceHeader describes a non-inlined byte value that is backed by
    62  // Bytes.buffer.
    63  //
    64  // We do not just use []byte for two reasons:
    65  // 1. we can easily overwrite the whole struct when a value is inlined without
    66  // causing any issues with the GC (because our sliceHeader doesn't contain an
    67  // actual slice, GC doesn't have to track whether the backing array is live or
    68  // not);
    69  // 2. our slice header remains valid even when Bytes.buffer is reallocated. If
    70  // we were to use []byte, then whenever that reallocation occurred, we would
    71  // have to iterate over all non-inlined values and update the byte slices to
    72  // point to the new Bytes.buffer.
    73  type sliceHeader struct {
    74  	// bufferOffset stores the offset into Bytes.buffer where the corresponding
    75  	// bytes value begins.
    76  	bufferOffset int
    77  	len          int
    78  	cap          int
    79  }
    80  
    81  // ElementSize is the size of element object. It is expected to be 32 on a 64
    82  // bit system.
    83  const ElementSize = int64(unsafe.Sizeof(element{}))
    84  
    85  // BytesMaxInlineLength is the maximum length of a []byte that can be inlined
    86  // within element.
    87  const BytesMaxInlineLength = int(unsafe.Offsetof(element{}.inlinedLength))
    88  
    89  // inlinedSlice returns 30 bytes of space within e that can be used for storing
    90  // a value inlined, as a slice.
    91  //
    92  //gcassert:inline
    93  func (e *element) inlinedSlice() []byte {
    94  	return (*(*[BytesMaxInlineLength]byte)(unsafe.Pointer(&e.header)))[:]
    95  }
    96  
    97  //gcassert:inline
    98  func (e *element) get(b *Bytes) []byte {
    99  	if e.inlined {
   100  		return e.inlinedSlice()[:e.inlinedLength:BytesMaxInlineLength]
   101  	}
   102  	return e.getNonInlined(b)
   103  }
   104  
   105  //gcassert:inline
   106  func (e *element) getNonInlined(b *Bytes) []byte {
   107  	return b.buffer[e.header.bufferOffset : e.header.bufferOffset+e.header.len : e.header.bufferOffset+e.header.cap]
   108  }
   109  
   110  func (e *element) set(v []byte, b *Bytes) {
   111  	if len(v) <= BytesMaxInlineLength {
   112  		*e = element{inlinedLength: byte(len(v)), inlined: true}
   113  		copy(e.inlinedSlice(), v)
   114  	} else {
   115  		e.setNonInlined(v, b)
   116  	}
   117  }
   118  
   119  // copy copies the value contained in other into the receiver.
   120  func (e *element) copy(other element, dest, src *Bytes) {
   121  	if other.inlined {
   122  		*e = other
   123  		return
   124  	}
   125  	e.setNonInlined(other.getNonInlined(src), dest)
   126  }
   127  
   128  func (e *element) setNonInlined(v []byte, b *Bytes) {
   129  	// Check if we there was an old non-inlined value we can overwrite.
   130  	if !e.inlined && e.header.cap >= len(v) {
   131  		e.header.len = len(v)
   132  		copy(e.getNonInlined(b), v)
   133  	} else {
   134  		*e = element{
   135  			header: sliceHeader{
   136  				bufferOffset: len(b.buffer),
   137  				len:          len(v),
   138  				cap:          len(v),
   139  			},
   140  			inlined: false,
   141  		}
   142  		// Use a custom append to grow the buffer faster than go does by default.
   143  		if rem := cap(b.buffer) - len(b.buffer); rem < len(v) {
   144  			increment := cap(b.buffer)                  // at least double the buffer
   145  			if need := len(v) - rem; increment < need { // grow enough to fit v
   146  				increment = need
   147  			}
   148  			const initialBufferSize = 256 // don't go smaller than this
   149  			if increment < initialBufferSize {
   150  				increment = initialBufferSize
   151  			}
   152  			realloc := make([]byte, len(b.buffer), cap(b.buffer)+increment)
   153  			copy(realloc, b.buffer)
   154  			b.buffer = realloc
   155  		}
   156  		b.buffer = append(b.buffer, v...)
   157  	}
   158  }
   159  
   160  // Bytes is a vector that stores []byte values.
   161  type Bytes struct {
   162  	// elements contains all values stored in this Bytes vector.
   163  	elements []element
   164  	// buffer contains all values that couldn't be inlined within the
   165  	// corresponding elements.
   166  	buffer []byte
   167  	// isWindow indicates whether this Bytes is a "window" into another Bytes.
   168  	// If it is, no modifications are allowed (all of them will panic).
   169  	isWindow bool
   170  }
   171  
   172  // NewBytes returns a Bytes struct with enough capacity to store n []byte
   173  // values.
   174  func NewBytes(n int) *Bytes {
   175  	return &Bytes{elements: make([]element, n)}
   176  }
   177  
   178  // Get returns the ith []byte in Bytes. Note that the returned byte slice is
   179  // unsafe for reuse if any write operation happens.
   180  //
   181  // If the returned value is then Set() into another Bytes, then use Bytes.Copy
   182  // instead.
   183  //
   184  // Note this function call is mostly inlined except in a handful of very large
   185  // generated functions, so we can't add the gcassert directive for it.
   186  func (b *Bytes) Get(i int) []byte {
   187  	return b.elements[i].get(b)
   188  }
   189  
   190  // Set sets the ith []byte in Bytes.
   191  //
   192  // If the provided value is obtained via Get() from another Bytes, then use
   193  // Bytes.Copy instead.
   194  //
   195  // Note this function call is mostly inlined except in a handful of very large
   196  // generated functions, so we can't add the gcassert directive for it.
   197  func (b *Bytes) Set(i int, v []byte) {
   198  	if buildutil.CrdbTestBuild {
   199  		if b.isWindow {
   200  			panic("Set is called on a window into Bytes")
   201  		}
   202  	}
   203  	b.elements[i].set(v, b)
   204  }
   205  
   206  // Window creates a "window" into the receiver. It behaves similarly to Golang's
   207  // slice, but the returned object is *not* allowed to be modified - it is
   208  // read-only. If b is modified, then the returned object becomes invalid.
   209  //
   210  // Window is a lightweight operation that doesn't involve copying the underlying
   211  // data.
   212  func (b *Bytes) Window(start, end int) *Bytes {
   213  	if start < 0 || start > end || end > b.Len() {
   214  		panic(
   215  			fmt.Sprintf(
   216  				"invalid window arguments: start=%d end=%d when Bytes.Len()=%d",
   217  				start, end, b.Len(),
   218  			),
   219  		)
   220  	}
   221  	return &Bytes{
   222  		elements: b.elements[start:end],
   223  		buffer:   b.buffer,
   224  		isWindow: true,
   225  	}
   226  }
   227  
   228  // Copy copies a single value from src at position srcIdx into position destIdx
   229  // of the receiver. It is faster than b.Set(destIdx, src.Get(srcIdx)).
   230  func (b *Bytes) Copy(src *Bytes, destIdx, srcIdx int) {
   231  	if buildutil.CrdbTestBuild {
   232  		if b.isWindow {
   233  			panic("copy is called on a window into Bytes")
   234  		}
   235  	}
   236  	b.elements[destIdx].copy(src.elements[srcIdx], b, src)
   237  }
   238  
   239  // copyElements copies the provided elements into the receiver starting at
   240  // position destIdx. The method assumes that there are enough elements in the
   241  // receiver in [destIdx:] range to fit all the source elements.
   242  func (b *Bytes) copyElements(srcElementsToCopy []element, src *Bytes, destIdx int) {
   243  	if len(srcElementsToCopy) == 0 {
   244  		return
   245  	}
   246  	destElements := b.elements[destIdx:]
   247  	if buildutil.CrdbTestBuild {
   248  		if len(destElements) < len(srcElementsToCopy) {
   249  			panic(errors.AssertionFailedf("unexpectedly not enough destination elements"))
   250  		}
   251  	}
   252  	// Optimize copying of the elements by copying all of them directly into the
   253  	// destination. This way all inlined values become correctly set, and we
   254  	// only need to set the non-inlined values separately.
   255  	//
   256  	// Note that this behavior results in losing the references to the old
   257  	// non-inlined values, even if they could be reused. If Bytes is not Reset,
   258  	// then that unused space in Bytes.buffer can accumulate. However, checking
   259  	// whether there are old non-inlined values with non-zero capacity leads to
   260  	// performance regressions, and in the production code we do reset the Bytes
   261  	// in all cases, so we accept this poor behavior in such a hypothetical /
   262  	// test-only scenario. See #78703 for more details.
   263  	copy(destElements, srcElementsToCopy)
   264  	// Early bounds checks.
   265  	_ = destElements[len(srcElementsToCopy)-1]
   266  	for i := 0; i < len(srcElementsToCopy); i++ {
   267  		//gcassert:bce
   268  		if e := &destElements[i]; !e.inlined {
   269  			// This value is non-inlined, so at the moment it is pointing
   270  			// into the buffer of the source - we have to explicitly set it
   271  			// in order to copy the actual value's data into the buffer of
   272  			// b.
   273  			//
   274  			// First, unset the element so that we don't try to reuse the old
   275  			// non-inlined value to write the new value into - we do want to
   276  			// append the new value to b.buffer.
   277  			*e = element{}
   278  			//gcassert:bce
   279  			e.setNonInlined(srcElementsToCopy[i].getNonInlined(src), b)
   280  		}
   281  	}
   282  }
   283  
   284  // TODO(yuzefovich): unexport some of the methods (like CopySlice and
   285  // AppendSlice).
   286  
   287  // CopySlice copies srcStartIdx inclusive and srcEndIdx exclusive []byte values
   288  // from src into the receiver starting at destIdx. Similar to the copy builtin,
   289  // min(dest.Len(), src.Len()) values will be copied.
   290  func (b *Bytes) CopySlice(src *Bytes, destIdx, srcStartIdx, srcEndIdx int) {
   291  	if b.isWindow {
   292  		panic("CopySlice is called on a window into Bytes")
   293  	}
   294  	if destIdx < 0 || destIdx > b.Len() {
   295  		panic(
   296  			fmt.Sprintf(
   297  				"dest index %d out of range (len=%d)", destIdx, b.Len(),
   298  			),
   299  		)
   300  	} else if srcStartIdx < 0 || srcStartIdx > src.Len() ||
   301  		srcEndIdx > src.Len() || srcStartIdx > srcEndIdx {
   302  		panic(
   303  			fmt.Sprintf(
   304  				"source index start %d or end %d invalid (len=%d)",
   305  				srcStartIdx, srcEndIdx, src.Len(),
   306  			),
   307  		)
   308  	}
   309  	toCopy := srcEndIdx - srcStartIdx
   310  	if toCopy == 0 || b.Len() == 0 || destIdx == b.Len() {
   311  		return
   312  	}
   313  
   314  	if destIdx+toCopy > b.Len() {
   315  		// Reduce the number of elements to copy to what can fit into the
   316  		// destination.
   317  		toCopy = b.Len() - destIdx
   318  	}
   319  
   320  	if b == src && destIdx == srcStartIdx {
   321  		// No need to do anything.
   322  		return
   323  	}
   324  
   325  	if b != src {
   326  		srcElementsToCopy := src.elements[srcStartIdx : srcStartIdx+toCopy]
   327  		b.copyElements(srcElementsToCopy, src, destIdx)
   328  	} else if destIdx > srcStartIdx {
   329  		// If we're copying from the vector into itself and from the earlier
   330  		// part into the later part, we need to reverse the order (otherwise we
   331  		// would overwrite the later values before copying them, thus corrupting
   332  		// the vector).
   333  		for i := toCopy - 1; i >= 0; i-- {
   334  			b.elements[destIdx+i].copy(src.elements[srcStartIdx+i], b, src)
   335  		}
   336  	} else {
   337  		for i := 0; i < toCopy; i++ {
   338  			b.elements[destIdx+i].copy(src.elements[srcStartIdx+i], b, src)
   339  		}
   340  	}
   341  }
   342  
   343  // ensureLengthForAppend makes sure that b has enough elements to support newLen
   344  // as the new length of the vector for an append operation.
   345  func (b *Bytes) ensureLengthForAppend(destIdx, newLen int) {
   346  	if cap(b.elements) < newLen {
   347  		// If the elements slice doesn't have enough capacity, then it must be
   348  		// reallocated.
   349  		oldElements := b.elements
   350  		// Figure out the new capacity of the elements slice. We will be
   351  		// doubling the existing capacity until newLen elements can fit, unless
   352  		// currently elements slice is nil, in which case we use newLen as the
   353  		// new capacity.
   354  		newCap := cap(b.elements)
   355  		if newCap > 0 {
   356  			for newCap < newLen {
   357  				newCap *= 2
   358  			}
   359  		} else {
   360  			newCap = newLen
   361  		}
   362  		b.elements = make([]element, newLen, newCap)
   363  		copy(b.elements[:destIdx], oldElements)
   364  	} else {
   365  		// We don't have to reallocate, so we can just reuse the old elements.
   366  		b.elements = b.elements[:newLen]
   367  	}
   368  }
   369  
   370  // AppendSlice appends srcStartIdx inclusive and srcEndIdx exclusive []byte
   371  // values from src into the receiver starting at destIdx.
   372  func (b *Bytes) AppendSlice(src *Bytes, destIdx, srcStartIdx, srcEndIdx int) {
   373  	if b == src {
   374  		panic("AppendSlice when b == src is not supported")
   375  	}
   376  	if b.isWindow {
   377  		panic("AppendSlice is called on a window into Bytes")
   378  	}
   379  	if destIdx < 0 || destIdx > b.Len() {
   380  		panic(
   381  			fmt.Sprintf(
   382  				"dest index %d out of range (len=%d)", destIdx, b.Len(),
   383  			),
   384  		)
   385  	} else if srcStartIdx < 0 || srcStartIdx > src.Len() ||
   386  		srcEndIdx > src.Len() || srcStartIdx > srcEndIdx {
   387  		panic(
   388  			fmt.Sprintf(
   389  				"source index start %d or end %d invalid (len=%d)",
   390  				srcStartIdx, srcEndIdx, src.Len(),
   391  			),
   392  		)
   393  	}
   394  	srcElementsToCopy := src.elements[srcStartIdx:srcEndIdx]
   395  	newLen := destIdx + srcEndIdx - srcStartIdx
   396  	b.ensureLengthForAppend(destIdx, newLen)
   397  	b.copyElements(srcElementsToCopy, src, destIdx)
   398  }
   399  
   400  // appendSliceWithSel appends all values specified in sel from the source into
   401  // the receiver starting at position destIdx.
   402  func (b *Bytes) appendSliceWithSel(src *Bytes, destIdx int, sel []int) {
   403  	if b == src {
   404  		panic("appendSliceWithSel when b == src is not supported")
   405  	}
   406  	if b.isWindow {
   407  		panic("appendSliceWithSel is called on a window into Bytes")
   408  	}
   409  	if destIdx < 0 || destIdx > b.Len() {
   410  		panic(
   411  			fmt.Sprintf(
   412  				"dest index %d out of range (len=%d)", destIdx, b.Len(),
   413  			),
   414  		)
   415  	}
   416  	newLen := destIdx + len(sel)
   417  	b.ensureLengthForAppend(destIdx, newLen)
   418  	for i, srcIdx := range sel {
   419  		b.elements[destIdx+i].copy(src.elements[srcIdx], b, src)
   420  	}
   421  }
   422  
   423  // Len returns how many []byte values the receiver contains.
   424  //
   425  //gcassert:inline
   426  func (b *Bytes) Len() int {
   427  	return len(b.elements)
   428  }
   429  
   430  // FlatBytesOverhead is the overhead of Bytes in bytes.
   431  const FlatBytesOverhead = int64(unsafe.Sizeof(Bytes{}))
   432  
   433  // Size returns the total size of the receiver in bytes.
   434  func (b *Bytes) Size() int64 {
   435  	if b == nil {
   436  		return 0
   437  	}
   438  	return FlatBytesOverhead + int64(cap(b.elements))*ElementSize + int64(cap(b.buffer))
   439  }
   440  
   441  // ProportionalSize returns the size of the receiver in bytes that is attributed
   442  // to only first n out of Len() elements.
   443  func (b *Bytes) ProportionalSize(n int64) int64 {
   444  	if n == 0 {
   445  		return 0
   446  	}
   447  	s := FlatBytesOverhead
   448  	for i := 0; i < int(n); i++ {
   449  		s += b.ElemSize(i)
   450  	}
   451  	return s
   452  }
   453  
   454  // ElemSize returns the size in bytes of the []byte elem at the given index.
   455  // Panics if passed an invalid element.
   456  //
   457  //gcassert:inline
   458  func (b *Bytes) ElemSize(idx int) int64 {
   459  	if b.elements[idx].inlined {
   460  		return ElementSize
   461  	}
   462  	return ElementSize + int64(b.elements[idx].header.cap)
   463  }
   464  
   465  // Abbreviated returns a uint64 slice where each uint64 represents the first
   466  // eight bytes of each []byte. It is used for byte comparison fast paths.
   467  //
   468  // Given Bytes b, and abbr = b.Abbreviated():
   469  //
   470  //   - abbr[i] > abbr[j] iff b.Get(i) > b.Get(j)
   471  //   - abbr[i] < abbr[j] iff b.Get(i) < b.Get(j)
   472  //   - If abbr[i] == abbr[j], it is unknown if b.Get(i) is greater than, less
   473  //     than, or equal to b.Get(j). A full comparison of all bytes in each is
   474  //     required.
   475  func (b *Bytes) Abbreviated() []uint64 {
   476  	r := make([]uint64, b.Len())
   477  	for i := range r {
   478  		bs := b.Get(i)
   479  		r[i] = abbreviate(bs)
   480  	}
   481  	return r
   482  }
   483  
   484  // abbreviate interprets up to the first 8 bytes of the slice as a big-endian
   485  // uint64. If the slice has less than 8 bytes, the value returned is the same as
   486  // if the slice was filled to 8 bytes with zero value bytes. For example:
   487  //
   488  //	abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
   489  //	  => 1
   490  //
   491  //	abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00})
   492  //	  => 256
   493  //
   494  //	abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
   495  //	  => 256
   496  func abbreviate(bs []byte) uint64 {
   497  	if len(bs) >= 8 {
   498  		return binary.BigEndian.Uint64(bs)
   499  	}
   500  	var v uint64
   501  	for _, b := range bs {
   502  		v <<= 8
   503  		v |= uint64(b)
   504  	}
   505  	return v << uint(8*(8-len(bs)))
   506  }
   507  
   508  var zeroElements = make([]element, MaxBatchSize)
   509  
   510  // Reset resets the underlying Bytes for reuse. It is a noop if b is a window
   511  // into another Bytes.
   512  //
   513  // Calling it is not required for correctness and is only an optimization.
   514  // Namely, this allows us to remove all "holes" (unused space) in b.buffer which
   515  // can occur when an old non-inlined element is overwritten by a new element
   516  // that is either fully-inlined or non-inlined but larger.
   517  //
   518  //gcassert:inline
   519  func (b *Bytes) Reset() {
   520  	if b.isWindow {
   521  		return
   522  	}
   523  	// Zero out all elements, up to the capacity, and then restore the length of
   524  	// the vector.
   525  	l := len(b.elements)
   526  	b.elements = b.elements[:cap(b.elements)]
   527  	for n := 0; n < len(b.elements); {
   528  		n += copy(b.elements[n:], zeroElements)
   529  	}
   530  	b.elements = b.elements[:l]
   531  	b.buffer = b.buffer[:0]
   532  }
   533  
   534  // String is used for debugging purposes.
   535  func (b *Bytes) String() string {
   536  	var builder strings.Builder
   537  	for i := range b.elements {
   538  		builder.WriteString(
   539  			fmt.Sprintf("%d: %v\n", i, b.Get(i)),
   540  		)
   541  	}
   542  	return builder.String()
   543  }
   544  
   545  // elementsAsBytes unsafely casts b.elements[:n] to []byte.
   546  func (b *Bytes) elementsAsBytes(n int) []byte {
   547  	var bytes []byte
   548  	//lint:ignore SA1019 SliceHeader is deprecated, but no clear replacement
   549  	elementsHeader := (*reflect.SliceHeader)(unsafe.Pointer(&b.elements))
   550  	//lint:ignore SA1019 SliceHeader is deprecated, but no clear replacement
   551  	bytesHeader := (*reflect.SliceHeader)(unsafe.Pointer(&bytes))
   552  	bytesHeader.Data = elementsHeader.Data
   553  	bytesHeader.Len = int(ElementSize) * n
   554  	bytesHeader.Cap = int(ElementSize) * n
   555  	return bytes
   556  }
   557  
   558  var zeroInt32Slice []int32
   559  
   560  func init() {
   561  	zeroInt32Slice = make([]int32, MaxBatchSize)
   562  }
   563  
   564  // Serialize converts b into the "arrow-like" (which is arrow-compatible)
   565  // format.
   566  //
   567  // We call this "arrow-like" because we're abusing the arrow format to get the
   568  // best speed, possibly at the cost of increased allocations (when Bytes vector
   569  // has been modified in-place many times via Sets at arbitrary positions with
   570  // values of different lengths).
   571  //
   572  // In particular, the arrow format represents bytes values via two slices - the
   573  // flat []byte buffer and the offsets where len(offsets) = n + 1 (where n is the
   574  // number of elements). ith element is then buffer[offsets[i]:offsets[i+1].
   575  // However, we squash b.elements (which is []element) and b.buffer to be stored
   576  // in that flat byte slice, and we only need two positions in offsets to
   577  // indicate the boundary between the two as well as the total data length. As a
   578  // result, we have the following representation (which defeats the spirit of the
   579  // arrow format but doesn't cause any issues anywhere):
   580  //
   581  //	 buffer = [<b.elements as []byte><b.buffer]
   582  //	offsets = [0, 0, ..., 0, len(<b.elements as []byte>), len(<b.elements as []byte>) + len(buffer)]
   583  //
   584  // Note: it is assumed that n is not larger than MaxBatchSize.
   585  func (b *Bytes) Serialize(n int, dataScratch []byte, offsetsScratch []int32) ([]byte, []int32) {
   586  	if buildutil.CrdbTestBuild {
   587  		if n > MaxBatchSize {
   588  			colexecerror.InternalError(errors.AssertionFailedf(
   589  				"too many bytes elements to serialize: %d vs MaxBatchSize of %d", n, MaxBatchSize,
   590  			))
   591  		}
   592  	}
   593  	data := dataScratch[:0]
   594  	offsets := offsetsScratch[:0]
   595  
   596  	// Handle the cases of 0 and 1 elements separately since then we cannot
   597  	// store two offsets that we need.
   598  	if n == 0 {
   599  		offsets = append(offsets, 0)
   600  		return data, offsets
   601  	} else if n == 1 {
   602  		data = append(data, b.Get(0)...)
   603  		offsets = append(offsets, 0)
   604  		offsets = append(offsets, int32(len(data)))
   605  		return data, offsets
   606  	}
   607  
   608  	// Copy over b.elements treated as []byte as well as b.buffer into data.
   609  	bytes := b.elementsAsBytes(n)
   610  	if cap(data) < len(bytes)+len(b.buffer) {
   611  		data = make([]byte, 0, len(bytes)+len(b.buffer))
   612  	}
   613  	data = append(data, bytes...)
   614  	data = append(data, b.buffer...)
   615  
   616  	// Now populate the offsets slice which conforms to the arrow format and has
   617  	// the correct length.
   618  	offsets = append(offsets, zeroInt32Slice[:n-1]...)
   619  	offsets = append(offsets, int32(len(bytes)))
   620  	offsets = append(offsets, int32(len(data)))
   621  	return data, offsets
   622  }
   623  
   624  // Deserialize updates b according to the "arrow-like" format that was produced
   625  // by Serialize.
   626  func (b *Bytes) Deserialize(data []byte, offsets []int32) {
   627  	n := len(offsets) - 1
   628  	if cap(b.elements) < n {
   629  		b.elements = make([]element, n)
   630  	} else {
   631  		b.elements = b.elements[:n]
   632  	}
   633  	b.buffer = b.buffer[:0]
   634  	if n == 0 {
   635  		return
   636  	} else if n == 1 {
   637  		b.elements[0] = element{}
   638  		b.Set(0, data)
   639  		return
   640  	}
   641  	bytes := b.elementsAsBytes(n)
   642  	copy(bytes, data)
   643  	b.buffer = append(b.buffer, data[len(bytes):]...)
   644  }
   645  
   646  // ProportionalSize calls the method of the same name on bytes-like vectors,
   647  // panicking if not bytes-like.
   648  func ProportionalSize(v Vec, length int64) int64 {
   649  	family := v.CanonicalTypeFamily()
   650  	switch family {
   651  	case types.BytesFamily:
   652  		return v.Bytes().ProportionalSize(length)
   653  	case types.JsonFamily:
   654  		return v.JSON().ProportionalSize(length)
   655  	default:
   656  		colexecerror.InternalError(errors.AssertionFailedf("unsupported type %s", family))
   657  	}
   658  	return 0
   659  }
   660  
   661  // ResetIfBytesLike calls Reset on v if it is bytes-like, noop otherwise.
   662  func ResetIfBytesLike(v Vec) {
   663  	switch v.CanonicalTypeFamily() {
   664  	case types.BytesFamily:
   665  		v.Bytes().Reset()
   666  	case types.JsonFamily:
   667  		v.JSON().Reset()
   668  	}
   669  }