Documentation
¶
Overview ¶
Package vm implements the core query-processing "physical operators" that process streams of ion-encoded data.
Each of the "physical operators" (such as Filter, FilterDistinct, Project, etc.) implements QuerySink, and generally operators also accept a QuerySink into which they will write their outputs. Typically, each of the operators will inspect the interfaces implemented by the output QuerySink and choose a method of passing rows to that QuerySink that is more efficient than passing serialized rows via an io.WriteCloser.
Data is fed to a chain of vm.QuerySink operators via a call to vm.SplitInput or vm.Table.WriteChunks, and the final output of the query sink is directed to an io.Writer that is wrapped with vm.LockedSink.
Index ¶
- Constants
- Variables
- func Allocated(buf []byte) bool
- func DFA6TGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFA6TZGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFA7TGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFA7TZGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFA8TGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFA8TZGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DFALZGo(data []byte, maskIn uint16, offsets, sizes [16]uint32, dsByte []byte) uint16
- func DfaGoImpl(op bcop, data []byte, inputK uint16, offsets, sizes [16]uint32, dsByte []byte) kRegData
- func Free(buf []byte)
- func HintEndSegment(w io.Writer)
- func LeakCheck(w io.Writer, fn func())
- func Locked(dst io.Writer) io.Writer
- func Malloc() []byte
- func PagesUsed() int
- func SetOptimizationLevel(opt OptimizationLevel)
- func SplitInput(dst QuerySink, parallel int, into func(io.Writer) error) error
- func Trace(w io.Writer, flags TraceFlags)
- type AggBinding
- type Aggregate
- type AggregateOp
- type AggregateOpFn
- type Aggregation
- type BufferedTable
- type Count
- type Data
- type DistinctFilter
- type EndSegmentWriter
- type Filter
- type HashAggregate
- func (h *HashAggregate) Close() error
- func (h *HashAggregate) Limit(n int)
- func (h *HashAggregate) Open() (io.WriteCloser, error)
- func (h *HashAggregate) OrderByAggregate(n int, ordering SortOrdering) error
- func (h *HashAggregate) OrderByGroup(n int, ordering SortOrdering) error
- func (h *HashAggregate) OrderByWindow(n int, ordering SortOrdering) error
- func (h *HashAggregate) SetSkipEmpty(skip bool)
- type LengthZ3
- type Limit
- type Needle
- type OffsetZ2
- type OptimizationLevel
- type Order
- type Projection
- type QueryBuffer
- func (q *QueryBuffer) Alignment() int
- func (q *QueryBuffer) Bytes() []byte
- func (q *QueryBuffer) Close() error
- func (q *QueryBuffer) Open() (io.WriteCloser, error)
- func (q *QueryBuffer) Reset()
- func (q *QueryBuffer) SetAlignment(align int)
- func (q *QueryBuffer) Size() int64
- func (q *QueryBuffer) Table() *BufferedTable
- func (q *QueryBuffer) Write(buf []byte) (int, error)
- type QuerySink
- type ReaderAtTable
- type Rematerializer
- type Selection
- type SortColumn
- type SortDirection
- type SortLimit
- type SortNullsOrder
- type SortOrdering
- type Table
- type TeeWriter
- type TraceFlags
- type Unnest
- type Unpivot
- type UnpivotAtDistinct
Constants ¶
const ( // MaxAggregateBuckets is the maximum cardinality // of a hash aggregate (SUM(...) ... GROUP BY ...); // this is chosen somewhat arbitrarily to prevent // ridiculous memory consumption and the higher // likelihood of hash collisions MaxAggregateBuckets = 1 << 18 // MaxAggregateMemory is the maximum number of // bytes that the aggregate groups or values // can occupy. (This limit is applied to groups // and values separately, so the true max memory use // is roughly double this value.) MaxAggregateMemory = 1 << 24 )
const ( // TraceSSAText causes all compiled SSA programs // to be dumped in their textual representation. TraceSSAText = 1 << iota // TraceSSADot causes all compiled SSA programs // to be dumped in a format suitable for processing // with graphviz(1). TraceSSADot // TraceBytecodeText causes all compiled bytecode // programs to be dumped in a text-based format. TraceBytecodeText )
const MaxSymbolID = (1 << 21) - 1
MaxSymbolID is the largest symbol ID supported by the system.
const MaximumIonObjectSize = 2*1024*1024 - 1
MaximumIonObjectSize object size is limited by assembly implementation. Up to three varuint bytes are parsed (it's 3*7=21 bits).
const ( // PageSize is the granularity // of the allocations returned // by Malloc PageSize = pageSize )
Variables ¶
var ( ConstNBytesUtf8 = [16]uint32{1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 3, 4} ConstTailMask = [16]uint32{0, 0xFF, 0xFFFF, 0xFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF, 0xFFFFFFFF} ConstBswap32 = [16]uint32{0x00010203, 0x04050607, 0x08090A0B, 0x0C0D0E0F, 0x00010203, 0x04050607, 0x08090A0B, 0x0C0D0E0F, 0x00010203, 0x04050607, 0x08090A0B, 0x0C0D0E0F, 0x00010203, 0x04050607, 0x08090A0B, 0x0C0D0E0F} )
constants
var Errorf func(f string, args ...any)
Errorf is a global diagnostic function that can be set during init() to capture additional diagnostic information from the vm.
var LeakCheckHook func(stack []byte, obj any)
LeakCheckHook is a hook that can be set in test code to look for leaked RowConsumers. LeakCheckHook should not be set in production code.
Functions ¶
func Allocated ¶
Allocated returns true if buf was returned from Malloc, or false otherwise.
NOTE: Allocated will return true for a buffer allocated from Malloc even after it has been returned via Free. Allocated does *not* indicate whether the buffer is actually safe to access.
func DFA6TZGo ¶
DFA6TZGo go implementation of DfaT6Z Deterministic Finite Automaton (DFA) with 6-bits lookup-key and Zero length remaining assertion
func DFA7TZGo ¶
DFA7TZGo go implementation of DfaT7Z Deterministic Finite Automaton (DFA) with 7-bits lookup-key and Zero length remaining assertion
func DFA8TGo ¶
DFA8TGo go implementation of DfaT8 Deterministic Finite Automaton (DFA) with 8-bits lookup-key
func DFA8TZGo ¶
DFA8TZGo go implementation of DfaT8Z Deterministic Finite Automaton 8-bits with Zero length remaining assertion
func DFALZGo ¶
DFALZGo go implementation of DfaLZ Deterministic Finite Automaton(DFA) with unlimited capacity (Large) and Remaining Length Zero Assertion (RLZA)
func Free ¶
func Free(buf []byte)
Free frees a buffer that was returned by Malloc so that it can be re-used. The caller may not use the contents of buf after it has called Free.
func HintEndSegment ¶
HintEndSegment calls EndSegment() on w if it can be cast to an EndSegmentWriter.
Callers that partition data into logical segments that begin with a fresh symbol table can use HintEndSegment as a hint to release temporary resources (like vm memory) that are specific to the most-recently-processed segment.
func LeakCheck ¶
LeakCheck runs fn and writes the stack traces of all the page allocation sites to w for each page that was allocated within fn and was not freed. LeakCheck is not reentrancy-safe.
Note that LeakCheck *just* runs fn() unless -tags=vmemleaks is set.
func Locked ¶
Locked turns an io.Writer into a goroutine-safe io.Writer where each write is serialized against other writes. Locked takes into account whether dst is the result of another call to Locked or LockedSink and optimizes accordingly.
func Malloc ¶
func Malloc() []byte
Malloc returns a new buffer suitable for passing to VM operations.
If there is no VM memory available, Malloc panics.
func PagesUsed ¶
func PagesUsed() int
PagesUsed returns the number of currently-active pages returned by Malloc that have not been deactivated with a call to Free.
func SetOptimizationLevel ¶
func SetOptimizationLevel(opt OptimizationLevel)
SetOptimizationLevel sets SSA instructions to use opcodes from given optimization level.
NOTE: This function is not thread safe and can be only used at startup time or during testing. Its always called on startup to setup the defaults, but some tests can call it to make sure we are testing all possible features.
func SplitInput ¶
SplitInput is a helper function for writing the implementation of Table.WriteChunks. SplitInput calls dst.Open() up to parallel times, and then passes the destination to separate calls to into() in different goroutines. SplitInput takes care of closing the outputs returned from dst.Open() and waits for each goroutine to return.
func Trace ¶
func Trace(w io.Writer, flags TraceFlags)
Trace enables or disables tracing of bytecode program compilation.
To enable tracing, Trace should be called with a non-nil io.Writer and non-zero flags. To disable tracing, Trace should be called with a nil io.Writer and flags equal to zero.
Types ¶
type AggBinding ¶
AggBinding is a binding of a single aggregate expression to a result
func (AggBinding) Equals ¶
func (a AggBinding) Equals(x AggBinding) bool
func (*AggBinding) String ¶
func (a *AggBinding) String() string
type Aggregate ¶
type Aggregate struct { // Aggregated values (results from executing queries, even in parallel) AggregatedData []byte // contains filtered or unexported fields }
Aggregate is a QuerySink implementation that computes simple aggregations that do not use groups.
func NewAggregate ¶
func NewAggregate(bind Aggregation, rest QuerySink) (*Aggregate, error)
NewAggregate constructs an aggregation QuerySink.
func (*Aggregate) SetSkipEmpty ¶
SetSkipEmpty configures whether or not the Aggregate flushes any data to its output QuerySink when [Close] is called if zero rows have been written. The default behavior is to flush the "zero value" of the rows (typically [NULL]).
type AggregateOp ¶
type AggregateOp struct {
// contains filtered or unexported fields
}
AggregateOp describes aggregate operation
type AggregateOpFn ¶
type AggregateOpFn uint8
AggregateOpFn specifies the aggregate operation and its type.
const ( AggregateOpNone AggregateOpFn = iota AggregateOpSumF AggregateOpTDigest AggregateOpAvgF AggregateOpMinF AggregateOpMaxF AggregateOpSumI AggregateOpSumC AggregateOpAvgI AggregateOpMinI AggregateOpMaxI AggregateOpAndI AggregateOpOrI AggregateOpXorI AggregateOpAndK AggregateOpOrK AggregateOpMinTS AggregateOpMaxTS AggregateOpCount AggregateOpApproxCountDistinct )
func (AggregateOpFn) String ¶
func (o AggregateOpFn) String() string
type Aggregation ¶
type Aggregation []AggBinding
Aggregation is a list of aggregate bindings
func (Aggregation) Equals ¶
func (a Aggregation) Equals(x Aggregation) bool
func (Aggregation) String ¶
func (a Aggregation) String() string
type BufferedTable ¶
type BufferedTable struct {
// contains filtered or unexported fields
}
BufferedTable is a Table implementation that uses bytes that are present in memory.
func BufferTable ¶
func BufferTable(buf []byte, align int) *BufferedTable
BufferTable converts a buffer with a known chunk alignment into a Table
func (*BufferedTable) Bytes ¶
func (b *BufferedTable) Bytes() int64
func (*BufferedTable) Hits ¶
func (b *BufferedTable) Hits() int64
func (*BufferedTable) Misses ¶
func (b *BufferedTable) Misses() int64
func (*BufferedTable) Reset ¶
func (b *BufferedTable) Reset()
Reset resets the current read offset of the table so that another call to WriteChunks can be made.
func (*BufferedTable) Size ¶
func (b *BufferedTable) Size() int64
Size returns the number of bytes in the table
func (*BufferedTable) WriteChunks ¶
func (b *BufferedTable) WriteChunks(dst QuerySink, parallel int) error
WriteChunks implements Table.WriteChunks
type Count ¶
type Count struct {
// contains filtered or unexported fields
}
Count is a utility QuerySink that simply counts the number of rows that it receives.
type DistinctFilter ¶
type DistinctFilter struct {
// contains filtered or unexported fields
}
DistinctFilter is a QuerySink that deduplicates rows using a tuple of input rows
Note that deduplicated rows are returned in their entirety, but the contents of the fields that are not part of the deduplication condition are left unspecified. In other words, when there are duplicate rows, the first row to be selected as "distinct" can be any of the distinct rows.
func NewDistinct ¶
func NewDistinct(on []expr.Node, dst QuerySink) (*DistinctFilter, error)
NewDistinct creates a new DistinctFilter that filters out duplicate rows for which the tuple of expressions 'on' are duplicated.
func (*DistinctFilter) Close ¶
func (d *DistinctFilter) Close() error
func (*DistinctFilter) Limit ¶
func (d *DistinctFilter) Limit(n int64)
Limit sets a limit on the number of distinct rows to produce. (A limit <= 0 means an unlimited number of rows.)
func (*DistinctFilter) Open ¶
func (d *DistinctFilter) Open() (io.WriteCloser, error)
type EndSegmentWriter ¶
type EndSegmentWriter interface {
EndSegment()
}
EndSegmentWriter is implemented by some io.WriteClosers returned by QuerySink.Open.
See also: HintEndSegment.
type Filter ¶
type Filter struct {
// contains filtered or unexported fields
}
Filter is a concrete implementation of QuerySink that applies a filter to incoming rows.
type HashAggregate ¶
type HashAggregate struct {
// contains filtered or unexported fields
}
func NewHashAggregate ¶
func NewHashAggregate(agg, windows Aggregation, by Selection, dst QuerySink) (*HashAggregate, error)
func (*HashAggregate) Close ¶
func (h *HashAggregate) Close() error
func (*HashAggregate) Limit ¶
func (h *HashAggregate) Limit(n int)
Limit sets the maximum number of output rows. Limit <= 0 means there is no limit.
func (*HashAggregate) Open ¶
func (h *HashAggregate) Open() (io.WriteCloser, error)
func (*HashAggregate) OrderByAggregate ¶
func (h *HashAggregate) OrderByAggregate(n int, ordering SortOrdering) error
func (*HashAggregate) OrderByGroup ¶
func (h *HashAggregate) OrderByGroup(n int, ordering SortOrdering) error
func (*HashAggregate) OrderByWindow ¶
func (h *HashAggregate) OrderByWindow(n int, ordering SortOrdering) error
func (*HashAggregate) SetSkipEmpty ¶
func (h *HashAggregate) SetSkipEmpty(skip bool)
SetSkipEmpty configures whether or not the HashAggregate flushes any data to its output QuerySink when [Close] is called if zero rows have been written. The default behavior is to flush the "zero value" of the rows (typically [NULL]).
type Limit ¶
type Limit struct {
// contains filtered or unexported fields
}
Limit is a QuerySink that limits the number of rows written to the next QuerySink.
See NewLimit
type OptimizationLevel ¶
type OptimizationLevel uint32
OptimizationLevel describes which optimizations Sneller can use.
const ( // Don't use any optimizations. OptimizationLevelNone OptimizationLevel = iota // Use AVX-512 level 1 optimizations (baseline). // // Baseline AVX-512 requires F, BW, DQ, CD, and VL features. OptimizationLevelAVX512V1 // Use AVX-512 level 2 optimizations (IceLake and Zen 4+). // // AVX-512 level 2 requires BITALG, GFNI, IFMA, VAES, VBMI, VBMI2, // VPCLMULQDQ, and VPOPCNTDQ. OptimizationLevelAVX512V2 // Autodetect optimizations based on environment variable // (SNELLER_OPT_LEVEL) and detected CPU features. OptimizationLevelDetect = OptimizationLevel(0xFFFFFFFF) )
func DetectOptimizationLevel ¶
func DetectOptimizationLevel() OptimizationLevel
DetectOptimizationLevel detects the optimization level to use based on both CPU and `SNELLER_OPT_LEVEL` environment variable, which is useful to override the detection.
func GetOptimizationLevel ¶
func GetOptimizationLevel() OptimizationLevel
GetOptimizationLevel returns the optimization level currently in use.
type Order ¶
type Order struct {
// contains filtered or unexported fields
}
Order implements a QuerySink that applies an ordering to its output rows.
func NewOrder ¶
func NewOrder(dst io.Writer, columns []SortColumn, limit *SortLimit, parallelism int) (*Order, error)
NewOrder constructs a new Order QuerySink that sorts the provided columns (in left-to-right order). If limit is non-nil, then the number of rows output by the Order will be less than or equal to the limit.
type Projection ¶
type Projection struct {
// contains filtered or unexported fields
}
func NewProjection ¶
func NewProjection(sel Selection, dst QuerySink) (*Projection, error)
NewProjection implements simple column projection from one set of values to a subset of those values, possibly re-named.
func (*Projection) Close ¶
func (p *Projection) Close() error
func (*Projection) Open ¶
func (p *Projection) Open() (io.WriteCloser, error)
type QueryBuffer ¶
type QueryBuffer struct {
// contains filtered or unexported fields
}
QueryBuffer is an in-memory implementation of QuerySink that can be trivially converted to a Table. It can be used to force a sub-query to be fully materialized before being consumed by another query. It also guarantees that the input chunks are padded to a fixed alignment.
func (*QueryBuffer) Alignment ¶
func (q *QueryBuffer) Alignment() int
Alignment returns the alignment of the table.
func (*QueryBuffer) Bytes ¶
func (q *QueryBuffer) Bytes() []byte
Bytes returns all of the bytes written to the buffer.
func (*QueryBuffer) Open ¶
func (q *QueryBuffer) Open() (io.WriteCloser, error)
Open implements QueryConsumer.Open
func (*QueryBuffer) Reset ¶
func (q *QueryBuffer) Reset()
Reset resets the buffer so that it contains no data.
func (*QueryBuffer) SetAlignment ¶
func (q *QueryBuffer) SetAlignment(align int)
SetAlignment sets the alignment to which subsequent calls to Write will be padded.
func (*QueryBuffer) Size ¶
func (q *QueryBuffer) Size() int64
Size returns the number of bytes in the table.
func (*QueryBuffer) Table ¶
func (q *QueryBuffer) Table() *BufferedTable
Table produces a view of the data in the QueryBuffer
type QuerySink ¶
type QuerySink interface { // Open opens a new stream for output. // Each stream is only safe to use from // a single goroutine. Multiple streams // may be opened for concurrent output. Open() (io.WriteCloser, error) io.Closer }
QuerySink represents a sink for query outputs. Every query writes into a QuerySink.
func LockedSink ¶
LockedSink returns a QuerySink for which all calls to Open return a wrapper of dst that serializes calls to io.Writer.Write. (See also Locked.)
func NewSystemDatashape ¶
NewSystemDatashape constucts a QuerySink implementing the `SYSTEM_DATASHAPE(*)` aggregation
func NewSystemDatashapeMerge ¶
NewSystemDatashapeMerge constructs a QuerySink that merges the results of multiple `SYSTEM_DATASHAPE(*)` aggregations
type ReaderAtTable ¶
type ReaderAtTable struct {
// contains filtered or unexported fields
}
ReaderAtTable is a Table implementation that wraps an io.ReaderAt.
func NewReaderAtTable ¶
func NewReaderAtTable(src io.ReaderAt, size int64, align int) *ReaderAtTable
NewReaderAtTable table constructs a ReaderAtTable that reads from the provided ReaderAt at the specified alignment and up to size bytes.
func (*ReaderAtTable) Align ¶
func (r *ReaderAtTable) Align() int
Align returns the configured alignment for chunks in the table
func (*ReaderAtTable) Bytes ¶
func (r *ReaderAtTable) Bytes() int64
func (*ReaderAtTable) Hits ¶
func (r *ReaderAtTable) Hits() int64
func (*ReaderAtTable) Misses ¶
func (r *ReaderAtTable) Misses() int64
func (*ReaderAtTable) Size ¶
func (r *ReaderAtTable) Size() int64
Size returns the number of bytes in the table
func (*ReaderAtTable) WriteChunks ¶
func (r *ReaderAtTable) WriteChunks(dst QuerySink, parallel int) error
WriteChunks implements Table.WriteChunks
type Rematerializer ¶
type Rematerializer struct {
// contains filtered or unexported fields
}
Rematerializer is a RowConsumer that rematerializes row data into contiguous blocks of ion data.
func Rematerialize ¶
func Rematerialize(dst io.WriteCloser) *Rematerializer
Rematerialize returns a RowConsumer that guarantees that the row data is fully rematerialized before being written to 'dst'
type Selection ¶
Selection represents a set of columns with optional re-naming
i.e. 'x, y, z' or 'x AS a, y AS b, z AS c'
type SortColumn ¶
type SortColumn struct { Node expr.Node Ordering SortOrdering }
SortColumn represents a single entry in the 'ORDER BY' clause: "column-name [ASC|DESC] [NULLS FIRST|NULLS LAST]"
type SortDirection ¶
type SortDirection int
SortDirection selectes ordering of non-null values: ascending or descending.
const ( SortAscending SortDirection = +1 SortDescending SortDirection = -1 )
func (SortDirection) String ¶
func (v SortDirection) String() string
type SortNullsOrder ¶
type SortNullsOrder int
SortNullsOrder selects ordering of null values.
const ( SortNullsFirst SortNullsOrder = 0 SortNullsLast SortNullsOrder = 1 )
func (SortNullsOrder) String ¶
func (v SortNullsOrder) String() string
type SortOrdering ¶
type SortOrdering struct { Direction SortDirection NullsOrder SortNullsOrder }
SortOrdering select ordering of null and non-null values
func (SortOrdering) Compare ¶
func (o SortOrdering) Compare(a, b []byte) int
Compare compares two Ion values according to ordering settings. Similarly to bytes.Compare, Compare returns -1 if a < b, 0 if a == b, or 1 if a > b
func (SortOrdering) String ¶
func (o SortOrdering) String() string
type Table ¶
type Table interface { // WriteChunks should write the table // contents into dst using the provided // parallelism hint. // // Each output stream should be created // with dst.Open(), followed by zero or // more calls to io.WriteCloser.Write, followed // by exactly one call to io.WriteCloser.Close. // See QuerySink.Open. Each call to io.WriteCloser.Write // must be at a "chunk boundary" -- the provided // data must begin with an ion BVM plus an ion symbol table // and be followed by zero or more ion structures. // // Typically callers will implement // WriteChunks in terms of SplitInput. WriteChunks(dst QuerySink, parallel int) error }
Table represents an ion-encoded collection of rows
type TeeWriter ¶
type TeeWriter struct {
// contains filtered or unexported fields
}
TeeWriter is an io.Writer that writes to multiple streams simultaneously, taking care to handle the errors from each stream separately.
func NewTeeWriter ¶
NewTeeWriter constructs a new TeeWriter with an io.Writer and an error handler. The returned TeeWriter does not return errors on calls to Write unless all of its constituent io.Writers have returned with errors, at which point it will return io.EOF.
func (*TeeWriter) Add ¶
Add adds a writer to the TeeWriter. Calls to t.Write will be forwarded to w for as long as it does not return an error. On the first encountered error, final(err) will be called and the writer will be disabled. If no errors are encountered, then final(nil) will be called at the point that t.Close (or t.CloseError) is called.
The final function provided to Add should not block; it is called synchronously with respect to calls to Write.
func (*TeeWriter) Close ¶
Close calls final(nil) for each of the remaining writers added via Add and then resets the content of t.
func (*TeeWriter) CloseError ¶
CloseError calls the final function for all the remaining writers with the provided error value, then resets the content of t.
func (*TeeWriter) ConfigureZion ¶
func (*TeeWriter) EndSegment ¶
func (t *TeeWriter) EndSegment()
EndSegment implements EndSegmentWriter.EndSegment. This calls HintEndSegment on all the remaining writers.
type TraceFlags ¶
type TraceFlags uint
TraceFlags is set of tracing options that can be passed to Trace.
type Unnest ¶
type Unnest struct {
// contains filtered or unexported fields
}
Unnest un-nests an array and produces rows that have their contents cross-joined with the array contents as an auxiliary binding
type Unpivot ¶
type Unpivot struct {
// contains filtered or unexported fields
}
func NewUnpivot ¶
NewUnpivot creates a new Unpivot kernel that unpivots a tuple into a set of pairs, per PartiQL.pdf, $5.2
type UnpivotAtDistinct ¶
type UnpivotAtDistinct struct {
// contains filtered or unexported fields
}
func NewUnpivotAtDistinct ¶
func NewUnpivotAtDistinct(at string, dst QuerySink) (*UnpivotAtDistinct, error)
NewUnpivotAtDistinct creates a new UnpivotAtDistinct kernel that returns the list of pairs describing the encountered columns
func (*UnpivotAtDistinct) Close ¶
func (u *UnpivotAtDistinct) Close() error
func (*UnpivotAtDistinct) Open ¶
func (u *UnpivotAtDistinct) Open() (io.WriteCloser, error)
Source Files
¶
- aggcountdistinct.go
- aggregate.go
- aggregate_sumf.go
- aggtdigest.go
- aligned-writer.go
- assembler.go
- avx512level.go
- bclayout.go
- bcsymbol.go
- bufferops.go
- bytecode.go
- bytecode_gen.go
- compare_tuple.go
- constraints.go
- distinct.go
- doc.go
- exprcompile.go
- filter.go
- hash_aggregate.go
- hash_aggregate_window.go
- interp.go
- interp_dfa.go
- interp_string.go
- interpagg.go
- interparrayvec.go
- interpbox.go
- interpcmp.go
- interpconvert.go
- interpdatetime.go
- interpfloat.go
- interpgeneral.go
- interphash.go
- interpi64.go
- interpmask.go
- interptrans.go
- interpunbox.go
- interpvalue.go
- ion.go
- leak.go
- limit.go
- log.go
- malloc.go
- malloc_linux.go
- mathconstants.go
- radix64.go
- reference_impl.go
- scan.go
- selector.go
- sfw.go
- simplify.go
- simplify1.go
- siphash_amd64.go
- slab.go
- sleefdp.go
- sleefrempitab.go
- sort.go
- ssa.go
- ssadefs.go
- ssastack.go
- symbols.go
- symtab.go
- sysdatashape.go
- table.go
- tee.go
- trace.go
- unnest.go
- unpivot.go
- unpivot_accelerators_amd64.go
- vmfence_none.go
- vmmemleaks_none.go
- zion.go
- zionflatten_amd64.go
Directories
¶
Path | Synopsis |
---|---|
Alternative to-{upper,lower} approach --------------------------------------------------
|
Alternative to-{upper,lower} approach -------------------------------------------------- |