proto

package
v0.61.5 Latest Latest
Warning

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

Go to latest
Published: Mar 8, 2024 License: Apache-2.0 Imports: 17 Imported by: 56

Documentation

Overview

Package proto implements ClickHouse wire protocol.

Index

Examples

Constants

View Source
const (
	Version = 54460
	Name    = "clickhouse/ch-go"
)

Defaults for ClientHello.

View Source
const DateLayout = "2006-01-02"

DateLayout is default time format for Date.

Variables

This section is empty.

Functions

func CardinalityKeyStrings

func CardinalityKeyStrings() []string

CardinalityKeyStrings returns a slice of all String values of the enum

func ClientCodeStrings

func ClientCodeStrings() []string

ClientCodeStrings returns a slice of all String values of the enum

func ClientQueryKindStrings

func ClientQueryKindStrings() []string

ClientQueryKindStrings returns a slice of all String values of the enum

func CompressionStrings

func CompressionStrings() []string

CompressionStrings returns a slice of all String values of the enum

func ErrorStrings

func ErrorStrings() []string

ErrorStrings returns a slice of all String values of the enum

func FeatureStrings

func FeatureStrings() []string

FeatureStrings returns a slice of all String values of the enum

func InterfaceStrings

func InterfaceStrings() []string

InterfaceStrings returns a slice of all String values of the enum

func IntervalScaleStrings added in v0.45.0

func IntervalScaleStrings() []string

IntervalScaleStrings returns a slice of all String values of the enum

func ProfileEventTypeStrings

func ProfileEventTypeStrings() []string

ProfileEventTypeStrings returns a slice of all String values of the enum

func Reset added in v0.47.2

func Reset(columns ...Resettable)

Reset is helper to reset columns.

func ServerCodeStrings

func ServerCodeStrings() []string

ServerCodeStrings returns a slice of all String values of the enum

func StageStrings

func StageStrings() []string

StageStrings returns a slice of all String values of the enum

Types

type Arrayable added in v0.41.0

type Arrayable[T any] interface {
	Array() *ColArr[T]
}

Arrayable constraint specifies ability of column T to be Array(T).

type AwareDecoder

type AwareDecoder interface {
	DecodeAware(r *Reader, version int) error
}

AwareDecoder implements encoding to Buffer that depends on version.

type AwareEncoder

type AwareEncoder interface {
	EncodeAware(b *Buffer, version int)
}

AwareEncoder implements encoding to Buffer that depends on version.

type Block

type Block struct {
	Info    BlockInfo
	Columns int
	Rows    int
}

func (*Block) DecodeBlock

func (b *Block) DecodeBlock(r *Reader, version int, target Result) error

func (*Block) DecodeRawBlock

func (b *Block) DecodeRawBlock(r *Reader, version int, target Result) error

func (Block) EncodeAware

func (b Block) EncodeAware(buf *Buffer, version int)

func (Block) EncodeBlock

func (b Block) EncodeBlock(buf *Buffer, version int, input []InputColumn) error
Example
package main

import (
	"bytes"
	"fmt"

	"github.com/ClickHouse/ch-go/proto"
)

func main() {
	// See ./internal/cmd/ch-native-dump for more sophisticated example.
	var (
		colK proto.ColInt64
		colV proto.ColInt64
	)
	// Generate some data.
	for i := 0; i < 100; i++ {
		colK.Append(int64(i))
		colV.Append(int64(i) + 1000)
	}
	// Write data to buffer.
	var buf proto.Buffer
	input := proto.Input{
		{"k", colK},
		{"v", colV},
	}
	b := proto.Block{
		Rows:    colK.Rows(),
		Columns: len(input),
	}
	// Note that we are using version 54451, proto.Version will fail.
	if err := b.EncodeRawBlock(&buf, 54451, input); err != nil {
		panic(err)
	}

	// You can write buf.Buf to io.Writer, e.g. os.Stdout or file.
	var out bytes.Buffer
	_, _ = out.Write(buf.Buf)

	// You can encode multiple buffers in sequence.
	//
	// To do this, reset buf and all columns, append new values
	// to columns and call EncodeRawBlock again.
	buf.Reset()
	colV.Reset()
	colV.Reset()

	fmt.Println(out.Len())
}
Output:

1618

func (Block) EncodeRawBlock

func (b Block) EncodeRawBlock(buf *Buffer, version int, input []InputColumn) error

func (*Block) End

func (b *Block) End() bool

type BlockInfo

type BlockInfo struct {
	Overflows bool
	BucketNum int
}

BlockInfo describes block.

func (*BlockInfo) Decode

func (i *BlockInfo) Decode(r *Reader) error

func (BlockInfo) Encode

func (i BlockInfo) Encode(b *Buffer)

Encode to Buffer.

func (BlockInfo) String

func (i BlockInfo) String() string

type Buffer

type Buffer struct {
	Buf []byte
}

Buffer implements ClickHouse binary protocol encoding.

func (*Buffer) Encode

func (b *Buffer) Encode(e Encoder)

Encode value that implements Encoder.

func (*Buffer) EncodeAware

func (b *Buffer) EncodeAware(e AwareEncoder, version int)

EncodeAware value that implements AwareEncoder.

func (*Buffer) Ensure

func (b *Buffer) Ensure(n int)

Ensure Buf length.

func (*Buffer) PutBool

func (b *Buffer) PutBool(v bool)

func (*Buffer) PutByte

func (b *Buffer) PutByte(x byte)

PutByte encodes byte as uint8.

func (*Buffer) PutFloat32

func (b *Buffer) PutFloat32(v float32)

func (*Buffer) PutFloat64

func (b *Buffer) PutFloat64(v float64)

func (*Buffer) PutInt

func (b *Buffer) PutInt(x int)

PutInt encodes integer as uvarint.

func (*Buffer) PutInt128

func (b *Buffer) PutInt128(x Int128)

func (*Buffer) PutInt16

func (b *Buffer) PutInt16(v int16)

func (*Buffer) PutInt32

func (b *Buffer) PutInt32(x int32)

func (*Buffer) PutInt64

func (b *Buffer) PutInt64(x int64)

func (*Buffer) PutInt8

func (b *Buffer) PutInt8(v int8)

func (*Buffer) PutLen

func (b *Buffer) PutLen(x int)

PutLen encodes length to buffer as uvarint.

func (*Buffer) PutRaw

func (b *Buffer) PutRaw(v []byte)

PutRaw writes v as raw bytes to buffer.

func (*Buffer) PutString

func (b *Buffer) PutString(s string)

PutString encodes sting value to buffer.

func (*Buffer) PutUInt128

func (b *Buffer) PutUInt128(x UInt128)

func (*Buffer) PutUInt16

func (b *Buffer) PutUInt16(x uint16)

func (*Buffer) PutUInt32

func (b *Buffer) PutUInt32(x uint32)

func (*Buffer) PutUInt64

func (b *Buffer) PutUInt64(x uint64)

func (*Buffer) PutUInt8

func (b *Buffer) PutUInt8(x uint8)

func (*Buffer) PutUVarInt

func (b *Buffer) PutUVarInt(x uint64)

PutUVarInt encodes x as uvarint.

func (*Buffer) Read

func (b *Buffer) Read(p []byte) (n int, err error)

Read implements io.Reader.

func (*Buffer) Reader

func (b *Buffer) Reader() *Reader

Reader returns new *Reader from *Buffer.

func (*Buffer) Reset

func (b *Buffer) Reset()

Reset buffer to zero length.

type CardinalityKey

type CardinalityKey byte

CardinalityKey is integer type of ColLowCardinality.Keys column.

const (
	KeyUInt8  CardinalityKey = 0
	KeyUInt16 CardinalityKey = 1
	KeyUInt32 CardinalityKey = 2
	KeyUInt64 CardinalityKey = 3
)

Possible integer types for ColLowCardinality.Keys.

func CardinalityKeyString

func CardinalityKeyString(s string) (CardinalityKey, error)

CardinalityKeyString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func CardinalityKeyValues

func CardinalityKeyValues() []CardinalityKey

CardinalityKeyValues returns all values of the enum

func (CardinalityKey) IsACardinalityKey

func (i CardinalityKey) IsACardinalityKey() bool

IsACardinalityKey returns "true" if the value is listed in the enum definition. "false" otherwise

func (CardinalityKey) String

func (i CardinalityKey) String() string

type ClientCode

type ClientCode byte

ClientCode is sent from client to server.

const (
	ClientCodeHello           ClientCode = 0 // client part of "handshake"
	ClientCodeQuery           ClientCode = 1 // query start
	ClientCodeData            ClientCode = 2 // data block (can be compressed)
	ClientCodeCancel          ClientCode = 3 // query cancel
	ClientCodePing            ClientCode = 4 // ping request to server
	ClientTablesStatusRequest ClientCode = 5 // tables status request
)

Possible client codes.

func ClientCodeString

func ClientCodeString(s string) (ClientCode, error)

ClientCodeString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func ClientCodeValues

func ClientCodeValues() []ClientCode

ClientCodeValues returns all values of the enum

func (ClientCode) Encode

func (c ClientCode) Encode(b *Buffer)

Encode to buffer.

func (ClientCode) IsAClientCode

func (i ClientCode) IsAClientCode() bool

IsAClientCode returns "true" if the value is listed in the enum definition. "false" otherwise

func (ClientCode) String

func (i ClientCode) String() string

type ClientData

type ClientData struct {
	TableName string
}

func (*ClientData) DecodeAware

func (c *ClientData) DecodeAware(r *Reader, version int) error

func (ClientData) EncodeAware

func (c ClientData) EncodeAware(b *Buffer, version int)

type ClientHello

type ClientHello struct {
	Name string

	Major int // client major version
	Minor int // client minor version

	// ProtocolVersion is TCP protocol version of client.
	//
	// Usually it is equal to the latest compatible server revision, but
	// should not be confused with it.
	ProtocolVersion int

	Database string
	User     string
	Password string
}

ClientHello represents ClientCodeHello message.

func (*ClientHello) Decode

func (c *ClientHello) Decode(r *Reader) error

func (ClientHello) Encode

func (c ClientHello) Encode(b *Buffer)

Encode to Buffer.

type ClientInfo

type ClientInfo struct {
	ProtocolVersion int

	Major int
	Minor int
	Patch int

	Interface Interface
	Query     ClientQueryKind

	InitialUser    string
	InitialQueryID string
	InitialAddress string
	InitialTime    int64

	OSUser         string
	ClientHostname string
	ClientName     string

	Span trace.SpanContext

	QuotaKey         string
	DistributedDepth int

	CollaborateWithInitiator   bool
	CountParticipatingReplicas int
	NumberOfCurrentReplica     int
}

ClientInfo message.

func (*ClientInfo) DecodeAware

func (c *ClientInfo) DecodeAware(r *Reader, version int) error

func (ClientInfo) EncodeAware

func (c ClientInfo) EncodeAware(b *Buffer, version int)

EncodeAware encodes to buffer version-aware.

type ClientQueryKind

type ClientQueryKind byte

ClientQueryKind is kind of query.

const (
	ClientQueryNone      ClientQueryKind = 0
	ClientQueryInitial   ClientQueryKind = 1
	ClientQuerySecondary ClientQueryKind = 2
)

Possible query kinds.

func ClientQueryKindString

func ClientQueryKindString(s string) (ClientQueryKind, error)

ClientQueryKindString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func ClientQueryKindValues

func ClientQueryKindValues() []ClientQueryKind

ClientQueryKindValues returns all values of the enum

func (ClientQueryKind) IsAClientQueryKind

func (i ClientQueryKind) IsAClientQueryKind() bool

IsAClientQueryKind returns "true" if the value is listed in the enum definition. "false" otherwise

func (ClientQueryKind) String

func (i ClientQueryKind) String() string

type ColArr

type ColArr[T any] struct {
	Offsets ColUInt64
	Data    ColumnOf[T]
}

ColArr is Array(T).

func NewArrDate

func NewArrDate() *ColArr[time.Time]

NewArrDate returns new Array(Date).

func NewArrDate32

func NewArrDate32() *ColArr[time.Time]

NewArrDate32 returns new Array(Date32).

func NewArrDateTime

func NewArrDateTime() *ColArr[time.Time]

NewArrDateTime returns new Array(DateTime).

func NewArrDecimal128

func NewArrDecimal128() *ColArr[Decimal128]

NewArrDecimal128 returns new Array(Decimal128).

func NewArrDecimal256

func NewArrDecimal256() *ColArr[Decimal256]

NewArrDecimal256 returns new Array(Decimal256).

func NewArrDecimal32

func NewArrDecimal32() *ColArr[Decimal32]

NewArrDecimal32 returns new Array(Decimal32).

func NewArrDecimal64

func NewArrDecimal64() *ColArr[Decimal64]

NewArrDecimal64 returns new Array(Decimal64).

func NewArrEnum16

func NewArrEnum16() *ColArr[Enum16]

NewArrEnum16 returns new Array(Enum16).

func NewArrEnum8

func NewArrEnum8() *ColArr[Enum8]

NewArrEnum8 returns new Array(Enum8).

func NewArrFixedStr128 added in v0.57.0

func NewArrFixedStr128() *ColArr[[128]byte]

NewArrFixedStr128 returns new Array(FixedStr128).

func NewArrFixedStr16 added in v0.57.0

func NewArrFixedStr16() *ColArr[[16]byte]

NewArrFixedStr16 returns new Array(FixedStr16).

func NewArrFixedStr256 added in v0.57.0

func NewArrFixedStr256() *ColArr[[256]byte]

NewArrFixedStr256 returns new Array(FixedStr256).

func NewArrFixedStr32 added in v0.57.0

func NewArrFixedStr32() *ColArr[[32]byte]

NewArrFixedStr32 returns new Array(FixedStr32).

func NewArrFixedStr512 added in v0.57.0

func NewArrFixedStr512() *ColArr[[512]byte]

NewArrFixedStr512 returns new Array(FixedStr512).

func NewArrFixedStr64 added in v0.57.0

func NewArrFixedStr64() *ColArr[[64]byte]

NewArrFixedStr64 returns new Array(FixedStr64).

func NewArrFixedStr8 added in v0.57.0

func NewArrFixedStr8() *ColArr[[8]byte]

NewArrFixedStr8 returns new Array(FixedStr8).

func NewArrFloat32

func NewArrFloat32() *ColArr[float32]

NewArrFloat32 returns new Array(Float32).

func NewArrFloat64

func NewArrFloat64() *ColArr[float64]

NewArrFloat64 returns new Array(Float64).

func NewArrIPv4

func NewArrIPv4() *ColArr[IPv4]

NewArrIPv4 returns new Array(IPv4).

func NewArrIPv6

func NewArrIPv6() *ColArr[IPv6]

NewArrIPv6 returns new Array(IPv6).

func NewArrInt128

func NewArrInt128() *ColArr[Int128]

NewArrInt128 returns new Array(Int128).

func NewArrInt16

func NewArrInt16() *ColArr[int16]

NewArrInt16 returns new Array(Int16).

func NewArrInt256

func NewArrInt256() *ColArr[Int256]

NewArrInt256 returns new Array(Int256).

func NewArrInt32

func NewArrInt32() *ColArr[int32]

NewArrInt32 returns new Array(Int32).

func NewArrInt64

func NewArrInt64() *ColArr[int64]

NewArrInt64 returns new Array(Int64).

func NewArrInt8

func NewArrInt8() *ColArr[int8]

NewArrInt8 returns new Array(Int8).

func NewArrUInt128

func NewArrUInt128() *ColArr[UInt128]

NewArrUInt128 returns new Array(UInt128).

func NewArrUInt16

func NewArrUInt16() *ColArr[uint16]

NewArrUInt16 returns new Array(UInt16).

func NewArrUInt256

func NewArrUInt256() *ColArr[UInt256]

NewArrUInt256 returns new Array(UInt256).

func NewArrUInt32

func NewArrUInt32() *ColArr[uint32]

NewArrUInt32 returns new Array(UInt32).

func NewArrUInt64

func NewArrUInt64() *ColArr[uint64]

NewArrUInt64 returns new Array(UInt64).

func NewArrUInt8

func NewArrUInt8() *ColArr[uint8]

NewArrUInt8 returns new Array(UInt8).

func NewArray added in v0.41.0

func NewArray[T any](c ColumnOf[T]) *ColArr[T]

NewArray returns ColArr of c.

Example: NewArray[string](new(ColStr))

func (*ColArr[T]) Append added in v0.41.0

func (c *ColArr[T]) Append(v []T)

Append appends new row to column.

func (*ColArr[T]) AppendArr added in v0.55.0

func (c *ColArr[T]) AppendArr(vs [][]T)

AppendArr appends new slice of rows to column.

func (*ColArr[T]) DecodeColumn

func (c *ColArr[T]) DecodeColumn(r *Reader, rows int) error

DecodeColumn implements ColResult.

func (*ColArr[T]) DecodeState

func (c *ColArr[T]) DecodeState(r *Reader) error

func (ColArr[T]) EncodeColumn

func (c ColArr[T]) EncodeColumn(b *Buffer)

EncodeColumn implements ColInput.

func (*ColArr[T]) EncodeState

func (c *ColArr[T]) EncodeState(b *Buffer)

func (*ColArr[T]) Infer added in v0.41.0

func (c *ColArr[T]) Infer(t ColumnType) error

Infer ensures Inferable column propagation.

func (*ColArr[T]) Prepare added in v0.41.0

func (c *ColArr[T]) Prepare() error

Prepare ensures Preparable column propagation.

func (*ColArr[T]) Reset

func (c *ColArr[T]) Reset()

Reset implements ColResult.

func (*ColArr[T]) Result added in v0.41.0

func (c *ColArr[T]) Result(column string) ResultColumn

Result for current column.

func (*ColArr[T]) Results added in v0.41.0

func (c *ColArr[T]) Results(column string) Results

Results return Results containing single column.

func (ColArr[T]) Row added in v0.41.0

func (c ColArr[T]) Row(i int) []T

Row returns i-th row.

func (ColArr[T]) RowAppend added in v0.41.0

func (c ColArr[T]) RowAppend(i int, target []T) []T

RowAppend appends i-th row to target and returns it.

func (ColArr[T]) Rows

func (c ColArr[T]) Rows() int

Rows returns rows count.

func (ColArr[T]) Type

func (c ColArr[T]) Type() ColumnType

Type returns type of array, i.e. Array(T).

type ColAuto

type ColAuto struct {
	Data     Column
	DataType ColumnType
}

ColAuto is column that is initialized during decoding.

func (ColAuto) DecodeColumn

func (c ColAuto) DecodeColumn(r *Reader, rows int) error

func (ColAuto) EncodeColumn

func (c ColAuto) EncodeColumn(b *Buffer)

func (*ColAuto) Infer

func (c *ColAuto) Infer(t ColumnType) error

Infer and initialize Column from ColumnType.

func (ColAuto) Reset

func (c ColAuto) Reset()

func (ColAuto) Rows

func (c ColAuto) Rows() int

func (ColAuto) Type

func (c ColAuto) Type() ColumnType

type ColBool

type ColBool []bool

ColBool is Bool column.

func (*ColBool) Append added in v0.41.0

func (c *ColBool) Append(v bool)

func (*ColBool) AppendArr added in v0.55.0

func (c *ColBool) AppendArr(vs []bool)

func (*ColBool) Array added in v0.41.0

func (c *ColBool) Array() *ColArr[bool]

Array is helper that creates Array(Bool).

func (*ColBool) DecodeColumn

func (c *ColBool) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Bool rows from *Reader.

func (ColBool) EncodeColumn

func (c ColBool) EncodeColumn(b *Buffer)

EncodeColumn encodes Bool rows to *Buffer.

func (*ColBool) Nullable added in v0.41.0

func (c *ColBool) Nullable() *ColNullable[bool]

Nullable is helper that creates Nullable(Bool).

func (*ColBool) Reset

func (c *ColBool) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColBool) Row added in v0.41.0

func (c ColBool) Row(i int) bool

func (ColBool) Rows

func (c ColBool) Rows() int

Rows returns count of rows in column.

func (ColBool) Type

func (ColBool) Type() ColumnType

Type returns ColumnType of Bool.

type ColBytes

type ColBytes struct {
	ColStr
}

ColBytes is ColStr wrapper to be ColumnOf for []byte.

func (*ColBytes) Append

func (c *ColBytes) Append(v []byte)

Append byte slice to column.

func (*ColBytes) AppendArr

func (c *ColBytes) AppendArr(v [][]byte)

AppendArr append slice of byte slices to column.

func (*ColBytes) Array

func (c *ColBytes) Array() *ColArr[[]byte]

Array is helper that creates Array(String).

func (*ColBytes) Nullable added in v0.41.0

func (c *ColBytes) Nullable() *ColNullable[[]byte]

Nullable is helper that creates Nullable(String).

func (ColBytes) Row

func (c ColBytes) Row(i int) []byte

Row returns row with number i.

type ColDate

type ColDate []Date

ColDate represents Date column.

func (*ColDate) Append

func (c *ColDate) Append(v time.Time)

func (*ColDate) AppendArr

func (c *ColDate) AppendArr(vs []time.Time)

func (*ColDate) Array

func (c *ColDate) Array() *ColArr[time.Time]

Array is helper that creates Array of Enum8.

func (*ColDate) DecodeColumn

func (c *ColDate) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Date rows from *Reader.

func (ColDate) EncodeColumn

func (c ColDate) EncodeColumn(b *Buffer)

EncodeColumn encodes Date rows to *Buffer.

func (*ColDate) LowCardinality

func (c *ColDate) LowCardinality() *ColLowCardinality[time.Time]

LowCardinality returns LowCardinality for Enum8 .

func (*ColDate) Nullable added in v0.41.0

func (c *ColDate) Nullable() *ColNullable[time.Time]

Nullable is helper that creates Nullable(Enum8).

func (*ColDate) Reset

func (c *ColDate) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDate) Row

func (c ColDate) Row(i int) time.Time

func (ColDate) Rows

func (c ColDate) Rows() int

Rows returns count of rows in column.

func (ColDate) Type

func (ColDate) Type() ColumnType

Type returns ColumnType of Date.

type ColDate32

type ColDate32 []Date32

ColDate32 represents Date32 column.

func (*ColDate32) Append

func (c *ColDate32) Append(v time.Time)

func (*ColDate32) AppendArr

func (c *ColDate32) AppendArr(vs []time.Time)

func (*ColDate32) Array

func (c *ColDate32) Array() *ColArr[time.Time]

Array is helper that creates Array of Enum8.

func (*ColDate32) DecodeColumn

func (c *ColDate32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Date32 rows from *Reader.

func (ColDate32) EncodeColumn

func (c ColDate32) EncodeColumn(b *Buffer)

EncodeColumn encodes Date32 rows to *Buffer.

func (*ColDate32) LowCardinality

func (c *ColDate32) LowCardinality() *ColLowCardinality[time.Time]

LowCardinality returns LowCardinality for Enum8 .

func (*ColDate32) Nullable added in v0.41.0

func (c *ColDate32) Nullable() *ColNullable[time.Time]

Nullable is helper that creates Nullable(Enum8).

func (*ColDate32) Reset

func (c *ColDate32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDate32) Row

func (c ColDate32) Row(i int) time.Time

func (ColDate32) Rows

func (c ColDate32) Rows() int

Rows returns count of rows in column.

func (ColDate32) Type

func (ColDate32) Type() ColumnType

Type returns ColumnType of Date32.

type ColDateTime

type ColDateTime struct {
	Data     []DateTime
	Location *time.Location
}

ColDateTime implements ColumnOf[time.Time].

func (*ColDateTime) Append

func (c *ColDateTime) Append(v time.Time)

func (*ColDateTime) AppendArr

func (c *ColDateTime) AppendArr(vs []time.Time)

func (*ColDateTime) Array

func (c *ColDateTime) Array() *ColArr[time.Time]

Array is helper that creates Array of Enum8.

func (*ColDateTime) DecodeColumn

func (c *ColDateTime) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes DateTime rows from *Reader.

func (ColDateTime) EncodeColumn

func (c ColDateTime) EncodeColumn(b *Buffer)

EncodeColumn encodes DateTime rows to *Buffer.

func (*ColDateTime) Infer added in v0.43.0

func (c *ColDateTime) Infer(t ColumnType) error

func (*ColDateTime) LowCardinality

func (c *ColDateTime) LowCardinality() *ColLowCardinality[time.Time]

LowCardinality returns LowCardinality for Enum8 .

func (*ColDateTime) Nullable added in v0.41.0

func (c *ColDateTime) Nullable() *ColNullable[time.Time]

Nullable is helper that creates Nullable(Enum8).

func (*ColDateTime) Reset

func (c *ColDateTime) Reset()

func (ColDateTime) Row

func (c ColDateTime) Row(i int) time.Time

func (ColDateTime) Rows

func (c ColDateTime) Rows() int

func (ColDateTime) Type

func (c ColDateTime) Type() ColumnType

type ColDateTime64

type ColDateTime64 struct {
	Data         []DateTime64
	Location     *time.Location
	Precision    Precision
	PrecisionSet bool
}

ColDateTime64 implements ColumnOf[time.Time].

If Precision is not set, Append and Row() panics. Use ColDateTime64Raw to work with raw DateTime64 values.

func (*ColDateTime64) Append

func (c *ColDateTime64) Append(v time.Time)

func (*ColDateTime64) AppendArr

func (c *ColDateTime64) AppendArr(v []time.Time)

func (*ColDateTime64) AppendRaw added in v0.44.0

func (c *ColDateTime64) AppendRaw(v DateTime64)

func (*ColDateTime64) Array

func (c *ColDateTime64) Array() *ColArr[time.Time]

func (*ColDateTime64) DecodeColumn

func (c *ColDateTime64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes DateTime64 rows from *Reader.

func (ColDateTime64) EncodeColumn

func (c ColDateTime64) EncodeColumn(b *Buffer)

EncodeColumn encodes DateTime64 rows to *Buffer.

func (*ColDateTime64) Infer added in v0.43.0

func (c *ColDateTime64) Infer(t ColumnType) error

func (ColDateTime64) Raw added in v0.44.0

Raw version of ColDateTime64 for ColumnOf[DateTime64].

func (*ColDateTime64) Reset

func (c *ColDateTime64) Reset()

func (ColDateTime64) Row

func (c ColDateTime64) Row(i int) time.Time

func (ColDateTime64) Rows

func (c ColDateTime64) Rows() int

func (ColDateTime64) Type

func (c ColDateTime64) Type() ColumnType

func (*ColDateTime64) WithLocation added in v0.43.0

func (c *ColDateTime64) WithLocation(loc *time.Location) *ColDateTime64

func (*ColDateTime64) WithPrecision added in v0.43.0

func (c *ColDateTime64) WithPrecision(p Precision) *ColDateTime64

type ColDateTime64Raw added in v0.44.0

type ColDateTime64Raw struct {
	ColDateTime64
}

ColDateTime64Raw is DateTime64 wrapper to implement ColumnOf[DateTime64].

func (*ColDateTime64Raw) Append added in v0.44.0

func (c *ColDateTime64Raw) Append(v DateTime64)

func (*ColDateTime64Raw) AppendArr added in v0.55.0

func (c *ColDateTime64Raw) AppendArr(vs []DateTime64)

func (ColDateTime64Raw) Row added in v0.44.0

func (c ColDateTime64Raw) Row(i int) DateTime64

type ColDecimal128

type ColDecimal128 []Decimal128

ColDecimal128 represents Decimal128 column.

func (*ColDecimal128) Append

func (c *ColDecimal128) Append(v Decimal128)

Append Decimal128 to column.

func (*ColDecimal128) AppendArr

func (c *ColDecimal128) AppendArr(vs []Decimal128)

Append Decimal128 slice to column.

func (*ColDecimal128) Array

func (c *ColDecimal128) Array() *ColArr[Decimal128]

Array is helper that creates Array of Decimal128.

func (*ColDecimal128) DecodeColumn

func (c *ColDecimal128) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Decimal128 rows from *Reader.

func (ColDecimal128) EncodeColumn

func (c ColDecimal128) EncodeColumn(b *Buffer)

EncodeColumn encodes Decimal128 rows to *Buffer.

func (*ColDecimal128) LowCardinality

func (c *ColDecimal128) LowCardinality() *ColLowCardinality[Decimal128]

LowCardinality returns LowCardinality for Decimal128 .

func (*ColDecimal128) Nullable added in v0.41.0

func (c *ColDecimal128) Nullable() *ColNullable[Decimal128]

Nullable is helper that creates Nullable(Decimal128).

func (*ColDecimal128) Reset

func (c *ColDecimal128) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDecimal128) Row

func (c ColDecimal128) Row(i int) Decimal128

Row returns i-th row of column.

func (ColDecimal128) Rows

func (c ColDecimal128) Rows() int

Rows returns count of rows in column.

func (ColDecimal128) Type

func (ColDecimal128) Type() ColumnType

Type returns ColumnType of Decimal128.

type ColDecimal256

type ColDecimal256 []Decimal256

ColDecimal256 represents Decimal256 column.

func (*ColDecimal256) Append

func (c *ColDecimal256) Append(v Decimal256)

Append Decimal256 to column.

func (*ColDecimal256) AppendArr

func (c *ColDecimal256) AppendArr(vs []Decimal256)

Append Decimal256 slice to column.

func (*ColDecimal256) Array

func (c *ColDecimal256) Array() *ColArr[Decimal256]

Array is helper that creates Array of Decimal256.

func (*ColDecimal256) DecodeColumn

func (c *ColDecimal256) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Decimal256 rows from *Reader.

func (ColDecimal256) EncodeColumn

func (c ColDecimal256) EncodeColumn(b *Buffer)

EncodeColumn encodes Decimal256 rows to *Buffer.

func (*ColDecimal256) LowCardinality

func (c *ColDecimal256) LowCardinality() *ColLowCardinality[Decimal256]

LowCardinality returns LowCardinality for Decimal256 .

func (*ColDecimal256) Nullable added in v0.41.0

func (c *ColDecimal256) Nullable() *ColNullable[Decimal256]

Nullable is helper that creates Nullable(Decimal256).

func (*ColDecimal256) Reset

func (c *ColDecimal256) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDecimal256) Row

func (c ColDecimal256) Row(i int) Decimal256

Row returns i-th row of column.

func (ColDecimal256) Rows

func (c ColDecimal256) Rows() int

Rows returns count of rows in column.

func (ColDecimal256) Type

func (ColDecimal256) Type() ColumnType

Type returns ColumnType of Decimal256.

type ColDecimal32

type ColDecimal32 []Decimal32

ColDecimal32 represents Decimal32 column.

func (*ColDecimal32) Append

func (c *ColDecimal32) Append(v Decimal32)

Append Decimal32 to column.

func (*ColDecimal32) AppendArr

func (c *ColDecimal32) AppendArr(vs []Decimal32)

Append Decimal32 slice to column.

func (*ColDecimal32) Array

func (c *ColDecimal32) Array() *ColArr[Decimal32]

Array is helper that creates Array of Decimal32.

func (*ColDecimal32) DecodeColumn

func (c *ColDecimal32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Decimal32 rows from *Reader.

func (ColDecimal32) EncodeColumn

func (c ColDecimal32) EncodeColumn(b *Buffer)

EncodeColumn encodes Decimal32 rows to *Buffer.

func (*ColDecimal32) LowCardinality

func (c *ColDecimal32) LowCardinality() *ColLowCardinality[Decimal32]

LowCardinality returns LowCardinality for Decimal32 .

func (*ColDecimal32) Nullable added in v0.41.0

func (c *ColDecimal32) Nullable() *ColNullable[Decimal32]

Nullable is helper that creates Nullable(Decimal32).

func (*ColDecimal32) Reset

func (c *ColDecimal32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDecimal32) Row

func (c ColDecimal32) Row(i int) Decimal32

Row returns i-th row of column.

func (ColDecimal32) Rows

func (c ColDecimal32) Rows() int

Rows returns count of rows in column.

func (ColDecimal32) Type

func (ColDecimal32) Type() ColumnType

Type returns ColumnType of Decimal32.

type ColDecimal64

type ColDecimal64 []Decimal64

ColDecimal64 represents Decimal64 column.

func (*ColDecimal64) Append

func (c *ColDecimal64) Append(v Decimal64)

Append Decimal64 to column.

func (*ColDecimal64) AppendArr

func (c *ColDecimal64) AppendArr(vs []Decimal64)

Append Decimal64 slice to column.

func (*ColDecimal64) Array

func (c *ColDecimal64) Array() *ColArr[Decimal64]

Array is helper that creates Array of Decimal64.

func (*ColDecimal64) DecodeColumn

func (c *ColDecimal64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Decimal64 rows from *Reader.

func (ColDecimal64) EncodeColumn

func (c ColDecimal64) EncodeColumn(b *Buffer)

EncodeColumn encodes Decimal64 rows to *Buffer.

func (*ColDecimal64) LowCardinality

func (c *ColDecimal64) LowCardinality() *ColLowCardinality[Decimal64]

LowCardinality returns LowCardinality for Decimal64 .

func (*ColDecimal64) Nullable added in v0.41.0

func (c *ColDecimal64) Nullable() *ColNullable[Decimal64]

Nullable is helper that creates Nullable(Decimal64).

func (*ColDecimal64) Reset

func (c *ColDecimal64) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColDecimal64) Row

func (c ColDecimal64) Row(i int) Decimal64

Row returns i-th row of column.

func (ColDecimal64) Rows

func (c ColDecimal64) Rows() int

Rows returns count of rows in column.

func (ColDecimal64) Type

func (ColDecimal64) Type() ColumnType

Type returns ColumnType of Decimal64.

type ColEnum added in v0.42.0

type ColEnum struct {

	// Values of ColEnum.
	Values []string
	// contains filtered or unexported fields
}

ColEnum is inference helper for enums.

You can set Values and actual enum mapping will be inferred during query execution.

func (*ColEnum) Append added in v0.42.0

func (e *ColEnum) Append(v string)

Append value to Enum8 column.

func (*ColEnum) AppendArr added in v0.55.0

func (e *ColEnum) AppendArr(vs []string)

func (*ColEnum) DecodeColumn added in v0.42.0

func (e *ColEnum) DecodeColumn(r *Reader, rows int) error

func (*ColEnum) EncodeColumn added in v0.42.0

func (e *ColEnum) EncodeColumn(b *Buffer)

func (*ColEnum) Infer added in v0.42.0

func (e *ColEnum) Infer(t ColumnType) error

func (*ColEnum) Prepare added in v0.42.0

func (e *ColEnum) Prepare() error

func (*ColEnum) Reset added in v0.42.0

func (e *ColEnum) Reset()

func (ColEnum) Row added in v0.42.0

func (e ColEnum) Row(i int) string

func (*ColEnum) Rows added in v0.42.0

func (e *ColEnum) Rows() int

func (*ColEnum) Type added in v0.42.0

func (e *ColEnum) Type() ColumnType

type ColEnum16

type ColEnum16 []Enum16

ColEnum16 represents Enum16 column.

func (*ColEnum16) Append

func (c *ColEnum16) Append(v Enum16)

Append Enum16 to column.

func (*ColEnum16) AppendArr

func (c *ColEnum16) AppendArr(vs []Enum16)

Append Enum16 slice to column.

func (*ColEnum16) Array

func (c *ColEnum16) Array() *ColArr[Enum16]

Array is helper that creates Array of Enum16.

func (*ColEnum16) DecodeColumn

func (c *ColEnum16) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Enum16 rows from *Reader.

func (ColEnum16) EncodeColumn

func (c ColEnum16) EncodeColumn(b *Buffer)

EncodeColumn encodes Enum16 rows to *Buffer.

func (*ColEnum16) LowCardinality

func (c *ColEnum16) LowCardinality() *ColLowCardinality[Enum16]

LowCardinality returns LowCardinality for Enum16 .

func (*ColEnum16) Nullable added in v0.41.0

func (c *ColEnum16) Nullable() *ColNullable[Enum16]

Nullable is helper that creates Nullable(Enum16).

func (*ColEnum16) Reset

func (c *ColEnum16) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColEnum16) Row

func (c ColEnum16) Row(i int) Enum16

Row returns i-th row of column.

func (ColEnum16) Rows

func (c ColEnum16) Rows() int

Rows returns count of rows in column.

func (ColEnum16) Type

func (ColEnum16) Type() ColumnType

Type returns ColumnType of Enum16.

type ColEnum8

type ColEnum8 []Enum8

ColEnum8 represents Enum8 column.

func (*ColEnum8) Append

func (c *ColEnum8) Append(v Enum8)

Append Enum8 to column.

func (*ColEnum8) AppendArr

func (c *ColEnum8) AppendArr(vs []Enum8)

Append Enum8 slice to column.

func (*ColEnum8) Array

func (c *ColEnum8) Array() *ColArr[Enum8]

Array is helper that creates Array of Enum8.

func (*ColEnum8) DecodeColumn

func (c *ColEnum8) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Enum8 rows from *Reader.

func (ColEnum8) EncodeColumn

func (c ColEnum8) EncodeColumn(b *Buffer)

EncodeColumn encodes Enum8 rows to *Buffer.

func (*ColEnum8) LowCardinality

func (c *ColEnum8) LowCardinality() *ColLowCardinality[Enum8]

LowCardinality returns LowCardinality for Enum8 .

func (*ColEnum8) Nullable added in v0.41.0

func (c *ColEnum8) Nullable() *ColNullable[Enum8]

Nullable is helper that creates Nullable(Enum8).

func (*ColEnum8) Reset

func (c *ColEnum8) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColEnum8) Row

func (c ColEnum8) Row(i int) Enum8

Row returns i-th row of column.

func (ColEnum8) Rows

func (c ColEnum8) Rows() int

Rows returns count of rows in column.

func (ColEnum8) Type

func (ColEnum8) Type() ColumnType

Type returns ColumnType of Enum8.

type ColFixedStr

type ColFixedStr struct {
	Buf  []byte
	Size int // N
}

ColFixedStr represents FixedString(Size) column. Size is required.

Can be used to store SHA256, MD5 or similar fixed size binary values. See https://clickhouse.com/docs/en/sql-reference/data-types/fixedstring/.

func (*ColFixedStr) Append

func (c *ColFixedStr) Append(b []byte)

Append value to column. Panics if len(b) != Size.

If Size is not set, will set to len of first value.

func (*ColFixedStr) AppendArr added in v0.55.0

func (c *ColFixedStr) AppendArr(vs [][]byte)

func (*ColFixedStr) Array added in v0.51.0

func (c *ColFixedStr) Array() *ColArr[[]byte]

Array returns new Array(FixedString).

func (*ColFixedStr) DecodeColumn

func (c *ColFixedStr) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes ColFixedStr rows from *Reader.

func (ColFixedStr) EncodeColumn

func (c ColFixedStr) EncodeColumn(b *Buffer)

EncodeColumn encodes ColFixedStr rows to *Buffer.

func (*ColFixedStr) Reset

func (c *ColFixedStr) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr) Row

func (c ColFixedStr) Row(i int) []byte

Row returns value of "i" row.

func (ColFixedStr) Rows

func (c ColFixedStr) Rows() int

Rows returns count of rows in column.

func (*ColFixedStr) SetSize

func (c *ColFixedStr) SetSize(n int)

SetSize sets Size of FixedString(Size) to n.

Can be called during decode to infer size from result.

func (ColFixedStr) Type

func (c ColFixedStr) Type() ColumnType

Type returns ColumnType of FixedString.

type ColFixedStr128 added in v0.57.0

type ColFixedStr128 [][128]byte

ColFixedStr128 represents FixedStr128 column.

func (*ColFixedStr128) Append added in v0.57.0

func (c *ColFixedStr128) Append(v [128]byte)

Append [128]byte to column.

func (*ColFixedStr128) AppendArr added in v0.57.0

func (c *ColFixedStr128) AppendArr(vs [][128]byte)

Append [128]byte slice to column.

func (*ColFixedStr128) Array added in v0.57.0

func (c *ColFixedStr128) Array() *ColArr[[128]byte]

Array is helper that creates Array of [128]byte.

func (*ColFixedStr128) DecodeColumn added in v0.57.0

func (c *ColFixedStr128) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr128 rows from *Reader.

func (ColFixedStr128) EncodeColumn added in v0.57.0

func (c ColFixedStr128) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr128 rows to *Buffer.

func (*ColFixedStr128) LowCardinality added in v0.57.0

func (c *ColFixedStr128) LowCardinality() *ColLowCardinality[[128]byte]

LowCardinality returns LowCardinality for FixedStr128 .

func (*ColFixedStr128) Nullable added in v0.57.0

func (c *ColFixedStr128) Nullable() *ColNullable[[128]byte]

Nullable is helper that creates Nullable([128]byte).

func (*ColFixedStr128) Reset added in v0.57.0

func (c *ColFixedStr128) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr128) Row added in v0.57.0

func (c ColFixedStr128) Row(i int) [128]byte

Row returns i-th row of column.

func (ColFixedStr128) Rows added in v0.57.0

func (c ColFixedStr128) Rows() int

Rows returns count of rows in column.

func (ColFixedStr128) Type added in v0.57.0

func (ColFixedStr128) Type() ColumnType

Type returns ColumnType of FixedStr128.

type ColFixedStr16 added in v0.57.0

type ColFixedStr16 [][16]byte

ColFixedStr16 represents FixedStr16 column.

func (*ColFixedStr16) Append added in v0.57.0

func (c *ColFixedStr16) Append(v [16]byte)

Append [16]byte to column.

func (*ColFixedStr16) AppendArr added in v0.57.0

func (c *ColFixedStr16) AppendArr(vs [][16]byte)

Append [16]byte slice to column.

func (*ColFixedStr16) Array added in v0.57.0

func (c *ColFixedStr16) Array() *ColArr[[16]byte]

Array is helper that creates Array of [16]byte.

func (*ColFixedStr16) DecodeColumn added in v0.57.0

func (c *ColFixedStr16) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr16 rows from *Reader.

func (ColFixedStr16) EncodeColumn added in v0.57.0

func (c ColFixedStr16) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr16 rows to *Buffer.

func (*ColFixedStr16) LowCardinality added in v0.57.0

func (c *ColFixedStr16) LowCardinality() *ColLowCardinality[[16]byte]

LowCardinality returns LowCardinality for FixedStr16 .

func (*ColFixedStr16) Nullable added in v0.57.0

func (c *ColFixedStr16) Nullable() *ColNullable[[16]byte]

Nullable is helper that creates Nullable([16]byte).

func (*ColFixedStr16) Reset added in v0.57.0

func (c *ColFixedStr16) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr16) Row added in v0.57.0

func (c ColFixedStr16) Row(i int) [16]byte

Row returns i-th row of column.

func (ColFixedStr16) Rows added in v0.57.0

func (c ColFixedStr16) Rows() int

Rows returns count of rows in column.

func (ColFixedStr16) Type added in v0.57.0

func (ColFixedStr16) Type() ColumnType

Type returns ColumnType of FixedStr16.

type ColFixedStr256 added in v0.57.0

type ColFixedStr256 [][256]byte

ColFixedStr256 represents FixedStr256 column.

func (*ColFixedStr256) Append added in v0.57.0

func (c *ColFixedStr256) Append(v [256]byte)

Append [256]byte to column.

func (*ColFixedStr256) AppendArr added in v0.57.0

func (c *ColFixedStr256) AppendArr(vs [][256]byte)

Append [256]byte slice to column.

func (*ColFixedStr256) Array added in v0.57.0

func (c *ColFixedStr256) Array() *ColArr[[256]byte]

Array is helper that creates Array of [256]byte.

func (*ColFixedStr256) DecodeColumn added in v0.57.0

func (c *ColFixedStr256) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr256 rows from *Reader.

func (ColFixedStr256) EncodeColumn added in v0.57.0

func (c ColFixedStr256) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr256 rows to *Buffer.

func (*ColFixedStr256) LowCardinality added in v0.57.0

func (c *ColFixedStr256) LowCardinality() *ColLowCardinality[[256]byte]

LowCardinality returns LowCardinality for FixedStr256 .

func (*ColFixedStr256) Nullable added in v0.57.0

func (c *ColFixedStr256) Nullable() *ColNullable[[256]byte]

Nullable is helper that creates Nullable([256]byte).

func (*ColFixedStr256) Reset added in v0.57.0

func (c *ColFixedStr256) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr256) Row added in v0.57.0

func (c ColFixedStr256) Row(i int) [256]byte

Row returns i-th row of column.

func (ColFixedStr256) Rows added in v0.57.0

func (c ColFixedStr256) Rows() int

Rows returns count of rows in column.

func (ColFixedStr256) Type added in v0.57.0

func (ColFixedStr256) Type() ColumnType

Type returns ColumnType of FixedStr256.

type ColFixedStr32 added in v0.57.0

type ColFixedStr32 [][32]byte

ColFixedStr32 represents FixedStr32 column.

func (*ColFixedStr32) Append added in v0.57.0

func (c *ColFixedStr32) Append(v [32]byte)

Append [32]byte to column.

func (*ColFixedStr32) AppendArr added in v0.57.0

func (c *ColFixedStr32) AppendArr(vs [][32]byte)

Append [32]byte slice to column.

func (*ColFixedStr32) Array added in v0.57.0

func (c *ColFixedStr32) Array() *ColArr[[32]byte]

Array is helper that creates Array of [32]byte.

func (*ColFixedStr32) DecodeColumn added in v0.57.0

func (c *ColFixedStr32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr32 rows from *Reader.

func (ColFixedStr32) EncodeColumn added in v0.57.0

func (c ColFixedStr32) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr32 rows to *Buffer.

func (*ColFixedStr32) LowCardinality added in v0.57.0

func (c *ColFixedStr32) LowCardinality() *ColLowCardinality[[32]byte]

LowCardinality returns LowCardinality for FixedStr32 .

func (*ColFixedStr32) Nullable added in v0.57.0

func (c *ColFixedStr32) Nullable() *ColNullable[[32]byte]

Nullable is helper that creates Nullable([32]byte).

func (*ColFixedStr32) Reset added in v0.57.0

func (c *ColFixedStr32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr32) Row added in v0.57.0

func (c ColFixedStr32) Row(i int) [32]byte

Row returns i-th row of column.

func (ColFixedStr32) Rows added in v0.57.0

func (c ColFixedStr32) Rows() int

Rows returns count of rows in column.

func (ColFixedStr32) Type added in v0.57.0

func (ColFixedStr32) Type() ColumnType

Type returns ColumnType of FixedStr32.

type ColFixedStr512 added in v0.57.0

type ColFixedStr512 [][512]byte

ColFixedStr512 represents FixedStr512 column.

func (*ColFixedStr512) Append added in v0.57.0

func (c *ColFixedStr512) Append(v [512]byte)

Append [512]byte to column.

func (*ColFixedStr512) AppendArr added in v0.57.0

func (c *ColFixedStr512) AppendArr(vs [][512]byte)

Append [512]byte slice to column.

func (*ColFixedStr512) Array added in v0.57.0

func (c *ColFixedStr512) Array() *ColArr[[512]byte]

Array is helper that creates Array of [512]byte.

func (*ColFixedStr512) DecodeColumn added in v0.57.0

func (c *ColFixedStr512) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr512 rows from *Reader.

func (ColFixedStr512) EncodeColumn added in v0.57.0

func (c ColFixedStr512) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr512 rows to *Buffer.

func (*ColFixedStr512) LowCardinality added in v0.57.0

func (c *ColFixedStr512) LowCardinality() *ColLowCardinality[[512]byte]

LowCardinality returns LowCardinality for FixedStr512 .

func (*ColFixedStr512) Nullable added in v0.57.0

func (c *ColFixedStr512) Nullable() *ColNullable[[512]byte]

Nullable is helper that creates Nullable([512]byte).

func (*ColFixedStr512) Reset added in v0.57.0

func (c *ColFixedStr512) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr512) Row added in v0.57.0

func (c ColFixedStr512) Row(i int) [512]byte

Row returns i-th row of column.

func (ColFixedStr512) Rows added in v0.57.0

func (c ColFixedStr512) Rows() int

Rows returns count of rows in column.

func (ColFixedStr512) Type added in v0.57.0

func (ColFixedStr512) Type() ColumnType

Type returns ColumnType of FixedStr512.

type ColFixedStr64 added in v0.57.0

type ColFixedStr64 [][64]byte

ColFixedStr64 represents FixedStr64 column.

func (*ColFixedStr64) Append added in v0.57.0

func (c *ColFixedStr64) Append(v [64]byte)

Append [64]byte to column.

func (*ColFixedStr64) AppendArr added in v0.57.0

func (c *ColFixedStr64) AppendArr(vs [][64]byte)

Append [64]byte slice to column.

func (*ColFixedStr64) Array added in v0.57.0

func (c *ColFixedStr64) Array() *ColArr[[64]byte]

Array is helper that creates Array of [64]byte.

func (*ColFixedStr64) DecodeColumn added in v0.57.0

func (c *ColFixedStr64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr64 rows from *Reader.

func (ColFixedStr64) EncodeColumn added in v0.57.0

func (c ColFixedStr64) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr64 rows to *Buffer.

func (*ColFixedStr64) LowCardinality added in v0.57.0

func (c *ColFixedStr64) LowCardinality() *ColLowCardinality[[64]byte]

LowCardinality returns LowCardinality for FixedStr64 .

func (*ColFixedStr64) Nullable added in v0.57.0

func (c *ColFixedStr64) Nullable() *ColNullable[[64]byte]

Nullable is helper that creates Nullable([64]byte).

func (*ColFixedStr64) Reset added in v0.57.0

func (c *ColFixedStr64) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr64) Row added in v0.57.0

func (c ColFixedStr64) Row(i int) [64]byte

Row returns i-th row of column.

func (ColFixedStr64) Rows added in v0.57.0

func (c ColFixedStr64) Rows() int

Rows returns count of rows in column.

func (ColFixedStr64) Type added in v0.57.0

func (ColFixedStr64) Type() ColumnType

Type returns ColumnType of FixedStr64.

type ColFixedStr8 added in v0.57.0

type ColFixedStr8 [][8]byte

ColFixedStr8 represents FixedStr8 column.

func (*ColFixedStr8) Append added in v0.57.0

func (c *ColFixedStr8) Append(v [8]byte)

Append [8]byte to column.

func (*ColFixedStr8) AppendArr added in v0.57.0

func (c *ColFixedStr8) AppendArr(vs [][8]byte)

Append [8]byte slice to column.

func (*ColFixedStr8) Array added in v0.57.0

func (c *ColFixedStr8) Array() *ColArr[[8]byte]

Array is helper that creates Array of [8]byte.

func (*ColFixedStr8) DecodeColumn added in v0.57.0

func (c *ColFixedStr8) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes FixedStr8 rows from *Reader.

func (ColFixedStr8) EncodeColumn added in v0.57.0

func (c ColFixedStr8) EncodeColumn(b *Buffer)

EncodeColumn encodes FixedStr8 rows to *Buffer.

func (*ColFixedStr8) LowCardinality added in v0.57.0

func (c *ColFixedStr8) LowCardinality() *ColLowCardinality[[8]byte]

LowCardinality returns LowCardinality for FixedStr8 .

func (*ColFixedStr8) Nullable added in v0.57.0

func (c *ColFixedStr8) Nullable() *ColNullable[[8]byte]

Nullable is helper that creates Nullable([8]byte).

func (*ColFixedStr8) Reset added in v0.57.0

func (c *ColFixedStr8) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFixedStr8) Row added in v0.57.0

func (c ColFixedStr8) Row(i int) [8]byte

Row returns i-th row of column.

func (ColFixedStr8) Rows added in v0.57.0

func (c ColFixedStr8) Rows() int

Rows returns count of rows in column.

func (ColFixedStr8) Type added in v0.57.0

func (ColFixedStr8) Type() ColumnType

Type returns ColumnType of FixedStr8.

type ColFloat32

type ColFloat32 []float32

ColFloat32 represents Float32 column.

func (*ColFloat32) Append

func (c *ColFloat32) Append(v float32)

Append float32 to column.

func (*ColFloat32) AppendArr

func (c *ColFloat32) AppendArr(vs []float32)

Append float32 slice to column.

func (*ColFloat32) Array

func (c *ColFloat32) Array() *ColArr[float32]

Array is helper that creates Array of float32.

func (*ColFloat32) DecodeColumn

func (c *ColFloat32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Float32 rows from *Reader.

func (ColFloat32) EncodeColumn

func (c ColFloat32) EncodeColumn(b *Buffer)

EncodeColumn encodes Float32 rows to *Buffer.

func (*ColFloat32) LowCardinality

func (c *ColFloat32) LowCardinality() *ColLowCardinality[float32]

LowCardinality returns LowCardinality for Float32 .

func (*ColFloat32) Nullable added in v0.41.0

func (c *ColFloat32) Nullable() *ColNullable[float32]

Nullable is helper that creates Nullable(float32).

func (*ColFloat32) Reset

func (c *ColFloat32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFloat32) Row

func (c ColFloat32) Row(i int) float32

Row returns i-th row of column.

func (ColFloat32) Rows

func (c ColFloat32) Rows() int

Rows returns count of rows in column.

func (ColFloat32) Type

func (ColFloat32) Type() ColumnType

Type returns ColumnType of Float32.

type ColFloat64

type ColFloat64 []float64

ColFloat64 represents Float64 column.

func (*ColFloat64) Append

func (c *ColFloat64) Append(v float64)

Append float64 to column.

func (*ColFloat64) AppendArr

func (c *ColFloat64) AppendArr(vs []float64)

Append float64 slice to column.

func (*ColFloat64) Array

func (c *ColFloat64) Array() *ColArr[float64]

Array is helper that creates Array of float64.

func (*ColFloat64) DecodeColumn

func (c *ColFloat64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Float64 rows from *Reader.

func (ColFloat64) EncodeColumn

func (c ColFloat64) EncodeColumn(b *Buffer)

EncodeColumn encodes Float64 rows to *Buffer.

func (*ColFloat64) LowCardinality

func (c *ColFloat64) LowCardinality() *ColLowCardinality[float64]

LowCardinality returns LowCardinality for Float64 .

func (*ColFloat64) Nullable added in v0.41.0

func (c *ColFloat64) Nullable() *ColNullable[float64]

Nullable is helper that creates Nullable(float64).

func (*ColFloat64) Reset

func (c *ColFloat64) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColFloat64) Row

func (c ColFloat64) Row(i int) float64

Row returns i-th row of column.

func (ColFloat64) Rows

func (c ColFloat64) Rows() int

Rows returns count of rows in column.

func (ColFloat64) Type

func (ColFloat64) Type() ColumnType

Type returns ColumnType of Float64.

type ColIPv4

type ColIPv4 []IPv4

ColIPv4 represents IPv4 column.

func (*ColIPv4) Append

func (c *ColIPv4) Append(v IPv4)

Append IPv4 to column.

func (*ColIPv4) AppendArr

func (c *ColIPv4) AppendArr(vs []IPv4)

Append IPv4 slice to column.

func (*ColIPv4) Array

func (c *ColIPv4) Array() *ColArr[IPv4]

Array is helper that creates Array of IPv4.

func (*ColIPv4) DecodeColumn

func (c *ColIPv4) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes IPv4 rows from *Reader.

func (ColIPv4) EncodeColumn

func (c ColIPv4) EncodeColumn(b *Buffer)

EncodeColumn encodes IPv4 rows to *Buffer.

func (*ColIPv4) LowCardinality

func (c *ColIPv4) LowCardinality() *ColLowCardinality[IPv4]

LowCardinality returns LowCardinality for IPv4 .

func (*ColIPv4) Nullable added in v0.41.0

func (c *ColIPv4) Nullable() *ColNullable[IPv4]

Nullable is helper that creates Nullable(IPv4).

func (*ColIPv4) Reset

func (c *ColIPv4) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColIPv4) Row

func (c ColIPv4) Row(i int) IPv4

Row returns i-th row of column.

func (ColIPv4) Rows

func (c ColIPv4) Rows() int

Rows returns count of rows in column.

func (ColIPv4) Type

func (ColIPv4) Type() ColumnType

Type returns ColumnType of IPv4.

type ColIPv6

type ColIPv6 []IPv6

ColIPv6 represents IPv6 column.

func (*ColIPv6) Append

func (c *ColIPv6) Append(v IPv6)

Append IPv6 to column.

func (*ColIPv6) AppendArr

func (c *ColIPv6) AppendArr(vs []IPv6)

Append IPv6 slice to column.

func (*ColIPv6) Array

func (c *ColIPv6) Array() *ColArr[IPv6]

Array is helper that creates Array of IPv6.

func (*ColIPv6) DecodeColumn

func (c *ColIPv6) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes IPv6 rows from *Reader.

func (ColIPv6) EncodeColumn

func (c ColIPv6) EncodeColumn(b *Buffer)

EncodeColumn encodes IPv6 rows to *Buffer.

func (*ColIPv6) LowCardinality

func (c *ColIPv6) LowCardinality() *ColLowCardinality[IPv6]

LowCardinality returns LowCardinality for IPv6 .

func (*ColIPv6) Nullable added in v0.41.0

func (c *ColIPv6) Nullable() *ColNullable[IPv6]

Nullable is helper that creates Nullable(IPv6).

func (*ColIPv6) Reset

func (c *ColIPv6) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColIPv6) Row

func (c ColIPv6) Row(i int) IPv6

Row returns i-th row of column.

func (ColIPv6) Rows

func (c ColIPv6) Rows() int

Rows returns count of rows in column.

func (ColIPv6) Type

func (ColIPv6) Type() ColumnType

Type returns ColumnType of IPv6.

type ColInfo

type ColInfo struct {
	Name string
	Type ColumnType
}

ColInfo wraps Name and Type of column.

type ColInfoInput

type ColInfoInput []ColInfo

ColInfoInput saves column info on decoding.

func (*ColInfoInput) DecodeResult

func (s *ColInfoInput) DecodeResult(r *Reader, version int, b Block) error

func (*ColInfoInput) Reset

func (s *ColInfoInput) Reset()

type ColInput

type ColInput interface {
	Type() ColumnType
	Rows() int
	EncodeColumn(b *Buffer)
}

ColInput column.

type ColInt128

type ColInt128 []Int128

ColInt128 represents Int128 column.

func (*ColInt128) Append

func (c *ColInt128) Append(v Int128)

Append Int128 to column.

func (*ColInt128) AppendArr

func (c *ColInt128) AppendArr(vs []Int128)

Append Int128 slice to column.

func (*ColInt128) Array

func (c *ColInt128) Array() *ColArr[Int128]

Array is helper that creates Array of Int128.

func (*ColInt128) DecodeColumn

func (c *ColInt128) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int128 rows from *Reader.

func (ColInt128) EncodeColumn

func (c ColInt128) EncodeColumn(b *Buffer)

EncodeColumn encodes Int128 rows to *Buffer.

func (*ColInt128) LowCardinality

func (c *ColInt128) LowCardinality() *ColLowCardinality[Int128]

LowCardinality returns LowCardinality for Int128 .

func (*ColInt128) Nullable added in v0.41.0

func (c *ColInt128) Nullable() *ColNullable[Int128]

Nullable is helper that creates Nullable(Int128).

func (*ColInt128) Reset

func (c *ColInt128) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt128) Row

func (c ColInt128) Row(i int) Int128

Row returns i-th row of column.

func (ColInt128) Rows

func (c ColInt128) Rows() int

Rows returns count of rows in column.

func (ColInt128) Type

func (ColInt128) Type() ColumnType

Type returns ColumnType of Int128.

type ColInt16

type ColInt16 []int16

ColInt16 represents Int16 column.

func (*ColInt16) Append

func (c *ColInt16) Append(v int16)

Append int16 to column.

func (*ColInt16) AppendArr

func (c *ColInt16) AppendArr(vs []int16)

Append int16 slice to column.

func (*ColInt16) Array

func (c *ColInt16) Array() *ColArr[int16]

Array is helper that creates Array of int16.

func (*ColInt16) DecodeColumn

func (c *ColInt16) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int16 rows from *Reader.

func (ColInt16) EncodeColumn

func (c ColInt16) EncodeColumn(b *Buffer)

EncodeColumn encodes Int16 rows to *Buffer.

func (*ColInt16) LowCardinality

func (c *ColInt16) LowCardinality() *ColLowCardinality[int16]

LowCardinality returns LowCardinality for Int16 .

func (*ColInt16) Nullable added in v0.41.0

func (c *ColInt16) Nullable() *ColNullable[int16]

Nullable is helper that creates Nullable(int16).

func (*ColInt16) Reset

func (c *ColInt16) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt16) Row

func (c ColInt16) Row(i int) int16

Row returns i-th row of column.

func (ColInt16) Rows

func (c ColInt16) Rows() int

Rows returns count of rows in column.

func (ColInt16) Type

func (ColInt16) Type() ColumnType

Type returns ColumnType of Int16.

type ColInt256

type ColInt256 []Int256

ColInt256 represents Int256 column.

func (*ColInt256) Append

func (c *ColInt256) Append(v Int256)

Append Int256 to column.

func (*ColInt256) AppendArr

func (c *ColInt256) AppendArr(vs []Int256)

Append Int256 slice to column.

func (*ColInt256) Array

func (c *ColInt256) Array() *ColArr[Int256]

Array is helper that creates Array of Int256.

func (*ColInt256) DecodeColumn

func (c *ColInt256) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int256 rows from *Reader.

func (ColInt256) EncodeColumn

func (c ColInt256) EncodeColumn(b *Buffer)

EncodeColumn encodes Int256 rows to *Buffer.

func (*ColInt256) LowCardinality

func (c *ColInt256) LowCardinality() *ColLowCardinality[Int256]

LowCardinality returns LowCardinality for Int256 .

func (*ColInt256) Nullable added in v0.41.0

func (c *ColInt256) Nullable() *ColNullable[Int256]

Nullable is helper that creates Nullable(Int256).

func (*ColInt256) Reset

func (c *ColInt256) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt256) Row

func (c ColInt256) Row(i int) Int256

Row returns i-th row of column.

func (ColInt256) Rows

func (c ColInt256) Rows() int

Rows returns count of rows in column.

func (ColInt256) Type

func (ColInt256) Type() ColumnType

Type returns ColumnType of Int256.

type ColInt32

type ColInt32 []int32

ColInt32 represents Int32 column.

func (*ColInt32) Append

func (c *ColInt32) Append(v int32)

Append int32 to column.

func (*ColInt32) AppendArr

func (c *ColInt32) AppendArr(vs []int32)

Append int32 slice to column.

func (*ColInt32) Array

func (c *ColInt32) Array() *ColArr[int32]

Array is helper that creates Array of int32.

func (*ColInt32) DecodeColumn

func (c *ColInt32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int32 rows from *Reader.

func (ColInt32) EncodeColumn

func (c ColInt32) EncodeColumn(b *Buffer)

EncodeColumn encodes Int32 rows to *Buffer.

func (*ColInt32) LowCardinality

func (c *ColInt32) LowCardinality() *ColLowCardinality[int32]

LowCardinality returns LowCardinality for Int32 .

func (*ColInt32) Nullable added in v0.41.0

func (c *ColInt32) Nullable() *ColNullable[int32]

Nullable is helper that creates Nullable(int32).

func (*ColInt32) Reset

func (c *ColInt32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt32) Row

func (c ColInt32) Row(i int) int32

Row returns i-th row of column.

func (ColInt32) Rows

func (c ColInt32) Rows() int

Rows returns count of rows in column.

func (ColInt32) Type

func (ColInt32) Type() ColumnType

Type returns ColumnType of Int32.

type ColInt64

type ColInt64 []int64

ColInt64 represents Int64 column.

func (*ColInt64) Append

func (c *ColInt64) Append(v int64)

Append int64 to column.

func (*ColInt64) AppendArr

func (c *ColInt64) AppendArr(vs []int64)

Append int64 slice to column.

func (*ColInt64) Array

func (c *ColInt64) Array() *ColArr[int64]

Array is helper that creates Array of int64.

func (*ColInt64) DecodeColumn

func (c *ColInt64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int64 rows from *Reader.

func (ColInt64) EncodeColumn

func (c ColInt64) EncodeColumn(b *Buffer)

EncodeColumn encodes Int64 rows to *Buffer.

func (*ColInt64) LowCardinality

func (c *ColInt64) LowCardinality() *ColLowCardinality[int64]

LowCardinality returns LowCardinality for Int64 .

func (*ColInt64) Nullable added in v0.41.0

func (c *ColInt64) Nullable() *ColNullable[int64]

Nullable is helper that creates Nullable(int64).

func (*ColInt64) Reset

func (c *ColInt64) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt64) Row

func (c ColInt64) Row(i int) int64

Row returns i-th row of column.

func (ColInt64) Rows

func (c ColInt64) Rows() int

Rows returns count of rows in column.

func (ColInt64) Type

func (ColInt64) Type() ColumnType

Type returns ColumnType of Int64.

type ColInt8

type ColInt8 []int8

ColInt8 represents Int8 column.

func (*ColInt8) Append

func (c *ColInt8) Append(v int8)

Append int8 to column.

func (*ColInt8) AppendArr

func (c *ColInt8) AppendArr(vs []int8)

Append int8 slice to column.

func (*ColInt8) Array

func (c *ColInt8) Array() *ColArr[int8]

Array is helper that creates Array of int8.

func (*ColInt8) DecodeColumn

func (c *ColInt8) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes Int8 rows from *Reader.

func (ColInt8) EncodeColumn

func (c ColInt8) EncodeColumn(b *Buffer)

EncodeColumn encodes Int8 rows to *Buffer.

func (*ColInt8) LowCardinality

func (c *ColInt8) LowCardinality() *ColLowCardinality[int8]

LowCardinality returns LowCardinality for Int8 .

func (*ColInt8) Nullable added in v0.41.0

func (c *ColInt8) Nullable() *ColNullable[int8]

Nullable is helper that creates Nullable(int8).

func (*ColInt8) Reset

func (c *ColInt8) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColInt8) Row

func (c ColInt8) Row(i int) int8

Row returns i-th row of column.

func (ColInt8) Rows

func (c ColInt8) Rows() int

Rows returns count of rows in column.

func (ColInt8) Type

func (ColInt8) Type() ColumnType

Type returns ColumnType of Int8.

type ColInterval added in v0.45.0

type ColInterval struct {
	Scale  IntervalScale
	Values ColInt64
}

func (*ColInterval) Append added in v0.45.0

func (c *ColInterval) Append(v Interval)

func (*ColInterval) DecodeColumn added in v0.45.0

func (c *ColInterval) DecodeColumn(r *Reader, rows int) error

func (ColInterval) EncodeColumn added in v0.45.0

func (c ColInterval) EncodeColumn(b *Buffer)

func (*ColInterval) Infer added in v0.45.0

func (c *ColInterval) Infer(t ColumnType) error

func (*ColInterval) Reset added in v0.45.0

func (c *ColInterval) Reset()

func (ColInterval) Row added in v0.45.0

func (c ColInterval) Row(i int) Interval

func (ColInterval) Rows added in v0.45.0

func (c ColInterval) Rows() int

func (ColInterval) Type added in v0.45.0

func (c ColInterval) Type() ColumnType

type ColLowCardinality

type ColLowCardinality[T comparable] struct {
	Values []T
	// contains filtered or unexported fields
}

ColLowCardinality is generic LowCardinality(T) column.

ColLowCardinality contains index and keys columns.

Index (i.e. dictionary) column contains unique values, Keys column contains sequence of indexes in Index column that represent actual values.

For example, ["Eko", "Eko", "Amadela", "Amadela", "Amadela", "Amadela"] can be encoded as:

Index: ["Eko", "Amadela"] (String)
Keys:  [0, 0, 1, 1, 1, 1] (UInt8)

The CardinalityKey is chosen depending on Index size, i.e. maximum value of chosen type should be able to represent any index of Index element.

func NewLowCardinality added in v0.41.0

func NewLowCardinality[T comparable](c ColumnOf[T]) *ColLowCardinality[T]

NewLowCardinality creates new LowCardinality column from another column for T.

func (*ColLowCardinality[T]) Append added in v0.41.0

func (c *ColLowCardinality[T]) Append(v T)

Append value to column.

func (*ColLowCardinality[T]) AppendArr added in v0.41.0

func (c *ColLowCardinality[T]) AppendArr(v []T)

AppendArr appends slice to column.

func (*ColLowCardinality[T]) Array added in v0.41.0

func (c *ColLowCardinality[T]) Array() *ColArr[T]

Array is helper that creates Array(ColLowCardinality(T)).

func (*ColLowCardinality[T]) DecodeColumn

func (c *ColLowCardinality[T]) DecodeColumn(r *Reader, rows int) error

func (*ColLowCardinality[T]) DecodeState

func (c *ColLowCardinality[T]) DecodeState(r *Reader) error

DecodeState implements StateDecoder, ensuring state for index column.

func (*ColLowCardinality[T]) EncodeColumn

func (c *ColLowCardinality[T]) EncodeColumn(b *Buffer)

func (ColLowCardinality[T]) EncodeState

func (c ColLowCardinality[T]) EncodeState(b *Buffer)

EncodeState implements StateEncoder, ensuring state for index column.

func (*ColLowCardinality[T]) Prepare added in v0.41.0

func (c *ColLowCardinality[T]) Prepare() error

Prepare column for ingestion.

func (*ColLowCardinality[T]) Reset

func (c *ColLowCardinality[T]) Reset()

func (ColLowCardinality[T]) Row added in v0.41.0

func (c ColLowCardinality[T]) Row(i int) T

Row returns i-th row.

func (ColLowCardinality[T]) Rows

func (c ColLowCardinality[T]) Rows() int

Rows returns rows count.

func (ColLowCardinality[T]) Type

func (c ColLowCardinality[T]) Type() ColumnType

type ColLowCardinalityRaw added in v0.49.0

type ColLowCardinalityRaw struct {
	Index Column // dictionary
	Key   CardinalityKey

	Keys8  ColUInt8
	Keys16 ColUInt16
	Keys32 ColUInt32
	Keys64 ColUInt64
}

ColLowCardinalityRaw is non-generic version of ColLowCardinality.

func (*ColLowCardinalityRaw) AppendKey added in v0.49.0

func (c *ColLowCardinalityRaw) AppendKey(i int)

func (*ColLowCardinalityRaw) DecodeColumn added in v0.49.0

func (c *ColLowCardinalityRaw) DecodeColumn(r *Reader, rows int) error

func (*ColLowCardinalityRaw) DecodeState added in v0.49.0

func (c *ColLowCardinalityRaw) DecodeState(r *Reader) error

func (ColLowCardinalityRaw) EncodeColumn added in v0.49.0

func (c ColLowCardinalityRaw) EncodeColumn(b *Buffer)

func (ColLowCardinalityRaw) EncodeState added in v0.49.0

func (c ColLowCardinalityRaw) EncodeState(b *Buffer)

func (*ColLowCardinalityRaw) Keys added in v0.49.0

func (c *ColLowCardinalityRaw) Keys() Column

func (*ColLowCardinalityRaw) Reset added in v0.49.0

func (c *ColLowCardinalityRaw) Reset()

func (ColLowCardinalityRaw) Rows added in v0.49.0

func (c ColLowCardinalityRaw) Rows() int

func (ColLowCardinalityRaw) Type added in v0.49.0

type ColMap

type ColMap[K comparable, V any] struct {
	Offsets ColUInt64
	Keys    ColumnOf[K]
	Values  ColumnOf[V]
}

ColMap implements Map(K, V) as ColumnOf[map[K]V].

func NewMap added in v0.41.0

func NewMap[K comparable, V any](k ColumnOf[K], v ColumnOf[V]) *ColMap[K, V]

NewMap constructs Map(K, V).

func (*ColMap[K, V]) Append added in v0.41.0

func (c *ColMap[K, V]) Append(m map[K]V)

func (*ColMap[K, V]) AppendArr added in v0.41.0

func (c *ColMap[K, V]) AppendArr(v []map[K]V)

func (*ColMap[K, V]) AppendKV added in v0.57.0

func (c *ColMap[K, V]) AppendKV(kv []KV[K, V])

AppendKV is a convenience method for appending a slice of KV[K, V].

func (*ColMap[K, V]) DecodeColumn

func (c *ColMap[K, V]) DecodeColumn(r *Reader, rows int) error

func (*ColMap[K, V]) DecodeState

func (c *ColMap[K, V]) DecodeState(r *Reader) error

func (ColMap[K, V]) EncodeColumn

func (c ColMap[K, V]) EncodeColumn(b *Buffer)

func (ColMap[K, V]) EncodeState

func (c ColMap[K, V]) EncodeState(b *Buffer)

func (*ColMap[K, V]) Infer added in v0.57.0

func (c *ColMap[K, V]) Infer(t ColumnType) error

Infer ensures Inferable column propagation.

func (ColMap[K, V]) Prepare added in v0.57.0

func (c ColMap[K, V]) Prepare() error

Prepare ensures Preparable column propagation.

func (*ColMap[K, V]) Reset

func (c *ColMap[K, V]) Reset()

func (ColMap[K, V]) Row added in v0.41.0

func (c ColMap[K, V]) Row(i int) map[K]V

func (ColMap[K, V]) RowKV added in v0.61.0

func (c ColMap[K, V]) RowKV(i int) []KV[K, V]

RowKV returns a slice of KV[K, V] for a given row.

func (ColMap[K, V]) Rows

func (c ColMap[K, V]) Rows() int

func (ColMap[K, V]) Type

func (c ColMap[K, V]) Type() ColumnType

type ColNamed added in v0.42.0

type ColNamed[T any] struct {
	ColumnOf[T]
	Name string
}

ColNamed is named column. Used in named tuples.

func Named added in v0.42.0

func Named[T any](data ColumnOf[T], name string) *ColNamed[T]

func (ColNamed[T]) ColumnName added in v0.42.0

func (c ColNamed[T]) ColumnName() string

func (ColNamed[T]) DecodeState added in v0.42.0

func (c ColNamed[T]) DecodeState(r *Reader) error

func (ColNamed[T]) EncodeState added in v0.42.0

func (c ColNamed[T]) EncodeState(b *Buffer)

func (*ColNamed[T]) Infer added in v0.42.0

func (c *ColNamed[T]) Infer(t ColumnType) error

func (*ColNamed[T]) Prepare added in v0.42.0

func (c *ColNamed[T]) Prepare() error

func (ColNamed[T]) Type added in v0.42.0

func (c ColNamed[T]) Type() ColumnType

type ColNothing added in v0.45.0

type ColNothing int

ColNothing represents column of null values. Value is row count.

https://clickhouse.com/docs/ru/sql-reference/data-types/special-data-types/nothing

func (*ColNothing) Append added in v0.45.0

func (c *ColNothing) Append(_ Nothing)

func (*ColNothing) AppendArr added in v0.55.0

func (c *ColNothing) AppendArr(vs []Nothing)

func (*ColNothing) Array added in v0.45.0

func (c *ColNothing) Array() *ColArr[Nothing]

func (*ColNothing) DecodeColumn added in v0.45.0

func (c *ColNothing) DecodeColumn(r *Reader, rows int) error

func (ColNothing) EncodeColumn added in v0.45.0

func (c ColNothing) EncodeColumn(b *Buffer)

func (*ColNothing) Nullable added in v0.45.0

func (c *ColNothing) Nullable() *ColNullable[Nothing]

func (*ColNothing) Reset added in v0.45.0

func (c *ColNothing) Reset()

func (ColNothing) Row added in v0.45.0

func (c ColNothing) Row(i int) Nothing

func (ColNothing) Rows added in v0.45.0

func (c ColNothing) Rows() int

func (ColNothing) Type added in v0.45.0

func (c ColNothing) Type() ColumnType

type ColNullable

type ColNullable[T any] struct {
	Nulls  ColUInt8
	Values ColumnOf[T]
}

ColNullable represents Nullable(T) column.

Nulls is nullable "mask" on Values column. For example, to encode [null, "", "hello", null, "world"]

Values: ["", "", "hello", "", "world"] (len: 5)
Nulls:  [ 1,  0,       0,  1,       0] (len: 5)

Values and Nulls row counts are always equal.

func NewColNullable added in v0.41.0

func NewColNullable[T any](v ColumnOf[T]) *ColNullable[T]

NewColNullable returns new Nullable(T) from v column.

func (*ColNullable[T]) Append added in v0.41.0

func (c *ColNullable[T]) Append(v Nullable[T])

func (*ColNullable[T]) AppendArr added in v0.41.0

func (c *ColNullable[T]) AppendArr(v []Nullable[T])

func (*ColNullable[T]) DecodeColumn

func (c *ColNullable[T]) DecodeColumn(r *Reader, rows int) error

func (*ColNullable[T]) DecodeState added in v0.41.0

func (c *ColNullable[T]) DecodeState(r *Reader) error

func (ColNullable[T]) EncodeColumn

func (c ColNullable[T]) EncodeColumn(b *Buffer)

func (ColNullable[T]) EncodeState added in v0.41.0

func (c ColNullable[T]) EncodeState(b *Buffer)

func (ColNullable[T]) IsElemNull

func (c ColNullable[T]) IsElemNull(i int) bool

func (*ColNullable[T]) Reset

func (c *ColNullable[T]) Reset()

func (ColNullable[T]) Row added in v0.41.0

func (c ColNullable[T]) Row(i int) Nullable[T]

func (ColNullable[T]) Rows

func (c ColNullable[T]) Rows() int

func (ColNullable[T]) Type

func (c ColNullable[T]) Type() ColumnType

type ColPoint

type ColPoint struct {
	X, Y ColFloat64
}

func (*ColPoint) Append

func (c *ColPoint) Append(v Point)

func (*ColPoint) AppendArr

func (c *ColPoint) AppendArr(v []Point)

func (*ColPoint) DecodeColumn

func (c *ColPoint) DecodeColumn(r *Reader, rows int) error

func (ColPoint) EncodeColumn

func (c ColPoint) EncodeColumn(b *Buffer)

func (*ColPoint) Reset

func (c *ColPoint) Reset()

func (ColPoint) Row

func (c ColPoint) Row(i int) Point

func (ColPoint) Rows

func (c ColPoint) Rows() int

func (ColPoint) Type

func (c ColPoint) Type() ColumnType

type ColRaw

type ColRaw struct {
	T    ColumnType // type of column
	Size int        // size of single value

	Data  []byte // raw value of column
	Count int    // count of rows
}

ColRaw is Column that performs zero decoding or encoding. T, Size are required.

TODO: support strings and T, Size inference.

Useful for copying from one source to another.

func (*ColRaw) DecodeColumn

func (c *ColRaw) DecodeColumn(r *Reader, rows int) error

func (ColRaw) EncodeColumn

func (c ColRaw) EncodeColumn(b *Buffer)

func (*ColRaw) Reset

func (c *ColRaw) Reset()

func (ColRaw) Rows

func (c ColRaw) Rows() int

func (ColRaw) Type

func (c ColRaw) Type() ColumnType

type ColRawOf

type ColRawOf[X comparable] []X

ColRawOf is generic raw column.

func (*ColRawOf[X]) Append

func (c *ColRawOf[X]) Append(v X)

Append value to column.

func (*ColRawOf[X]) AppendArr

func (c *ColRawOf[X]) AppendArr(v []X)

func (*ColRawOf[X]) DecodeColumn

func (c *ColRawOf[X]) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes ColRawOf rows from *Reader.

func (ColRawOf[X]) EncodeColumn

func (c ColRawOf[X]) EncodeColumn(b *Buffer)

EncodeColumn encodes ColRawOf rows to *Buffer.

func (*ColRawOf[X]) Reset

func (c *ColRawOf[X]) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColRawOf[X]) Row

func (c ColRawOf[X]) Row(i int) X

Row returns value of "i" row.

func (ColRawOf[X]) Rows

func (c ColRawOf[X]) Rows() int

Rows returns count of rows in column.

func (ColRawOf[X]) Size

func (c ColRawOf[X]) Size() int

func (ColRawOf[X]) Type

func (c ColRawOf[X]) Type() ColumnType

Type returns ColumnType of ColRawOf.

type ColResult

type ColResult interface {
	Type() ColumnType
	Rows() int
	DecodeColumn(r *Reader, rows int) error
	Resettable
}

ColResult column.

type ColStr

type ColStr struct {
	Buf []byte
	Pos []Position
}

ColStr represents String column.

Use ColBytes for []bytes ColumnOf implementation.

func (*ColStr) Append

func (c *ColStr) Append(v string)

Append string to column.

func (*ColStr) AppendArr

func (c *ColStr) AppendArr(v []string)

func (*ColStr) AppendBytes

func (c *ColStr) AppendBytes(v []byte)

AppendBytes append byte slice as string to column.

func (*ColStr) Array

func (c *ColStr) Array() *ColArr[string]

Array is helper that creates Array(String).

func (*ColStr) DecodeColumn

func (c *ColStr) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes String rows from *Reader.

func (ColStr) EncodeColumn

func (c ColStr) EncodeColumn(b *Buffer)

EncodeColumn encodes String rows to *Buffer.

func (ColStr) First

func (c ColStr) First() string

First returns first row of column.

func (ColStr) ForEach

func (c ColStr) ForEach(f func(i int, s string) error) error

ForEach calls f on each string from column.

func (ColStr) ForEachBytes

func (c ColStr) ForEachBytes(f func(i int, b []byte) error) error

ForEachBytes calls f on each string from column as byte slice.

func (*ColStr) LowCardinality

func (c *ColStr) LowCardinality() *ColLowCardinality[string]

LowCardinality returns LowCardinality(String).

func (*ColStr) Nullable added in v0.41.0

func (c *ColStr) Nullable() *ColNullable[string]

Nullable is helper that creates Nullable(String).

func (*ColStr) Reset

func (c *ColStr) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColStr) Row

func (c ColStr) Row(i int) string

Row returns row with number i.

func (ColStr) RowBytes

func (c ColStr) RowBytes(i int) []byte

RowBytes returns row with number i as byte slice.

func (ColStr) Rows

func (c ColStr) Rows() int

Rows returns count of rows in column.

func (ColStr) Type

func (ColStr) Type() ColumnType

Type returns ColumnType of String.

type ColTuple

type ColTuple []Column

ColTuple is Tuple column.

Basically it is just a group of columns.

func (ColTuple) DecodeColumn

func (c ColTuple) DecodeColumn(r *Reader, rows int) error

func (ColTuple) DecodeState

func (c ColTuple) DecodeState(r *Reader) error

func (ColTuple) EncodeColumn

func (c ColTuple) EncodeColumn(b *Buffer)

func (ColTuple) EncodeState

func (c ColTuple) EncodeState(b *Buffer)

func (ColTuple) First

func (c ColTuple) First() Column

func (ColTuple) Infer added in v0.42.0

func (c ColTuple) Infer(t ColumnType) error

func (ColTuple) Prepare added in v0.42.0

func (c ColTuple) Prepare() error

func (ColTuple) Reset

func (c ColTuple) Reset()

func (ColTuple) Rows

func (c ColTuple) Rows() int

func (ColTuple) Type

func (c ColTuple) Type() ColumnType

type ColUInt128

type ColUInt128 []UInt128

ColUInt128 represents UInt128 column.

func (*ColUInt128) Append

func (c *ColUInt128) Append(v UInt128)

Append UInt128 to column.

func (*ColUInt128) AppendArr

func (c *ColUInt128) AppendArr(vs []UInt128)

Append UInt128 slice to column.

func (*ColUInt128) Array

func (c *ColUInt128) Array() *ColArr[UInt128]

Array is helper that creates Array of UInt128.

func (*ColUInt128) DecodeColumn

func (c *ColUInt128) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt128 rows from *Reader.

func (ColUInt128) EncodeColumn

func (c ColUInt128) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt128 rows to *Buffer.

func (*ColUInt128) LowCardinality

func (c *ColUInt128) LowCardinality() *ColLowCardinality[UInt128]

LowCardinality returns LowCardinality for UInt128 .

func (*ColUInt128) Nullable added in v0.41.0

func (c *ColUInt128) Nullable() *ColNullable[UInt128]

Nullable is helper that creates Nullable(UInt128).

func (*ColUInt128) Reset

func (c *ColUInt128) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt128) Row

func (c ColUInt128) Row(i int) UInt128

Row returns i-th row of column.

func (ColUInt128) Rows

func (c ColUInt128) Rows() int

Rows returns count of rows in column.

func (ColUInt128) Type

func (ColUInt128) Type() ColumnType

Type returns ColumnType of UInt128.

type ColUInt16

type ColUInt16 []uint16

ColUInt16 represents UInt16 column.

func (*ColUInt16) Append

func (c *ColUInt16) Append(v uint16)

Append uint16 to column.

func (*ColUInt16) AppendArr

func (c *ColUInt16) AppendArr(vs []uint16)

Append uint16 slice to column.

func (*ColUInt16) Array

func (c *ColUInt16) Array() *ColArr[uint16]

Array is helper that creates Array of uint16.

func (*ColUInt16) DecodeColumn

func (c *ColUInt16) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt16 rows from *Reader.

func (ColUInt16) EncodeColumn

func (c ColUInt16) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt16 rows to *Buffer.

func (*ColUInt16) LowCardinality

func (c *ColUInt16) LowCardinality() *ColLowCardinality[uint16]

LowCardinality returns LowCardinality for UInt16 .

func (*ColUInt16) Nullable added in v0.41.0

func (c *ColUInt16) Nullable() *ColNullable[uint16]

Nullable is helper that creates Nullable(uint16).

func (*ColUInt16) Reset

func (c *ColUInt16) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt16) Row

func (c ColUInt16) Row(i int) uint16

Row returns i-th row of column.

func (ColUInt16) Rows

func (c ColUInt16) Rows() int

Rows returns count of rows in column.

func (ColUInt16) Type

func (ColUInt16) Type() ColumnType

Type returns ColumnType of UInt16.

type ColUInt256

type ColUInt256 []UInt256

ColUInt256 represents UInt256 column.

func (*ColUInt256) Append

func (c *ColUInt256) Append(v UInt256)

Append UInt256 to column.

func (*ColUInt256) AppendArr

func (c *ColUInt256) AppendArr(vs []UInt256)

Append UInt256 slice to column.

func (*ColUInt256) Array

func (c *ColUInt256) Array() *ColArr[UInt256]

Array is helper that creates Array of UInt256.

func (*ColUInt256) DecodeColumn

func (c *ColUInt256) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt256 rows from *Reader.

func (ColUInt256) EncodeColumn

func (c ColUInt256) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt256 rows to *Buffer.

func (*ColUInt256) LowCardinality

func (c *ColUInt256) LowCardinality() *ColLowCardinality[UInt256]

LowCardinality returns LowCardinality for UInt256 .

func (*ColUInt256) Nullable added in v0.41.0

func (c *ColUInt256) Nullable() *ColNullable[UInt256]

Nullable is helper that creates Nullable(UInt256).

func (*ColUInt256) Reset

func (c *ColUInt256) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt256) Row

func (c ColUInt256) Row(i int) UInt256

Row returns i-th row of column.

func (ColUInt256) Rows

func (c ColUInt256) Rows() int

Rows returns count of rows in column.

func (ColUInt256) Type

func (ColUInt256) Type() ColumnType

Type returns ColumnType of UInt256.

type ColUInt32

type ColUInt32 []uint32

ColUInt32 represents UInt32 column.

func (*ColUInt32) Append

func (c *ColUInt32) Append(v uint32)

Append uint32 to column.

func (*ColUInt32) AppendArr

func (c *ColUInt32) AppendArr(vs []uint32)

Append uint32 slice to column.

func (*ColUInt32) Array

func (c *ColUInt32) Array() *ColArr[uint32]

Array is helper that creates Array of uint32.

func (*ColUInt32) DecodeColumn

func (c *ColUInt32) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt32 rows from *Reader.

func (ColUInt32) EncodeColumn

func (c ColUInt32) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt32 rows to *Buffer.

func (*ColUInt32) LowCardinality

func (c *ColUInt32) LowCardinality() *ColLowCardinality[uint32]

LowCardinality returns LowCardinality for UInt32 .

func (*ColUInt32) Nullable added in v0.41.0

func (c *ColUInt32) Nullable() *ColNullable[uint32]

Nullable is helper that creates Nullable(uint32).

func (*ColUInt32) Reset

func (c *ColUInt32) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt32) Row

func (c ColUInt32) Row(i int) uint32

Row returns i-th row of column.

func (ColUInt32) Rows

func (c ColUInt32) Rows() int

Rows returns count of rows in column.

func (ColUInt32) Type

func (ColUInt32) Type() ColumnType

Type returns ColumnType of UInt32.

type ColUInt64

type ColUInt64 []uint64

ColUInt64 represents UInt64 column.

func (*ColUInt64) Append

func (c *ColUInt64) Append(v uint64)

Append uint64 to column.

func (*ColUInt64) AppendArr

func (c *ColUInt64) AppendArr(vs []uint64)

Append uint64 slice to column.

func (*ColUInt64) Array

func (c *ColUInt64) Array() *ColArr[uint64]

Array is helper that creates Array of uint64.

func (*ColUInt64) DecodeColumn

func (c *ColUInt64) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt64 rows from *Reader.

func (ColUInt64) EncodeColumn

func (c ColUInt64) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt64 rows to *Buffer.

func (*ColUInt64) LowCardinality

func (c *ColUInt64) LowCardinality() *ColLowCardinality[uint64]

LowCardinality returns LowCardinality for UInt64 .

func (*ColUInt64) Nullable added in v0.41.0

func (c *ColUInt64) Nullable() *ColNullable[uint64]

Nullable is helper that creates Nullable(uint64).

func (*ColUInt64) Reset

func (c *ColUInt64) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt64) Row

func (c ColUInt64) Row(i int) uint64

Row returns i-th row of column.

func (ColUInt64) Rows

func (c ColUInt64) Rows() int

Rows returns count of rows in column.

func (ColUInt64) Type

func (ColUInt64) Type() ColumnType

Type returns ColumnType of UInt64.

type ColUInt8

type ColUInt8 []uint8

ColUInt8 represents UInt8 column.

func (*ColUInt8) Append

func (c *ColUInt8) Append(v uint8)

Append uint8 to column.

func (*ColUInt8) AppendArr

func (c *ColUInt8) AppendArr(vs []uint8)

Append uint8 slice to column.

func (*ColUInt8) Array

func (c *ColUInt8) Array() *ColArr[uint8]

Array is helper that creates Array of uint8.

func (*ColUInt8) DecodeColumn

func (c *ColUInt8) DecodeColumn(r *Reader, rows int) error

DecodeColumn decodes UInt8 rows from *Reader.

func (ColUInt8) EncodeColumn

func (c ColUInt8) EncodeColumn(b *Buffer)

EncodeColumn encodes UInt8 rows to *Buffer.

func (*ColUInt8) LowCardinality

func (c *ColUInt8) LowCardinality() *ColLowCardinality[uint8]

LowCardinality returns LowCardinality for UInt8 .

func (*ColUInt8) Nullable added in v0.41.0

func (c *ColUInt8) Nullable() *ColNullable[uint8]

Nullable is helper that creates Nullable(uint8).

func (*ColUInt8) Reset

func (c *ColUInt8) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColUInt8) Row

func (c ColUInt8) Row(i int) uint8

Row returns i-th row of column.

func (ColUInt8) Rows

func (c ColUInt8) Rows() int

Rows returns count of rows in column.

func (ColUInt8) Type

func (ColUInt8) Type() ColumnType

Type returns ColumnType of UInt8.

type ColUUID

type ColUUID []uuid.UUID

ColUUID is UUID column.

func (*ColUUID) Append

func (c *ColUUID) Append(v uuid.UUID)

func (*ColUUID) AppendArr

func (c *ColUUID) AppendArr(v []uuid.UUID)

func (*ColUUID) Array added in v0.57.0

func (c *ColUUID) Array() *ColArr[uuid.UUID]

Array is helper that creates Array of uuid.UUID.

func (*ColUUID) DecodeColumn

func (c *ColUUID) DecodeColumn(r *Reader, rows int) error

func (ColUUID) EncodeColumn

func (c ColUUID) EncodeColumn(b *Buffer)

EncodeColumn encodes ColUUID rows to *Buffer.

func (*ColUUID) Nullable added in v0.57.0

func (c *ColUUID) Nullable() *ColNullable[uuid.UUID]

Nullable is helper that creates Nullable(uuid.UUID).

func (*ColUUID) Reset

func (c *ColUUID) Reset()

func (ColUUID) Row

func (c ColUUID) Row(i int) uuid.UUID

func (ColUUID) Rows

func (c ColUUID) Rows() int

func (ColUUID) Type

func (c ColUUID) Type() ColumnType

type Column

type Column interface {
	ColResult
	ColInput
}

func Alias

func Alias(c Column, t ColumnType) Column

Alias column as other type.

E.g. Bool is domain of UInt8, so can be aliased from UInt8.

func Wrap

func Wrap(c Column, args ...interface{}) Column

Wrap Column with type parameters.

So if c type is T, result type will be T(arg0, arg1, ...).

type ColumnOf

type ColumnOf[T any] interface {
	Column
	Append(v T)
	AppendArr(v []T)
	Row(i int) T
}

ColumnOf is generic Column(T) constraint.

type ColumnType

type ColumnType string

ColumnType is type of column element.

const (
	ColumnTypeNone           ColumnType = ""
	ColumnTypeInt8           ColumnType = "Int8"
	ColumnTypeInt16          ColumnType = "Int16"
	ColumnTypeInt32          ColumnType = "Int32"
	ColumnTypeInt64          ColumnType = "Int64"
	ColumnTypeInt128         ColumnType = "Int128"
	ColumnTypeInt256         ColumnType = "Int256"
	ColumnTypeUInt8          ColumnType = "UInt8"
	ColumnTypeUInt16         ColumnType = "UInt16"
	ColumnTypeUInt32         ColumnType = "UInt32"
	ColumnTypeUInt64         ColumnType = "UInt64"
	ColumnTypeUInt128        ColumnType = "UInt128"
	ColumnTypeUInt256        ColumnType = "UInt256"
	ColumnTypeFloat32        ColumnType = "Float32"
	ColumnTypeFloat64        ColumnType = "Float64"
	ColumnTypeString         ColumnType = "String"
	ColumnTypeFixedString    ColumnType = "FixedString"
	ColumnTypeArray          ColumnType = "Array"
	ColumnTypeIPv4           ColumnType = "IPv4"
	ColumnTypeIPv6           ColumnType = "IPv6"
	ColumnTypeDateTime       ColumnType = "DateTime"
	ColumnTypeDateTime64     ColumnType = "DateTime64"
	ColumnTypeDate           ColumnType = "Date"
	ColumnTypeDate32         ColumnType = "Date32"
	ColumnTypeUUID           ColumnType = "UUID"
	ColumnTypeEnum8          ColumnType = "Enum8"
	ColumnTypeEnum16         ColumnType = "Enum16"
	ColumnTypeLowCardinality ColumnType = "LowCardinality"
	ColumnTypeMap            ColumnType = "Map"
	ColumnTypeBool           ColumnType = "Bool"
	ColumnTypeTuple          ColumnType = "Tuple"
	ColumnTypeNullable       ColumnType = "Nullable"
	ColumnTypeDecimal32      ColumnType = "Decimal32"
	ColumnTypeDecimal64      ColumnType = "Decimal64"
	ColumnTypeDecimal128     ColumnType = "Decimal128"
	ColumnTypeDecimal256     ColumnType = "Decimal256"
	ColumnTypePoint          ColumnType = "Point"
	ColumnTypeInterval       ColumnType = "Interval"
	ColumnTypeNothing        ColumnType = "Nothing"
)

Common colum type names. Does not represent full set of supported types, because ColumnTypeArray is composable; actual type is composite.

For example: Array(Int8) or even Array(Array(String)).

func (ColumnType) Array

func (c ColumnType) Array() ColumnType

Array returns Array(ColumnType).

func (ColumnType) Base

func (c ColumnType) Base() ColumnType

func (ColumnType) Conflicts

func (c ColumnType) Conflicts(b ColumnType) bool

Conflicts reports whether two types conflict.

func (ColumnType) Elem

func (c ColumnType) Elem() ColumnType

func (ColumnType) IsArray

func (c ColumnType) IsArray() bool

IsArray reports whether ColumnType is composite.

func (ColumnType) String

func (c ColumnType) String() string

func (ColumnType) Sub

func (c ColumnType) Sub(subtypes ...ColumnType) ColumnType

Sub of T returns T(A, B, ...).

func (ColumnType) With

func (c ColumnType) With(params ...string) ColumnType

With returns ColumnType(p1, p2, ...) from ColumnType.

type Compression

type Compression byte

Compression status.

const (
	CompressionDisabled Compression = 0
	CompressionEnabled  Compression = 1
)

Compression statuses.

func CompressionString

func CompressionString(s string) (Compression, error)

CompressionString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func CompressionValues

func CompressionValues() []Compression

CompressionValues returns all values of the enum

func (Compression) Encode

func (c Compression) Encode(b *Buffer)

Encode to buffer.

func (Compression) IsACompression

func (i Compression) IsACompression() bool

IsACompression returns "true" if the value is listed in the enum definition. "false" otherwise

func (Compression) String

func (i Compression) String() string

type Date

type Date uint16

Date represents Date value.

https://clickhouse.com/docs/en/sql-reference/data-types/date/

func NewDate

func NewDate(year int, month time.Month, day int) Date

NewDate returns the Date corresponding to year, month and day in UTC.

func ToDate

func ToDate(t time.Time) Date

ToDate returns Date of time.Time.

func (Date) String

func (d Date) String() string

func (Date) Time

func (d Date) Time() time.Time

Time returns UTC starting time.Time of Date.

You can use time.Unix(d.Unix(), 0) to get Time in time.Local location.

func (Date) Unix

func (d Date) Unix() int64

Unix returns unix timestamp of Date.

type Date32

type Date32 int32

Date32 represents Date32 value.

https://clickhouse.com/docs/en/sql-reference/data-types/date32/

func NewDate32

func NewDate32(year int, month time.Month, day int) Date32

NewDate32 returns the Date32 corresponding to year, month and day in UTC.

func ToDate32

func ToDate32(t time.Time) Date32

ToDate32 returns Date32 of time.Time.

func (Date32) String

func (d Date32) String() string

func (Date32) Time

func (d Date32) Time() time.Time

Time returns UTC starting time.Time of Date32.

func (Date32) Unix

func (d Date32) Unix() int64

Unix returns unix timestamp of Date32.

You can use time.Unix(d.Unix(), 0) to get Time in time.Local location.

type DateTime

type DateTime uint32

DateTime represents DateTime type.

func ToDateTime

func ToDateTime(t time.Time) DateTime

ToDateTime converts time.Time to DateTime.

func (DateTime) Time

func (d DateTime) Time() time.Time

Time returns DateTime as time.Time.

type DateTime64

type DateTime64 int64

DateTime64 represents DateTime64 type.

See https://clickhouse.com/docs/en/sql-reference/data-types/datetime64/.

func ToDateTime64

func ToDateTime64(t time.Time, p Precision) DateTime64

ToDateTime64 converts time.Time to DateTime64.

func (DateTime64) Time

func (d DateTime64) Time(p Precision) time.Time

Time returns DateTime64 as time.Time.

type Decimal128

type Decimal128 Int128

Decimal128 represents Decimal128 value.

type Decimal256

type Decimal256 Int256

Decimal256 represents Decimal256 value.

type Decimal32

type Decimal32 int32

Decimal32 represents Decimal32 value.

type Decimal64

type Decimal64 int64

Decimal64 represents Decimal32 value.

type Decoder

type Decoder interface {
	Decode(r *Reader) error
}

Decoder implements decoding from Reader.

type Encoder

type Encoder interface {
	Encode(b *Buffer)
}

Encoder implements encoding to Buffer.

type Enum16

type Enum16 int16

Enum16 represents raw Enum16 value.

Actual values should be taken from DDL.

type Enum8

type Enum8 int8

Enum8 represents raw Enum8 value.

Actual values should be taken from DDL.

type Error

type Error int

Error on server side.

const (
	ErrUnsupportedMethod                            Error = 1
	ErrUnsupportedParameter                         Error = 2
	ErrUnexpectedEndOfFile                          Error = 3
	ErrExpectedEndOfFile                            Error = 4
	ErrCannotParseText                              Error = 6
	ErrIncorrectNumberOfColumns                     Error = 7
	ErrThereIsNoColumn                              Error = 8
	ErrSizesOfColumnsDoesntMatch                    Error = 9
	ErrNotFoundColumnInBlock                        Error = 10
	ErrPositionOutOfBound                           Error = 11
	ErrParameterOutOfBound                          Error = 12
	ErrSizesOfColumnsInTupleDoesntMatch             Error = 13
	ErrDuplicateColumn                              Error = 15
	ErrNoSuchColumnInTable                          Error = 16
	ErrDelimiterInStringLiteralDoesntMatch          Error = 17
	ErrCannotInsertElementIntoConstantColumn        Error = 18
	ErrSizeOfFixedStringDoesntMatch                 Error = 19
	ErrNumberOfColumnsDoesntMatch                   Error = 20
	ErrCannotReadAllDataFromTabSeparatedInput       Error = 21
	ErrCannotParseAllValueFromTabSeparatedInput     Error = 22
	ErrCannotReadFromIstream                        Error = 23
	ErrCannotWriteToOstream                         Error = 24
	ErrCannotParseEscapeSequence                    Error = 25
	ErrCannotParseQuotedString                      Error = 26
	ErrCannotParseInputAssertionFailed              Error = 27
	ErrCannotPrintFloatOrDoubleNumber               Error = 28
	ErrCannotPrintInteger                           Error = 29
	ErrCannotReadSizeOfCompressedChunk              Error = 30
	ErrCannotReadCompressedChunk                    Error = 31
	ErrAttemptToReadAfterEOF                        Error = 32
	ErrCannotReadAllData                            Error = 33
	ErrTooManyArgumentsForFunction                  Error = 34
	ErrTooLessArgumentsForFunction                  Error = 35
	ErrBadArguments                                 Error = 36
	ErrUnknownElementInAst                          Error = 37
	ErrCannotParseDate                              Error = 38
	ErrTooLargeSizeCompressed                       Error = 39
	ErrChecksumDoesntMatch                          Error = 40
	ErrCannotParseDatetime                          Error = 41
	ErrNumberOfArgumentsDoesntMatch                 Error = 42
	ErrIllegalTypeOfArgument                        Error = 43
	ErrIllegalColumn                                Error = 44
	ErrIllegalNumberOfResultColumns                 Error = 45
	ErrUnknownFunction                              Error = 46
	ErrUnknownIdentifier                            Error = 47
	ErrNotImplemented                               Error = 48
	ErrLogicalError                                 Error = 49
	ErrUnknownType                                  Error = 50
	ErrEmptyListOfColumnsQueried                    Error = 51
	ErrColumnQueriedMoreThanOnce                    Error = 52
	ErrTypeMismatch                                 Error = 53
	ErrStorageDoesntAllowParameters                 Error = 54
	ErrStorageRequiresParameter                     Error = 55
	ErrUnknownStorage                               Error = 56
	ErrTableAlreadyExists                           Error = 57
	ErrTableMetadataAlreadyExists                   Error = 58
	ErrIllegalTypeOfColumnForFilter                 Error = 59
	ErrUnknownTable                                 Error = 60
	ErrOnlyFilterColumnInBlock                      Error = 61
	ErrSyntaxError                                  Error = 62
	ErrUnknownAggregateFunction                     Error = 63
	ErrCannotReadAggregateFunctionFromText          Error = 64
	ErrCannotWriteAggregateFunctionAsText           Error = 65
	ErrNotAColumn                                   Error = 66
	ErrIllegalKeyOfAggregation                      Error = 67
	ErrCannotGetSizeOfField                         Error = 68
	ErrArgumentOutOfBound                           Error = 69
	ErrCannotConvertType                            Error = 70
	ErrCannotWriteAfterEndOfBuffer                  Error = 71
	ErrCannotParseNumber                            Error = 72
	ErrUnknownFormat                                Error = 73
	ErrCannotReadFromFileDescriptor                 Error = 74
	ErrCannotWriteToFileDescriptor                  Error = 75
	ErrCannotOpenFile                               Error = 76
	ErrCannotCloseFile                              Error = 77
	ErrUnknownTypeOfQuery                           Error = 78
	ErrIncorrectFileName                            Error = 79
	ErrIncorrectQuery                               Error = 80
	ErrUnknownDatabase                              Error = 81
	ErrDatabaseAlreadyExists                        Error = 82
	ErrDirectoryDoesntExist                         Error = 83
	ErrDirectoryAlreadyExists                       Error = 84
	ErrFormatIsNotSuitableForInput                  Error = 85
	ErrReceivedErrorFromRemoteIoServer              Error = 86
	ErrCannotSeekThroughFile                        Error = 87
	ErrCannotTruncateFile                           Error = 88
	ErrUnknownCompressionMethod                     Error = 89
	ErrEmptyListOfColumnsPassed                     Error = 90
	ErrSizesOfMarksFilesAreInconsistent             Error = 91
	ErrEmptyDataPassed                              Error = 92
	ErrUnknownAggregatedDataVariant                 Error = 93
	ErrCannotMergeDifferentAggregatedDataVariants   Error = 94
	ErrCannotReadFromSocket                         Error = 95
	ErrCannotWriteToSocket                          Error = 96
	ErrCannotReadAllDataFromChunkedInput            Error = 97
	ErrCannotWriteToEmptyBlockOutputStream          Error = 98
	ErrUnknownPacketFromClient                      Error = 99
	ErrUnknownPacketFromServer                      Error = 100
	ErrUnexpectedPacketFromClient                   Error = 101
	ErrUnexpectedPacketFromServer                   Error = 102
	ErrReceivedDataForWrongQueryID                  Error = 103
	ErrTooSmallBufferSize                           Error = 104
	ErrCannotReadHistory                            Error = 105
	ErrCannotAppendHistory                          Error = 106
	ErrFileDoesntExist                              Error = 107
	ErrNoDataToInsert                               Error = 108
	ErrCannotBlockSignal                            Error = 109
	ErrCannotUnblockSignal                          Error = 110
	ErrCannotManipulateSigset                       Error = 111
	ErrCannotWaitForSignal                          Error = 112
	ErrThereIsNoSession                             Error = 113
	ErrCannotClockGettime                           Error = 114
	ErrUnknownSetting                               Error = 115
	ErrThereIsNoDefaultValue                        Error = 116
	ErrIncorrectData                                Error = 117
	ErrEngineRequired                               Error = 119
	ErrCannotInsertValueOfDifferentSizeIntoTuple    Error = 120
	ErrUnknownSetDataVariant                        Error = 121
	ErrIncompatibleColumns                          Error = 122
	ErrUnknownTypeOfAstNode                         Error = 123
	ErrIncorrectElementOfSet                        Error = 124
	ErrIncorrectResultOfScalarSubquery              Error = 125
	ErrCannotGetReturnType                          Error = 126
	ErrIllegalIndex                                 Error = 127
	ErrTooLargeArraySize                            Error = 128
	ErrFunctionIsSpecial                            Error = 129
	ErrCannotReadArrayFromText                      Error = 130
	ErrTooLargeStringSize                           Error = 131
	ErrCannotCreateTableFromMetadata                Error = 132
	ErrAggregateFunctionDoesntAllowParameters       Error = 133
	ErrParametersToAggregateFunctionsMustBeLiterals Error = 134
	ErrZeroArrayOrTupleIndex                        Error = 135
	ErrUnknownElementInConfig                       Error = 137
	ErrExcessiveElementInConfig                     Error = 138
	ErrNoElementsInConfig                           Error = 139
	ErrAllRequestedColumnsAreMissing                Error = 140
	ErrSamplingNotSupported                         Error = 141
	ErrNotFoundNode                                 Error = 142
	ErrFoundMoreThanOneNode                         Error = 143
	ErrFirstDateIsBiggerThanLastDate                Error = 144
	ErrUnknownOverflowMode                          Error = 145
	ErrQuerySectionDoesntMakeSense                  Error = 146
	ErrNotFoundFunctionElementForAggregate          Error = 147
	ErrNotFoundRelationElementForCondition          Error = 148
	ErrNotFoundRHSElementForCondition               Error = 149
	ErrNoAttributesListed                           Error = 150
	ErrIndexOfColumnInSortClauseIsOutOfRange        Error = 151
	ErrUnknownDirectionOfSorting                    Error = 152
	ErrIllegalDivision                              Error = 153
	ErrAggregateFunctionNotApplicable               Error = 154
	ErrUnknownRelation                              Error = 155
	ErrDictionariesWasNotLoaded                     Error = 156
	ErrIllegalOverflowMode                          Error = 157
	ErrTooManyRows                                  Error = 158
	ErrTimeoutExceeded                              Error = 159
	ErrTooSlow                                      Error = 160
	ErrTooManyColumns                               Error = 161
	ErrTooDeepSubqueries                            Error = 162
	ErrTooDeepPipeline                              Error = 163
	ErrReadonly                                     Error = 164
	ErrTooManyTemporaryColumns                      Error = 165
	ErrTooManyTemporaryNonConstColumns              Error = 166
	ErrTooDeepAst                                   Error = 167
	ErrTooBigAst                                    Error = 168
	ErrBadTypeOfField                               Error = 169
	ErrBadGet                                       Error = 170
	ErrBlocksHaveDifferentStructure                 Error = 171
	ErrCannotCreateDirectory                        Error = 172
	ErrCannotAllocateMemory                         Error = 173
	ErrCyclicAliases                                Error = 174
	ErrChunkNotFound                                Error = 176
	ErrDuplicateChunkName                           Error = 177
	ErrMultipleAliasesForExpression                 Error = 178
	ErrMultipleExpressionsForAlias                  Error = 179
	ErrThereIsNoProfile                             Error = 180
	ErrIllegalFinal                                 Error = 181
	ErrIllegalPrewhere                              Error = 182
	ErrUnexpectedExpression                         Error = 183
	ErrIllegalAggregation                           Error = 184
	ErrUnsupportedMyisamBlockType                   Error = 185
	ErrUnsupportedCollationLocale                   Error = 186
	ErrCollationComparisonFailed                    Error = 187
	ErrUnknownAction                                Error = 188
	ErrTableMustNotBeCreatedManually                Error = 189
	ErrSizesOfArraysDoesntMatch                     Error = 190
	ErrSetSizeLimitExceeded                         Error = 191
	ErrUnknownUser                                  Error = 192
	ErrWrongPassword                                Error = 193
	ErrRequiredPassword                             Error = 194
	ErrIPAddressNotAllowed                          Error = 195
	ErrUnknownAddressPatternType                    Error = 196
	ErrServerRevisionIsTooOld                       Error = 197
	ErrDNSError                                     Error = 198
	ErrUnknownQuota                                 Error = 199
	ErrQuotaDoesntAllowKeys                         Error = 200
	ErrQuotaExpired                                 Error = 201
	ErrTooManySimultaneousQueries                   Error = 202
	ErrNoFreeConnection                             Error = 203
	ErrCannotFsync                                  Error = 204
	ErrNestedTypeTooDeep                            Error = 205
	ErrAliasRequired                                Error = 206
	ErrAmbiguousIdentifier                          Error = 207
	ErrEmptyNestedTable                             Error = 208
	ErrSocketTimeout                                Error = 209
	ErrNetworkError                                 Error = 210
	ErrEmptyQuery                                   Error = 211
	ErrUnknownLoadBalancing                         Error = 212
	ErrUnknownTotalsMode                            Error = 213
	ErrCannotStatvfs                                Error = 214
	ErrNotAnAggregate                               Error = 215
	ErrQueryWithSameIDIsAlreadyRunning              Error = 216
	ErrClientHasConnectedToWrongPort                Error = 217
	ErrTableIsDropped                               Error = 218
	ErrDatabaseNotEmpty                             Error = 219
	ErrDuplicateInterserverIoEndpoint               Error = 220
	ErrNoSuchInterserverIoEndpoint                  Error = 221
	ErrAddingReplicaToNonEmptyTable                 Error = 222
	ErrUnexpectedAstStructure                       Error = 223
	ErrReplicaIsAlreadyActive                       Error = 224
	ErrNoZookeeper                                  Error = 225
	ErrNoFileInDataPart                             Error = 226
	ErrUnexpectedFileInDataPart                     Error = 227
	ErrBadSizeOfFileInDataPart                      Error = 228
	ErrQueryIsTooLarge                              Error = 229
	ErrNotFoundExpectedDataPart                     Error = 230
	ErrTooManyUnexpectedDataParts                   Error = 231
	ErrNoSuchDataPart                               Error = 232
	ErrBadDataPartName                              Error = 233
	ErrNoReplicaHasPart                             Error = 234
	ErrDuplicateDataPart                            Error = 235
	ErrAborted                                      Error = 236
	ErrNoReplicaNameGiven                           Error = 237
	ErrFormatVersionTooOld                          Error = 238
	ErrCannotMunmap                                 Error = 239
	ErrCannotMremap                                 Error = 240
	ErrMemoryLimitExceeded                          Error = 241
	ErrTableIsReadOnly                              Error = 242
	ErrNotEnoughSpace                               Error = 243
	ErrUnexpectedZookeeperError                     Error = 244
	ErrCorruptedData                                Error = 246
	ErrIncorrectMark                                Error = 247
	ErrInvalidPartitionValue                        Error = 248
	ErrNotEnoughBlockNumbers                        Error = 250
	ErrNoSuchReplica                                Error = 251
	ErrTooManyParts                                 Error = 252
	ErrReplicaIsAlreadyExist                        Error = 253
	ErrNoActiveReplicas                             Error = 254
	ErrTooManyRetriesToFetchParts                   Error = 255
	ErrPartitionAlreadyExists                       Error = 256
	ErrPartitionDoesntExist                         Error = 257
	ErrUnionAllResultStructuresMismatch             Error = 258
	ErrClientOutputFormatSpecified                  Error = 260
	ErrUnknownBlockInfoField                        Error = 261
	ErrBadCollation                                 Error = 262
	ErrCannotCompileCode                            Error = 263
	ErrIncompatibleTypeOfJoin                       Error = 264
	ErrNoAvailableReplica                           Error = 265
	ErrMismatchReplicasDataSources                  Error = 266
	ErrStorageDoesntSupportParallelReplicas         Error = 267
	ErrCPUIDError                                   Error = 268
	ErrInfiniteLoop                                 Error = 269
	ErrCannotCompress                               Error = 270
	ErrCannotDecompress                             Error = 271
	ErrAioSubmitError                               Error = 272
	ErrAioCompletionError                           Error = 273
	ErrAioReadError                                 Error = 274
	ErrAioWriteError                                Error = 275
	ErrIndexNotUsed                                 Error = 277
	ErrLeadershipLost                               Error = 278
	ErrAllConnectionTriesFailed                     Error = 279
	ErrNoAvailableData                              Error = 280
	ErrDictionaryIsEmpty                            Error = 281
	ErrIncorrectIndex                               Error = 282
	ErrUnknownDistributedProductMode                Error = 283
	ErrUnknownGlobalSubqueriesMethod                Error = 284
	ErrTooLessLiveReplicas                          Error = 285
	ErrUnsatisfiedQuorumForPreviousWrite            Error = 286
	ErrUnknownFormatVersion                         Error = 287
	ErrDistributedInJoinSubqueryDenied              Error = 288
	ErrReplicaIsNotInQuorum                         Error = 289
	ErrLimitExceeded                                Error = 290
	ErrDatabaseAccessDenied                         Error = 291
	ErrLeadershipChanged                            Error = 292
	ErrMongodbCannotAuthenticate                    Error = 293
	ErrInvalidBlockExtraInfo                        Error = 294
	ErrReceivedEmptyData                            Error = 295
	ErrNoRemoteShardFound                           Error = 296
	ErrShardHasNoConnections                        Error = 297
	ErrCannotPipe                                   Error = 298
	ErrCannotFork                                   Error = 299
	ErrCannotDlsym                                  Error = 300
	ErrCannotCreateChildProcess                     Error = 301
	ErrChildWasNotExitedNormally                    Error = 302
	ErrCannotSelect                                 Error = 303
	ErrCannotWaitpid                                Error = 304
	ErrTableWasNotDropped                           Error = 305
	ErrTooDeepRecursion                             Error = 306
	ErrTooManyBytes                                 Error = 307
	ErrUnexpectedNodeInZookeeper                    Error = 308
	ErrFunctionCannotHaveParameters                 Error = 309
	ErrInvalidShardWeight                           Error = 317
	ErrInvalidConfigParameter                       Error = 318
	ErrUnknownStatusOfInsert                        Error = 319
	ErrValueIsOutOfRangeOfDataType                  Error = 321
	ErrBarrierTimeout                               Error = 335
	ErrUnknownDatabaseEngine                        Error = 336
	ErrDdlGuardIsActive                             Error = 337
	ErrUnfinished                                   Error = 341
	ErrMetadataMismatch                             Error = 342
	ErrSupportIsDisabled                            Error = 344
	ErrTableDiffersTooMuch                          Error = 345
	ErrCannotConvertCharset                         Error = 346
	ErrCannotLoadConfig                             Error = 347
	ErrCannotInsertNullInOrdinaryColumn             Error = 349
	ErrIncompatibleSourceTables                     Error = 350
	ErrAmbiguousTableName                           Error = 351
	ErrAmbiguousColumnName                          Error = 352
	ErrIndexOfPositionalArgumentIsOutOfRange        Error = 353
	ErrZlibInflateFailed                            Error = 354
	ErrZlibDeflateFailed                            Error = 355
	ErrBadLambda                                    Error = 356
	ErrReservedIdentifierName                       Error = 357
	ErrIntoOutfileNotAllowed                        Error = 358
	ErrTableSizeExceedsMaxDropSizeLimit             Error = 359
	ErrCannotCreateCharsetConverter                 Error = 360
	ErrSeekPositionOutOfBound                       Error = 361
	ErrCurrentWriteBufferIsExhausted                Error = 362
	ErrCannotCreateIoBuffer                         Error = 363
	ErrReceivedErrorTooManyRequests                 Error = 364
	ErrOutputIsNotSorted                            Error = 365
	ErrSizesOfNestedColumnsAreInconsistent          Error = 366
	ErrTooManyFetches                               Error = 367
	ErrBadCast                                      Error = 368
	ErrAllReplicasAreStale                          Error = 369
	ErrDataTypeCannotBeUsedInTables                 Error = 370
	ErrInconsistentClusterDefinition                Error = 371
	ErrSessionNotFound                              Error = 372
	ErrSessionIsLocked                              Error = 373
	ErrInvalidSessionTimeout                        Error = 374
	ErrCannotDlopen                                 Error = 375
	ErrCannotParseUUID                              Error = 376
	ErrIllegalSyntaxForDataType                     Error = 377
	ErrDataTypeCannotHaveArguments                  Error = 378
	ErrUnknownStatusOfDistributedDdlTask            Error = 379
	ErrCannotKill                                   Error = 380
	ErrHTTPLengthRequired                           Error = 381
	ErrCannotLoadCatboostModel                      Error = 382
	ErrCannotApplyCatboostModel                     Error = 383
	ErrPartIsTemporarilyLocked                      Error = 384
	ErrMultipleStreamsRequired                      Error = 385
	ErrNoCommonType                                 Error = 386
	ErrExternalLoadableAlreadyExists                Error = 387
	ErrCannotAssignOptimize                         Error = 388
	ErrInsertWasDeduplicated                        Error = 389
	ErrCannotGetCreateTableQuery                    Error = 390
	ErrExternalLibraryError                         Error = 391
	ErrQueryIsProhibited                            Error = 392
	ErrThereIsNoQuery                               Error = 393
	ErrQueryWasCancelled                            Error = 394
	ErrFunctionThrowIfValueIsNonZero                Error = 395
	ErrTooManyRowsOrBytes                           Error = 396
	ErrQueryIsNotSupportedInMaterializedView        Error = 397
	ErrCannotParseDomainValueFromString             Error = 441
	ErrKeeperException                              Error = 999
	ErrPocoException                                Error = 1000
	ErrStdException                                 Error = 1001
	ErrUnknownException                             Error = 1002
	ErrConditionalTreeParentNotFound                Error = 2001
	ErrIllegalProjectionManipulator                 Error = 2002

	ErrAuthenticationFailed Error = 516
)

Subset of possible errors.

func ErrorString

func ErrorString(s string) (Error, error)

ErrorString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func ErrorValues

func ErrorValues() []Error

ErrorValues returns all values of the enum

func (Error) Error

func (e Error) Error() string

func (Error) IsAError

func (i Error) IsAError() bool

IsAError returns "true" if the value is listed in the enum definition. "false" otherwise

func (Error) String

func (i Error) String() string

type Exception

type Exception struct {
	Code    Error
	Name    string
	Message string
	Stack   string
	Nested  bool
}

Exception is server-side error.

func (*Exception) DecodeAware

func (e *Exception) DecodeAware(r *Reader, _ int) error

DecodeAware decodes exception.

func (*Exception) EncodeAware

func (e *Exception) EncodeAware(b *Buffer, _ int)

EncodeAware encodes exception.

type Feature

type Feature int

Feature represents server side feature.

const (
	FeatureBlockInfo                   Feature = 51903
	FeatureTimezone                    Feature = 54058
	FeatureQuotaKeyInClientInfo        Feature = 54060
	FeatureDisplayName                 Feature = 54372
	FeatureVersionPatch                Feature = 54401
	FeatureTempTables                  Feature = 50264
	FeatureServerLogs                  Feature = 54406
	FeatureColumnDefaultsMetadata      Feature = 54410
	FeatureClientWriteInfo             Feature = 54420
	FeatureSettingsSerializedAsStrings Feature = 54429
	FeatureInterServerSecret           Feature = 54441
	FeatureOpenTelemetry               Feature = 54442
	FeatureXForwardedForInClientInfo   Feature = 54443
	FeatureRefererInClientInfo         Feature = 54447
	FeatureDistributedDepth            Feature = 54448
	FeatureQueryStartTime              Feature = 54449
	FeatureProfileEvents               Feature = 54451
	FeatureParallelReplicas            Feature = 54453
	FeatureCustomSerialization         Feature = 54454
	FeatureQuotaKey                    Feature = 54458
	FeatureAddendum                    Feature = 54458
	FeatureParameters                  Feature = 54459
	FeatureServerQueryTimeInProgress   Feature = 54460
)

Possible features.

func FeatureString

func FeatureString(s string) (Feature, error)

FeatureString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func FeatureValues

func FeatureValues() []Feature

FeatureValues returns all values of the enum

func (Feature) In

func (f Feature) In(v int) bool

In reports whether feature is implemented in provided protocol version.

func (Feature) IsAFeature

func (i Feature) IsAFeature() bool

IsAFeature returns "true" if the value is listed in the enum definition. "false" otherwise

func (Feature) String

func (i Feature) String() string

func (Feature) Version

func (f Feature) Version() int

Version reports protocol version when Feature was introduced.

type IPv4

type IPv4 uint32

IPv4 represents IPv4 address as uint32 number.

Not using netip.Addr because uint32 is 5 times faster, consumes 6 times less memory and better represents IPv4.

Use ToIP helper for convenience.

func ToIPv4

func ToIPv4(ip netip.Addr) IPv4

ToIPv4 represents ip as IPv4. Panics if ip is not ipv4.

func (IPv4) String

func (v IPv4) String() string

func (IPv4) ToIP

func (v IPv4) ToIP() netip.Addr

ToIP represents IPv4 as netaddr.IP.

type IPv6

type IPv6 [16]byte

IPv6 represents IPv6 address.

Same as FixedString(16) internally in ClickHouse.

func ToIPv6

func ToIPv6(ip netip.Addr) IPv6

ToIPv6 represents ip as IPv6.

func (IPv6) String

func (v IPv6) String() string

func (IPv6) ToIP

func (v IPv6) ToIP() netip.Addr

ToIP represents IPv6 as netip.IP.

type Inferable added in v0.41.0

type Inferable interface {
	Infer(t ColumnType) error
}

Inferable can be inferenced from type.

type Input

type Input []InputColumn

Input of query.

func (Input) Columns

func (i Input) Columns() string

Columns returns "(foo, bar, baz)" formatted list of Input column names.

func (Input) Into

func (i Input) Into(table string) string

Into returns INSERT INTO table (c0, c..., cn) VALUES query.

func (Input) Reset added in v0.47.2

func (i Input) Reset()

Reset all columns that implement proto.Resettable.

type InputColumn

type InputColumn struct {
	Name string
	Data ColInput
}

func (InputColumn) EncodeStart

func (c InputColumn) EncodeStart(buf *Buffer, version int)

type Int128

type Int128 struct {
	Low  uint64 // first 64 bits
	High uint64 // last 64 bits
}

Int128 represents Int128 type.

func Int128FromInt

func Int128FromInt(v int) Int128

Int128FromInt creates new Int128 from int.

func Int128FromUInt64

func Int128FromUInt64(v uint64) Int128

Int128FromUInt64 creates new Int128 from uint64.

func (Int128) Int

func (i Int128) Int() int

Int value of Int128.

Returns math.MaxInt if High is set.

func (Int128) UInt64

func (i Int128) UInt64() uint64

UInt64 value of Int128.

type Int256

type Int256 struct {
	Low  UInt128 // first 128 bits
	High UInt128 // last 128 bits
}

Int256 is 256-bit signed integer.

func Int256FromInt

func Int256FromInt(v int) Int256

Int256FromInt creates new Int256 from int.

type Interface

type Interface byte

Interface is interface of client.

const (
	InterfaceTCP  Interface = 1
	InterfaceHTTP Interface = 2
)

Possible interfaces.

func InterfaceString

func InterfaceString(s string) (Interface, error)

InterfaceString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func InterfaceValues

func InterfaceValues() []Interface

InterfaceValues returns all values of the enum

func (Interface) IsAInterface

func (i Interface) IsAInterface() bool

IsAInterface returns "true" if the value is listed in the enum definition. "false" otherwise

func (Interface) String

func (i Interface) String() string

type Interval added in v0.45.0

type Interval struct {
	Scale IntervalScale
	Value int64
}

func (Interval) Add added in v0.45.0

func (i Interval) Add(t time.Time) time.Time

Add Interval to time.Time.

func (Interval) String added in v0.45.0

func (i Interval) String() string

type IntervalScale added in v0.45.0

type IntervalScale byte
const (
	IntervalSecond IntervalScale = iota
	IntervalMinute
	IntervalHour
	IntervalDay
	IntervalWeek
	IntervalMonth
	IntervalQuarter
	IntervalYear
)

func IntervalScaleString added in v0.45.0

func IntervalScaleString(s string) (IntervalScale, error)

IntervalScaleString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func IntervalScaleValues added in v0.45.0

func IntervalScaleValues() []IntervalScale

IntervalScaleValues returns all values of the enum

func (IntervalScale) IsAIntervalScale added in v0.45.0

func (i IntervalScale) IsAIntervalScale() bool

IsAIntervalScale returns "true" if the value is listed in the enum definition. "false" otherwise

func (IntervalScale) String added in v0.45.0

func (i IntervalScale) String() string

type KV added in v0.57.0

type KV[K comparable, V any] struct {
	Key   K
	Value V
}

KV is a key-value pair.

type Log

type Log struct {
	QueryID  string    `json:"query_id"`
	Source   string    `json:"source"`
	Text     string    `json:"text"`
	Time     time.Time `json:"event_time"`
	Host     string    `json:"host_name"`
	ThreadID uint64    `json:"thread_id"`
	Priority int8      `json:"priority"`
}

Log from server.

type Logs

type Logs struct {
	Time      ColDateTime
	TimeMicro ColUInt32
	HostName  ColStr
	QueryID   ColStr
	ThreadID  ColUInt64
	Priority  ColInt8
	Source    ColStr
	Text      ColStr
}

Logs from ServerCodeLog packet.

func (Logs) All

func (s Logs) All() []Log

func (*Logs) Result

func (s *Logs) Result() Results

type Nothing added in v0.45.0

type Nothing struct{}

Nothing represents NULL value.

type Nullable

type Nullable[T any] struct {
	Set   bool
	Value T
}

Nullable is T value that can be null.

func NewNullable

func NewNullable[T any](v T) Nullable[T]

NewNullable returns set value of Nullable[T] to v.

func Null

func Null[T any]() Nullable[T]

Null returns null value for Nullable[T].

func (Nullable[T]) IsSet

func (n Nullable[T]) IsSet() bool

func (Nullable[T]) Or

func (n Nullable[T]) Or(v T) T

type Parameter added in v0.48.0

type Parameter struct {
	Key   string
	Value string
}

func (*Parameter) Decode added in v0.48.0

func (p *Parameter) Decode(r *Reader) error

func (Parameter) Encode added in v0.48.0

func (p Parameter) Encode(b *Buffer)

type Point

type Point struct {
	X, Y float64
}

type Position

type Position struct {
	Start int
	End   int
}

type Precision

type Precision byte

Precision of DateTime64.

Tick size (precision): 10^(-precision) seconds. Valid range: [0:9].

const (
	// PrecisionSecond is one second precision.
	PrecisionSecond Precision = 0
	// PrecisionMilli is millisecond precision.
	PrecisionMilli Precision = 3
	// PrecisionMicro is microsecond precision.
	PrecisionMicro Precision = 6
	// PrecisionNano is nanosecond precision.
	PrecisionNano Precision = 9

	// PrecisionMax is maximum precision (nanosecond).
	PrecisionMax = PrecisionNano
)

func (Precision) Duration

func (p Precision) Duration() time.Duration

Duration returns duration of single tick for precision.

func (Precision) Scale added in v0.44.0

func (p Precision) Scale() int64

func (Precision) Valid

func (p Precision) Valid() bool

Valid reports whether precision is valid.

type Preparable

type Preparable interface {
	Prepare() error
}

Preparable should be prepared before encoding or decoding.

type Profile

type Profile struct {
	Rows                      uint64
	Blocks                    uint64
	Bytes                     uint64
	AppliedLimit              bool
	RowsBeforeLimit           uint64
	CalculatedRowsBeforeLimit bool
}

func (*Profile) DecodeAware

func (p *Profile) DecodeAware(r *Reader, _ int) error

func (Profile) EncodeAware

func (p Profile) EncodeAware(b *Buffer, _ int)

type ProfileEvent

type ProfileEvent struct {
	Type     ProfileEventType `json:"type"`
	Name     string           `json:"name"`
	Value    int64            `json:"value"`
	Host     string           `json:"host_name"`
	Time     time.Time        `json:"current_time"`
	ThreadID uint64           `json:"thread_id"`
}

ProfileEvent is detailed profiling event from Server.

type ProfileEventType

type ProfileEventType byte
const (
	ProfileIncrement ProfileEventType = 1
	ProfileGauge     ProfileEventType = 2
)

func ProfileEventTypeString

func ProfileEventTypeString(s string) (ProfileEventType, error)

ProfileEventTypeString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func ProfileEventTypeValues

func ProfileEventTypeValues() []ProfileEventType

ProfileEventTypeValues returns all values of the enum

func (ProfileEventType) IsAProfileEventType

func (i ProfileEventType) IsAProfileEventType() bool

IsAProfileEventType returns "true" if the value is listed in the enum definition. "false" otherwise

func (ProfileEventType) MarshalJSON added in v0.59.0

func (i ProfileEventType) MarshalJSON() ([]byte, error)

MarshalJSON implements the json.Marshaler interface for ProfileEventType

func (ProfileEventType) MarshalText added in v0.59.0

func (i ProfileEventType) MarshalText() ([]byte, error)

MarshalText implements the encoding.TextMarshaler interface for ProfileEventType

func (ProfileEventType) String

func (i ProfileEventType) String() string

func (*ProfileEventType) UnmarshalJSON added in v0.59.0

func (i *ProfileEventType) UnmarshalJSON(data []byte) error

UnmarshalJSON implements the json.Unmarshaler interface for ProfileEventType

func (*ProfileEventType) UnmarshalText added in v0.59.0

func (i *ProfileEventType) UnmarshalText(text []byte) error

UnmarshalText implements the encoding.TextUnmarshaler interface for ProfileEventType

type ProfileEvents

type ProfileEvents struct {
	Host     ColStr
	Time     ColDateTime
	ThreadID ColUInt64
	Type     ColInt8
	Name     ColStr
	Value    ColAuto // UInt64 or Int64 depending on version
}

ProfileEvents is data of ServerProfileEvents packet.

func (*ProfileEvents) All

func (d *ProfileEvents) All() ([]ProfileEvent, error)

func (*ProfileEvents) Result

func (d *ProfileEvents) Result() Results

type Progress

type Progress struct {
	Rows      uint64
	Bytes     uint64
	TotalRows uint64

	WroteRows  uint64
	WroteBytes uint64
	ElapsedNs  uint64
}

Progress of query execution.

func (*Progress) DecodeAware

func (p *Progress) DecodeAware(r *Reader, version int) error

func (Progress) EncodeAware

func (p Progress) EncodeAware(b *Buffer, version int)

type Query

type Query struct {
	ID          string
	Body        string
	Secret      string
	Stage       Stage
	Compression Compression
	Info        ClientInfo
	Settings    []Setting
	Parameters  []Parameter
}

func (*Query) DecodeAware

func (q *Query) DecodeAware(r *Reader, version int) error

func (Query) EncodeAware

func (q Query) EncodeAware(b *Buffer, version int)

type Reader

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

Reader implements ClickHouse protocol decoding from buffered reader. Not goroutine-safe.

func NewReader

func NewReader(r io.Reader) *Reader

NewReader initializes new Reader from provided io.Reader.

func (*Reader) Bool

func (r *Reader) Bool() (bool, error)

Bool decodes bool as uint8.

func (*Reader) Byte

func (r *Reader) Byte() (byte, error)

Byte decodes byte value.

func (*Reader) Decode

func (r *Reader) Decode(v Decoder) error

Decode value.

func (*Reader) DisableCompression

func (r *Reader) DisableCompression()

DisableCompression makes next read use raw source of data.

func (*Reader) EnableCompression

func (r *Reader) EnableCompression()

EnableCompression makes next reads use decompressed source of data.

func (*Reader) Float32

func (r *Reader) Float32() (float32, error)

Float32 decodes float32 value.

func (*Reader) Float64

func (r *Reader) Float64() (float64, error)

Float64 decodes float64 value.

func (*Reader) Int

func (r *Reader) Int() (int, error)

Int decodes uvarint as int.

func (*Reader) Int128

func (r *Reader) Int128() (Int128, error)

Int128 decodes Int128 value.

func (*Reader) Int16

func (r *Reader) Int16() (int16, error)

Int16 decodes int16 value.

func (*Reader) Int32

func (r *Reader) Int32() (int32, error)

Int32 decodes int32 value.

func (*Reader) Int64

func (r *Reader) Int64() (int64, error)

Int64 decodes int64 value.

func (*Reader) Int8

func (r *Reader) Int8() (int8, error)

Int8 decodes int8 value.

func (*Reader) Read

func (r *Reader) Read(p []byte) (n int, err error)

func (*Reader) ReadByte

func (r *Reader) ReadByte() (byte, error)

func (*Reader) ReadFull

func (r *Reader) ReadFull(buf []byte) error

func (*Reader) ReadRaw

func (r *Reader) ReadRaw(n int) ([]byte, error)

ReadRaw reads raw n bytes.

func (*Reader) Str

func (r *Reader) Str() (string, error)

Str decodes string.

func (*Reader) StrAppend

func (r *Reader) StrAppend(buf []byte) ([]byte, error)

StrAppend decodes string and appends it to provided buf.

func (*Reader) StrBytes

func (r *Reader) StrBytes() ([]byte, error)

StrBytes decodes string and allocates new byte slice with result.

func (*Reader) StrLen

func (r *Reader) StrLen() (int, error)

func (*Reader) StrRaw

func (r *Reader) StrRaw() ([]byte, error)

StrRaw decodes string to internal buffer and returns it directly.

Do not retain returned slice.

func (*Reader) UInt128

func (r *Reader) UInt128() (UInt128, error)

UInt128 decodes UInt128 value.

func (*Reader) UInt16

func (r *Reader) UInt16() (uint16, error)

UInt16 decodes uint16 value.

func (*Reader) UInt32

func (r *Reader) UInt32() (uint32, error)

UInt32 decodes uint32 value.

func (*Reader) UInt64

func (r *Reader) UInt64() (uint64, error)

UInt64 decodes uint64 value.

func (*Reader) UInt8

func (r *Reader) UInt8() (uint8, error)

UInt8 decodes uint8 value.

func (*Reader) UVarInt

func (r *Reader) UVarInt() (uint64, error)

UVarInt reads uint64 from internal reader.

type Resettable added in v0.47.2

type Resettable interface {
	Reset()
}

type Result

type Result interface {
	DecodeResult(r *Reader, version int, b Block) error
}

Result of Query.

type ResultColumn

type ResultColumn struct {
	Name string    // Name of column. Inferred if not provided.
	Data ColResult // Data of column, required.
}

ResultColumn can be uses as part of Results or as single Result.

func AutoResult

func AutoResult(name string) ResultColumn

AutoResult is ResultColumn with type inference.

func (ResultColumn) DecodeResult

func (c ResultColumn) DecodeResult(r *Reader, version int, b Block) error

DecodeResult implements Result as "single result" helper.

type Results

type Results []ResultColumn

Results wrap []ResultColumn to implement Result.

func (*Results) Auto

func (s *Results) Auto() Result

func (Results) DecodeResult

func (s Results) DecodeResult(r *Reader, version int, b Block) error

func (Results) Rows

func (s Results) Rows() int

type ServerCode

type ServerCode byte

ServerCode is sent by server to client.

const (
	ServerCodeHello        ServerCode = 0  // Server part of "handshake"
	ServerCodeData         ServerCode = 1  // data block (can be compressed)
	ServerCodeException    ServerCode = 2  // runtime exception
	ServerCodeProgress     ServerCode = 3  // query execution progress (bytes, lines)
	ServerCodePong         ServerCode = 4  // ping response (ClientPing)
	ServerCodeEndOfStream  ServerCode = 5  // all packets were transmitted
	ServerCodeProfile      ServerCode = 6  // profiling info
	ServerCodeTotals       ServerCode = 7  // packet with total values (can be compressed)
	ServerCodeExtremes     ServerCode = 8  // packet with minimums and maximums (can be compressed)
	ServerCodeTablesStatus ServerCode = 9  // response to TablesStatus
	ServerCodeLog          ServerCode = 10 // query execution system log
	ServerCodeTableColumns ServerCode = 11 // columns description
	ServerPartUUIDs        ServerCode = 12 // list of unique parts ids.
	ServerReadTaskRequest  ServerCode = 13 // String (UUID) describes a request for which next task is needed
	ServerProfileEvents    ServerCode = 14 // Packet with profile events from server
)

Possible server codes.

func ServerCodeString

func ServerCodeString(s string) (ServerCode, error)

ServerCodeString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func ServerCodeValues

func ServerCodeValues() []ServerCode

ServerCodeValues returns all values of the enum

func (ServerCode) Compressible

func (c ServerCode) Compressible() bool

Compressible reports whether message can be compressed.

func (ServerCode) Encode

func (c ServerCode) Encode(b *Buffer)

Encode to buffer.

func (ServerCode) IsAServerCode

func (i ServerCode) IsAServerCode() bool

IsAServerCode returns "true" if the value is listed in the enum definition. "false" otherwise

func (ServerCode) String

func (i ServerCode) String() string

type ServerHello

type ServerHello struct {
	Name        string
	Major       int
	Minor       int
	Revision    int
	Timezone    string
	DisplayName string
	Patch       int
}

ServerHello is answer to ClientHello and represents ServerCodeHello message.

func (*ServerHello) DecodeAware

func (s *ServerHello) DecodeAware(r *Reader, v int) error

DecodeAware decodes ServerHello message from Reader.

func (*ServerHello) EncodeAware

func (s *ServerHello) EncodeAware(b *Buffer, v int)

func (ServerHello) Features

func (s ServerHello) Features() []Feature

Features implemented by server.

func (ServerHello) Has

func (s ServerHello) Has(f Feature) bool

Has reports whether Feature is implemented.

func (ServerHello) String

func (s ServerHello) String() string

type Setting

type Setting struct {
	Key   string
	Value string

	Important bool
	Custom    bool
	Obsolete  bool
}

func (*Setting) Decode

func (s *Setting) Decode(r *Reader) error

func (Setting) Encode

func (s Setting) Encode(b *Buffer)

type Stage

type Stage byte

Stage of query till SELECT should be executed.

const (
	StageFetchColumns       Stage = 0
	StageWithMergeableState Stage = 1
	StageComplete           Stage = 2
)

StageComplete is query complete.

func StageString

func StageString(s string) (Stage, error)

StageString retrieves an enum value from the enum constants string name. Throws an error if the param is not part of the enum.

func StageValues

func StageValues() []Stage

StageValues returns all values of the enum

func (Stage) Encode

func (s Stage) Encode(b *Buffer)

Encode to buffer.

func (Stage) IsAStage

func (i Stage) IsAStage() bool

IsAStage returns "true" if the value is listed in the enum definition. "false" otherwise

func (Stage) String

func (i Stage) String() string

type StateDecoder

type StateDecoder interface {
	DecodeState(r *Reader) error
}

type StateEncoder

type StateEncoder interface {
	EncodeState(b *Buffer)
}

type Stateful added in v0.41.0

type Stateful interface {
	StateEncoder
	StateDecoder
}

type TableColumns

type TableColumns struct {
	First  string
	Second string
}

func (*TableColumns) DecodeAware

func (c *TableColumns) DecodeAware(r *Reader, _ int) error

func (TableColumns) EncodeAware

func (c TableColumns) EncodeAware(b *Buffer, _ int)

type UInt128

type UInt128 struct {
	Low  uint64 // first 64 bits
	High uint64 // last 64 bits
}

UInt128 represents UInt128 type.

func UInt128FromInt

func UInt128FromInt(v int) UInt128

UInt128FromInt creates new UInt128 from int.

func UInt128FromUInt64

func UInt128FromUInt64(v uint64) UInt128

UInt128FromUInt64 creates new UInt128 from uint64.

func (UInt128) Int

func (i UInt128) Int() int

Int returns Int value of UInt128.

func (UInt128) UInt64

func (i UInt128) UInt64() uint64

UInt64 returns UInt64 value of UInt128.

type UInt256

type UInt256 struct {
	Low  UInt128 // first 128 bits
	High UInt128 // last 128 bits
}

UInt256 is 256-bit unsigned integer.

func UInt256FromInt

func UInt256FromInt(v int) UInt256

UInt256FromInt creates new UInt256 from int.

func UInt256FromUInt64

func UInt256FromUInt64(v uint64) UInt256

UInt256FromUInt64 creates new UInt256 from uint64.

Source Files

Directories

Path Synopsis
cmd

Jump to

Keyboard shortcuts

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