ClickHouse / clickhouse-go

Golang driver for ClickHouse
Apache License 2.0
2.92k stars 562 forks source link

v2.27.0 breaks compatibility with OrderedMap for inserts #1365

Open mdonkers opened 3 months ago

mdonkers commented 3 months ago

Observed

  1. Updated ClickHouse-go from v2.26.0 to v2.27.0
  2. Inserts fail with
    "error": "clickhouse [AppendRow]: Events.Attributes clickhouse [AppendRow]: converting column.clickHouseAttributeOrderedMap[github.com/dash0hq/pkg/common/column.Tuple] to Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3))) is unsupported. try using map[string]map[string]interface {}"
  3. Downgraded back to v2.26.0 and everything works fine again.

(suspecting https://github.com/ClickHouse/clickhouse-go/pull/1350 for the change)

We're using an implementation of OrderedMap (https://github.com/ClickHouse/clickhouse-go/blob/20fb5918d16eb0ea2a9ae1230658b319ef484920/lib/column/map.go#L40) for inserts. With a column of type Attributes Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3)))`

Expected behaviour

Ordered map inserts work, also for nested types

Error log

See above

Details

See above

Environment

jkaflik commented 3 months ago

Hi @mdonkers

Could you please share how does your (it can be anonymized) OrderedMap implementation looks like?

mdonkers commented 3 months ago

Hey Kuba,

Thx for looking into this. Unfortunately our OrderedMap implementation is complex in that it allows wrapping other OrderedMap instances. So I'll try to post the minimum necessary. Due to holiday not too much time for a full reproducer, perhaps if the below is not sufficient I can try to get more details later.

We're using a OrderedMap implementation 'implementing' the MapIter:

type attrToTupleMapIter struct {
    wmi            *maps.MapIter[pcommon.Value]
}

func newAttrToTupleMapIter(wmi *maps.MapIter[pcommon.Value]) *attrToTupleMapIter {
    return &attrToTupleMapIter{
        wmi:            wmi,
    }
}

func (i *attrToTupleMapIter) Next() bool {
    return i.wmi.Next()
}

func (i *attrToTupleMapIter) Key() any {
    return i.wmi.Key()
}

func (i *attrToTupleMapIter) Value() any {
    return column.TupleFromValue(i.wmi.ValueAsType())
}

type clickHouseAttributeOrderedMap[V otlp.AnyValueTuple] struct {
    wrappedMap     maps.OrderedMap[pcommon.Value]
    skipDash0Attrs bool
}

func (om *clickHouseAttributeOrderedMap[_]) Iterator() column.MapIterator {
    originalIterator := om.wrappedMap.Iterator()

    if mi, ok := originalIterator.(*maps.MapIter[pcommon.Value]); ok {
        return newAttrToTupleMapIter(mi)
    }

    panic("unexpected ordered map, needs to return maps.MapIter")
}

Then for the Value() function which does a conversion to tuples: (which should match the column definition: Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3)))

func TupleFromValue(value pcommon.Value) []any {
    switch value.Type() {
    case pcommon.ValueTypeEmpty:
        return []any{"", ValueTypeEmpty}
    case pcommon.ValueTypeStr:
        return []any{value.AsString(), ValueTypeString}
    case pcommon.ValueTypeBool:
        return []any{value.AsString(), ValueTypeBool}
    case pcommon.ValueTypeInt:
        return []any{value.AsString(), ValueTypeInt}
    }
    panic("unknown value type")
}

I'm expecting that []any{} array containing a string and int cannot get mapped correctly?

jkaflik commented 3 months ago

Thanks @mdonkers . I will be looking into this week.

jkaflik commented 3 months ago

@mdonkers that's very interesting case, because I got a reproducible example:

package issues

import (
    "context"
    "testing"

    "github.com/ClickHouse/clickhouse-go/v2/examples/clickhouse_api"
    "github.com/ClickHouse/clickhouse-go/v2/tests"
    "github.com/stretchr/testify/require"
)

func TestIssue1365(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 test_1365 (
                Col1 Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3)))
        ) Engine MergeTree() ORDER BY tuple()
        `
    err = conn.Exec(ctx, ddl)
    require.NoError(t, err)
    defer conn.Exec(ctx, "DROP TABLE test_1365")

    mapValue := map[string]string{
        "Value": "value1",
        "Type":  "String",
    }

    om := clickhouse_api.OrderedMap{}
    om.Put("key", mapValue)

    batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_1365")
    require.NoError(t, err)
    require.NoError(t, batch.Append(om))
    require.NoError(t, batch.Send())
}

However, it fails regardless of the driver's version. I tried every minor version down till v2.22.0:

clickhouse [AppendRow]: Col1 clickhouse [AppendRow]: converting clickhouse_api.OrderedMap to Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3))) is unsupported. try using map[string]map[string]interface {}

then, this works when you pass OrderedMap implementation as pointer to batch.Append:

require.NoError(t, batch.Append(&om))

Could you please confirm, before my further investigation, that this error didn't exist for past driver versions?

mdonkers commented 3 months ago

Sorry, just back from vacation. And thanks a lot for looking into this!

We're currently using v2.26.0 without problems... So at least from our perspective the issue did not appear earlier. Only when upgrading to v2.27.0 the problems showed up.

But I'm also very confused since I tried upgrading again (to the now released v2.28.0) and cannot get the problem reproduced again. Everything seems to behave fine now. The problem did not appear in our integration-tests but only after it got deployed. And in the mean time we did update ClickHouse (though I don't believe that should make a big difference). I'll see if I can get it reproduced some other way on our end but I have some doubts whether that will be possible.

jkaflik commented 3 months ago

Thanks @mdonkers . It's interesting and would confirm my investigation. Please let me know if you are able to reproduce this again.

mdonkers commented 3 months ago

Hi @jkaflik ,

Took a bit of effort, but managed to reproduce. And I understand the issue now (it's more complex than I initially thought). The problem is due to https://github.com/ClickHouse/clickhouse-go/pull/1350 though.

The problem doesn't surface with 'normal' IterableOrderedMap / OrderedMap usage, but only when having Nested columns. We have a column;

`Events` Nested (
        `Timestamp` DateTime64(9),
        `Name` LowCardinality(String),
        `Attributes` Map(LowCardinality(String), Tuple(Value String, Type Enum8('Empty' = 0, 'String' = 1, 'Bool' = 2, 'Int' = 3)))
    )

But I overlooked the Events.Attributes initially in the error message since we also have similar Attributes directly as column.

When calling batch.Append(), the Attributes are inserted as []column.IterableOrderedMap, which due to IterableOrderedMap being an interface, holds pointers to actual instances of our attrToTupleMapIter type.

What happens now due to the changed code in https://github.com/ClickHouse/clickhouse-go/pull/1350, is that the map.Append() code isn't getting called with a pointer to attrToTupleMapIter, but instead the pointer is dereferenced. That then results in the type conversion orderedMap, ok := v.(IterableOrderedMap) or orderedMap, ok := v.(OrderedMap) to not succeed. Because an interface is always it a pointer, but doesn't have a pointer anymore.

Hope this helps...

mdonkers commented 3 months ago

(I couldn't initially get it reproduced since we have a go.work file and that one still contained a replace directive and as such I was still running with v2.26.0 before)

jkaflik commented 2 months ago

@mdonkers thanks. That's a good piece of information. I will take a look on this tomorrow.

jkaflik commented 2 months ago

@mdonkers, unfortunately (given the short amount of time), I couldn't be able to get this reproduced reliably. I will have a look after my vacation in two weeks.

mdonkers commented 2 months ago

@jkaflik no worries! Just came back from vacation myself so I know the feeling with work piling up :-) We can simply stay on the current version. Enjoy your vacation!

joschi commented 2 months ago

@jkaflik Unfortunately the issue still exists in clickhouse-go v2.28.2.

earwin commented 1 month ago

@mdonkers, unfortunately (given the short amount of time), I couldn't be able to get this reproduced reliably.

There you are : ) 1365_test.go:

package issues

import (
    "context"
    "github.com/ClickHouse/clickhouse-go/v2/lib/column"
    "github.com/ClickHouse/clickhouse-go/v2/tests"
    "github.com/stretchr/testify/require"
    "testing"
)

type T1365OrderedMap int

func (t *T1365OrderedMap) Put(k any, v any) {
    if k == "K" && v == "V" {
        *t = 0xDEDEDEAD
    }
}
func (t *T1365OrderedMap) Iterator() column.MapIterator { return t }
func (t *T1365OrderedMap) Next() bool                   { *t++; return *t == 1 }
func (t *T1365OrderedMap) Key() any                     { return "K" }
func (t *T1365OrderedMap) Value() any                   { return "V" }

func TestIssue1365(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 test_1365 (
                Col1 Array(Map(String,String))
        ) Engine MergeTree() ORDER BY tuple()
        `
    err = conn.Exec(ctx, ddl)
    require.NoError(t, err)
    defer conn.Exec(ctx, "DROP TABLE test_1365")

    batch, err := conn.PrepareBatch(ctx, "INSERT INTO test_1365")
    require.NoError(t, err)

    var writeMaps []column.IterableOrderedMap
    writeMaps = append(writeMaps, new(T1365OrderedMap))
    writeMaps = append(writeMaps, new(T1365OrderedMap))

    err = batch.Append(writeMaps)
    require.NoError(t, err)

    err = batch.Send()
    require.NoError(t, err)

    rows, err := conn.Query(ctx, "SELECT * FROM test_1365")
    require.NoError(t, err)

    require.True(t, rows.Next())

    //var readMaps []*T1365OrderedMap
    //
    //err = rows.Scan(&readMaps)
    //require.NoError(t, err)
}

Works with v2.26.0, doesn't work with main. It is triggered with a slice of IOMs, so somewhat different from initial example, but I believe it has the same underlying problem — *IOM nested in something else gets dereferenced during traversal, and is no longer recognized by v.(column.IterableOrderedMap) stanza. I couldn't get it to Scan() back the slice even on v2.26.0, but that's a separate issue.

earwin commented 1 month ago

1350 feels weird, but the array handling approach itself feels even weirder...

Normally, when you're scanning some deeply nested structures, when you encounter a slice, you just iterate over it and recurse to your top-level type dispatcher. Top-level is where all the any unpacking and pointer dereferencing happens, in an unified manner, so it doesn't matter if you have a pointer-to-something as an array element, as a map key or value, as a structure field or whatever.

But here, array nesting Array(Array(Array(String))) is special-cased, you have that level var, and until you're down to the deepest level, you don't get a chance to call that top-level dispatcher. So #1350's author did not really have many options.

earwin commented 1 month ago

There you are. It's simple, it fixes our specific cases (or at least I hope so :D), but doesn't take away from the fact that CH types don't seem to be truly composable, as handled by this library.

mdonkers commented 1 month ago

Yep, that looks good to me. Thx for the effort @earwin

@jkaflik would you agree that change looks good and could be merged?