Skip to content

Commit

Permalink
Merge pull request #685 from ClickHouse/named_tuples
Browse files Browse the repository at this point in the history
Typed Maps/Slices for tuples
  • Loading branch information
gingerwizard authored Jul 25, 2022
2 parents cd34002 + 4fff759 commit 67708b0
Show file tree
Hide file tree
Showing 4 changed files with 413 additions and 181 deletions.
5 changes: 2 additions & 3 deletions lib/column/json.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,8 +217,7 @@ func getStructFieldName(field reflect.StructField) (string, bool) {
}

// ensures numeric keys and ` are escaped properly
func getMapFieldName(key reflect.Value) string {
name := key.Interface().(string)
func getMapFieldName(name string) string {
if !escapeColRegex.MatchString(name) {
return fmt.Sprintf("`%s`", colEscape.Replace(name))
}
Expand Down Expand Up @@ -427,7 +426,7 @@ func iterateMap(mapVal reflect.Value, col JSONParent, preFill int) error {
addedColumns := make([]string, len(mapVal.MapKeys()), len(mapVal.MapKeys()))
newColumn := false
for i, key := range mapVal.MapKeys() {
name := getMapFieldName(key)
name := getMapFieldName(key.Interface().(string))
if _, ok := columnLookup[name]; !ok && len(currentColumns) > 0 {
// new column - need to handle
preFill = numRows
Expand Down
72 changes: 35 additions & 37 deletions lib/column/tuple.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,17 +439,10 @@ func (col *Tuple) ScanRow(dest interface{}, row int) error {
}

func (col *Tuple) Append(v interface{}) (nulls []uint8, err error) {
switch v := v.(type) {
case [][]interface{}:
for _, v := range v {
if err := col.AppendRow(v); err != nil {
return nil, err
}
}
return nil, nil
case []*[]interface{}:
for _, v := range v {
if err := col.AppendRow(v); err != nil {
value := reflect.ValueOf(v)
if value.Kind() == reflect.Slice {
for i := 0; i < value.Len(); i++ {
if err := col.AppendRow(value.Index(i).Interface()); err != nil {
return nil, err
}
}
Expand All @@ -463,55 +456,60 @@ func (col *Tuple) Append(v interface{}) (nulls []uint8, err error) {
}

func (col *Tuple) AppendRow(v interface{}) error {
switch v := v.(type) {
case []interface{}:
if len(v) != len(col.columns) {
// allows support of tuples where map or slice is typed and NOT interface{}. Will fail if tuple isn't consistent
value := reflect.ValueOf(v)
if value.Kind() == reflect.Pointer {
value = value.Elem()
}
switch value.Kind() {
case reflect.Map:
if !col.isNamed {
return &Error{
ColumnType: string(col.chType),
Err: fmt.Errorf("invalid size. expected %d got %d", len(col.columns), len(v)),
}
}
for i, v := range v {
if err := col.columns[i].AppendRow(v); err != nil {
return err
Err: fmt.Errorf("converting from %T is not supported for unnamed tuples - use a slice", v),
}
}
return nil
case *[]interface{}:
if v == nil {
return &ColumnConverterError{
Op: "AppendRow",
To: string(col.chType),
From: fmt.Sprintf("%T", v),
Hint: "invalid (nil) pointer value",
if value.Type().Key().Kind() != reflect.String {
return &Error{
ColumnType: fmt.Sprint(value.Type().Key().Kind()),
Err: fmt.Errorf("map keys must be string for column %s", col.Name()),
}
}
if len(*v) != len(col.columns) {
if value.Len() != len(col.columns) {
return &Error{
ColumnType: string(col.chType),
Err: fmt.Errorf("invalid size. expected %d got %d", len(col.columns), len(*v)),
Err: fmt.Errorf("invalid size. expected %d got %d", len(col.columns), value.Len()),
}
}
for i, v := range *v {
if err := col.columns[i].AppendRow(v); err != nil {
for _, key := range value.MapKeys() {
name := getMapFieldName(key.Interface().(string))
if _, ok := col.index[name]; !ok {
return &Error{
ColumnType: string(col.chType),
Err: fmt.Errorf("sub column '%s' does not exist in %s", name, col.Name()),
}
}
if err := col.columns[col.index[name]].AppendRow(value.MapIndex(key).Interface()); err != nil {
return err
}
}
return nil
case map[string]interface{}:
if !col.isNamed {
case reflect.Slice:
if value.Len() != len(col.columns) {
return &Error{
ColumnType: string(col.chType),
Err: fmt.Errorf("converting from %T is not supported for unnamed tuples - use a slice", v),
Err: fmt.Errorf("invalid size. expected %d got %d", len(col.columns), value.Len()),
}
}
for name, v := range v {
if err := col.columns[col.index[name]].AppendRow(v); err != nil {
for i := 0; i < value.Len(); i++ {
elem := value.Index(i)
if err := col.columns[i].AppendRow(elem.Interface()); err != nil {
return err
}
}
return nil
}

return &ColumnConverterError{
Op: "AppendRow",
To: string(col.chType),
Expand Down
116 changes: 116 additions & 0 deletions tests/std/tuples_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
// 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 (
"github.com/stretchr/testify/require"
"testing"
"time"

"github.com/ClickHouse/clickhouse-go/v2"
"github.com/stretchr/testify/assert"
)

func TestTuple(t *testing.T) {

var (
conn = clickhouse.OpenDB(&clickhouse.Options{
Addr: []string{"127.0.0.1:9000"},
Auth: clickhouse.Auth{
Database: "default",
Username: "default",
Password: "",
},
})
)
loc, err := time.LoadLocation("Europe/Lisbon")
require.NoError(t, err)
localTime := testDate.In(loc)

if err := checkMinServerVersion(conn, 21, 9, 0); err != nil {
t.Skip(err.Error())
return
}
const ddl = `
CREATE TABLE test_tuple (
Col1 Tuple(String, Int64)
, Col2 Tuple(String, Int8, DateTime('Europe/Lisbon'))
, Col3 Tuple(name1 DateTime('Europe/Lisbon'), name2 FixedString(2), name3 Map(String, String))
, Col4 Array(Array( Tuple(String, Int64) ))
, Col5 Tuple(LowCardinality(String), Array(LowCardinality(String)))
, Col6 Tuple(LowCardinality(Nullable(String)), Array(LowCardinality(Nullable(String))))
, Col7 Tuple(String, Int64)
) Engine Memory
`
defer func() {
conn.Exec("DROP TABLE test_tuple")
}()
_, err = conn.Exec(ddl)
require.NoError(t, err)
scope, err := conn.Begin()
require.NoError(t, err)
batch, err := scope.Prepare("INSERT INTO test_tuple")
require.NoError(t, err)
var (
col1Data = []interface{}{"A", int64(42)}
col2Data = []interface{}{"B", int8(1), localTime.Truncate(time.Second)}
col3Data = map[string]interface{}{
"name1": localTime.Truncate(time.Second),
"name2": "CH",
"name3": map[string]string{
"key": "value",
},
}
col4Data = [][][]interface{}{
[][]interface{}{
[]interface{}{"Hi", int64(42)},
},
}
col5Data = []interface{}{
"LCString",
[]string{"A", "B", "C"},
}
str = "LCString"
col6Data = []interface{}{
&str,
[]*string{&str, nil, &str},
}
col7Data = &[]interface{}{"C", int64(42)}
)
_, err = batch.Exec(col1Data, col2Data, col3Data, col4Data, col5Data, col6Data, col7Data)
require.NoError(t, err)
require.NoError(t, scope.Commit())
var (
col1 interface{}
col2 interface{}
// col3 is a named tuple - we can use map
col3 interface{}
col4 interface{}
col5 interface{}
col6 interface{}
col7 interface{}
)
require.NoError(t, conn.QueryRow("SELECT * FROM test_tuple").Scan(&col1, &col2, &col3, &col4, &col5, &col6, &col7))
assert.NoError(t, err)
assert.Equal(t, toJson(col1Data), toJson(col1))
assert.Equal(t, toJson(col2Data), toJson(col2))
assert.JSONEq(t, toJson(col3Data), toJson(col3))
assert.Equal(t, toJson(col4Data), toJson(col4))
assert.Equal(t, toJson(col5Data), toJson(col5))
assert.Equal(t, toJson(col6Data), toJson(col6))
assert.Equal(t, toJson(col7Data), toJson(col7))
}
Loading

0 comments on commit 67708b0

Please sign in to comment.