coldata

package
v0.23.2 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Feb 12, 2024 License: Apache-2.0 Imports: 19 Imported by: 0

Documentation

Overview

Package coldata exposes utilities for handling columnarized data.

Index

Constants

View Source
const BytesMaxInlineLength = int(unsafe.Offsetof(element{}.inlinedLength))

BytesMaxInlineLength is the maximum length of a []byte that can be inlined within element.

View Source
const DefaultColdataBatchSize = 1024

DefaultColdataBatchSize is the default value of coldata-batch-size.

View Source
const ElementSize = int64(unsafe.Sizeof(element{}))

ElementSize is the size of element object. It is expected to be 32 on a 64 bit system.

View Source
const FlatBytesOverhead = int64(unsafe.Sizeof(Bytes{}))

FlatBytesOverhead is the overhead of Bytes in bytes.

View Source
const MaxBatchSize = 4096

MaxBatchSize is the maximum acceptable size of batches.

Variables

View Source
var GetBatchMemSize func(Batch) int64

GetBatchMemSize returns the total memory footprint of the batch.

The implementation lives in the sql/colmem package since it depends on sem/tree, and we don't want to make coldata depend on that.

View Source
var VecsToStringWithRowPrefix func(vecs []Vec, length int, sel []int, prefix string) []string

VecsToStringWithRowPrefix returns a pretty representation of the vectors. This method will convert all vectors to datums in order to print everything in the same manner as the tree.Datum representation does. Each row is printed in a separate string.

The implementation lives in colconv package and is injected during the initialization.

View Source
var ZeroBatch = &zeroBatch{
	MemBatch: NewMemBatchWithCapacity(
		nil, 0, StandardColumnFactory,
	).(*MemBatch),
}

ZeroBatch is a schema-less Batch of length 0.

Functions

func AssertEquivalentBatches

func AssertEquivalentBatches(t testingT, expected, actual Batch)

AssertEquivalentBatches is a testing function that asserts that expected and actual are equivalent.

func BatchSize

func BatchSize() int

BatchSize is the maximum number of tuples that fit in a column batch.

func GetValueAt

func GetValueAt(v Vec, rowIdx int) interface{}

GetValueAt is an inefficient helper to get the value in a Vec when the type is unknown.

func ProportionalSize

func ProportionalSize(v Vec, length int64) int64

ProportionalSize calls the method of the same name on bytes-like vectors, panicking if not bytes-like.

func ResetIfBytesLike

func ResetIfBytesLike(v Vec)

ResetIfBytesLike calls Reset on v if it is bytes-like, noop otherwise.

func SetBatchSizeForTests

func SetBatchSizeForTests(newBatchSize int) error

SetBatchSizeForTests modifies batchSize variable. It should only be used in tests. batch sizes greater than MaxBatchSize will return an error.

func SetValueAt

func SetValueAt(v Vec, elem interface{}, rowIdx int)

SetValueAt is an inefficient helper to set the value in a Vec when the type is unknown.

Types

type Batch

type Batch interface {
	// Length returns the number of values in the columns in the batch.
	Length() int
	// SetLength sets the number of values in the columns in the batch. Note
	// that if the selection vector will be set or updated on the batch, it must
	// be set **before** setting the length.
	SetLength(int)
	// Capacity returns the maximum number of values that can be stored in the
	// columns in the batch. Note that it could be a lower bound meaning some
	// of the Vecs could actually have larger underlying capacity (for example,
	// if they have been appended to).
	Capacity() int
	// Width returns the number of columns in the batch.
	Width() int
	// ColVec returns the ith Vec in this batch.
	ColVec(i int) Vec
	// ColVecs returns all of the underlying Vecs in this batch.
	ColVecs() []Vec
	// Selection, if not nil, returns the selection vector on this batch: a
	// densely-packed list of the *increasing* indices in each column that have
	// not been filtered out by a previous step.
	// TODO(yuzefovich): consider ensuring that the length of the returned slice
	// equals the length of the batch.
	Selection() []int
	// SetSelection sets whether this batch is using its selection vector or not.
	SetSelection(bool)
	// AppendCol appends the given Vec to this batch.
	AppendCol(Vec)
	// ReplaceCol replaces the current Vec at the provided index with the
	// provided Vec. The original and the replacement vectors *must* be of the
	// same type.
	ReplaceCol(Vec, int)
	// Reset modifies the caller in-place to have the given length and columns
	// with the given types. If it's possible, Reset will reuse the existing
	// columns and allocations, invalidating existing references to the Batch or
	// its Vecs. However, Reset does _not_ zero out the column data.
	//
	// NOTE: Reset can allocate a new Batch, so when calling from the vectorized
	// engine consider either allocating a new Batch explicitly via
	// colmem.Allocator or calling ResetInternalBatch.
	Reset(typs []*types.T, length int, factory ColumnFactory)
	// ResetInternalBatch resets a batch and its underlying Vecs for reuse. It's
	// important for callers to call ResetInternalBatch if they own internal
	// batches that they reuse as not doing this could result in correctness
	// or memory blowup issues. It unsets the selection and sets the length to
	// 0.
	ResetInternalBatch()
	// String returns a pretty representation of this batch.
	String() string
}

Batch is the type that columnar operators receive and produce. It represents a set of column vectors (partial data columns) as well as metadata about a batch, like the selection vector (which rows in the column batch are selected).

func NewMemBatch

func NewMemBatch(typs []*types.T, factory ColumnFactory) Batch

NewMemBatch allocates a new in-memory Batch. TODO(jordan): pool these allocations.

func NewMemBatchNoCols

func NewMemBatchNoCols(typs []*types.T, capacity int) Batch

NewMemBatchNoCols creates a "skeleton" of new in-memory Batch. It allocates memory for the selection vector but does *not* allocate any memory for the column vectors - those will have to be added separately.

func NewMemBatchWithCapacity

func NewMemBatchWithCapacity(typs []*types.T, capacity int, factory ColumnFactory) Batch

NewMemBatchWithCapacity allocates a new in-memory Batch with the given column size. Use for operators that have a precisely-sized output batch.

type Bools

type Bools []bool

Bools is a slice of bool.

func (Bools) CopySlice

func (c Bools) CopySlice(src Bools, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Bools) Get

func (c Bools) Get(idx int) bool

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Bools) Len

func (c Bools) Len() int

Len returns the length of the vector.

func (Bools) Set

func (c Bools) Set(idx int, val bool)

Set sets the element at index idx of the vector to val.

func (Bools) Window

func (c Bools) Window(start, end int) Bools

Window returns the window into the vector.

type Bytes

type Bytes struct {
	// contains filtered or unexported fields
}

Bytes is a vector that stores []byte values.

func NewBytes

func NewBytes(n int) *Bytes

NewBytes returns a Bytes struct with enough capacity to store n []byte values.

func (*Bytes) Abbreviated

func (b *Bytes) Abbreviated() []uint64

Abbreviated returns a uint64 slice where each uint64 represents the first eight bytes of each []byte. It is used for byte comparison fast paths.

Given Bytes b, and abbr = b.Abbreviated():

  • abbr[i] > abbr[j] iff b.Get(i) > b.Get(j)
  • abbr[i] < abbr[j] iff b.Get(i) < b.Get(j)
  • If abbr[i] == abbr[j], it is unknown if b.Get(i) is greater than, less than, or equal to b.Get(j). A full comparison of all bytes in each is required.

func (*Bytes) AppendSlice

func (b *Bytes) AppendSlice(src *Bytes, destIdx, srcStartIdx, srcEndIdx int)

AppendSlice appends srcStartIdx inclusive and srcEndIdx exclusive []byte values from src into the receiver starting at destIdx.

func (*Bytes) Copy

func (b *Bytes) Copy(src *Bytes, destIdx, srcIdx int)

Copy copies a single value from src at position srcIdx into position destIdx of the receiver. It is faster than b.Set(destIdx, src.Get(srcIdx)).

func (*Bytes) CopySlice

func (b *Bytes) CopySlice(src *Bytes, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies srcStartIdx inclusive and srcEndIdx exclusive []byte values from src into the receiver starting at destIdx. Similar to the copy builtin, min(dest.Len(), src.Len()) values will be copied.

func (*Bytes) Deserialize

func (b *Bytes) Deserialize(data []byte, offsets []int32)

Deserialize updates b according to the "arrow-like" format that was produced by Serialize.

func (*Bytes) ElemSize

func (b *Bytes) ElemSize(idx int) int64

ElemSize returns the size in bytes of the []byte elem at the given index. Panics if passed an invalid element.

func (*Bytes) Get

func (b *Bytes) Get(i int) []byte

Get returns the ith []byte in Bytes. Note that the returned byte slice is unsafe for reuse if any write operation happens.

If the returned value is then Set() into another Bytes, then use Bytes.Copy instead.

Note this function call is mostly inlined except in a handful of very large generated functions, so we can't add the gcassert directive for it.

func (*Bytes) Len

func (b *Bytes) Len() int

Len returns how many []byte values the receiver contains.

func (*Bytes) ProportionalSize

func (b *Bytes) ProportionalSize(n int64) int64

ProportionalSize returns the size of the receiver in bytes that is attributed to only first n out of Len() elements.

func (*Bytes) Reset

func (b *Bytes) Reset()

Reset resets the underlying Bytes for reuse. It is a noop if b is a window into another Bytes.

Calling it is not required for correctness and is only an optimization. Namely, this allows us to remove all "holes" (unused space) in b.buffer which can occur when an old non-inlined element is overwritten by a new element that is either fully-inlined or non-inlined but larger.

func (*Bytes) Serialize

func (b *Bytes) Serialize(n int, dataScratch []byte, offsetsScratch []int32) ([]byte, []int32)

Serialize converts b into the "arrow-like" (which is arrow-compatible) format.

We call this "arrow-like" because we're abusing the arrow format to get the best speed, possibly at the cost of increased allocations (when Bytes vector has been modified in-place many times via Sets at arbitrary positions with values of different lengths).

In particular, the arrow format represents bytes values via two slices - the flat []byte buffer and the offsets where len(offsets) = n + 1 (where n is the number of elements). ith element is then buffer[offsets[i]:offsets[i+1]. However, we squash b.elements (which is []element) and b.buffer to be stored in that flat byte slice, and we only need two positions in offsets to indicate the boundary between the two as well as the total data length. As a result, we have the following representation (which defeats the spirit of the arrow format but doesn't cause any issues anywhere):

 buffer = [<b.elements as []byte><b.buffer]
offsets = [0, 0, ..., 0, len(<b.elements as []byte>), len(<b.elements as []byte>) + len(buffer)]

Note: it is assumed that n is not larger than MaxBatchSize.

func (*Bytes) Set

func (b *Bytes) Set(i int, v []byte)

Set sets the ith []byte in Bytes.

If the provided value is obtained via Get() from another Bytes, then use Bytes.Copy instead.

Note this function call is mostly inlined except in a handful of very large generated functions, so we can't add the gcassert directive for it.

func (*Bytes) Size

func (b *Bytes) Size() int64

Size returns the total size of the receiver in bytes.

func (*Bytes) String

func (b *Bytes) String() string

String is used for debugging purposes.

func (*Bytes) Window

func (b *Bytes) Window(start, end int) *Bytes

Window creates a "window" into the receiver. It behaves similarly to Golang's slice, but the returned object is *not* allowed to be modified - it is read-only. If b is modified, then the returned object becomes invalid.

Window is a lightweight operation that doesn't involve copying the underlying data.

type Column

type Column interface {
	// Len returns the number of elements in the Column.
	Len() int
}

Column is an interface that represents a raw array of a Go native type.

type ColumnFactory

type ColumnFactory interface {
	MakeColumn(t *types.T, length int) Column
}

ColumnFactory is an interface that can construct columns for Batches.

var StandardColumnFactory ColumnFactory = &defaultColumnFactory{}

StandardColumnFactory is a factory that produces columns of types that are explicitly supported by the vectorized engine (i.e. not datum-backed).

type Datum

type Datum interface{}

Datum is abstract type for elements inside DatumVec, this type in reality should be tree.Datum. However, in order to avoid pulling in 'tree' package into the 'coldata' package, we use a runtime cast instead.

type DatumVec

type DatumVec interface {
	// Get returns the datum at index i in the vector. The datum cannot be used
	// anymore once the vector is modified.
	Get(i int) Datum
	// Set sets the datum at index i in the vector. It must check whether the
	// provided datum is compatible with the type that the DatumVec stores.
	Set(i int, v Datum)
	// Window creates a "window" into the vector. It behaves similarly to
	// Golang's slice.
	Window(start, end int) DatumVec
	// CopySlice copies srcStartIdx inclusive and srcEndIdx exclusive
	// tree.Datum values from src into the vector starting at destIdx.
	CopySlice(src DatumVec, destIdx, srcStartIdx, srcEndIdx int)
	// AppendSlice appends srcStartIdx inclusive and srcEndIdx exclusive
	// tree.Datum values from src into the vector starting at destIdx.
	AppendSlice(src DatumVec, destIdx, srcStartIdx, srcEndIdx int)
	// AppendVal appends the given tree.Datum value to the end of the vector.
	AppendVal(v Datum)
	// SetLength sets the length of the vector.
	SetLength(l int)
	// Len returns the length of the vector.
	Len() int
	// Cap returns the underlying capacity of the vector.
	Cap() int
	// MarshalAt returns the marshaled representation of datum at index i.
	MarshalAt(appendTo []byte, i int) ([]byte, error)
	// UnmarshalTo unmarshals the byte representation of a datum and sets it at
	// index i.
	UnmarshalTo(i int, b []byte) error
	// Size returns the total memory footprint of the vector (including the
	// internal memory used by tree.Datums) in bytes. It only accounts for the
	// size of the datum objects starting from the given index. So, Size is
	// relatively cheap when startIdx >= length, and expensive when
	// startIdx < length (with a maximum at zero). A nonzero startIdx should only
	// be used when elements before startIdx are guaranteed not to have been
	// modified.
	Size(startIdx int) int64
	// SetEvalCtx updates the vector with the provided *eval.Context.
	SetEvalCtx(evalCtx interface{})
}

DatumVec is the interface for a specialized vector that operates on tree.Datums in the vectorized engine. In order to avoid import of 'tree' package the implementation of DatumVec lives in 'coldataext' package.

type Decimals

type Decimals []apd.Decimal

Decimals is a slice of apd.Decimal.

func (Decimals) CopySlice

func (c Decimals) CopySlice(src Decimals, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

Note that this method is usually inlined, but it isn't in case of the memColumn.Copy generated code (probably because of the size of that function), so we don't assert the inlining with the GCAssert linter.

func (Decimals) Get

func (c Decimals) Get(idx int) apd.Decimal

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Decimals) Len

func (c Decimals) Len() int

Len returns the length of the vector.

func (Decimals) Set

func (c Decimals) Set(idx int, val apd.Decimal)

Set sets the element at index idx of the vector to val.

Note that this method is usually inlined, but it isn't in case of the merge joiner generated code (probably because of the size of the functions), so we don't assert the inlining with the GCAssert linter.

func (Decimals) Window

func (c Decimals) Window(start, end int) Decimals

Window returns the window into the vector.

type Durations

type Durations []duration.Duration

Durations is a slice of duration.Duration.

func (Durations) CopySlice

func (c Durations) CopySlice(src Durations, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Durations) Get

func (c Durations) Get(idx int) duration.Duration

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Durations) Len

func (c Durations) Len() int

Len returns the length of the vector.

func (Durations) Set

func (c Durations) Set(idx int, val duration.Duration)

Set sets the element at index idx of the vector to val.

func (Durations) Window

func (c Durations) Window(start, end int) Durations

Window returns the window into the vector.

type Float64s

type Float64s []float64

Float64s is a slice of float64.

func (Float64s) CopySlice

func (c Float64s) CopySlice(src Float64s, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Float64s) Get

func (c Float64s) Get(idx int) float64

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Float64s) Len

func (c Float64s) Len() int

Len returns the length of the vector.

func (Float64s) Set

func (c Float64s) Set(idx int, val float64)

Set sets the element at index idx of the vector to val.

func (Float64s) Window

func (c Float64s) Window(start, end int) Float64s

Window returns the window into the vector.

type Int16s

type Int16s []int16

Int16s is a slice of int16.

func (Int16s) CopySlice

func (c Int16s) CopySlice(src Int16s, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Int16s) Get

func (c Int16s) Get(idx int) int16

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Int16s) Len

func (c Int16s) Len() int

Len returns the length of the vector.

func (Int16s) Set

func (c Int16s) Set(idx int, val int16)

Set sets the element at index idx of the vector to val.

func (Int16s) Window

func (c Int16s) Window(start, end int) Int16s

Window returns the window into the vector.

type Int32s

type Int32s []int32

Int32s is a slice of int32.

func (Int32s) CopySlice

func (c Int32s) CopySlice(src Int32s, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Int32s) Get

func (c Int32s) Get(idx int) int32

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Int32s) Len

func (c Int32s) Len() int

Len returns the length of the vector.

func (Int32s) Set

func (c Int32s) Set(idx int, val int32)

Set sets the element at index idx of the vector to val.

func (Int32s) Window

func (c Int32s) Window(start, end int) Int32s

Window returns the window into the vector.

type Int64s

type Int64s []int64

Int64s is a slice of int64.

func (Int64s) CopySlice

func (c Int64s) CopySlice(src Int64s, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Int64s) Get

func (c Int64s) Get(idx int) int64

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Int64s) Len

func (c Int64s) Len() int

Len returns the length of the vector.

func (Int64s) Set

func (c Int64s) Set(idx int, val int64)

Set sets the element at index idx of the vector to val.

func (Int64s) Window

func (c Int64s) Window(start, end int) Int64s

Window returns the window into the vector.

type JSONs

type JSONs struct {
	Bytes
	// contains filtered or unexported fields
}

JSONs is a representation of columnar JSON data. It's simply a wrapper around the flat Bytes structure. To pull a JSON out of the structure, we construct a new "encodedJSON" object from scratch on demand.

func NewJSONs

func NewJSONs(n int) *JSONs

NewJSONs returns a new JSONs presized to n elements.

func (*JSONs) AppendSlice

func (js *JSONs) AppendSlice(src *JSONs, destIdx, srcStartIdx, srcEndIdx int)

AppendSlice appends srcStartIdx inclusive and srcEndIdx exclusive JSON values from src into the receiver starting at destIdx.

func (*JSONs) Copy

func (js *JSONs) Copy(src *JSONs, destIdx, srcIdx int)

Copy copies a single value from src at position srcIdx into position destIdx of the receiver.

func (*JSONs) CopySlice

func (js *JSONs) CopySlice(src *JSONs, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies srcStartIdx inclusive and srcEndIdx exclusive []byte values from src into the receiver starting at destIdx. See Bytes.CopySlice.

func (*JSONs) Get

func (js *JSONs) Get(i int) json.JSON

Get returns the ith JSON in JSONs. Note that the returned JSON is unsafe for reuse if any write operation happens. NOTE: if ith element was never set in any way, the behavior of Get is undefined.

func (*JSONs) Set

func (js *JSONs) Set(i int, j json.JSON)

Set sets the ith JSON in JSONs.

func (*JSONs) String

func (js *JSONs) String() string

String is used for debugging purposes.

func (*JSONs) Window

func (js *JSONs) Window(start, end int) *JSONs

Window creates a "window" into the receiver. It behaves similarly to Golang's slice, but the returned object is *not* allowed to be modified - it is read-only. Window is a lightweight operation that doesn't involve copying the underlying data.

type MemBatch

type MemBatch struct {
	// contains filtered or unexported fields
}

MemBatch is an in-memory implementation of Batch.

func (*MemBatch) AppendCol

func (m *MemBatch) AppendCol(col Vec)

AppendCol implements the Batch interface.

func (*MemBatch) Capacity

func (m *MemBatch) Capacity() int

Capacity implements the Batch interface.

func (*MemBatch) ColVec

func (m *MemBatch) ColVec(i int) Vec

ColVec implements the Batch interface.

func (*MemBatch) ColVecs

func (m *MemBatch) ColVecs() []Vec

ColVecs implements the Batch interface.

func (*MemBatch) Length

func (m *MemBatch) Length() int

Length implements the Batch interface.

func (*MemBatch) ReplaceCol

func (m *MemBatch) ReplaceCol(col Vec, colIdx int)

ReplaceCol implements the Batch interface.

func (*MemBatch) Reset

func (m *MemBatch) Reset(typs []*types.T, length int, factory ColumnFactory)

Reset implements the Batch interface.

func (*MemBatch) ResetInternalBatch

func (m *MemBatch) ResetInternalBatch()

ResetInternalBatch implements the Batch interface.

func (*MemBatch) Selection

func (m *MemBatch) Selection() []int

Selection implements the Batch interface.

func (*MemBatch) SetLength

func (m *MemBatch) SetLength(length int)

SetLength implements the Batch interface.

func (*MemBatch) SetSelection

func (m *MemBatch) SetSelection(b bool)

SetSelection implements the Batch interface.

func (*MemBatch) String

func (m *MemBatch) String() string

String returns a pretty representation of this batch.

func (*MemBatch) Width

func (m *MemBatch) Width() int

Width implements the Batch interface.

type Nulls

type Nulls struct {
	// contains filtered or unexported fields
}

Nulls represents a list of potentially nullable values using a bitmap. It is intended to be used alongside a slice (e.g. in the Vec interface) -- if the ith bit is off, then the ith element in that slice should be treated as NULL.

func NewNulls

func NewNulls(len int) Nulls

NewNulls returns a new nulls vector, initialized with a length.

func (*Nulls) Copy

func (n *Nulls) Copy(other *Nulls)

Copy copies the contents of other into n.

func (*Nulls) MaybeHasNulls

func (n *Nulls) MaybeHasNulls() bool

MaybeHasNulls returns true if the column possibly has any null values, and returns false if the column definitely has no null values.

func (*Nulls) NullAt

func (n *Nulls) NullAt(i int) bool

NullAt returns true if the ith value of the column is null.

func (*Nulls) NullAtChecked

func (n *Nulls) NullAtChecked(i int) bool

NullAtChecked returns true if the ith value of the column is null and allows an uninitialized Nulls to represent "no nulls".

func (*Nulls) NullBitmap

func (n *Nulls) NullBitmap() []byte

NullBitmap returns the null bitmap.

func (Nulls) Or

func (n Nulls) Or(n2 Nulls) Nulls

Or returns a new Nulls vector where NullAt(i) iff n1.NullAt(i) or n2.NullAt(i).

func (*Nulls) SetNull

func (n *Nulls) SetNull(i int)

SetNull sets the ith value of the column to null.

func (*Nulls) SetNullBitmap

func (n *Nulls) SetNullBitmap(bm []byte, size int)

SetNullBitmap sets the validity of first size elements in n according to bm. The bits past the end of this size will be set to valid. It is assumed that n has enough capacity to store size number of elements. If bm is zero length or if size is 0, then all elements will be set to valid.

func (*Nulls) SetNullRange

func (n *Nulls) SetNullRange(startIdx int, endIdx int)

SetNullRange sets all the values in [startIdx, endIdx) to null.

func (*Nulls) SetNulls

func (n *Nulls) SetNulls()

SetNulls sets the column to have only null values.

func (*Nulls) Slice

func (n *Nulls) Slice(start int, end int) Nulls

Slice returns a new Nulls representing a slice of the current Nulls from [start, end).

func (*Nulls) Truncate

func (n *Nulls) Truncate(start int)

Truncate sets all values with index greater than or equal to start to null.

func (*Nulls) UnsetNull

func (n *Nulls) UnsetNull(i int)

UnsetNull unsets the ith values of the column.

func (*Nulls) UnsetNullRange

func (n *Nulls) UnsetNullRange(startIdx, endIdx int)

UnsetNullRange unsets all the nulls in the range [startIdx, endIdx). After using UnsetNullRange, n might not contain any null values, but maybeHasNulls could still be true.

func (*Nulls) UnsetNulls

func (n *Nulls) UnsetNulls()

UnsetNulls sets the column to have no null values.

func (*Nulls) UnsetNullsAfter

func (n *Nulls) UnsetNullsAfter(idx int)

UnsetNullsAfter sets all values with index greater than or equal to idx to non-null.

type SliceArgs

type SliceArgs struct {
	// Src is the data being appended.
	Src Vec
	// Sel is an optional slice specifying indices to append to the destination
	// slice. Note that Src{Start,End}Idx apply to Sel.
	Sel []int
	// DestIdx is the first index that Append will append to.
	DestIdx int
	// SrcStartIdx is the index of the first element in Src that Append will
	// append.
	SrcStartIdx int
	// SrcEndIdx is the exclusive end index of Src. i.e. the element in the index
	// before SrcEndIdx is the last element appended to the destination slice,
	// similar to Src[SrcStartIdx:SrcEndIdx].
	SrcEndIdx int
}

SliceArgs represents the arguments passed in to Vec.Append and Nulls.set.

type Times

type Times []time.Time

Times is a slice of time.Time.

func (Times) CopySlice

func (c Times) CopySlice(src Times, destIdx, srcStartIdx, srcEndIdx int)

CopySlice copies src[srcStartIdx:srcEndIdx] into c starting at position destIdx.

func (Times) Get

func (c Times) Get(idx int) time.Time

Get returns the element at index idx of the vector. The element cannot be used anymore once the vector is modified.

func (Times) Len

func (c Times) Len() int

Len returns the length of the vector.

func (Times) Set

func (c Times) Set(idx int, val time.Time)

Set sets the element at index idx of the vector to val.

func (Times) Window

func (c Times) Window(start, end int) Times

Window returns the window into the vector.

type TypedVecs

type TypedVecs struct {
	Vecs  []Vec
	Nulls []*Nulls

	// Fields below need to be accessed by an index mapped via ColsMap.
	BoolCols      []Bools
	BytesCols     []*Bytes
	DecimalCols   []Decimals
	Int16Cols     []Int16s
	Int32Cols     []Int32s
	Int64Cols     []Int64s
	Float64Cols   []Float64s
	TimestampCols []Times
	IntervalCols  []Durations
	JSONCols      []*JSONs
	DatumCols     []DatumVec
	// ColsMap contains the positions of the corresponding vectors in the slice
	// for the same types. For example, if we have a batch with
	//   types = [Int64, Int64, Bool, Bytes, Bool, Int64],
	// then ColsMap will be
	//                      [0, 1, 0, 0, 1, 2]
	//                       ^  ^  ^  ^  ^  ^
	//                       |  |  |  |  |  |
	//                       |  |  |  |  |  3rd among all Int64's
	//                       |  |  |  |  2nd among all Bool's
	//                       |  |  |  1st among all Bytes's
	//                       |  |  1st among all Bool's
	//                       |  2nd among all Int64's
	//                       1st among all Int64's
	ColsMap []int
}

TypedVecs represents a slice of Vecs that have been converted into the typed columns. The idea is that every Vec is stored both in Vecs slice as well as in the typed slice, in order. Components that know the type of the vector they are working with can then access the typed column directly, avoiding expensive type casts.

func (*TypedVecs) Reset

func (v *TypedVecs) Reset()

Reset performs a deep reset of v while keeping the references to the slices.

func (*TypedVecs) SetBatch

func (v *TypedVecs) SetBatch(batch Batch)

SetBatch updates TypedVecs to represent all vectors from batch.

type Vec

type Vec interface {
	// Type returns the type of data stored in this Vec. Consider whether
	// CanonicalTypeFamily() should be used instead.
	Type() *types.T
	// CanonicalTypeFamily returns the canonical type family of data stored in
	// this Vec.
	CanonicalTypeFamily() types.Family

	// Bool returns a bool list.
	Bool() Bools
	// Int16 returns an int16 slice.
	Int16() Int16s
	// Int32 returns an int32 slice.
	Int32() Int32s
	// Int64 returns an int64 slice.
	Int64() Int64s
	// Float64 returns a float64 slice.
	Float64() Float64s
	// Bytes returns a flat Bytes representation.
	Bytes() *Bytes
	// Decimal returns an apd.Decimal slice.
	Decimal() Decimals
	// Timestamp returns a time.Time slice.
	Timestamp() Times
	// Interval returns a duration.Duration slice.
	Interval() Durations
	// JSON returns a vector of JSONs.
	JSON() *JSONs
	// Datum returns a vector of Datums.
	Datum() DatumVec

	// Col returns the raw, typeless backing storage for this Vec.
	Col() Column

	// SetCol sets the member column (in the case of mutable columns).
	SetCol(Column)

	// TemplateType returns an []interface{} and is used for operator templates.
	// Do not call this from normal code - it'll always panic.
	TemplateType() []interface{}

	// Append uses SliceArgs to append elements of a source Vec into this Vec.
	// It is logically equivalent to:
	// destVec = append(destVec[:args.DestIdx], args.Src[args.SrcStartIdx:args.SrcEndIdx])
	// An optional Sel slice can also be provided to apply a filter on the source
	// Vec.
	// Refer to the SliceArgs comment for specifics and TestAppend for examples.
	//
	// Note: Append()'ing from a Vector into itself is not supported.
	Append(SliceArgs)

	// Copy uses SliceArgs to copy elements of a source Vec into this Vec. It is
	// logically equivalent to:
	// copy(destVec[args.DestIdx:], args.Src[args.SrcStartIdx:args.SrcEndIdx])
	// An optional Sel slice can also be provided to apply a filter on the source
	// Vec.
	// Refer to the SliceArgs comment for specifics and TestCopy for examples.
	Copy(SliceArgs)

	// CopyWithReorderedSource copies a value at position order[sel[i]] in src
	// into the receiver at position sel[i]. len(sel) elements are copied.
	// Resulting values of elements not mentioned in sel are undefined after
	// this function.
	CopyWithReorderedSource(src Vec, sel, order []int)

	// Window returns a "window" into the Vec. A "window" is similar to Golang's
	// slice of the current Vec from [start, end), but the returned object is NOT
	// allowed to be modified (the modification might result in an undefined
	// behavior).
	Window(start int, end int) Vec

	// MaybeHasNulls returns true if the column possibly has any null values, and
	// returns false if the column definitely has no null values.
	MaybeHasNulls() bool

	// Nulls returns the nulls vector for the column.
	Nulls() *Nulls

	// SetNulls sets the nulls vector for this column.
	SetNulls(Nulls)

	// Length returns the length of the slice that is underlying this Vec.
	Length() int

	// Capacity returns the capacity of the Golang's slice that is underlying
	// this Vec. Note that if there is no "slice" (like in case of flat bytes),
	// then "capacity" of such object is equal to the number of elements.
	Capacity() int
}

Vec is an interface that represents a column vector that's accessible by Go native types.

func NewMemColumn

func NewMemColumn(t *types.T, length int, factory ColumnFactory) Vec

NewMemColumn returns a new memColumn, initialized with a length using the given column factory.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL