From fde04722c923b75efccd1664ccd291bc9f245cb5 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 20 Dec 2024 01:20:18 -0500 Subject: [PATCH 01/18] add variant type --- chcol.go | 37 +++ examples/clickhouse_api/main_test.go | 4 + examples/clickhouse_api/variant.go | 136 ++++++++++ lib/chcol/variant.go | 126 +++++++++ lib/chcol/variant_test.go | 72 +++++ lib/column/codegen/column.tpl | 6 +- lib/column/column_gen.go | 4 + lib/column/variant.go | 385 +++++++++++++++++++++++++++ lib/column/variant_test.go | 91 +++++++ tests/variant_test.go | 247 +++++++++++++++++ 10 files changed, 1107 insertions(+), 1 deletion(-) create mode 100644 chcol.go create mode 100644 examples/clickhouse_api/variant.go create mode 100644 lib/chcol/variant.go create mode 100644 lib/chcol/variant_test.go create mode 100644 lib/column/variant.go create mode 100644 lib/column/variant_test.go create mode 100644 tests/variant_test.go diff --git a/chcol.go b/chcol.go new file mode 100644 index 0000000000..7adbb8a1da --- /dev/null +++ b/chcol.go @@ -0,0 +1,37 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package clickhouse + +import "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + +// Re-export chcol types/funcs to top level clickhouse package + +type ( + Variant = chcol.Variant + VariantWithType = chcol.VariantWithType +) + +// NewVariant creates a new Variant with the given value +func NewVariant(v any) Variant { + return chcol.NewVariant(v) +} + +// NewVariantWithType creates a new Variant with the given value and ClickHouse type +func NewVariantWithType(v any, chType string) VariantWithType { + return chcol.NewVariantWithType(v, chType) +} diff --git a/examples/clickhouse_api/main_test.go b/examples/clickhouse_api/main_test.go index 85c054e258..40b59e36b2 100644 --- a/examples/clickhouse_api/main_test.go +++ b/examples/clickhouse_api/main_test.go @@ -209,3 +209,7 @@ func TestSSL(t *testing.T) { func TestSSLNoVerify(t *testing.T) { require.NoError(t, SSLNoVerifyVersion()) } + +func TestVariantExample(t *testing.T) { + require.NoError(t, VariantExample()) +} diff --git a/examples/clickhouse_api/variant.go b/examples/clickhouse_api/variant.go new file mode 100644 index 0000000000..b90a231e08 --- /dev/null +++ b/examples/clickhouse_api/variant.go @@ -0,0 +1,136 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package clickhouse_api + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" +) + +func VariantExample() error { + ctx := context.Background() + + conn, err := GetNativeConnection(clickhouse.Settings{ + "allow_experimental_variant_type": true, + }, nil, nil) + if err != nil { + return err + } + + err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_variant_example") + if err != nil { + return err + } + + err = conn.Exec(ctx, ` + CREATE TABLE go_variant_example ( + c Variant(Bool, Int64, String) + ) ENGINE = Memory + `) + if err != nil { + return err + } + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO go_variant_example (c)") + if err != nil { + return err + } + + if err = batch.Append(true); err != nil { + return err + } + + if err = batch.Append(int64(42)); err != nil { + return err + } + + if err = batch.Append("example"); err != nil { + return err + } + + if err = batch.Append(clickhouse.NewVariant("example variant")); err != nil { + return err + } + + if err = batch.Append(clickhouse.NewVariantWithType("example variant with specific type", "String")); err != nil { + return err + } + + if err = batch.Append(nil); err != nil { + return err + } + + if err = batch.Send(); err != nil { + return err + } + + // Switch on Go Type + + rows, err := conn.Query(ctx, "SELECT c FROM go_variant_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Variant + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Any().(type) { + case bool: + fmt.Printf("row at index %d is Bool: %v\n", i, row.Any()) + case int64: + fmt.Printf("row at index %d is Int64: %v\n", i, row.Any()) + case string: + fmt.Printf("row at index %d is String: %v\n", i, row.Any()) + case nil: + fmt.Printf("row at index %d is NULL\n", i) + } + } + + // Switch on ClickHouse Type + + rows, err = conn.Query(ctx, "SELECT c FROM go_variant_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.VariantWithType + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Type() { + case "Bool": + fmt.Printf("row at index %d is bool: %v\n", i, row.Any()) + case "Int64": + fmt.Printf("row at index %d is int64: %v\n", i, row.Any()) + case "String": + fmt.Printf("row at index %d is string: %v\n", i, row.Any()) + case "": + fmt.Printf("row at index %d is nil\n", i) + } + } + + return nil +} diff --git a/lib/chcol/variant.go b/lib/chcol/variant.go new file mode 100644 index 0000000000..f40b5fcda5 --- /dev/null +++ b/lib/chcol/variant.go @@ -0,0 +1,126 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package chcol + +import ( + "database/sql/driver" + "encoding/json" +) + +// Variant represents a ClickHouse Variant type that can hold multiple possible types +type Variant struct { + value any +} + +// NewVariant creates a new Variant with the given value +func NewVariant(v any) Variant { + return Variant{value: v} +} + +// Nil returns true if the underlying value is nil. +func (v Variant) Nil() bool { + return v.value == nil +} + +// Any returns the underlying value as any. Same as Interface. +func (v Variant) Any() any { + return v.value +} + +// Interface returns the underlying value as interface{}. Same as Any. +func (v Variant) Interface() interface{} { + return v.value +} + +// Int returns the value as an int if possible +func (v Variant) Int() (int, bool) { + if i, ok := v.value.(int); ok { + return i, true + } + + return 0, false +} + +// Int64 returns the value as an int64 if possible +func (v Variant) Int64() (int64, bool) { + if i, ok := v.value.(int64); ok { + return i, true + } + + return 0, false +} + +// String returns the value as a string if possible +func (v Variant) String() (string, bool) { + if s, ok := v.value.(string); ok { + return s, true + } + + return "", false +} + +// Bool returns the value as an bool if possible +func (v Variant) Bool() (bool, bool) { + if b, ok := v.value.(bool); ok { + return b, true + } + + return false, false +} + +// MarshalJSON implements the json.Marshaler interface +func (v *Variant) MarshalJSON() ([]byte, error) { + return json.Marshal(v.value) +} + +// Scan implements the sql.Scanner interface +func (v *Variant) Scan(value interface{}) error { + v.value = value + return nil +} + +// Value implements the driver.Valuer interface +func (v Variant) Value() (driver.Value, error) { + return v.value, nil +} + +func (v Variant) WithType(chType string) VariantWithType { + return VariantWithType{ + Variant: v, + chType: chType, + } +} + +// VariantWithType is Variant with an extra value for specifying the preferred ClickHouse type for column encoding +type VariantWithType struct { + Variant + chType string +} + +// NewVariantWithType creates a new Variant with the given value and ClickHouse type +func NewVariantWithType(v any, chType string) VariantWithType { + return VariantWithType{ + Variant: Variant{value: v}, + chType: chType, + } +} + +// Type returns the ClickHouse type as a string. +func (v VariantWithType) Type() string { + return v.chType +} diff --git a/lib/chcol/variant_test.go b/lib/chcol/variant_test.go new file mode 100644 index 0000000000..f5aa352493 --- /dev/null +++ b/lib/chcol/variant_test.go @@ -0,0 +1,72 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package chcol + +import ( + "testing" +) + +func TestVariant_Nil(t *testing.T) { + v := NewVariant(nil) + + if !v.Nil() { + t.Fatalf("expected variant to be nil") + } +} + +func TestVariant_Int64(t *testing.T) { + var in int64 = 42 + + v := NewVariant(in) + + out, ok := v.Int64() + if !ok { + t.Fatalf("failed to get int64 from variant") + } else if out != in { + t.Fatalf("incorrect value from variant. expected: %d got: %d", in, out) + } +} + +func TestVariant_String(t *testing.T) { + in := "test" + + v := NewVariant(in) + + out, ok := v.String() + if !ok { + t.Fatalf("failed to get string from variant") + } else if out != in { + t.Fatalf("incorrect value from variant. expected: %s got: %s", in, out) + } +} + +func TestVariant_TypeSwitch(t *testing.T) { + var in any + + v := NewVariant(in) + + switch v.Any().(type) { + case int64: + t.Fatalf("unexpected int64 value from variant") + case string: + t.Fatalf("unexpected string value from variant") + case nil: + default: + t.Fatalf("expected nil value from variant") + } +} diff --git a/lib/column/codegen/column.tpl b/lib/column/codegen/column.tpl index fb76a8999b..8697a67930 100644 --- a/lib/column/codegen/column.tpl +++ b/lib/column/codegen/column.tpl @@ -33,6 +33,7 @@ import ( "database/sql" "database/sql/driver" "github.com/ClickHouse/ch-go/proto" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" ) func (t Type) Column(name string, tz *time.Location) (Interface, error) { @@ -120,7 +121,7 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { case "Point": return &Point{name: name}, nil case "String": - return &String{name: name}, nil + return &String{name: name, col: colStrProvider()}, nil case "Object('json')": return &JSONObject{name: name, root: true, tz: tz}, nil } @@ -130,6 +131,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return (&Map{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Tuple("): return (&Tuple{name: name}).parse(t, tz) + case strings.HasPrefix(string(t), "Variant("): + return (&Variant{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Decimal("): return (&Decimal{name: name}).parse(t) case strings.HasPrefix(strType, "Nested("): @@ -191,6 +194,7 @@ var ( scanTypePolygon = reflect.TypeOf(orb.Polygon{}) scanTypeDecimal = reflect.TypeOf(decimal.Decimal{}) scanTypeMultiPolygon = reflect.TypeOf(orb.MultiPolygon{}) + scanTypeVariant = reflect.TypeOf(chcol.Variant{}) ) {{- range . }} diff --git a/lib/column/column_gen.go b/lib/column/column_gen.go index 5bd1180a3b..8367e3fe14 100644 --- a/lib/column/column_gen.go +++ b/lib/column/column_gen.go @@ -25,6 +25,7 @@ import ( "database/sql/driver" "fmt" "github.com/ClickHouse/ch-go/proto" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" "github.com/google/uuid" "github.com/paulmach/orb" "github.com/shopspring/decimal" @@ -146,6 +147,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return (&Map{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Tuple("): return (&Tuple{name: name}).parse(t, tz) + case strings.HasPrefix(string(t), "Variant("): + return (&Variant{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Decimal("): return (&Decimal{name: name}).parse(t) case strings.HasPrefix(strType, "Nested("): @@ -255,6 +258,7 @@ var ( scanTypePolygon = reflect.TypeOf(orb.Polygon{}) scanTypeDecimal = reflect.TypeOf(decimal.Decimal{}) scanTypeMultiPolygon = reflect.TypeOf(orb.MultiPolygon{}) + scanTypeVariant = reflect.TypeOf(chcol.Variant{}) ) func (col *Float32) Name() string { diff --git a/lib/column/variant.go b/lib/column/variant.go new file mode 100644 index 0000000000..a8b0668a2d --- /dev/null +++ b/lib/column/variant.go @@ -0,0 +1,385 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package column + +import ( + "database/sql/driver" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + "reflect" + "strings" + "time" + + "github.com/ClickHouse/ch-go/proto" +) + +const SupportedVariantSerializationVersion = 0 +const NullVariantDiscriminator uint8 = 255 + +type Variant struct { + chType Type + name string + + discriminators []uint8 + offsets []int + + columns []Interface + columnTypeIndex map[string]uint8 +} + +func (c *Variant) parse(t Type, tz *time.Location) (_ *Variant, err error) { + c.chType = t + var ( + element []rune + elements []Type + brackets int + appendElement = func() { + if len(element) != 0 { + cType := strings.TrimSpace(string(element)) + if parts := strings.SplitN(cType, " ", 2); len(parts) == 2 { + if !strings.Contains(parts[0], "(") { + cType = parts[1] + } + } + + elements = append(elements, Type(strings.TrimSpace(cType))) + } + } + ) + + for _, r := range t.params() { + switch r { + case '(': + brackets++ + case ')': + brackets-- + case ',': + if brackets == 0 { + appendElement() + element = element[:0] + continue + } + } + element = append(element, r) + } + + appendElement() + + c.columnTypeIndex = make(map[string]uint8, len(elements)) + for _, columnType := range elements { + column, err := columnType.Column("", tz) + if err != nil { + return nil, err + } + + c.addColumn(column) + } + + if len(c.columns) != 0 { + return c, nil + } + + return nil, &UnsupportedColumnTypeError{ + t: t, + } +} + +func (c *Variant) addColumn(col Interface) { + c.columns = append(c.columns, col) + c.columnTypeIndex[string(col.Type())] = uint8(len(c.columns) - 1) +} + +func (c *Variant) appendDiscriminatorRow(d uint8) { + c.discriminators = append(c.discriminators, d) +} + +func (c *Variant) appendNullRow() { + c.appendDiscriminatorRow(NullVariantDiscriminator) +} + +func (c *Variant) Name() string { + return c.name +} + +func (c *Variant) Type() Type { + return c.chType +} + +func (c *Variant) Rows() int { + return len(c.discriminators) +} + +func (c *Variant) Row(i int, ptr bool) any { + typeIndex := c.discriminators[i] + offsetIndex := c.offsets[i] + var value any + var chType string + if typeIndex != NullVariantDiscriminator { + value = c.columns[typeIndex].Row(offsetIndex, ptr) + chType = string(c.columns[typeIndex].Type()) + } + + vt := chcol.NewVariantWithType(value, chType) + if ptr { + return &vt + } + + return vt +} + +func (c *Variant) ScanRow(dest any, row int) error { + typeIndex := c.discriminators[row] + offsetIndex := c.offsets[row] + var value any + var chType string + if typeIndex != NullVariantDiscriminator { + value = c.columns[typeIndex].Row(offsetIndex, false) + chType = string(c.columns[typeIndex].Type()) + } + + switch v := dest.(type) { + case *chcol.Variant: + vt := chcol.NewVariant(value) + *v = vt + case **chcol.Variant: + vt := chcol.NewVariant(value) + **v = vt + case *chcol.VariantWithType: + vt := chcol.NewVariantWithType(value, chType) + *v = vt + case **chcol.VariantWithType: + vt := chcol.NewVariantWithType(value, chType) + **v = vt + default: + if typeIndex == NullVariantDiscriminator { + return nil + } + + if err := c.columns[typeIndex].ScanRow(dest, offsetIndex); err != nil { + return err + } + } + + return nil +} + +func (c *Variant) Append(v any) (nulls []uint8, err error) { + switch v.(type) { + case []chcol.Variant: + for i, vt := range v.([]chcol.Variant) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []*chcol.Variant: + for i, vt := range v.([]*chcol.Variant) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []chcol.VariantWithType: + for i, vt := range v.([]chcol.VariantWithType) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []*chcol.VariantWithType: + for i, vt := range v.([]*chcol.VariantWithType) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + default: + if valuer, ok := v.(driver.Valuer); ok { + val, err := valuer.Value() + if err != nil { + return nil, &ColumnConverterError{ + Op: "Append", + To: string(c.chType), + From: fmt.Sprintf("%T", v), + Hint: "could not get driver.Valuer value", + } + } + + return c.Append(val) + } + + return nil, &ColumnConverterError{ + Op: "Append", + To: string(c.chType), + From: fmt.Sprintf("%T", v), + } + } +} + +func (c *Variant) AppendRow(v any) error { + var requestedType string + switch v.(type) { + case nil: + c.appendNullRow() + return nil + case chcol.Variant: + v = v.(chcol.Variant).Any() + if v == nil { + c.appendNullRow() + return nil + } + case *chcol.Variant: + v = v.(*chcol.Variant).Any() + if v == nil { + c.appendNullRow() + return nil + } + case chcol.VariantWithType: + requestedType = v.(chcol.VariantWithType).Type() + v = v.(chcol.VariantWithType).Any() + if v == nil { + c.appendNullRow() + return nil + } + case *chcol.VariantWithType: + requestedType = v.(*chcol.VariantWithType).Type() + v = v.(*chcol.VariantWithType).Any() + if v == nil { + c.appendNullRow() + return nil + } + } + + if requestedType != "" { + typeIndex, ok := c.columnTypeIndex[requestedType] + if !ok { + return fmt.Errorf("value %v cannot be stored in variant column %s with requested type %s: type not present in variant", v, c.chType, requestedType) + } + + if err := c.columns[typeIndex].AppendRow(v); err != nil { + return fmt.Errorf("failed to append row to variant column with requested type %s: %w", requestedType, err) + } + + c.appendDiscriminatorRow(typeIndex) + return nil + } + + // If preferred type wasn't provided, try each column + var err error + for i, col := range c.columns { + if err = col.AppendRow(v); err == nil { + c.appendDiscriminatorRow(uint8(i)) + return nil + } + } + + return fmt.Errorf("value \"%v\" cannot be stored in variant column: no compatible types", v) +} + +func (c *Variant) encodeHeader(buffer *proto.Buffer) { + buffer.PutUInt64(SupportedVariantSerializationVersion) +} + +func (c *Variant) encodeData(buffer *proto.Buffer) { + buffer.PutRaw(c.discriminators) + + for _, col := range c.columns { + col.Encode(buffer) + } +} + +func (c *Variant) Encode(buffer *proto.Buffer) { + c.encodeHeader(buffer) + c.encodeData(buffer) +} + +func (c *Variant) ScanType() reflect.Type { + return scanTypeVariant +} + +func (c *Variant) Reset() { + c.discriminators = c.discriminators[:0] + + for _, col := range c.columns { + col.Reset() + } +} + +func (c *Variant) decodeHeader(reader *proto.Reader) error { + variantSerializationVersion, err := reader.UInt64() + if err != nil { + return fmt.Errorf("failed to read variant discriminator version: %w", err) + } else if variantSerializationVersion != SupportedVariantSerializationVersion { + return fmt.Errorf("unsupported variant discriminator version: %d", variantSerializationVersion) + } + + return nil +} + +func (c *Variant) decodeData(reader *proto.Reader, rows int) error { + c.discriminators = make([]uint8, rows) + c.offsets = make([]int, rows) + rowCountByType := make(map[uint8]int, len(c.columns)) + + for i := 0; i < rows; i++ { + disc, err := reader.ReadByte() + if err != nil { + return fmt.Errorf("failed to read discriminator at index %d: %w", i, err) + } + + c.discriminators[i] = disc + if rowCountByType[disc] == 0 { + rowCountByType[disc] = 1 + } else { + rowCountByType[disc]++ + } + + c.offsets[i] = rowCountByType[disc] - 1 + } + + for i, col := range c.columns { + cRows := rowCountByType[uint8(i)] + if err := col.Decode(reader, cRows); err != nil { + return fmt.Errorf("failed to decode variant column with %s type: %w", col.Type(), err) + } + } + + return nil +} + +func (c *Variant) Decode(reader *proto.Reader, rows int) error { + err := c.decodeHeader(reader) + if err != nil { + return fmt.Errorf("failed to decode variant header: %w", err) + } + + err = c.decodeData(reader, rows) + if err != nil { + return fmt.Errorf("failed to decode variant data: %w", err) + } + + return nil +} diff --git a/lib/column/variant_test.go b/lib/column/variant_test.go new file mode 100644 index 0000000000..8ef1ab6542 --- /dev/null +++ b/lib/column/variant_test.go @@ -0,0 +1,91 @@ +package column + +import ( + "github.com/stretchr/testify/require" + "testing" +) + +func TestColVariant_parse(t *testing.T) { + cases := []struct { + typ Type + expectedTypes []Type + }{ + {typ: "Variant(Int64)", expectedTypes: []Type{"Int64"}}, + {typ: "Variant(Int64, String)", expectedTypes: []Type{"Int64", "String"}}, + {typ: "Variant(Array(String), Int64, String)", expectedTypes: []Type{"Array(String)", "Int64", "String"}}, + {typ: "Variant(Array(Map(String, String)), Map(String, Int64))", expectedTypes: []Type{"Array(Map(String, String))", "Map(String, Int64)"}}, + {typ: "Variant(Array(Map(String, Tuple(a String, b Int64))), Map(String, Int64))", expectedTypes: []Type{"Array(Map(String, Tuple(a String, b Int64)))", "Map(String, Int64)"}}, + } + + for i, c := range cases { + col, err := (&Variant{name: "vt"}).parse(c.typ, nil) + if err != nil { + t.Fatalf("case index %d failed to parse Variant column: %s", i, err) + } + + require.Equal(t, "vt", col.Name()) + require.Equal(t, c.typ, col.chType) + require.Equal(t, len(c.expectedTypes), len(col.columns)) + require.Equal(t, len(c.expectedTypes), len(col.columnTypeIndex)) + + for j, subCol := range col.columns { + expectedType := c.expectedTypes[j] + actualType := subCol.Type() + if actualType != expectedType { + t.Fatalf("case index %d Variant type index %d column type does not match: expected: \"%s\" actual: \"%s\"", i, j, expectedType, actualType) + } + + expectedColumnTypeIndex := uint8(j) + actualColumnTypeIndex := col.columnTypeIndex[string(actualType)] + if actualColumnTypeIndex != expectedColumnTypeIndex { + t.Fatalf("case index %d Variant type index %d columnTypeIndex does not match: expected: %d actual: %d", i, j, expectedColumnTypeIndex, actualColumnTypeIndex) + } + } + } +} + +func TestColVariant_parse_invalid(t *testing.T) { + cases := []Type{ + "", + "Variant", + "Variant(Array(Map(String)), Map(String, Int64))", + "Variant(Array(Tuple(String, b Int64)), Map(String, Int64), FakeType)", + } + + for i, typeName := range cases { + _, err := (&Variant{name: "vt"}).parse(typeName, nil) + if err == nil { + t.Fatalf("expected error for case index %d (\"%s\"), but received nil", i, typeName) + } + } +} + +func TestColVariant_addColumn(t *testing.T) { + col := Variant{columnTypeIndex: make(map[string]uint8, 1)} + + col.addColumn(&Int64{}) + + require.Equal(t, 1, len(col.columns)) + require.Equal(t, 1, len(col.columnTypeIndex)) + require.Equal(t, Type("Int64"), col.columns[0].Type()) + require.Equal(t, uint8(0), col.columnTypeIndex["Int64"]) +} + +func TestColVariant_appendDiscriminatorRow(t *testing.T) { + col := Variant{} + var discriminator uint8 = 8 + + col.appendDiscriminatorRow(discriminator) + + require.Equal(t, 1, len(col.discriminators)) + require.Equal(t, discriminator, col.discriminators[0]) +} + +func TestColVariant_appendNullRow(t *testing.T) { + col := Variant{} + + col.appendNullRow() + + require.Equal(t, 1, len(col.discriminators)) + require.Equal(t, NullVariantDiscriminator, col.discriminators[0]) +} diff --git a/tests/variant_test.go b/tests/variant_test.go new file mode 100644 index 0000000000..e5e7c08ffd --- /dev/null +++ b/tests/variant_test.go @@ -0,0 +1,247 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package tests + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +var variantTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") + +func setupVariantTest(t *testing.T) driver.Conn { + conn, err := GetNativeConnection(clickhouse.Settings{ + "max_execution_time": 60, + "allow_experimental_variant_type": true, + }, nil, &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + }) + require.NoError(t, err) + + if !CheckMinServerServerVersion(conn, 24, 1, 0) { + t.Skip(fmt.Errorf("unsupported clickhouse version for Variant type")) + return nil + } + + return conn +} + +func TestVariant(t *testing.T) { + ctx := context.Background() + conn := setupVariantTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_variant ( + c Variant( + Bool, + Int64, + String, + DateTime64(3), + Array(String), + Array(UInt8), + Array(Map(String, String)), + Map(String, String), + Map(String, Int64), + ) + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_variant")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_variant (c)") + require.NoError(t, err) + + require.NoError(t, batch.Append(true)) + colInt64 := int64(42) + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(colInt64, "Int64"))) + colString := "test" + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(colString, "String"))) + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(variantTestDate, "DateTime64(3)"))) + var colNil any = nil + require.NoError(t, batch.Append(colNil)) + colSliceString := []string{"a", "b"} + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(colSliceString, "Array(String)"))) + colSliceUInt8 := []uint8{0xA, 0xB, 0xC} + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(colSliceUInt8, "Array(UInt8)"))) + colSliceMapStringString := []map[string]string{{"key1": "value1", "key2": "value2"}, {"key3": "value3"}} + require.NoError(t, batch.Append(colSliceMapStringString)) + colMapStringString := map[string]string{"key1": "value1", "key2": "value2"} + require.NoError(t, batch.Append(colMapStringString)) + colMapStringInt64 := map[string]int64{"key1": 42, "key2": 84} + require.NoError(t, batch.Append(colMapStringInt64)) + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_variant") + require.NoError(t, err) + + var row chcol.Variant + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colInt64, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, variantTestDate, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colNil, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceUInt8, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringInt64, row.Any()) +} + +func TestVariant_ScanWithType(t *testing.T) { + ctx := context.Background() + conn := setupVariantTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_variant ( + c Variant(Bool, Int64) + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_variant")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_variant (c)") + require.NoError(t, err) + + require.NoError(t, batch.Append(true)) + require.NoError(t, batch.Append(clickhouse.NewVariantWithType(int64(42), "Int64"))) + require.NoError(t, batch.Append(nil)) + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_variant") + require.NoError(t, err) + + var row chcol.VariantWithType + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + require.Equal(t, "Bool", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, int64(42), row.Any()) + require.Equal(t, "Int64", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, nil, row.Any()) + require.Equal(t, "", row.Type()) +} + +func TestVariant_BatchFlush(t *testing.T) { + ctx := context.Background() + conn := setupVariantTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_variant ( + c Variant(Bool, Int64) + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_variant")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_variant (c)") + require.NoError(t, err) + + vals := make([]clickhouse.VariantWithType, 0, 1000) + for i := 0; i < 1000; i++ { + if i%2 == 0 { + vals = append(vals, clickhouse.NewVariantWithType(int64(i), "Int64")) + } else { + vals = append(vals, clickhouse.NewVariantWithType(i%5 == 0, "Bool")) + } + + require.NoError(t, batch.Append(vals[i])) + require.NoError(t, batch.Flush()) + } + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_variant") + require.NoError(t, err) + + i := 0 + for rows.Next() { + var row chcol.VariantWithType + err = rows.Scan(&row) + + if i%2 == 0 { + require.Equal(t, int64(i), row.Any()) + require.Equal(t, "Int64", row.Type()) + } else { + require.Equal(t, i%5 == 0, row.Any()) + require.Equal(t, "Bool", row.Type()) + } + + i++ + } +} From 33abd0482177b2c9742dd81e200e772998d84368 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 20 Dec 2024 01:28:42 -0500 Subject: [PATCH 02/18] add dynamic type --- chcol.go | 12 + examples/clickhouse_api/dynamic.go | 136 +++++++++ examples/clickhouse_api/main_test.go | 4 + lib/chcol/dynamic.go | 35 +++ lib/column/codegen/column.tpl | 5 + lib/column/codegen/dynamic.tpl | 60 ++++ lib/column/codegen/main.go | 45 ++- lib/column/column_gen.go | 5 + lib/column/dynamic.go | 442 +++++++++++++++++++++++++++ lib/column/dynamic_gen.go | 269 ++++++++++++++++ lib/column/sharedvariant.go | 72 +++++ tests/dynamic_test.go | 233 ++++++++++++++ 12 files changed, 1315 insertions(+), 3 deletions(-) create mode 100644 examples/clickhouse_api/dynamic.go create mode 100644 lib/chcol/dynamic.go create mode 100644 lib/column/codegen/dynamic.tpl create mode 100644 lib/column/dynamic.go create mode 100644 lib/column/dynamic_gen.go create mode 100644 lib/column/sharedvariant.go create mode 100644 tests/dynamic_test.go diff --git a/chcol.go b/chcol.go index 7adbb8a1da..fff58c831f 100644 --- a/chcol.go +++ b/chcol.go @@ -24,6 +24,8 @@ import "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" type ( Variant = chcol.Variant VariantWithType = chcol.VariantWithType + Dynamic = chcol.Dynamic + DynamicWithType = chcol.DynamicWithType ) // NewVariant creates a new Variant with the given value @@ -35,3 +37,13 @@ func NewVariant(v any) Variant { func NewVariantWithType(v any, chType string) VariantWithType { return chcol.NewVariantWithType(v, chType) } + +// NewDynamic creates a new Dynamic with the given value +func NewDynamic(v any) Dynamic { + return chcol.NewDynamic(v) +} + +// NewDynamicWithType creates a new Dynamic with the given value and ClickHouse type +func NewDynamicWithType(v any, chType string) DynamicWithType { + return chcol.NewDynamicWithType(v, chType) +} diff --git a/examples/clickhouse_api/dynamic.go b/examples/clickhouse_api/dynamic.go new file mode 100644 index 0000000000..ef354b769e --- /dev/null +++ b/examples/clickhouse_api/dynamic.go @@ -0,0 +1,136 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package clickhouse_api + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" +) + +func DynamicExample() error { + ctx := context.Background() + + conn, err := GetNativeConnection(clickhouse.Settings{ + "allow_experimental_dynamic_type": true, + }, nil, nil) + if err != nil { + return err + } + + err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_dynamic_example") + if err != nil { + return err + } + + err = conn.Exec(ctx, ` + CREATE TABLE go_dynamic_example ( + c Dynamic + ) ENGINE = Memory + `) + if err != nil { + return err + } + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO go_dynamic_example (c)") + if err != nil { + return err + } + + if err = batch.Append(true); err != nil { + return err + } + + if err = batch.Append(int64(42)); err != nil { + return err + } + + if err = batch.Append("example"); err != nil { + return err + } + + if err = batch.Append(clickhouse.NewVariant("example dynamic")); err != nil { + return err + } + + if err = batch.Append(clickhouse.NewVariantWithType("example dynamic with specific type", "String")); err != nil { + return err + } + + if err = batch.Append(nil); err != nil { + return err + } + + if err = batch.Send(); err != nil { + return err + } + + // Switch on Go Type + + rows, err := conn.Query(ctx, "SELECT c FROM go_dynamic_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Variant + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Any().(type) { + case bool: + fmt.Printf("row at index %d is Bool: %v\n", i, row.Any()) + case int64: + fmt.Printf("row at index %d is Int64: %v\n", i, row.Any()) + case string: + fmt.Printf("row at index %d is String: %v\n", i, row.Any()) + case nil: + fmt.Printf("row at index %d is NULL\n", i) + } + } + + // Switch on ClickHouse Type + + rows, err = conn.Query(ctx, "SELECT c FROM go_dynamic_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.VariantWithType + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Type() { + case "Bool": + fmt.Printf("row at index %d is bool: %v\n", i, row.Any()) + case "Int64": + fmt.Printf("row at index %d is int64: %v\n", i, row.Any()) + case "String": + fmt.Printf("row at index %d is string: %v\n", i, row.Any()) + case "": + fmt.Printf("row at index %d is nil\n", i) + } + } + + return nil +} diff --git a/examples/clickhouse_api/main_test.go b/examples/clickhouse_api/main_test.go index 40b59e36b2..d1b15637d9 100644 --- a/examples/clickhouse_api/main_test.go +++ b/examples/clickhouse_api/main_test.go @@ -213,3 +213,7 @@ func TestSSLNoVerify(t *testing.T) { func TestVariantExample(t *testing.T) { require.NoError(t, VariantExample()) } + +func TestDynamicExample(t *testing.T) { + require.NoError(t, DynamicExample()) +} diff --git a/lib/chcol/dynamic.go b/lib/chcol/dynamic.go new file mode 100644 index 0000000000..7e081376a7 --- /dev/null +++ b/lib/chcol/dynamic.go @@ -0,0 +1,35 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package chcol + +type Dynamic = Variant + +// NewDynamic creates a new Dynamic with the given value +func NewDynamic(v any) Dynamic { + return Dynamic{value: v} +} + +type DynamicWithType = VariantWithType + +// NewDynamicWithType creates a new Dynamic with the given value and ClickHouse type +func NewDynamicWithType(v any, chType string) DynamicWithType { + return DynamicWithType{ + Variant: Variant{value: v}, + chType: chType, + } +} diff --git a/lib/column/codegen/column.tpl b/lib/column/codegen/column.tpl index 8697a67930..78eed58c56 100644 --- a/lib/column/codegen/column.tpl +++ b/lib/column/codegen/column.tpl @@ -122,6 +122,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return &Point{name: name}, nil case "String": return &String{name: name, col: colStrProvider()}, nil + case "SharedVariant": + return &SharedVariant{name: name}, nil case "Object('json')": return &JSONObject{name: name, root: true, tz: tz}, nil } @@ -133,6 +135,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return (&Tuple{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Variant("): return (&Variant{name: name}).parse(t, tz) + case strings.HasPrefix(string(t), "Dynamic"): + return (&Dynamic{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Decimal("): return (&Decimal{name: name}).parse(t) case strings.HasPrefix(strType, "Nested("): @@ -195,6 +199,7 @@ var ( scanTypeDecimal = reflect.TypeOf(decimal.Decimal{}) scanTypeMultiPolygon = reflect.TypeOf(orb.MultiPolygon{}) scanTypeVariant = reflect.TypeOf(chcol.Variant{}) + scanTypeDynamic = reflect.TypeOf(chcol.Dynamic{}) ) {{- range . }} diff --git a/lib/column/codegen/dynamic.tpl b/lib/column/codegen/dynamic.tpl new file mode 100644 index 0000000000..b725c069c5 --- /dev/null +++ b/lib/column/codegen/dynamic.tpl @@ -0,0 +1,60 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Code generated by make codegen DO NOT EDIT. +// source: lib/column/codegen/dynamic.tpl + +package column + +import ( + "database/sql" + "encoding/json" + "github.com/ClickHouse/ch-go/proto" + "github.com/google/uuid" + "github.com/paulmach/orb" + "time" +) + +// inferClickHouseTypeFromGoType takes a Go interface{} and converts it to a ClickHouse type. +// Returns empty string if type was not matched. +// This is best effort and does not work for all types. +// Optimally, users should provide a type using DynamicWithType. +func inferClickHouseTypeFromGoType(v any) string { + switch v.(type) { + {{- range . }} + case {{ .GoType }}: + return "{{ .ChType }}" + case *{{ .GoType }}: + return "{{ .ChType }}" + {{- end }} + {{- range . }} + {{- if .SkipArray }} + {{- else}} + case []{{ .GoType }}: + return "Array({{ .ChType }})" + {{- end}} + case []*{{ .GoType }}: + return "Array({{ .ChType }})" + {{- end }} + {{- range . }} + case map[string]{{ .GoType }}: + return "Map(String, {{ .ChType }})" + {{- end }} + default: + return "" + } +} diff --git a/lib/column/codegen/main.go b/lib/column/codegen/main.go index f427acdd54..46f8875040 100644 --- a/lib/column/codegen/main.go +++ b/lib/column/codegen/main.go @@ -34,16 +34,22 @@ var ( columnSrc string //go:embed array.tpl arraySrc string + //go:embed dynamic.tpl + dynamicSrc string ) var ( types []_type supportedGoTypes []string + dynamicTypes []_type ) type _type struct { - Size int + Size int + ChType string GoType string + + SkipArray bool } func init() { @@ -72,6 +78,7 @@ func init() { for _, typ := range types { supportedGoTypes = append(supportedGoTypes, typ.GoType) } + supportedGoTypes = append(supportedGoTypes, "string", "[]byte", "sql.NullString", "int", "uint", "big.Int", "decimal.Decimal", @@ -81,6 +88,37 @@ func init() { "netip.Addr", "net.IP", "proto.IPv6", "[16]byte", "orb.MultiPolygon", "orb.Point", "orb.Polygon", "orb.Ring", ) + + dynamicTypes = make([]_type, 0, len(types)) + for _, typ := range types { + + if typ.GoType == "uint8" { + // Prevent conflict with []byte and []uint8 + typ.SkipArray = true + dynamicTypes = append(dynamicTypes, typ) + continue + } + + dynamicTypes = append(dynamicTypes, typ) + } + + // Best-effort type matching for Dynamic inference + dynamicTypes = append(dynamicTypes, []_type{ + {ChType: "String", GoType: "string"}, + {ChType: "String", GoType: "json.RawMessage"}, + {ChType: "String", GoType: "sql.NullString"}, + {ChType: "Bool", GoType: "bool"}, + {ChType: "Bool", GoType: "sql.NullBool"}, + {ChType: "DateTime64(3)", GoType: "time.Time"}, + {ChType: "DateTime64(3)", GoType: "sql.NullTime"}, + {ChType: "UUID", GoType: "uuid.UUID"}, + {ChType: "IPv6", GoType: "proto.IPv6"}, + {ChType: "MultiPolygon", GoType: "orb.MultiPolygon"}, + {ChType: "Point", GoType: "orb.Point"}, + {ChType: "Polygon", GoType: "orb.Polygon"}, + {ChType: "Ring", GoType: "orb.Ring"}, + }...) + } func write(name string, v any, t *template.Template) error { out := new(bytes.Buffer) @@ -107,8 +145,9 @@ func main() { template *template.Template args any }{ - "column_gen": {template.Must(template.New("column").Parse(columnSrc)), types}, - "array_gen": {template.Must(template.New("array").Parse(arraySrc)), supportedGoTypes}, + "column_gen": {template.Must(template.New("column").Parse(columnSrc)), types}, + "array_gen": {template.Must(template.New("array").Parse(arraySrc)), supportedGoTypes}, + "dynamic_gen": {template.Must(template.New("dynamic").Parse(dynamicSrc)), dynamicTypes}, } { if err := write(name, tpl.args, tpl.template); err != nil { log.Fatal(err) diff --git a/lib/column/column_gen.go b/lib/column/column_gen.go index 8367e3fe14..cc601212f5 100644 --- a/lib/column/column_gen.go +++ b/lib/column/column_gen.go @@ -138,6 +138,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return &Point{name: name}, nil case "String": return &String{name: name, col: colStrProvider()}, nil + case "SharedVariant": + return &SharedVariant{name: name}, nil case "Object('json')": return &JSONObject{name: name, root: true, tz: tz}, nil } @@ -149,6 +151,8 @@ func (t Type) Column(name string, tz *time.Location) (Interface, error) { return (&Tuple{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Variant("): return (&Variant{name: name}).parse(t, tz) + case strings.HasPrefix(string(t), "Dynamic"): + return (&Dynamic{name: name}).parse(t, tz) case strings.HasPrefix(string(t), "Decimal("): return (&Decimal{name: name}).parse(t) case strings.HasPrefix(strType, "Nested("): @@ -259,6 +263,7 @@ var ( scanTypeDecimal = reflect.TypeOf(decimal.Decimal{}) scanTypeMultiPolygon = reflect.TypeOf(orb.MultiPolygon{}) scanTypeVariant = reflect.TypeOf(chcol.Variant{}) + scanTypeDynamic = reflect.TypeOf(chcol.Dynamic{}) ) func (col *Float32) Name() string { diff --git a/lib/column/dynamic.go b/lib/column/dynamic.go new file mode 100644 index 0000000000..2378df104b --- /dev/null +++ b/lib/column/dynamic.go @@ -0,0 +1,442 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package column + +import ( + "database/sql/driver" + "fmt" + "reflect" + "slices" + "strconv" + "strings" + "time" + + "github.com/ClickHouse/ch-go/proto" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" +) + +const SupportedDynamicSerializationVersion = 1 +const DefaultMaxDynamicTypes = 32 + +type Dynamic struct { + chType Type + tz *time.Location + + name string + + maxTypes uint8 + totalTypes uint8 + typeNames []string + typeNamesIndex map[string]int + + variant Variant +} + +func (c *Dynamic) parse(t Type, tz *time.Location) (_ *Dynamic, err error) { + c.chType = t + c.tz = tz + tStr := string(t) + + // SharedVariant is special, and does not count against totalTypes + c.typeNamesIndex = make(map[string]int) + c.variant.columnTypeIndex = make(map[string]uint8) + sv, _ := Type("SharedVariant").Column("", tz) + c.addColumn(sv) + + c.maxTypes = DefaultMaxDynamicTypes + c.totalTypes = 0 // Reset to 0 after adding SharedVariant + + if tStr == "Dynamic" { + return c, nil + } + + if !strings.HasPrefix(tStr, "Dynamic(") || !strings.HasSuffix(tStr, ")") { + return nil, &UnsupportedColumnTypeError{t: t} + } + + typeParamsStr := strings.TrimPrefix(tStr, "Dynamic(") + typeParamsStr = strings.TrimSuffix(typeParamsStr, ")") + + if strings.HasPrefix(typeParamsStr, "max_types=") { + v := strings.TrimPrefix(typeParamsStr, "max_types=") + if maxTypes, err := strconv.Atoi(v); err == nil { + c.maxTypes = uint8(maxTypes) + } + } + + return c, nil +} + +func (c *Dynamic) addColumn(col Interface) { + typeName := string(col.Type()) + c.typeNames = append(c.typeNames, typeName) + c.typeNamesIndex[typeName] = len(c.typeNames) - 1 + c.totalTypes++ + c.variant.addColumn(col) +} + +func (c *Dynamic) Name() string { + return c.name +} + +func (c *Dynamic) Type() Type { + return c.chType +} + +func (c *Dynamic) Rows() int { + return c.variant.Rows() +} + +func (c *Dynamic) Row(i int, ptr bool) any { + typeIndex := c.variant.discriminators[i] + offsetIndex := c.variant.offsets[i] + var value any + var chType string + if typeIndex != NullVariantDiscriminator { + value = c.variant.columns[typeIndex].Row(offsetIndex, ptr) + chType = string(c.variant.columns[typeIndex].Type()) + } + + dyn := chcol.NewDynamicWithType(value, chType) + if ptr { + return &dyn + } + + return dyn +} + +func (c *Dynamic) ScanRow(dest any, row int) error { + typeIndex := c.variant.discriminators[row] + offsetIndex := c.variant.offsets[row] + var value any + var chType string + if typeIndex != NullVariantDiscriminator { + value = c.variant.columns[typeIndex].Row(offsetIndex, false) + chType = string(c.variant.columns[typeIndex].Type()) + } + + switch v := dest.(type) { + case *chcol.Dynamic: + vt := chcol.NewDynamic(value) + *v = vt + case **chcol.Dynamic: + vt := chcol.NewDynamic(value) + **v = vt + case *chcol.DynamicWithType: + vt := chcol.NewDynamicWithType(value, chType) + *v = vt + case **chcol.DynamicWithType: + vt := chcol.NewDynamicWithType(value, chType) + **v = vt + default: + if typeIndex == NullVariantDiscriminator { + return nil + } + + if err := c.variant.columns[typeIndex].ScanRow(dest, offsetIndex); err != nil { + return err + } + } + + return nil +} + +func (c *Dynamic) Append(v any) (nulls []uint8, err error) { + switch v.(type) { + case []chcol.Dynamic: + for i, vt := range v.([]chcol.Dynamic) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []*chcol.Dynamic: + for i, vt := range v.([]*chcol.Dynamic) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []chcol.DynamicWithType: + for i, vt := range v.([]chcol.DynamicWithType) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + case []*chcol.DynamicWithType: + for i, vt := range v.([]*chcol.DynamicWithType) { + err := c.AppendRow(vt) + if err != nil { + return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) + } + } + + return nil, nil + default: + if valuer, ok := v.(driver.Valuer); ok { + val, err := valuer.Value() + if err != nil { + return nil, &ColumnConverterError{ + Op: "Append", + To: string(c.chType), + From: fmt.Sprintf("%T", v), + Hint: "could not get driver.Valuer value", + } + } + + return c.Append(val) + } + + return nil, &ColumnConverterError{ + Op: "Append", + To: string(c.chType), + From: fmt.Sprintf("%T", v), + } + } +} + +func (c *Dynamic) AppendRow(v any) error { + var requestedType string + switch v.(type) { + case nil: + c.variant.appendNullRow() + return nil + case chcol.Dynamic: + v = v.(chcol.Dynamic).Any() + if v == nil { + c.variant.appendNullRow() + return nil + } + case *chcol.Dynamic: + v = v.(*chcol.Dynamic).Any() + if v == nil { + c.variant.appendNullRow() + return nil + } + case chcol.DynamicWithType: + requestedType = v.(chcol.DynamicWithType).Type() + v = v.(chcol.DynamicWithType).Any() + if v == nil { + c.variant.appendNullRow() + return nil + } + case *chcol.DynamicWithType: + requestedType = v.(*chcol.DynamicWithType).Type() + v = v.(*chcol.DynamicWithType).Any() + if v == nil { + c.variant.appendNullRow() + return nil + } + } + + if requestedType != "" { + var col Interface + colIndex, ok := c.typeNamesIndex[requestedType] + if ok { + col = c.variant.columns[colIndex] + } else { + newCol, err := Type(requestedType).Column("", c.tz) + if err != nil { + return fmt.Errorf("value \"%v\" cannot be stored in dynamic column %s with requested type %s: unable to append type: %w", v, c.chType, requestedType, err) + } + + c.addColumn(newCol) + colIndex = int(c.totalTypes) + col = newCol + } + + if err := col.AppendRow(v); err != nil { + return fmt.Errorf("value \"%v\" cannot be stored in dynamic column %s with requested type %s: %w", v, c.chType, requestedType, err) + } + + c.variant.appendDiscriminatorRow(uint8(colIndex)) + return nil + } + + // If preferred type wasn't provided, try each column + for i, col := range c.variant.columns { + if c.typeNames[i] == "SharedVariant" { + // Do not try to fit into SharedVariant + continue + } + + if err := col.AppendRow(v); err == nil { + c.variant.appendDiscriminatorRow(uint8(i)) + return nil + } + } + + // If no existing columns match, try matching a ClickHouse type from common Go types + inferredTypeName := inferClickHouseTypeFromGoType(v) + if inferredTypeName != "" { + return c.AppendRow(chcol.NewDynamicWithType(v, inferredTypeName)) + } + + return fmt.Errorf("value \"%v\" cannot be stored in dynamic column: no compatible types. hint: use clickhouse.DynamicWithType to wrap the value", v) +} + +func (c *Dynamic) sortColumnsForEncoding() { + previousTypeNames := make([]string, 0, len(c.typeNames)) + previousTypeNames = append(previousTypeNames, c.typeNames...) + slices.Sort(c.typeNames) + + for i, typeName := range c.typeNames { + c.typeNamesIndex[typeName] = i + c.variant.columnTypeIndex[typeName] = uint8(i) + } + + sortedDiscriminatorMap := make([]uint8, len(c.variant.columns)) + sortedColumns := make([]Interface, len(c.variant.columns)) + for i, typeName := range previousTypeNames { + correctIndex := c.typeNamesIndex[typeName] + + sortedDiscriminatorMap[i] = uint8(correctIndex) + sortedColumns[correctIndex] = c.variant.columns[i] + } + c.variant.columns = sortedColumns + + for i := range c.variant.discriminators { + if c.variant.discriminators[i] == NullVariantDiscriminator { + continue + } + + c.variant.discriminators[i] = sortedDiscriminatorMap[c.variant.discriminators[i]] + } +} + +func (c *Dynamic) encodeHeader(buffer *proto.Buffer) { + c.sortColumnsForEncoding() + + buffer.PutUInt64(SupportedDynamicSerializationVersion) + buffer.PutUVarInt(uint64(c.maxTypes)) + buffer.PutUVarInt(uint64(c.totalTypes)) + + for _, typeName := range c.typeNames { + if typeName == "SharedVariant" { + // SharedVariant is implicitly present in Dynamic, do not append to type names + continue + } + + buffer.PutString(typeName) + } + + c.variant.encodeHeader(buffer) +} + +func (c *Dynamic) encodeData(buffer *proto.Buffer) { + c.variant.encodeData(buffer) +} + +func (c *Dynamic) Encode(buffer *proto.Buffer) { + c.encodeHeader(buffer) + c.encodeData(buffer) +} + +func (c *Dynamic) ScanType() reflect.Type { + return scanTypeDynamic +} + +func (c *Dynamic) Reset() { + c.variant.Reset() +} + +func (c *Dynamic) decodeHeader(reader *proto.Reader) error { + dynamicSerializationVersion, err := reader.UInt64() + if err != nil { + return fmt.Errorf("failed to read dynamic serialization version: %w", err) + } else if dynamicSerializationVersion != SupportedDynamicSerializationVersion { + return fmt.Errorf("unsupported dynamic serialization version: %d", dynamicSerializationVersion) + } + + maxTypes, err := reader.UVarInt() + if err != nil { + return fmt.Errorf("failed to read max types for dynamic column: %w", err) + } + c.maxTypes = uint8(maxTypes) + + totalTypes, err := reader.UVarInt() + if err != nil { + return fmt.Errorf("failed to read total types for dynamic column: %w", err) + } + + sortedTypeNames := make([]string, 0, totalTypes+1) + for i := uint64(0); i < totalTypes; i++ { + typeName, err := reader.Str() + if err != nil { + return fmt.Errorf("failed to read type name at index %d for dynamic column: %w", i, err) + } + + sortedTypeNames = append(sortedTypeNames, typeName) + } + + sortedTypeNames = append(sortedTypeNames, "SharedVariant") + slices.Sort(sortedTypeNames) // Re-sort after adding SharedVariant + + c.typeNames = make([]string, 0, len(sortedTypeNames)) + c.typeNamesIndex = make(map[string]int, len(sortedTypeNames)) + c.variant.columns = make([]Interface, 0, len(sortedTypeNames)) + c.variant.columnTypeIndex = make(map[string]uint8, len(sortedTypeNames)) + + for _, typeName := range sortedTypeNames { + col, err := Type(typeName).Column("", c.tz) + if err != nil { + return fmt.Errorf("failed to add dynamic column with type %s: %w", typeName, err) + } + + c.addColumn(col) + } + + c.totalTypes = uint8(totalTypes) // Reset to server's totalTypes + + err = c.variant.decodeHeader(reader) + if err != nil { + return fmt.Errorf("failed to decode variant header: %w", err) + } + + return nil +} + +func (c *Dynamic) decodeData(reader *proto.Reader, rows int) error { + err := c.variant.decodeData(reader, rows) + if err != nil { + return fmt.Errorf("failed to decode variant data: %w", err) + } + + return nil +} + +func (c *Dynamic) Decode(reader *proto.Reader, rows int) error { + err := c.decodeHeader(reader) + if err != nil { + return fmt.Errorf("failed to decode dynamic header: %w", err) + } + + err = c.decodeData(reader, rows) + if err != nil { + return fmt.Errorf("failed to decode dynamic data: %w", err) + } + + return nil +} diff --git a/lib/column/dynamic_gen.go b/lib/column/dynamic_gen.go new file mode 100644 index 0000000000..425b14892d --- /dev/null +++ b/lib/column/dynamic_gen.go @@ -0,0 +1,269 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Code generated by make codegen DO NOT EDIT. +// source: lib/column/codegen/dynamic.tpl + +package column + +import ( + "database/sql" + "encoding/json" + "github.com/ClickHouse/ch-go/proto" + "github.com/google/uuid" + "github.com/paulmach/orb" + "time" +) + +// inferClickHouseTypeFromGoType takes a Go interface{} and converts it to a ClickHouse type. +// Returns empty string if type was not matched. +// This is best effort and does not work for all types. +// Optimally, users should provide a type using DynamicWithType. +func inferClickHouseTypeFromGoType(v any) string { + switch v.(type) { + case float32: + return "Float32" + case *float32: + return "Float32" + case float64: + return "Float64" + case *float64: + return "Float64" + case int8: + return "Int8" + case *int8: + return "Int8" + case int16: + return "Int16" + case *int16: + return "Int16" + case int32: + return "Int32" + case *int32: + return "Int32" + case int64: + return "Int64" + case *int64: + return "Int64" + case uint8: + return "UInt8" + case *uint8: + return "UInt8" + case uint16: + return "UInt16" + case *uint16: + return "UInt16" + case uint32: + return "UInt32" + case *uint32: + return "UInt32" + case uint64: + return "UInt64" + case *uint64: + return "UInt64" + case string: + return "String" + case *string: + return "String" + case json.RawMessage: + return "String" + case *json.RawMessage: + return "String" + case sql.NullString: + return "String" + case *sql.NullString: + return "String" + case bool: + return "Bool" + case *bool: + return "Bool" + case sql.NullBool: + return "Bool" + case *sql.NullBool: + return "Bool" + case time.Time: + return "DateTime64(3)" + case *time.Time: + return "DateTime64(3)" + case sql.NullTime: + return "DateTime64(3)" + case *sql.NullTime: + return "DateTime64(3)" + case uuid.UUID: + return "UUID" + case *uuid.UUID: + return "UUID" + case proto.IPv6: + return "IPv6" + case *proto.IPv6: + return "IPv6" + case orb.MultiPolygon: + return "MultiPolygon" + case *orb.MultiPolygon: + return "MultiPolygon" + case orb.Point: + return "Point" + case *orb.Point: + return "Point" + case orb.Polygon: + return "Polygon" + case *orb.Polygon: + return "Polygon" + case orb.Ring: + return "Ring" + case *orb.Ring: + return "Ring" + case []float32: + return "Array(Float32)" + case []*float32: + return "Array(Float32)" + case []float64: + return "Array(Float64)" + case []*float64: + return "Array(Float64)" + case []int8: + return "Array(Int8)" + case []*int8: + return "Array(Int8)" + case []int16: + return "Array(Int16)" + case []*int16: + return "Array(Int16)" + case []int32: + return "Array(Int32)" + case []*int32: + return "Array(Int32)" + case []int64: + return "Array(Int64)" + case []*int64: + return "Array(Int64)" + case []*uint8: + return "Array(UInt8)" + case []uint16: + return "Array(UInt16)" + case []*uint16: + return "Array(UInt16)" + case []uint32: + return "Array(UInt32)" + case []*uint32: + return "Array(UInt32)" + case []uint64: + return "Array(UInt64)" + case []*uint64: + return "Array(UInt64)" + case []string: + return "Array(String)" + case []*string: + return "Array(String)" + case []json.RawMessage: + return "Array(String)" + case []*json.RawMessage: + return "Array(String)" + case []sql.NullString: + return "Array(String)" + case []*sql.NullString: + return "Array(String)" + case []bool: + return "Array(Bool)" + case []*bool: + return "Array(Bool)" + case []sql.NullBool: + return "Array(Bool)" + case []*sql.NullBool: + return "Array(Bool)" + case []time.Time: + return "Array(DateTime64(3))" + case []*time.Time: + return "Array(DateTime64(3))" + case []sql.NullTime: + return "Array(DateTime64(3))" + case []*sql.NullTime: + return "Array(DateTime64(3))" + case []uuid.UUID: + return "Array(UUID)" + case []*uuid.UUID: + return "Array(UUID)" + case []proto.IPv6: + return "Array(IPv6)" + case []*proto.IPv6: + return "Array(IPv6)" + case []orb.MultiPolygon: + return "Array(MultiPolygon)" + case []*orb.MultiPolygon: + return "Array(MultiPolygon)" + case []orb.Point: + return "Array(Point)" + case []*orb.Point: + return "Array(Point)" + case []orb.Polygon: + return "Array(Polygon)" + case []*orb.Polygon: + return "Array(Polygon)" + case []orb.Ring: + return "Array(Ring)" + case []*orb.Ring: + return "Array(Ring)" + case map[string]float32: + return "Map(String, Float32)" + case map[string]float64: + return "Map(String, Float64)" + case map[string]int8: + return "Map(String, Int8)" + case map[string]int16: + return "Map(String, Int16)" + case map[string]int32: + return "Map(String, Int32)" + case map[string]int64: + return "Map(String, Int64)" + case map[string]uint8: + return "Map(String, UInt8)" + case map[string]uint16: + return "Map(String, UInt16)" + case map[string]uint32: + return "Map(String, UInt32)" + case map[string]uint64: + return "Map(String, UInt64)" + case map[string]string: + return "Map(String, String)" + case map[string]json.RawMessage: + return "Map(String, String)" + case map[string]sql.NullString: + return "Map(String, String)" + case map[string]bool: + return "Map(String, Bool)" + case map[string]sql.NullBool: + return "Map(String, Bool)" + case map[string]time.Time: + return "Map(String, DateTime64(3))" + case map[string]sql.NullTime: + return "Map(String, DateTime64(3))" + case map[string]uuid.UUID: + return "Map(String, UUID)" + case map[string]proto.IPv6: + return "Map(String, IPv6)" + case map[string]orb.MultiPolygon: + return "Map(String, MultiPolygon)" + case map[string]orb.Point: + return "Map(String, Point)" + case map[string]orb.Polygon: + return "Map(String, Polygon)" + case map[string]orb.Ring: + return "Map(String, Ring)" + default: + return "" + } +} diff --git a/lib/column/sharedvariant.go b/lib/column/sharedvariant.go new file mode 100644 index 0000000000..b356f1ac61 --- /dev/null +++ b/lib/column/sharedvariant.go @@ -0,0 +1,72 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package column + +import ( + "github.com/ClickHouse/ch-go/proto" + "reflect" +) + +type SharedVariant struct { + name string + stringData String +} + +func (c *SharedVariant) Name() string { + return c.name +} + +func (c *SharedVariant) Type() Type { + return "SharedVariant" +} + +func (c *SharedVariant) Rows() int { + return c.stringData.Rows() +} + +func (c *SharedVariant) Row(i int, ptr bool) any { + return c.stringData.Row(i, ptr) +} + +func (c *SharedVariant) ScanRow(dest any, row int) error { + return c.stringData.ScanRow(dest, row) +} + +func (c *SharedVariant) Append(v any) (nulls []uint8, err error) { + return c.stringData.Append(v) +} + +func (c *SharedVariant) AppendRow(v any) error { + return c.stringData.AppendRow(v) +} + +func (c *SharedVariant) Decode(reader *proto.Reader, rows int) error { + return c.stringData.Decode(reader, rows) +} + +func (c *SharedVariant) Encode(buffer *proto.Buffer) { + c.stringData.Encode(buffer) +} + +func (c *SharedVariant) ScanType() reflect.Type { + return c.stringData.ScanType() +} + +func (c *SharedVariant) Reset() { + c.stringData.Reset() +} diff --git a/tests/dynamic_test.go b/tests/dynamic_test.go new file mode 100644 index 0000000000..6e98eda085 --- /dev/null +++ b/tests/dynamic_test.go @@ -0,0 +1,233 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package tests + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +var dynamicTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") + +func setupDynamicTest(t *testing.T) driver.Conn { + conn, err := GetNativeConnection(clickhouse.Settings{ + "max_execution_time": 60, + "allow_experimental_dynamic_type": true, + }, nil, &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + }) + require.NoError(t, err) + + if !CheckMinServerServerVersion(conn, 24, 1, 0) { + t.Skip(fmt.Errorf("unsupported clickhouse version for Dynamic type")) + return nil + } + + return conn +} + +func TestDynamic(t *testing.T) { + ctx := context.Background() + conn := setupDynamicTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_dynamic ( + c Dynamic + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_dynamic")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_dynamic (c)") + require.NoError(t, err) + + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(true, "Bool"))) + colInt64 := int64(42) + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colInt64, "Int64"))) + colString := "test" + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colString, "String"))) + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(dynamicTestDate, "DateTime64(3)"))) + var colNil any = nil + require.NoError(t, batch.Append(colNil)) + colSliceUInt8 := []uint8{0xA, 0xB, 0xC} + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colSliceUInt8, "Array(UInt8)"))) + colSliceMapStringString := []map[string]string{{"key1": "value1", "key2": "value2"}, {"key3": "value3"}} + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colSliceMapStringString, "Array(Map(String, String))"))) + colMapStringString := map[string]string{"key1": "value1", "key2": "value2"} + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colMapStringString, "Map(String, String)"))) + colMapStringInt64 := map[string]int64{"key1": 42, "key2": 84} + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(colMapStringInt64, "Map(String, Int64)"))) + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_dynamic") + require.NoError(t, err) + + var row chcol.Dynamic + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colInt64, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, dynamicTestDate, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colNil, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceUInt8, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringInt64, row.Any()) +} + +func TestDynamic_ScanWithType(t *testing.T) { + ctx := context.Background() + conn := setupDynamicTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_dynamic ( + c Dynamic + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_dynamic")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_dynamic (c)") + require.NoError(t, err) + + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(true, "Bool"))) + require.NoError(t, batch.Append(clickhouse.NewDynamicWithType(int64(42), "Int64"))) + require.NoError(t, batch.Append(nil)) + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_dynamic") + require.NoError(t, err) + + var row chcol.DynamicWithType + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + require.Equal(t, "Bool", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, int64(42), row.Any()) + require.Equal(t, "Int64", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, nil, row.Any()) + require.Equal(t, "", row.Type()) +} + +func TestDynamic_BatchFlush(t *testing.T) { + t.Skip(fmt.Errorf("server-side Dynamic bug")) + + ctx := context.Background() + conn := setupDynamicTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_dynamic ( + c Dynamic + ) Engine = MergeTree() ORDER BY tuple() + ` + require.NoError(t, conn.Exec(ctx, ddl)) + defer func() { + require.NoError(t, conn.Exec(ctx, "DROP TABLE IF EXISTS test_dynamic")) + }() + + batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_dynamic (c)") + require.NoError(t, err) + + vals := make([]clickhouse.DynamicWithType, 0, 1000) + for i := 0; i < 1000; i++ { + if i%2 == 0 { + vals = append(vals, clickhouse.NewDynamicWithType(int64(i), "Int64")) + } else { + vals = append(vals, clickhouse.NewDynamicWithType(i%5 == 0, "Bool")) + } + + require.NoError(t, batch.Append(vals[i])) + require.NoError(t, batch.Flush()) + } + require.NoError(t, batch.Send()) + + rows, err := conn.Query(ctx, "SELECT c FROM test_dynamic") + require.NoError(t, err) + + i := 0 + for rows.Next() { + var row clickhouse.DynamicWithType + err = rows.Scan(&row) + require.NoError(t, err) + + if i%2 == 0 { + require.Equal(t, int64(i), row.Any()) + require.Equal(t, "Int64", row.Type()) + } else { + require.Equal(t, i%5 == 0, row.Any()) + require.Equal(t, "Bool", row.Type()) + } + + i++ + } +} From dfc593b5a038ae636ebb7122ba1003fe65b56bbc Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Mon, 6 Jan 2025 12:21:45 -0500 Subject: [PATCH 03/18] Remove forced string conversions for Tuple --- lib/column/tuple.go | 7 ---- tests/issues/1446_test.go | 68 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 68 insertions(+), 7 deletions(-) create mode 100644 tests/issues/1446_test.go diff --git a/lib/column/tuple.go b/lib/column/tuple.go index 95c4e24209..0310a3686e 100644 --- a/lib/column/tuple.go +++ b/lib/column/tuple.go @@ -200,13 +200,6 @@ func setJSONFieldValue(field reflect.Value, value reflect.Value) error { } } - // check if our target is a string - if field.Kind() == reflect.String { - if v := reflect.ValueOf(fmt.Sprint(value.Interface())); v.Type().AssignableTo(field.Type()) { - field.Set(v) - return nil - } - } if value.CanConvert(field.Type()) { field.Set(value.Convert(field.Type())) return nil diff --git a/tests/issues/1446_test.go b/tests/issues/1446_test.go new file mode 100644 index 0000000000..4763b475d4 --- /dev/null +++ b/tests/issues/1446_test.go @@ -0,0 +1,68 @@ +package issues + +import ( + "context" + "testing" + + "github.com/ClickHouse/clickhouse-go/v2/tests" + "github.com/stretchr/testify/require" +) + +type sampleFailTuple struct { + TupleOne string `ch:"tuple_one"` + TupleTwo string `ch:"tuple_two"` +} + +type sampleFailRow struct { + MyId uint64 `ch:"my_id"` + MyTuple sampleFailTuple `ch:"my_tuple"` +} + +type sampleOkTuple struct { + TupleOne *string `ch:"tuple_one"` + TupleTwo *string `ch:"tuple_two"` +} + +type sampleOkRow struct { + MyId uint64 `ch:"my_id"` + MyTuple sampleOkTuple `ch:"my_tuple"` +} + +func TestIssue1446(t *testing.T) { + ctx := context.Background() + + conn, err := tests.GetConnection("issues", nil, nil, nil) + require.NoError(t, err) + defer conn.Close() + + const ddl = ` + CREATE TABLE IF NOT EXISTS issue_1446( + my_id UInt64, + my_tuple Tuple(tuple_one Nullable(String), tuple_two Nullable(String)) + ) ENGINE = MergeTree PRIMARY KEY (my_id) ORDER BY (my_id) + ` + err = conn.Exec(ctx, ddl) + require.NoError(t, err) + defer conn.Exec(ctx, "DROP TABLE issue_1446") + + err = conn.Exec(ctx, "INSERT INTO issue_1446(my_id, my_tuple) VALUES (1, tuple('one', 'two'))") + require.NoError(t, err) + + failRow := sampleFailRow{} + err = conn.QueryRow(ctx, "SELECT * FROM issue_1446 LIMIT 1").ScanStruct(&failRow) + if err == nil { + t.Errorf("expected column convert error for tuple *string to string") + } + + okRow := sampleOkRow{} + err = conn.QueryRow(ctx, "SELECT * FROM issue_1446 LIMIT 1").ScanStruct(&okRow) + require.NoError(t, err) + + if *okRow.MyTuple.TupleOne != "one" { + t.Errorf("expected 'one' but got '%s'", *okRow.MyTuple.TupleOne) + } + + if *okRow.MyTuple.TupleTwo != "two" { + t.Errorf("expected 'two' but got '%s'", *okRow.MyTuple.TupleTwo) + } +} From bbf06d8a3f4dc659f721fb94af2a4da180aaca1f Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Tue, 7 Jan 2025 10:00:46 -0500 Subject: [PATCH 04/18] update tests --- tests/issues/1446_test.go | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/issues/1446_test.go b/tests/issues/1446_test.go index 4763b475d4..d52ca92640 100644 --- a/tests/issues/1446_test.go +++ b/tests/issues/1446_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/ClickHouse/clickhouse-go/v2/tests" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -50,19 +51,12 @@ func TestIssue1446(t *testing.T) { failRow := sampleFailRow{} err = conn.QueryRow(ctx, "SELECT * FROM issue_1446 LIMIT 1").ScanStruct(&failRow) - if err == nil { - t.Errorf("expected column convert error for tuple *string to string") - } + assert.EqualError(t, err, "clickhouse [ScanRow]: (my_tuple) converting *string to string is unsupported") okRow := sampleOkRow{} err = conn.QueryRow(ctx, "SELECT * FROM issue_1446 LIMIT 1").ScanStruct(&okRow) require.NoError(t, err) - if *okRow.MyTuple.TupleOne != "one" { - t.Errorf("expected 'one' but got '%s'", *okRow.MyTuple.TupleOne) - } - - if *okRow.MyTuple.TupleTwo != "two" { - t.Errorf("expected 'two' but got '%s'", *okRow.MyTuple.TupleTwo) - } + assert.Equal(t, "one", *okRow.MyTuple.TupleOne) + assert.Equal(t, "two", *okRow.MyTuple.TupleTwo) } From 6c5abcbf2df8434fb1a45a6906f2aeaaa8717a99 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Wed, 8 Jan 2025 20:45:10 -0500 Subject: [PATCH 05/18] VariantWithType is now Variant, update Scan/Valuer interfaces, add stdlib tests, fix nits --- chcol.go | 5 +- examples/clickhouse_api/variant.go | 2 +- examples/std/main_test.go | 4 + examples/std/variant.go | 144 +++++++++++++++++++++++++++++ lib/chcol/variant.go | 111 +++++++++------------- lib/chcol/variant_test.go | 72 --------------- lib/column/variant.go | 62 +++---------- lib/column/variant_test.go | 17 +--- tests/variant_test.go | 6 +- 9 files changed, 215 insertions(+), 208 deletions(-) create mode 100644 examples/std/variant.go delete mode 100644 lib/chcol/variant_test.go diff --git a/chcol.go b/chcol.go index 7adbb8a1da..151459033c 100644 --- a/chcol.go +++ b/chcol.go @@ -22,8 +22,7 @@ import "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" // Re-export chcol types/funcs to top level clickhouse package type ( - Variant = chcol.Variant - VariantWithType = chcol.VariantWithType + Variant = chcol.Variant ) // NewVariant creates a new Variant with the given value @@ -32,6 +31,6 @@ func NewVariant(v any) Variant { } // NewVariantWithType creates a new Variant with the given value and ClickHouse type -func NewVariantWithType(v any, chType string) VariantWithType { +func NewVariantWithType(v any, chType string) Variant { return chcol.NewVariantWithType(v, chType) } diff --git a/examples/clickhouse_api/variant.go b/examples/clickhouse_api/variant.go index b90a231e08..3727a8ae21 100644 --- a/examples/clickhouse_api/variant.go +++ b/examples/clickhouse_api/variant.go @@ -114,7 +114,7 @@ func VariantExample() error { } for i := 0; rows.Next(); i++ { - var row clickhouse.VariantWithType + var row clickhouse.Variant err := rows.Scan(&row) if err != nil { return fmt.Errorf("failed to scan row index %d: %w", i, err) diff --git a/examples/std/main_test.go b/examples/std/main_test.go index f3c5364110..0ba13eebf1 100644 --- a/examples/std/main_test.go +++ b/examples/std/main_test.go @@ -148,3 +148,7 @@ func TestOpenDb(t *testing.T) { func TestConnectionSettings(t *testing.T) { require.NoError(t, ConnectSettings()) } + +func TestVariantExample(t *testing.T) { + require.NoError(t, VariantExample()) +} diff --git a/examples/std/variant.go b/examples/std/variant.go new file mode 100644 index 0000000000..9f62fcde47 --- /dev/null +++ b/examples/std/variant.go @@ -0,0 +1,144 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package std + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" +) + +func VariantExample() error { + ctx := context.Background() + + conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ + "allow_experimental_variant_type": true, + }, nil, nil) + if err != nil { + return err + } + defer func() { + conn.Exec("DROP TABLE go_variant_example") + }() + + _, err = conn.ExecContext(ctx, "DROP TABLE IF EXISTS go_variant_example") + if err != nil { + return err + } + + _, err = conn.ExecContext(ctx, ` + CREATE TABLE go_variant_example ( + c Variant(Bool, Int64, String) + ) ENGINE = Memory + `) + if err != nil { + return err + } + + tx, err := conn.BeginTx(ctx, nil) + if err != nil { + return err + } + + batch, err := tx.PrepareContext(ctx, "INSERT INTO go_variant_example (c)") + if err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, true); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, int64(42)); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, "example"); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, clickhouse.NewVariant("example variant")); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType("example variant with specific type", "String")); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, nil); err != nil { + return err + } + + if err = tx.Commit(); err != nil { + return err + } + + // Switch on Go Type + + rows, err := conn.QueryContext(ctx, "SELECT c FROM go_variant_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Variant + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Any().(type) { + case bool: + fmt.Printf("row at index %d is Bool: %v\n", i, row.Any()) + case int64: + fmt.Printf("row at index %d is Int64: %v\n", i, row.Any()) + case string: + fmt.Printf("row at index %d is String: %v\n", i, row.Any()) + case nil: + fmt.Printf("row at index %d is NULL\n", i) + } + } + + // Switch on ClickHouse Type + + rows, err = conn.QueryContext(ctx, "SELECT c FROM go_variant_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Variant + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Type() { + case "Bool": + fmt.Printf("row at index %d is bool: %v\n", i, row.Any()) + case "Int64": + fmt.Printf("row at index %d is int64: %v\n", i, row.Any()) + case "String": + fmt.Printf("row at index %d is string: %v\n", i, row.Any()) + case "": + fmt.Printf("row at index %d is nil\n", i) + } + } + + return nil +} diff --git a/lib/chcol/variant.go b/lib/chcol/variant.go index f40b5fcda5..5a5adee473 100644 --- a/lib/chcol/variant.go +++ b/lib/chcol/variant.go @@ -24,63 +24,52 @@ import ( // Variant represents a ClickHouse Variant type that can hold multiple possible types type Variant struct { - value any + value any + chType string } // NewVariant creates a new Variant with the given value func NewVariant(v any) Variant { - return Variant{value: v} -} - -// Nil returns true if the underlying value is nil. -func (v Variant) Nil() bool { - return v.value == nil -} - -// Any returns the underlying value as any. Same as Interface. -func (v Variant) Any() any { - return v.value -} - -// Interface returns the underlying value as interface{}. Same as Any. -func (v Variant) Interface() interface{} { - return v.value + return Variant{ + value: v, + chType: "", + } } -// Int returns the value as an int if possible -func (v Variant) Int() (int, bool) { - if i, ok := v.value.(int); ok { - return i, true +// NewVariantWithType creates a new Variant with the given value and ClickHouse type +func NewVariantWithType(v any, chType string) Variant { + return Variant{ + value: v, + chType: chType, } - - return 0, false } -// Int64 returns the value as an int64 if possible -func (v Variant) Int64() (int64, bool) { - if i, ok := v.value.(int64); ok { - return i, true +// WithType creates a new Variant with the current value and given ClickHouse type +func (v Variant) WithType(chType string) Variant { + return Variant{ + value: v.value, + chType: chType, } - - return 0, false } -// String returns the value as a string if possible -func (v Variant) String() (string, bool) { - if s, ok := v.value.(string); ok { - return s, true - } +// Type returns the ClickHouse type as a string. +func (v Variant) Type() string { + return v.chType +} - return "", false +// HasType returns true if the value has a type ClickHouse included. +func (v Variant) HasType() bool { + return v.chType == "" } -// Bool returns the value as an bool if possible -func (v Variant) Bool() (bool, bool) { - if b, ok := v.value.(bool); ok { - return b, true - } +// Nil returns true if the underlying value is nil. +func (v Variant) Nil() bool { + return v.value == nil +} - return false, false +// Any returns the underlying value as any. +func (v Variant) Any() any { + return v.value } // MarshalJSON implements the json.Marshaler interface @@ -90,37 +79,21 @@ func (v *Variant) MarshalJSON() ([]byte, error) { // Scan implements the sql.Scanner interface func (v *Variant) Scan(value interface{}) error { - v.value = value + switch vv := value.(type) { + case Variant: + v.value = vv.value + v.chType = vv.chType + case *Variant: + v.value = vv.value + v.chType = vv.chType + default: + v.value = value + } + return nil } // Value implements the driver.Valuer interface func (v Variant) Value() (driver.Value, error) { - return v.value, nil -} - -func (v Variant) WithType(chType string) VariantWithType { - return VariantWithType{ - Variant: v, - chType: chType, - } -} - -// VariantWithType is Variant with an extra value for specifying the preferred ClickHouse type for column encoding -type VariantWithType struct { - Variant - chType string -} - -// NewVariantWithType creates a new Variant with the given value and ClickHouse type -func NewVariantWithType(v any, chType string) VariantWithType { - return VariantWithType{ - Variant: Variant{value: v}, - chType: chType, - } -} - -// Type returns the ClickHouse type as a string. -func (v VariantWithType) Type() string { - return v.chType + return v, nil } diff --git a/lib/chcol/variant_test.go b/lib/chcol/variant_test.go deleted file mode 100644 index f5aa352493..0000000000 --- a/lib/chcol/variant_test.go +++ /dev/null @@ -1,72 +0,0 @@ -// Licensed to ClickHouse, Inc. under one or more contributor -// license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright -// ownership. ClickHouse, Inc. licenses this file to you under -// the Apache License, Version 2.0 (the "License"); you may -// not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package chcol - -import ( - "testing" -) - -func TestVariant_Nil(t *testing.T) { - v := NewVariant(nil) - - if !v.Nil() { - t.Fatalf("expected variant to be nil") - } -} - -func TestVariant_Int64(t *testing.T) { - var in int64 = 42 - - v := NewVariant(in) - - out, ok := v.Int64() - if !ok { - t.Fatalf("failed to get int64 from variant") - } else if out != in { - t.Fatalf("incorrect value from variant. expected: %d got: %d", in, out) - } -} - -func TestVariant_String(t *testing.T) { - in := "test" - - v := NewVariant(in) - - out, ok := v.String() - if !ok { - t.Fatalf("failed to get string from variant") - } else if out != in { - t.Fatalf("incorrect value from variant. expected: %s got: %s", in, out) - } -} - -func TestVariant_TypeSwitch(t *testing.T) { - var in any - - v := NewVariant(in) - - switch v.Any().(type) { - case int64: - t.Fatalf("unexpected int64 value from variant") - case string: - t.Fatalf("unexpected string value from variant") - case nil: - default: - t.Fatalf("expected nil value from variant") - } -} diff --git a/lib/column/variant.go b/lib/column/variant.go index a8b0668a2d..6073f97797 100644 --- a/lib/column/variant.go +++ b/lib/column/variant.go @@ -154,15 +154,9 @@ func (c *Variant) ScanRow(dest any, row int) error { switch v := dest.(type) { case *chcol.Variant: - vt := chcol.NewVariant(value) - *v = vt - case **chcol.Variant: - vt := chcol.NewVariant(value) - **v = vt - case *chcol.VariantWithType: vt := chcol.NewVariantWithType(value, chType) *v = vt - case **chcol.VariantWithType: + case **chcol.Variant: vt := chcol.NewVariantWithType(value, chType) **v = vt default: @@ -179,9 +173,9 @@ func (c *Variant) ScanRow(dest any, row int) error { } func (c *Variant) Append(v any) (nulls []uint8, err error) { - switch v.(type) { + switch vv := v.(type) { case []chcol.Variant: - for i, vt := range v.([]chcol.Variant) { + for i, vt := range vv { err := c.AppendRow(vt) if err != nil { return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) @@ -190,25 +184,7 @@ func (c *Variant) Append(v any) (nulls []uint8, err error) { return nil, nil case []*chcol.Variant: - for i, vt := range v.([]*chcol.Variant) { - err := c.AppendRow(vt) - if err != nil { - return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) - } - } - - return nil, nil - case []chcol.VariantWithType: - for i, vt := range v.([]chcol.VariantWithType) { - err := c.AppendRow(vt) - if err != nil { - return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) - } - } - - return nil, nil - case []*chcol.VariantWithType: - for i, vt := range v.([]*chcol.VariantWithType) { + for i, vt := range vv { err := c.AppendRow(vt) if err != nil { return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) @@ -241,33 +217,21 @@ func (c *Variant) Append(v any) (nulls []uint8, err error) { func (c *Variant) AppendRow(v any) error { var requestedType string - switch v.(type) { + switch vv := v.(type) { case nil: c.appendNullRow() return nil case chcol.Variant: - v = v.(chcol.Variant).Any() - if v == nil { + requestedType = vv.Type() + v = vv.Any() + if vv.Nil() { c.appendNullRow() return nil } case *chcol.Variant: - v = v.(*chcol.Variant).Any() - if v == nil { - c.appendNullRow() - return nil - } - case chcol.VariantWithType: - requestedType = v.(chcol.VariantWithType).Type() - v = v.(chcol.VariantWithType).Any() - if v == nil { - c.appendNullRow() - return nil - } - case *chcol.VariantWithType: - requestedType = v.(*chcol.VariantWithType).Type() - v = v.(*chcol.VariantWithType).Any() - if v == nil { + requestedType = vv.Type() + v = vv.Any() + if vv.Nil() { c.appendNullRow() return nil } @@ -332,7 +296,9 @@ func (c *Variant) decodeHeader(reader *proto.Reader) error { variantSerializationVersion, err := reader.UInt64() if err != nil { return fmt.Errorf("failed to read variant discriminator version: %w", err) - } else if variantSerializationVersion != SupportedVariantSerializationVersion { + } + + if variantSerializationVersion != SupportedVariantSerializationVersion { return fmt.Errorf("unsupported variant discriminator version: %d", variantSerializationVersion) } diff --git a/lib/column/variant_test.go b/lib/column/variant_test.go index 8ef1ab6542..ff8bf4bf46 100644 --- a/lib/column/variant_test.go +++ b/lib/column/variant_test.go @@ -1,6 +1,7 @@ package column import ( + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "testing" ) @@ -19,9 +20,7 @@ func TestColVariant_parse(t *testing.T) { for i, c := range cases { col, err := (&Variant{name: "vt"}).parse(c.typ, nil) - if err != nil { - t.Fatalf("case index %d failed to parse Variant column: %s", i, err) - } + require.NoError(t, err, "case index %d failed to parse Variant column", i) require.Equal(t, "vt", col.Name()) require.Equal(t, c.typ, col.chType) @@ -31,15 +30,11 @@ func TestColVariant_parse(t *testing.T) { for j, subCol := range col.columns { expectedType := c.expectedTypes[j] actualType := subCol.Type() - if actualType != expectedType { - t.Fatalf("case index %d Variant type index %d column type does not match: expected: \"%s\" actual: \"%s\"", i, j, expectedType, actualType) - } + assert.Equal(t, expectedType, actualType, "case index %d Variant type index %d column type does not match", i, j) expectedColumnTypeIndex := uint8(j) actualColumnTypeIndex := col.columnTypeIndex[string(actualType)] - if actualColumnTypeIndex != expectedColumnTypeIndex { - t.Fatalf("case index %d Variant type index %d columnTypeIndex does not match: expected: %d actual: %d", i, j, expectedColumnTypeIndex, actualColumnTypeIndex) - } + assert.Equal(t, expectedColumnTypeIndex, actualColumnTypeIndex, "case index %d Variant type index %d columnTypeIndex does not match", i, j) } } } @@ -54,9 +49,7 @@ func TestColVariant_parse_invalid(t *testing.T) { for i, typeName := range cases { _, err := (&Variant{name: "vt"}).parse(typeName, nil) - if err == nil { - t.Fatalf("expected error for case index %d (\"%s\"), but received nil", i, typeName) - } + require.Error(t, err, "expected error for case index %d (\"%s\"), but received nil", i, typeName) } } diff --git a/tests/variant_test.go b/tests/variant_test.go index e5e7c08ffd..ae871194de 100644 --- a/tests/variant_test.go +++ b/tests/variant_test.go @@ -175,7 +175,7 @@ func TestVariant_ScanWithType(t *testing.T) { rows, err := conn.Query(ctx, "SELECT c FROM test_variant") require.NoError(t, err) - var row chcol.VariantWithType + var row chcol.Variant require.True(t, rows.Next()) err = rows.Scan(&row) @@ -213,7 +213,7 @@ func TestVariant_BatchFlush(t *testing.T) { batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_variant (c)") require.NoError(t, err) - vals := make([]clickhouse.VariantWithType, 0, 1000) + vals := make([]clickhouse.Variant, 0, 1000) for i := 0; i < 1000; i++ { if i%2 == 0 { vals = append(vals, clickhouse.NewVariantWithType(int64(i), "Int64")) @@ -231,7 +231,7 @@ func TestVariant_BatchFlush(t *testing.T) { i := 0 for rows.Next() { - var row chcol.VariantWithType + var row chcol.Variant err = rows.Scan(&row) if i%2 == 0 { From 1c6c14ef67ee86a3f271f5b21611e500b301476f Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Wed, 8 Jan 2025 22:04:00 -0500 Subject: [PATCH 06/18] enable `allow_suspicious_variant_types` for Variant tests --- examples/clickhouse_api/variant.go | 2 ++ examples/std/variant.go | 2 ++ tests/variant_test.go | 6 ++++-- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/examples/clickhouse_api/variant.go b/examples/clickhouse_api/variant.go index 3727a8ae21..5d300b49a2 100644 --- a/examples/clickhouse_api/variant.go +++ b/examples/clickhouse_api/variant.go @@ -20,6 +20,7 @@ package clickhouse_api import ( "context" "fmt" + "github.com/ClickHouse/clickhouse-go/v2" ) @@ -28,6 +29,7 @@ func VariantExample() error { conn, err := GetNativeConnection(clickhouse.Settings{ "allow_experimental_variant_type": true, + "allow_suspicious_variant_types": true, }, nil, nil) if err != nil { return err diff --git a/examples/std/variant.go b/examples/std/variant.go index 9f62fcde47..65062c7587 100644 --- a/examples/std/variant.go +++ b/examples/std/variant.go @@ -20,6 +20,7 @@ package std import ( "context" "fmt" + "github.com/ClickHouse/clickhouse-go/v2" ) @@ -28,6 +29,7 @@ func VariantExample() error { conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ "allow_experimental_variant_type": true, + "allow_suspicious_variant_types": true, }, nil, nil) if err != nil { return err diff --git a/tests/variant_test.go b/tests/variant_test.go index ae871194de..9e71cab10b 100644 --- a/tests/variant_test.go +++ b/tests/variant_test.go @@ -20,12 +20,13 @@ package tests import ( "context" "fmt" + "testing" + "time" + "github.com/ClickHouse/clickhouse-go/v2" "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/stretchr/testify/require" - "testing" - "time" ) var variantTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") @@ -34,6 +35,7 @@ func setupVariantTest(t *testing.T) driver.Conn { conn, err := GetNativeConnection(clickhouse.Settings{ "max_execution_time": 60, "allow_experimental_variant_type": true, + "allow_suspicious_variant_types": true, }, nil, &clickhouse.Compression{ Method: clickhouse.CompressionLZ4, }) From 1dbf0582ba4285756c406137139f8d262151dd51 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Wed, 8 Jan 2025 22:21:52 -0500 Subject: [PATCH 07/18] update minimum server version for variant test --- tests/variant_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/variant_test.go b/tests/variant_test.go index 9e71cab10b..afdca8495d 100644 --- a/tests/variant_test.go +++ b/tests/variant_test.go @@ -41,7 +41,7 @@ func setupVariantTest(t *testing.T) driver.Conn { }) require.NoError(t, err) - if !CheckMinServerServerVersion(conn, 24, 1, 0) { + if !CheckMinServerServerVersion(conn, 24, 4, 0) { t.Skip(fmt.Errorf("unsupported clickhouse version for Variant type")) return nil } From 017f3792e8d05106177739a80f4b1a8379642c85 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 9 Jan 2025 10:50:28 +0100 Subject: [PATCH 08/18] Bump golang.org/x/net from 0.33.0 to 0.34.0 (#1466) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.33.0 to 0.34.0. - [Commits](https://github.com/golang/net/compare/v0.33.0...v0.34.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 6 +++--- go.sum | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/go.mod b/go.mod index 29f57f30f6..1725ae292a 100644 --- a/go.mod +++ b/go.mod @@ -17,7 +17,7 @@ require ( github.com/stretchr/testify v1.10.0 github.com/testcontainers/testcontainers-go v0.33.0 go.opentelemetry.io/otel/trace v1.26.0 - golang.org/x/net v0.33.0 + golang.org/x/net v0.34.0 gopkg.in/yaml.v3 v3.0.1 ) @@ -69,8 +69,8 @@ require ( go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0 // indirect go.opentelemetry.io/otel/metric v1.26.0 // indirect - golang.org/x/crypto v0.31.0 // indirect - golang.org/x/sys v0.28.0 // indirect + golang.org/x/crypto v0.32.0 // indirect + golang.org/x/sys v0.29.0 // indirect golang.org/x/time v0.0.0-20220210224613-90d013bbcef8 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237 // indirect diff --git a/go.sum b/go.sum index 6290c28909..6b71896fcb 100644 --- a/go.sum +++ b/go.sum @@ -192,8 +192,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= -golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/crypto v0.32.0 h1:euUpcYgM8WcP71gNpTqQCn6rC2t6ULUPiOzfWaXVVfc= +golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -201,8 +201,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= -golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= +golang.org/x/net v0.34.0 h1:Mb7Mrk043xzHgnRM88suvJFwzVrRfHEHJEl5/71CKw0= +golang.org/x/net v0.34.0/go.mod h1:di0qlW3YNM5oh6GqDGQr92MyTozJPmybPK4Ev/Gm31k= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -221,11 +221,11 @@ golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= -golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.29.0 h1:TPYlXGxvx1MGTn2GiZDhnjPA9wZzZeGKHHmKhHYvgaU= +golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= -golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= +golang.org/x/term v0.28.0 h1:/Ts8HFuMR2E6IP/jlo7QVLZHggjKQbhu/7H0LJFr3Gg= +golang.org/x/term v0.28.0/go.mod h1:Sw/lC2IAUZ92udQNf3WodGtn4k/XoLyZoh8v/8uiwek= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= From b5e27f0e897344e3b17110d93dcae0b5b0c77cbb Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Thu, 9 Jan 2025 21:39:34 -0500 Subject: [PATCH 09/18] stdlib sql return precision for DateTime64 --- clickhouse_std.go | 6 ++++++ lib/column/datetime64.go | 4 ++++ tests/std/datetime64_test.go | 21 +++++++++++++++++++++ 3 files changed, 31 insertions(+) diff --git a/clickhouse_std.go b/clickhouse_std.go index 466d345fb2..e747c0ffe4 100644 --- a/clickhouse_std.go +++ b/clickhouse_std.go @@ -445,10 +445,16 @@ func (r *stdRows) ColumnTypePrecisionScale(idx int) (precision, scale int64, ok switch col := r.rows.block.Columns[idx].(type) { case *column.Decimal: return col.Precision(), col.Scale(), true + case *column.DateTime64: + p, ok := col.Precision() + return p, 0, ok case interface{ Base() column.Interface }: switch col := col.Base().(type) { case *column.Decimal: return col.Precision(), col.Scale(), true + case *column.DateTime64: + p, ok := col.Precision() + return p, 0, ok } } return 0, 0, false diff --git a/lib/column/datetime64.go b/lib/column/datetime64.go index 3e1f438b9f..668bc2cea8 100644 --- a/lib/column/datetime64.go +++ b/lib/column/datetime64.go @@ -96,6 +96,10 @@ func (col *DateTime64) ScanType() reflect.Type { return scanTypeTime } +func (col *DateTime64) Precision() (int64, bool) { + return int64(col.col.Precision), col.col.PrecisionSet +} + func (col *DateTime64) Rows() int { return col.col.Rows() } diff --git a/tests/std/datetime64_test.go b/tests/std/datetime64_test.go index 20ba9432d3..406246767a 100644 --- a/tests/std/datetime64_test.go +++ b/tests/std/datetime64_test.go @@ -118,5 +118,26 @@ func TestStdDateTime64(t *testing.T) { require.Equal(t, time.Date(1900, 01, 01, 0, 0, 0, 0, time.UTC), col9) require.Equal(t, time.Unix(0, 0).UTC(), col10) }) + + t.Run("DateTime64 precision", func(t *testing.T) { + conn, err := GetStdDSNConnection(protocol, useSSL, nil) + require.NoError(t, err) + if !CheckMinServerVersion(conn, 20, 3, 0) { + t.Skip(fmt.Errorf("unsupported clickhouse version")) + return + } + + rows, err := conn.Query("SELECT toDateTime64(1546300800.123, 3)") + require.NoError(t, err) + + columnTypes, err := rows.ColumnTypes() + require.NoError(t, err) + require.Len(t, columnTypes, 1) + + precision, scale, ok := columnTypes[0].DecimalSize() + require.Equal(t, int64(3), precision) + require.Equal(t, int64(0), scale) + require.True(t, ok) + }) } } From 192524a8594ca55beadd3d7b27ce37b0ab75d991 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 12:50:49 -0500 Subject: [PATCH 10/18] skip variant examples --- examples/clickhouse_api/main_test.go | 1 + examples/std/main_test.go | 1 + 2 files changed, 2 insertions(+) diff --git a/examples/clickhouse_api/main_test.go b/examples/clickhouse_api/main_test.go index 40b59e36b2..49fab4f317 100644 --- a/examples/clickhouse_api/main_test.go +++ b/examples/clickhouse_api/main_test.go @@ -211,5 +211,6 @@ func TestSSLNoVerify(t *testing.T) { } func TestVariantExample(t *testing.T) { + t.Skip("unsupported version") require.NoError(t, VariantExample()) } diff --git a/examples/std/main_test.go b/examples/std/main_test.go index 0ba13eebf1..bfbb9808a2 100644 --- a/examples/std/main_test.go +++ b/examples/std/main_test.go @@ -150,5 +150,6 @@ func TestConnectionSettings(t *testing.T) { } func TestVariantExample(t *testing.T) { + t.Skip("unsupported version") require.NoError(t, VariantExample()) } From c00a8f43a3aba96883d4173c197c05f6053b4159 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 13:50:22 -0500 Subject: [PATCH 11/18] remove DynamicWithType in favor of Dynamic --- lib/chcol/dynamic.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/lib/chcol/dynamic.go b/lib/chcol/dynamic.go index 7e081376a7..17d1e266f6 100644 --- a/lib/chcol/dynamic.go +++ b/lib/chcol/dynamic.go @@ -24,12 +24,10 @@ func NewDynamic(v any) Dynamic { return Dynamic{value: v} } -type DynamicWithType = VariantWithType - // NewDynamicWithType creates a new Dynamic with the given value and ClickHouse type -func NewDynamicWithType(v any, chType string) DynamicWithType { - return DynamicWithType{ - Variant: Variant{value: v}, - chType: chType, +func NewDynamicWithType(v any, chType string) Dynamic { + return Dynamic{ + value: v, + chType: chType, } } From 50147b66921a5b07ee1a874d620bda6860a0d5eb Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 13:51:10 -0500 Subject: [PATCH 12/18] carry over nits from Variant branch --- lib/column/dynamic.go | 72 ++++++++++++------------------------------- 1 file changed, 19 insertions(+), 53 deletions(-) diff --git a/lib/column/dynamic.go b/lib/column/dynamic.go index 2378df104b..310a1678f4 100644 --- a/lib/column/dynamic.go +++ b/lib/column/dynamic.go @@ -132,17 +132,11 @@ func (c *Dynamic) ScanRow(dest any, row int) error { switch v := dest.(type) { case *chcol.Dynamic: - vt := chcol.NewDynamic(value) - *v = vt + dyn := chcol.NewDynamicWithType(value, chType) + *v = dyn case **chcol.Dynamic: - vt := chcol.NewDynamic(value) - **v = vt - case *chcol.DynamicWithType: - vt := chcol.NewDynamicWithType(value, chType) - *v = vt - case **chcol.DynamicWithType: - vt := chcol.NewDynamicWithType(value, chType) - **v = vt + dyn := chcol.NewDynamicWithType(value, chType) + **v = dyn default: if typeIndex == NullVariantDiscriminator { return nil @@ -157,10 +151,10 @@ func (c *Dynamic) ScanRow(dest any, row int) error { } func (c *Dynamic) Append(v any) (nulls []uint8, err error) { - switch v.(type) { + switch vv := v.(type) { case []chcol.Dynamic: - for i, vt := range v.([]chcol.Dynamic) { - err := c.AppendRow(vt) + for i, dyn := range vv { + err := c.AppendRow(dyn) if err != nil { return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) } @@ -168,26 +162,8 @@ func (c *Dynamic) Append(v any) (nulls []uint8, err error) { return nil, nil case []*chcol.Dynamic: - for i, vt := range v.([]*chcol.Dynamic) { - err := c.AppendRow(vt) - if err != nil { - return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) - } - } - - return nil, nil - case []chcol.DynamicWithType: - for i, vt := range v.([]chcol.DynamicWithType) { - err := c.AppendRow(vt) - if err != nil { - return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) - } - } - - return nil, nil - case []*chcol.DynamicWithType: - for i, vt := range v.([]*chcol.DynamicWithType) { - err := c.AppendRow(vt) + for i, dyn := range vv { + err := c.AppendRow(dyn) if err != nil { return nil, fmt.Errorf("failed to AppendRow at index %d: %w", i, err) } @@ -219,33 +195,21 @@ func (c *Dynamic) Append(v any) (nulls []uint8, err error) { func (c *Dynamic) AppendRow(v any) error { var requestedType string - switch v.(type) { + switch vv := v.(type) { case nil: c.variant.appendNullRow() return nil case chcol.Dynamic: - v = v.(chcol.Dynamic).Any() - if v == nil { + requestedType = vv.Type() + v = vv.Any() + if vv.Nil() { c.variant.appendNullRow() return nil } case *chcol.Dynamic: - v = v.(*chcol.Dynamic).Any() - if v == nil { - c.variant.appendNullRow() - return nil - } - case chcol.DynamicWithType: - requestedType = v.(chcol.DynamicWithType).Type() - v = v.(chcol.DynamicWithType).Any() - if v == nil { - c.variant.appendNullRow() - return nil - } - case *chcol.DynamicWithType: - requestedType = v.(*chcol.DynamicWithType).Type() - v = v.(*chcol.DynamicWithType).Any() - if v == nil { + requestedType = vv.Type() + v = vv.Any() + if vv.Nil() { c.variant.appendNullRow() return nil } @@ -366,7 +330,9 @@ func (c *Dynamic) decodeHeader(reader *proto.Reader) error { dynamicSerializationVersion, err := reader.UInt64() if err != nil { return fmt.Errorf("failed to read dynamic serialization version: %w", err) - } else if dynamicSerializationVersion != SupportedDynamicSerializationVersion { + } + + if dynamicSerializationVersion != SupportedDynamicSerializationVersion { return fmt.Errorf("unsupported dynamic serialization version: %d", dynamicSerializationVersion) } From 93b460a4642afb51fa394520baa8f538f0c24260 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 13:52:26 -0500 Subject: [PATCH 13/18] Add CheckMinServerVersion func to clickhouse_api and std examples --- examples/clickhouse_api/utils.go | 4 ++++ examples/std/utils.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/examples/clickhouse_api/utils.go b/examples/clickhouse_api/utils.go index 388a0a25f8..4f9a2af13c 100644 --- a/examples/clickhouse_api/utils.go +++ b/examples/clickhouse_api/utils.go @@ -37,3 +37,7 @@ func GetNativeTestEnvironment() (clickhouse_tests.ClickHouseTestEnvironment, err func GetNativeConnectionWithOptions(settings clickhouse.Settings, tlsConfig *tls.Config, compression *clickhouse.Compression) (driver.Conn, error) { return clickhouse_tests.GetConnection(TestSet, settings, tlsConfig, compression) } + +func CheckMinServerVersion(conn driver.Conn, major, minor, patch uint64) bool { + return clickhouse_tests.CheckMinServerServerVersion(conn, major, minor, patch) +} diff --git a/examples/std/utils.go b/examples/std/utils.go index 972a4af8e4..88b6e0fdfb 100644 --- a/examples/std/utils.go +++ b/examples/std/utils.go @@ -38,3 +38,7 @@ func GetStdOpenDBConnection(protocol clickhouse.Protocol, settings clickhouse.Se func GetStdTestEnvironment() (clickhouse_tests.ClickHouseTestEnvironment, error) { return clickhouse_tests.GetTestEnvironment(TestSet) } + +func CheckMinServerVersion(conn *sql.DB, major, minor, patch uint64) bool { + return clickhouse_tests_std.CheckMinServerVersion(conn, major, minor, patch) +} From 60d395b8d8a02670d06ee407bd4132de3072d306 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 13:53:45 -0500 Subject: [PATCH 14/18] Add Dynamic examples + tests for clickhouse_api + std sql --- examples/clickhouse_api/dynamic.go | 13 +- examples/std/dynamic.go | 150 +++++++++++++++++++++ examples/std/main_test.go | 4 + tests/dynamic_test.go | 8 +- tests/std/dynamic_test.go | 206 +++++++++++++++++++++++++++++ 5 files changed, 373 insertions(+), 8 deletions(-) create mode 100644 examples/std/dynamic.go create mode 100644 tests/std/dynamic_test.go diff --git a/examples/clickhouse_api/dynamic.go b/examples/clickhouse_api/dynamic.go index ef354b769e..6b0fa28d3b 100644 --- a/examples/clickhouse_api/dynamic.go +++ b/examples/clickhouse_api/dynamic.go @@ -33,6 +33,11 @@ func DynamicExample() error { return err } + if !CheckMinServerVersion(conn, 24, 4, 0) { + fmt.Print("unsupported clickhouse version for Dynamic type") + return nil + } + err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_dynamic_example") if err != nil { return err @@ -64,11 +69,11 @@ func DynamicExample() error { return err } - if err = batch.Append(clickhouse.NewVariant("example dynamic")); err != nil { + if err = batch.Append(clickhouse.NewDynamic("example dynamic")); err != nil { return err } - if err = batch.Append(clickhouse.NewVariantWithType("example dynamic with specific type", "String")); err != nil { + if err = batch.Append(clickhouse.NewDynamicWithType("example dynamic with specific type", "String")); err != nil { return err } @@ -88,7 +93,7 @@ func DynamicExample() error { } for i := 0; rows.Next(); i++ { - var row clickhouse.Variant + var row clickhouse.Dynamic err := rows.Scan(&row) if err != nil { return fmt.Errorf("failed to scan row index %d: %w", i, err) @@ -114,7 +119,7 @@ func DynamicExample() error { } for i := 0; rows.Next(); i++ { - var row clickhouse.VariantWithType + var row clickhouse.Dynamic err := rows.Scan(&row) if err != nil { return fmt.Errorf("failed to scan row index %d: %w", i, err) diff --git a/examples/std/dynamic.go b/examples/std/dynamic.go new file mode 100644 index 0000000000..861d2e9ef5 --- /dev/null +++ b/examples/std/dynamic.go @@ -0,0 +1,150 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package std + +import ( + "context" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" +) + +func DynamicExample() error { + ctx := context.Background() + + conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ + "allow_experimental_dynamic_type": true, + }, nil, nil) + if err != nil { + return err + } + + if !CheckMinServerVersion(conn, 24, 4, 0) { + fmt.Print("unsupported clickhouse version for Dynamic type") + return nil + } + + defer func() { + conn.Exec("DROP TABLE go_dynamic_example") + }() + + _, err = conn.ExecContext(ctx, "DROP TABLE IF EXISTS go_dynamic_example") + if err != nil { + return err + } + + _, err = conn.ExecContext(ctx, ` + CREATE TABLE go_dynamic_example ( + c Dynamic + ) ENGINE = Memory + `) + if err != nil { + return err + } + + tx, err := conn.BeginTx(ctx, nil) + if err != nil { + return err + } + + batch, err := tx.PrepareContext(ctx, "INSERT INTO go_dynamic_example (c)") + if err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, true); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, int64(42)); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, "example"); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, clickhouse.NewDynamic("example dynamic")); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType("example dynamic with specific type", "String")); err != nil { + return err + } + + if _, err = batch.ExecContext(ctx, nil); err != nil { + return err + } + + if err = tx.Commit(); err != nil { + return err + } + + // Switch on Go Type + + rows, err := conn.QueryContext(ctx, "SELECT c FROM go_dynamic_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Dynamic + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Any().(type) { + case bool: + fmt.Printf("row at index %d is Bool: %v\n", i, row.Any()) + case int64: + fmt.Printf("row at index %d is Int64: %v\n", i, row.Any()) + case string: + fmt.Printf("row at index %d is String: %v\n", i, row.Any()) + case nil: + fmt.Printf("row at index %d is NULL\n", i) + } + } + + // Switch on ClickHouse Type + + rows, err = conn.QueryContext(ctx, "SELECT c FROM go_dynamic_example") + if err != nil { + return err + } + + for i := 0; rows.Next(); i++ { + var row clickhouse.Dynamic + err := rows.Scan(&row) + if err != nil { + return fmt.Errorf("failed to scan row index %d: %w", i, err) + } + + switch row.Type() { + case "Bool": + fmt.Printf("row at index %d is bool: %v\n", i, row.Any()) + case "Int64": + fmt.Printf("row at index %d is int64: %v\n", i, row.Any()) + case "String": + fmt.Printf("row at index %d is string: %v\n", i, row.Any()) + case "": + fmt.Printf("row at index %d is nil\n", i) + } + } + + return nil +} diff --git a/examples/std/main_test.go b/examples/std/main_test.go index bfbb9808a2..012e3627b5 100644 --- a/examples/std/main_test.go +++ b/examples/std/main_test.go @@ -153,3 +153,7 @@ func TestVariantExample(t *testing.T) { t.Skip("unsupported version") require.NoError(t, VariantExample()) } + +func TestDynamicExample(t *testing.T) { + require.NoError(t, DynamicExample()) +} diff --git a/tests/dynamic_test.go b/tests/dynamic_test.go index 6e98eda085..3bda9fe601 100644 --- a/tests/dynamic_test.go +++ b/tests/dynamic_test.go @@ -39,7 +39,7 @@ func setupDynamicTest(t *testing.T) driver.Conn { }) require.NoError(t, err) - if !CheckMinServerServerVersion(conn, 24, 1, 0) { + if !CheckMinServerServerVersion(conn, 24, 4, 0) { t.Skip(fmt.Errorf("unsupported clickhouse version for Dynamic type")) return nil } @@ -158,7 +158,7 @@ func TestDynamic_ScanWithType(t *testing.T) { rows, err := conn.Query(ctx, "SELECT c FROM test_dynamic") require.NoError(t, err) - var row chcol.DynamicWithType + var row chcol.Dynamic require.True(t, rows.Next()) err = rows.Scan(&row) @@ -198,7 +198,7 @@ func TestDynamic_BatchFlush(t *testing.T) { batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_dynamic (c)") require.NoError(t, err) - vals := make([]clickhouse.DynamicWithType, 0, 1000) + vals := make([]clickhouse.Dynamic, 0, 1000) for i := 0; i < 1000; i++ { if i%2 == 0 { vals = append(vals, clickhouse.NewDynamicWithType(int64(i), "Int64")) @@ -216,7 +216,7 @@ func TestDynamic_BatchFlush(t *testing.T) { i := 0 for rows.Next() { - var row clickhouse.DynamicWithType + var row clickhouse.Dynamic err = rows.Scan(&row) require.NoError(t, err) diff --git a/tests/std/dynamic_test.go b/tests/std/dynamic_test.go new file mode 100644 index 0000000000..a7d0cceff6 --- /dev/null +++ b/tests/std/dynamic_test.go @@ -0,0 +1,206 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package std + +import ( + "context" + "database/sql" + + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +var dynamicTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") + +func setupDynamicTest(t *testing.T) *sql.DB { + conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ + "max_execution_time": 60, + "allow_experimental_dynamic_type": true, + }, nil, &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + }) + require.NoError(t, err) + + if !CheckMinServerVersion(conn, 24, 4, 0) { + t.Skip(fmt.Errorf("unsupported clickhouse version for Dynamic type")) + return nil + } + + return conn +} + +func TestDynamic(t *testing.T) { + ctx := context.Background() + conn := setupDynamicTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_dynamic ( + c Dynamic + ) Engine = MergeTree() ORDER BY tuple() + ` + _, err := conn.ExecContext(ctx, ddl) + require.NoError(t, err) + defer func() { + _, err := conn.ExecContext(ctx, "DROP TABLE IF EXISTS test_dynamic") + require.NoError(t, err) + }() + + tx, err := conn.BeginTx(ctx, nil) + require.NoError(t, err) + + batch, err := tx.PrepareContext(ctx, "INSERT INTO test_dynamic (c)") + require.NoError(t, err) + + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(true, "Bool")) + require.NoError(t, err) + colInt64 := int64(42) + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colInt64, "Int64")) + require.NoError(t, err) + colString := "test" + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colString, "String")) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(dynamicTestDate, "DateTime64(3)")) + require.NoError(t, err) + var colNil any = nil + _, err = batch.ExecContext(ctx, colNil) + require.NoError(t, err) + colSliceUInt8 := []uint8{0xA, 0xB, 0xC} + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colSliceUInt8, "Array(UInt8)")) + require.NoError(t, err) + colSliceMapStringString := []map[string]string{{"key1": "value1", "key2": "value2"}, {"key3": "value3"}} + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colSliceMapStringString, "Array(Map(String, String))")) + require.NoError(t, err) + colMapStringString := map[string]string{"key1": "value1", "key2": "value2"} + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colMapStringString, "Map(String, String)")) + require.NoError(t, err) + colMapStringInt64 := map[string]int64{"key1": 42, "key2": 84} + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(colMapStringInt64, "Map(String, Int64)")) + require.NoError(t, err) + + require.NoError(t, tx.Commit()) + + rows, err := conn.QueryContext(ctx, "SELECT c FROM test_dynamic") + require.NoError(t, err) + + var row chcol.Dynamic + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colInt64, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, dynamicTestDate, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colNil, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceUInt8, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringInt64, row.Any()) +} + +func TestDynamic_ScanWithType(t *testing.T) { + ctx := context.Background() + conn := setupDynamicTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_dynamic ( + c Dynamic + ) Engine = MergeTree() ORDER BY tuple() + ` + _, err := conn.ExecContext(ctx, ddl) + require.NoError(t, err) + + defer func() { + _, err := conn.ExecContext(ctx, "DROP TABLE IF EXISTS test_dynamic") + require.NoError(t, err) + }() + + tx, err := conn.BeginTx(ctx, nil) + require.NoError(t, err) + + batch, err := tx.PrepareContext(ctx, "INSERT INTO test_dynamic (c)") + require.NoError(t, err) + + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(true, "Bool")) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, clickhouse.NewDynamicWithType(int64(42), "Int64")) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, nil) + require.NoError(t, err) + + require.NoError(t, tx.Commit()) + + rows, err := conn.QueryContext(ctx, "SELECT c FROM test_dynamic") + require.NoError(t, err) + + var row chcol.Dynamic + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + require.Equal(t, "Bool", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, int64(42), row.Any()) + require.Equal(t, "Int64", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, nil, row.Any()) + require.Equal(t, "", row.Type()) +} From b8a324f4d796991d49c01e69566731834620f261 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 13:55:07 -0500 Subject: [PATCH 15/18] Add std sql test for Variant + min server version for examples --- examples/clickhouse_api/main_test.go | 1 - examples/clickhouse_api/variant.go | 5 + examples/std/main_test.go | 1 - examples/std/variant.go | 6 + tests/std/variant_test.go | 224 +++++++++++++++++++++++++++ 5 files changed, 235 insertions(+), 2 deletions(-) create mode 100644 tests/std/variant_test.go diff --git a/examples/clickhouse_api/main_test.go b/examples/clickhouse_api/main_test.go index c96d8884a4..d1b15637d9 100644 --- a/examples/clickhouse_api/main_test.go +++ b/examples/clickhouse_api/main_test.go @@ -211,7 +211,6 @@ func TestSSLNoVerify(t *testing.T) { } func TestVariantExample(t *testing.T) { - t.Skip("unsupported version") require.NoError(t, VariantExample()) } diff --git a/examples/clickhouse_api/variant.go b/examples/clickhouse_api/variant.go index 5d300b49a2..e6d4a656ff 100644 --- a/examples/clickhouse_api/variant.go +++ b/examples/clickhouse_api/variant.go @@ -35,6 +35,11 @@ func VariantExample() error { return err } + if !CheckMinServerVersion(conn, 24, 4, 0) { + fmt.Print("unsupported clickhouse version for Variant type") + return nil + } + err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_variant_example") if err != nil { return err diff --git a/examples/std/main_test.go b/examples/std/main_test.go index 012e3627b5..c1995c46fe 100644 --- a/examples/std/main_test.go +++ b/examples/std/main_test.go @@ -150,7 +150,6 @@ func TestConnectionSettings(t *testing.T) { } func TestVariantExample(t *testing.T) { - t.Skip("unsupported version") require.NoError(t, VariantExample()) } diff --git a/examples/std/variant.go b/examples/std/variant.go index 65062c7587..50bde691be 100644 --- a/examples/std/variant.go +++ b/examples/std/variant.go @@ -34,6 +34,12 @@ func VariantExample() error { if err != nil { return err } + + if !CheckMinServerVersion(conn, 24, 4, 0) { + fmt.Print("unsupported clickhouse version for Variant type") + return nil + } + defer func() { conn.Exec("DROP TABLE go_variant_example") }() diff --git a/tests/std/variant_test.go b/tests/std/variant_test.go new file mode 100644 index 0000000000..b0c6f99dae --- /dev/null +++ b/tests/std/variant_test.go @@ -0,0 +1,224 @@ +// Licensed to ClickHouse, Inc. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. ClickHouse, Inc. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package std + +import ( + "context" + "database/sql" + + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/chcol" + "github.com/stretchr/testify/require" + "testing" + "time" +) + +var variantTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") + +func setupVariantTest(t *testing.T) *sql.DB { + conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ + "max_execution_time": 60, + "allow_experimental_variant_type": true, + "allow_suspicious_variant_types": true, + }, nil, &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + }) + require.NoError(t, err) + + if !CheckMinServerVersion(conn, 24, 4, 0) { + t.Skip(fmt.Errorf("unsupported clickhouse version for Variant type")) + return nil + } + + return conn +} + +func TestVariant(t *testing.T) { + ctx := context.Background() + conn := setupVariantTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_variant ( + c Variant( + Bool, + Int64, + String, + DateTime64(3), + Array(String), + Array(UInt8), + Array(Map(String, String)), + Map(String, String), + Map(String, Int64), + ) + ) Engine = MergeTree() ORDER BY tuple() + ` + _, err := conn.ExecContext(ctx, ddl) + require.NoError(t, err) + defer func() { + _, err := conn.ExecContext(ctx, "DROP TABLE IF EXISTS test_variant") + require.NoError(t, err) + }() + + tx, err := conn.BeginTx(ctx, nil) + require.NoError(t, err) + + batch, err := tx.PrepareContext(ctx, "INSERT INTO test_variant (c)") + require.NoError(t, err) + + _, err = batch.ExecContext(ctx, true) + require.NoError(t, err) + colInt64 := int64(42) + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(colInt64, "Int64")) + require.NoError(t, err) + colString := "test" + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(colString, "String")) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(variantTestDate, "DateTime64(3)")) + require.NoError(t, err) + var colNil any = nil + _, err = batch.ExecContext(ctx, colNil) + require.NoError(t, err) + colSliceString := []string{"a", "b"} + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(colSliceString, "Array(String)")) + require.NoError(t, err) + colSliceUInt8 := []uint8{0xA, 0xB, 0xC} + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(colSliceUInt8, "Array(UInt8)")) + require.NoError(t, err) + colSliceMapStringString := []map[string]string{{"key1": "value1", "key2": "value2"}, {"key3": "value3"}} + _, err = batch.ExecContext(ctx, colSliceMapStringString) + require.NoError(t, err) + colMapStringString := map[string]string{"key1": "value1", "key2": "value2"} + _, err = batch.ExecContext(ctx, colMapStringString) + require.NoError(t, err) + colMapStringInt64 := map[string]int64{"key1": 42, "key2": 84} + _, err = batch.ExecContext(ctx, colMapStringInt64) + require.NoError(t, err) + + require.NoError(t, tx.Commit()) + + rows, err := conn.QueryContext(ctx, "SELECT c FROM test_variant") + require.NoError(t, err) + + var row chcol.Variant + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colInt64, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, variantTestDate, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colNil, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceUInt8, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colSliceMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringString, row.Any()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, colMapStringInt64, row.Any()) +} + +func TestVariant_ScanWithType(t *testing.T) { + ctx := context.Background() + conn := setupVariantTest(t) + + const ddl = ` + CREATE TABLE IF NOT EXISTS test_variant ( + c Variant(Bool, Int64) + ) Engine = MergeTree() ORDER BY tuple() + ` + _, err := conn.ExecContext(ctx, ddl) + require.NoError(t, err) + defer func() { + _, err := conn.ExecContext(ctx, "DROP TABLE IF EXISTS test_variant") + require.NoError(t, err) + }() + + tx, err := conn.BeginTx(ctx, nil) + require.NoError(t, err) + + batch, err := tx.PrepareContext(ctx, "INSERT INTO test_variant (c)") + require.NoError(t, err) + + _, err = batch.ExecContext(ctx, true) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, clickhouse.NewVariantWithType(int64(42), "Int64")) + require.NoError(t, err) + _, err = batch.ExecContext(ctx, nil) + require.NoError(t, err) + + require.NoError(t, tx.Commit()) + + rows, err := conn.QueryContext(ctx, "SELECT c FROM test_variant") + require.NoError(t, err) + + var row chcol.Variant + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, true, row.Any()) + require.Equal(t, "Bool", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, int64(42), row.Any()) + require.Equal(t, "Int64", row.Type()) + + require.True(t, rows.Next()) + err = rows.Scan(&row) + require.NoError(t, err) + require.Equal(t, nil, row.Any()) + require.Equal(t, "", row.Type()) +} From 016808979eb88eddf4c1e6e8b9a83ca81b0705d6 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 14:07:20 -0500 Subject: [PATCH 16/18] update minimum version for dynamic --- examples/clickhouse_api/dynamic.go | 2 +- examples/std/dynamic.go | 2 +- tests/dynamic_test.go | 2 +- tests/std/dynamic_test.go | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/clickhouse_api/dynamic.go b/examples/clickhouse_api/dynamic.go index 6b0fa28d3b..1f98b28bed 100644 --- a/examples/clickhouse_api/dynamic.go +++ b/examples/clickhouse_api/dynamic.go @@ -33,7 +33,7 @@ func DynamicExample() error { return err } - if !CheckMinServerVersion(conn, 24, 4, 0) { + if !CheckMinServerVersion(conn, 24, 8, 0) { fmt.Print("unsupported clickhouse version for Dynamic type") return nil } diff --git a/examples/std/dynamic.go b/examples/std/dynamic.go index 861d2e9ef5..4e917a70a5 100644 --- a/examples/std/dynamic.go +++ b/examples/std/dynamic.go @@ -33,7 +33,7 @@ func DynamicExample() error { return err } - if !CheckMinServerVersion(conn, 24, 4, 0) { + if !CheckMinServerVersion(conn, 24, 8, 0) { fmt.Print("unsupported clickhouse version for Dynamic type") return nil } diff --git a/tests/dynamic_test.go b/tests/dynamic_test.go index 3bda9fe601..76a52329c0 100644 --- a/tests/dynamic_test.go +++ b/tests/dynamic_test.go @@ -39,7 +39,7 @@ func setupDynamicTest(t *testing.T) driver.Conn { }) require.NoError(t, err) - if !CheckMinServerServerVersion(conn, 24, 4, 0) { + if !CheckMinServerServerVersion(conn, 24, 8, 0) { t.Skip(fmt.Errorf("unsupported clickhouse version for Dynamic type")) return nil } diff --git a/tests/std/dynamic_test.go b/tests/std/dynamic_test.go index a7d0cceff6..743c3ab98b 100644 --- a/tests/std/dynamic_test.go +++ b/tests/std/dynamic_test.go @@ -40,7 +40,7 @@ func setupDynamicTest(t *testing.T) *sql.DB { }) require.NoError(t, err) - if !CheckMinServerVersion(conn, 24, 4, 0) { + if !CheckMinServerVersion(conn, 24, 8, 0) { t.Skip(fmt.Errorf("unsupported clickhouse version for Dynamic type")) return nil } From ecd8d93921b8ac34a70cab0a1118883db8479317 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Fri, 10 Jan 2025 14:27:08 -0500 Subject: [PATCH 17/18] move settings outside initial connect for std sql tests --- examples/std/dynamic.go | 9 ++++++--- examples/std/variant.go | 15 +++++++++++---- tests/std/dynamic_test.go | 11 +++++++---- tests/std/variant_test.go | 18 +++++++++++++----- 4 files changed, 37 insertions(+), 16 deletions(-) diff --git a/examples/std/dynamic.go b/examples/std/dynamic.go index 4e917a70a5..c366a63ad5 100644 --- a/examples/std/dynamic.go +++ b/examples/std/dynamic.go @@ -26,9 +26,7 @@ import ( func DynamicExample() error { ctx := context.Background() - conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ - "allow_experimental_dynamic_type": true, - }, nil, nil) + conn, err := GetStdOpenDBConnection(clickhouse.Native, nil, nil, nil) if err != nil { return err } @@ -38,6 +36,11 @@ func DynamicExample() error { return nil } + _, err = conn.ExecContext(ctx, "SET allow_experimental_dynamic_type = 1") + if err != nil { + return err + } + defer func() { conn.Exec("DROP TABLE go_dynamic_example") }() diff --git a/examples/std/variant.go b/examples/std/variant.go index 50bde691be..e05767e289 100644 --- a/examples/std/variant.go +++ b/examples/std/variant.go @@ -27,10 +27,7 @@ import ( func VariantExample() error { ctx := context.Background() - conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ - "allow_experimental_variant_type": true, - "allow_suspicious_variant_types": true, - }, nil, nil) + conn, err := GetStdOpenDBConnection(clickhouse.Native, nil, nil, nil) if err != nil { return err } @@ -40,6 +37,16 @@ func VariantExample() error { return nil } + _, err = conn.ExecContext(ctx, "SET allow_experimental_variant_type = 1") + if err != nil { + return err + } + + _, err = conn.ExecContext(ctx, "SET allow_suspicious_variant_types = 1") + if err != nil { + return err + } + defer func() { conn.Exec("DROP TABLE go_variant_example") }() diff --git a/tests/std/dynamic_test.go b/tests/std/dynamic_test.go index 743c3ab98b..15d53cf634 100644 --- a/tests/std/dynamic_test.go +++ b/tests/std/dynamic_test.go @@ -32,10 +32,7 @@ import ( var dynamicTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") func setupDynamicTest(t *testing.T) *sql.DB { - conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ - "max_execution_time": 60, - "allow_experimental_dynamic_type": true, - }, nil, &clickhouse.Compression{ + conn, err := GetStdOpenDBConnection(clickhouse.Native, nil, nil, &clickhouse.Compression{ Method: clickhouse.CompressionLZ4, }) require.NoError(t, err) @@ -45,6 +42,12 @@ func setupDynamicTest(t *testing.T) *sql.DB { return nil } + _, err = conn.ExecContext(context.Background(), "SET allow_experimental_dynamic_type = 1") + if err != nil { + t.Fatal(err) + return nil + } + return conn } diff --git a/tests/std/variant_test.go b/tests/std/variant_test.go index b0c6f99dae..16b815f081 100644 --- a/tests/std/variant_test.go +++ b/tests/std/variant_test.go @@ -32,11 +32,7 @@ import ( var variantTestDate, _ = time.Parse(time.RFC3339, "2024-12-13T02:09:30.123Z") func setupVariantTest(t *testing.T) *sql.DB { - conn, err := GetStdOpenDBConnection(clickhouse.Native, clickhouse.Settings{ - "max_execution_time": 60, - "allow_experimental_variant_type": true, - "allow_suspicious_variant_types": true, - }, nil, &clickhouse.Compression{ + conn, err := GetStdOpenDBConnection(clickhouse.Native, nil, nil, &clickhouse.Compression{ Method: clickhouse.CompressionLZ4, }) require.NoError(t, err) @@ -46,6 +42,18 @@ func setupVariantTest(t *testing.T) *sql.DB { return nil } + _, err = conn.ExecContext(context.Background(), "SET allow_experimental_variant_type = 1") + if err != nil { + t.Fatal(err) + return nil + } + + _, err = conn.ExecContext(context.Background(), "SET allow_suspicious_variant_types = 1") + if err != nil { + t.Fatal(err) + return nil + } + return conn } From a1e7a29313c85c78e779466caefee495b56052d4 Mon Sep 17 00:00:00 2001 From: Spencer Torres Date: Mon, 13 Jan 2025 10:19:45 -0500 Subject: [PATCH 18/18] remove outdated version check --- tests/std/datetime64_test.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/tests/std/datetime64_test.go b/tests/std/datetime64_test.go index 406246767a..1dd05f7edb 100644 --- a/tests/std/datetime64_test.go +++ b/tests/std/datetime64_test.go @@ -38,10 +38,7 @@ func TestStdDateTime64(t *testing.T) { t.Run(fmt.Sprintf("%s Protocol", name), func(t *testing.T) { conn, err := GetStdDSNConnection(protocol, useSSL, nil) require.NoError(t, err) - if !CheckMinServerVersion(conn, 20, 3, 0) { - t.Skip(fmt.Errorf("unsupported clickhouse version")) - return - } + const ddl = ` CREATE TABLE test_datetime64 ( Col1 DateTime64(3) @@ -122,10 +119,6 @@ func TestStdDateTime64(t *testing.T) { t.Run("DateTime64 precision", func(t *testing.T) { conn, err := GetStdDSNConnection(protocol, useSSL, nil) require.NoError(t, err) - if !CheckMinServerVersion(conn, 20, 3, 0) { - t.Skip(fmt.Errorf("unsupported clickhouse version")) - return - } rows, err := conn.Query("SELECT toDateTime64(1546300800.123, 3)") require.NoError(t, err)