Giter VIP home page Giter VIP logo

ch-go's Introduction

ch

Low level TCP ClickHouse client and protocol implementation in Go. Designed for very fast data block streaming with low network, cpu and memory overhead.

NB: No pooling, reconnects and not goroutine-safe by default, only single connection. Use clickhouse-go for high-level database/sql-compatible client, pooling for ch-go is available as chpool package.

ClickHouse is an open-source, high performance columnar OLAP database management system for real-time analytics using SQL.

go get github.com/ClickHouse/ch-go@latest

Example

package main

import (
  "context"
  "fmt"

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

func main() {
  ctx := context.Background()
  c, err := ch.Dial(ctx, ch.Options{Address: "localhost:9000"})
  if err != nil {
    panic(err)
  }
  var (
    numbers int
    data    proto.ColUInt64
  )
  if err := c.Do(ctx, ch.Query{
    Body: "SELECT number FROM system.numbers LIMIT 500000000",
    Result: proto.Results{
      {Name: "number", Data: &data},
    },
    // OnResult will be called on next received data block.
    OnResult: func(ctx context.Context, b proto.Block) error {
      numbers += len(data)
      return nil
    },
  }); err != nil {
    panic(err)
  }
  fmt.Println("numbers:", numbers)
}
393ms 0.5B rows  4GB  10GB/s 1 job
874ms 2.0B rows 16GB  18GB/s 4 jobs

Results

To stream query results, set Result and OnResult fields of Query. The OnResult will be called after Result is filled with received data block.

The OnResult is optional, but query will fail if more than single block is received, so it is ok to solely set the Result if only one row is expected.

Automatic result inference

var result proto.Results
q := ch.Query{
  Body:   "SELECT * FROM table",
  Result: result.Auto(),
}

Single result with column name inference

var res proto.ColBool
q := ch.Query{
  Body:   "SELECT v FROM test_table",
  Result: proto.ResultColumn{Data: &res},
}

Writing data

See examples/insert.

For table

CREATE TABLE test_table_insert
(
    ts                DateTime64(9),
    severity_text     Enum8('INFO'=1, 'DEBUG'=2),
    severity_number   UInt8,
    body              String,
    name              String,
    arr               Array(String)
) ENGINE = Memory

We prepare data block for insertion as follows:

var (
	body      proto.ColStr
	name      proto.ColStr
	sevText   proto.ColEnum
	sevNumber proto.ColUInt8

	ts  = new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano) // DateTime64(9)
	arr = new(proto.ColStr).Array()                                   // Array(String)
	now = time.Date(2010, 1, 1, 10, 22, 33, 345678, time.UTC)
)

// Append 10 rows to initial data block.
for i := 0; i < 10; i++ {
	body.AppendBytes([]byte("Hello"))
	ts.Append(now)
	name.Append("name")
	sevText.Append("INFO")
	sevNumber.Append(10)
	arr.Append([]string{"foo", "bar", "baz"})
}

input := proto.Input{
	{Name: "ts", Data: ts},
	{Name: "severity_text", Data: &sevText},
	{Name: "severity_number", Data: sevNumber},
	{Name: "body", Data: body},
	{Name: "name", Data: name},
	{Name: "arr", Data: arr},
}

Single data block

if err := conn.Do(ctx, ch.Query{
	// Or "INSERT INTO test_table_insert (ts, severity_text, severity_number, body, name, arr) VALUES"
	// Or input.Into("test_table_insert")
	Body: "INSERT INTO test_table_insert VALUES",
	Input: input,
}); err != nil {
	panic(err)
}

Stream data

// Stream data to ClickHouse server in multiple data blocks.
var blocks int
if err := conn.Do(ctx, ch.Query{
	Body:  input.Into("test_table_insert"), // helper that generates INSERT INTO query with all columns
	Input: input,

	// OnInput is called to prepare Input data before encoding and sending
	// to ClickHouse server.
	OnInput: func(ctx context.Context) error {
		// On OnInput call, you should fill the input data.
		//
		// NB: You should reset the input columns, they are
		// not reset automatically.
		//
		// That is, we are re-using the same input columns and
		// if we will return nil without doing anything, data will be
		// just duplicated.

		input.Reset() // calls "Reset" on each column

		if blocks >= 10 {
			// Stop streaming.
			//
			// This will also write tailing input data if any,
			// but we just reset the input, so it is currently blank.
			return io.EOF
		}

		// Append new values:
		for i := 0; i < 10; i++ {
			body.AppendBytes([]byte("Hello"))
			ts.Append(now)
			name.Append("name")
			sevText.Append("DEBUG")
			sevNumber.Append(10)
			arr.Append([]string{"foo", "bar", "baz"})
		}

		// Data will be encoded and sent to ClickHouse server after returning nil.
		// The Do method will return error if any.
		blocks++
		return nil
	},
}); err != nil {
	panic(err)
}

Writing dumps in Native format

You can use ch-go to write ClickHouse dumps in Native format:

The most efficient format. Data is written and read by blocks in binary format. For each block, the number of rows, number of columns, column names and types, and parts of columns in this block are recorded one after another. In other words, this format is “columnar” – it does not convert columns to rows. This is the format used in the native interface for interaction between servers, for using the command-line client, and for C++ clients.

See ./internal/cmd/ch-native-dump for more sophisticated example.

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

Features

  • OpenTelemetry support
  • No reflection or interface{}
  • Generics (go1.18) for Array[T], LowCardinaliy[T], Map[K, V], Nullable[T]
  • Reading or writing ClickHouse dumps in Native format
  • Column-oriented design that operates directly with blocks of data
    • Dramatically more efficient
    • Up to 100x faster than row-first design around sql
    • Up to 700x faster than HTTP API
    • Low memory overhead (data blocks are slices, i.e. continuous memory)
    • Highly efficient input and output block streaming
    • As close to ClickHouse as possible
  • Structured query execution telemetry streaming
  • LZ4, ZSTD or None (just checksums for integrity check) compression
  • External data support
  • Rigorously tested
    • Windows, Mac, Linux (also x86)
    • Unit tests for encoding and decoding
      • ClickHouse Server in Go for faster tests
      • Golden files for all packets, columns
      • Both server and client structures
      • Ensuring that partial read leads to failure
    • End-to-end tests on multiple LTS and stable versions
    • Fuzzing

Supported types

  • UInt8, UInt16, UInt32, UInt64, UInt128, UInt256
  • Int8, Int16, Int32, Int64, Int128, Int256
  • Date, Date32, DateTime, DateTime64
  • Decimal32, Decimal64, Decimal128, Decimal256 (only low-level raw values)
  • IPv4, IPv6
  • String, FixedString(N)
  • UUID
  • Array(T)
  • Enum8, Enum16
  • LowCardinality(T)
  • Map(K, V)
  • Bool
  • Tuple(T1, T2, ..., Tn)
  • Nullable(T)
  • Point
  • Nothing, Interval

Enums

You can use automatic enum inference in proto.ColEnum, this will come with some performance penalty.

To use proto.ColEnum8 and proto.ColEnum16, you need to explicitly provide DDL for them via proto.Wrap:

var v proto.ColEnum8

const ddl = `'Foo'=1, 'Bar'=2, 'Baz'=3`
input := []proto.InputColumn{
  {Name: "v", Data: proto.Wrap(&v, ddl)},
}

Generics

Most columns implement proto.ColumnOf[T] generic constraint:

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

For example, ColStr (and ColStr.LowCardinality) implements ColumnOf[string]. Same for arrays: new(proto.ColStr).Array() implements ColumnOf[[]string], column of []string values.

Array

Generic for Array(T)

// Array(String)
arr := proto.NewArray[string](new(proto.ColStr))
// Or
arr := new(proto.ColStr).Array()
q := ch.Query{
  Body:   "SELECT ['foo', 'bar', 'baz']::Array(String) as v",
  Result: arr.Results("v"),
}
// Do ...
arr.Row(0) // ["foo", "bar", "baz"]

Dumps

Reading

Use proto.Block.DecodeRawBlock on proto.NewReader:

func TestDump(t *testing.T) {
	// Testing decoding of Native format dump.
	//
	// CREATE TABLE test_dump (id Int8, v String)
	//   ENGINE = MergeTree()
	// ORDER BY id;
	//
	// SELECT * FROM test_dump
	//   ORDER BY id
	// INTO OUTFILE 'test_dump_native.raw' FORMAT Native;
	data, err := os.ReadFile(filepath.Join("_testdata", "test_dump_native.raw"))
	require.NoError(t, err)
	var (
		dec    proto.Block
		ids    proto.ColInt8
		values proto.ColStr
	)
	require.NoError(t, dec.DecodeRawBlock(
		proto.NewReader(bytes.NewReader(data)),
		proto.Results{
			{Name: "id", Data: &ids},
			{Name: "v", Data: &values},
		}),
	)
}

Writing

Use proto.Block.EncodeRawBlock with version 54451 on proto.Buffer with Rows and Columns set:

func TestLocalNativeDump(t *testing.T) {
	ctx := context.Background()
	// Testing clickhouse-local.
	var v proto.ColStr
	for _, s := range data {
		v.Append(s)
	}
	buf := new(proto.Buffer)
	b := proto.Block{Rows: 2, Columns: 2}
	require.NoError(t, b.EncodeRawBlock(buf, 54451, []proto.InputColumn{
		{Name: "title", Data: v},
		{Name: "data", Data: proto.ColInt64{1, 2}},
	}), "encode")

	dir := t.TempDir()
	inFile := filepath.Join(dir, "data.native")
	require.NoError(t, os.WriteFile(inFile, buf.Buf, 0600), "write file")

	cmd := exec.Command("clickhouse-local", "local",
		"--logger.console",
		"--log-level", "trace",
		"--file", inFile,
		"--input-format", "Native",
		"--output-format", "JSON",
		"--query", "SELECT * FROM table",
	)
	out := new(bytes.Buffer)
	errOut := new(bytes.Buffer)
	cmd.Stdout = out
	cmd.Stderr = errOut

	t.Log(cmd.Args)
	require.NoError(t, cmd.Run(), "run: %s", errOut)
	t.Log(errOut)

	v := struct {
		Rows int `json:"rows"`
		Data []struct {
			Title string `json:"title"`
			Data  int    `json:"data,string"`
		}
	}{}
	require.NoError(t, json.Unmarshal(out.Bytes(), &v), "json")
	assert.Equal(t, 2, v.Rows)
	if assert.Len(t, v.Data, 2) {
		for i, r := range []struct {
			Title string `json:"title"`
			Data  int    `json:"data,string"`
		}{
			{"Foo", 1},
			{"Bar", 2},
		} {
			assert.Equal(t, r, v.Data[i])
		}
	}
}

TODO

  • Types
    • Decimal(P, S) API
    • JSON
    • SimpleAggregateFunction
    • AggregateFunction
    • Nothing
    • Interval
    • Nested
    • Geo types
      • Point
      • Ring
      • Polygon
      • MultiPolygon
  • Improved i/o timeout handling for reading packets from server
    • Close connection on context cancellation in all cases
    • Ensure that reads can't block forever

Reference

License

Apache License 2.0

ch-go's People

Contributors

alpinskiy avatar arnaudbriche avatar charredlot avatar dependabot[bot] avatar develar avatar epicstep avatar ernado avatar filimonov avatar fzambia avatar genzgd avatar jkaflik avatar kaworu avatar kevinjoiner avatar msaf1980 avatar mshustov avatar taiyang-li avatar tdakkota avatar tomershafir avatar vadimalekseev avatar yehonatanwe avatar yenchangchan avatar

Stargazers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

Watchers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

ch-go's Issues

group by with totals fails with 'handle packet: unexpected packet "Totals"'

Describe the bug

Any query with GROUP BY WITH TOTALS statement, fails with handle packet: unexpected packet "Totals"

Steps to reproduce

We can take the totals test from clickhouse-go for example

SELECT
	number AS n, COUNT()
FROM (
	SELECT number FROM system.numbers LIMIT 100
) GROUP BY n WITH TOTALS

Expected behaviour

Return The totals row as part of the result.

Code example

package main

import (
	"context"
	"fmt"

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

func main() {
	ctx := context.Background()
	client, err := ch.Dial(ctx, ch.Options{Address: "localhost:9000"})
	if err != nil {
		panic(err)
	}
	n := proto.ColUInt64{}
	c := proto.ColUInt64{}
	query := `
		SELECT
			number AS n, COUNT() AS c
		FROM (
			SELECT number FROM system.numbers LIMIT 100
		) GROUP BY n WITH TOTALS
	`
	data := map[string][]uint64{"n": {}, "c": {}}
	r := proto.Results{{Name: "n", Data: &n}, {Name: "c", Data: &c}}
	onResult := func(context.Context, proto.Block) error {
		for i := 0; i < r.Rows(); i++ {
			data["n"] = append(data["n"], n.Row(i))
			data["c"] = append(data["c"], c.Row(i))
		}
		return nil
	}
	if err := client.Do(ctx, ch.Query{Body: query, Result: &r, OnResult: onResult}); err != nil {
		panic(err)
	}
	fmt.Printf("%+v\n", data)
}

Error log

handle packet: unexpected packet "Totals"

Configuration

Any

Environment

Any

ClickHouse server

Any

feat: improve decimal API

I found a problem when I tested the code. Is there something wrong?

CREATE TABLE default.dev
(
    test   Decimal32(2)
)
    ENGINE = MergeTree()
        ORDER BY tuple();
var data proto.ColDecimal32
data.Append(1)
if err := c.Do(ctx, ch.Query{
    Body: "INSERT INTO default.dev VALUES",
    Input: []proto.InputColumn{
        {
            Name: "test",
            Data: data,
	},
    },
}); err != nil {
    panic(err)
}

run error:

panic: handle packet: NUMBER_OF_ARGUMENTS_DOESNT_MATCH (42): DB::Exception: Decimal data type family must have exactly two arguments: precision and scale

ColDecimal32/ColDecimal64/ColDecimal128 the same

bug: write-deadline should be cleared if it gets set

in the client:

		if err := c.conn.SetWriteDeadline(deadline); err != nil {
			return errors.Wrap(err, "set write deadline")
		}

.. it's not properly clearing the deadline, and so repeated use of the same connection ends up throwing up flush/timeout errors.

suggest something like a deferred call to SetWriteDeadline(time.Time{}), similar to what's done w/ SetReadDeadline elsewhere

async_insert with ch-go

Is it possible to use async_insert with ch-go? Every way I tried I always get this debug statement in the server logs:

<Debug> executeQuery: Setting async_insert=1, but INSERT query will be executed synchronously (reason: insert query doesn't have inlined data)

ColBytes support for LowCardinality

i'm trying to avoid converting []byte types into string for bulk inserts w/ low-cardinality cols but the API doesn't seem to allow that since []byte is not comparable

as a workaround, I can use ColString and then coerce the []byte vals into string with the unfortunate side-effect of additional allocations. ideally this would not be needed and the API would support low-cardinality w/ []byte natively.

Streams leading to incomplete parts

I'm experimenting streaming blocks of data into the server and I can't seem to be able to get a full stream until the server flushes to a part.

For example, I'm trying to blocks of 100 rows and return io.EOF every 10_000 blocks. I would expect the server to create a part after reaching 1_000_000 rows but instead it created these parts:

part with 98400 rows
part with 98600 rows
part with 98500 rows
part with 98400 rows
part with 98500 rows
part with 98400 rows
part with 98500 rows
part with 98500 rows
part with 98400 rows
part with 98500 rows
part with 15300 rows

That leads to a total of 1_000_000 rows as expected but why is the server flushing to a part before the client returns io.EOF (and I assume sends an empty block and end of stream?).

Is there a setting on the server side that can be adjusted to change how often the data is flushed to a part and hopefully flush less often to create bigger parts?

Thanks!

Date32 wrong proto representation

Describe the bug

Steps to reproduce

  1. Append any value to proto.ColDate32
  2. Send data to ClickHouse
  3. The date32Epoch constant shifts values. In most cases, it causes an overflow with Date32 represent as uint32

const date32Epoch = -1420070400

Expected behaviour

Date32 value should be represented as int32 and not be shifted. It represents the number of days since the start of Unix time: https://clickhouse.com/docs/en/sql-reference/data-types/date32

Code example

package main

import (
	"context"
	"io"
	"time"

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

func main() {
	ctx := context.Background()

	conn, err := ch.Dial(ctx, ch.Options{})
	if err != nil {
		panic(err)
	}

	if err := conn.Do(ctx, ch.Query{
		Body: `CREATE TABLE IF NOT EXISTS test_table_insert
(
    d32			      Date32,
) ENGINE = Memory`,
	}); err != nil {
		panic(err)
	}

	// Define all columns of table.
	var (
		d32       proto.ColDate32
	)

	// Append 10 rows to initial data block.
	for i := 0; i < 10; i++ {
		d32.Append(time.Now())
	}

	// Insert single data block.
	input := proto.Input{
		{Name: "d32", Data: &d32},
	}
	if err := conn.Do(ctx, ch.Query{
		Body: "INSERT INTO test_table_insert VALUES",
		// Or "INSERT INTO test_table_insert (ts, severity_text, severity_number, body, name, arr) VALUES"
		Input: input,
	}); err != nil {
		panic(err)
	}

	// Stream data to ClickHouse server in multiple data blocks.
	var blocks int
	if err := conn.Do(ctx, ch.Query{
		Body:  input.Into("test_table_insert"), // helper that generates INSERT INTO query with all columns
		Input: input,

		// OnInput is called to prepare Input data before encoding and sending
		// to ClickHouse server.
		OnInput: func(ctx context.Context) error {
			// On OnInput call, you should fill the input data.
			//
			// NB: You should reset the input columns, they are
			// not reset automatically.
			//
			// That is, we are re-using the same input columns and
			// if we will return nil without doing anything, data will be
			// just duplicated.

			input.Reset() // calls "Reset" on each column

			if blocks >= 10 {
				// Stop streaming.
				//
				// This will also write tailing input data if any,
				// but we just reset the input, so it is currently blank.
				return io.EOF
			}

			// Append new values:
			for i := 0; i < 10; i++ {
				d32.Append(time.Now())
			}

			// Data will be encoded and sent to ClickHouse server after returning nil.
			// The Do method will return error if any.
			blocks++
			return nil
		},
	}); err != nil {
		panic(err)
	}
}

Configuration

Environment

  • Client version: HEAD

ClickHouse server

  • ClickHouse Server version: latest

how to append data?

auto infer

col := &proto.ColAuto{}
if err := col.Infer(proto.ColumnType("DateTime")); err != nil {
	fmt.Printf("Auto Infer Error: %s\n", err)
}

// how to append data to column???
col.append()???

After renaming the package can't be installed

➜  awesomeProject go install github.com/ClickHouse/ch-go@latest
package github.com/ClickHouse/ch-go is not a main package
➜  awesomeProject go install github.com/go-faster/ch@latest
go: github.com/go-faster/ch@latest: github.com/go-faster/[email protected]: parsing go.mod:
        module declares its path as: github.com/ClickHouse/ch-go
                but was required as: github.com/go-faster/ch

ColMap support for LowCardinality(String) for both Keys and Values

I cannot find a proper way to define a Map(LowCardinality(String), LowCardinality(String)), does anybody try to convert a map[string]string into this?

by defining it as proto.NewMap[string, string](new(proto.ColStr), new(proto.ColStr)), clickhouse-server will raise an Exception, Cannot convert: Map(String, String) to Map(LowCardinality(String), LowCardinality(String)), server version 22.8

Request: add option for struct/primitive scan

Hi, can we make this interface to be more generic? and work more like lets say sqlx works?
i.e pass in the struct/primitive we want to scan, and fill up the fields automatically
i think it will imrove the usabilty of this driver

How to read from array of low cardinality column?

Discussed in #22

Originally posted by kant777 January 25, 2022
How to read from array of low cardinality string column?

Below is the parts of the code I used to Read LowCardinality Column

tagsKey = proto.ColStr{}
tagsKeyLC = proto.ColLowCardinality{
	Index: &tagsKey,
}
tagsKeyArr = proto.ColArr{
	Data: &tagsKeyLC,
}

Result: proto.Results{
	{Name: "tags.key", Data: &tagsKeyArr},
},

I get the following error

panic: decode block: decode block: target: tags.key: decode data: got version 30, expected 1

any example?

go1.18.4, go1.19: internal compiler error: assertion failed

Not sure if this is a bug here or with the go compiler itself, but I get the following error when building in a Docker container:

/home/circleci/go/pkg/mod/github.com/!click!house/[email protected]/proto/col_map.go:124:11: internal compiler error: assertion failed

Minimal Dockerfile to reproduce:

FROM golang:1.18
WORKDIR /project
COPY . .
RUN go build -v .

main.go (same as the example in the README):

package main

import (
  "context"
  "fmt"

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

func main() {
  ctx := context.Background()
  c, err := ch.Dial(ctx, ch.Options{Address: "localhost:9000"})
  if err != nil {
    panic(err)
  }
  var (
    numbers int
    data    proto.ColUInt64
  )
  if err := c.Do(ctx, ch.Query{
    Body: "SELECT number FROM system.numbers LIMIT 500000000",
    Result: proto.Results{
      {Name: "number", Data: &data},
    },
    // OnResult will be called on next received data block.
    OnResult: func(ctx context.Context, b proto.Block) error {
      numbers += len(data)
      return nil
    },
  }); err != nil {
    panic(err)
  }
  fmt.Println("numbers:", numbers)
}

Mock server causing client to panic

I tried running the mock server

[a-dot ~/repos/ch-go/internal/cmd/ch-bench-server (main)]$ go build
[a-dot ~/repos/ch-go/internal/cmd/ch-bench-server (main)]$ ./ch-bench-server 
starting with chunk of 1.6 MB

And then tried running the ch-write-bench-faster benchmark. (I had to add one line to specify port 9001 instead of the default 9000):

[a-dot ~/repos/ch-bench/ch-write-bench-faster (main)]$ go build
[a-dot ~/repos/ch-bench/ch-write-bench-faster (main)]$ ./ch-write-bench-faster 
Error: decode block:
    github.com/ClickHouse/ch-go.(*Client).Do.func4
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/query.go:640
  - decode block:
    github.com/ClickHouse/ch-go.(*Client).decodeBlock
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/query.go:219
  - info:
    github.com/ClickHouse/ch-go/proto.(*Block).DecodeBlock
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/proto/block.go:254
  - field id:
    github.com/ClickHouse/ch-go/proto.(*BlockInfo).Decode
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/proto/block.go:44
  - read:
    github.com/ClickHouse/ch-go/proto.(*Reader).UVarInt
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:92
  - read:
    github.com/ClickHouse/ch-go/proto.(*Reader).ReadFull
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:66
  - read next block:
    github.com/ClickHouse/ch-go/internal/compress.(*Reader).Read
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/internal/compress/reader.go:117
  - data size should be 0 < 913600073 < 134217728:
    github.com/ClickHouse/ch-go/internal/compress.(*Reader).readBlock
        /home/a-dot/go/pkg/mod/github.com/!click!house/[email protected]/internal/compress/reader.go:46

But then I get this panic. How do I use the mock server?

invalid memory address or nil pointer dereference in query.go:99

Describe the bug

Under heavy load I get the following error:

golang.org/x/[email protected]/errgroup/errgroup.go:72 +0xa5
created by golang.org/x/sync/errgroup.(*Group).Go
golang.org/x/[email protected]/errgroup/errgroup.go:75 +0x64
golang.org/x/sync/errgroup.(*Group).Go.func1()
github.com/ClickHouse/[email protected]/query.go:622 +0x85
github.com/ClickHouse/ch-go.(*Client).Do.func3()
github.com/ClickHouse/[email protected]/query.go:99 +0x635
github.com/ClickHouse/ch-go.(*Client).sendQuery(_, {_, _}, {{0xc0011722c0, 0x1a6}, {0xc002823dd0, 0x24}, {0x0, 0x0}, {0x0, ...}, ...})
goroutine 26759 [running]:
[signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x86ec35]
panic: runtime error: invalid memory address or nil pointer dereference

Steps to reproduce

Unfortunately, the issue is reproducible only in production and I can't reproduce it locally.

Environment

  • Client version: 0.56.0
  • Language version: 1.20.4
  • OS: Linux

ClickHouse server

  • ClickHouse Server version: 22.10.3.27

Refactoring and low-level specialization

This repo should be kept mostly low-level, providing primitives on ClickHouse TCP protocol and Native format.

So, some higher-level abstractions should be extracted.
Also, some refactoring should be done based on acquired feedback and expertise, e.g. on using generics, json column type implementation and others.

This issue should track overall work.

  • Support JSON column type
  • Rework DataTime64 to contain precision #130
  • Rework Enum8, Enum16 to contain enum definition
  • Add Apache header to all files

List is updating.

This will require some breaking changes.

[Question] Is it possible to accept exception that are not utf-8?

ch-go/proto/reader.go

Lines 144 to 154 in e0933a2

func (r *Reader) Str() (string, error) {
s, err := r.StrBytes()
if err != nil {
return "", errors.Wrap(err, "bytes")
}
if !utf8.Valid(s) {
return "", errors.New("invalid utf8")
}
return string(s), err
}

If the user enters some incorrect query statement

e.g:

SELECT 世界;

I can never get a specific exception message in this case


Worse still, the reader will be retained and interfere with the next query :-(

bug: UNEXPECTED_PACKET_FROM_CLIENT after context cancellation

Connection gets unusable (always returns an error) after context cancellation.

Reproducible example:

package main

import (
	"context"
	"fmt"
	"github.com/ClickHouse/ch-go"
	"github.com/ClickHouse/ch-go/proto"
	"github.com/stretchr/testify/require"
	"testing"
)

const tableName = "ch_go_test"

func Test1(t *testing.T) {
	c, err := ch.Dial(context.Background(), ch.Options{})
	require.NoError(t, err)
	err = c.Do(context.Background(), ch.Query{
		Body: fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s (v Int32) ENGINE = GenerateRandom(0, 0, 0)", tableName),
	})
	require.NoError(t, err)
	for i := 1; ; i++ {
		var (
			rows int
			data proto.ColInt32
		)
		ctx, cancel := context.WithCancel(context.Background())
		err = c.Do(ctx, ch.Query{
			Body:   fmt.Sprintf("SELECT * FROM %s LIMIT %d", tableName, 1_000_000),
			Result: proto.Results{{Name: "v", Data: &data}},
			OnResult: func(_ context.Context, block proto.Block) error {
				rows += block.Rows
				if 500_000 < rows {
					cancel()
				}
				return nil
			},
		})
		cancel()
		if err != nil {
			require.ErrorIs(t, err, context.Canceled, "rows #%d, iteration #%d", rows, i)
		}
	}
}

In my case code above 100% ends up with:

=== RUN   Test1
    main_test.go:40: 
        	Error Trace:	/home/u/GolandProjects/ch-go-test/main_test.go:40
        	Error:      	Target error should be in err chain:
        	            	expected: "context canceled"
        	            	in chain: "handle packet: UNEXPECTED_PACKET_FROM_CLIENT (101): DB::NetException: Unexpected packet Hello received from client"
        	            		"UNEXPECTED_PACKET_FROM_CLIENT (101): DB::NetException: Unexpected packet Hello received from client"
        	Test:       	Test1
        	Messages:   	rows #0, iteration #3
--- FAIL: Test1 (0.01s)

check slice capacity to avoid unnecessary memory allocation

  1. When I use function https://github.com/ClickHouse/ch-go/blob/main/compress/writer.go#L21-L52, find https://github.com/ClickHouse/ch-go/blob/main/compress/writer.go#L23 don't need to createmake([]byte, maxSize+headerSize)when cap(w.Data) >= maxSize+headerSize
  2. And for EncodeColumn, when b.Buf has enough capacity, then no need to create make([]byte, size*len(v)), here are some examples:
    https://github.com/ClickHouse/ch-go/blob/main/proto/col_decimal64_unsafe_gen.go#L31-L45
    https://github.com/ClickHouse/ch-go/blob/main/proto/col_int64_unsafe_gen.go#L31-L45

Possible stuck when we set big timeout

Describe the bug

Stuck conn.Do(query) if we set big timeout through the context.Context

Steps to reproduce

  1. Create context with big timeout, e.g.: ctx, cancel := context.WithTimeout(context.Background(), time.Hour)
  2. Do some invalid request (check my example below)

Expected behaviour

It seems to me that we should immediately get a response from the .Do(query) function

Code example

package main

import (
	"context"
	"fmt"
	"time"

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

func main() {
	ctx, cancel := context.WithTimeout(context.Background(), time.Hour)
	defer cancel()

	conn, err := chpool.Dial(ctx, chpool.Options{
		ClientOptions: ch.Options{
			Address: "127.0.0.1:9001",
		},
	})
	if err != nil {
		panic(err)
	}

	err = conn.Do(ctx, ch.Query{Body: `DROP TABLE IF EXISTS test_table_insert`})
	if err != nil {
		panic(err)
	}

	err = conn.Do(ctx, ch.Query{
		Body: `CREATE TABLE IF NOT EXISTS test_table_insert
(
    level Enum8('error'=1, 'warn'=2, 'info'=3, 'debug'=4)
) ENGINE = Memory;`})
	if err != nil {
		panic(err)
	}

	level := new(proto.ColEnum)
	err = level.Infer("Enum8('error'=1, 'warn'=2, 'info'=3, 'debug'=4)")
	if err != nil {
		panic(err)
	}

	level.Append("unknown level") // ! this level does not exist in the enum

	err = conn.Do(ctx, ch.Query{
		Body: `INSERT INTO test_table_insert VALUES`,
		Input: proto.Input{
			{Name: "level", Data: level},
		},
	}) // lock here for an 1 hour
	if err != nil {
		panic(err)
	}

	fmt.Println("done")
}

Some parts of the goroutines dump:

1 @ 0x100a74e14 0x100a41b04 0x100a41674 0x100f64da4 0x100f5c7ec 0x100aa6334
#	0x100f64da3	github.com/ClickHouse/ch-go.(*Client).Do.func5+0x43	/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/query.go:688
#	0x100f5c7eb	golang.org/x/sync/errgroup.(*Group).Go.func1+0x5b	/Users/valekseev/go/pkg/mod/golang.org/x/[email protected]/errgroup/errgroup.go:75



1 @ 0x100a74e14 0x100a6dfd8 0x100aa00d0 0x100ada4e8 0x100adb810 0x100adb801 0x100cb97d8 0x100ccac04 0x100b51020 0x100ed677c 0x100ad56f0 0x100ed685c 0x100ed683d 0x100ed65f4 0x100ed65f5 0x100b76bb0 0x100ed6a48 0x100f5d4b4 0x100f64f40 0x100f5c7ec 0x100aa6334
#	0x100aa00cf	internal/poll.runtime_pollWait+0x9f				/opt/homebrew/Cellar/go/1.19.2/libexec/src/runtime/netpoll.go:305
#	0x100ada4e7	internal/poll.(*pollDesc).wait+0x27				/opt/homebrew/Cellar/go/1.19.2/libexec/src/internal/poll/fd_poll_runtime.go:84
#	0x100adb80f	internal/poll.(*pollDesc).waitRead+0x1df			/opt/homebrew/Cellar/go/1.19.2/libexec/src/internal/poll/fd_poll_runtime.go:89
#	0x100adb800	internal/poll.(*FD).Read+0x1d0					/opt/homebrew/Cellar/go/1.19.2/libexec/src/internal/poll/fd_unix.go:167
#	0x100cb97d7	net.(*netFD).Read+0x27						/opt/homebrew/Cellar/go/1.19.2/libexec/src/net/fd_posix.go:55
#	0x100ccac03	net.(*conn).Read+0x33						/opt/homebrew/Cellar/go/1.19.2/libexec/src/net/net.go:183
#	0x100b5101f	bufio.(*Reader).Read+0x1df					/opt/homebrew/Cellar/go/1.19.2/libexec/src/bufio/bufio.go:237
#	0x100ed677b	github.com/ClickHouse/ch-go/proto.(*Reader).Read+0x2b		/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:53
#	0x100ad56ef	io.ReadAtLeast+0x9f						/opt/homebrew/Cellar/go/1.19.2/libexec/src/io/io.go:332
#	0x100ed685b	io.ReadFull+0x3b						/opt/homebrew/Cellar/go/1.19.2/libexec/src/io/io.go:351
#	0x100ed683c	github.com/ClickHouse/ch-go/proto.(*Reader).ReadFull+0x1c	/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:62
#	0x100ed65f3	github.com/ClickHouse/ch-go/proto.(*Reader).readFull+0xb3	/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:70
#	0x100ed65f4	github.com/ClickHouse/ch-go/proto.(*Reader).ReadByte+0xb4	/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:36
#	0x100b76baf	encoding/binary.ReadUvarint+0x7f				/opt/homebrew/Cellar/go/1.19.2/libexec/src/encoding/binary/varint.go:133
#	0x100ed6a47	github.com/ClickHouse/ch-go/proto.(*Reader).UVarInt+0x27	/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/proto/reader.go:84
#	0x100f5d4b3	github.com/ClickHouse/ch-go.(*Client).packet+0x113		/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/client.go:224
#	0x100f64f3f	github.com/ClickHouse/ch-go.(*Client).Do.func4+0xff		/Users/valekseev/go/pkg/mod/github.com/!click!house/[email protected]/query.go:657
#	0x100f5c7eb	golang.org/x/sync/errgroup.(*Group).Go.func1+0x5b		/Users/valekseev/go/pkg/mod/golang.org/x/[email protected]/errgroup/errgroup.go:75

Researching brought me to the function func (c *Client) packet(ctx context.Context) (proto.ServerCode, error)

-- we check and override the default timeout from the context:

	deadline := time.Now().Add(defaultTimeout)
	if d, ok := ctx.Deadline(); ok {
		deadline = d
	}

Therefore, we have come out of this function for so long, so the done (/[email protected]/query.go:688) channel does not close, so we cannot get an answer.

Configuration

Environment

  • Client version: v0.52.0
  • Language version: 1.19

ClickHouse server

  • ClickHouse Server version: 22.1.3.7

Data loss when EOF in first block

Describe the bug

If we don't fill any data before conn.Do and then return io.EOF instead of nil on first OnInput call client doesn't write tail of input data.

If we return nil at least once in OnInput and after that return io.EOF after adding some data then this data will be recorded to table

Steps to reproduce

  1. don't fill any column data in input before conn.Do
  2. return io.EOF on first OnInput call

Expected behaviour

This code must write 10 rows instead of 0

Code example

package main

import (
	"context"
	"io"
	"time"

	"github.com/ClickHouse/ch-go"
	"github.com/ClickHouse/ch-go/proto"
	"go.uber.org/zap"
)

func main() {
	ctx := context.Background()

	logger, _ := zap.NewDevelopment()

	conn, err := ch.Dial(ctx, ch.Options{
		Logger: logger,
	})
	if err != nil {
		panic(err)
	}

	if err := conn.Do(ctx, ch.Query{
		Body: `CREATE TABLE IF NOT EXISTS test_table_insert
(
    ts                DateTime64(9),
    severity_text     Enum8('INFO'=1, 'DEBUG'=2),
    severity_number   UInt8,
    body              String,
    name              String,
    arr               Array(String)
) ENGINE = Memory`,
	}); err != nil {
		panic(err)
	}

	var (
		body      proto.ColStr
		name      proto.ColStr
		sevText   proto.ColEnum
		sevNumber proto.ColUInt8

		ts  = new(proto.ColDateTime64).WithPrecision(proto.PrecisionNano)
		arr = new(proto.ColStr).Array()
		now = time.Date(2010, 1, 1, 10, 22, 33, 345678, time.UTC)
	)

	// just prepare empty data
	input := proto.Input{
		{Name: "ts", Data: ts},
		{Name: "severity_text", Data: &sevText},
		{Name: "severity_number", Data: &sevNumber},
		{Name: "body", Data: &body},
		{Name: "name", Data: &name},
		{Name: "arr", Data: arr},
	}

	var blocks int
	if err := conn.Do(ctx, ch.Query{
		Body:  input.Into("test_table_insert"),
		Input: input,
		OnInput: func(ctx context.Context) error {
			input.Reset()

			for i := 0; i < 10; i++ {
				body.AppendBytes([]byte("Hello"))
				ts.Append(now)
				name.Append("name")
				sevText.Append("DEBUG")
				sevNumber.Append(10)
				arr.Append([]string{"foo", "bar", "baz"})
			}

			if blocks >= 10 {
				// we call this after adding data, in that case we get 110 rows when nil returned in last statement
				return io.EOF
			}

			blocks++
			return io.EOF // instead of nil
		},
	}); err != nil {
		panic(err)
	}
}

Configuration

Environment

  • Client version: v0.58.2
  • Language version: go1.21.1
  • OS: linux

ClickHouse server

  • ClickHouse Server version: 22.1.3.7

"got rows without target" when creating database on cluster

Describe the bug

I'm using ch-go to creating database and tables. Bug ch-go would failed instantly after creating database, and after failure I can see err logs in clickhouse console with database created.

logs from clickhouse:

2023.09.06 17:30:56.757502 [ 50 ] {} <Error> TCPHandler: Code: 33. DB::Exception: Cannot read all data. Bytes read: 0. Bytes expected: 3. (CANNOT_READ_ALL_DATA), Stack trace (when copying this message, always include the lines below):

0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c604bf7 in /usr/bin/clickhouse
1. DB::Exception::Exception<unsigned long&, String>(int, FormatStringHelperImpl<std::type_identity<unsigned long&>::type, std::type_identity<String>::type>, unsigned long&, String&&) @ 0x00000000073116d4 in /usr/bin/clickhouse
2. DB::ReadBuffer::readStrict(char*, unsigned long) @ 0x00000000073115d7 in /usr/bin/clickhouse
3. DB::TCPHandler::receiveUnexpectedHello() @ 0x0000000013119d72 in /usr/bin/clickhouse
4. DB::TCPHandler::receivePacket() @ 0x0000000013114f20 in /usr/bin/clickhouse
5. DB::TCPHandler::runImpl() @ 0x000000001310bcbe in /usr/bin/clickhouse
6. DB::TCPHandler::run() @ 0x000000001311e839 in /usr/bin/clickhouse
7. Poco::Net::TCPServerConnection::start() @ 0x0000000015b104d4 in /usr/bin/clickhouse
8. Poco::Net::TCPServerDispatcher::run() @ 0x0000000015b116d1 in /usr/bin/clickhouse
9. Poco::PooledThread::run() @ 0x0000000015c47f07 in /usr/bin/clickhouse
10. Poco::ThreadImpl::runnableEntry(void*) @ 0x0000000015c461dc in /usr/bin/clickhouse
11. ? @ 0x00007f49373cf609 in ?
12. ? @ 0x00007f49372f4133 in ?

logs from ch-go's zap-logger:

decode block: raw block: got rows without target

Steps to reproduce

I've make a smallest reproducible project at here.

  1. docker compose up -d
  2. go test ./

Support for AggregateFunction data type

This is a parent issue to add support for the data type AggregateFunction. Several different categories of quantile functions can be used with AggregateFunction. I think each of them deserves their own type as they have custom serialization/deserialization. I would like to help with this effort starting with quantile(s)DDSketch and quantileBFloat16 and more to follow. I have a quick prototype for DDSketch here main...srikanthccv:ch-go:ddsketch.

@ernado do you have anything in mind how the API should look like?

Tasks

AppendArr is not implemented everywhere

I have a used case where I must insert in batch in a wide table with more than 100 columns and about 40 array columns.
The data is coming in small batches of a few hundreds rows and is appended to a ColInput until there is enough rows to send to the server.
I benchmarked a bit and find out that Appending new data to ColInputs was quite expensive, sometimes taking 10s of milliseconds.
I modified the code a bit to ensure AppendArr method was usable on every ColInput, and find out that it lead to a 10-fold improvement in the time taken to append new batches.
I'm preparing a PR that implements this change.

Streaming Data Example doesn't change values

Tried this example for streaming data into ClickHouse https://github.com/ClickHouse/ch-go/blob/main/examples/insert/main.go
with changed values for body, name

...
        // Append 10 rows to initial data block.
	for i := 0; i < 10; i++ {
		body.AppendBytes([]byte("Hello"))
		ts.Append(now)
		name.Append("name")
		sevText.Append("INFO")
		sevNumber.Append(10)
		arr.Append([]string{"foo", "bar", "baz"})
	}
        // Insert single data block.
	input := proto.Input{
		{Name: "ts", Data: ts},
		{Name: "severity_text", Data: &sevText},
		{Name: "severity_number", Data: sevNumber},
		{Name: "body", Data: body},
		{Name: "name", Data: name},
		{Name: "arr", Data: arr},
	}
	if err := conn.Do(ctx, ch.Query{
		Body: "INSERT INTO test_table_insert VALUES",
		// Or "INSERT INTO test_table_insert (ts, severity_text, severity_number, body, name, arr) VALUES"
		Input: input,
	}); err != nil {
		panic(err)
	}

	// Stream data to ClickHouse server in multiple data blocks.
	var blocks int
	if err := conn.Do(ctx, ch.Query{
		Body:  input.Into("test_table_insert"), // helper that generates INSERT INTO query with all columns
		Input: input,

		// OnInput is called to prepare Input data before encoding and sending
		// to ClickHouse server.
		OnInput: func(ctx context.Context) error {
...
                        // Append new values:
			for i := 0; i < 10; i++ {
				body.AppendBytes([]byte("Hello2"))
				ts.Append(now)
				name.Append("name2")
				sevText.Append("DEBUG")
				sevNumber.Append(11)
				arr.Append([]string{"foo", "bar", "baz"})
			}
...

Why severity_text changed to DEBUG, but severity_number, body, name are the same as in initial data block?

SELECT *
FROM test_table_insert

Query id: e59eeb35-3942-42b0-90d0-2f905025cdf3

┌────────────────────────────ts─┬─severity_text─┬─severity_number─┬─body──┬─name─┬─arr─────────────────┐
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
└───────────────────────────────┴───────────────┴─────────────────┴───────┴──────┴─────────────────────┘
┌────────────────────────────ts─┬─severity_text─┬─severity_number─┬─body──┬─name─┬─arr─────────────────┐
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ INFO          │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
│ 2010-01-01 13:22:33.000345678 │ DEBUG         │              10 │ Hello │ name │ ['foo','bar','baz'] │
└───────────────────────────────┴───────────────┴─────────────────┴───────┴──────┴─────────────────────┘

120 rows in set. Elapsed: 0.002 sec.

Or(v T) of Nullable[T] (in proto package) works improperly

Describe the bug

Or(v T) of Nullable[T] (in proto package) works improperly

Steps to reproduce

  1. Use Or(v T) in parsing results of query
  2. If underlying value is set, Or() will anyway return default value (v T), due to this bug

Expected behaviour

Should return default value when underlying is not Set

Code example

Looks like this:

func (n Nullable[T]) Or(v T) T {
	if n.Set {
		return v
	}
	return n.Value
}

Should be:

func (n Nullable[T]) Or(v T) T {
	if n.Set {
		return n.Value
	}
	return v
}

Error log

Configuration

Environment

  • Client version: 0.50, 0.61
  • Language version: any
  • OS: any

ColMap should support Prepare for Keys and Values

Describe the bug

Using LowCardinality for ColMap Key or Value will receive an error from server

Steps to reproduce

Expected behaviour

Send OK.

Code example

TraceId := new(proto.ColStr)
ResourceAttributes := proto.NewMap[string, string](new(proto.ColStr).LowCardinality(), new(proto.ColStr).LowCardinality())
SpanAttributes := proto.NewMap[string, string](new(proto.ColStr).LowCardinality(), new(proto.ColStr).LowCardinality())
input := proto.Input{
	{Name: "TraceId", Data: &TraceId},
	{Name: "ResourceAttributes", Data: ResourceAttributes},
	{Name: "SpanAttributes", Data: SpanAttributes},
}

Error log

INCORRECT_DATA (117): DB::Exception: Index for LowCardinality is out of range. Dictionary size is 0, but found index with value 4

Configuration

Environment

  • Client version: v0.51.2 but v0.55.0 is the same code for proto.ColMap
  • Language version:
  • OS: Darwin and Linux

ClickHouse server

  • ClickHouse Server version: 23.1.3.5
  • ClickHouse Server non-default settings, if any:
  • CREATE TABLE statements for tables involved:
  • Sample data for all these tables, use clickhouse-obfuscator if necessary

bug(proto): ArrayOf[T] of LowCardinality[T] does not work

t.Run("ArrayLowCardinality", func(t *testing.T) {
	t.Parallel()
	conn := Conn(t)
	require.NoError(t, conn.Do(ctx, Query{
		Body: "CREATE TABLE test_table (v Array(LowCardinality(String))) ENGINE = Memory",
	}), "create table")

	v := proto.ArrayOf[string](new(proto.ColStr).LowCardinality())
	v.Append([]string{"foo", "bar"})
	v.Append([]string{"baz"})

	require.NoError(t, conn.Do(ctx, Query{
		Body: "INSERT INTO test_table VALUES",
		Input: []proto.InputColumn{
			{Name: "v", Data: v},
		},
	}), "insert")

	gotData := proto.ArrayOf[string](new(proto.ColStr).LowCardinality())
	require.NoError(t, conn.Do(ctx, Query{
		Body: "SELECT * FROM test_table",
		Result: proto.Results{
			{Name: "v", Data: gotData},
		},
	}), "select")

	require.Equal(t, []string{"foo", "bar"}, gotData.Row(0)) // {"foo", "foo"}
	require.Equal(t, []string{"baz"}, gotData.Row(1)) // {"foo"}
})

Default values are not triggered when inserting data???

CREATE TABLE test_materialization
(
    key UInt32,
    value UInt32,
    inserted_at DateTime DEFAULT now()
)
ENGINE = MergeTree
ORDER BY key;

use ch-go insert key and value column

key value inserted_at
1 1 1970-01-01 00:00:00

use clickhouse-client key and value column

insert into test_materialization(key,value)values(2,2);
select * from test_materialization;
key value inserted_at
1 1 1970-01-01 00:00:00
2 2 2023-02-15 16:34:11

Chpool depends on puddle package version 1.2.2 which does not exist

Chpool package requires github.com/jackc/puddle v1.2.2-0.20220404125616-4e959849469a to be installed

$ go mod tidy          
go: finding module for package github.com/jackc/puddle/puddleg
.../storage imports
        github.com/ClickHouse/ch-go/chpool imports
        github.com/jackc/puddle/puddleg: module github.com/jackc/puddle@latest found (v1.3.0), but does not contain package github.com/jackc/puddle/puddleg

nil pointer dereference on attempt to execute query in case DB is missing

(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:628\n  - UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist"}
{"level":"error","msg":"cannot handle http request","error":"packet: uvarint: read: read: read tcp 127.0.0.1:61219->127.0.0.1:9000: read: connection reset by peer","errorVerbose":"packet:\n    github.com/go-faster/ch.(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:609\n  - uvarint:\n    github.com/go-faster/ch.(*Client).packet\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/client.go:214\n  - read:\n    github.com/go-faster/ch/proto.(*Reader).UVarInt\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/proto/reader.go:92\n  - read:\n    github.com/go-faster/ch/proto.(*Reader).ReadFull\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/proto/reader.go:66\n  - read tcp 127.0.0.1:61219->127.0.0.1:9000: read: connection reset by peer"}
{"level":"error","msg":"cannot handle http request","error":"handle packet: UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist","errorVerbose":"handle packet:\n    github.com/go-faster/ch.(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:628\n  - UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist"}
{"level":"error","msg":"cannot handle http request","error":"packet: uvarint: read: read: read tcp 127.0.0.1:61220->127.0.0.1:9000: read: connection reset by peer","errorVerbose":"packet:\n    github.com/go-faster/ch.(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:609\n  - uvarint:\n    github.com/go-faster/ch.(*Client).packet\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/client.go:214\n  - read:\n    github.com/go-faster/ch/proto.(*Reader).UVarInt\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/proto/reader.go:92\n  - read:\n    github.com/go-faster/ch/proto.(*Reader).ReadFull\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/proto/reader.go:66\n  - read tcp 127.0.0.1:61220->127.0.0.1:9000: read: connection reset by peer"}
{"level":"error","msg":"cannot handle http request","error":"handle packet: UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist","errorVerbose":"handle packet:\n    github.com/go-faster/ch.(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:628\n  - UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist"}
{"level":"error","msg":"cannot handle http request","error":"handle packet: UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist","errorVerbose":"handle packet:\n    github.com/go-faster/ch.(*Client).Do.func4\n        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:628\n  - UNKNOWN_DATABASE (81): DB::Exception: Database ij doesn't exist"}
panic: runtime error: invalid memory address or nil pointer dereference
[signal SIGSEGV: segmentation violation code=0x2 addr=0x0 pc=0x102b3bec8]

goroutine 54 [running]:
github.com/go-faster/ch.(*Client).sendQuery(0x140000b6000, {_, _}, {{0x14000380000, 0x1a8}, {0x140025322d0, 0x24}, {0x0, 0x0}, {0x0, ...}, ...})
        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:90 +0x228
github.com/go-faster/ch.(*Client).Do.func3()
        /Users/maxim.kolmakov/go/pkg/mod/github.com/go-faster/[email protected]/query.go:568 +0x7c
golang.org/x/sync/errgroup.(*Group).Go.func1()
        /Users/maxim.kolmakov/go/pkg/mod/golang.org/x/[email protected]/errgroup/errgroup.go:57 +0x88
created by golang.org/x/sync/errgroup.(*Group).Go
        /Users/maxim.kolmakov/go/pkg/mod/golang.org/x/[email protected]/errgroup/errgroup.go:54 +0xc0

feat: add query builder with escaping

So, instead of writing

query := fmt.Sprintf("SELECT * FROM table WHERE foo = '%s' AND bar != %d", "foo", 1234)

We can do something like

query := Build("SELECT * FROM table WHERE foo = ? AND bar != ?", "foo", 1234)

And this should be safe to use without input sanitizing.

Issues when trying to do simple data copy between tables

From telegram channel:

[...]

Я тут копирую данные из одной таблицы в другую. Структура идентичная, селект * с условиями, и инсерт куда следует. Копирование между базами, драйвер ch-go (golang). Более высокоуровневый драйвер работает по строкам неоптимально, я его не пользую т.к. зачем тогда это всё columnar db и проч?. А этот работает блоками и массивами. Это правильно. Но в нем проблема1 и проблема2. Поясняю.

проблема1) состоит в том, что невозможно нормально получить массив данных из селект и тут же отправить его в инсерт. Промежуточная структура данных для LowCardinality в указанном драйвере енумерирует (переводит из строки в инт) мне все значения из колонки во время дальнейшей сериализации. А вообще, хранит как вычисленные строки для потребления так и хештейбл (map) внутри себя. Ну и пусть бы. Проблема возникает по причине того, что я хочу отправить это всё в соседний тред для записи в другую базу, а следующий блок получить в свежее место. Драйвер написан так, что нет возможности дать ему новое место куда ложить следующий блок (или я не нашел в примерах?). Поэтому я хочу сделать копию полученных данных, пусть хоть с итерацией по строкам во время промежуточных енумераций (что позор), и отправить ее в saver thread. Обычный цикл внутри .Do очищает и засовывает новую порцию в те же буфера (Result), и по кругу, а копия уже ушла на запись. Неидеально, но пусть хоть так.

проблема2) состоит в том, что копию полученных данных делать не получается по-человечески, но только ковыряясь в кишках структур буферов, вдобавок с приватными полями: речь идет про LowCardinality тип данных, который уже я упомянул выше. Поэтому я подумал, а почему бы мне не сделать SerializeToBlock/DeserializeFromBlock как это делают все ленивые белые люди, для получения глубокой копии данных с использованием авторской методики. Баг в том, что там блок, до этого десериализованный из wire stream в Result, когда сериализуешь его снова, не чистит одно из своих полей, и при сериализации получается потеря данных (индексы смотрят не туда), поэтому далее снова десериализовать это уже не получается - приходит с потерянными данными.

Подробно:

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

делает

c.index.Append(v)

ошибочно предполагая что длина c.index == 0 в начале функции, потому что автор написал что var last int ( == 0) перед циклом, хотя у блока который только что десериализирован из протокола (и сериализируется опять) там не ноль, а уже что-то есть. Поэтому надо

if c.kv == nil {
	c.kv = map[T]int{}
	c.index.Reset()                       //  ++++++ добавить это
}

Более глобально, помимо этого глюка, драма состоит в том, что я-то хочу отправить на инсерт такой же блок, что мне и пришел, а в драйвере делается много ненужной работы, например переиндексируются LowCardinality.

Infer error: automatic column inference not supported for UUID

Describe the bug

Auto inference not supported for UUID column

Code example

func TestUUIDInfer(t *testing.T) {
	col := &ColAuto{}
	if err := col.Infer(ColumnType("UUID")); err != nil {
		t.Errorf("Auto Infer Error: %s\n", err)
		t.Fail()
	}
}

Error log

Auto Infer Error: automatic column inference not supported for "UUID"

[Question] How to decode Array(Tuple()) column

Hi, I have the following column in my schema

map_arr Map(Int32, Array(Tuple(String, Array(Int32))))

But I couldn't find a way to create a ColArr of type ColTuple. I am new to go-lang, so it might very well be my ignorance.

I am not using the connection/query, but rather using the proto.* with DecodeRawBlock as described in native dump reading section to build an executable UDF.

Also, the Dump section in main README.md do not pass version for DecodeRawBlock and EncodeRawBlock as the current latest version requires, and it wasn't obvious which version to use.

Recommend Projects

  • React photo React

    A declarative, efficient, and flexible JavaScript library for building user interfaces.

  • Vue.js photo Vue.js

    🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.

  • Typescript photo Typescript

    TypeScript is a superset of JavaScript that compiles to clean JavaScript output.

  • TensorFlow photo TensorFlow

    An Open Source Machine Learning Framework for Everyone

  • Django photo Django

    The Web framework for perfectionists with deadlines.

  • D3 photo D3

    Bring data to life with SVG, Canvas and HTML. 📊📈🎉

Recommend Topics

  • javascript

    JavaScript (JS) is a lightweight interpreted programming language with first-class functions.

  • web

    Some thing interesting about web. New door for the world.

  • server

    A server is a program made to process requests and deliver data to clients.

  • Machine learning

    Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.

  • Game

    Some thing interesting about game, make everyone happy.

Recommend Org

  • Facebook photo Facebook

    We are working to build community through open source technology. NB: members must have two-factor auth.

  • Microsoft photo Microsoft

    Open source projects and samples from Microsoft.

  • Google photo Google

    Google ❤️ Open Source for everyone.

  • D3 photo D3

    Data-Driven Documents codes.