proto

package
v0.69.0 Latest Latest
Warning

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

Go to latest
Published: Oct 4, 2025 License: Apache-2.0 Imports: 20 Imported by: 105

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.

View Source
const JSONStringSerializationVersion uint64 = 1

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

func (Block) WriteBlock added in v0.63.0

func (b Block) WriteBlock(w *Writer, version int, input []InputColumn) error

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
	ClientCodeSSHChallengeRequest  ClientCode = 11 // request for SSH signature challenge
	ClientCodeSSHChallengeResponse ClientCode = 12 // response with SSH signature
)

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 NewArrTime32 added in v0.67.0

func NewArrTime32() *ColArr[Time32]

func NewArrTime64 added in v0.67.0

func NewArrTime64() *ColArr[Time64]

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]) RowLen added in v0.68.0

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

RowLen returns i-th row array length.

func (ColArr[T]) RowRange added in v0.63.0

func (c ColArr[T]) RowRange(i int) iter.Seq[T]

RowRange returns a iter.Seq iterator over i-th row.

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).

func (ColArr[T]) WriteColumn added in v0.63.0

func (c ColArr[T]) WriteColumn(w *Writer)

WriteColumn implements ColInput.

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

func (ColAuto) WriteColumn added in v0.63.0

func (c ColAuto) WriteColumn(w *Writer)

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.

func (ColBool) WriteColumn added in v0.63.0

func (c ColBool) WriteColumn(w *Writer)

WriteColumn writes Bool rows to *Writer.

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.

func (ColDate) WriteColumn added in v0.63.0

func (c ColDate) WriteColumn(w *Writer)

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.

func (ColDate32) WriteColumn added in v0.63.0

func (c ColDate32) WriteColumn(w *Writer)

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) AppendRaw added in v0.64.0

func (c *ColDateTime) AppendRaw(v DateTime)

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

func (ColDateTime) WriteColumn added in v0.63.0

func (c ColDateTime) WriteColumn(w *Writer)

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) Nullable added in v0.41.0

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

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

func (ColDateTime64) WriteColumn added in v0.63.0

func (c ColDateTime64) WriteColumn(w *Writer)

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.

func (ColDecimal128) WriteColumn added in v0.63.0

func (c ColDecimal128) WriteColumn(w *Writer)

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.

func (ColDecimal256) WriteColumn added in v0.63.0

func (c ColDecimal256) WriteColumn(w *Writer)

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.

func (ColDecimal32) WriteColumn added in v0.63.0

func (c ColDecimal32) WriteColumn(w *Writer)

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.

func (ColDecimal64) WriteColumn added in v0.63.0

func (c ColDecimal64) WriteColumn(w *Writer)

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

func (*ColEnum) WriteColumn added in v0.63.0

func (e *ColEnum) WriteColumn(w *Writer)

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.

func (ColEnum16) WriteColumn added in v0.63.0

func (c ColEnum16) WriteColumn(w *Writer)

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.

func (ColEnum8) WriteColumn added in v0.63.0

func (c ColEnum8) WriteColumn(w *Writer)

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.

func (ColFixedStr) WriteColumn added in v0.63.0

func (c ColFixedStr) WriteColumn(w *Writer)

WriteColumn writes ColFixedStr rows to *Writer.

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.

func (ColFixedStr128) WriteColumn added in v0.63.0

func (c ColFixedStr128) WriteColumn(w *Writer)

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.

func (ColFixedStr16) WriteColumn added in v0.63.0

func (c ColFixedStr16) WriteColumn(w *Writer)

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.

func (ColFixedStr256) WriteColumn added in v0.63.0

func (c ColFixedStr256) WriteColumn(w *Writer)

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.

func (ColFixedStr32) WriteColumn added in v0.63.0

func (c ColFixedStr32) WriteColumn(w *Writer)

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.

func (ColFixedStr512) WriteColumn added in v0.63.0

func (c ColFixedStr512) WriteColumn(w *Writer)

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.

func (ColFixedStr64) WriteColumn added in v0.63.0

func (c ColFixedStr64) WriteColumn(w *Writer)

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.

func (ColFixedStr8) WriteColumn added in v0.63.0

func (c ColFixedStr8) WriteColumn(w *Writer)

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.

func (ColFloat32) WriteColumn added in v0.63.0

func (c ColFloat32) WriteColumn(w *Writer)

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.

func (ColFloat64) WriteColumn added in v0.63.0

func (c ColFloat64) WriteColumn(w *Writer)

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.

func (ColIPv4) WriteColumn added in v0.63.0

func (c ColIPv4) WriteColumn(w *Writer)

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.

func (ColIPv6) WriteColumn added in v0.63.0

func (c ColIPv6) WriteColumn(w *Writer)

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)
	WriteColumn(w *Writer)
}

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.

func (ColInt128) WriteColumn added in v0.63.0

func (c ColInt128) WriteColumn(w *Writer)

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.

func (ColInt16) WriteColumn added in v0.63.0

func (c ColInt16) WriteColumn(w *Writer)

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.

func (ColInt256) WriteColumn added in v0.63.0

func (c ColInt256) WriteColumn(w *Writer)

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.

func (ColInt32) WriteColumn added in v0.63.0

func (c ColInt32) WriteColumn(w *Writer)

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.

func (ColInt64) WriteColumn added in v0.63.0

func (c ColInt64) WriteColumn(w *Writer)

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.

func (ColInt8) WriteColumn added in v0.63.0

func (c ColInt8) WriteColumn(w *Writer)

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

func (ColInterval) WriteColumn added in v0.63.0

func (c ColInterval) WriteColumn(w *Writer)

type ColJSONBytes added in v0.64.0

type ColJSONBytes struct {
	ColJSONStr
}

ColJSONBytes is ColJSONStr wrapper to be ColumnOf for []byte.

func (*ColJSONBytes) Append added in v0.64.0

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

Append byte slice to column.

func (*ColJSONBytes) AppendArr added in v0.64.0

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

AppendArr append slice of byte slices to column.

func (*ColJSONBytes) Array added in v0.64.0

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

Array is helper that creates Array(JSON).

func (*ColJSONBytes) Nullable added in v0.64.0

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

Nullable is helper that creates Nullable(JSON).

func (ColJSONBytes) Row added in v0.64.0

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

Row returns row with number i.

type ColJSONStr added in v0.64.0

type ColJSONStr struct {
	Str ColStr
}

ColJSONStr represents String column.

Use ColJSONBytes for []bytes ColumnOf implementation.

func (*ColJSONStr) Append added in v0.64.0

func (c *ColJSONStr) Append(v string)

Append string to column.

func (*ColJSONStr) AppendArr added in v0.64.0

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

func (*ColJSONStr) AppendBytes added in v0.64.0

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

AppendBytes append byte slice as string to column.

func (*ColJSONStr) Array added in v0.64.0

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

Array is helper that creates Array(JSON).

func (*ColJSONStr) DecodeColumn added in v0.64.0

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

DecodeColumn decodes String rows from *Reader.

func (*ColJSONStr) DecodeState added in v0.65.1

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

DecodeState decodes the JSON serialization version

func (ColJSONStr) EncodeColumn added in v0.64.0

func (c ColJSONStr) EncodeColumn(b *Buffer)

EncodeColumn encodes String rows to *Buffer.

func (*ColJSONStr) EncodeState added in v0.65.1

func (c *ColJSONStr) EncodeState(b *Buffer)

EncodeState encodes the JSON serialization version

func (ColJSONStr) First added in v0.64.0

func (c ColJSONStr) First() string

First returns the first row of the column.

func (ColJSONStr) ForEach added in v0.64.0

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

ForEach calls f on each string from column.

func (ColJSONStr) ForEachBytes added in v0.64.0

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

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

func (*ColJSONStr) LowCardinality added in v0.64.0

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

LowCardinality returns LowCardinality(JSON).

func (*ColJSONStr) Nullable added in v0.64.0

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

Nullable is helper that creates Nullable(JSON).

func (*ColJSONStr) Reset added in v0.64.0

func (c *ColJSONStr) Reset()

Reset resets data in row, preserving capacity for efficiency.

func (ColJSONStr) Row added in v0.64.0

func (c ColJSONStr) Row(i int) string

Row returns row with number i.

func (ColJSONStr) RowBytes added in v0.64.0

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

RowBytes returns row with number i as byte slice.

func (ColJSONStr) Rows added in v0.64.0

func (c ColJSONStr) Rows() int

Rows returns count of rows in column.

func (ColJSONStr) Type added in v0.64.0

func (ColJSONStr) Type() ColumnType

Type returns ColumnType of JSON.

func (ColJSONStr) WriteColumn added in v0.64.0

func (c ColJSONStr) WriteColumn(w *Writer)

WriteColumn writes JSON rows to *Writer.

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

func (*ColLowCardinality[T]) WriteColumn added in v0.63.0

func (c *ColLowCardinality[T]) WriteColumn(w *Writer)

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

func (ColLowCardinalityRaw) WriteColumn added in v0.63.0

func (c ColLowCardinalityRaw) WriteColumn(w *Writer)

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]) RowRange added in v0.63.0

func (c ColMap[K, V]) RowRange(i int) iter.Seq2[K, V]

RowRange returns a iter.Seq2 iterator over i-th 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

func (ColMap[K, V]) WriteColumn added in v0.63.0

func (c ColMap[K, V]) WriteColumn(w *Writer)

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

func (ColNothing) WriteColumn added in v0.63.0

func (c ColNothing) WriteColumn(w *Writer)

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]) Array added in v0.63.0

func (c *ColNullable[T]) Array() *ColArr[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

func (ColNullable[T]) WriteColumn added in v0.63.0

func (c ColNullable[T]) WriteColumn(w *Writer)

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

func (ColPoint) WriteColumn added in v0.63.0

func (c ColPoint) WriteColumn(w *Writer)

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.

func (ColRawOf[X]) WriteColumn added in v0.63.0

func (c ColRawOf[X]) WriteColumn(w *Writer)

WriteColumn write ColRawOf rows to *Writer.

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.

func (ColStr) WriteColumn added in v0.63.0

func (c ColStr) WriteColumn(w *Writer)

WriteColumn writes String rows to *Writer.

type ColTime added in v0.67.0

type ColTime = ColTime32

ColTime is an alias for ColTime32

type ColTime32 added in v0.67.0

type ColTime32 struct {
	Data         []Time32
	Precision    Precision
	PrecisionSet bool
}

ColTime32 implements ColumnOf[Time32].

func (*ColTime32) Append added in v0.67.0

func (c *ColTime32) Append(v Time32)

func (*ColTime32) AppendArr added in v0.67.0

func (c *ColTime32) AppendArr(vs []Time32)

func (*ColTime32) Array added in v0.67.0

func (c *ColTime32) Array() *ColArr[Time32]

func (*ColTime32) DecodeColumn added in v0.67.0

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

func (ColTime32) EncodeColumn added in v0.67.0

func (c ColTime32) EncodeColumn(b *Buffer)

EncodeColumn encodes Time32 rows to *Buffer.

func (*ColTime32) Infer added in v0.67.0

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

func (*ColTime32) LowCardinality added in v0.67.0

func (c *ColTime32) LowCardinality() *ColLowCardinality[Time32]

func (*ColTime32) Nullable added in v0.67.0

func (c *ColTime32) Nullable() *ColNullable[Time32]

func (*ColTime32) Reset added in v0.67.0

func (c *ColTime32) Reset()

func (ColTime32) Row added in v0.67.0

func (c ColTime32) Row(i int) Time32

func (ColTime32) Rows added in v0.67.0

func (c ColTime32) Rows() int

func (ColTime32) Type added in v0.67.0

func (c ColTime32) Type() ColumnType

func (*ColTime32) WithPrecision added in v0.67.0

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

func (ColTime32) WriteColumn added in v0.67.0

func (c ColTime32) WriteColumn(w *Writer)

type ColTime64 added in v0.67.0

type ColTime64 struct {
	Data         []Time64
	Precision    Precision
	PrecisionSet bool
}

ColTime64 implements ColumnOf[Time64].

func (*ColTime64) Append added in v0.67.0

func (c *ColTime64) Append(v Time64)

func (*ColTime64) AppendArr added in v0.67.0

func (c *ColTime64) AppendArr(vs []Time64)

func (*ColTime64) Array added in v0.67.0

func (c *ColTime64) Array() *ColArr[Time64]

func (*ColTime64) DecodeColumn added in v0.67.0

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

func (ColTime64) EncodeColumn added in v0.67.0

func (c ColTime64) EncodeColumn(b *Buffer)

EncodeColumn encodes Time64 rows to *Buffer.

func (*ColTime64) Infer added in v0.67.0

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

func (*ColTime64) LowCardinality added in v0.67.0

func (c *ColTime64) LowCardinality() *ColLowCardinality[Time64]

func (*ColTime64) Nullable added in v0.67.0

func (c *ColTime64) Nullable() *ColNullable[Time64]

func (*ColTime64) Reset added in v0.67.0

func (c *ColTime64) Reset()

func (ColTime64) Row added in v0.67.0

func (c ColTime64) Row(i int) Time64

func (ColTime64) Rows added in v0.67.0

func (c ColTime64) Rows() int

func (ColTime64) Type added in v0.67.0

func (c ColTime64) Type() ColumnType

func (*ColTime64) WithPrecision added in v0.67.0

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

func (ColTime64) WriteColumn added in v0.67.0

func (c ColTime64) WriteColumn(w *Writer)

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

func (ColTuple) WriteColumn added in v0.63.0

func (c ColTuple) WriteColumn(w *Writer)

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.

func (ColUInt128) WriteColumn added in v0.63.0

func (c ColUInt128) WriteColumn(w *Writer)

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.

func (ColUInt16) WriteColumn added in v0.63.0

func (c ColUInt16) WriteColumn(w *Writer)

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.

func (ColUInt256) WriteColumn added in v0.63.0

func (c ColUInt256) WriteColumn(w *Writer)

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.

func (ColUInt32) WriteColumn added in v0.63.0

func (c ColUInt32) WriteColumn(w *Writer)

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.

func (ColUInt64) WriteColumn added in v0.63.0

func (c ColUInt64) WriteColumn(w *Writer)

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.

func (ColUInt8) WriteColumn added in v0.63.0

func (c ColUInt8) WriteColumn(w *Writer)

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

func (ColUUID) WriteColumn added in v0.63.0

func (c ColUUID) WriteColumn(w *Writer)

WriteColumn encodes ColUUID rows to *Writer.

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"
	ColumnTypeTime32         ColumnType = "Time32"
	ColumnTypeTime64         ColumnType = "Time64"
	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"
	ColumnTypeDecimal        ColumnType = "Decimal"
	ColumnTypeDecimal32      ColumnType = "Decimal32"
	ColumnTypeDecimal64      ColumnType = "Decimal64"
	ColumnTypeDecimal128     ColumnType = "Decimal128"
	ColumnTypeDecimal256     ColumnType = "Decimal256"
	ColumnTypePoint          ColumnType = "Point"
	ColumnTypeInterval       ColumnType = "Interval"
	ColumnTypeNothing        ColumnType = "Nothing"
	ColumnTypeJSON           ColumnType = "JSON"
)

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 // Deprecated: Error removed from ClickHouse
	ErrCannotInsertElementIntoConstantColumn               Error = 18 // Deprecated: Error removed from ClickHouse
	ErrSizeOfFixedStringDoesntMatch                        Error = 19
	ErrNumberOfColumnsDoesntMatch                          Error = 20
	ErrCannotReadAllDataFromTabSeparatedInput              Error = 21 // Deprecated: Error removed from ClickHouse
	ErrCannotParseAllValueFromTabSeparatedInput            Error = 22 // Deprecated: Error removed from ClickHouse
	ErrCannotReadFromIstream                               Error = 23
	ErrCannotWriteToOstream                                Error = 24
	ErrCannotParseEscapeSequence                           Error = 25
	ErrCannotParseQuotedString                             Error = 26
	ErrCannotParseInputAssertionFailed                     Error = 27
	ErrCannotPrintFloatOrDoubleNumber                      Error = 28
	ErrCannotPrintInteger                                  Error = 29 // Deprecated: Error removed from ClickHouse
	ErrCannotReadSizeOfCompressedChunk                     Error = 30 // Deprecated: Error removed from ClickHouse
	ErrCannotReadCompressedChunk                           Error = 31 // Deprecated: Error removed from ClickHouse
	ErrAttemptToReadAfterEOF                               Error = 32
	ErrCannotReadAllData                                   Error = 33
	ErrTooManyArgumentsForFunction                         Error = 34
	ErrTooFewArgumentsForFunction                          Error = 35
	ErrTooLessArgumentsForFunction                         Error = 35 // Deprecated: Use ErrTooFewArgumentsForFunction instead
	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 // Deprecated: Error removed from ClickHouse
	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 // Deprecated: Error removed from ClickHouse
	ErrStorageRequiresParameter                            Error = 55
	ErrUnknownStorage                                      Error = 56
	ErrTableAlreadyExists                                  Error = 57
	ErrTableMetadataAlreadyExists                          Error = 58
	ErrIllegalTypeOfColumnForFilter                        Error = 59
	ErrUnknownTable                                        Error = 60
	ErrOnlyFilterColumnInBlock                             Error = 61 // Deprecated: Error removed from ClickHouse
	ErrSyntaxError                                         Error = 62
	ErrUnknownAggregateFunction                            Error = 63
	ErrCannotReadAggregateFunctionFromText                 Error = 64 // Deprecated: Error removed from ClickHouse
	ErrCannotWriteAggregateFunctionAsText                  Error = 65 // Deprecated: Error removed from ClickHouse
	ErrNotAColumn                                          Error = 66 // Deprecated: Error removed from ClickHouse
	ErrIllegalKeyOfAggregation                             Error = 67 // Deprecated: Error removed from ClickHouse
	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 // Deprecated: Use ErrReceivedErrorFromRemoteIOServer instead
	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 // Deprecated: Error removed from ClickHouse
	ErrCannotWriteToEmptyBlockOutputStream                 Error = 98 // Deprecated: Error removed from ClickHouse
	ErrUnknownPacketFromClient                             Error = 99
	ErrUnknownPacketFromServer                             Error = 100
	ErrUnexpectedPacketFromClient                          Error = 101
	ErrUnexpectedPacketFromServer                          Error = 102
	ErrReceivedDataForWrongQueryID                         Error = 103 // Deprecated: Error removed from ClickHouse
	ErrTooSmallBufferSize                                  Error = 104
	ErrCannotReadHistory                                   Error = 105 // Deprecated: Error removed from ClickHouse
	ErrCannotAppendHistory                                 Error = 106 // Deprecated: Error removed from ClickHouse
	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
	ErrUnsupportedJoinKeys                                 Error = 121
	ErrUnknownSetDataVariant                               Error = 121 // Deprecated: Use ErrUnsupportedJoinKeys instead
	ErrIncompatibleColumns                                 Error = 122
	ErrUnknownTypeOfAstNode                                Error = 123
	ErrIncorrectElementOfSet                               Error = 124
	ErrIncorrectResultOfScalarSubquery                     Error = 125
	ErrCannotGetReturnType                                 Error = 126 // Deprecated: Error removed from ClickHouse
	ErrIllegalIndex                                        Error = 127
	ErrTooLargeArraySize                                   Error = 128
	ErrFunctionIsSpecial                                   Error = 129
	ErrCannotReadArrayFromText                             Error = 130
	ErrTooLargeStringSize                                  Error = 131
	ErrCannotCreateTableFromMetadata                       Error = 132 // Deprecated: Error removed from ClickHouse
	ErrAggregateFunctionDoesntAllowParameters              Error = 133
	ErrParametersToAggregateFunctionsMustBeLiterals        Error = 134
	ErrZeroArrayOrTupleIndex                               Error = 135
	ErrUnknownElementInConfig                              Error = 137
	ErrExcessiveElementInConfig                            Error = 138
	ErrNoElementsInConfig                                  Error = 139
	ErrAllRequestedColumnsAreMissing                       Error = 140 // Deprecated: Error removed from ClickHouse
	ErrSamplingNotSupported                                Error = 141
	ErrNotFoundNode                                        Error = 142
	ErrFoundMoreThanOneNode                                Error = 143 // Deprecated: Error removed from ClickHouse
	ErrFirstDateIsBiggerThanLastDate                       Error = 144 // Deprecated: Error removed from ClickHouse
	ErrUnknownOverflowMode                                 Error = 145
	ErrQuerySectionDoesntMakeSense                         Error = 146 // Deprecated: Error removed from ClickHouse
	ErrNotFoundFunctionElementForAggregate                 Error = 147 // Deprecated: Error removed from ClickHouse
	ErrNotFoundRelationElementForCondition                 Error = 148 // Deprecated: Error removed from ClickHouse
	ErrNotFoundRHSElementForCondition                      Error = 149 // Deprecated: Error removed from ClickHouse
	ErrNoAttributesListed                                  Error = 150 // Deprecated: Error removed from ClickHouse
	ErrIndexOfColumnInSortClauseIsOutOfRange               Error = 151 // Deprecated: Error removed from ClickHouse
	ErrUnknownDirectionOfSorting                           Error = 152
	ErrIllegalDivision                                     Error = 153
	ErrAggregateFunctionNotApplicable                      Error = 154 // Deprecated: Error removed from ClickHouse
	ErrUnknownRelation                                     Error = 155 // Deprecated: Error removed from ClickHouse
	ErrDictionariesWasNotLoaded                            Error = 156
	ErrIllegalOverflowMode                                 Error = 157 // Deprecated: Error removed from ClickHouse
	ErrTooManyRows                                         Error = 158
	ErrTimeoutExceeded                                     Error = 159
	ErrTooSlow                                             Error = 160
	ErrTooManyColumns                                      Error = 161
	ErrTooDeepSubqueries                                   Error = 162
	ErrTooDeepPipeline                                     Error = 163 // Deprecated: Error removed from ClickHouse
	ErrReadonly                                            Error = 164
	ErrTooManyTemporaryColumns                             Error = 165
	ErrTooManyTemporaryNonConstColumns                     Error = 166
	ErrTooDeepAst                                          Error = 167
	ErrTooBigAst                                           Error = 168
	ErrBadTypeOfField                                      Error = 169
	ErrBadGet                                              Error = 170
	ErrBlocksHaveDifferentStructure                        Error = 171 // Deprecated: Error removed from ClickHouse
	ErrCannotCreateDirectory                               Error = 172
	ErrCannotAllocateMemory                                Error = 173
	ErrCyclicAliases                                       Error = 174
	ErrChunkNotFound                                       Error = 176 // Deprecated: Error removed from ClickHouse
	ErrDuplicateChunkName                                  Error = 177 // Deprecated: Error removed from ClickHouse
	ErrMultipleAliasesForExpression                        Error = 178 // Deprecated: Error removed from ClickHouse
	ErrMultipleExpressionsForAlias                         Error = 179
	ErrThereIsNoProfile                                    Error = 180
	ErrIllegalFinal                                        Error = 181
	ErrIllegalPrewhere                                     Error = 182
	ErrUnexpectedExpression                                Error = 183
	ErrIllegalAggregation                                  Error = 184
	ErrUnsupportedMyisamBlockType                          Error = 185 // Deprecated: Error removed from ClickHouse
	ErrUnsupportedCollationLocale                          Error = 186
	ErrCollationComparisonFailed                           Error = 187
	ErrUnknownAction                                       Error = 188 // Deprecated: Error removed from ClickHouse
	ErrTableMustNotBeCreatedManually                       Error = 189 // Deprecated: Error removed from ClickHouse
	ErrSizesOfArraysDontMatch                              Error = 190
	ErrSizesOfArraysDoesntMatch                            Error = 190 // Deprecated: Use ErrSizesOfArraysDontMatch instead
	ErrSetSizeLimitExceeded                                Error = 191
	ErrUnknownUser                                         Error = 192
	ErrWrongPassword                                       Error = 193
	ErrRequiredPassword                                    Error = 194
	ErrIPAddressNotAllowed                                 Error = 195
	ErrUnknownAddressPatternType                           Error = 196
	ErrServerRevisionIsTooOld                              Error = 197 // Deprecated: Error removed from ClickHouse
	ErrDNSError                                            Error = 198
	ErrUnknownQuota                                        Error = 199
	ErrQuotaDoesntAllowKeys                                Error = 200 // Deprecated: Error removed from ClickHouse
	ErrQuotaExpired                                        Error = 201 // Deprecated: Use ErrQuotaExceeded instead
	ErrQuotaExceeded                                       Error = 201
	ErrTooManySimultaneousQueries                          Error = 202
	ErrNoFreeConnection                                    Error = 203
	ErrCannotFsync                                         Error = 204
	ErrNestedTypeTooDeep                                   Error = 205 // Deprecated: Error removed from ClickHouse
	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
	ErrDuplicateInterserverIoEndpoint                      Error = 220 // Deprecated: Use ErrDuplicateInterserverIOEndpoint instead
	ErrNoSuchInterserverIoEndpoint                         Error = 221 // Deprecated: Use ErrNoSuchInterserverIOEndpoint instead
	ErrNoSuchInterserverIOEndpoint                         Error = 221
	ErrAddingReplicaToNonEmptyTable                        Error = 222 // Deprecated: Error removed from ClickHouse
	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 // Deprecated: Error removed from ClickHouse
	ErrInvalidPartitionValue                               Error = 248
	ErrNotEnoughBlockNumbers                               Error = 250 // Deprecated: Error removed from ClickHouse
	ErrNoSuchReplica                                       Error = 251
	ErrTooManyParts                                        Error = 252
	ErrReplicaIsAlreadyExist                               Error = 253 // Deprecated: Use ErrReplicaAlreadyExists instead
	ErrReplicaAlreadyExists                                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 // Deprecated: Error removed from ClickHouse
	ErrCPUIDError                                          Error = 268 // Deprecated: Error removed from ClickHouse
	ErrInfiniteLoop                                        Error = 269
	ErrCannotCompress                                      Error = 270
	ErrCannotDecompress                                    Error = 271
	ErrCannotIOSubmit                                      Error = 272
	ErrAioSubmitError                                      Error = 272 // Deprecated: Use ErrCannotIOSubmit instead
	ErrCannotIOGetevents                                   Error = 273
	ErrAioCompletionError                                  Error = 273 // Deprecated: Use ErrCannotIOGetevents instead
	ErrAioReadError                                        Error = 274
	ErrAioWriteError                                       Error = 275
	ErrIndexNotUsed                                        Error = 277
	ErrLeadershipLost                                      Error = 278 // Deprecated: Error removed from ClickHouse
	ErrAllConnectionTriesFailed                            Error = 279
	ErrNoAvailableData                                     Error = 280
	ErrDictionaryIsEmpty                                   Error = 281
	ErrIncorrectIndex                                      Error = 282
	ErrUnknownDistributedProductMode                       Error = 283
	ErrWrongGlobalSubquery                                 Error = 284
	ErrUnknownGlobalSubqueriesMethod                       Error = 284 // Deprecated: Use ErrWrongGlobalSubquery instead
	ErrTooFewLiveReplicas                                  Error = 285
	ErrTooLessLiveReplicas                                 Error = 285 // Deprecated: Use ErrTooFewLiveReplicas instead
	ErrUnsatisfiedQuorumForPreviousWrite                   Error = 286
	ErrUnknownFormatVersion                                Error = 287
	ErrDistributedInJoinSubqueryDenied                     Error = 288
	ErrReplicaIsNotInQuorum                                Error = 289
	ErrLimitExceeded                                       Error = 290
	ErrDatabaseAccessDenied                                Error = 291
	ErrLeadershipChanged                                   Error = 292 // Deprecated: Error removed from ClickHouse
	ErrMongodbCannotAuthenticate                           Error = 293
	ErrCannotWriteToFile                                   Error = 294
	ErrInvalidBlockExtraInfo                               Error = 294 // Deprecated: Use ErrCannotWriteToFile instead
	ErrReceivedEmptyData                                   Error = 295
	ErrNoRemoteShardFound                                  Error = 296 // Deprecated: Error removed from ClickHouse
	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 // Deprecated: Error removed from ClickHouse
	ErrInvalidConfigParameter                              Error = 318
	ErrUnknownStatusOfInsert                               Error = 319
	ErrValueIsOutOfRangeOfDataType                         Error = 321
	ErrBarrierTimeout                                      Error = 335 // Deprecated: Error removed from ClickHouse
	ErrUnknownDatabaseEngine                               Error = 336
	ErrDdlGuardIsActive                                    Error = 337 // Deprecated: Error removed from ClickHouse
	ErrUnfinished                                          Error = 341
	ErrMetadataMismatch                                    Error = 342
	ErrSupportIsDisabled                                   Error = 344
	ErrTableDiffersTooMuch                                 Error = 345
	ErrCannotConvertCharset                                Error = 346
	ErrCannotLoadConfig                                    Error = 347
	ErrCannotInsertNullInOrdinaryColumn                    Error = 349
	ErrIncompatibleSourceTables                            Error = 350 // Deprecated: Error removed from ClickHouse
	ErrAmbiguousTableName                                  Error = 351 // Deprecated: Error removed from ClickHouse
	ErrAmbiguousColumnName                                 Error = 352
	ErrIndexOfPositionalArgumentIsOutOfRange               Error = 353
	ErrZlibInflateFailed                                   Error = 354
	ErrZlibDeflateFailed                                   Error = 355
	ErrBadLambda                                           Error = 356 // Deprecated: Error removed from ClickHouse
	ErrReservedIdentifierName                              Error = 357 // Deprecated: Error removed from ClickHouse
	ErrIntoOutfileNotAllowed                               Error = 358
	ErrTableSizeExceedsMaxDropSizeLimit                    Error = 359
	ErrCannotCreateCharsetConverter                        Error = 360
	ErrSeekPositionOutOfBound                              Error = 361
	ErrCurrentWriteBufferIsExhausted                       Error = 362
	ErrCannotCreateIoBuffer                                Error = 363 // Deprecated: Use ErrCannotCreateIOBuffer instead
	ErrCannotCreateIOBuffer                                Error = 363
	ErrReceivedErrorTooManyRequests                        Error = 364
	ErrOutputIsNotSorted                                   Error = 365 // Deprecated: Error removed from ClickHouse
	ErrSizesOfNestedColumnsAreInconsistent                 Error = 366
	ErrTooManyFetches                                      Error = 367 // Deprecated: Error removed from ClickHouse
	ErrBadCast                                             Error = 368 // Deprecated: Error removed from ClickHouse
	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 // Deprecated: Error removed from ClickHouse
	ErrCannotKill                                          Error = 380
	ErrHTTPLengthRequired                                  Error = 381
	ErrCannotLoadCatboostModel                             Error = 382
	ErrCannotApplyCatboostModel                            Error = 383
	ErrPartIsTemporarilyLocked                             Error = 384
	ErrMultipleStreamsRequired                             Error = 385
	ErrNoCommonType                                        Error = 386
	ErrExternalLoadableAlreadyExists                       Error = 387 // Deprecated: Use ErrDictionaryAlreadyExists instead
	ErrDictionaryAlreadyExists                             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
	ErrUnknownMutationCommand                              Error = 398
	ErrFormatIsNotSuitableForOutput                        Error = 399
	ErrCannotStat                                          Error = 400
	ErrFeatureIsNotEnabledAtBuildTime                      Error = 401
	ErrCannotIosetup                                       Error = 402
	ErrInvalidJoinOnExpression                             Error = 403
	ErrBadOdbcConnectionString                             Error = 404
	ErrTopAndLimitTogether                                 Error = 406
	ErrDecimalOverflow                                     Error = 407
	ErrBadRequestParameter                                 Error = 408
	ErrExternalServerIsNotResponding                       Error = 410
	ErrPthreadError                                        Error = 411
	ErrNetlinkError                                        Error = 412
	ErrCannotSetSignalHandler                              Error = 413
	ErrAllReplicasLost                                     Error = 415
	ErrReplicaStatusChanged                                Error = 416
	ErrExpectedAllOrAny                                    Error = 417
	ErrUnknownJoin                                         Error = 418
	ErrMultipleAssignmentsToColumn                         Error = 419
	ErrCannotUpdateColumn                                  Error = 420
	ErrCannotAddDifferentAggregateStates                   Error = 421
	ErrUnsupportedURIScheme                                Error = 422
	ErrCannotGettimeofday                                  Error = 423
	ErrCannotLink                                          Error = 424
	ErrSystemError                                         Error = 425
	ErrCannotCompileRegexp                                 Error = 427
	ErrFailedToGetpwuid                                    Error = 429
	ErrMismatchingUsersForProcessAndData                   Error = 430
	ErrIllegalSyntaxForCodecType                           Error = 431
	ErrUnknownCodec                                        Error = 432
	ErrIllegalCodecParameter                               Error = 433
	ErrCannotParseProtobufSchema                           Error = 434
	ErrNoColumnSerializedToRequiredProtobufField           Error = 435
	ErrProtobufBadCast                                     Error = 436
	ErrProtobufFieldNotRepeated                            Error = 437
	ErrDataTypeCannotBePromoted                            Error = 438
	ErrCannotScheduleTask                                  Error = 439
	ErrInvalidLimitExpression                              Error = 440
	ErrCannotParseDomainValueFromString                    Error = 441
	ErrBadDatabaseForTemporaryTable                        Error = 442
	ErrNoColumnsSerializedToProtobufFields                 Error = 443
	ErrUnknownProtobufFormat                               Error = 444
	ErrCannotMprotect                                      Error = 445
	ErrFunctionNotAllowed                                  Error = 446
	ErrHyperscanCannotScanText                             Error = 447
	ErrBrotliReadFailed                                    Error = 448
	ErrBrotliWriteFailed                                   Error = 449
	ErrBadTTLExpression                                    Error = 450
	ErrBadTTLFile                                          Error = 451
	ErrSettingConstraintViolation                          Error = 452
	ErrMysqlClientInsufficientCapabilities                 Error = 453
	ErrOpensslError                                        Error = 454
	ErrSuspiciousTypeForLowCardinality                     Error = 455
	ErrUnknownQueryParameter                               Error = 456
	ErrBadQueryParameter                                   Error = 457
	ErrCannotUnlink                                        Error = 458
	ErrCannotSetThreadPriority                             Error = 459
	ErrCannotCreateTimer                                   Error = 460
	ErrCannotSetTimerPeriod                                Error = 461
	ErrCannotFcntl                                         Error = 463
	ErrCannotParseElf                                      Error = 464
	ErrCannotParseDwarf                                    Error = 465
	ErrInsecurePath                                        Error = 466
	ErrCannotParseBool                                     Error = 467
	ErrCannotPthreadAttr                                   Error = 468
	ErrViolatedConstraint                                  Error = 469
	ErrInvalidSettingValue                                 Error = 471
	ErrReadonlySetting                                     Error = 472
	ErrDeadlockAvoided                                     Error = 473
	ErrInvalidTemplateFormat                               Error = 474
	ErrInvalidWithFillExpression                           Error = 475
	ErrWithTiesWithoutOrderBy                              Error = 476
	ErrInvalidUsageOfInput                                 Error = 477
	ErrUnknownPolicy                                       Error = 478
	ErrUnknownDisk                                         Error = 479
	ErrUnknownProtocol                                     Error = 480
	ErrPathAccessDenied                                    Error = 481
	ErrDictionaryAccessDenied                              Error = 482
	ErrTooManyRedirects                                    Error = 483
	ErrInternalRedisError                                  Error = 484
	ErrCannotGetCreateDictionaryQuery                      Error = 487
	ErrIncorrectDictionaryDefinition                       Error = 489
	ErrCannotFormatDatetime                                Error = 490
	ErrUnacceptableURL                                     Error = 491
	ErrAccessEntityNotFound                                Error = 492
	ErrAccessEntityAlreadyExists                           Error = 493
	ErrAccessStorageReadonly                               Error = 495
	ErrQuotaRequiresClientKey                              Error = 496
	ErrAccessDenied                                        Error = 497
	ErrLimitByWithTiesIsNotSupported                       Error = 498
	ErrS3Error                                             Error = 499
	ErrAzureBlobStorageError                               Error = 500
	ErrCannotCreateDatabase                                Error = 501
	ErrCannotSigqueue                                      Error = 502
	ErrAggregateFunctionThrow                              Error = 503
	ErrFileAlreadyExists                                   Error = 504
	ErrUnableToSkipUnusedShards                            Error = 507
	ErrUnknownAccessType                                   Error = 508
	ErrInvalidGrant                                        Error = 509
	ErrCacheDictionaryUpdateFail                           Error = 510
	ErrUnknownRole                                         Error = 511
	ErrSetNonGrantedRole                                   Error = 512
	ErrUnknownPartType                                     Error = 513
	ErrAccessStorageForInsertionNotFound                   Error = 514
	ErrIncorrectAccessEntityDefinition                     Error = 515
	ErrAuthenticationFailed                                Error = 516
	ErrCannotAssignAlter                                   Error = 517
	ErrCannotCommitOffset                                  Error = 518
	ErrNoRemoteShardAvailable                              Error = 519
	ErrCannotDetachDictionaryAsTable                       Error = 520
	ErrAtomicRenameFail                                    Error = 521
	ErrUnknownRowPolicy                                    Error = 523
	ErrAlterOfColumnIsForbidden                            Error = 524
	ErrIncorrectDiskIndex                                  Error = 525
	ErrNoSuitableFunctionImplementation                    Error = 527
	ErrCassandraInternalError                              Error = 528
	ErrNotALeader                                          Error = 529
	ErrCannotConnectRabbitmq                               Error = 530
	ErrCannotFstat                                         Error = 531
	ErrLdapError                                           Error = 532
	ErrUnknownRaidType                                     Error = 535
	ErrCannotRestoreFromFieldDump                          Error = 536
	ErrIllegalMysqlVariable                                Error = 537
	ErrMysqlSyntaxError                                    Error = 538
	ErrCannotBindRabbitmqExchange                          Error = 539
	ErrCannotDeclareRabbitmqExchange                       Error = 540
	ErrCannotCreateRabbitmqQueueBinding                    Error = 541
	ErrCannotRemoveRabbitmqExchange                        Error = 542
	ErrUnknownMysqlDatatypesSupportLevel                   Error = 543
	ErrRowAndRowsTogether                                  Error = 544
	ErrFirstAndNextTogether                                Error = 545
	ErrNoRowDelimiter                                      Error = 546
	ErrInvalidRaidType                                     Error = 547
	ErrUnknownVolume                                       Error = 548
	ErrDataTypeCannotBeUsedInKey                           Error = 549
	ErrUnrecognizedArguments                               Error = 552
	ErrLzmaStreamEncoderFailed                             Error = 553
	ErrLzmaStreamDecoderFailed                             Error = 554
	ErrRocksdbError                                        Error = 555
	ErrSyncMysqlUserAccessError                            Error = 556
	ErrUnknownUnion                                        Error = 557
	ErrExpectedAllOrDistinct                               Error = 558
	ErrInvalidGrpcQueryInfo                                Error = 559
	ErrZstdEncoderFailed                                   Error = 560
	ErrZstdDecoderFailed                                   Error = 561
	ErrTldListNotFound                                     Error = 562
	ErrCannotReadMapFromText                               Error = 563
	ErrInterserverSchemeDoesntMatch                        Error = 564
	ErrTooManyPartitions                                   Error = 565
	ErrCannotRmdir                                         Error = 566
	ErrDuplicatedPartUuids                                 Error = 567
	ErrRaftError                                           Error = 568
	ErrMultipleColumnsSerializedToSameProtobufField        Error = 569
	ErrDataTypeIncompatibleWithProtobufField               Error = 570
	ErrDatabaseReplicationFailed                           Error = 571
	ErrTooManyQueryPlanOptimizations                       Error = 572
	ErrEpollError                                          Error = 573
	ErrDistributedTooManyPendingBytes                      Error = 574
	ErrUnknownSnapshot                                     Error = 575
	ErrKerberosError                                       Error = 576
	ErrInvalidShardID                                      Error = 577
	ErrInvalidFormatInsertQueryWithData                    Error = 578
	ErrIncorrectPartType                                   Error = 579
	ErrCannotSetRoundingMode                               Error = 580
	ErrTooLargeDistributedDepth                            Error = 581
	ErrNoSuchProjectionInTable                             Error = 582
	ErrIllegalProjection                                   Error = 583
	ErrProjectionNotUsed                                   Error = 584
	ErrCannotParseYaml                                     Error = 585
	ErrCannotCreateFile                                    Error = 586
	ErrConcurrentAccessNotSupported                        Error = 587
	ErrDistributedBrokenBatchInfo                          Error = 588
	ErrDistributedBrokenBatchFiles                         Error = 589
	ErrCannotSysconf                                       Error = 590
	ErrSqliteEngineError                                   Error = 591
	ErrDataEncryptionError                                 Error = 592
	ErrZeroCopyReplicationError                            Error = 593
	ErrBzip2StreamDecoderFailed                            Error = 594
	ErrBzip2StreamEncoderFailed                            Error = 595
	ErrIntersectOrExceptResultStructuresMismatch           Error = 596
	ErrNoSuchErrorCode                                     Error = 597
	ErrBackupAlreadyExists                                 Error = 598
	ErrBackupNotFound                                      Error = 599
	ErrBackupVersionNotSupported                           Error = 600
	ErrBackupDamaged                                       Error = 601
	ErrNoBaseBackup                                        Error = 602
	ErrWrongBaseBackup                                     Error = 603
	ErrBackupEntryAlreadyExists                            Error = 604
	ErrBackupEntryNotFound                                 Error = 605
	ErrBackupIsEmpty                                       Error = 606
	ErrCannotRestoreDatabase                               Error = 607
	ErrCannotRestoreTable                                  Error = 608
	ErrFunctionAlreadyExists                               Error = 609
	ErrCannotDropFunction                                  Error = 610
	ErrCannotCreateRecursiveFunction                       Error = 611
	ErrPostgresqlConnectionFailure                         Error = 614
	ErrCannotAdvise                                        Error = 615
	ErrUnknownReadMethod                                   Error = 616
	ErrLz4EncoderFailed                                    Error = 617
	ErrLz4DecoderFailed                                    Error = 618
	ErrPostgresqlReplicationInternalError                  Error = 619
	ErrQueryNotAllowed                                     Error = 620
	ErrCannotNormalizeString                               Error = 621
	ErrCannotParseCapnProtoSchema                          Error = 622
	ErrCapnProtoBadCast                                    Error = 623
	ErrBadFileType                                         Error = 624
	ErrIOSetupError                                        Error = 625
	ErrCannotSkipUnknownField                              Error = 626
	ErrBackupEngineNotFound                                Error = 627
	ErrOffsetFetchWithoutOrderBy                           Error = 628
	ErrHTTPRangeNotSatisfiable                             Error = 629
	ErrHaveDependentObjects                                Error = 630
	ErrUnknownFileSize                                     Error = 631
	ErrUnexpectedDataAfterParsedValue                      Error = 632
	ErrQueryIsNotSupportedInWindowView                     Error = 633
	ErrMongodbError                                        Error = 634
	ErrCannotPoll                                          Error = 635
	ErrCannotExtractTableStructure                         Error = 636
	ErrInvalidTableOverride                                Error = 637
	ErrSnappyUncompressFailed                              Error = 638
	ErrSnappyCompressFailed                                Error = 639
	ErrNoHivemetastore                                     Error = 640
	ErrCannotAppendToFile                                  Error = 641
	ErrCannotPackArchive                                   Error = 642
	ErrCannotUnpackArchive                                 Error = 643
	ErrNumberOfDimensionsMismatched                        Error = 645
	ErrCannotBackupTable                                   Error = 647
	ErrWrongDdlRenamingSettings                            Error = 648
	ErrInvalidTransaction                                  Error = 649
	ErrSerializationError                                  Error = 650
	ErrCapnProtoBadType                                    Error = 651
	ErrOnlyNullsWhileReadingSchema                         Error = 652
	ErrCannotParseBackupSettings                           Error = 653
	ErrWrongBackupSettings                                 Error = 654
	ErrFailedToSyncBackupOrRestore                         Error = 655
	ErrUnknownStatusOfTransaction                          Error = 659
	ErrHdfsError                                           Error = 660
	ErrCannotSendSignal                                    Error = 661
	ErrFsMetadataError                                     Error = 662
	ErrInconsistentMetadataForBackup                       Error = 663
	ErrAccessStorageDoesntAllowBackup                      Error = 664
	ErrCannotConnectNats                                   Error = 665
	ErrNotInitialized                                      Error = 667
	ErrInvalidState                                        Error = 668
	ErrNamedCollectionDoesntExist                          Error = 669
	ErrNamedCollectionAlreadyExists                        Error = 670
	ErrNamedCollectionIsImmutable                          Error = 671
	ErrInvalidSchedulerNode                                Error = 672
	ErrResourceAccessDenied                                Error = 673
	ErrResourceNotFound                                    Error = 674
	ErrCannotParseIpv4                                     Error = 675
	ErrCannotParseIpv6                                     Error = 676
	ErrThreadWasCanceled                                   Error = 677
	ErrIOUringInitFailed                                   Error = 678
	ErrIOUringSubmitError                                  Error = 679
	ErrMixedAccessParameterTypes                           Error = 690
	ErrUnknownElementOfEnum                                Error = 691
	ErrTooManyMutations                                    Error = 692
	ErrAwsError                                            Error = 693
	ErrAsyncLoadCycle                                      Error = 694
	ErrAsyncLoadFailed                                     Error = 695
	ErrAsyncLoadCanceled                                   Error = 696
	ErrCannotRestoreToNonencryptedDisk                     Error = 697
	ErrInvalidRedisStorageType                             Error = 698
	ErrInvalidRedisTableStructure                          Error = 699
	ErrUserSessionLimitExceeded                            Error = 700
	ErrClusterDoesntExist                                  Error = 701
	ErrClientInfoDoesNotMatch                              Error = 702
	ErrInvalidIdentifier                                   Error = 703
	ErrQueryCacheUsedWithNondeterministicFunctions         Error = 704
	ErrTableNotEmpty                                       Error = 705
	ErrLibsshError                                         Error = 706
	ErrGcpError                                            Error = 707
	ErrIllegalStatistics                                   Error = 708
	ErrCannotGetReplicatedDatabaseSnapshot                 Error = 709
	ErrFaultInjected                                       Error = 710
	ErrFilecacheAccessDenied                               Error = 711
	ErrTooManyMaterializedViews                            Error = 712
	ErrBrokenProjection                                    Error = 713
	ErrUnexpectedCluster                                   Error = 714
	ErrCannotDetectFormat                                  Error = 715
	ErrCannotForgetPartition                               Error = 716
	ErrExperimentalFeatureError                            Error = 717
	ErrTooSlowParsing                                      Error = 718
	ErrQueryCacheUsedWithSystemTable                       Error = 719
	ErrUserExpired                                         Error = 720
	ErrDeprecatedFunction                                  Error = 721
	ErrAsyncLoadWaitFailed                                 Error = 722
	ErrParquetException                                    Error = 723
	ErrTooManyTables                                       Error = 724
	ErrTooManyDatabases                                    Error = 725
	ErrUnexpectedHTTPHeaders                               Error = 726
	ErrUnexpectedTableEngine                               Error = 727
	ErrUnexpectedDataType                                  Error = 728
	ErrIllegalTimeSeriesTags                               Error = 729
	ErrRefreshFailed                                       Error = 730
	ErrQueryCacheUsedWithNonThrowOverflowMode              Error = 731
	ErrTableIsBeingRestarted                               Error = 733
	ErrCannotWriteAfterBufferCanceled                      Error = 734
	ErrQueryWasCancelledByClient                           Error = 735
	ErrDatalakeDatabaseError                               Error = 736
	ErrGoogleCloudError                                    Error = 737
	ErrPartIsLocked                                        Error = 738
	ErrBuzzhouse                                           Error = 739
	ErrPotentiallyBrokenDataPart                           Error = 740
	ErrTableUUIDMismatch                                   Error = 741
	ErrDeltaKernelError                                    Error = 742
	ErrIcebergSpecificationViolation                       Error = 743
	ErrSessionIDEmpty                                      Error = 744
	ErrServerOverloaded                                    Error = 745
	ErrDependenciesNotFound                                Error = 746
	ErrFilecacheCannotWriteThroughCacheWithConcurrentReads Error = 747
	ErrDistributedCacheError                               Error = 900
	ErrCannotUseDistributedCache                           Error = 901
	ErrProtocolVersionMismatch                             Error = 902
	ErrLicenseExpired                                      Error = 903
	ErrKeeperException                                     Error = 999
	ErrPocoException                                       Error = 1000
	ErrStdException                                        Error = 1001
	ErrUnknownException                                    Error = 1002
	ErrSSHException                                        Error = 1003
	ErrStartupScriptsError                                 Error = 1004
	ErrConditionalTreeParentNotFound                       Error = 2001 // Deprecated: Error removed from ClickHouse
	ErrIllegalProjectionManipulator                        Error = 2002 // Deprecated: Error removed from ClickHouse
)

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
	FeatureJSONStrings                 Feature = 54475
)

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 and Time64.

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
	ServerCodeSSHChallenge ServerCode = 18 // return challenge for SSH signature signing
)

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 Time32 added in v0.67.0

type Time32 int32

Time32 represents duration in seconds.

func IntoTime32 added in v0.69.0

func IntoTime32(t time.Duration) Time32

IntoTime32 converts time.Druation into Time32 up to seconds precision.

func ParseTime32 added in v0.67.0

func ParseTime32(s string) (Time32, error)

ParseTime32 parses string of form "12:34:56" to valid Time32 type.

func (Time32) Duration added in v0.69.0

func (t Time32) Duration() time.Duration

Duration converts Time32 into time.Duration up to seconds precision

func (Time32) String added in v0.67.0

func (t Time32) String() string

String implements formatting Time32 to string of form Hour:Minutes:Seconds.

type Time64 added in v0.67.0

type Time64 int64

Time64 represents duration up until nanoseconds.

func IntoTime64 added in v0.69.0

func IntoTime64(t time.Duration) Time64

IntoTime64 converts time.Duration to Time64 up to nanoseconds precision

func IntoTime64WithPrecision added in v0.69.0

func IntoTime64WithPrecision(d time.Duration, precision Precision) Time64

IntoTime64WithPrecision converts time.Duration to Time64 with specified precision

func ParseTime64 added in v0.67.0

func ParseTime64(s string) (Time64, error)

ParseTime64 parses string of form "12:34:56.789" to valid Time64 type.

func (Time64) Duration added in v0.69.0

func (t Time64) Duration() time.Duration

Duration converts Time64 into time.Duration up to nanoseconds precision

func (Time64) String added in v0.67.0

func (t Time64) String() string

String implements formatting Time64 to string of form Hour:Minutes:Seconds.NanoSeconds.

func (Time64) ToDurationWithPrecision added in v0.69.0

func (t Time64) ToDurationWithPrecision(precision Precision) time.Duration

ToDurationWithPrecision converts Time64 to time.Duration with specified precision up until PrecisionMax (nanoseconds)

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.

type Writer added in v0.63.0

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

Writer is a column writer.

It helps to reduce memory footprint by writing column using vector I/O.

func NewWriter added in v0.63.0

func NewWriter(conn io.Writer, buf *Buffer) *Writer

NewWriter creates new Writer.

func (*Writer) ChainBuffer added in v0.63.0

func (w *Writer) ChainBuffer(cb func(*Buffer))

ChainBuffer creates a temporary buffer and adds it to the vector to write later.

Data is not written immediately, call Writer.Flush to flush data.

NB: do not retain buffer.

func (*Writer) ChainWrite added in v0.63.0

func (w *Writer) ChainWrite(data []byte)

ChainWrite adds buffer to the vector to write later.

Passed byte slice may be captured until Writer.Flush is called.

func (*Writer) Flush added in v0.63.0

func (w *Writer) Flush() (n int64, err error)

Flush flushes all data to writer.

Source Files

Directories

Path Synopsis
cmd
ch-gen-col command

Jump to

Keyboard shortcuts

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