From 248bc2aeef74d18df605dc5e858f877df6ff8586 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sat, 19 Oct 2024 09:43:39 -0700 Subject: [PATCH 01/33] feat(sink): meter event --- .../kafkaingest/serializer/serializer.go | 16 +- openmeter/meter/parse.go | 96 +++++++++ .../meter/{validate_test.go => parse_test.go} | 84 +++++++- openmeter/meter/validate.go | 68 ------- openmeter/sink/models/models.go | 9 +- openmeter/sink/namespaces.go | 58 +++--- openmeter/sink/storage.go | 161 +++++++-------- openmeter/sink/storage_test.go | 79 -------- .../clickhouse_connector/connector.go | 18 ++ .../clickhouse_connector/event_query.go | 174 ++++++++++++++++ .../clickhouse_connector/event_query_test.go | 185 ++++++++++++++++++ .../clickhouse_connector/meter_event_query.go | 131 +++++++++++++ .../streaming/clickhouse_connector/query.go | 106 ---------- .../clickhouse_connector/query_test.go | 124 ------------ 14 files changed, 811 insertions(+), 498 deletions(-) create mode 100644 openmeter/meter/parse.go rename openmeter/meter/{validate_test.go => parse_test.go} (54%) delete mode 100644 openmeter/meter/validate.go delete mode 100644 openmeter/sink/storage_test.go create mode 100644 openmeter/streaming/clickhouse_connector/event_query.go create mode 100644 openmeter/streaming/clickhouse_connector/event_query_test.go create mode 100644 openmeter/streaming/clickhouse_connector/meter_event_query.go diff --git a/openmeter/ingest/kafkaingest/serializer/serializer.go b/openmeter/ingest/kafkaingest/serializer/serializer.go index e486ca31f..d99a9f585 100644 --- a/openmeter/ingest/kafkaingest/serializer/serializer.go +++ b/openmeter/ingest/kafkaingest/serializer/serializer.go @@ -3,6 +3,7 @@ package serializer import ( _ "embed" "encoding/json" + "time" "github.com/cloudevents/sdk-go/v2/event" ) @@ -16,13 +17,12 @@ type Serializer interface { } type CloudEventsKafkaPayload struct { - Id string `json:"id"` - Type string `json:"type"` - Source string `json:"source"` - Subject string `json:"subject"` - // Note: By converting to unix timestamp we loose timezone information. - Time int64 `json:"time"` - Data string `json:"data"` + Id string `json:"id"` + Type string `json:"type"` + Source string `json:"source"` + Subject string `json:"subject"` + Time time.Time `json:"time"` + Data string `json:"data"` } func toCloudEventsKafkaPayload(ev event.Event) (CloudEventsKafkaPayload, error) { @@ -31,7 +31,7 @@ func toCloudEventsKafkaPayload(ev event.Event) (CloudEventsKafkaPayload, error) Type: ev.Type(), Source: ev.Source(), Subject: ev.Subject(), - Time: ev.Time().Unix(), + Time: ev.Time(), } // We try to parse data as JSON. diff --git a/openmeter/meter/parse.go b/openmeter/meter/parse.go new file mode 100644 index 000000000..29d5a6ba4 --- /dev/null +++ b/openmeter/meter/parse.go @@ -0,0 +1,96 @@ +package meter + +import ( + "errors" + "fmt" + "strconv" + + "github.com/cloudevents/sdk-go/v2/event" + "github.com/oliveagle/jsonpath" + + "github.com/openmeterio/openmeter/pkg/models" +) + +// ParseEvent validates and parses an event against a meter. +func ParseEvent(meter Meter, ev event.Event) (float64, map[string]string, error) { + // Parse CloudEvents data + var data interface{} + + err := ev.DataAs(&data) + if err != nil { + return 0, map[string]string{}, errors.New("cannot unmarshal event data") + } + + // Parse group by fields + groupBy := parseGroupBy(meter, data) + + // We can skip count events as they don't have value property + if meter.Aggregation == MeterAggregationCount { + return 1, groupBy, nil + } + + // Get value from event data by value property + rawValue, err := jsonpath.JsonPathLookup(data, meter.ValueProperty) + if err != nil { + return 0, groupBy, fmt.Errorf("event data is missing value property at %q", meter.ValueProperty) + } + + if rawValue == nil { + return 0, groupBy, errors.New("event data value cannot be null") + } + + // Aggregation specific value validation + switch meter.Aggregation { + // UNIQUE_COUNT aggregation requires string property value + case MeterAggregationUniqueCount: + switch value := rawValue.(type) { + case string: + return 1, groupBy, nil + case float64: + return value, groupBy, nil + + default: + return 0, groupBy, errors.New("event data value property must be string for unique count aggregation") + } + + // SUM, AVG, MIN, MAX aggregations require float64 parsable value property value + case MeterAggregationSum, MeterAggregationAvg, MeterAggregationMin, MeterAggregationMax: + switch value := rawValue.(type) { + case string: + _, err = strconv.ParseFloat(value, 64) + if err != nil { + // TODO: omit value or make sure it's length is not too long + return 0, groupBy, fmt.Errorf("event data value cannot be parsed as float64: %s", value) + } + + case float64: + return value, groupBy, nil + + default: + return 0, groupBy, errors.New("event data value property cannot be parsed") + } + } + + return 0, groupBy, fmt.Errorf("unknown meter aggregation: %s", meter.Aggregation) +} + +// parseGroupBy parses the group by fields from the event data +func parseGroupBy(meter models.Meter, data interface{}) map[string]string { + groupBy := map[string]string{} + + // Group by fields + for groupByKey, groupByPath := range meter.GroupBy { + var groupByValue string + + rawGroupBy, err := jsonpath.JsonPathLookup(data, groupByPath) + if err != nil { + groupByValue = "" + } else { + groupByValue = fmt.Sprintf("%v", rawGroupBy) + } + + groupBy[groupByKey] = groupByValue + } + + return groupBy +} diff --git a/openmeter/meter/validate_test.go b/openmeter/meter/parse_test.go similarity index 54% rename from openmeter/meter/validate_test.go rename to openmeter/meter/parse_test.go index 0c485677a..0adb37c9e 100644 --- a/openmeter/meter/validate_test.go +++ b/openmeter/meter/parse_test.go @@ -11,8 +11,8 @@ import ( "github.com/openmeterio/openmeter/openmeter/meter" ) -func TestValidateEvent(t *testing.T) { - m := meter.Meter{ +func TestParseEvent(t *testing.T) { + meterSum := meter.Meter{ Namespace: "default", Slug: "m1", Description: "", @@ -26,13 +26,71 @@ func TestValidateEvent(t *testing.T) { WindowSize: meter.WindowSizeMinute, } + meterCount := meter.Meter{ + Namespace: "default", + Slug: "m2", + Description: "", + Aggregation: "COUNT", + EventType: "api-calls", + WindowSize: meter.WindowSizeMinute, + } + tests := []struct { description string + meter meter.Meter event func(t *testing.T) event.Event want error + value float64 + groupBy map[string]string }{ + { + description: "should parse event", + meter: meterSum, + event: func(t *testing.T) event.Event { + ev := event.New() + ev.SetType("api-calls") + + err := ev.SetData(event.ApplicationJSON, []byte(`{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`)) + require.NoError(t, err) + + return ev + }, + value: 100, + groupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, + }, + { + description: "should parse count as value one", + meter: meterCount, + event: func(t *testing.T) event.Event { + ev := event.New() + ev.SetType("api-calls") + + return ev + }, + value: 1, + groupBy: map[string]string{}, + }, + { + description: "should parse event with missing group by properties", + meter: meterSum, + event: func(t *testing.T) event.Event { + ev := event.New() + ev.SetType("api-calls") + + err := ev.SetData(event.ApplicationJSON, []byte(`{"duration_ms": 100}`)) + require.NoError(t, err) + + return ev + }, + value: 100, + groupBy: map[string]string{}, + }, { description: "should return error with invalid json", + meter: meterSum, event: func(t *testing.T) event.Event { ev := event.New() ev.SetType("api-calls") @@ -42,10 +100,12 @@ func TestValidateEvent(t *testing.T) { return ev }, - want: errors.New("cannot unmarshal event data"), + want: errors.New("cannot unmarshal event data"), + groupBy: map[string]string{}, }, { description: "should return error with value property not found", + meter: meterSum, event: func(t *testing.T) event.Event { ev := event.New() ev.SetType("api-calls") @@ -56,9 +116,14 @@ func TestValidateEvent(t *testing.T) { return ev }, want: errors.New("event data is missing value property at \"$.duration_ms\""), + groupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, }, { description: "should return error when value property is null", + meter: meterSum, event: func(t *testing.T) event.Event { ev := event.New() ev.SetType("api-calls") @@ -69,9 +134,14 @@ func TestValidateEvent(t *testing.T) { return ev }, want: errors.New("event data value cannot be null"), + groupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, }, { description: "should return error when value property cannot be parsed as number", + meter: meterSum, event: func(t *testing.T) event.Event { ev := event.New() ev.SetType("api-calls") @@ -82,6 +152,10 @@ func TestValidateEvent(t *testing.T) { return ev }, want: errors.New("event data value cannot be parsed as float64: not a number"), + groupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, }, } @@ -89,7 +163,7 @@ func TestValidateEvent(t *testing.T) { test := test t.Run(test.description, func(t *testing.T) { - err := meter.ValidateEvent(m, test.event(t)) + value, groupBy, err := meter.ParseEvent(test.meter, test.event(t)) if test.want == nil { assert.Nil(t, err) @@ -97,6 +171,8 @@ func TestValidateEvent(t *testing.T) { } assert.Equal(t, test.want, err) + assert.Equal(t, test.value, value) + assert.Equal(t, test.groupBy, groupBy) }) } } diff --git a/openmeter/meter/validate.go b/openmeter/meter/validate.go deleted file mode 100644 index dfe186503..000000000 --- a/openmeter/meter/validate.go +++ /dev/null @@ -1,68 +0,0 @@ -package meter - -import ( - "errors" - "fmt" - "strconv" - - "github.com/cloudevents/sdk-go/v2/event" - "github.com/oliveagle/jsonpath" -) - -// ValidateEvent validates an event against a meter. -func ValidateEvent(meter Meter, ev event.Event) error { - // Parse CloudEvents data - var data interface{} - - err := ev.DataAs(&data) - if err != nil { - return errors.New("cannot unmarshal event data") - } - - // We can skip count events as they don't have value property - if meter.Aggregation == MeterAggregationCount { - return nil - } - - // Get value from event data by value property - rawValue, err := jsonpath.JsonPathLookup(data, meter.ValueProperty) - if err != nil { - return fmt.Errorf("event data is missing value property at %q", meter.ValueProperty) - } - - if rawValue == nil { - return errors.New("event data value cannot be null") - } - - // Aggregation specific value validation - switch meter.Aggregation { - // UNIQUE_COUNT aggregation requires string property value - case MeterAggregationUniqueCount: - switch rawValue.(type) { - case string, float64: - // No need to do anything - - default: - return errors.New("event data value property must be string for unique count aggregation") - } - - // SUM, AVG, MIN, MAX aggregations require float64 parsable value property value - case MeterAggregationSum, MeterAggregationAvg, MeterAggregationMin, MeterAggregationMax: - switch value := rawValue.(type) { - case string: - _, err = strconv.ParseFloat(value, 64) - if err != nil { - // TODO: omit value or make sure it's length is not too long - return fmt.Errorf("event data value cannot be parsed as float64: %s", value) - } - - case float64: - // No need to do anything - - default: - return errors.New("event data value property cannot be parsed") - } - } - - return nil -} diff --git a/openmeter/sink/models/models.go b/openmeter/sink/models/models.go index 3db49cd3b..6673686d8 100644 --- a/openmeter/sink/models/models.go +++ b/openmeter/sink/models/models.go @@ -15,7 +15,14 @@ type SinkMessage struct { Serialized *serializer.CloudEventsKafkaPayload Status ProcessingStatus // Meters contains the list of meters this message affects - Meters []models.Meter + Meters []models.Meter + MeterEvents []MeterEvent +} + +type MeterEvent struct { + Meter *models.Meter + Value float64 + GroupBy map[string]string } type ProcessingState int8 diff --git a/openmeter/sink/namespaces.go b/openmeter/sink/namespaces.go index 16a81749c..2f3a81814 100644 --- a/openmeter/sink/namespaces.go +++ b/openmeter/sink/namespaces.go @@ -4,12 +4,11 @@ import ( "context" "errors" "fmt" - "time" "github.com/cloudevents/sdk-go/v2/event" "github.com/openmeterio/openmeter/openmeter/ingest/kafkaingest/serializer" - "github.com/openmeterio/openmeter/openmeter/meter" + ommeter "github.com/openmeterio/openmeter/openmeter/meter" sinkmodels "github.com/openmeterio/openmeter/openmeter/sink/models" "github.com/openmeterio/openmeter/pkg/models" ) @@ -60,9 +59,35 @@ func (n *NamespaceStore) ValidateEvent(_ context.Context, m *sinkmodels.SinkMess // // On the other hand we still want to collect the list of affected meters // for the FlushEventHandler. - if m.Status.Error == nil { - validateEventWithMeter(meter, m) + if m.Status.Error != nil { + return } + + // Parse kafka event + event, err := kafkaPayloadToCloudEvents(*m.Serialized) + if err != nil { + m.Status = sinkmodels.ProcessingStatus{ + State: sinkmodels.INVALID, + Error: errors.New("cannot parse event"), + } + } + + // Validate event against meter + value, groupBy, err := ommeter.ParseEvent(meter, event) + if err != nil { + m.Status = sinkmodels.ProcessingStatus{ + State: sinkmodels.INVALID, + Error: err, + } + + return + } + + m.MeterEvents = append(m.MeterEvents, sinkmodels.MeterEvent{ + Meter: &meter, + Value: value, + GroupBy: groupBy, + }) } } @@ -82,7 +107,7 @@ func kafkaPayloadToCloudEvents(payload serializer.CloudEventsKafkaPayload) (even ev.SetType(payload.Type) ev.SetSource(payload.Source) ev.SetSubject(payload.Subject) - ev.SetTime(time.Unix(payload.Time, 0)) + ev.SetTime(payload.Time) err := ev.SetData(event.ApplicationJSON, []byte(payload.Data)) if err != nil { @@ -91,26 +116,3 @@ func kafkaPayloadToCloudEvents(payload serializer.CloudEventsKafkaPayload) (even return ev, nil } - -// validateEventWithMeter validates a single event against a single meter -func validateEventWithMeter(m models.Meter, sm *sinkmodels.SinkMessage) { - ev, err := kafkaPayloadToCloudEvents(*sm.Serialized) - if err != nil { - sm.Status = sinkmodels.ProcessingStatus{ - State: sinkmodels.INVALID, - Error: errors.New("cannot parse event"), - } - - return - } - - err = meter.ValidateEvent(m, ev) - if err != nil { - sm.Status = sinkmodels.ProcessingStatus{ - State: sinkmodels.INVALID, - Error: err, - } - - return - } -} diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index bd7109532..62aa9768d 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -3,11 +3,9 @@ package sink import ( "context" "fmt" - "strings" "time" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/huandu/go-sqlbuilder" sinkmodels "github.com/openmeterio/openmeter/openmeter/sink/models" "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" @@ -51,67 +49,18 @@ type ClickHouseStorage struct { config ClickHouseStorageConfig } +// BatchInsert inserts multiple messages into ClickHouse. func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmodels.SinkMessage) error { - query := InsertEventsQuery{ - Clock: realClock{}, - Database: c.config.Database, - Messages: messages, - QuerySettings: c.config.QuerySettings, - } - sql, args, err := query.ToSQL() - if err != nil { - return err - } - - // By default, ClickHouse is writing data synchronously. - // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts - if c.config.AsyncInsert { - // With the `wait_for_async_insert` setting, you can configure - // if you want an insert statement to return with an acknowledgment - // either immediately after the data got inserted into the buffer. - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } + var rawEvents []clickhouse_connector.CHEvent + var meterEvents []clickhouse_connector.CHMeterEvent - if err != nil { - return fmt.Errorf("failed to batch insert events: %w", err) - } - - return nil -} - -type InsertEventsQuery struct { - Clock Clock - Database string - Messages []sinkmodels.SinkMessage - QuerySettings map[string]string -} - -func (q InsertEventsQuery) ToSQL() (string, []interface{}, error) { - tableName := clickhouse_connector.GetEventsTableName(q.Database) - - query := sqlbuilder.ClickHouse.NewInsertBuilder() - query.InsertInto(tableName) - query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") - - // Add settings - var settings []string - for key, value := range q.QuerySettings { - settings = append(settings, fmt.Sprintf("%s = %s", key, value)) - } - - if len(settings) > 0 { - query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) - } - - for _, message := range q.Messages { + for _, message := range messages { var eventErr string if message.Status.Error != nil { eventErr = message.Status.Error.Error() } - storedAt := q.Clock.Now() + storedAt := time.Now() ingestedAt := storedAt if message.KafkaMessage != nil { @@ -128,33 +77,85 @@ func (q InsertEventsQuery) ToSQL() (string, []interface{}, error) { } } - query.Values( - message.Namespace, - eventErr, - message.Serialized.Id, - message.Serialized.Type, - message.Serialized.Source, - message.Serialized.Subject, - message.Serialized.Time, - message.Serialized.Data, - ingestedAt, - storedAt, - ) + rawEvent := clickhouse_connector.CHEvent{ + Namespace: message.Namespace, + ValidationError: eventErr, + ID: message.Serialized.Id, + Type: message.Serialized.Type, + Source: message.Serialized.Source, + Subject: message.Serialized.Subject, + Time: message.Serialized.Time, + Data: message.Serialized.Data, + IngestedAt: ingestedAt, + StoredAt: storedAt, + } + + rawEvents = append(rawEvents, rawEvent) + + // Meter events per meter + for _, meterEvent := range message.MeterEvents { + meterEvent := clickhouse_connector.CHMeterEvent{ + Namespace: message.Namespace, + Time: rawEvent.Time, + Meter: meterEvent.Meter.ID, + Subject: rawEvent.Subject, + Value: meterEvent.Value, + GroupBy: meterEvent.GroupBy, + EventID: rawEvent.ID, + EventSource: rawEvent.Source, + EventType: rawEvent.Type, + StoredAt: rawEvent.StoredAt, + IngestedAt: rawEvent.IngestedAt, + } + + meterEvents = append(meterEvents, meterEvent) + } } - sql, args := query.Build() - return sql, args, nil -} + var err error -// Clock is an interface for getting the current time. -// It is used to make the code testable. -type Clock interface { - Now() time.Time -} + // Insert raw events + query := clickhouse_connector.InsertEventsQuery{ + Database: c.config.Database, + Events: rawEvents, + QuerySettings: c.config.QuerySettings, + } + sql, args := query.ToSQL() + + // By default, ClickHouse is writing data synchronously. + // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts + if c.config.AsyncInsert { + // With the `wait_for_async_insert` setting, you can configure + // if you want an insert statement to return with an acknowledgment + // either immediately after the data got inserted into the buffer. + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } -// realClock implements Clock using the system clock. -type realClock struct{} + if err != nil { + return fmt.Errorf("failed to batch insert raw events: %w", err) + } -func (realClock) Now() time.Time { - return time.Now() + // Insert meter events + if len(meterEvents) > 0 { + query := clickhouse_connector.InsertMeterEventsQuery{ + Database: c.config.Database, + MeterEvents: meterEvents, + QuerySettings: c.config.QuerySettings, + } + sql, args := query.ToSQL() + + if c.config.AsyncInsert { + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert meter events: %w", err) + } + } + + return nil } diff --git a/openmeter/sink/storage_test.go b/openmeter/sink/storage_test.go deleted file mode 100644 index 16aacd5bf..000000000 --- a/openmeter/sink/storage_test.go +++ /dev/null @@ -1,79 +0,0 @@ -package sink_test - -import ( - "errors" - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/openmeter/ingest/kafkaingest/serializer" - "github.com/openmeterio/openmeter/openmeter/sink" - sinkmodels "github.com/openmeterio/openmeter/openmeter/sink/models" -) - -func TestInsertEventsQuery(t *testing.T) { - now := time.Now() - - query := sink.InsertEventsQuery{ - Clock: mockClock{now: now}, - Database: "database", - Messages: []sinkmodels.SinkMessage{ - { - Namespace: "my_namespace", - Serialized: &serializer.CloudEventsKafkaPayload{ - Id: "1", - Source: "source", - Subject: "subject-1", - Time: now.UnixMilli(), - Type: "api-calls", - Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, - }, - }, - { - Namespace: "my_namespace", - Serialized: &serializer.CloudEventsKafkaPayload{ - Id: "2", - Source: "source", - Subject: "subject-2", - Time: now.UnixMilli(), - Type: "api-calls", - Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, - }, - }, - { - Namespace: "my_namespace", - Status: sinkmodels.ProcessingStatus{ - State: sinkmodels.INVALID, - Error: errors.New("event data value cannot be parsed as float64: not a number"), - }, - Serialized: &serializer.CloudEventsKafkaPayload{ - Id: "3", - Source: "source", - Subject: "subject-2", - Time: now.UnixMilli(), - Type: "api-calls", - Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, - }, - }, - }, - } - - sql, args, err := query.ToSQL() - assert.NoError(t, err) - - assert.Equal(t, []interface{}{ - "my_namespace", "", "1", "api-calls", "source", "subject-1", now.UnixMilli(), `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "", "2", "api-calls", "source", "subject-2", now.UnixMilli(), `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now.UnixMilli(), `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, - }, args) - assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) -} - -type mockClock struct { - now time.Time -} - -func (m mockClock) Now() time.Time { - return m.now -} diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index d999b08a5..dec57e9d2 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -151,6 +151,11 @@ func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace str return fmt.Errorf("create namespace in clickhouse: %w", err) } + err = c.createMeterEventTable(ctx) + if err != nil { + return fmt.Errorf("create namespace in clickhouse: %w", err) + } + return nil } @@ -216,6 +221,19 @@ func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { return nil } +func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { + table := createMeterEventTable{ + Database: c.config.Database, + } + + err := c.config.ClickHouse.Exec(ctx, table.toSQL()) + if err != nil { + return fmt.Errorf("create meter event table: %w", err) + } + + return nil +} + func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { table := queryEventsTable{ Database: c.config.Database, diff --git a/openmeter/streaming/clickhouse_connector/event_query.go b/openmeter/streaming/clickhouse_connector/event_query.go new file mode 100644 index 000000000..126deddc6 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector/event_query.go @@ -0,0 +1,174 @@ +package clickhouse_connector + +import ( + _ "embed" + "fmt" + "strings" + "time" + + "github.com/huandu/go-sqlbuilder" +) + +// CHEvent represents a raw event in ClickHouse +type CHEvent struct { + Namespace string `ch:"namespace"` + ValidationError string `ch:"validation_error"` + ID string `ch:"id"` + Type string `ch:"type"` + Source string `ch:"source"` + Subject string `ch:"subject"` + Time time.Time `ch:"time"` + Data string `ch:"data"` + IngestedAt time.Time `ch:"ingested_at"` + StoredAt time.Time `ch:"stored_at"` +} + +// Create Events Table +type createEventsTable struct { + Database string +} + +func (d createEventsTable) toSQL() string { + tableName := GetEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() + sb.CreateTable(tableName) + sb.IfNotExists() + sb.Define("namespace", "String") + sb.Define("validation_error", "String") + sb.Define("id", "String") + sb.Define("type", "LowCardinality(String)") + sb.Define("subject", "String") + sb.Define("source", "String") + sb.Define("time", "DateTime") + sb.Define("data", "String") + sb.Define("ingested_at", "DateTime") + sb.Define("stored_at", "DateTime") + sb.SQL("ENGINE = MergeTree") + sb.SQL("PARTITION BY toYYYYMM(time)") + sb.SQL("ORDER BY (namespace, time, type, subject)") + + sql, _ := sb.Build() + return sql +} + +// Query Events Table +type queryEventsTable struct { + Database string + Namespace string + From *time.Time + To *time.Time + IngestedAtFrom *time.Time + IngestedAtTo *time.Time + ID *string + Subject *string + HasError *bool + Limit int +} + +func (d queryEventsTable) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + where := []string{} + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") + query.From(tableName) + + where = append(where, query.Equal("namespace", d.Namespace)) + if d.From != nil { + where = append(where, query.GreaterEqualThan("time", d.From.Unix())) + } + if d.To != nil { + where = append(where, query.LessEqualThan("time", d.To.Unix())) + } + if d.IngestedAtFrom != nil { + where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) + } + if d.IngestedAtTo != nil { + where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) + } + if d.ID != nil { + where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) + } + if d.Subject != nil { + where = append(where, query.Equal("subject", *d.Subject)) + } + if d.HasError != nil { + if *d.HasError { + where = append(where, "notEmpty(validation_error) = 1") + } else { + where = append(where, "empty(validation_error) = 1") + } + } + query.Where(where...) + + query.Desc().OrderBy("time") + query.Limit(d.Limit) + + sql, args := query.Build() + return sql, args +} + +type queryCountEvents struct { + Database string + Namespace string + From time.Time +} + +func (d queryCountEvents) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") + query.From(tableName) + + query.Where(query.Equal("namespace", d.Namespace)) + query.Where(query.GreaterEqualThan("time", d.From.Unix())) + query.GroupBy("subject", "is_error") + + sql, args := query.Build() + return sql, args +} + +// Insert Events Query +type InsertEventsQuery struct { + Database string + Events []CHEvent + QuerySettings map[string]string +} + +func (q InsertEventsQuery) ToSQL() (string, []interface{}) { + tableName := GetEventsTableName(q.Database) + + query := sqlbuilder.ClickHouse.NewInsertBuilder() + query.InsertInto(tableName) + query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") + + // Add settings + var settings []string + for key, value := range q.QuerySettings { + settings = append(settings, fmt.Sprintf("%s = %s", key, value)) + } + + if len(settings) > 0 { + query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) + } + + for _, event := range q.Events { + query.Values( + event.Namespace, + event.ValidationError, + event.ID, + event.Type, + event.Source, + event.Subject, + event.Time, + event.Data, + event.IngestedAt, + event.StoredAt, + ) + } + + sql, args := query.Build() + return sql, args +} diff --git a/openmeter/streaming/clickhouse_connector/event_query_test.go b/openmeter/streaming/clickhouse_connector/event_query_test.go new file mode 100644 index 000000000..ec4e70e3c --- /dev/null +++ b/openmeter/streaming/clickhouse_connector/event_query_test.go @@ -0,0 +1,185 @@ +package clickhouse_connector + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestCreateEventsTable(t *testing.T) { + tests := []struct { + data createEventsTable + want string + }{ + { + data: createEventsTable{ + Database: "openmeter", + }, + want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + got := tt.data.toSQL() + assert.Equal(t, tt.want, got) + }) + } +} + +func TestQueryEventsTable(t *testing.T) { + subjectFilter := "customer-1" + idFilter := "event-id-1" + hasErrorTrue := true + hasErrorFalse := false + + tests := []struct { + query queryEventsTable + wantSQL string + wantArgs []interface{} + }{ + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + Subject: &subjectFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", subjectFilter}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + ID: &idFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorTrue, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorFalse, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestQueryEventsCount(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + tests := []struct { + query queryCountEvents + wantSQL string + wantArgs []interface{} + }{ + { + query: queryCountEvents{ + Database: "openmeter", + Namespace: "my_namespace", + From: from, + }, + wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", + wantArgs: []interface{}{"my_namespace", from.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestInsertEventsQuery(t *testing.T) { + now := time.Now() + + query := InsertEventsQuery{ + Database: "database", + Events: []CHEvent{ + { + Namespace: "my_namespace", + ID: "1", + Source: "source", + Subject: "subject-1", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ID: "2", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ValidationError: "event data value cannot be parsed as float64: not a number", + ID: "3", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, + }, + }, + } + + sql, args := query.ToSQL() + + assert.Equal(t, []interface{}{ + "my_namespace", "", "1", "api-calls", "source", "subject-1", now, `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "", "2", "api-calls", "source", "subject-2", now, `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now, `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, + }, args) + assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) +} diff --git a/openmeter/streaming/clickhouse_connector/meter_event_query.go b/openmeter/streaming/clickhouse_connector/meter_event_query.go new file mode 100644 index 000000000..5b9eee318 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector/meter_event_query.go @@ -0,0 +1,131 @@ +package clickhouse_connector + +import ( + _ "embed" + "fmt" + "strings" + "time" + + "github.com/huandu/go-sqlbuilder" +) + +const ( + MeterEventTableName = "meter_events" +) + +// Meter Event represents a single meter event in ClickHouse +type CHMeterEvent struct { + // Identifiers + Namespace string `ch:"namespace"` + Time time.Time `ch:"time"` + Meter string `ch:"meter"` + Subject string `ch:"subject"` + + // Usage + Value float64 `ch:"value"` + GroupBy map[string]string `ch:"group_by"` + + // Metadata + EventID string `ch:"event_id"` + EventSource string `ch:"event_source"` + EventType string `ch:"event_type"` + IngestedAt time.Time `ch:"ingested_at"` + StoredAt time.Time `ch:"stored_at"` +} + +// Create Meter Event Table +type createMeterEventTable struct { + Database string +} + +func (d createMeterEventTable) toSQL() string { + tableName := GetMeterEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() + sb.CreateTable(tableName) + sb.IfNotExists() + + // Identifiers + sb.Define("namespace", "String") + sb.Define("time", "DateTime") + sb.Define("meter", "LowCardinality(String)") + sb.Define("subject", "String") + + // Usage + sb.Define("value", "Decimal(14, 4)") + sb.Define("group_by", "Map(String, String)") + + // Metadata + sb.Define("event_id", "String") + sb.Define("event_type", "LowCardinality(String)") + sb.Define("event_source", "String") + sb.Define("ingested_at", "DateTime") + sb.Define("stored_at", "DateTime") + sb.SQL("ENGINE = MergeTree") + sb.SQL("PARTITION BY toYYYYMM(time)") + sb.SQL("ORDER BY (namespace, time, meter, subject)") + + sql, _ := sb.Build() + return sql +} + +// Insert Meter Events Query +type InsertMeterEventsQuery struct { + Database string + MeterEvents []CHMeterEvent + QuerySettings map[string]string +} + +func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { + tableName := GetMeterEventsTableName(q.Database) + + query := sqlbuilder.ClickHouse.NewInsertBuilder() + query.InsertInto(tableName) + query.Cols( + "namespace", + "time", + "meter", + "subject", + "value", + "group_by", + "event_id", + "event_source", + "event_type", + "ingested_at", + "stored_at", + ) + + // Add settings + var settings []string + for key, value := range q.QuerySettings { + settings = append(settings, fmt.Sprintf("%s = %s", key, value)) + } + + if len(settings) > 0 { + query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) + } + + for _, meterEvent := range q.MeterEvents { + query.Values( + meterEvent.Namespace, + meterEvent.Time, + meterEvent.Meter, + meterEvent.Subject, + meterEvent.Value, + meterEvent.GroupBy, + meterEvent.EventID, + meterEvent.EventSource, + meterEvent.EventType, + meterEvent.IngestedAt, + meterEvent.StoredAt, + ) + } + + sql, args := query.Build() + return sql, args +} + +// Get Meter Events Table Name +func GetMeterEventsTableName(database string) string { + return fmt.Sprintf("%s.%s%s", sqlbuilder.Escape(database), tablePrefix, MeterEventTableName) +} diff --git a/openmeter/streaming/clickhouse_connector/query.go b/openmeter/streaming/clickhouse_connector/query.go index 2dbc4f8ee..29cc7ddce 100644 --- a/openmeter/streaming/clickhouse_connector/query.go +++ b/openmeter/streaming/clickhouse_connector/query.go @@ -18,112 +18,6 @@ type column struct { Type string } -// Create Events Table -type createEventsTable struct { - Database string -} - -func (d createEventsTable) toSQL() string { - tableName := GetEventsTableName(d.Database) - - sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() - sb.CreateTable(tableName) - sb.IfNotExists() - sb.Define("namespace", "String") - sb.Define("validation_error", "String") - sb.Define("id", "String") - sb.Define("type", "LowCardinality(String)") - sb.Define("subject", "String") - sb.Define("source", "String") - sb.Define("time", "DateTime") - sb.Define("data", "String") - sb.Define("ingested_at", "DateTime") - sb.Define("stored_at", "DateTime") - sb.SQL("ENGINE = MergeTree") - sb.SQL("PARTITION BY toYYYYMM(time)") - sb.SQL("ORDER BY (namespace, time, type, subject)") - - sql, _ := sb.Build() - return sql -} - -type queryEventsTable struct { - Database string - Namespace string - From *time.Time - To *time.Time - IngestedAtFrom *time.Time - IngestedAtTo *time.Time - ID *string - Subject *string - HasError *bool - Limit int -} - -func (d queryEventsTable) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - where := []string{} - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") - query.From(tableName) - - where = append(where, query.Equal("namespace", d.Namespace)) - if d.From != nil { - where = append(where, query.GreaterEqualThan("time", d.From.Unix())) - } - if d.To != nil { - where = append(where, query.LessEqualThan("time", d.To.Unix())) - } - if d.IngestedAtFrom != nil { - where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) - } - if d.IngestedAtTo != nil { - where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) - } - if d.ID != nil { - where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) - } - if d.Subject != nil { - where = append(where, query.Equal("subject", *d.Subject)) - } - if d.HasError != nil { - if *d.HasError { - where = append(where, "notEmpty(validation_error) = 1") - } else { - where = append(where, "empty(validation_error) = 1") - } - } - query.Where(where...) - - query.Desc().OrderBy("time") - query.Limit(d.Limit) - - sql, args := query.Build() - return sql, args -} - -type queryCountEvents struct { - Database string - Namespace string - From time.Time -} - -func (d queryCountEvents) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") - query.From(tableName) - - query.Where(query.Equal("namespace", d.Namespace)) - query.Where(query.GreaterEqualThan("time", d.From.Unix())) - query.GroupBy("subject", "is_error") - - sql, args := query.Build() - return sql, args -} - type createMeterView struct { Database string Aggregation models.MeterAggregation diff --git a/openmeter/streaming/clickhouse_connector/query_test.go b/openmeter/streaming/clickhouse_connector/query_test.go index b9cb17743..d97a4809f 100644 --- a/openmeter/streaming/clickhouse_connector/query_test.go +++ b/openmeter/streaming/clickhouse_connector/query_test.go @@ -9,130 +9,6 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) -func TestCreateEventsTable(t *testing.T) { - tests := []struct { - data createEventsTable - want string - }{ - { - data: createEventsTable{ - Database: "openmeter", - }, - want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - got := tt.data.toSQL() - assert.Equal(t, tt.want, got) - }) - } -} - -func TestQueryEventsTable(t *testing.T) { - subjectFilter := "customer-1" - idFilter := "event-id-1" - hasErrorTrue := true - hasErrorFalse := false - - tests := []struct { - query queryEventsTable - wantSQL string - wantArgs []interface{} - }{ - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - Subject: &subjectFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", subjectFilter}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - ID: &idFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorTrue, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorFalse, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - -func TestQueryEventsCount(t *testing.T) { - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - tests := []struct { - query queryCountEvents - wantSQL string - wantArgs []interface{} - }{ - { - query: queryCountEvents{ - Database: "openmeter", - Namespace: "my_namespace", - From: from, - }, - wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", - wantArgs: []interface{}{"my_namespace", from.Unix()}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - func TestCreateMeterView(t *testing.T) { tests := []struct { query createMeterView From 9bcf353b47f2a1ea4a4669095baa544ee22c6a9e Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sat, 19 Oct 2024 09:50:31 -0700 Subject: [PATCH 02/33] fix(storage): meter id --- openmeter/sink/storage.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index 62aa9768d..f2af44f8e 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -94,10 +94,16 @@ func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmode // Meter events per meter for _, meterEvent := range message.MeterEvents { + meterID := meterEvent.Meter.ID + + if meterID == "" { + meterID = meterEvent.Meter.Slug + } + meterEvent := clickhouse_connector.CHMeterEvent{ Namespace: message.Namespace, Time: rawEvent.Time, - Meter: meterEvent.Meter.ID, + Meter: meterID, Subject: rawEvent.Subject, Value: meterEvent.Value, GroupBy: meterEvent.GroupBy, From 4a83e6dcad10a720451925d31c4d1bf2b66d0746 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sat, 19 Oct 2024 10:09:53 -0700 Subject: [PATCH 03/33] feat(query): meter --- .../clickhouse_connector/connector.go | 141 ++--------- .../clickhouse_connector/event_query.go | 6 + .../streaming/clickhouse_connector/query.go | 226 +++--------------- .../clickhouse_connector/query_test.go | 200 ++-------------- 4 files changed, 76 insertions(+), 497 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index dec57e9d2..014105c81 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -15,12 +15,10 @@ import ( "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/pkg/models" + "github.com/shopspring/decimal" ) -var ( - tablePrefix = "om_" - EventsTableName = "events" -) +var tablePrefix = "om_" // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. type ClickhouseConnector struct { @@ -62,35 +60,12 @@ func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, } func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error { - if namespace == "" { - return fmt.Errorf("namespace is required") - } - - err := c.createMeterView(ctx, namespace, meter) - if err != nil { - return fmt.Errorf("init: %w", err) - } - + // Do nothing return nil } func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meterSlug string) error { - if namespace == "" { - return fmt.Errorf("namespace is required") - } - if meterSlug == "" { - return fmt.Errorf("slug is required") - } - - err := c.deleteMeterView(ctx, namespace, meterSlug) - if err != nil { - if _, ok := err.(*models.MeterNotFoundError); ok { - return err - } - - return fmt.Errorf("delete meter view: %w", err) - } - + // Do nothing return nil } @@ -99,7 +74,7 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return nil, fmt.Errorf("namespace is required") } - values, err := c.queryMeterView(ctx, namespace, meterSlug, params) + values, err := c.queryMeter(ctx, namespace, meterSlug, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { return nil, err @@ -160,34 +135,7 @@ func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace str } func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { - err := c.deleteNamespace(ctx, namespace) - if err != nil { - return fmt.Errorf("delete namespace in clickhouse: %w", err) - } - return nil -} - -// DeleteNamespace deletes the namespace related resources from Clickhouse -// We don't delete the events table as it it reused between namespaces -// We only delete the materialized views for the meters -func (c *ClickhouseConnector) deleteNamespace(ctx context.Context, namespace string) error { - // Retrieve meters belonging to the namespace - meters, err := c.config.Meters.ListMeters(ctx, namespace) - if err != nil { - return fmt.Errorf("failed to list meters: %w", err) - } - - for _, meter := range meters { - err := c.deleteMeterView(ctx, namespace, meter.Slug) - if err != nil { - // If the meter view does not exist, we ignore the error - if _, ok := err.(*models.MeterNotFoundError); ok { - return nil - } - return fmt.Errorf("delete meter view: %w", err) - } - } - + // We don't delete the event tables as it it reused between namespaces return nil } @@ -344,61 +292,8 @@ func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace st return results, nil } -func (c *ClickhouseConnector) createMeterView(ctx context.Context, namespace string, meter *models.Meter) error { - // CreateOrReplace is used to force the recreation of the materialized view - // This is not safe to use in production as it will drop the existing views - if c.config.CreateOrReplaceMeter { - err := c.deleteMeterView(ctx, namespace, meter.Slug) - if err != nil { - return fmt.Errorf("drop meter view: %w", err) - } - } - - view := createMeterView{ - Populate: c.config.PopulateMeter, - Database: c.config.Database, - Namespace: namespace, - MeterSlug: meter.Slug, - Aggregation: meter.Aggregation, - EventType: meter.EventType, - ValueProperty: meter.ValueProperty, - GroupBy: meter.GroupBy, - } - sql, args, err := view.toSQL() - if err != nil { - return fmt.Errorf("create meter view: %w", err) - } - err = c.config.ClickHouse.Exec(ctx, sql, args...) - if err != nil { - return fmt.Errorf("create meter view: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) deleteMeterView(ctx context.Context, namespace string, meterSlug string) error { - query := deleteMeterView{ - Database: c.config.Database, - Namespace: namespace, - MeterSlug: meterSlug, - } - - sql := query.toSQL() - - err := c.config.ClickHouse.Exec(ctx, sql) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return &models.MeterNotFoundError{MeterSlug: meterSlug} - } - - return fmt.Errorf("delete meter view: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { - queryMeter := queryMeterView{ +func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { + queryMeter := queryMeter{ Database: c.config.Database, Namespace: namespace, MeterSlug: meterSlug, @@ -432,11 +327,12 @@ func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace stri slog.Debug("query meter view", "elapsed", elapsed.String(), "sql", sql, "args", args) for rows.Next() { - value := models.MeterQueryRow{ + row := models.MeterQueryRow{ GroupBy: map[string]*string{}, } - args := []interface{}{&value.WindowStart, &value.WindowEnd, &value.Value} + var value decimal.Decimal + args := []interface{}{&row.WindowStart, &row.WindowEnd, &value} argCount := len(args) for range queryMeter.GroupBy { @@ -448,28 +344,31 @@ func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace stri return values, fmt.Errorf("query meter view row scan: %w", err) } + // TODO: should we use decima all the way? + row.Value, _ = value.Float64() + for i, key := range queryMeter.GroupBy { if s, ok := args[i+argCount].(*string); ok { if key == "subject" { - value.Subject = s + row.Subject = s continue } // We treat empty string as nil if s != nil && *s == "" { - value.GroupBy[key] = nil + row.GroupBy[key] = nil } else { - value.GroupBy[key] = s + row.GroupBy[key] = s } } } // an empty row is returned when there are no values for the meter - if value.WindowStart.IsZero() && value.WindowEnd.IsZero() && value.Value == 0 { + if row.WindowStart.IsZero() && row.WindowEnd.IsZero() && row.Value == 0 { continue } - values = append(values, value) + values = append(values, row) } rows.Close() err = rows.Err() @@ -481,7 +380,7 @@ func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace stri } func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { - query := listMeterViewSubjects{ + query := listMeterSubjectsQuery{ Database: c.config.Database, Namespace: namespace, MeterSlug: meterSlug, diff --git a/openmeter/streaming/clickhouse_connector/event_query.go b/openmeter/streaming/clickhouse_connector/event_query.go index 126deddc6..0da046ce6 100644 --- a/openmeter/streaming/clickhouse_connector/event_query.go +++ b/openmeter/streaming/clickhouse_connector/event_query.go @@ -9,6 +9,8 @@ import ( "github.com/huandu/go-sqlbuilder" ) +const EventsTableName = "events" + // CHEvent represents a raw event in ClickHouse type CHEvent struct { Namespace string `ch:"namespace"` @@ -172,3 +174,7 @@ func (q InsertEventsQuery) ToSQL() (string, []interface{}) { sql, args := query.Build() return sql, args } + +func GetEventsTableName(database string) string { + return fmt.Sprintf("%s.%s%s", sqlbuilder.Escape(database), tablePrefix, EventsTableName) +} diff --git a/openmeter/streaming/clickhouse_connector/query.go b/openmeter/streaming/clickhouse_connector/query.go index 29cc7ddce..0e7368c69 100644 --- a/openmeter/streaming/clickhouse_connector/query.go +++ b/openmeter/streaming/clickhouse_connector/query.go @@ -4,7 +4,6 @@ import ( _ "embed" "fmt" "sort" - "strings" "time" "github.com/huandu/go-sqlbuilder" @@ -18,159 +17,7 @@ type column struct { Type string } -type createMeterView struct { - Database string - Aggregation models.MeterAggregation - Namespace string - MeterSlug string - EventType string - ValueProperty string - GroupBy map[string]string - // Populate creates the materialized view with data from the events table - // This is not safe to use in production as requires to stop ingestion - Populate bool -} - -func (d createMeterView) toSQL() (string, []interface{}, error) { - viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) - columns := []column{ - {Name: "subject", Type: "String"}, - {Name: "windowstart", Type: "DateTime"}, - {Name: "windowend", Type: "DateTime"}, - } - - // Value - agg := "" - - switch d.Aggregation { - case models.MeterAggregationSum: - agg = "sum" - case models.MeterAggregationAvg: - agg = "avg" - case models.MeterAggregationMin: - agg = "min" - case models.MeterAggregationMax: - agg = "max" - case models.MeterAggregationCount: - agg = "count" - case models.MeterAggregationUniqueCount: - agg = "uniq" - default: - return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) - } - - if d.Aggregation == models.MeterAggregationUniqueCount { - columns = append(columns, column{Name: "value", Type: fmt.Sprintf("AggregateFunction(%s, String)", agg)}) - } else { - columns = append(columns, column{Name: "value", Type: fmt.Sprintf("AggregateFunction(%s, Float64)", agg)}) - } - - // Group by - orderBy := []string{"windowstart", "windowend", "subject"} - sortedGroupBy := sortedKeys(d.GroupBy) - for _, k := range sortedGroupBy { - columnName := sqlbuilder.Escape(k) - orderBy = append(orderBy, sqlbuilder.Escape(columnName)) - columns = append(columns, column{Name: columnName, Type: "String"}) - } - - sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() - sb.CreateTable(viewName) - sb.IfNotExists() - for _, column := range columns { - sb.Define(column.Name, column.Type) - } - sb.SQL("ENGINE = AggregatingMergeTree()") - sb.SQL(fmt.Sprintf("ORDER BY (%s)", strings.Join(orderBy, ", "))) - if d.Populate { - sb.SQL("POPULATE") - } - sb.SQL("AS") - - selectQuery, err := d.toSelectSQL() - if err != nil { - return "", nil, err - } - - sb.SQL(selectQuery) - sql, args := sb.Build() - - // TODO: can we do it differently? - sql = strings.Replace(sql, "CREATE TABLE", "CREATE MATERIALIZED VIEW", 1) - - return sql, args, nil -} - -func (d createMeterView) toSelectSQL() (string, error) { - eventsTableName := GetEventsTableName(d.Database) - - aggStateFn := "" - switch d.Aggregation { - case models.MeterAggregationSum: - aggStateFn = "sumState" - case models.MeterAggregationAvg: - aggStateFn = "avgState" - case models.MeterAggregationMin: - aggStateFn = "minState" - case models.MeterAggregationMax: - aggStateFn = "maxState" - case models.MeterAggregationUniqueCount: - aggStateFn = "uniqState" - case models.MeterAggregationCount: - aggStateFn = "countState" - default: - return "", fmt.Errorf("invalid aggregation type: %s", d.Aggregation) - } - - // Selects - selects := []string{ - "subject", - "tumbleStart(time, toIntervalMinute(1)) AS windowstart", - "tumbleEnd(time, toIntervalMinute(1)) AS windowend", - } - if d.ValueProperty == "" && d.Aggregation == models.MeterAggregationCount { - selects = append(selects, fmt.Sprintf("%s(*) AS value", aggStateFn)) - } else if d.Aggregation == models.MeterAggregationUniqueCount { - selects = append(selects, fmt.Sprintf("%s(JSON_VALUE(data, '%s')) AS value", aggStateFn, sqlbuilder.Escape(d.ValueProperty))) - } else { - selects = append(selects, fmt.Sprintf("%s(cast(JSON_VALUE(data, '%s'), 'Float64')) AS value", aggStateFn, sqlbuilder.Escape(d.ValueProperty))) - } - - // Group by - orderBy := []string{"windowstart", "windowend", "subject"} - sortedGroupBy := sortedKeys(d.GroupBy) - for _, k := range sortedGroupBy { - v := d.GroupBy[k] - columnName := sqlbuilder.Escape(k) - orderBy = append(orderBy, sqlbuilder.Escape(columnName)) - selects = append(selects, fmt.Sprintf("JSON_VALUE(data, '%s') as %s", sqlbuilder.Escape(v), sqlbuilder.Escape(k))) - } - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select(selects...) - query.From(eventsTableName) - // We use absolute paths to avoid shadowing in the case the materialized view have a `namespace` or `type` group by - query.Where(fmt.Sprintf("%s.namespace = '%s'", eventsTableName, sqlbuilder.Escape(d.Namespace))) - query.Where(fmt.Sprintf("empty(%s.validation_error) = 1", eventsTableName)) - query.Where(fmt.Sprintf("%s.type = '%s'", eventsTableName, sqlbuilder.Escape(d.EventType))) - query.GroupBy(orderBy...) - - return query.String(), nil -} - -type deleteMeterView struct { - Database string - Namespace string - MeterSlug string -} - -func (d deleteMeterView) toSQL() string { - viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) - - return fmt.Sprintf("DROP VIEW %s", viewName) -} - -type queryMeterView struct { +type queryMeter struct { Database string Namespace string MeterSlug string @@ -184,9 +31,9 @@ type queryMeterView struct { WindowTimeZone *time.Location } -func (d queryMeterView) toSQL() (string, []interface{}, error) { +func (d queryMeter) toSQL() (string, []interface{}, error) { viewAlias := "meter" - viewName := fmt.Sprintf("%s %s", GetMeterViewName(d.Database, d.Namespace, d.MeterSlug), viewAlias) + tableName := fmt.Sprintf("%s %s", GetMeterEventsTableName(d.Database), viewAlias) getColumn := columnFactory(viewAlias) var selectColumns, groupByColumns, where []string @@ -203,22 +50,22 @@ func (d queryMeterView) toSQL() (string, []interface{}, error) { case models.WindowSizeMinute: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(windowstart, toIntervalMinute(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(windowstart, toIntervalMinute(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(time, toIntervalMinute(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(time, toIntervalMinute(1), '%s') AS windowend", tz), ) case models.WindowSizeHour: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(windowstart, toIntervalHour(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(windowstart, toIntervalHour(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(time, toIntervalHour(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(time, toIntervalHour(1), '%s') AS windowend", tz), ) case models.WindowSizeDay: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(windowstart, toIntervalDay(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(windowstart, toIntervalDay(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(time, toIntervalDay(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(time, toIntervalDay(1), '%s') AS windowend", tz), ) default: @@ -232,30 +79,34 @@ func (d queryMeterView) toSQL() (string, []interface{}, error) { switch d.Aggregation { case models.MeterAggregationSum: - selectColumns = append(selectColumns, "sumMerge(value) AS value") + selectColumns = append(selectColumns, "sum(value) AS value") case models.MeterAggregationAvg: - selectColumns = append(selectColumns, "avgMerge(value) AS value") + selectColumns = append(selectColumns, "avg(value) AS value") case models.MeterAggregationMin: - selectColumns = append(selectColumns, "minMerge(value) AS value") + selectColumns = append(selectColumns, "min(value) AS value") case models.MeterAggregationMax: - selectColumns = append(selectColumns, "maxMerge(value) AS value") + selectColumns = append(selectColumns, "max(value) AS value") case models.MeterAggregationUniqueCount: - selectColumns = append(selectColumns, "toFloat64(uniqMerge(value)) AS value") + // FIXME: value is a number, not a string + selectColumns = append(selectColumns, "toFloat64(uniq(value)) AS value") case models.MeterAggregationCount: - selectColumns = append(selectColumns, "toFloat64(countMerge(value)) AS value") + selectColumns = append(selectColumns, "sum(value) AS value") default: return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) } for _, column := range d.GroupBy { c := sqlbuilder.Escape(column) - selectColumns = append(selectColumns, c) + selectColumn := fmt.Sprintf("group_by['%s'] as %s", c, c) + selectColumns = append(selectColumns, selectColumn) groupByColumns = append(groupByColumns, c) } queryView := sqlbuilder.ClickHouse.NewSelectBuilder() queryView.Select(selectColumns...) - queryView.From(viewName) + queryView.From(tableName) + queryView.Where(queryView.Equal("namespace", d.Namespace)) + queryView.Where(queryView.Equal("meter", d.MeterSlug)) if len(d.Subject) > 0 { mapFunc := func(subject string) string { @@ -287,11 +138,11 @@ func (d queryMeterView) toSQL() (string, []interface{}, error) { } if d.From != nil { - where = append(where, queryView.GreaterEqualThan(getColumn("windowstart"), d.From.Unix())) + where = append(where, queryView.GreaterEqualThan(getColumn("time"), d.From.Unix())) } if d.To != nil { - where = append(where, queryView.LessEqualThan(getColumn("windowend"), d.To.Unix())) + where = append(where, queryView.LessEqualThan(getColumn("time"), d.To.Unix())) } if len(where) > 0 { @@ -319,7 +170,7 @@ func sortedKeys(m map[string]string) []string { return keys } -type listMeterViewSubjects struct { +type listMeterSubjectsQuery struct { Database string Namespace string MeterSlug string @@ -327,41 +178,28 @@ type listMeterViewSubjects struct { To *time.Time } -func (d listMeterViewSubjects) toSQL() (string, []interface{}) { - viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) +func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { + tableName := GetMeterEventsTableName(d.Database) - var where []string sb := sqlbuilder.ClickHouse.NewSelectBuilder() sb.Select("DISTINCT subject") - sb.From(viewName) + sb.Where(sb.Equal("namespace", d.Namespace)) + sb.Where(sb.Equal("meter", d.MeterSlug)) + sb.From(tableName) + sb.OrderBy("subject") if d.From != nil { - where = append(where, sb.GreaterEqualThan("windowstart", d.From.Unix())) + sb.Where(sb.GreaterEqualThan("time", d.From.Unix())) } if d.To != nil { - where = append(where, sb.LessEqualThan("windowend", d.To.Unix())) + sb.Where(sb.LessEqualThan("time", d.To.Unix())) } - if len(where) > 0 { - sb.Where(where...) - } - - sb.OrderBy("subject") - sql, args := sb.Build() return sql, args } -func GetEventsTableName(database string) string { - return fmt.Sprintf("%s.%s%s", sqlbuilder.Escape(database), tablePrefix, EventsTableName) -} - -func GetMeterViewName(database string, namespace string, meterSlug string) string { - meterViewName := fmt.Sprintf("%s%s_%s", tablePrefix, namespace, meterSlug) - return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), sqlbuilder.Escape(meterViewName)) -} - func columnFactory(alias string) func(string) string { return func(column string) string { return fmt.Sprintf("%s.%s", alias, column) diff --git a/openmeter/streaming/clickhouse_connector/query_test.go b/openmeter/streaming/clickhouse_connector/query_test.go index d97a4809f..16016d03a 100644 --- a/openmeter/streaming/clickhouse_connector/query_test.go +++ b/openmeter/streaming/clickhouse_connector/query_test.go @@ -9,120 +9,6 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) -func TestCreateMeterView(t *testing.T) { - tests := []struct { - query createMeterView - wantSQL string - wantArgs []interface{} - }{ - { - query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - EventType: "myevent", - ValueProperty: "$.duration_ms", - GroupBy: map[string]string{"group1": "$.group1", "group2": "$.group2"}, - }, - wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(sum, Float64), group1 String, group2 String) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject, group1, group2) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, sumState(cast(JSON_VALUE(data, '$.duration_ms'), 'Float64')) AS value, JSON_VALUE(data, '$.group1') as group1, JSON_VALUE(data, '$.group2') as group2 FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject, group1, group2", - wantArgs: nil, - }, - { - query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationAvg, - EventType: "myevent", - ValueProperty: "$.token_count", - GroupBy: map[string]string{}, - }, - wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(avg, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, avgState(cast(JSON_VALUE(data, '$.token_count'), 'Float64')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", - wantArgs: nil, - }, - { - query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, - EventType: "myevent", - ValueProperty: "", - GroupBy: map[string]string{}, - }, - wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", - wantArgs: nil, - }, - { - query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, - EventType: "myevent", - ValueProperty: "", - GroupBy: map[string]string{}, - }, - wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", - wantArgs: nil, - }, - { - query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationUniqueCount, - EventType: "myevent", - ValueProperty: "$.trace_id", - GroupBy: map[string]string{}, - }, - wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(uniq, String)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, uniqState(JSON_VALUE(data, '$.trace_id')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", - wantArgs: nil, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs, err := tt.query.toSQL() - if err != nil { - t.Error(err) - return - } - - assert.Equal(t, tt.wantSQL, gotSql) - assert.Equal(t, tt.wantArgs, gotArgs) - }) - } -} - -func TestDeleteMeterView(t *testing.T) { - tests := []struct { - data deleteMeterView - wantSQL string - wantArgs []interface{} - }{ - { - data: deleteMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - }, - wantSQL: "DROP VIEW openmeter.om_my_namespace_meter1", - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql := tt.data.toSQL() - - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - func TestQueryMeterView(t *testing.T) { subject := "subject1" from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") @@ -131,12 +17,12 @@ func TestQueryMeterView(t *testing.T) { windowSize := models.WindowSizeHour tests := []struct { - query queryMeterView + query queryMeter wantSQL string wantArgs []interface{} }{ { - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -151,7 +37,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"subject1", from.Unix(), to.Unix()}, }, { // Aggregate all available data - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -161,7 +47,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: nil, }, { // Aggregate with count aggregation - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -171,7 +57,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: nil, }, { // Aggregate data from start - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -182,7 +68,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{from.Unix()}, }, { // Aggregate data between period - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -194,7 +80,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -207,7 +93,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data between period in a different timezone, groupped by window size - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -221,7 +107,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data for a single subject - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -233,7 +119,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"subject1"}, }, { // Aggregate data for a single subject and group by additional fields - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -245,7 +131,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"subject1"}, }, { // Aggregate data for a multiple subjects - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -257,7 +143,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -268,7 +154,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"g1v1"}, }, { // Aggregate data with filtering for a single group and multiple values - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -279,7 +165,7 @@ func TestQueryMeterView(t *testing.T) { wantArgs: []interface{}{"g1v1", "g1v2"}, }, { // Aggregate data with filtering for multiple groups and multiple values - query: queryMeterView{ + query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -311,12 +197,12 @@ func TestListMeterViewSubjects(t *testing.T) { to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") tests := []struct { - query listMeterViewSubjects + query listMeterSubjectsQuery wantSQL string wantArgs []interface{} }{ { - query: listMeterViewSubjects{ + query: listMeterSubjectsQuery{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -325,7 +211,7 @@ func TestListMeterViewSubjects(t *testing.T) { wantArgs: nil, }, { - query: listMeterViewSubjects{ + query: listMeterSubjectsQuery{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -335,7 +221,7 @@ func TestListMeterViewSubjects(t *testing.T) { wantArgs: []interface{}{from.Unix()}, }, { - query: listMeterViewSubjects{ + query: listMeterSubjectsQuery{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", @@ -357,53 +243,3 @@ func TestListMeterViewSubjects(t *testing.T) { }) } } - -func TestQueryEvents(t *testing.T) { - fromTime, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00Z") - toTime, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") - - tests := []struct { - query queryEventsTable - wantSQL string - wantArgs []interface{} - }{ - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - From: &fromTime, - To: &toTime, - Limit: 10, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND time >= ? AND time <= ? ORDER BY time DESC LIMIT 10", - wantArgs: []interface{}{"my_namespace", fromTime.Unix(), toTime.Unix()}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - From: &fromTime, - Limit: 10, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND time >= ? ORDER BY time DESC LIMIT 10", - wantArgs: []interface{}{"my_namespace", fromTime.Unix()}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - To: &toTime, - Limit: 10, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND time <= ? ORDER BY time DESC LIMIT 10", - wantArgs: []interface{}{"my_namespace", toTime.Unix()}, - }, - } - - for _, tt := range tests { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantSQL, gotSql) - assert.Equal(t, tt.wantArgs, gotArgs) - } -} From eea1d465639acddee03b31a339bffab2d62ef83b Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sat, 19 Oct 2024 11:02:54 -0700 Subject: [PATCH 04/33] feat(meter): query --- openmeter/meter/parse.go | 10 +- .../clickhouse_connector/connector.go | 2 - .../clickhouse_connector/event_query.go | 4 +- .../clickhouse_connector/meter_event_query.go | 4 +- .../{query.go => meter_query.go} | 93 ++++--- .../clickhouse_connector/meter_query_test.go | 245 ++++++++++++++++++ .../clickhouse_connector/query_test.go | 245 ------------------ 7 files changed, 307 insertions(+), 296 deletions(-) rename openmeter/streaming/clickhouse_connector/{query.go => meter_query.go} (54%) create mode 100644 openmeter/streaming/clickhouse_connector/meter_query_test.go delete mode 100644 openmeter/streaming/clickhouse_connector/query_test.go diff --git a/openmeter/meter/parse.go b/openmeter/meter/parse.go index 29d5a6ba4..735dbaecc 100644 --- a/openmeter/meter/parse.go +++ b/openmeter/meter/parse.go @@ -43,11 +43,11 @@ func ParseEvent(meter Meter, ev event.Event) (float64, map[string]string, error) switch meter.Aggregation { // UNIQUE_COUNT aggregation requires string property value case MeterAggregationUniqueCount: - switch value := rawValue.(type) { - case string: - return 1, groupBy, nil - case float64: - return value, groupBy, nil + switch rawValue.(type) { + case string, float64: + // We return 0 because the meter is unique by this string value. + // The actual value is not used in the aggregation even if it's number. + return 0, groupBy, nil default: return 0, groupBy, errors.New("event data value property must be string for unique count aggregation") diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index 014105c81..ef7ca095a 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -18,8 +18,6 @@ import ( "github.com/shopspring/decimal" ) -var tablePrefix = "om_" - // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. type ClickhouseConnector struct { config ClickhouseConnectorConfig diff --git a/openmeter/streaming/clickhouse_connector/event_query.go b/openmeter/streaming/clickhouse_connector/event_query.go index 0da046ce6..7dcf9aa0f 100644 --- a/openmeter/streaming/clickhouse_connector/event_query.go +++ b/openmeter/streaming/clickhouse_connector/event_query.go @@ -9,7 +9,7 @@ import ( "github.com/huandu/go-sqlbuilder" ) -const EventsTableName = "events" +const EventsTableName = "om_events" // CHEvent represents a raw event in ClickHouse type CHEvent struct { @@ -176,5 +176,5 @@ func (q InsertEventsQuery) ToSQL() (string, []interface{}) { } func GetEventsTableName(database string) string { - return fmt.Sprintf("%s.%s%s", sqlbuilder.Escape(database), tablePrefix, EventsTableName) + return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) } diff --git a/openmeter/streaming/clickhouse_connector/meter_event_query.go b/openmeter/streaming/clickhouse_connector/meter_event_query.go index 5b9eee318..d9a0b321f 100644 --- a/openmeter/streaming/clickhouse_connector/meter_event_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_event_query.go @@ -10,7 +10,7 @@ import ( ) const ( - MeterEventTableName = "meter_events" + MeterEventTableName = "om_meter_events" ) // Meter Event represents a single meter event in ClickHouse @@ -127,5 +127,5 @@ func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { // Get Meter Events Table Name func GetMeterEventsTableName(database string) string { - return fmt.Sprintf("%s.%s%s", sqlbuilder.Escape(database), tablePrefix, MeterEventTableName) + return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), MeterEventTableName) } diff --git a/openmeter/streaming/clickhouse_connector/query.go b/openmeter/streaming/clickhouse_connector/meter_query.go similarity index 54% rename from openmeter/streaming/clickhouse_connector/query.go rename to openmeter/streaming/clickhouse_connector/meter_query.go index 0e7368c69..9d0b09efd 100644 --- a/openmeter/streaming/clickhouse_connector/query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -32,9 +32,9 @@ type queryMeter struct { } func (d queryMeter) toSQL() (string, []interface{}, error) { - viewAlias := "meter" - tableName := fmt.Sprintf("%s %s", GetMeterEventsTableName(d.Database), viewAlias) - getColumn := columnFactory(viewAlias) + tableName := GetMeterEventsTableName(d.Database) + getColumn := columnFactory(MeterEventTableName) + timeColumn := getColumn("time") var selectColumns, groupByColumns, where []string @@ -50,22 +50,22 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { case models.WindowSizeMinute: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(time, toIntervalMinute(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(time, toIntervalMinute(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(%s, toIntervalMinute(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalMinute(1), '%s') AS windowend", timeColumn, tz), ) case models.WindowSizeHour: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(time, toIntervalHour(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(time, toIntervalHour(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(%s, toIntervalHour(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalHour(1), '%s') AS windowend", timeColumn, tz), ) case models.WindowSizeDay: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(time, toIntervalDay(1), '%s') AS windowstart", tz), - fmt.Sprintf("tumbleEnd(time, toIntervalDay(1), '%s') AS windowend", tz), + fmt.Sprintf("tumbleStart(%s, toIntervalDay(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalDay(1), '%s') AS windowend", timeColumn, tz), ) default: @@ -79,83 +79,96 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { switch d.Aggregation { case models.MeterAggregationSum: - selectColumns = append(selectColumns, "sum(value) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) case models.MeterAggregationAvg: - selectColumns = append(selectColumns, "avg(value) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("avg(%s) AS value", getColumn("value"))) case models.MeterAggregationMin: - selectColumns = append(selectColumns, "min(value) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("min(%s) AS value", getColumn("value"))) case models.MeterAggregationMax: - selectColumns = append(selectColumns, "max(value) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("max(%s) AS value", getColumn("value"))) case models.MeterAggregationUniqueCount: // FIXME: value is a number, not a string - selectColumns = append(selectColumns, "toFloat64(uniq(value)) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("toFloat64(uniq(%s)) AS value", getColumn("value"))) case models.MeterAggregationCount: - selectColumns = append(selectColumns, "sum(value) AS value") + selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) default: return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) } - for _, column := range d.GroupBy { - c := sqlbuilder.Escape(column) - selectColumn := fmt.Sprintf("group_by['%s'] as %s", c, c) + for _, groupByKey := range d.GroupBy { + c := sqlbuilder.Escape(groupByKey) + selectColumn := fmt.Sprintf("%s['%s'] as %s", getColumn("group_by"), c, c) + + // Subject is a special case + if groupByKey == "subject" { + selectColumn = getColumn("subject") + } + selectColumns = append(selectColumns, selectColumn) groupByColumns = append(groupByColumns, c) } - queryView := sqlbuilder.ClickHouse.NewSelectBuilder() - queryView.Select(selectColumns...) - queryView.From(tableName) - queryView.Where(queryView.Equal("namespace", d.Namespace)) - queryView.Where(queryView.Equal("meter", d.MeterSlug)) + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select(selectColumns...) + query.From(tableName) + query.Where(query.Equal(getColumn("namespace"), d.Namespace)) + query.Where(query.Equal(getColumn("meter"), d.MeterSlug)) if len(d.Subject) > 0 { mapFunc := func(subject string) string { - return queryView.Equal(getColumn("subject"), subject) + return query.Equal(getColumn("subject"), subject) } - where = append(where, queryView.Or(slicesx.Map(d.Subject, mapFunc)...)) + where = append(where, query.Or(slicesx.Map(d.Subject, mapFunc)...)) } if len(d.FilterGroupBy) > 0 { - // We sort the columns to ensure the query is deterministic - columns := make([]string, 0, len(d.FilterGroupBy)) + // We sort the group by s to ensure the query is deterministic + groupByKeys := make([]string, 0, len(d.FilterGroupBy)) for k := range d.FilterGroupBy { - columns = append(columns, k) + groupByKeys = append(groupByKeys, k) } - sort.Strings(columns) + sort.Strings(groupByKeys) - for _, column := range columns { - values := d.FilterGroupBy[column] + for _, groupByKey := range groupByKeys { + values := d.FilterGroupBy[groupByKey] if len(values) == 0 { - return "", nil, fmt.Errorf("empty filter for group by: %s", column) + return "", nil, fmt.Errorf("empty filter for group by: %s", groupByKey) } mapFunc := func(value string) string { - return queryView.Equal(sqlbuilder.Escape(getColumn(column)), value) + column := sqlbuilder.Escape(fmt.Sprintf("%s['%s']", getColumn("group_by"), groupByKey)) + + // Subject is a special case + if groupByKey == "subject" { + column = fmt.Sprintf("subject") + } + + return query.Equal(column, value) } - where = append(where, queryView.Or(slicesx.Map(values, mapFunc)...)) + where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) } } if d.From != nil { - where = append(where, queryView.GreaterEqualThan(getColumn("time"), d.From.Unix())) + where = append(where, query.GreaterEqualThan(getColumn("time"), d.From.Unix())) } if d.To != nil { - where = append(where, queryView.LessEqualThan(getColumn("time"), d.To.Unix())) + where = append(where, query.LessEqualThan(getColumn("time"), d.To.Unix())) } if len(where) > 0 { - queryView.Where(where...) + query.Where(where...) } - queryView.GroupBy(groupByColumns...) + query.GroupBy(groupByColumns...) if groupByWindowSize { - queryView.OrderBy("windowstart") + query.OrderBy("windowstart") } - sql, args := queryView.Build() + sql, args := query.Build() return sql, args, nil } diff --git a/openmeter/streaming/clickhouse_connector/meter_query_test.go b/openmeter/streaming/clickhouse_connector/meter_query_test.go new file mode 100644 index 000000000..7d4cd7424 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector/meter_query_test.go @@ -0,0 +1,245 @@ +package clickhouse_connector + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/pkg/models" +) + +func TestQueryMeter(t *testing.T) { + subject := "subject1" + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + tz, _ := time.LoadLocation("Asia/Shanghai") + windowSize := models.WindowSizeHour + + tests := []struct { + query queryMeter + wantSQL string + wantArgs []interface{} + }{ + { + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + From: &from, + To: &to, + GroupBy: []string{"subject", "group1", "group2"}, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, + }, + { // Aggregate all available data + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate with count aggregation + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate data from start + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { // Aggregate data between period + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + To: &to, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + To: &to, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period in a different timezone, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + To: &to, + WindowSize: &windowSize, + WindowTimeZone: tz, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data for a single subject + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a single subject and group by additional fields + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + GroupBy: []string{"subject", "group1", "group2"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a multiple subjects + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject, "subject2"}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, + }, + { // Aggregate data with filtering for a single group and single value + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, + }, + { // Aggregate data with filtering for a single group and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, + }, + { // Aggregate data with filtering for multiple groups and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs, err := tt.query.toSQL() + if err != nil { + t.Error(err) + return + } + + assert.Equal(t, tt.wantSQL, gotSql) + assert.Equal(t, tt.wantArgs, gotArgs) + }) + } +} + +func TestListMeterSubjects(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + + tests := []struct { + query listMeterSubjectsQuery + wantSQL string + wantArgs []interface{} + }{ + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + From: &from, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + From: &from, + To: &to, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} diff --git a/openmeter/streaming/clickhouse_connector/query_test.go b/openmeter/streaming/clickhouse_connector/query_test.go deleted file mode 100644 index 16016d03a..000000000 --- a/openmeter/streaming/clickhouse_connector/query_test.go +++ /dev/null @@ -1,245 +0,0 @@ -package clickhouse_connector - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/pkg/models" -) - -func TestQueryMeterView(t *testing.T) { - subject := "subject1" - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") - tz, _ := time.LoadLocation("Asia/Shanghai") - windowSize := models.WindowSizeHour - - tests := []struct { - query queryMeter - wantSQL string - wantArgs []interface{} - }{ - { - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - From: &from, - To: &to, - GroupBy: []string{"subject", "group1", "group2"}, - WindowSize: &windowSize, - }, - wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'UTC') AS windowend, sumMerge(value) AS value, subject, group1, group2 FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) AND meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", - wantArgs: []interface{}{"subject1", from.Unix(), to.Unix()}, - }, - { // Aggregate all available data - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter", - wantArgs: nil, - }, - { // Aggregate with count aggregation - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, - }, - wantSQL: "SELECT min(windowstart), max(windowend), toFloat64(countMerge(value)) AS value FROM openmeter.om_my_namespace_meter1 meter", - wantArgs: nil, - }, - { // Aggregate data from start - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ?", - wantArgs: []interface{}{from.Unix()}, - }, - { // Aggregate data between period - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - To: &to, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ?", - wantArgs: []interface{}{from.Unix(), to.Unix()}, - }, - { // Aggregate data between period, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - To: &to, - WindowSize: &windowSize, - }, - wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'UTC') AS windowend, sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{from.Unix(), to.Unix()}, - }, - { // Aggregate data between period in a different timezone, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - To: &to, - WindowSize: &windowSize, - WindowTimeZone: tz, - }, - wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{from.Unix(), to.Unix()}, - }, - { // Aggregate data for a single subject - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - GroupBy: []string{"subject"}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"subject1"}, - }, - { // Aggregate data for a single subject and group by additional fields - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - GroupBy: []string{"subject", "group1", "group2"}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject, group1, group2 FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) GROUP BY subject, group1, group2", - wantArgs: []interface{}{"subject1"}, - }, - { // Aggregate data for a multiple subjects - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject, "subject2"}, - GroupBy: []string{"subject"}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ? OR meter.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"subject1", "subject2"}, - }, - { // Aggregate data with filtering for a single group and single value - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ?)", - wantArgs: []interface{}{"g1v1"}, - }, - { // Aggregate data with filtering for a single group and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ? OR meter.g1 = ?)", - wantArgs: []interface{}{"g1v1", "g1v2"}, - }, - { // Aggregate data with filtering for multiple groups and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, - }, - wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ? OR meter.g1 = ?) AND (meter.g2 = ? OR meter.g2 = ?)", - wantArgs: []interface{}{"g1v1", "g1v2", "g2v1", "g2v2"}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs, err := tt.query.toSQL() - if err != nil { - t.Error(err) - return - } - - assert.Equal(t, tt.wantSQL, gotSql) - assert.Equal(t, tt.wantArgs, gotArgs) - }) - } -} - -func TestListMeterViewSubjects(t *testing.T) { - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") - - tests := []struct { - query listMeterSubjectsQuery - wantSQL string - wantArgs []interface{} - }{ - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 ORDER BY subject", - wantArgs: nil, - }, - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - From: &from, - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 WHERE windowstart >= ? ORDER BY subject", - wantArgs: []interface{}{from.Unix()}, - }, - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - From: &from, - To: &to, - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 WHERE windowstart >= ? AND windowend <= ? ORDER BY subject", - wantArgs: []interface{}{from.Unix(), to.Unix()}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} From 59fbd59f7338541d3e0699daf154f0ca90091b05 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 09:26:23 -0700 Subject: [PATCH 05/33] feat(streaming): unique aggregation --- openmeter/credit/balance.go | 8 +- openmeter/credit/engine/engine_test.go | 8 +- openmeter/credit/grant.go | 4 +- openmeter/credit/grant/owner_connector.go | 3 +- openmeter/entitlement/metered/balance.go | 10 +- .../metered/grant_owner_adapter.go | 7 +- openmeter/meter/parse.go | 31 ++-- openmeter/meter/parse_test.go | 66 ++++--- openmeter/server/router/meter_query.go | 9 +- openmeter/server/server_test.go | 2 +- openmeter/sink/models/models.go | 7 +- openmeter/sink/namespaces.go | 22 ++- openmeter/sink/storage.go | 9 +- .../clickhouse_connector/connector.go | 13 +- .../clickhouse_connector/meter_event_query.go | 9 +- .../clickhouse_connector/meter_query.go | 30 +--- .../clickhouse_connector/meter_query_test.go | 164 ++++++++++-------- openmeter/streaming/connector.go | 2 +- openmeter/streaming/query_params.go | 1 - openmeter/streaming/testutils/streaming.go | 8 +- .../streaming/testutils/streaming_test.go | 6 +- pkg/models/meter.go | 8 + 22 files changed, 238 insertions(+), 189 deletions(-) diff --git a/openmeter/credit/balance.go b/openmeter/credit/balance.go index cda8bd19b..b1ba7e87b 100644 --- a/openmeter/credit/balance.go +++ b/openmeter/credit/balance.go @@ -229,7 +229,7 @@ func (m *connector) ResetUsageForOwner(ctx context.Context, owner grant.Namespac return nil, fmt.Errorf("failed to get owner query params for owner %s: %w", owner.ID, err) } - at := params.At.Truncate(ownerMeter.WindowSize.Duration()) + at := params.At.Truncate(ownerMeter.Meter.WindowSize.Duration()) // check if reset is possible (after last reset) periodStart, err := m.ownerConnector.GetUsagePeriodStartAt(ctx, owner, clock.Now()) @@ -425,9 +425,9 @@ func (m *connector) getQueryUsageFn(ctx context.Context, owner grant.NamespacedO params.From = &from params.To = &to params.FilterSubject = []string{subjectKey} - rows, err := m.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.MeterSlug, params) + rows, err := m.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.Meter, params) if err != nil { - return 0.0, fmt.Errorf("failed to query meter %s: %w", ownerMeter.MeterSlug, err) + return 0.0, fmt.Errorf("failed to query meter %s: %w", ownerMeter.Meter.Slug, err) } if len(rows) > 1 { return 0.0, fmt.Errorf("expected 1 row, got %d", len(rows)) @@ -437,7 +437,7 @@ func (m *connector) getQueryUsageFn(ctx context.Context, owner grant.NamespacedO } return rows[0].Value, nil }, - Grantuality: ownerMeter.WindowSize, + Grantuality: ownerMeter.Meter.WindowSize, }, nil } diff --git a/openmeter/credit/engine/engine_test.go b/openmeter/credit/engine/engine_test.go index fb7a046e9..0610c5bca 100644 --- a/openmeter/credit/engine/engine_test.go +++ b/openmeter/credit/engine/engine_test.go @@ -26,6 +26,10 @@ func TestEngine(t *testing.T) { assert.NoError(t, err) meterSlug := "meter-1" + meter := models.Meter{ + Slug: meterSlug, + } + grant1 := makeGrant(grant.Grant{ ID: "grant-1", Amount: 100.0, @@ -795,7 +799,7 @@ func TestEngine(t *testing.T) { streamingConnector := testutils.NewMockStreamingConnector(t) queryFeatureUsage := func(ctx context.Context, from, to time.Time) (float64, error) { - rows, err := streamingConnector.QueryMeter(ctx, "default", meterSlug, &streaming.QueryParams{ + rows, err := streamingConnector.QueryMeter(ctx, "default", meter, &streaming.QueryParams{ From: &from, To: &to, }) @@ -1091,7 +1095,7 @@ func TestEngine(t *testing.T) { streamingConnector := testutils.NewMockStreamingConnector(t) queryFeatureUsage := func(ctx context.Context, from, to time.Time) (float64, error) { - rows, err := streamingConnector.QueryMeter(ctx, "default", meterSlug, &streaming.QueryParams{ + rows, err := streamingConnector.QueryMeter(ctx, "default", meter, &streaming.QueryParams{ From: &from, To: &to, }) diff --git a/openmeter/credit/grant.go b/openmeter/credit/grant.go index 3eeec0aea..2afa0aeb3 100644 --- a/openmeter/credit/grant.go +++ b/openmeter/credit/grant.go @@ -41,11 +41,11 @@ func (m *connector) CreateGrant(ctx context.Context, owner grant.NamespacedOwner // All metering information is stored in windowSize chunks, // so we cannot do accurate calculations unless we follow that same windowing. - meter, err := m.ownerConnector.GetMeter(ctx, owner) + ownerMeter, err := m.ownerConnector.GetMeter(ctx, owner) if err != nil { return nil, err } - granularity := meter.WindowSize.Duration() + granularity := ownerMeter.Meter.WindowSize.Duration() input.EffectiveAt = input.EffectiveAt.Truncate(granularity) if input.Recurrence != nil { input.Recurrence.Anchor = input.Recurrence.Anchor.Truncate(granularity) diff --git a/openmeter/credit/grant/owner_connector.go b/openmeter/credit/grant/owner_connector.go index 50cea5629..a2cd83539 100644 --- a/openmeter/credit/grant/owner_connector.go +++ b/openmeter/credit/grant/owner_connector.go @@ -15,9 +15,8 @@ type EndCurrentUsagePeriodParams struct { } type OwnerMeter struct { - MeterSlug string + Meter models.Meter DefaultParams *streaming.QueryParams - WindowSize models.WindowSize SubjectKey string } diff --git a/openmeter/entitlement/metered/balance.go b/openmeter/entitlement/metered/balance.go index 29b4d5353..4e9ac931d 100644 --- a/openmeter/entitlement/metered/balance.go +++ b/openmeter/entitlement/metered/balance.go @@ -81,7 +81,7 @@ func (e *connector) GetEntitlementBalance(ctx context.Context, entitlementID mod meterQuery.To = convert.ToPointer(trunc.Add(time.Minute)) } - rows, err := e.streamingConnector.QueryMeter(ctx, entitlementID.Namespace, ownerMeter.MeterSlug, meterQuery) + rows, err := e.streamingConnector.QueryMeter(ctx, entitlementID.Namespace, ownerMeter.Meter, meterQuery) if err != nil { return nil, fmt.Errorf("failed to query meter: %w", err) } @@ -144,8 +144,8 @@ func (e *connector) GetEntitlementBalanceHistory(ctx context.Context, entitlemen // 1. we get the burndown history burndownHistory, err := e.balanceConnector.GetBalanceHistoryOfOwner(ctx, owner, credit.BalanceHistoryParams{ - From: params.From.Truncate(ownerMeter.WindowSize.Duration()), - To: params.To.Truncate(ownerMeter.WindowSize.Duration()), + From: params.From.Truncate(ownerMeter.Meter.WindowSize.Duration()), + To: params.To.Truncate(ownerMeter.Meter.WindowSize.Duration()), }) if err != nil { return nil, engine.GrantBurnDownHistory{}, fmt.Errorf("failed to get balance history: %w", err) @@ -159,7 +159,7 @@ func (e *connector) GetEntitlementBalanceHistory(ctx context.Context, entitlemen meterQuery.WindowSize = convert.ToPointer(models.WindowSize(params.WindowSize)) meterQuery.WindowTimeZone = ¶ms.WindowTimeZone - meterRows, err := e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.MeterSlug, meterQuery) + meterRows, err := e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.Meter, meterQuery) if err != nil { return nil, engine.GrantBurnDownHistory{}, fmt.Errorf("failed to query meter: %w", err) } @@ -171,7 +171,7 @@ func (e *connector) GetEntitlementBalanceHistory(ctx context.Context, entitlemen nonWindowedParams.FilterSubject = []string{ownerMeter.SubjectKey} nonWindowedParams.WindowSize = nil nonWindowedParams.WindowTimeZone = nil - meterRows, err = e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.MeterSlug, &nonWindowedParams) + meterRows, err = e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.Meter, &nonWindowedParams) if err != nil { return nil, engine.GrantBurnDownHistory{}, fmt.Errorf("failed to query meter: %w", err) } diff --git a/openmeter/entitlement/metered/grant_owner_adapter.go b/openmeter/entitlement/metered/grant_owner_adapter.go index b14861489..1ff5583ea 100644 --- a/openmeter/entitlement/metered/grant_owner_adapter.go +++ b/openmeter/entitlement/metered/grant_owner_adapter.go @@ -65,9 +65,7 @@ func (e *entitlementGrantOwner) GetMeter(ctx context.Context, owner grant.Namesp return nil, fmt.Errorf("failed to get meter: %w", err) } - queryParams := &streaming.QueryParams{ - Aggregation: meter.Aggregation, - } + queryParams := &streaming.QueryParams{} if feature.MeterGroupByFilters != nil { queryParams.FilterGroupBy = map[string][]string{} @@ -77,9 +75,8 @@ func (e *entitlementGrantOwner) GetMeter(ctx context.Context, owner grant.Namesp } return &grant.OwnerMeter{ - MeterSlug: meter.Slug, + Meter: meter, DefaultParams: queryParams, - WindowSize: meter.WindowSize, SubjectKey: entitlement.SubjectKey, }, nil } diff --git a/openmeter/meter/parse.go b/openmeter/meter/parse.go index 735dbaecc..15f3eecae 100644 --- a/openmeter/meter/parse.go +++ b/openmeter/meter/parse.go @@ -12,13 +12,13 @@ import ( ) // ParseEvent validates and parses an event against a meter. -func ParseEvent(meter Meter, ev event.Event) (float64, map[string]string, error) { +func ParseEvent(meter Meter, ev event.Event) (*float64, *string, map[string]string, error) { // Parse CloudEvents data var data interface{} err := ev.DataAs(&data) if err != nil { - return 0, map[string]string{}, errors.New("cannot unmarshal event data") + return nil, nil, map[string]string{}, errors.New("cannot unmarshal event data") } // Parse group by fields @@ -26,32 +26,27 @@ func ParseEvent(meter Meter, ev event.Event) (float64, map[string]string, error) // We can skip count events as they don't have value property if meter.Aggregation == MeterAggregationCount { - return 1, groupBy, nil + value := 1.0 + return &value, nil, groupBy, nil } // Get value from event data by value property rawValue, err := jsonpath.JsonPathLookup(data, meter.ValueProperty) if err != nil { - return 0, groupBy, fmt.Errorf("event data is missing value property at %q", meter.ValueProperty) + return nil, nil, groupBy, fmt.Errorf("event data is missing value property at %q", meter.ValueProperty) } if rawValue == nil { - return 0, groupBy, errors.New("event data value cannot be null") + return nil, nil, groupBy, errors.New("event data value cannot be null") } // Aggregation specific value validation switch meter.Aggregation { // UNIQUE_COUNT aggregation requires string property value case MeterAggregationUniqueCount: - switch rawValue.(type) { - case string, float64: - // We return 0 because the meter is unique by this string value. - // The actual value is not used in the aggregation even if it's number. - return 0, groupBy, nil - - default: - return 0, groupBy, errors.New("event data value property must be string for unique count aggregation") - } + // We convert the value to string + val := fmt.Sprintf("%v", rawValue) + return nil, &val, groupBy, nil // SUM, AVG, MIN, MAX aggregations require float64 parsable value property value case MeterAggregationSum, MeterAggregationAvg, MeterAggregationMin, MeterAggregationMax: @@ -60,18 +55,18 @@ func ParseEvent(meter Meter, ev event.Event) (float64, map[string]string, error) _, err = strconv.ParseFloat(value, 64) if err != nil { // TODO: omit value or make sure it's length is not too long - return 0, groupBy, fmt.Errorf("event data value cannot be parsed as float64: %s", value) + return nil, nil, groupBy, fmt.Errorf("event data value cannot be parsed as float64: %s", value) } case float64: - return value, groupBy, nil + return &value, nil, groupBy, nil default: - return 0, groupBy, errors.New("event data value property cannot be parsed") + return nil, nil, groupBy, errors.New("event data value property cannot be parsed") } } - return 0, groupBy, fmt.Errorf("unknown meter aggregation: %s", meter.Aggregation) + return nil, nil, groupBy, fmt.Errorf("unknown meter aggregation: %s", meter.Aggregation) } // parseGroupBy parses the group by fields from the event data diff --git a/openmeter/meter/parse_test.go b/openmeter/meter/parse_test.go index 0adb37c9e..e68d686c3 100644 --- a/openmeter/meter/parse_test.go +++ b/openmeter/meter/parse_test.go @@ -5,18 +5,19 @@ import ( "testing" "github.com/cloudevents/sdk-go/v2/event" + "github.com/samber/lo" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/openmeterio/openmeter/openmeter/meter" + "github.com/openmeterio/openmeter/pkg/models" ) func TestParseEvent(t *testing.T) { meterSum := meter.Meter{ Namespace: "default", Slug: "m1", - Description: "", - Aggregation: "SUM", + Aggregation: models.MeterAggregationSum, EventType: "api-calls", ValueProperty: "$.duration_ms", GroupBy: map[string]string{ @@ -29,18 +30,27 @@ func TestParseEvent(t *testing.T) { meterCount := meter.Meter{ Namespace: "default", Slug: "m2", - Description: "", - Aggregation: "COUNT", + Aggregation: models.MeterAggregationCount, EventType: "api-calls", WindowSize: meter.WindowSizeMinute, } + meterUniqueCount := meter.Meter{ + Namespace: "default", + Slug: "m3", + Aggregation: models.MeterAggregationUniqueCount, + EventType: "spans", + ValueProperty: "$.trace_id", + WindowSize: meter.WindowSizeMinute, + } + tests := []struct { description string meter meter.Meter event func(t *testing.T) event.Event - want error - value float64 + err error + value *float64 + valueStr *string groupBy map[string]string }{ { @@ -55,7 +65,7 @@ func TestParseEvent(t *testing.T) { return ev }, - value: 100, + value: lo.ToPtr(100.0), groupBy: map[string]string{ "method": "GET", "path": "/api/v1", @@ -70,9 +80,24 @@ func TestParseEvent(t *testing.T) { return ev }, - value: 1, + value: lo.ToPtr(1.0), groupBy: map[string]string{}, }, + { + description: "should parse unique count as string", + meter: meterUniqueCount, + event: func(t *testing.T) event.Event { + ev := event.New() + ev.SetType("spans") + + err := ev.SetData(event.ApplicationJSON, []byte(`{"trace_id": "test_trace_id"}`)) + require.NoError(t, err) + + return ev + }, + valueStr: lo.ToPtr("test_trace_id"), + groupBy: map[string]string{}, + }, { description: "should parse event with missing group by properties", meter: meterSum, @@ -85,8 +110,11 @@ func TestParseEvent(t *testing.T) { return ev }, - value: 100, - groupBy: map[string]string{}, + value: lo.ToPtr(100.0), + groupBy: map[string]string{ + "method": "", + "path": "", + }, }, { description: "should return error with invalid json", @@ -100,7 +128,7 @@ func TestParseEvent(t *testing.T) { return ev }, - want: errors.New("cannot unmarshal event data"), + err: errors.New("cannot unmarshal event data"), groupBy: map[string]string{}, }, { @@ -115,7 +143,7 @@ func TestParseEvent(t *testing.T) { return ev }, - want: errors.New("event data is missing value property at \"$.duration_ms\""), + err: errors.New("event data is missing value property at \"$.duration_ms\""), groupBy: map[string]string{ "method": "GET", "path": "/api/v1", @@ -133,7 +161,7 @@ func TestParseEvent(t *testing.T) { return ev }, - want: errors.New("event data value cannot be null"), + err: errors.New("event data value cannot be null"), groupBy: map[string]string{ "method": "GET", "path": "/api/v1", @@ -151,7 +179,7 @@ func TestParseEvent(t *testing.T) { return ev }, - want: errors.New("event data value cannot be parsed as float64: not a number"), + err: errors.New("event data value cannot be parsed as float64: not a number"), groupBy: map[string]string{ "method": "GET", "path": "/api/v1", @@ -163,15 +191,11 @@ func TestParseEvent(t *testing.T) { test := test t.Run(test.description, func(t *testing.T) { - value, groupBy, err := meter.ParseEvent(test.meter, test.event(t)) - if test.want == nil { - assert.Nil(t, err) - - return - } + value, valueStr, groupBy, err := meter.ParseEvent(test.meter, test.event(t)) - assert.Equal(t, test.want, err) + assert.Equal(t, test.err, err) assert.Equal(t, test.value, value) + assert.Equal(t, test.valueStr, valueStr) assert.Equal(t, test.groupBy, groupBy) }) } diff --git a/openmeter/server/router/meter_query.go b/openmeter/server/router/meter_query.go index 3bc4cf7b1..9659e3607 100644 --- a/openmeter/server/router/meter_query.go +++ b/openmeter/server/router/meter_query.go @@ -52,10 +52,9 @@ func (a *Router) QueryMeter(w http.ResponseWriter, r *http.Request, meterIDOrSlu func (a *Router) QueryMeterWithMeter(ctx context.Context, w http.ResponseWriter, r *http.Request, logger *slog.Logger, meter models.Meter, params api.QueryMeterParams) { // Query Params queryParams := &streaming.QueryParams{ - From: params.From, - To: params.To, - WindowSize: params.WindowSize, - Aggregation: meter.Aggregation, + From: params.From, + To: params.To, + WindowSize: params.WindowSize, } if params.GroupBy != nil { @@ -122,7 +121,7 @@ func (a *Router) QueryMeterWithMeter(ctx context.Context, w http.ResponseWriter, } // Query connector - data, err := a.config.StreamingConnector.QueryMeter(ctx, meter.Namespace, meter.Slug, queryParams) + data, err := a.config.StreamingConnector.QueryMeter(ctx, meter.Namespace, meter, queryParams) if err != nil { err := fmt.Errorf("query meter: %w", err) diff --git a/openmeter/server/server_test.go b/openmeter/server/server_test.go index 8cbcb4f53..15222a0b8 100644 --- a/openmeter/server/server_test.go +++ b/openmeter/server/server_test.go @@ -96,7 +96,7 @@ func (c *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace stri return nil } -func (c *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { value := mockQueryValue if params.FilterSubject == nil { diff --git a/openmeter/sink/models/models.go b/openmeter/sink/models/models.go index 6673686d8..3d0d372e2 100644 --- a/openmeter/sink/models/models.go +++ b/openmeter/sink/models/models.go @@ -20,9 +20,10 @@ type SinkMessage struct { } type MeterEvent struct { - Meter *models.Meter - Value float64 - GroupBy map[string]string + Meter *models.Meter + Value float64 + ValueString string + GroupBy map[string]string } type ProcessingState int8 diff --git a/openmeter/sink/namespaces.go b/openmeter/sink/namespaces.go index 2f3a81814..7108f7a54 100644 --- a/openmeter/sink/namespaces.go +++ b/openmeter/sink/namespaces.go @@ -72,8 +72,8 @@ func (n *NamespaceStore) ValidateEvent(_ context.Context, m *sinkmodels.SinkMess } } - // Validate event against meter - value, groupBy, err := ommeter.ParseEvent(meter, event) + // Parse event with meter + value, valueString, groupBy, err := ommeter.ParseEvent(meter, event) if err != nil { m.Status = sinkmodels.ProcessingStatus{ State: sinkmodels.INVALID, @@ -83,11 +83,23 @@ func (n *NamespaceStore) ValidateEvent(_ context.Context, m *sinkmodels.SinkMess return } - m.MeterEvents = append(m.MeterEvents, sinkmodels.MeterEvent{ + // Create meter event + meterEvent := sinkmodels.MeterEvent{ Meter: &meter, - Value: value, GroupBy: groupBy, - }) + } + + // Meterring numeric value + if value != nil { + meterEvent.Value = *value + } + + // Meterring string value + if valueString != nil { + meterEvent.ValueString = *valueString + } + + m.MeterEvents = append(m.MeterEvents, meterEvent) } } diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index f2af44f8e..d94112078 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -94,18 +94,13 @@ func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmode // Meter events per meter for _, meterEvent := range message.MeterEvents { - meterID := meterEvent.Meter.ID - - if meterID == "" { - meterID = meterEvent.Meter.Slug - } - meterEvent := clickhouse_connector.CHMeterEvent{ Namespace: message.Namespace, Time: rawEvent.Time, - Meter: meterID, + Meter: meterEvent.Meter.GetID(), Subject: rawEvent.Subject, Value: meterEvent.Value, + ValueString: meterEvent.ValueString, GroupBy: meterEvent.GroupBy, EventID: rawEvent.ID, EventSource: rawEvent.Source, diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index ef7ca095a..e827c2b5b 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -10,12 +10,12 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" "github.com/cloudevents/sdk-go/v2/event" + "github.com/shopspring/decimal" "github.com/openmeterio/openmeter/api" "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/pkg/models" - "github.com/shopspring/decimal" ) // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. @@ -67,12 +67,12 @@ func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, return nil } -func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } - values, err := c.queryMeter(ctx, namespace, meterSlug, params) + values, err := c.queryMeter(ctx, namespace, meter, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { return nil, err @@ -290,12 +290,11 @@ func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace st return results, nil } -func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeter{ Database: c.config.Database, Namespace: namespace, - MeterSlug: meterSlug, - Aggregation: params.Aggregation, + Meter: meter, From: params.From, To: params.To, Subject: params.FilterSubject, @@ -316,7 +315,7 @@ func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, rows, err := c.config.ClickHouse.Query(ctx, sql, args...) if err != nil { if strings.Contains(err.Error(), "code: 60") { - return nil, &models.MeterNotFoundError{MeterSlug: meterSlug} + return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} } return values, fmt.Errorf("query meter view query: %w", err) diff --git a/openmeter/streaming/clickhouse_connector/meter_event_query.go b/openmeter/streaming/clickhouse_connector/meter_event_query.go index d9a0b321f..dffc38790 100644 --- a/openmeter/streaming/clickhouse_connector/meter_event_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_event_query.go @@ -22,8 +22,9 @@ type CHMeterEvent struct { Subject string `ch:"subject"` // Usage - Value float64 `ch:"value"` - GroupBy map[string]string `ch:"group_by"` + Value float64 `ch:"value"` + ValueString string `ch:"value_str"` + GroupBy map[string]string `ch:"group_by"` // Metadata EventID string `ch:"event_id"` @@ -53,6 +54,8 @@ func (d createMeterEventTable) toSQL() string { // Usage sb.Define("value", "Decimal(14, 4)") + // For unique aggregation we need to store the value as a string + sb.Define("value_str", "String") sb.Define("group_by", "Map(String, String)") // Metadata @@ -87,6 +90,7 @@ func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { "meter", "subject", "value", + "value_str", "group_by", "event_id", "event_source", @@ -112,6 +116,7 @@ func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { meterEvent.Meter, meterEvent.Subject, meterEvent.Value, + meterEvent.ValueString, meterEvent.GroupBy, meterEvent.EventID, meterEvent.EventSource, diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index 9d0b09efd..be846bf94 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -12,16 +12,10 @@ import ( "github.com/openmeterio/openmeter/pkg/slicesx" ) -type column struct { - Name string - Type string -} - type queryMeter struct { Database string Namespace string - MeterSlug string - Aggregation models.MeterAggregation + Meter models.Meter Subject []string FilterGroupBy map[string][]string From *time.Time @@ -77,7 +71,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") } - switch d.Aggregation { + switch d.Meter.Aggregation { case models.MeterAggregationSum: selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) case models.MeterAggregationAvg: @@ -87,12 +81,11 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { case models.MeterAggregationMax: selectColumns = append(selectColumns, fmt.Sprintf("max(%s) AS value", getColumn("value"))) case models.MeterAggregationUniqueCount: - // FIXME: value is a number, not a string - selectColumns = append(selectColumns, fmt.Sprintf("toFloat64(uniq(%s)) AS value", getColumn("value"))) + selectColumns = append(selectColumns, fmt.Sprintf("toDecimal(uniq(%s)) AS value", getColumn("value_str"))) case models.MeterAggregationCount: selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) default: - return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) + return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Meter.Aggregation) } for _, groupByKey := range d.GroupBy { @@ -112,7 +105,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { query.Select(selectColumns...) query.From(tableName) query.Where(query.Equal(getColumn("namespace"), d.Namespace)) - query.Where(query.Equal(getColumn("meter"), d.MeterSlug)) + query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) if len(d.Subject) > 0 { mapFunc := func(subject string) string { @@ -140,7 +133,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { // Subject is a special case if groupByKey == "subject" { - column = fmt.Sprintf("subject") + column = "subject" } return query.Equal(column, value) @@ -172,17 +165,6 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { return sql, args, nil } -func sortedKeys(m map[string]string) []string { - keys := make([]string, len(m)) - i := 0 - for k := range m { - keys[i] = k - i++ - } - sort.Strings(keys) - return keys -} - type listMeterSubjectsQuery struct { Database string Namespace string diff --git a/openmeter/streaming/clickhouse_connector/meter_query_test.go b/openmeter/streaming/clickhouse_connector/meter_query_test.go index 7d4cd7424..bb27758fa 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector/meter_query_test.go @@ -23,81 +23,95 @@ func TestQueryMeter(t *testing.T) { }{ { query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - From: &from, - To: &to, - GroupBy: []string{"subject", "group1", "group2"}, - WindowSize: &windowSize, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + From: &from, + To: &to, + GroupBy: []string{"subject", "group1", "group2"}, + WindowSize: &windowSize, }, wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, }, { // Aggregate all available data query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationCount, + }, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - To: &to, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - From: &from, - To: &to, - WindowSize: &windowSize, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + WindowSize: &windowSize, }, wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period in a different timezone, groupped by window size query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, From: &from, To: &to, WindowSize: &windowSize, @@ -108,46 +122,54 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data for a single subject query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - GroupBy: []string{"subject"}, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject"}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject}, - GroupBy: []string{"subject", "group1", "group2"}, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject", "group1", "group2"}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - Subject: []string{subject, "subject2"}, - GroupBy: []string{"subject"}, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject, "subject2"}, + GroupBy: []string{"subject"}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", @@ -155,10 +177,12 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data with filtering for a single group and multiple values query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", @@ -166,10 +190,12 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data with filtering for multiple groups and multiple values query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", diff --git a/openmeter/streaming/connector.go b/openmeter/streaming/connector.go index 3d61b3cbd..5700c3716 100644 --- a/openmeter/streaming/connector.go +++ b/openmeter/streaming/connector.go @@ -35,7 +35,7 @@ type Connector interface { ListEvents(ctx context.Context, namespace string, params ListEventsParams) ([]api.IngestedEvent, error) CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error DeleteMeter(ctx context.Context, namespace string, meterSlug string) error - QueryMeter(ctx context.Context, namespace string, meterSlug string, params *QueryParams) ([]models.MeterQueryRow, error) + QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *QueryParams) ([]models.MeterQueryRow, error) ListMeterSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) // Add more methods as needed ... } diff --git a/openmeter/streaming/query_params.go b/openmeter/streaming/query_params.go index 27bdccbb8..4fdd3e023 100644 --- a/openmeter/streaming/query_params.go +++ b/openmeter/streaming/query_params.go @@ -14,7 +14,6 @@ type QueryParams struct { FilterSubject []string FilterGroupBy map[string][]string GroupBy []string - Aggregation models.MeterAggregation WindowSize *models.WindowSize WindowTimeZone *time.Location } diff --git a/openmeter/streaming/testutils/streaming.go b/openmeter/streaming/testutils/streaming.go index 071cb28ad..1bae5e87d 100644 --- a/openmeter/streaming/testutils/streaming.go +++ b/openmeter/streaming/testutils/streaming.go @@ -67,18 +67,18 @@ func (m *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace stri // Returns the result query set for the given params. If the query set is not found, // it will try to approximate the result by aggregating the simple events -func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { rows := []models.MeterQueryRow{} - _, rowOk := m.rows[meterSlug] + _, rowOk := m.rows[meter.Slug] if rowOk { - for _, row := range m.rows[meterSlug] { + for _, row := range m.rows[meter.Slug] { if row.WindowStart.Equal(*params.From) && row.WindowEnd.Equal(*params.To) { rows = append(rows, row) } } } else { - row, err := m.aggregateEvents(meterSlug, params) + row, err := m.aggregateEvents(meter.Slug, params) if err != nil { return rows, err } diff --git a/openmeter/streaming/testutils/streaming_test.go b/openmeter/streaming/testutils/streaming_test.go index c02ba631c..13b6bd1bd 100644 --- a/openmeter/streaming/testutils/streaming_test.go +++ b/openmeter/streaming/testutils/streaming_test.go @@ -16,6 +16,10 @@ import ( func TestMockStreamingConnector(t *testing.T) { defaultMeterSlug := "default-meter" + defaultMeter := models.Meter{ + Slug: defaultMeterSlug, + } + type tc struct { Name string Events []SimpleEvent @@ -203,7 +207,7 @@ func TestMockStreamingConnector(t *testing.T) { streamingConnector.AddRow(defaultMeterSlug, row) } - result, err := streamingConnector.QueryMeter(context.Background(), "namespace", defaultMeterSlug, tc.Query) + result, err := streamingConnector.QueryMeter(context.Background(), "namespace", defaultMeter, tc.Query) if tc.ExpectedError != nil { assert.Error(t, err) assert.Equal(t, tc.ExpectedError, err) diff --git a/pkg/models/meter.go b/pkg/models/meter.go index 7f9e6356c..59154ae08 100644 --- a/pkg/models/meter.go +++ b/pkg/models/meter.go @@ -114,6 +114,14 @@ type Meter struct { WindowSize WindowSize `json:"windowSize,omitempty" yaml:"windowSize,omitempty"` } +func (m *Meter) GetID() string { + if m.ID == "" { + return m.Slug + } + + return m.ID +} + type MeterOptions struct { ID string Description string From a38468b4171564b2fc15f9c50a7d427954655412 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 17:31:14 -0700 Subject: [PATCH 06/33] feat(streaming): raw events connector --- cmd/server/main.go | 7 +- openmeter/credit/engine/engine_test.go | 4 +- openmeter/credit/grant/owner_connector.go | 2 +- openmeter/entitlement/metered/balance.go | 4 +- .../metered/grant_owner_adapter.go | 2 +- openmeter/server/router/meter_query.go | 2 +- openmeter/server/router/meter_subject.go | 22 +- openmeter/server/server_test.go | 8 +- .../clickhouse_connector/connector.go | 18 +- .../clickhouse_connector_raw/connector.go | 394 ++++++++++++++++++ .../clickhouse_connector_raw/event_query.go | 180 ++++++++ .../event_query_test.go | 185 ++++++++ .../clickhouse_connector_raw/meter_query.go | 229 ++++++++++ .../meter_query_test.go | 283 +++++++++++++ openmeter/streaming/connector.go | 13 +- openmeter/streaming/testutils/streaming.go | 12 +- .../streaming/testutils/streaming_test.go | 18 +- 17 files changed, 1344 insertions(+), 39 deletions(-) create mode 100644 openmeter/streaming/clickhouse_connector_raw/connector.go create mode 100644 openmeter/streaming/clickhouse_connector_raw/event_query.go create mode 100644 openmeter/streaming/clickhouse_connector_raw/event_query_test.go create mode 100644 openmeter/streaming/clickhouse_connector_raw/meter_query.go create mode 100644 openmeter/streaming/clickhouse_connector_raw/meter_query_test.go diff --git a/cmd/server/main.go b/cmd/server/main.go index 26479b4ea..8eef91969 100644 --- a/cmd/server/main.go +++ b/cmd/server/main.go @@ -336,7 +336,12 @@ func main() { }) for _, meter := range conf.Meters { - err := app.StreamingConnector.CreateMeter(ctx, app.NamespaceManager.GetDefaultNamespace(), meter) + if meter == nil { + logger.Error("meter configuration is nil") + os.Exit(1) + } + + err := app.StreamingConnector.CreateMeter(ctx, app.NamespaceManager.GetDefaultNamespace(), *meter) if err != nil { slog.Warn("failed to initialize meter", "error", err) os.Exit(1) diff --git a/openmeter/credit/engine/engine_test.go b/openmeter/credit/engine/engine_test.go index 0610c5bca..25c9bfb53 100644 --- a/openmeter/credit/engine/engine_test.go +++ b/openmeter/credit/engine/engine_test.go @@ -799,7 +799,7 @@ func TestEngine(t *testing.T) { streamingConnector := testutils.NewMockStreamingConnector(t) queryFeatureUsage := func(ctx context.Context, from, to time.Time) (float64, error) { - rows, err := streamingConnector.QueryMeter(ctx, "default", meter, &streaming.QueryParams{ + rows, err := streamingConnector.QueryMeter(ctx, "default", meter, streaming.QueryParams{ From: &from, To: &to, }) @@ -1095,7 +1095,7 @@ func TestEngine(t *testing.T) { streamingConnector := testutils.NewMockStreamingConnector(t) queryFeatureUsage := func(ctx context.Context, from, to time.Time) (float64, error) { - rows, err := streamingConnector.QueryMeter(ctx, "default", meter, &streaming.QueryParams{ + rows, err := streamingConnector.QueryMeter(ctx, "default", meter, streaming.QueryParams{ From: &from, To: &to, }) diff --git a/openmeter/credit/grant/owner_connector.go b/openmeter/credit/grant/owner_connector.go index a2cd83539..ce7ad14e9 100644 --- a/openmeter/credit/grant/owner_connector.go +++ b/openmeter/credit/grant/owner_connector.go @@ -16,7 +16,7 @@ type EndCurrentUsagePeriodParams struct { type OwnerMeter struct { Meter models.Meter - DefaultParams *streaming.QueryParams + DefaultParams streaming.QueryParams SubjectKey string } diff --git a/openmeter/entitlement/metered/balance.go b/openmeter/entitlement/metered/balance.go index 4e9ac931d..af8c8f807 100644 --- a/openmeter/entitlement/metered/balance.go +++ b/openmeter/entitlement/metered/balance.go @@ -167,11 +167,11 @@ func (e *connector) GetEntitlementBalanceHistory(ctx context.Context, entitlemen // If we get 0 rows that means the windowsize is larger than the queried period. // In this case we simply query for the entire period. if len(meterRows) == 0 { - nonWindowedParams := *meterQuery + nonWindowedParams := meterQuery nonWindowedParams.FilterSubject = []string{ownerMeter.SubjectKey} nonWindowedParams.WindowSize = nil nonWindowedParams.WindowTimeZone = nil - meterRows, err = e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.Meter, &nonWindowedParams) + meterRows, err = e.streamingConnector.QueryMeter(ctx, owner.Namespace, ownerMeter.Meter, nonWindowedParams) if err != nil { return nil, engine.GrantBurnDownHistory{}, fmt.Errorf("failed to query meter: %w", err) } diff --git a/openmeter/entitlement/metered/grant_owner_adapter.go b/openmeter/entitlement/metered/grant_owner_adapter.go index 1ff5583ea..8ee5fb7f4 100644 --- a/openmeter/entitlement/metered/grant_owner_adapter.go +++ b/openmeter/entitlement/metered/grant_owner_adapter.go @@ -65,7 +65,7 @@ func (e *entitlementGrantOwner) GetMeter(ctx context.Context, owner grant.Namesp return nil, fmt.Errorf("failed to get meter: %w", err) } - queryParams := &streaming.QueryParams{} + queryParams := streaming.QueryParams{} if feature.MeterGroupByFilters != nil { queryParams.FilterGroupBy = map[string][]string{} diff --git a/openmeter/server/router/meter_query.go b/openmeter/server/router/meter_query.go index 9659e3607..53d6963f3 100644 --- a/openmeter/server/router/meter_query.go +++ b/openmeter/server/router/meter_query.go @@ -51,7 +51,7 @@ func (a *Router) QueryMeter(w http.ResponseWriter, r *http.Request, meterIDOrSlu // QueryMeter queries the values stored for a meter. func (a *Router) QueryMeterWithMeter(ctx context.Context, w http.ResponseWriter, r *http.Request, logger *slog.Logger, meter models.Meter, params api.QueryMeterParams) { // Query Params - queryParams := &streaming.QueryParams{ + queryParams := streaming.QueryParams{ From: params.From, To: params.To, WindowSize: params.WindowSize, diff --git a/openmeter/server/router/meter_subject.go b/openmeter/server/router/meter_subject.go index 14a2c1949..6738e7ed2 100644 --- a/openmeter/server/router/meter_subject.go +++ b/openmeter/server/router/meter_subject.go @@ -6,6 +6,7 @@ import ( "github.com/go-chi/render" + "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/pkg/contextx" "github.com/openmeterio/openmeter/pkg/models" ) @@ -17,7 +18,26 @@ func (a *Router) ListMeterSubjects(w http.ResponseWriter, r *http.Request, meter namespace := a.config.NamespaceManager.GetDefaultNamespace() - subjects, err := a.config.StreamingConnector.ListMeterSubjects(ctx, namespace, meterIDOrSlug, nil, nil) + // Get meter + meter, err := a.config.Meters.GetMeterByIDOrSlug(ctx, namespace, meterIDOrSlug) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + err := fmt.Errorf("meter not found: %w", err) + + models.NewStatusProblem(ctx, err, http.StatusNotFound).Respond(w) + + return + } + + err := fmt.Errorf("get meter: %w", err) + + a.config.ErrorHandler.HandleContext(ctx, err) + models.NewStatusProblem(ctx, err, http.StatusInternalServerError).Respond(w) + + return + } + + subjects, err := a.config.StreamingConnector.ListMeterSubjects(ctx, namespace, meter, streaming.ListMeterSubjectsParams{}) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { err := fmt.Errorf("meter not found: %w", err) diff --git a/openmeter/server/server_test.go b/openmeter/server/server_test.go index 15222a0b8..4777d9afa 100644 --- a/openmeter/server/server_test.go +++ b/openmeter/server/server_test.go @@ -88,15 +88,15 @@ func (c *MockStreamingConnector) ListEvents(ctx context.Context, namespace strin return events, nil } -func (c *MockStreamingConnector) CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error { +func (c *MockStreamingConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { return nil } -func (c *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace string, meterSlug string) error { +func (c *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { return nil } -func (c *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { value := mockQueryValue if params.FilterSubject == nil { @@ -106,7 +106,7 @@ func (c *MockStreamingConnector) QueryMeter(ctx context.Context, namespace strin return []models.MeterQueryRow{value}, nil } -func (c *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { +func (c *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { return []string{"s1"}, nil } diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index e827c2b5b..8f5b5d24c 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -18,6 +18,8 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) +var _ streaming.Connector = (*ClickhouseConnector)(nil) + // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. type ClickhouseConnector struct { config ClickhouseConnectorConfig @@ -57,17 +59,17 @@ func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, return events, nil } -func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error { +func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { // Do nothing return nil } -func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meterSlug string) error { +func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { // Do nothing return nil } -func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -98,15 +100,15 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return values, nil } -func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { +func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } - if meterSlug == "" { - return nil, fmt.Errorf("slug is required") + if meter.Slug == "" { + return nil, fmt.Errorf("meter is required") } - subjects, err := c.listMeterViewSubjects(ctx, namespace, meterSlug, from, to) + subjects, err := c.listMeterViewSubjects(ctx, namespace, meter.Slug, params.From, params.To) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { return nil, err @@ -290,7 +292,7 @@ func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace st return results, nil } -func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeter{ Database: c.config.Database, Namespace: namespace, diff --git a/openmeter/streaming/clickhouse_connector_raw/connector.go b/openmeter/streaming/clickhouse_connector_raw/connector.go new file mode 100644 index 000000000..354a68cd3 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_raw/connector.go @@ -0,0 +1,394 @@ +package clickhouse_connector_raw + +import ( + "context" + "encoding/json" + "fmt" + "log/slog" + "strings" + "time" + + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/cloudevents/sdk-go/v2/event" + "github.com/shopspring/decimal" + + "github.com/openmeterio/openmeter/api" + "github.com/openmeterio/openmeter/openmeter/meter" + "github.com/openmeterio/openmeter/openmeter/streaming" + "github.com/openmeterio/openmeter/pkg/models" +) + +var _ streaming.Connector = (*ClickhouseConnector)(nil) + +// ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. +type ClickhouseConnector struct { + config ClickhouseConnectorConfig +} + +type ClickhouseConnectorConfig struct { + Logger *slog.Logger + ClickHouse clickhouse.Conn + Database string + Meters meter.Repository + CreateOrReplaceMeter bool + PopulateMeter bool +} + +func NewClickhouseConnector(config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { + connector := &ClickhouseConnector{ + config: config, + } + + return connector, nil +} + +func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + events, err := c.queryEventsTable(ctx, namespace, params) + if err != nil { + if _, ok := err.(*models.NamespaceNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("query events: %w", err) + } + + return events, nil +} + +func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { + // Do nothing + return nil +} + +func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { + // Do nothing + return nil +} + +func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + values, err := c.queryMeter(ctx, namespace, meter, params) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("get values: %w", err) + } + + // If the total usage is queried for a single period (no window size), + // replace the window start and end with the period for each row. + // We can still have multiple rows for a single period due to group bys. + if params.WindowSize == nil { + for i := range values { + if params.From != nil { + values[i].WindowStart = *params.From + } + if params.To != nil { + values[i].WindowEnd = *params.To + } + } + } + + return values, nil +} + +func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + if meter.Slug == "" { + return nil, fmt.Errorf("meter is required") + } + + subjects, err := c.listMeterViewSubjects(ctx, namespace, meter, params.From, params.To) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("list meter subjects: %w", err) + } + + return subjects, nil +} + +func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { + err := c.createEventsTable(ctx) + if err != nil { + return fmt.Errorf("create namespace in clickhouse: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { + // We don't delete the event tables as it it reused between namespaces + return nil +} + +func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + rows, err := c.queryCountEvents(ctx, namespace, params) + if err != nil { + if _, ok := err.(*models.NamespaceNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("query count events: %w", err) + } + + return rows, nil +} + +func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { + table := createEventsTable{ + Database: c.config.Database, + } + + err := c.config.ClickHouse.Exec(ctx, table.toSQL()) + if err != nil { + return fmt.Errorf("create events table: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + table := queryEventsTable{ + Database: c.config.Database, + Namespace: namespace, + From: params.From, + To: params.To, + IngestedAtFrom: params.IngestedAtFrom, + IngestedAtTo: params.IngestedAtTo, + ID: params.ID, + Subject: params.Subject, + HasError: params.HasError, + Limit: params.Limit, + } + + sql, args := table.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.NamespaceNotFoundError{Namespace: namespace} + } + + return nil, fmt.Errorf("query events table query: %w", err) + } + + events := []api.IngestedEvent{} + + for rows.Next() { + var id string + var eventType string + var subject string + var source string + var eventTime time.Time + var dataStr string + var validationError string + var ingestedAt time.Time + var storedAt time.Time + + if err = rows.Scan(&id, &eventType, &subject, &source, &eventTime, &dataStr, &validationError, &ingestedAt, &storedAt); err != nil { + return nil, err + } + + // Parse data + var data interface{} + err := json.Unmarshal([]byte(dataStr), &data) + if err != nil { + return nil, fmt.Errorf("query events parse data: %w", err) + } + + event := event.New() + event.SetID(id) + event.SetType(eventType) + event.SetSubject(subject) + event.SetSource(source) + event.SetTime(eventTime) + err = event.SetData("application/json", data) + if err != nil { + return nil, fmt.Errorf("query events set data: %w", err) + } + + ingestedEvent := api.IngestedEvent{ + Event: event, + } + + if validationError != "" { + ingestedEvent.ValidationError = &validationError + } + + ingestedEvent.IngestedAt = ingestedAt + ingestedEvent.StoredAt = storedAt + + events = append(events, ingestedEvent) + } + + return events, nil +} + +func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + table := queryCountEvents{ + Database: c.config.Database, + Namespace: namespace, + From: params.From, + } + + sql, args := table.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.NamespaceNotFoundError{Namespace: namespace} + } + + return nil, fmt.Errorf("query events count query: %w", err) + } + + results := []streaming.CountEventRow{} + + for rows.Next() { + result := streaming.CountEventRow{} + + if err = rows.Scan(&result.Count, &result.Subject, &result.IsError); err != nil { + return nil, err + } + + results = append(results, result) + } + + return results, nil +} + +func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { + queryMeter := queryMeter{ + Database: c.config.Database, + Namespace: namespace, + Meter: meter, + From: params.From, + To: params.To, + Subject: params.FilterSubject, + FilterGroupBy: params.FilterGroupBy, + GroupBy: params.GroupBy, + WindowSize: params.WindowSize, + WindowTimeZone: params.WindowTimeZone, + } + + values := []models.MeterQueryRow{} + + sql, args, err := queryMeter.toSQL() + if err != nil { + return values, fmt.Errorf("query meter view: %w", err) + } + + start := time.Now() + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} + } + + return values, fmt.Errorf("query meter view query: %w", err) + } + elapsed := time.Since(start) + slog.Debug("query meter view", "elapsed", elapsed.String(), "sql", sql, "args", args) + + for rows.Next() { + row := models.MeterQueryRow{ + GroupBy: map[string]*string{}, + } + + var value decimal.Decimal + args := []interface{}{&row.WindowStart, &row.WindowEnd, &value} + argCount := len(args) + + for range queryMeter.GroupBy { + tmp := "" + args = append(args, &tmp) + } + + if err := rows.Scan(args...); err != nil { + return values, fmt.Errorf("query meter view row scan: %w", err) + } + + // TODO: should we use decima all the way? + row.Value, _ = value.Float64() + + for i, key := range queryMeter.GroupBy { + if s, ok := args[i+argCount].(*string); ok { + if key == "subject" { + row.Subject = s + continue + } + + // We treat empty string as nil + if s != nil && *s == "" { + row.GroupBy[key] = nil + } else { + row.GroupBy[key] = s + } + } + } + + // an empty row is returned when there are no values for the meter + if row.WindowStart.IsZero() && row.WindowEnd.IsZero() && row.Value == 0 { + continue + } + + values = append(values, row) + } + rows.Close() + err = rows.Err() + if err != nil { + return values, fmt.Errorf("query meter rows error: %w", err) + } + + return values, nil +} + +func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, from *time.Time, to *time.Time) ([]string, error) { + query := listMeterSubjectsQuery{ + Database: c.config.Database, + Namespace: namespace, + Meter: meter, + From: from, + To: to, + } + + sql, args := query.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} + } + + return nil, fmt.Errorf("list meter view subjects: %w", err) + } + + subjects := []string{} + for rows.Next() { + var subject string + if err = rows.Scan(&subject); err != nil { + return nil, err + } + + subjects = append(subjects, subject) + } + + return subjects, nil +} diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query.go b/openmeter/streaming/clickhouse_connector_raw/event_query.go new file mode 100644 index 000000000..abbb68eac --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_raw/event_query.go @@ -0,0 +1,180 @@ +package clickhouse_connector_raw + +import ( + _ "embed" + "fmt" + "strings" + "time" + + "github.com/huandu/go-sqlbuilder" +) + +const EventsTableName = "om_events" + +// CHEvent represents a raw event in ClickHouse +type CHEvent struct { + Namespace string `ch:"namespace"` + ValidationError string `ch:"validation_error"` + ID string `ch:"id"` + Type string `ch:"type"` + Source string `ch:"source"` + Subject string `ch:"subject"` + Time time.Time `ch:"time"` + Data string `ch:"data"` + IngestedAt time.Time `ch:"ingested_at"` + StoredAt time.Time `ch:"stored_at"` +} + +// Create Events Table +type createEventsTable struct { + Database string +} + +func (d createEventsTable) toSQL() string { + tableName := GetEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() + sb.CreateTable(tableName) + sb.IfNotExists() + sb.Define("namespace", "String") + sb.Define("validation_error", "String") + sb.Define("id", "String") + sb.Define("type", "LowCardinality(String)") + sb.Define("subject", "String") + sb.Define("source", "String") + sb.Define("time", "DateTime") + sb.Define("data", "String") + sb.Define("ingested_at", "DateTime") + sb.Define("stored_at", "DateTime") + sb.SQL("ENGINE = MergeTree") + sb.SQL("PARTITION BY toYYYYMM(time)") + sb.SQL("ORDER BY (namespace, time, type, subject)") + + sql, _ := sb.Build() + return sql +} + +// Query Events Table +type queryEventsTable struct { + Database string + Namespace string + From *time.Time + To *time.Time + IngestedAtFrom *time.Time + IngestedAtTo *time.Time + ID *string + Subject *string + HasError *bool + Limit int +} + +func (d queryEventsTable) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + where := []string{} + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") + query.From(tableName) + + where = append(where, query.Equal("namespace", d.Namespace)) + if d.From != nil { + where = append(where, query.GreaterEqualThan("time", d.From.Unix())) + } + if d.To != nil { + where = append(where, query.LessEqualThan("time", d.To.Unix())) + } + if d.IngestedAtFrom != nil { + where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) + } + if d.IngestedAtTo != nil { + where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) + } + if d.ID != nil { + where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) + } + if d.Subject != nil { + where = append(where, query.Equal("subject", *d.Subject)) + } + if d.HasError != nil { + if *d.HasError { + where = append(where, "notEmpty(validation_error) = 1") + } else { + where = append(where, "empty(validation_error) = 1") + } + } + query.Where(where...) + + query.Desc().OrderBy("time") + query.Limit(d.Limit) + + sql, args := query.Build() + return sql, args +} + +type queryCountEvents struct { + Database string + Namespace string + From time.Time +} + +func (d queryCountEvents) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") + query.From(tableName) + + query.Where(query.Equal("namespace", d.Namespace)) + query.Where(query.GreaterEqualThan("time", d.From.Unix())) + query.GroupBy("subject", "is_error") + + sql, args := query.Build() + return sql, args +} + +// Insert Events Query +type InsertEventsQuery struct { + Database string + Events []CHEvent + QuerySettings map[string]string +} + +func (q InsertEventsQuery) ToSQL() (string, []interface{}) { + tableName := GetEventsTableName(q.Database) + + query := sqlbuilder.ClickHouse.NewInsertBuilder() + query.InsertInto(tableName) + query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") + + // Add settings + var settings []string + for key, value := range q.QuerySettings { + settings = append(settings, fmt.Sprintf("%s = %s", key, value)) + } + + if len(settings) > 0 { + query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) + } + + for _, event := range q.Events { + query.Values( + event.Namespace, + event.ValidationError, + event.ID, + event.Type, + event.Source, + event.Subject, + event.Time, + event.Data, + event.IngestedAt, + event.StoredAt, + ) + } + + sql, args := query.Build() + return sql, args +} + +func GetEventsTableName(database string) string { + return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) +} diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go new file mode 100644 index 000000000..4fd7ea0d5 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go @@ -0,0 +1,185 @@ +package clickhouse_connector_raw + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestCreateEventsTable(t *testing.T) { + tests := []struct { + data createEventsTable + want string + }{ + { + data: createEventsTable{ + Database: "openmeter", + }, + want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + got := tt.data.toSQL() + assert.Equal(t, tt.want, got) + }) + } +} + +func TestQueryEventsTable(t *testing.T) { + subjectFilter := "customer-1" + idFilter := "event-id-1" + hasErrorTrue := true + hasErrorFalse := false + + tests := []struct { + query queryEventsTable + wantSQL string + wantArgs []interface{} + }{ + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + Subject: &subjectFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", subjectFilter}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + ID: &idFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorTrue, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorFalse, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestQueryEventsCount(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + tests := []struct { + query queryCountEvents + wantSQL string + wantArgs []interface{} + }{ + { + query: queryCountEvents{ + Database: "openmeter", + Namespace: "my_namespace", + From: from, + }, + wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", + wantArgs: []interface{}{"my_namespace", from.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestInsertEventsQuery(t *testing.T) { + now := time.Now() + + query := InsertEventsQuery{ + Database: "database", + Events: []CHEvent{ + { + Namespace: "my_namespace", + ID: "1", + Source: "source", + Subject: "subject-1", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ID: "2", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ValidationError: "event data value cannot be parsed as float64: not a number", + ID: "3", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, + }, + }, + } + + sql, args := query.ToSQL() + + assert.Equal(t, []interface{}{ + "my_namespace", "", "1", "api-calls", "source", "subject-1", now, `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "", "2", "api-calls", "source", "subject-2", now, `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now, `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, + }, args) + assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) +} diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse_connector_raw/meter_query.go new file mode 100644 index 000000000..1b5c1fdba --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query.go @@ -0,0 +1,229 @@ +package clickhouse_connector_raw + +import ( + _ "embed" + "fmt" + "sort" + "time" + + "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/pkg/models" + "github.com/openmeterio/openmeter/pkg/slicesx" +) + +type queryMeter struct { + Database string + Namespace string + Meter models.Meter + Subject []string + FilterGroupBy map[string][]string + From *time.Time + To *time.Time + GroupBy []string + WindowSize *models.WindowSize + WindowTimeZone *time.Location +} + +func (d queryMeter) toSQL() (string, []interface{}, error) { + tableName := GetEventsTableName(d.Database) + getColumn := columnFactory(EventsTableName) + timeColumn := getColumn("time") + + var selectColumns, groupByColumns, where []string + + // Select windows if any + groupByWindowSize := d.WindowSize != nil + + tz := "UTC" + if d.WindowTimeZone != nil { + tz = d.WindowTimeZone.String() + } + + if groupByWindowSize { + switch *d.WindowSize { + case models.WindowSizeMinute: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalMinute(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalMinute(1), '%s') AS windowend", timeColumn, tz), + ) + + case models.WindowSizeHour: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalHour(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalHour(1), '%s') AS windowend", timeColumn, tz), + ) + + case models.WindowSizeDay: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalDay(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalDay(1), '%s') AS windowend", timeColumn, tz), + ) + + default: + return "", nil, fmt.Errorf("invalid window size type: %s", *d.WindowSize) + } + + groupByColumns = append(groupByColumns, "windowstart", "windowend") + } else { + selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") + } + + // Select Value + sqlAggregation := "" + switch d.Meter.Aggregation { + case models.MeterAggregationSum: + sqlAggregation = "sum" + case models.MeterAggregationAvg: + sqlAggregation = "avg" + case models.MeterAggregationMin: + sqlAggregation = "min" + case models.MeterAggregationMax: + sqlAggregation = "max" + case models.MeterAggregationUniqueCount: + sqlAggregation = "uniq" + case models.MeterAggregationCount: + sqlAggregation = "count" + default: + return "", []interface{}{}, fmt.Errorf("invalid aggregation type: %s", d.Meter.Aggregation) + } + + if d.Meter.ValueProperty == "" && d.Meter.Aggregation == models.MeterAggregationCount { + selectColumns = append(selectColumns, fmt.Sprintf("%s(*) AS value", sqlAggregation)) + } else if d.Meter.Aggregation == models.MeterAggregationUniqueCount { + selectColumns = append(selectColumns, fmt.Sprintf("%s(JSON_VALUE(%s, '%s')) AS value", sqlAggregation, getColumn("data"), sqlbuilder.Escape(d.Meter.ValueProperty))) + } else { + selectColumns = append(selectColumns, fmt.Sprintf("%s(cast(JSON_VALUE(%s, '%s'), 'Float64')) AS value", sqlAggregation, getColumn("data"), sqlbuilder.Escape(d.Meter.ValueProperty))) + } + + // Select Group By + sortedGroupBy := sortedKeys(d.Meter.GroupBy) + for _, groupByKey := range sortedGroupBy { + groupByColumn := sqlbuilder.Escape(groupByKey) + groupByJSONPath := sqlbuilder.Escape(d.Meter.GroupBy[groupByKey]) + selectColumn := fmt.Sprintf("JSON_VALUE(%s, '%s') as %s", getColumn("data"), groupByJSONPath, groupByColumn) + + // Subject is a special case + if groupByKey == "subject" { + selectColumn = getColumn("subject") + } + + selectColumns = append(selectColumns, selectColumn) + groupByColumns = append(groupByColumns, groupByColumn) + } + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select(selectColumns...) + query.From(tableName) + query.Where(query.Equal(getColumn("namespace"), d.Namespace)) + query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) + + if len(d.Subject) > 0 { + mapFunc := func(subject string) string { + return query.Equal(getColumn("subject"), subject) + } + + where = append(where, query.Or(slicesx.Map(d.Subject, mapFunc)...)) + } + + if len(d.FilterGroupBy) > 0 { + // We sort the group by s to ensure the query is deterministic + groupByKeys := make([]string, 0, len(d.FilterGroupBy)) + for k := range d.FilterGroupBy { + groupByKeys = append(groupByKeys, k) + } + sort.Strings(groupByKeys) + + for _, groupByKey := range groupByKeys { + groupByJSONPath := sqlbuilder.Escape(d.Meter.GroupBy[groupByKey]) + + values := d.FilterGroupBy[groupByKey] + if len(values) == 0 { + return "", nil, fmt.Errorf("empty filter for group by: %s", groupByKey) + } + mapFunc := func(value string) string { + column := fmt.Sprintf("JSON_VALUE(%s, '%s')", getColumn("data"), groupByJSONPath) + + // Subject is a special case + if groupByKey == "subject" { + column = "subject" + } + + return query.Equal(column, value) + } + + where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) + } + } + + if d.From != nil { + where = append(where, query.GreaterEqualThan(getColumn("time"), d.From.Unix())) + } + + if d.To != nil { + where = append(where, query.LessEqualThan(getColumn("time"), d.To.Unix())) + } + + if len(where) > 0 { + query.Where(where...) + } + + query.GroupBy(groupByColumns...) + + if groupByWindowSize { + query.OrderBy("windowstart") + } + + sql, args := query.Build() + return sql, args, nil +} + +type listMeterSubjectsQuery struct { + Database string + Namespace string + Meter models.Meter + From *time.Time + To *time.Time +} + +func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewSelectBuilder() + sb.Select("DISTINCT subject") + sb.Where(sb.Equal("namespace", d.Namespace)) + sb.Where(sb.Equal("type", d.Meter.EventType)) + sb.From(tableName) + sb.OrderBy("subject") + + if d.From != nil { + sb.Where(sb.GreaterEqualThan("time", d.From.Unix())) + } + + if d.To != nil { + sb.Where(sb.LessEqualThan("time", d.To.Unix())) + } + + sql, args := sb.Build() + return sql, args +} + +func columnFactory(alias string) func(string) string { + return func(column string) string { + return fmt.Sprintf("%s.%s", alias, column) + } +} + +func sortedKeys(m map[string]string) []string { + keys := make([]string, len(m)) + i := 0 + for k := range m { + keys[i] = k + i++ + } + sort.Strings(keys) + return keys +} diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go new file mode 100644 index 000000000..b2f3cde14 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go @@ -0,0 +1,283 @@ +package clickhouse_connector_raw + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/pkg/models" +) + +func TestQueryMeter(t *testing.T) { + subject := "subject1" + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + tz, _ := time.LoadLocation("Asia/Shanghai") + windowSize := models.WindowSizeHour + + tests := []struct { + query queryMeter + wantSQL string + wantArgs []interface{} + }{ + { + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + From: &from, + To: &to, + GroupBy: []string{"subject", "group1", "group2"}, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, + }, + { // Aggregate all available data + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate with count aggregation + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationCount, + }, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate data from start + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { // Aggregate data between period + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period in a different timezone, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + WindowSize: &windowSize, + WindowTimeZone: tz, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data for a single subject + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a single subject and group by additional fields + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject", "group1", "group2"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a multiple subjects + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject, "subject2"}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, + }, + { // Aggregate data with filtering for a single group and single value + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, + }, + { // Aggregate data with filtering for a single group and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, + }, + { // Aggregate data with filtering for multiple groups and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs, err := tt.query.toSQL() + if err != nil { + t.Error(err) + return + } + + assert.Equal(t, tt.wantSQL, gotSql) + assert.Equal(t, tt.wantArgs, gotArgs) + }) + } +} + +func TestListMeterSubjects(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + + tests := []struct { + query listMeterSubjectsQuery + wantSQL string + wantArgs []interface{} + }{ + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + EventType: "event1", + Aggregation: models.MeterAggregationSum, + }, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + EventType: "event1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + EventType: "event1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} diff --git a/openmeter/streaming/connector.go b/openmeter/streaming/connector.go index 5700c3716..fbbda903c 100644 --- a/openmeter/streaming/connector.go +++ b/openmeter/streaming/connector.go @@ -30,12 +30,17 @@ type CountEventRow struct { IsError bool } +type ListMeterSubjectsParams struct { + From *time.Time + To *time.Time +} + type Connector interface { CountEvents(ctx context.Context, namespace string, params CountEventsParams) ([]CountEventRow, error) ListEvents(ctx context.Context, namespace string, params ListEventsParams) ([]api.IngestedEvent, error) - CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error - DeleteMeter(ctx context.Context, namespace string, meterSlug string) error - QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *QueryParams) ([]models.MeterQueryRow, error) - ListMeterSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) + CreateMeter(ctx context.Context, namespace string, meter models.Meter) error + DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error + QueryMeter(ctx context.Context, namespace string, meter models.Meter, params QueryParams) ([]models.MeterQueryRow, error) + ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params ListMeterSubjectsParams) ([]string, error) // Add more methods as needed ... } diff --git a/openmeter/streaming/testutils/streaming.go b/openmeter/streaming/testutils/streaming.go index 1bae5e87d..089ebbf51 100644 --- a/openmeter/streaming/testutils/streaming.go +++ b/openmeter/streaming/testutils/streaming.go @@ -11,6 +11,8 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) +var _ streaming.Connector = &MockStreamingConnector{} + func NewMockStreamingConnector(t testing.TB) *MockStreamingConnector { t.Helper() return &MockStreamingConnector{ @@ -57,17 +59,17 @@ func (m *MockStreamingConnector) ListEvents(ctx context.Context, namespace strin return []api.IngestedEvent{}, nil } -func (m *MockStreamingConnector) CreateMeter(ctx context.Context, namespace string, meter *models.Meter) error { +func (m *MockStreamingConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { return nil } -func (m *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace string, meterSlug string) error { +func (m *MockStreamingConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { return nil } // Returns the result query set for the given params. If the query set is not found, // it will try to approximate the result by aggregating the simple events -func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { rows := []models.MeterQueryRow{} _, rowOk := m.rows[meter.Slug] @@ -93,7 +95,7 @@ func windowSizeToDuration(windowSize models.WindowSize) time.Duration { } // We approximate the actual logic by a simple filter + aggregation for most cases -func (m *MockStreamingConnector) aggregateEvents(meterSlug string, params *streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (m *MockStreamingConnector) aggregateEvents(meterSlug string, params streaming.QueryParams) ([]models.MeterQueryRow, error) { events, ok := m.events[meterSlug] from := defaultx.WithDefault(params.From, time.Now().AddDate(-10, 0, 0)) to := defaultx.WithDefault(params.To, time.Now()) @@ -154,6 +156,6 @@ func (m *MockStreamingConnector) aggregateEvents(meterSlug string, params *strea return rows, nil } -func (m *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { +func (m *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { return []string{}, nil } diff --git a/openmeter/streaming/testutils/streaming_test.go b/openmeter/streaming/testutils/streaming_test.go index 13b6bd1bd..669eccb54 100644 --- a/openmeter/streaming/testutils/streaming_test.go +++ b/openmeter/streaming/testutils/streaming_test.go @@ -24,7 +24,7 @@ func TestMockStreamingConnector(t *testing.T) { Name string Events []SimpleEvent Rows []models.MeterQueryRow - Query *streaming.QueryParams + Query streaming.QueryParams Expected []models.MeterQueryRow ExpectedError error } @@ -35,7 +35,7 @@ func TestMockStreamingConnector(t *testing.T) { tt := []tc{ { Name: "Should return error if meter not found", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Hour)), To: convert.ToPointer(now), }, @@ -43,7 +43,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should error if meter exists but doesnt match", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Hour)), To: convert.ToPointer(now), }, @@ -52,7 +52,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should return empty rows if no rows and no events", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Hour)), To: convert.ToPointer(now), }, @@ -68,7 +68,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should return exact row", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Hour)), To: convert.ToPointer(now), }, @@ -87,7 +87,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should return event sum", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Hour)), To: convert.ToPointer(now), }, @@ -104,7 +104,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should aggregate events as if they were windowed", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Truncate(time.Minute).Add(time.Second * 30).Add(-time.Minute * 2)), To: convert.ToPointer(now.Truncate(time.Minute).Add(time.Second * 30)), }, @@ -137,7 +137,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should return events windowed", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Minute * 3)), To: convert.ToPointer(now), WindowSize: convert.ToPointer(models.WindowSizeMinute), @@ -172,7 +172,7 @@ func TestMockStreamingConnector(t *testing.T) { }, { Name: "Should return row for queried period if window is larger than period", - Query: &streaming.QueryParams{ + Query: streaming.QueryParams{ From: convert.ToPointer(now.Add(-time.Minute * 3)), To: convert.ToPointer(now), WindowSize: convert.ToPointer(models.WindowSizeHour), From 6bc872c3c65c7149f239504362d5231a3e863dd4 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 17:39:20 -0700 Subject: [PATCH 07/33] feat(streaming): add back mv --- openmeter/sink/storage.go | 2 +- .../clickhouse_connector/connector.go | 167 +++++-- .../clickhouse_connector/meter_query.go | 299 ++++++++++--- .../clickhouse_connector/meter_query_test.go | 382 +++++++++------- .../clickhouse_connector_map/connector.go | 412 ++++++++++++++++++ .../clickhouse_connector_map/event_query.go | 180 ++++++++ .../event_query_test.go | 185 ++++++++ .../meter_event_query.go | 2 +- .../clickhouse_connector_map/meter_query.go | 202 +++++++++ .../meter_query_test.go | 271 ++++++++++++ .../model.go | 2 +- 11 files changed, 1843 insertions(+), 261 deletions(-) create mode 100644 openmeter/streaming/clickhouse_connector_map/connector.go create mode 100644 openmeter/streaming/clickhouse_connector_map/event_query.go create mode 100644 openmeter/streaming/clickhouse_connector_map/event_query_test.go rename openmeter/streaming/{clickhouse_connector => clickhouse_connector_map}/meter_event_query.go (98%) create mode 100644 openmeter/streaming/clickhouse_connector_map/meter_query.go create mode 100644 openmeter/streaming/clickhouse_connector_map/meter_query_test.go rename openmeter/streaming/{clickhouse_connector => clickhouse_connector_map}/model.go (82%) diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index d94112078..040708c98 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -8,7 +8,7 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" sinkmodels "github.com/openmeterio/openmeter/openmeter/sink/models" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_map" ) type Storage interface { diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index 8f5b5d24c..36d1cfe69 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -10,7 +10,6 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" "github.com/cloudevents/sdk-go/v2/event" - "github.com/shopspring/decimal" "github.com/openmeterio/openmeter/api" "github.com/openmeterio/openmeter/openmeter/meter" @@ -60,12 +59,35 @@ func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, } func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { - // Do nothing + if namespace == "" { + return fmt.Errorf("namespace is required") + } + + err := c.createMeterView(ctx, namespace, meter) + if err != nil { + return fmt.Errorf("init: %w", err) + } + return nil } func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { - // Do nothing + if namespace == "" { + return fmt.Errorf("namespace is required") + } + if meter.Slug == "" { + return fmt.Errorf("meter is required") + } + + err := c.deleteMeterView(ctx, namespace, meter) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + return err + } + + return fmt.Errorf("delete meter view: %w", err) + } + return nil } @@ -74,7 +96,7 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return nil, fmt.Errorf("namespace is required") } - values, err := c.queryMeter(ctx, namespace, meter, params) + values, err := c.queryMeterView(ctx, namespace, meter, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { return nil, err @@ -108,7 +130,7 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s return nil, fmt.Errorf("meter is required") } - subjects, err := c.listMeterViewSubjects(ctx, namespace, meter.Slug, params.From, params.To) + subjects, err := c.listMeterViewSubjects(ctx, namespace, meter, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { return nil, err @@ -126,16 +148,38 @@ func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace str return fmt.Errorf("create namespace in clickhouse: %w", err) } - err = c.createMeterEventTable(ctx) + return nil +} + +func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { + err := c.deleteNamespace(ctx, namespace) if err != nil { - return fmt.Errorf("create namespace in clickhouse: %w", err) + return fmt.Errorf("delete namespace in clickhouse: %w", err) } - return nil } -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { - // We don't delete the event tables as it it reused between namespaces +// DeleteNamespace deletes the namespace related resources from Clickhouse +// We don't delete the events table as it it reused between namespaces +// We only delete the materialized views for the meters +func (c *ClickhouseConnector) deleteNamespace(ctx context.Context, namespace string) error { + // Retrieve meters belonging to the namespace + meters, err := c.config.Meters.ListMeters(ctx, namespace) + if err != nil { + return fmt.Errorf("failed to list meters: %w", err) + } + + for _, meter := range meters { + err := c.deleteMeterView(ctx, namespace, meter) + if err != nil { + // If the meter view does not exist, we ignore the error + if _, ok := err.(*models.MeterNotFoundError); ok { + return nil + } + return fmt.Errorf("delete meter view: %w", err) + } + } + return nil } @@ -169,19 +213,6 @@ func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { return nil } -func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { - table := createMeterEventTable{ - Database: c.config.Database, - } - - err := c.config.ClickHouse.Exec(ctx, table.toSQL()) - if err != nil { - return fmt.Errorf("create meter event table: %w", err) - } - - return nil -} - func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { table := queryEventsTable{ Database: c.config.Database, @@ -292,11 +323,65 @@ func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace st return results, nil } -func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { - queryMeter := queryMeter{ +func (c *ClickhouseConnector) createMeterView(ctx context.Context, namespace string, meter models.Meter) error { + // CreateOrReplace is used to force the recreation of the materialized view + // This is not safe to use in production as it will drop the existing views + if c.config.CreateOrReplaceMeter { + err := c.deleteMeterView(ctx, namespace, meter) + if err != nil { + return fmt.Errorf("drop meter view: %w", err) + } + } + + view := createMeterView{ + Populate: c.config.PopulateMeter, + Database: c.config.Database, + Namespace: namespace, + MeterSlug: meter.Slug, + Aggregation: meter.Aggregation, + EventType: meter.EventType, + ValueProperty: meter.ValueProperty, + GroupBy: meter.GroupBy, + } + sql, args, err := view.toSQL() + if err != nil { + return fmt.Errorf("create meter view: %w", err) + } + err = c.config.ClickHouse.Exec(ctx, sql, args...) + if err != nil { + return fmt.Errorf("create meter view: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) deleteMeterView(ctx context.Context, namespace string, meter models.Meter) error { + query := deleteMeterView{ + Database: c.config.Database, + Namespace: namespace, + MeterSlug: meter.Slug, + } + + sql := query.toSQL() + + err := c.config.ClickHouse.Exec(ctx, sql) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return &models.MeterNotFoundError{MeterSlug: meter.Slug} + } + + return fmt.Errorf("delete meter view: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { + queryMeter := queryMeterView{ Database: c.config.Database, Namespace: namespace, - Meter: meter, + MeterSlug: meter.Slug, + Aggregation: meter.Aggregation, From: params.From, To: params.To, Subject: params.FilterSubject, @@ -326,12 +411,11 @@ func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, slog.Debug("query meter view", "elapsed", elapsed.String(), "sql", sql, "args", args) for rows.Next() { - row := models.MeterQueryRow{ + value := models.MeterQueryRow{ GroupBy: map[string]*string{}, } - var value decimal.Decimal - args := []interface{}{&row.WindowStart, &row.WindowEnd, &value} + args := []interface{}{&value.WindowStart, &value.WindowEnd, &value.Value} argCount := len(args) for range queryMeter.GroupBy { @@ -343,31 +427,28 @@ func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, return values, fmt.Errorf("query meter view row scan: %w", err) } - // TODO: should we use decima all the way? - row.Value, _ = value.Float64() - for i, key := range queryMeter.GroupBy { if s, ok := args[i+argCount].(*string); ok { if key == "subject" { - row.Subject = s + value.Subject = s continue } // We treat empty string as nil if s != nil && *s == "" { - row.GroupBy[key] = nil + value.GroupBy[key] = nil } else { - row.GroupBy[key] = s + value.GroupBy[key] = s } } } // an empty row is returned when there are no values for the meter - if row.WindowStart.IsZero() && row.WindowEnd.IsZero() && row.Value == 0 { + if value.WindowStart.IsZero() && value.WindowEnd.IsZero() && value.Value == 0 { continue } - values = append(values, row) + values = append(values, value) } rows.Close() err = rows.Err() @@ -378,13 +459,13 @@ func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, return values, nil } -func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { - query := listMeterSubjectsQuery{ +func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { + query := listMeterViewSubjects{ Database: c.config.Database, Namespace: namespace, - MeterSlug: meterSlug, - From: from, - To: to, + MeterSlug: meter.Slug, + From: params.From, + To: params.To, } sql, args := query.toSQL() @@ -392,7 +473,7 @@ func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespa rows, err := c.config.ClickHouse.Query(ctx, sql, args...) if err != nil { if strings.Contains(err.Error(), "code: 60") { - return nil, &models.MeterNotFoundError{MeterSlug: meterSlug} + return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} } return nil, fmt.Errorf("list meter view subjects: %w", err) diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index be846bf94..343d2d9d2 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -4,6 +4,7 @@ import ( _ "embed" "fmt" "sort" + "strings" "time" "github.com/huandu/go-sqlbuilder" @@ -12,10 +13,168 @@ import ( "github.com/openmeterio/openmeter/pkg/slicesx" ) -type queryMeter struct { +type column struct { + Name string + Type string +} + +type createMeterView struct { + Database string + Aggregation models.MeterAggregation + Namespace string + MeterSlug string + EventType string + ValueProperty string + GroupBy map[string]string + // Populate creates the materialized view with data from the events table + // This is not safe to use in production as requires to stop ingestion + Populate bool +} + +func (d createMeterView) toSQL() (string, []interface{}, error) { + viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) + columns := []column{ + {Name: "subject", Type: "String"}, + {Name: "windowstart", Type: "DateTime"}, + {Name: "windowend", Type: "DateTime"}, + } + + // Value + agg := "" + + switch d.Aggregation { + case models.MeterAggregationSum: + agg = "sum" + case models.MeterAggregationAvg: + agg = "avg" + case models.MeterAggregationMin: + agg = "min" + case models.MeterAggregationMax: + agg = "max" + case models.MeterAggregationCount: + agg = "count" + case models.MeterAggregationUniqueCount: + agg = "uniq" + default: + return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) + } + + if d.Aggregation == models.MeterAggregationUniqueCount { + columns = append(columns, column{Name: "value", Type: fmt.Sprintf("AggregateFunction(%s, String)", agg)}) + } else { + columns = append(columns, column{Name: "value", Type: fmt.Sprintf("AggregateFunction(%s, Float64)", agg)}) + } + + // Group by + orderBy := []string{"windowstart", "windowend", "subject"} + sortedGroupBy := sortedKeys(d.GroupBy) + for _, k := range sortedGroupBy { + columnName := sqlbuilder.Escape(k) + orderBy = append(orderBy, sqlbuilder.Escape(columnName)) + columns = append(columns, column{Name: columnName, Type: "String"}) + } + + sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() + sb.CreateTable(viewName) + sb.IfNotExists() + for _, column := range columns { + sb.Define(column.Name, column.Type) + } + sb.SQL("ENGINE = AggregatingMergeTree()") + sb.SQL(fmt.Sprintf("ORDER BY (%s)", strings.Join(orderBy, ", "))) + if d.Populate { + sb.SQL("POPULATE") + } + sb.SQL("AS") + + selectQuery, err := d.toSelectSQL() + if err != nil { + return "", nil, err + } + + sb.SQL(selectQuery) + sql, args := sb.Build() + + // TODO: can we do it differently? + sql = strings.Replace(sql, "CREATE TABLE", "CREATE MATERIALIZED VIEW", 1) + + return sql, args, nil +} + +func (d createMeterView) toSelectSQL() (string, error) { + eventsTableName := GetEventsTableName(d.Database) + + aggStateFn := "" + switch d.Aggregation { + case models.MeterAggregationSum: + aggStateFn = "sumState" + case models.MeterAggregationAvg: + aggStateFn = "avgState" + case models.MeterAggregationMin: + aggStateFn = "minState" + case models.MeterAggregationMax: + aggStateFn = "maxState" + case models.MeterAggregationUniqueCount: + aggStateFn = "uniqState" + case models.MeterAggregationCount: + aggStateFn = "countState" + default: + return "", fmt.Errorf("invalid aggregation type: %s", d.Aggregation) + } + + // Selects + selects := []string{ + "subject", + "tumbleStart(time, toIntervalMinute(1)) AS windowstart", + "tumbleEnd(time, toIntervalMinute(1)) AS windowend", + } + if d.ValueProperty == "" && d.Aggregation == models.MeterAggregationCount { + selects = append(selects, fmt.Sprintf("%s(*) AS value", aggStateFn)) + } else if d.Aggregation == models.MeterAggregationUniqueCount { + selects = append(selects, fmt.Sprintf("%s(JSON_VALUE(data, '%s')) AS value", aggStateFn, sqlbuilder.Escape(d.ValueProperty))) + } else { + selects = append(selects, fmt.Sprintf("%s(cast(JSON_VALUE(data, '%s'), 'Float64')) AS value", aggStateFn, sqlbuilder.Escape(d.ValueProperty))) + } + + // Group by + orderBy := []string{"windowstart", "windowend", "subject"} + sortedGroupBy := sortedKeys(d.GroupBy) + for _, k := range sortedGroupBy { + v := d.GroupBy[k] + columnName := sqlbuilder.Escape(k) + orderBy = append(orderBy, sqlbuilder.Escape(columnName)) + selects = append(selects, fmt.Sprintf("JSON_VALUE(data, '%s') as %s", sqlbuilder.Escape(v), sqlbuilder.Escape(k))) + } + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select(selects...) + query.From(eventsTableName) + // We use absolute paths to avoid shadowing in the case the materialized view have a `namespace` or `type` group by + query.Where(fmt.Sprintf("%s.namespace = '%s'", eventsTableName, sqlbuilder.Escape(d.Namespace))) + query.Where(fmt.Sprintf("empty(%s.validation_error) = 1", eventsTableName)) + query.Where(fmt.Sprintf("%s.type = '%s'", eventsTableName, sqlbuilder.Escape(d.EventType))) + query.GroupBy(orderBy...) + + return query.String(), nil +} + +type deleteMeterView struct { + Database string + Namespace string + MeterSlug string +} + +func (d deleteMeterView) toSQL() string { + viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) + + return fmt.Sprintf("DROP VIEW %s", viewName) +} + +type queryMeterView struct { Database string Namespace string - Meter models.Meter + MeterSlug string + Aggregation models.MeterAggregation Subject []string FilterGroupBy map[string][]string From *time.Time @@ -25,10 +184,10 @@ type queryMeter struct { WindowTimeZone *time.Location } -func (d queryMeter) toSQL() (string, []interface{}, error) { - tableName := GetMeterEventsTableName(d.Database) - getColumn := columnFactory(MeterEventTableName) - timeColumn := getColumn("time") +func (d queryMeterView) toSQL() (string, []interface{}, error) { + viewAlias := "meter" + viewName := fmt.Sprintf("%s %s", GetMeterViewName(d.Database, d.Namespace, d.MeterSlug), viewAlias) + getColumn := columnFactory(viewAlias) var selectColumns, groupByColumns, where []string @@ -44,22 +203,22 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { case models.WindowSizeMinute: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalMinute(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalMinute(1), '%s') AS windowend", timeColumn, tz), + fmt.Sprintf("tumbleStart(windowstart, toIntervalMinute(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(windowstart, toIntervalMinute(1), '%s') AS windowend", tz), ) case models.WindowSizeHour: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalHour(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalHour(1), '%s') AS windowend", timeColumn, tz), + fmt.Sprintf("tumbleStart(windowstart, toIntervalHour(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(windowstart, toIntervalHour(1), '%s') AS windowend", tz), ) case models.WindowSizeDay: selectColumns = append( selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalDay(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalDay(1), '%s') AS windowend", timeColumn, tz), + fmt.Sprintf("tumbleStart(windowstart, toIntervalDay(1), '%s') AS windowstart", tz), + fmt.Sprintf("tumbleEnd(windowstart, toIntervalDay(1), '%s') AS windowend", tz), ) default: @@ -71,101 +230,96 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") } - switch d.Meter.Aggregation { + switch d.Aggregation { case models.MeterAggregationSum: - selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) + selectColumns = append(selectColumns, "sumMerge(value) AS value") case models.MeterAggregationAvg: - selectColumns = append(selectColumns, fmt.Sprintf("avg(%s) AS value", getColumn("value"))) + selectColumns = append(selectColumns, "avgMerge(value) AS value") case models.MeterAggregationMin: - selectColumns = append(selectColumns, fmt.Sprintf("min(%s) AS value", getColumn("value"))) + selectColumns = append(selectColumns, "minMerge(value) AS value") case models.MeterAggregationMax: - selectColumns = append(selectColumns, fmt.Sprintf("max(%s) AS value", getColumn("value"))) + selectColumns = append(selectColumns, "maxMerge(value) AS value") case models.MeterAggregationUniqueCount: - selectColumns = append(selectColumns, fmt.Sprintf("toDecimal(uniq(%s)) AS value", getColumn("value_str"))) + selectColumns = append(selectColumns, "toFloat64(uniqMerge(value)) AS value") case models.MeterAggregationCount: - selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) + selectColumns = append(selectColumns, "toFloat64(countMerge(value)) AS value") default: - return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Meter.Aggregation) + return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Aggregation) } - for _, groupByKey := range d.GroupBy { - c := sqlbuilder.Escape(groupByKey) - selectColumn := fmt.Sprintf("%s['%s'] as %s", getColumn("group_by"), c, c) - - // Subject is a special case - if groupByKey == "subject" { - selectColumn = getColumn("subject") - } - - selectColumns = append(selectColumns, selectColumn) + for _, column := range d.GroupBy { + c := sqlbuilder.Escape(column) + selectColumns = append(selectColumns, c) groupByColumns = append(groupByColumns, c) } - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select(selectColumns...) - query.From(tableName) - query.Where(query.Equal(getColumn("namespace"), d.Namespace)) - query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) + queryView := sqlbuilder.ClickHouse.NewSelectBuilder() + queryView.Select(selectColumns...) + queryView.From(viewName) if len(d.Subject) > 0 { mapFunc := func(subject string) string { - return query.Equal(getColumn("subject"), subject) + return queryView.Equal(getColumn("subject"), subject) } - where = append(where, query.Or(slicesx.Map(d.Subject, mapFunc)...)) + where = append(where, queryView.Or(slicesx.Map(d.Subject, mapFunc)...)) } if len(d.FilterGroupBy) > 0 { - // We sort the group by s to ensure the query is deterministic - groupByKeys := make([]string, 0, len(d.FilterGroupBy)) + // We sort the columns to ensure the query is deterministic + columns := make([]string, 0, len(d.FilterGroupBy)) for k := range d.FilterGroupBy { - groupByKeys = append(groupByKeys, k) + columns = append(columns, k) } - sort.Strings(groupByKeys) + sort.Strings(columns) - for _, groupByKey := range groupByKeys { - values := d.FilterGroupBy[groupByKey] + for _, column := range columns { + values := d.FilterGroupBy[column] if len(values) == 0 { - return "", nil, fmt.Errorf("empty filter for group by: %s", groupByKey) + return "", nil, fmt.Errorf("empty filter for group by: %s", column) } mapFunc := func(value string) string { - column := sqlbuilder.Escape(fmt.Sprintf("%s['%s']", getColumn("group_by"), groupByKey)) - - // Subject is a special case - if groupByKey == "subject" { - column = "subject" - } - - return query.Equal(column, value) + return queryView.Equal(sqlbuilder.Escape(getColumn(column)), value) } - where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) + where = append(where, queryView.Or(slicesx.Map(values, mapFunc)...)) } } if d.From != nil { - where = append(where, query.GreaterEqualThan(getColumn("time"), d.From.Unix())) + where = append(where, queryView.GreaterEqualThan(getColumn("windowstart"), d.From.Unix())) } if d.To != nil { - where = append(where, query.LessEqualThan(getColumn("time"), d.To.Unix())) + where = append(where, queryView.LessEqualThan(getColumn("windowend"), d.To.Unix())) } if len(where) > 0 { - query.Where(where...) + queryView.Where(where...) } - query.GroupBy(groupByColumns...) + queryView.GroupBy(groupByColumns...) if groupByWindowSize { - query.OrderBy("windowstart") + queryView.OrderBy("windowstart") } - sql, args := query.Build() + sql, args := queryView.Build() return sql, args, nil } -type listMeterSubjectsQuery struct { +func sortedKeys(m map[string]string) []string { + keys := make([]string, len(m)) + i := 0 + for k := range m { + keys[i] = k + i++ + } + sort.Strings(keys) + return keys +} + +type listMeterViewSubjects struct { Database string Namespace string MeterSlug string @@ -173,28 +327,37 @@ type listMeterSubjectsQuery struct { To *time.Time } -func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { - tableName := GetMeterEventsTableName(d.Database) +func (d listMeterViewSubjects) toSQL() (string, []interface{}) { + viewName := GetMeterViewName(d.Database, d.Namespace, d.MeterSlug) + var where []string sb := sqlbuilder.ClickHouse.NewSelectBuilder() sb.Select("DISTINCT subject") - sb.Where(sb.Equal("namespace", d.Namespace)) - sb.Where(sb.Equal("meter", d.MeterSlug)) - sb.From(tableName) - sb.OrderBy("subject") + sb.From(viewName) if d.From != nil { - sb.Where(sb.GreaterEqualThan("time", d.From.Unix())) + where = append(where, sb.GreaterEqualThan("windowstart", d.From.Unix())) } if d.To != nil { - sb.Where(sb.LessEqualThan("time", d.To.Unix())) + where = append(where, sb.LessEqualThan("windowend", d.To.Unix())) + } + + if len(where) > 0 { + sb.Where(where...) } + sb.OrderBy("subject") + sql, args := sb.Build() return sql, args } +func GetMeterViewName(database string, namespace string, meterSlug string) string { + meterViewName := fmt.Sprintf("%s_%s", namespace, meterSlug) + return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), sqlbuilder.Escape(meterViewName)) +} + func columnFactory(alias string) func(string) string { return func(column string) string { return fmt.Sprintf("%s.%s", alias, column) diff --git a/openmeter/streaming/clickhouse_connector/meter_query_test.go b/openmeter/streaming/clickhouse_connector/meter_query_test.go index bb27758fa..9a3bee96c 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector/meter_query_test.go @@ -9,7 +9,121 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) -func TestQueryMeter(t *testing.T) { +func TestCreateMeterView(t *testing.T) { + tests := []struct { + query createMeterView + wantSQL string + wantArgs []interface{} + }{ + { + query: createMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + EventType: "myevent", + ValueProperty: "$.duration_ms", + GroupBy: map[string]string{"group1": "$.group1", "group2": "$.group2"}, + }, + wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(sum, Float64), group1 String, group2 String) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject, group1, group2) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, sumState(cast(JSON_VALUE(data, '$.duration_ms'), 'Float64')) AS value, JSON_VALUE(data, '$.group1') as group1, JSON_VALUE(data, '$.group2') as group2 FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject, group1, group2", + wantArgs: nil, + }, + { + query: createMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationAvg, + EventType: "myevent", + ValueProperty: "$.token_count", + GroupBy: map[string]string{}, + }, + wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(avg, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, avgState(cast(JSON_VALUE(data, '$.token_count'), 'Float64')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", + wantArgs: nil, + }, + { + query: createMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, + EventType: "myevent", + ValueProperty: "", + GroupBy: map[string]string{}, + }, + wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", + wantArgs: nil, + }, + { + query: createMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, + EventType: "myevent", + ValueProperty: "", + GroupBy: map[string]string{}, + }, + wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", + wantArgs: nil, + }, + { + query: createMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationUniqueCount, + EventType: "myevent", + ValueProperty: "$.trace_id", + GroupBy: map[string]string{}, + }, + wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(uniq, String)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, uniqState(JSON_VALUE(data, '$.trace_id')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", + wantArgs: nil, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs, err := tt.query.toSQL() + if err != nil { + t.Error(err) + return + } + + assert.Equal(t, tt.wantSQL, gotSql) + assert.Equal(t, tt.wantArgs, gotArgs) + }) + } +} + +func TestDeleteMeterView(t *testing.T) { + tests := []struct { + data deleteMeterView + wantSQL string + wantArgs []interface{} + }{ + { + data: deleteMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + }, + wantSQL: "DROP VIEW openmeter.om_my_namespace_meter1", + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql := tt.data.toSQL() + + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestQueryMeterView(t *testing.T) { subject := "subject1" from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") @@ -17,189 +131,163 @@ func TestQueryMeter(t *testing.T) { windowSize := models.WindowSizeHour tests := []struct { - query queryMeter + query queryMeterView wantSQL string wantArgs []interface{} }{ { - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - From: &from, - To: &to, - GroupBy: []string{"subject", "group1", "group2"}, - WindowSize: &windowSize, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + From: &from, + To: &to, + GroupBy: []string{"subject", "group1", "group2"}, + WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'UTC') AS windowend, sumMerge(value) AS value, subject, group1, group2 FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) AND meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantArgs: []interface{}{"subject1", from.Unix(), to.Unix()}, }, { // Aggregate all available data - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter", + wantArgs: nil, }, { // Aggregate with count aggregation - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationCount, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT min(windowstart), max(windowend), toFloat64(countMerge(value)) AS value FROM openmeter.om_my_namespace_meter1 meter", + wantArgs: nil, }, { // Aggregate data from start - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ?", + wantArgs: []interface{}{from.Unix()}, }, { // Aggregate data between period - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - To: &to, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + To: &to, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ?", + wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - To: &to, - WindowSize: &windowSize, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + From: &from, + To: &to, + WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'UTC') AS windowend, sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data between period in a different timezone, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, From: &from, To: &to, WindowSize: &windowSize, WindowTimeZone: tz, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(windowstart, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(windowstart, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE meter.windowstart >= ? AND meter.windowend <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{from.Unix(), to.Unix()}, }, { // Aggregate data for a single subject - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - GroupBy: []string{"subject"}, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"subject1"}, }, { // Aggregate data for a single subject and group by additional fields - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - GroupBy: []string{"subject", "group1", "group2"}, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject}, + GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject, group1, group2 FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ?) GROUP BY subject, group1, group2", + wantArgs: []interface{}{"subject1"}, }, { // Aggregate data for a multiple subjects - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject, "subject2"}, - GroupBy: []string{"subject"}, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + Subject: []string{subject, "subject2"}, + GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value, subject FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.subject = ? OR meter.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ?)", + wantArgs: []interface{}{"g1v1"}, }, { // Aggregate data with filtering for a single group and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ? OR meter.g1 = ?)", + wantArgs: []interface{}{"g1v1", "g1v2"}, }, { // Aggregate data with filtering for multiple groups and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, + query: queryMeterView{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, + wantSQL: "SELECT min(windowstart), max(windowend), sumMerge(value) AS value FROM openmeter.om_my_namespace_meter1 meter WHERE (meter.g1 = ? OR meter.g1 = ?) AND (meter.g2 = ? OR meter.g2 = ?)", + wantArgs: []interface{}{"g1v1", "g1v2", "g2v1", "g2v2"}, }, } @@ -218,44 +306,44 @@ func TestQueryMeter(t *testing.T) { } } -func TestListMeterSubjects(t *testing.T) { +func TestListMeterViewSubjects(t *testing.T) { from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") tests := []struct { - query listMeterSubjectsQuery + query listMeterViewSubjects wantSQL string wantArgs []interface{} }{ { - query: listMeterSubjectsQuery{ + query: listMeterViewSubjects{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 ORDER BY subject", + wantArgs: nil, }, { - query: listMeterSubjectsQuery{ + query: listMeterViewSubjects{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", From: &from, }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 WHERE windowstart >= ? ORDER BY subject", + wantArgs: []interface{}{from.Unix()}, }, { - query: listMeterSubjectsQuery{ + query: listMeterViewSubjects{ Database: "openmeter", Namespace: "my_namespace", MeterSlug: "meter1", From: &from, To: &to, }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_my_namespace_meter1 WHERE windowstart >= ? AND windowend <= ? ORDER BY subject", + wantArgs: []interface{}{from.Unix(), to.Unix()}, }, } diff --git a/openmeter/streaming/clickhouse_connector_map/connector.go b/openmeter/streaming/clickhouse_connector_map/connector.go new file mode 100644 index 000000000..9940d510c --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_map/connector.go @@ -0,0 +1,412 @@ +package clickhouse_connector_map + +import ( + "context" + "encoding/json" + "fmt" + "log/slog" + "strings" + "time" + + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/cloudevents/sdk-go/v2/event" + "github.com/shopspring/decimal" + + "github.com/openmeterio/openmeter/api" + "github.com/openmeterio/openmeter/openmeter/meter" + "github.com/openmeterio/openmeter/openmeter/streaming" + "github.com/openmeterio/openmeter/pkg/models" +) + +var _ streaming.Connector = (*ClickhouseConnector)(nil) + +// ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. +type ClickhouseConnector struct { + config ClickhouseConnectorConfig +} + +type ClickhouseConnectorConfig struct { + Logger *slog.Logger + ClickHouse clickhouse.Conn + Database string + Meters meter.Repository + CreateOrReplaceMeter bool + PopulateMeter bool +} + +func NewClickhouseConnector(config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { + connector := &ClickhouseConnector{ + config: config, + } + + return connector, nil +} + +func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + events, err := c.queryEventsTable(ctx, namespace, params) + if err != nil { + if _, ok := err.(*models.NamespaceNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("query events: %w", err) + } + + return events, nil +} + +func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { + // Do nothing + return nil +} + +func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { + // Do nothing + return nil +} + +func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + values, err := c.queryMeter(ctx, namespace, meter, params) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("get values: %w", err) + } + + // If the total usage is queried for a single period (no window size), + // replace the window start and end with the period for each row. + // We can still have multiple rows for a single period due to group bys. + if params.WindowSize == nil { + for i := range values { + if params.From != nil { + values[i].WindowStart = *params.From + } + if params.To != nil { + values[i].WindowEnd = *params.To + } + } + } + + return values, nil +} + +func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + if meter.Slug == "" { + return nil, fmt.Errorf("meter is required") + } + + subjects, err := c.listMeterViewSubjects(ctx, namespace, meter.Slug, params.From, params.To) + if err != nil { + if _, ok := err.(*models.MeterNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("list meter subjects: %w", err) + } + + return subjects, nil +} + +func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { + err := c.createEventsTable(ctx) + if err != nil { + return fmt.Errorf("create namespace in clickhouse: %w", err) + } + + err = c.createMeterEventTable(ctx) + if err != nil { + return fmt.Errorf("create namespace in clickhouse: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { + // We don't delete the event tables as it it reused between namespaces + return nil +} + +func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + if namespace == "" { + return nil, fmt.Errorf("namespace is required") + } + + rows, err := c.queryCountEvents(ctx, namespace, params) + if err != nil { + if _, ok := err.(*models.NamespaceNotFoundError); ok { + return nil, err + } + + return nil, fmt.Errorf("query count events: %w", err) + } + + return rows, nil +} + +func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { + table := createEventsTable{ + Database: c.config.Database, + } + + err := c.config.ClickHouse.Exec(ctx, table.toSQL()) + if err != nil { + return fmt.Errorf("create events table: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { + table := createMeterEventTable{ + Database: c.config.Database, + } + + err := c.config.ClickHouse.Exec(ctx, table.toSQL()) + if err != nil { + return fmt.Errorf("create meter event table: %w", err) + } + + return nil +} + +func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + table := queryEventsTable{ + Database: c.config.Database, + Namespace: namespace, + From: params.From, + To: params.To, + IngestedAtFrom: params.IngestedAtFrom, + IngestedAtTo: params.IngestedAtTo, + ID: params.ID, + Subject: params.Subject, + HasError: params.HasError, + Limit: params.Limit, + } + + sql, args := table.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.NamespaceNotFoundError{Namespace: namespace} + } + + return nil, fmt.Errorf("query events table query: %w", err) + } + + events := []api.IngestedEvent{} + + for rows.Next() { + var id string + var eventType string + var subject string + var source string + var eventTime time.Time + var dataStr string + var validationError string + var ingestedAt time.Time + var storedAt time.Time + + if err = rows.Scan(&id, &eventType, &subject, &source, &eventTime, &dataStr, &validationError, &ingestedAt, &storedAt); err != nil { + return nil, err + } + + // Parse data + var data interface{} + err := json.Unmarshal([]byte(dataStr), &data) + if err != nil { + return nil, fmt.Errorf("query events parse data: %w", err) + } + + event := event.New() + event.SetID(id) + event.SetType(eventType) + event.SetSubject(subject) + event.SetSource(source) + event.SetTime(eventTime) + err = event.SetData("application/json", data) + if err != nil { + return nil, fmt.Errorf("query events set data: %w", err) + } + + ingestedEvent := api.IngestedEvent{ + Event: event, + } + + if validationError != "" { + ingestedEvent.ValidationError = &validationError + } + + ingestedEvent.IngestedAt = ingestedAt + ingestedEvent.StoredAt = storedAt + + events = append(events, ingestedEvent) + } + + return events, nil +} + +func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + table := queryCountEvents{ + Database: c.config.Database, + Namespace: namespace, + From: params.From, + } + + sql, args := table.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.NamespaceNotFoundError{Namespace: namespace} + } + + return nil, fmt.Errorf("query events count query: %w", err) + } + + results := []streaming.CountEventRow{} + + for rows.Next() { + result := streaming.CountEventRow{} + + if err = rows.Scan(&result.Count, &result.Subject, &result.IsError); err != nil { + return nil, err + } + + results = append(results, result) + } + + return results, nil +} + +func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { + queryMeter := queryMeter{ + Database: c.config.Database, + Namespace: namespace, + Meter: meter, + From: params.From, + To: params.To, + Subject: params.FilterSubject, + FilterGroupBy: params.FilterGroupBy, + GroupBy: params.GroupBy, + WindowSize: params.WindowSize, + WindowTimeZone: params.WindowTimeZone, + } + + values := []models.MeterQueryRow{} + + sql, args, err := queryMeter.toSQL() + if err != nil { + return values, fmt.Errorf("query meter view: %w", err) + } + + start := time.Now() + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} + } + + return values, fmt.Errorf("query meter view query: %w", err) + } + elapsed := time.Since(start) + slog.Debug("query meter view", "elapsed", elapsed.String(), "sql", sql, "args", args) + + for rows.Next() { + row := models.MeterQueryRow{ + GroupBy: map[string]*string{}, + } + + var value decimal.Decimal + args := []interface{}{&row.WindowStart, &row.WindowEnd, &value} + argCount := len(args) + + for range queryMeter.GroupBy { + tmp := "" + args = append(args, &tmp) + } + + if err := rows.Scan(args...); err != nil { + return values, fmt.Errorf("query meter view row scan: %w", err) + } + + // TODO: should we use decima all the way? + row.Value, _ = value.Float64() + + for i, key := range queryMeter.GroupBy { + if s, ok := args[i+argCount].(*string); ok { + if key == "subject" { + row.Subject = s + continue + } + + // We treat empty string as nil + if s != nil && *s == "" { + row.GroupBy[key] = nil + } else { + row.GroupBy[key] = s + } + } + } + + // an empty row is returned when there are no values for the meter + if row.WindowStart.IsZero() && row.WindowEnd.IsZero() && row.Value == 0 { + continue + } + + values = append(values, row) + } + rows.Close() + err = rows.Err() + if err != nil { + return values, fmt.Errorf("query meter rows error: %w", err) + } + + return values, nil +} + +func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { + query := listMeterSubjectsQuery{ + Database: c.config.Database, + Namespace: namespace, + MeterSlug: meterSlug, + From: from, + To: to, + } + + sql, args := query.toSQL() + + rows, err := c.config.ClickHouse.Query(ctx, sql, args...) + if err != nil { + if strings.Contains(err.Error(), "code: 60") { + return nil, &models.MeterNotFoundError{MeterSlug: meterSlug} + } + + return nil, fmt.Errorf("list meter view subjects: %w", err) + } + + subjects := []string{} + for rows.Next() { + var subject string + if err = rows.Scan(&subject); err != nil { + return nil, err + } + + subjects = append(subjects, subject) + } + + return subjects, nil +} diff --git a/openmeter/streaming/clickhouse_connector_map/event_query.go b/openmeter/streaming/clickhouse_connector_map/event_query.go new file mode 100644 index 000000000..9ac1155b8 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_map/event_query.go @@ -0,0 +1,180 @@ +package clickhouse_connector_map + +import ( + _ "embed" + "fmt" + "strings" + "time" + + "github.com/huandu/go-sqlbuilder" +) + +const EventsTableName = "om_events" + +// CHEvent represents a raw event in ClickHouse +type CHEvent struct { + Namespace string `ch:"namespace"` + ValidationError string `ch:"validation_error"` + ID string `ch:"id"` + Type string `ch:"type"` + Source string `ch:"source"` + Subject string `ch:"subject"` + Time time.Time `ch:"time"` + Data string `ch:"data"` + IngestedAt time.Time `ch:"ingested_at"` + StoredAt time.Time `ch:"stored_at"` +} + +// Create Events Table +type createEventsTable struct { + Database string +} + +func (d createEventsTable) toSQL() string { + tableName := GetEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() + sb.CreateTable(tableName) + sb.IfNotExists() + sb.Define("namespace", "String") + sb.Define("validation_error", "String") + sb.Define("id", "String") + sb.Define("type", "LowCardinality(String)") + sb.Define("subject", "String") + sb.Define("source", "String") + sb.Define("time", "DateTime") + sb.Define("data", "String") + sb.Define("ingested_at", "DateTime") + sb.Define("stored_at", "DateTime") + sb.SQL("ENGINE = MergeTree") + sb.SQL("PARTITION BY toYYYYMM(time)") + sb.SQL("ORDER BY (namespace, time, type, subject)") + + sql, _ := sb.Build() + return sql +} + +// Query Events Table +type queryEventsTable struct { + Database string + Namespace string + From *time.Time + To *time.Time + IngestedAtFrom *time.Time + IngestedAtTo *time.Time + ID *string + Subject *string + HasError *bool + Limit int +} + +func (d queryEventsTable) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + where := []string{} + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") + query.From(tableName) + + where = append(where, query.Equal("namespace", d.Namespace)) + if d.From != nil { + where = append(where, query.GreaterEqualThan("time", d.From.Unix())) + } + if d.To != nil { + where = append(where, query.LessEqualThan("time", d.To.Unix())) + } + if d.IngestedAtFrom != nil { + where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) + } + if d.IngestedAtTo != nil { + where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) + } + if d.ID != nil { + where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) + } + if d.Subject != nil { + where = append(where, query.Equal("subject", *d.Subject)) + } + if d.HasError != nil { + if *d.HasError { + where = append(where, "notEmpty(validation_error) = 1") + } else { + where = append(where, "empty(validation_error) = 1") + } + } + query.Where(where...) + + query.Desc().OrderBy("time") + query.Limit(d.Limit) + + sql, args := query.Build() + return sql, args +} + +type queryCountEvents struct { + Database string + Namespace string + From time.Time +} + +func (d queryCountEvents) toSQL() (string, []interface{}) { + tableName := GetEventsTableName(d.Database) + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") + query.From(tableName) + + query.Where(query.Equal("namespace", d.Namespace)) + query.Where(query.GreaterEqualThan("time", d.From.Unix())) + query.GroupBy("subject", "is_error") + + sql, args := query.Build() + return sql, args +} + +// Insert Events Query +type InsertEventsQuery struct { + Database string + Events []CHEvent + QuerySettings map[string]string +} + +func (q InsertEventsQuery) ToSQL() (string, []interface{}) { + tableName := GetEventsTableName(q.Database) + + query := sqlbuilder.ClickHouse.NewInsertBuilder() + query.InsertInto(tableName) + query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") + + // Add settings + var settings []string + for key, value := range q.QuerySettings { + settings = append(settings, fmt.Sprintf("%s = %s", key, value)) + } + + if len(settings) > 0 { + query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) + } + + for _, event := range q.Events { + query.Values( + event.Namespace, + event.ValidationError, + event.ID, + event.Type, + event.Source, + event.Subject, + event.Time, + event.Data, + event.IngestedAt, + event.StoredAt, + ) + } + + sql, args := query.Build() + return sql, args +} + +func GetEventsTableName(database string) string { + return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) +} diff --git a/openmeter/streaming/clickhouse_connector_map/event_query_test.go b/openmeter/streaming/clickhouse_connector_map/event_query_test.go new file mode 100644 index 000000000..726cec9a3 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_map/event_query_test.go @@ -0,0 +1,185 @@ +package clickhouse_connector_map + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestCreateEventsTable(t *testing.T) { + tests := []struct { + data createEventsTable + want string + }{ + { + data: createEventsTable{ + Database: "openmeter", + }, + want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + got := tt.data.toSQL() + assert.Equal(t, tt.want, got) + }) + } +} + +func TestQueryEventsTable(t *testing.T) { + subjectFilter := "customer-1" + idFilter := "event-id-1" + hasErrorTrue := true + hasErrorFalse := false + + tests := []struct { + query queryEventsTable + wantSQL string + wantArgs []interface{} + }{ + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + Subject: &subjectFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", subjectFilter}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + ID: &idFilter, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorTrue, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + { + query: queryEventsTable{ + Database: "openmeter", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorFalse, + }, + wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", + wantArgs: []interface{}{"my_namespace"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestQueryEventsCount(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + tests := []struct { + query queryCountEvents + wantSQL string + wantArgs []interface{} + }{ + { + query: queryCountEvents{ + Database: "openmeter", + Namespace: "my_namespace", + From: from, + }, + wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", + wantArgs: []interface{}{"my_namespace", from.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} + +func TestInsertEventsQuery(t *testing.T) { + now := time.Now() + + query := InsertEventsQuery{ + Database: "database", + Events: []CHEvent{ + { + Namespace: "my_namespace", + ID: "1", + Source: "source", + Subject: "subject-1", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ID: "2", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, + }, + { + Namespace: "my_namespace", + ValidationError: "event data value cannot be parsed as float64: not a number", + ID: "3", + Source: "source", + Subject: "subject-2", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, + }, + }, + } + + sql, args := query.ToSQL() + + assert.Equal(t, []interface{}{ + "my_namespace", "", "1", "api-calls", "source", "subject-1", now, `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "", "2", "api-calls", "source", "subject-2", now, `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, + "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now, `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, + }, args) + assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) +} diff --git a/openmeter/streaming/clickhouse_connector/meter_event_query.go b/openmeter/streaming/clickhouse_connector_map/meter_event_query.go similarity index 98% rename from openmeter/streaming/clickhouse_connector/meter_event_query.go rename to openmeter/streaming/clickhouse_connector_map/meter_event_query.go index dffc38790..f6051e722 100644 --- a/openmeter/streaming/clickhouse_connector/meter_event_query.go +++ b/openmeter/streaming/clickhouse_connector_map/meter_event_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector +package clickhouse_connector_map import ( _ "embed" diff --git a/openmeter/streaming/clickhouse_connector_map/meter_query.go b/openmeter/streaming/clickhouse_connector_map/meter_query.go new file mode 100644 index 000000000..625fb3db2 --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_map/meter_query.go @@ -0,0 +1,202 @@ +package clickhouse_connector_map + +import ( + _ "embed" + "fmt" + "sort" + "time" + + "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/pkg/models" + "github.com/openmeterio/openmeter/pkg/slicesx" +) + +type queryMeter struct { + Database string + Namespace string + Meter models.Meter + Subject []string + FilterGroupBy map[string][]string + From *time.Time + To *time.Time + GroupBy []string + WindowSize *models.WindowSize + WindowTimeZone *time.Location +} + +func (d queryMeter) toSQL() (string, []interface{}, error) { + tableName := GetMeterEventsTableName(d.Database) + getColumn := columnFactory(MeterEventTableName) + timeColumn := getColumn("time") + + var selectColumns, groupByColumns, where []string + + groupByWindowSize := d.WindowSize != nil + + tz := "UTC" + if d.WindowTimeZone != nil { + tz = d.WindowTimeZone.String() + } + + if groupByWindowSize { + switch *d.WindowSize { + case models.WindowSizeMinute: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalMinute(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalMinute(1), '%s') AS windowend", timeColumn, tz), + ) + + case models.WindowSizeHour: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalHour(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalHour(1), '%s') AS windowend", timeColumn, tz), + ) + + case models.WindowSizeDay: + selectColumns = append( + selectColumns, + fmt.Sprintf("tumbleStart(%s, toIntervalDay(1), '%s') AS windowstart", timeColumn, tz), + fmt.Sprintf("tumbleEnd(%s, toIntervalDay(1), '%s') AS windowend", timeColumn, tz), + ) + + default: + return "", nil, fmt.Errorf("invalid window size type: %s", *d.WindowSize) + } + + groupByColumns = append(groupByColumns, "windowstart", "windowend") + } else { + selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") + } + + switch d.Meter.Aggregation { + case models.MeterAggregationSum: + selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) + case models.MeterAggregationAvg: + selectColumns = append(selectColumns, fmt.Sprintf("avg(%s) AS value", getColumn("value"))) + case models.MeterAggregationMin: + selectColumns = append(selectColumns, fmt.Sprintf("min(%s) AS value", getColumn("value"))) + case models.MeterAggregationMax: + selectColumns = append(selectColumns, fmt.Sprintf("max(%s) AS value", getColumn("value"))) + case models.MeterAggregationUniqueCount: + selectColumns = append(selectColumns, fmt.Sprintf("toDecimal(uniq(%s)) AS value", getColumn("value_str"))) + case models.MeterAggregationCount: + selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) + default: + return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Meter.Aggregation) + } + + for _, groupByKey := range d.GroupBy { + c := sqlbuilder.Escape(groupByKey) + selectColumn := fmt.Sprintf("%s['%s'] as %s", getColumn("group_by"), c, c) + + // Subject is a special case + if groupByKey == "subject" { + selectColumn = getColumn("subject") + } + + selectColumns = append(selectColumns, selectColumn) + groupByColumns = append(groupByColumns, c) + } + + query := sqlbuilder.ClickHouse.NewSelectBuilder() + query.Select(selectColumns...) + query.From(tableName) + query.Where(query.Equal(getColumn("namespace"), d.Namespace)) + query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) + + if len(d.Subject) > 0 { + mapFunc := func(subject string) string { + return query.Equal(getColumn("subject"), subject) + } + + where = append(where, query.Or(slicesx.Map(d.Subject, mapFunc)...)) + } + + if len(d.FilterGroupBy) > 0 { + // We sort the group by s to ensure the query is deterministic + groupByKeys := make([]string, 0, len(d.FilterGroupBy)) + for k := range d.FilterGroupBy { + groupByKeys = append(groupByKeys, k) + } + sort.Strings(groupByKeys) + + for _, groupByKey := range groupByKeys { + values := d.FilterGroupBy[groupByKey] + if len(values) == 0 { + return "", nil, fmt.Errorf("empty filter for group by: %s", groupByKey) + } + mapFunc := func(value string) string { + column := sqlbuilder.Escape(fmt.Sprintf("%s['%s']", getColumn("group_by"), groupByKey)) + + // Subject is a special case + if groupByKey == "subject" { + column = "subject" + } + + return query.Equal(column, value) + } + + where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) + } + } + + if d.From != nil { + where = append(where, query.GreaterEqualThan(getColumn("time"), d.From.Unix())) + } + + if d.To != nil { + where = append(where, query.LessEqualThan(getColumn("time"), d.To.Unix())) + } + + if len(where) > 0 { + query.Where(where...) + } + + query.GroupBy(groupByColumns...) + + if groupByWindowSize { + query.OrderBy("windowstart") + } + + sql, args := query.Build() + return sql, args, nil +} + +type listMeterSubjectsQuery struct { + Database string + Namespace string + MeterSlug string + From *time.Time + To *time.Time +} + +func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { + tableName := GetMeterEventsTableName(d.Database) + + sb := sqlbuilder.ClickHouse.NewSelectBuilder() + sb.Select("DISTINCT subject") + sb.Where(sb.Equal("namespace", d.Namespace)) + sb.Where(sb.Equal("meter", d.MeterSlug)) + sb.From(tableName) + sb.OrderBy("subject") + + if d.From != nil { + sb.Where(sb.GreaterEqualThan("time", d.From.Unix())) + } + + if d.To != nil { + sb.Where(sb.LessEqualThan("time", d.To.Unix())) + } + + sql, args := sb.Build() + return sql, args +} + +func columnFactory(alias string) func(string) string { + return func(column string) string { + return fmt.Sprintf("%s.%s", alias, column) + } +} diff --git a/openmeter/streaming/clickhouse_connector_map/meter_query_test.go b/openmeter/streaming/clickhouse_connector_map/meter_query_test.go new file mode 100644 index 000000000..421a3e2ec --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_map/meter_query_test.go @@ -0,0 +1,271 @@ +package clickhouse_connector_map + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/pkg/models" +) + +func TestQueryMeter(t *testing.T) { + subject := "subject1" + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + tz, _ := time.LoadLocation("Asia/Shanghai") + windowSize := models.WindowSizeHour + + tests := []struct { + query queryMeter + wantSQL string + wantArgs []interface{} + }{ + { + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + From: &from, + To: &to, + GroupBy: []string{"subject", "group1", "group2"}, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, + }, + { // Aggregate all available data + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate with count aggregation + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationCount, + }, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { // Aggregate data from start + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { // Aggregate data between period + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + WindowSize: &windowSize, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data between period in a different timezone, groupped by window size + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + From: &from, + To: &to, + WindowSize: &windowSize, + WindowTimeZone: tz, + }, + wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + { // Aggregate data for a single subject + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a single subject and group by additional fields + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject}, + GroupBy: []string{"subject", "group1", "group2"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + }, + { // Aggregate data for a multiple subjects + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + Subject: []string{subject, "subject2"}, + GroupBy: []string{"subject"}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, + }, + { // Aggregate data with filtering for a single group and single value + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, + }, + { // Aggregate data with filtering for a single group and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, + }, + { // Aggregate data with filtering for multiple groups and multiple values + query: queryMeter{ + Database: "openmeter", + Namespace: "my_namespace", + Meter: models.Meter{ + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + }, + FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, + }, + wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", + wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs, err := tt.query.toSQL() + if err != nil { + t.Error(err) + return + } + + assert.Equal(t, tt.wantSQL, gotSql) + assert.Equal(t, tt.wantArgs, gotArgs) + }) + } +} + +func TestListMeterSubjects(t *testing.T) { + from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") + to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") + + tests := []struct { + query listMeterSubjectsQuery + wantSQL string + wantArgs []interface{} + }{ + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1"}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + From: &from, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + }, + { + query: listMeterSubjectsQuery{ + Database: "openmeter", + Namespace: "my_namespace", + MeterSlug: "meter1", + From: &from, + To: &to, + }, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + }, + } + + for _, tt := range tests { + tt := tt + t.Run("", func(t *testing.T) { + gotSql, gotArgs := tt.query.toSQL() + + assert.Equal(t, tt.wantArgs, gotArgs) + assert.Equal(t, tt.wantSQL, gotSql) + }) + } +} diff --git a/openmeter/streaming/clickhouse_connector/model.go b/openmeter/streaming/clickhouse_connector_map/model.go similarity index 82% rename from openmeter/streaming/clickhouse_connector/model.go rename to openmeter/streaming/clickhouse_connector_map/model.go index 915b5def3..11d9c3dd8 100644 --- a/openmeter/streaming/clickhouse_connector/model.go +++ b/openmeter/streaming/clickhouse_connector_map/model.go @@ -1,4 +1,4 @@ -package clickhouse_connector +package clickhouse_connector_map import "github.com/openmeterio/openmeter/pkg/models" From ec443fad723def840d5f21c377af8e1290174e5c Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 18:49:07 -0700 Subject: [PATCH 08/33] feat(connector): isnert --- app/common/openmeter.go | 3 +- cmd/balance-worker/wire_gen.go | 2 +- cmd/jobs/entitlement/init.go | 2 +- cmd/notification-service/wire_gen.go | 2 +- cmd/server/wire_gen.go | 2 +- cmd/sink-worker/main.go | 32 ++++-- openmeter/server/server_test.go | 8 ++ openmeter/sink/storage.go | 80 ++----------- .../clickhouse_connector/connector.go | 68 ++++++++++- .../clickhouse_connector/event_query.go | 18 +-- .../clickhouse_connector/event_query_test.go | 4 +- .../connector.go | 106 ++++++++++++++---- .../event_query.go | 20 +--- .../event_query_test.go | 6 +- .../meter_event_query.go | 34 ++---- .../meter_query.go | 2 +- .../meter_query_test.go | 2 +- .../model.go | 2 +- .../clickhouse_connector_raw/connector.go | 74 +++++++++--- .../clickhouse_connector_raw/event_query.go | 18 +-- .../event_query_test.go | 4 +- openmeter/streaming/connector.go | 24 ++++ openmeter/streaming/testutils/streaming.go | 8 ++ 23 files changed, 318 insertions(+), 203 deletions(-) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/connector.go (80%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/event_query.go (88%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/event_query_test.go (97%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/meter_event_query.go (75%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/meter_query.go (99%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/meter_query_test.go (99%) rename openmeter/streaming/{clickhouse_connector_map => clickhouse_connector_parse}/model.go (82%) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 6b05467eb..39790d1bf 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -35,12 +35,13 @@ func NewMeterRepository(meters []*models.Meter) *meter.InMemoryRepository { } func NewClickHouseStreamingConnector( + ctx context.Context, conf config.AggregationConfiguration, clickHouse clickhouse.Conn, meterRepository meter.Repository, logger *slog.Logger, ) (*clickhouse_connector.ClickhouseConnector, error) { - streamingConnector, err := clickhouse_connector.NewClickhouseConnector(clickhouse_connector.ClickhouseConnectorConfig{ + streamingConnector, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, Meters: meterRepository, diff --git a/cmd/balance-worker/wire_gen.go b/cmd/balance-worker/wire_gen.go index c572f76e3..99b49dfb5 100644 --- a/cmd/balance-worker/wire_gen.go +++ b/cmd/balance-worker/wire_gen.go @@ -134,7 +134,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v3 := conf.Meters inMemoryRepository := common.NewMeterRepository(v3) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(aggregationConfiguration, v2, inMemoryRepository, logger) + clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v2, inMemoryRepository, logger) if err != nil { cleanup5() cleanup4() diff --git a/cmd/jobs/entitlement/init.go b/cmd/jobs/entitlement/init.go index 89c2ee12e..fb119dc6c 100644 --- a/cmd/jobs/entitlement/init.go +++ b/cmd/jobs/entitlement/init.go @@ -50,7 +50,7 @@ func initEntitlements(ctx context.Context, conf config.Configuration, logger *sl return nil, fmt.Errorf("failed to initialize clickhouse client: %w", err) } - streamingConnector, err := clickhouse_connector.NewClickhouseConnector(clickhouse_connector.ClickhouseConnectorConfig{ + streamingConnector, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ Logger: logger, ClickHouse: clickHouseClient, Database: conf.Aggregation.ClickHouse.Database, diff --git a/cmd/notification-service/wire_gen.go b/cmd/notification-service/wire_gen.go index d96175fee..8185b92ab 100644 --- a/cmd/notification-service/wire_gen.go +++ b/cmd/notification-service/wire_gen.go @@ -74,7 +74,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(aggregationConfiguration, v, inMemoryRepository, logger) + clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() diff --git a/cmd/server/wire_gen.go b/cmd/server/wire_gen.go index cc4c32719..83fd3776d 100644 --- a/cmd/server/wire_gen.go +++ b/cmd/server/wire_gen.go @@ -78,7 +78,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(aggregationConfiguration, v, inMemoryRepository, logger) + clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() diff --git a/cmd/sink-worker/main.go b/cmd/sink-worker/main.go index 792f1d27a..f4bfae910 100644 --- a/cmd/sink-worker/main.go +++ b/cmd/sink-worker/main.go @@ -23,6 +23,7 @@ import ( "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/sink" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" + clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" pkgkafka "github.com/openmeterio/openmeter/pkg/kafka" ) @@ -96,7 +97,7 @@ func main() { var group run.Group // Initialize sink worker - sink, err := initSink(conf, logger, app.Meter, app.Tracer, app.MeterRepository, app.FlushHandler) + sink, err := initSink(ctx, conf, logger, app.Meter, app.Tracer, app.MeterRepository, app.FlushHandler) if err != nil { logger.Error("failed to initialize sink worker", "error", err) os.Exit(1) @@ -135,12 +136,27 @@ func main() { } } -func initSink(config config.Configuration, logger *slog.Logger, metricMeter metric.Meter, tracer trace.Tracer, meterRepository meter.Repository, flushHandler flushhandler.FlushEventHandler) (*sink.Sink, error) { +func initSink(ctx context.Context, config config.Configuration, logger *slog.Logger, metricMeter metric.Meter, tracer trace.Tracer, meterRepository meter.Repository, flushHandler flushhandler.FlushEventHandler) (*sink.Sink, error) { + // Initialize ClickHouse client clickhouseClient, err := clickhouse.Open(config.Aggregation.ClickHouse.GetClientOptions()) if err != nil { return nil, fmt.Errorf("init clickhouse client: %w", err) } + // Initialize streaming connector + streaming, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ + Logger: logger, + ClickHouse: clickhouseClient, + Database: config.Aggregation.ClickHouse.Database, + AsyncInsert: config.Sink.Storage.AsyncInsert, + AsyncInsertWait: config.Sink.Storage.AsyncInsertWait, + InsertQuerySettings: config.Sink.Storage.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse streaming connector: %w", err) + } + + // Initialize deduplicator if enabled var deduplicator dedupe.Deduplicator if config.Sink.Dedupe.Enabled { deduplicator, err = config.Sink.Dedupe.NewDeduplicator() @@ -150,15 +166,9 @@ func initSink(config config.Configuration, logger *slog.Logger, metricMeter metr } // Initialize storage - storage, err := sink.NewClickhouseStorage( - sink.ClickHouseStorageConfig{ - ClickHouse: clickhouseClient, - Database: config.Aggregation.ClickHouse.Database, - AsyncInsert: config.Sink.Storage.AsyncInsert, - AsyncInsertWait: config.Sink.Storage.AsyncInsertWait, - QuerySettings: config.Sink.Storage.QuerySettings, - }, - ) + storage, err := sink.NewClickhouseStorage(sink.ClickHouseStorageConfig{ + Streaming: streaming, + }) if err != nil { return nil, fmt.Errorf("failed to initialize storage: %w", err) } diff --git a/openmeter/server/server_test.go b/openmeter/server/server_test.go index 4777d9afa..a520d55eb 100644 --- a/openmeter/server/server_test.go +++ b/openmeter/server/server_test.go @@ -75,6 +75,10 @@ var ( type MockStreamingConnector struct{} +func (c *MockStreamingConnector) Init(ctx context.Context) error { + return nil +} + func (c *MockStreamingConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { return []streaming.CountEventRow{}, nil } @@ -110,6 +114,10 @@ func (c *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespac return []string{"s1"}, nil } +func (c *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + return nil +} + type MockDebugHandler struct{} func (h MockDebugHandler) GetDebugMetrics(ctx context.Context, namespace string) (string, error) { diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index 040708c98..41626e37a 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -5,10 +5,8 @@ import ( "fmt" "time" - "github.com/ClickHouse/clickhouse-go/v2" - sinkmodels "github.com/openmeterio/openmeter/openmeter/sink/models" - clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_map" + "github.com/openmeterio/openmeter/openmeter/streaming" ) type Storage interface { @@ -16,20 +14,12 @@ type Storage interface { } type ClickHouseStorageConfig struct { - ClickHouse clickhouse.Conn - Database string - AsyncInsert bool - AsyncInsertWait bool - QuerySettings map[string]string + Streaming streaming.Connector } func (c ClickHouseStorageConfig) Validate() error { - if c.ClickHouse == nil { - return fmt.Errorf("clickhouse connection is required") - } - - if c.Database == "" { - return fmt.Errorf("database is required") + if c.Streaming == nil { + return fmt.Errorf("streaming connection is required") } return nil @@ -51,8 +41,8 @@ type ClickHouseStorage struct { // BatchInsert inserts multiple messages into ClickHouse. func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmodels.SinkMessage) error { - var rawEvents []clickhouse_connector.CHEvent - var meterEvents []clickhouse_connector.CHMeterEvent + var rawEvents []streaming.RawEvent + var meterEvents []streaming.MeterEvent for _, message := range messages { var eventErr string @@ -77,7 +67,7 @@ func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmode } } - rawEvent := clickhouse_connector.CHEvent{ + rawEvent := streaming.RawEvent{ Namespace: message.Namespace, ValidationError: eventErr, ID: message.Serialized.Id, @@ -94,68 +84,20 @@ func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmode // Meter events per meter for _, meterEvent := range message.MeterEvents { - meterEvent := clickhouse_connector.CHMeterEvent{ - Namespace: message.Namespace, - Time: rawEvent.Time, + meterEvent := streaming.MeterEvent{ + RawEvent: rawEvent, Meter: meterEvent.Meter.GetID(), - Subject: rawEvent.Subject, Value: meterEvent.Value, ValueString: meterEvent.ValueString, GroupBy: meterEvent.GroupBy, - EventID: rawEvent.ID, - EventSource: rawEvent.Source, - EventType: rawEvent.Type, - StoredAt: rawEvent.StoredAt, - IngestedAt: rawEvent.IngestedAt, } meterEvents = append(meterEvents, meterEvent) } } - var err error - - // Insert raw events - query := clickhouse_connector.InsertEventsQuery{ - Database: c.config.Database, - Events: rawEvents, - QuerySettings: c.config.QuerySettings, - } - sql, args := query.ToSQL() - - // By default, ClickHouse is writing data synchronously. - // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts - if c.config.AsyncInsert { - // With the `wait_for_async_insert` setting, you can configure - // if you want an insert statement to return with an acknowledgment - // either immediately after the data got inserted into the buffer. - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert raw events: %w", err) - } - - // Insert meter events - if len(meterEvents) > 0 { - query := clickhouse_connector.InsertMeterEventsQuery{ - Database: c.config.Database, - MeterEvents: meterEvents, - QuerySettings: c.config.QuerySettings, - } - sql, args := query.ToSQL() - - if c.config.AsyncInsert { - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert meter events: %w", err) - } + if err := c.config.Streaming.BatchInsert(ctx, rawEvents, meterEvents); err != nil { + return fmt.Errorf("failed to store events: %w", err) } return nil diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index 36d1cfe69..facf7e9a6 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -31,13 +31,45 @@ type ClickhouseConnectorConfig struct { Meters meter.Repository CreateOrReplaceMeter bool PopulateMeter bool + AsyncInsert bool + AsyncInsertWait bool + InsertQuerySettings map[string]string } -func NewClickhouseConnector(config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { +func (c ClickhouseConnectorConfig) Validate() error { + if c.Logger == nil { + return fmt.Errorf("logger is required") + } + + if c.ClickHouse == nil { + return fmt.Errorf("clickhouse connection is required") + } + + if c.Database == "" { + return fmt.Errorf("database is required") + } + + if c.Meters == nil { + return fmt.Errorf("meters repository is required") + } + + return nil +} + +func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { + if err := config.Validate(); err != nil { + return nil, fmt.Errorf("validate config: %w", err) + } + connector := &ClickhouseConnector{ config: config, } + err := connector.createEventsTable(ctx) + if err != nil { + return nil, fmt.Errorf("create events table in clickhouse: %w", err) + } + return connector, nil } @@ -143,11 +175,6 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s } func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - err := c.createEventsTable(ctx) - if err != nil { - return fmt.Errorf("create namespace in clickhouse: %w", err) - } - return nil } @@ -200,6 +227,35 @@ func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, return rows, nil } +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + var err error + + // Insert raw events + query := InsertEventsQuery{ + Database: c.config.Database, + Events: rawEvents, + QuerySettings: c.config.InsertQuerySettings, + } + sql, args := query.ToSQL() + + // By default, ClickHouse is writing data synchronously. + // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts + if c.config.AsyncInsert { + // With the `wait_for_async_insert` setting, you can configure + // if you want an insert statement to return with an acknowledgment + // either immediately after the data got inserted into the buffer. + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert raw events: %w", err) + } + + return nil +} + func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { table := createEventsTable{ Database: c.config.Database, diff --git a/openmeter/streaming/clickhouse_connector/event_query.go b/openmeter/streaming/clickhouse_connector/event_query.go index 7dcf9aa0f..4e0e562fc 100644 --- a/openmeter/streaming/clickhouse_connector/event_query.go +++ b/openmeter/streaming/clickhouse_connector/event_query.go @@ -7,24 +7,12 @@ import ( "time" "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) const EventsTableName = "om_events" -// CHEvent represents a raw event in ClickHouse -type CHEvent struct { - Namespace string `ch:"namespace"` - ValidationError string `ch:"validation_error"` - ID string `ch:"id"` - Type string `ch:"type"` - Source string `ch:"source"` - Subject string `ch:"subject"` - Time time.Time `ch:"time"` - Data string `ch:"data"` - IngestedAt time.Time `ch:"ingested_at"` - StoredAt time.Time `ch:"stored_at"` -} - // Create Events Table type createEventsTable struct { Database string @@ -135,7 +123,7 @@ func (d queryCountEvents) toSQL() (string, []interface{}) { // Insert Events Query type InsertEventsQuery struct { Database string - Events []CHEvent + Events []streaming.RawEvent QuerySettings map[string]string } diff --git a/openmeter/streaming/clickhouse_connector/event_query_test.go b/openmeter/streaming/clickhouse_connector/event_query_test.go index ec4e70e3c..24b28d6e4 100644 --- a/openmeter/streaming/clickhouse_connector/event_query_test.go +++ b/openmeter/streaming/clickhouse_connector/event_query_test.go @@ -5,6 +5,8 @@ import ( "time" "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) func TestCreateEventsTable(t *testing.T) { @@ -136,7 +138,7 @@ func TestInsertEventsQuery(t *testing.T) { query := InsertEventsQuery{ Database: "database", - Events: []CHEvent{ + Events: []streaming.RawEvent{ { Namespace: "my_namespace", ID: "1", diff --git a/openmeter/streaming/clickhouse_connector_map/connector.go b/openmeter/streaming/clickhouse_connector_parse/connector.go similarity index 80% rename from openmeter/streaming/clickhouse_connector_map/connector.go rename to openmeter/streaming/clickhouse_connector_parse/connector.go index 9940d510c..a642a8f5e 100644 --- a/openmeter/streaming/clickhouse_connector_map/connector.go +++ b/openmeter/streaming/clickhouse_connector_parse/connector.go @@ -1,4 +1,4 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( "context" @@ -13,7 +13,6 @@ import ( "github.com/shopspring/decimal" "github.com/openmeterio/openmeter/api" - "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/pkg/models" ) @@ -26,19 +25,49 @@ type ClickhouseConnector struct { } type ClickhouseConnectorConfig struct { - Logger *slog.Logger - ClickHouse clickhouse.Conn - Database string - Meters meter.Repository - CreateOrReplaceMeter bool - PopulateMeter bool + Logger *slog.Logger + ClickHouse clickhouse.Conn + Database string + AsyncInsert bool + AsyncInsertWait bool + InsertQuerySettings map[string]string } -func NewClickhouseConnector(config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { +func (c ClickhouseConnectorConfig) Validate() error { + if c.Logger == nil { + return fmt.Errorf("logger is required") + } + + if c.ClickHouse == nil { + return fmt.Errorf("clickhouse connection is required") + } + + if c.Database == "" { + return fmt.Errorf("database is required") + } + + return nil +} + +func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { + if err := config.Validate(); err != nil { + return nil, fmt.Errorf("validate config: %w", err) + } + connector := &ClickhouseConnector{ config: config, } + err := connector.createEventsTable(ctx) + if err != nil { + return nil, fmt.Errorf("create events table in clickhouse: %w", err) + } + + err = connector.createMeterEventTable(ctx) + if err != nil { + return nil, fmt.Errorf("create meter events table in clickhouse: %w", err) + } + return connector, nil } @@ -121,16 +150,6 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s } func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - err := c.createEventsTable(ctx) - if err != nil { - return fmt.Errorf("create namespace in clickhouse: %w", err) - } - - err = c.createMeterEventTable(ctx) - if err != nil { - return fmt.Errorf("create namespace in clickhouse: %w", err) - } - return nil } @@ -156,6 +175,55 @@ func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, return rows, nil } +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + var err error + + // Insert raw events + query := InsertEventsQuery{ + Database: c.config.Database, + Events: rawEvents, + QuerySettings: c.config.InsertQuerySettings, + } + sql, args := query.ToSQL() + + // By default, ClickHouse is writing data synchronously. + // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts + if c.config.AsyncInsert { + // With the `wait_for_async_insert` setting, you can configure + // if you want an insert statement to return with an acknowledgment + // either immediately after the data got inserted into the buffer. + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert raw events: %w", err) + } + + // Insert meter events + if len(meterEvents) > 0 { + query := InsertMeterEventsQuery{ + Database: c.config.Database, + MeterEvents: meterEvents, + QuerySettings: c.config.InsertQuerySettings, + } + sql, args := query.ToSQL() + + if c.config.AsyncInsert { + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert meter events: %w", err) + } + } + + return nil +} + func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { table := createEventsTable{ Database: c.config.Database, diff --git a/openmeter/streaming/clickhouse_connector_map/event_query.go b/openmeter/streaming/clickhouse_connector_parse/event_query.go similarity index 88% rename from openmeter/streaming/clickhouse_connector_map/event_query.go rename to openmeter/streaming/clickhouse_connector_parse/event_query.go index 9ac1155b8..50e4264b1 100644 --- a/openmeter/streaming/clickhouse_connector_map/event_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/event_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( _ "embed" @@ -7,24 +7,12 @@ import ( "time" "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) const EventsTableName = "om_events" -// CHEvent represents a raw event in ClickHouse -type CHEvent struct { - Namespace string `ch:"namespace"` - ValidationError string `ch:"validation_error"` - ID string `ch:"id"` - Type string `ch:"type"` - Source string `ch:"source"` - Subject string `ch:"subject"` - Time time.Time `ch:"time"` - Data string `ch:"data"` - IngestedAt time.Time `ch:"ingested_at"` - StoredAt time.Time `ch:"stored_at"` -} - // Create Events Table type createEventsTable struct { Database string @@ -135,7 +123,7 @@ func (d queryCountEvents) toSQL() (string, []interface{}) { // Insert Events Query type InsertEventsQuery struct { Database string - Events []CHEvent + Events []streaming.RawEvent QuerySettings map[string]string } diff --git a/openmeter/streaming/clickhouse_connector_map/event_query_test.go b/openmeter/streaming/clickhouse_connector_parse/event_query_test.go similarity index 97% rename from openmeter/streaming/clickhouse_connector_map/event_query_test.go rename to openmeter/streaming/clickhouse_connector_parse/event_query_test.go index 726cec9a3..3f75f431e 100644 --- a/openmeter/streaming/clickhouse_connector_map/event_query_test.go +++ b/openmeter/streaming/clickhouse_connector_parse/event_query_test.go @@ -1,10 +1,12 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( "testing" "time" "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) func TestCreateEventsTable(t *testing.T) { @@ -136,7 +138,7 @@ func TestInsertEventsQuery(t *testing.T) { query := InsertEventsQuery{ Database: "database", - Events: []CHEvent{ + Events: []streaming.RawEvent{ { Namespace: "my_namespace", ID: "1", diff --git a/openmeter/streaming/clickhouse_connector_map/meter_event_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go similarity index 75% rename from openmeter/streaming/clickhouse_connector_map/meter_event_query.go rename to openmeter/streaming/clickhouse_connector_parse/meter_event_query.go index f6051e722..7bb5f5d83 100644 --- a/openmeter/streaming/clickhouse_connector_map/meter_event_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go @@ -1,39 +1,19 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( _ "embed" "fmt" "strings" - "time" "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) const ( MeterEventTableName = "om_meter_events" ) -// Meter Event represents a single meter event in ClickHouse -type CHMeterEvent struct { - // Identifiers - Namespace string `ch:"namespace"` - Time time.Time `ch:"time"` - Meter string `ch:"meter"` - Subject string `ch:"subject"` - - // Usage - Value float64 `ch:"value"` - ValueString string `ch:"value_str"` - GroupBy map[string]string `ch:"group_by"` - - // Metadata - EventID string `ch:"event_id"` - EventSource string `ch:"event_source"` - EventType string `ch:"event_type"` - IngestedAt time.Time `ch:"ingested_at"` - StoredAt time.Time `ch:"stored_at"` -} - // Create Meter Event Table type createMeterEventTable struct { Database string @@ -75,7 +55,7 @@ func (d createMeterEventTable) toSQL() string { // Insert Meter Events Query type InsertMeterEventsQuery struct { Database string - MeterEvents []CHMeterEvent + MeterEvents []streaming.MeterEvent QuerySettings map[string]string } @@ -118,9 +98,9 @@ func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { meterEvent.Value, meterEvent.ValueString, meterEvent.GroupBy, - meterEvent.EventID, - meterEvent.EventSource, - meterEvent.EventType, + meterEvent.RawEvent.ID, + meterEvent.RawEvent.Source, + meterEvent.RawEvent.Type, meterEvent.IngestedAt, meterEvent.StoredAt, ) diff --git a/openmeter/streaming/clickhouse_connector_map/meter_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_query.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_map/meter_query.go rename to openmeter/streaming/clickhouse_connector_parse/meter_query.go index 625fb3db2..194041a3b 100644 --- a/openmeter/streaming/clickhouse_connector_map/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( _ "embed" diff --git a/openmeter/streaming/clickhouse_connector_map/meter_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_map/meter_query_test.go rename to openmeter/streaming/clickhouse_connector_parse/meter_query_test.go index 421a3e2ec..db491ea29 100644 --- a/openmeter/streaming/clickhouse_connector_map/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go @@ -1,4 +1,4 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import ( "testing" diff --git a/openmeter/streaming/clickhouse_connector_map/model.go b/openmeter/streaming/clickhouse_connector_parse/model.go similarity index 82% rename from openmeter/streaming/clickhouse_connector_map/model.go rename to openmeter/streaming/clickhouse_connector_parse/model.go index 11d9c3dd8..308838dfc 100644 --- a/openmeter/streaming/clickhouse_connector_map/model.go +++ b/openmeter/streaming/clickhouse_connector_parse/model.go @@ -1,4 +1,4 @@ -package clickhouse_connector_map +package clickhouse_connector_parse import "github.com/openmeterio/openmeter/pkg/models" diff --git a/openmeter/streaming/clickhouse_connector_raw/connector.go b/openmeter/streaming/clickhouse_connector_raw/connector.go index 354a68cd3..fcee96fc4 100644 --- a/openmeter/streaming/clickhouse_connector_raw/connector.go +++ b/openmeter/streaming/clickhouse_connector_raw/connector.go @@ -13,7 +13,6 @@ import ( "github.com/shopspring/decimal" "github.com/openmeterio/openmeter/api" - "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/pkg/models" ) @@ -26,19 +25,44 @@ type ClickhouseConnector struct { } type ClickhouseConnectorConfig struct { - Logger *slog.Logger - ClickHouse clickhouse.Conn - Database string - Meters meter.Repository - CreateOrReplaceMeter bool - PopulateMeter bool + Logger *slog.Logger + ClickHouse clickhouse.Conn + Database string + AsyncInsert bool + AsyncInsertWait bool + InsertQuerySettings map[string]string } -func NewClickhouseConnector(config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { +func (c ClickhouseConnectorConfig) Validate() error { + if c.Logger == nil { + return fmt.Errorf("logger is required") + } + + if c.ClickHouse == nil { + return fmt.Errorf("clickhouse connection is required") + } + + if c.Database == "" { + return fmt.Errorf("database is required") + } + + return nil +} + +func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { + if err := config.Validate(); err != nil { + return nil, fmt.Errorf("validate config: %w", err) + } + connector := &ClickhouseConnector{ config: config, } + err := connector.createEventsTable(ctx) + if err != nil { + return nil, fmt.Errorf("create events table in clickhouse: %w", err) + } + return connector, nil } @@ -121,11 +145,6 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s } func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - err := c.createEventsTable(ctx) - if err != nil { - return fmt.Errorf("create namespace in clickhouse: %w", err) - } - return nil } @@ -151,6 +170,35 @@ func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, return rows, nil } +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + var err error + + // Insert raw events + query := InsertEventsQuery{ + Database: c.config.Database, + Events: rawEvents, + QuerySettings: c.config.InsertQuerySettings, + } + sql, args := query.ToSQL() + + // By default, ClickHouse is writing data synchronously. + // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts + if c.config.AsyncInsert { + // With the `wait_for_async_insert` setting, you can configure + // if you want an insert statement to return with an acknowledgment + // either immediately after the data got inserted into the buffer. + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert raw events: %w", err) + } + + return nil +} + func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { table := createEventsTable{ Database: c.config.Database, diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query.go b/openmeter/streaming/clickhouse_connector_raw/event_query.go index abbb68eac..4c3bc353e 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/event_query.go @@ -7,24 +7,12 @@ import ( "time" "github.com/huandu/go-sqlbuilder" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) const EventsTableName = "om_events" -// CHEvent represents a raw event in ClickHouse -type CHEvent struct { - Namespace string `ch:"namespace"` - ValidationError string `ch:"validation_error"` - ID string `ch:"id"` - Type string `ch:"type"` - Source string `ch:"source"` - Subject string `ch:"subject"` - Time time.Time `ch:"time"` - Data string `ch:"data"` - IngestedAt time.Time `ch:"ingested_at"` - StoredAt time.Time `ch:"stored_at"` -} - // Create Events Table type createEventsTable struct { Database string @@ -135,7 +123,7 @@ func (d queryCountEvents) toSQL() (string, []interface{}) { // Insert Events Query type InsertEventsQuery struct { Database string - Events []CHEvent + Events []streaming.RawEvent QuerySettings map[string]string } diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go index 4fd7ea0d5..231b6ef1e 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go @@ -5,6 +5,8 @@ import ( "time" "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/openmeter/streaming" ) func TestCreateEventsTable(t *testing.T) { @@ -136,7 +138,7 @@ func TestInsertEventsQuery(t *testing.T) { query := InsertEventsQuery{ Database: "database", - Events: []CHEvent{ + Events: []streaming.RawEvent{ { Namespace: "my_namespace", ID: "1", diff --git a/openmeter/streaming/connector.go b/openmeter/streaming/connector.go index fbbda903c..6710b8bd9 100644 --- a/openmeter/streaming/connector.go +++ b/openmeter/streaming/connector.go @@ -35,6 +35,29 @@ type ListMeterSubjectsParams struct { To *time.Time } +// RawEvent represents a single raw event +type RawEvent struct { + Namespace string + ValidationError string + ID string + Type string + Source string + Subject string + Time time.Time + Data string + IngestedAt time.Time + StoredAt time.Time +} + +// Meter Event represents a single event related to a meter +type MeterEvent struct { + RawEvent + Meter string + Value float64 + ValueString string + GroupBy map[string]string +} + type Connector interface { CountEvents(ctx context.Context, namespace string, params CountEventsParams) ([]CountEventRow, error) ListEvents(ctx context.Context, namespace string, params ListEventsParams) ([]api.IngestedEvent, error) @@ -42,5 +65,6 @@ type Connector interface { DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error QueryMeter(ctx context.Context, namespace string, meter models.Meter, params QueryParams) ([]models.MeterQueryRow, error) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params ListMeterSubjectsParams) ([]string, error) + BatchInsert(ctx context.Context, events []RawEvent, meterEvents []MeterEvent) error // Add more methods as needed ... } diff --git a/openmeter/streaming/testutils/streaming.go b/openmeter/streaming/testutils/streaming.go index 089ebbf51..d396d700a 100644 --- a/openmeter/streaming/testutils/streaming.go +++ b/openmeter/streaming/testutils/streaming.go @@ -51,6 +51,10 @@ func (m *MockStreamingConnector) AddRow(meterSlug string, row models.MeterQueryR m.rows[meterSlug] = append(m.rows[meterSlug], row) } +func (m *MockStreamingConnector) Init(ctx context.Context) error { + return nil +} + func (m *MockStreamingConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { return []streaming.CountEventRow{}, nil } @@ -90,6 +94,10 @@ func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace strin return rows, nil } +func (m *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + return nil +} + func windowSizeToDuration(windowSize models.WindowSize) time.Duration { return windowSize.Duration() } From 97920f68a87d5d3c695ac651e327fbe7a68a3a2a Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 19:37:04 -0700 Subject: [PATCH 09/33] fix(raw): query --- .../clickhouse_connector/meter_query.go | 2 +- .../clickhouse_connector_raw/meter_query.go | 40 ++--- .../meter_query_test.go | 161 ++++++++++++------ 3 files changed, 135 insertions(+), 68 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index 343d2d9d2..52203ce50 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -354,7 +354,7 @@ func (d listMeterViewSubjects) toSQL() (string, []interface{}) { } func GetMeterViewName(database string, namespace string, meterSlug string) string { - meterViewName := fmt.Sprintf("%s_%s", namespace, meterSlug) + meterViewName := fmt.Sprintf("om_%s_%s", namespace, meterSlug) return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), sqlbuilder.Escape(meterViewName)) } diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse_connector_raw/meter_query.go index 1b5c1fdba..7e6af13a3 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query.go @@ -3,6 +3,7 @@ package clickhouse_connector_raw import ( _ "embed" "fmt" + "slices" "sort" "time" @@ -99,18 +100,26 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { selectColumns = append(selectColumns, fmt.Sprintf("%s(cast(JSON_VALUE(%s, '%s'), 'Float64')) AS value", sqlAggregation, getColumn("data"), sqlbuilder.Escape(d.Meter.ValueProperty))) } + groupBys := make([]string, 0, len(d.GroupBy)) + + for _, groupBy := range d.GroupBy { + if groupBy == "subject" { + selectColumns = append(selectColumns, getColumn("subject")) + groupByColumns = append(groupByColumns, "subject") + continue + } + + groupBys = append(groupBys, groupBy) + } + // Select Group By - sortedGroupBy := sortedKeys(d.Meter.GroupBy) - for _, groupByKey := range sortedGroupBy { + slices.Sort(groupBys) + + for _, groupByKey := range groupBys { groupByColumn := sqlbuilder.Escape(groupByKey) groupByJSONPath := sqlbuilder.Escape(d.Meter.GroupBy[groupByKey]) selectColumn := fmt.Sprintf("JSON_VALUE(%s, '%s') as %s", getColumn("data"), groupByJSONPath, groupByColumn) - // Subject is a special case - if groupByKey == "subject" { - selectColumn = getColumn("subject") - } - selectColumns = append(selectColumns, selectColumn) groupByColumns = append(groupByColumns, groupByColumn) } @@ -138,6 +147,10 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { sort.Strings(groupByKeys) for _, groupByKey := range groupByKeys { + if _, ok := d.Meter.GroupBy[groupByKey]; !ok { + return "", nil, fmt.Errorf("meter does not have group by: %s", groupByKey) + } + groupByJSONPath := sqlbuilder.Escape(d.Meter.GroupBy[groupByKey]) values := d.FilterGroupBy[groupByKey] @@ -152,7 +165,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { column = "subject" } - return query.Equal(column, value) + return fmt.Sprintf("%s = '%s'", column, sqlbuilder.Escape((value))) } where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) @@ -216,14 +229,3 @@ func columnFactory(alias string) func(string) string { return fmt.Sprintf("%s.%s", alias, column) } } - -func sortedKeys(m map[string]string) []string { - keys := make([]string, len(m)) - i := 0 - for k := range m { - keys[i] = k - i++ - } - sort.Strings(keys) - return keys -} diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go index b2f3cde14..744a107d7 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go @@ -26,8 +26,13 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, Subject: []string{subject}, From: &from, @@ -35,7 +40,7 @@ func TestQueryMeter(t *testing.T) { GroupBy: []string{"subject", "group1", "group2"}, WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, }, { // Aggregate all available data @@ -43,11 +48,16 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation @@ -55,11 +65,16 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationCount, + Slug: "meter1", + Aggregation: models.MeterAggregationCount, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT min(windowstart), max(windowend), count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start @@ -67,12 +82,17 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, From: &from, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period @@ -80,13 +100,18 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, From: &from, To: &to, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size @@ -94,14 +119,19 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, From: &from, To: &to, WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period in a different timezone, groupped by window size @@ -109,15 +139,20 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, From: &from, To: &to, WindowSize: &windowSize, WindowTimeZone: tz, }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data for a single subject @@ -125,13 +160,18 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields @@ -139,13 +179,18 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects @@ -153,13 +198,18 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "group1": "$.group1", + "group2": "$.group2", + }, }, Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value @@ -167,39 +217,54 @@ func TestQueryMeter(t *testing.T) { Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "g1": "$.group1", + "g2": "$.group2", + }, }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", + wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for a single group and multiple values query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "g1": "$.group1", + "g2": "$.group2", + }, }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", + wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for multiple groups and multiple values query: queryMeter{ Database: "openmeter", Namespace: "my_namespace", Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, + Slug: "meter1", + Aggregation: models.MeterAggregationSum, + ValueProperty: "$.value", + GroupBy: map[string]string{ + "g1": "$.group1", + "g2": "$.group2", + }, }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, + wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", + wantArgs: []interface{}{"my_namespace", "meter1"}, }, } @@ -237,8 +302,8 @@ func TestListMeterSubjects(t *testing.T) { Aggregation: models.MeterAggregationSum, }, }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_events WHERE namespace = ? AND type = ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "event1"}, }, { query: listMeterSubjectsQuery{ @@ -251,8 +316,8 @@ func TestListMeterSubjects(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_events WHERE namespace = ? AND type = ? AND time >= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix()}, }, { query: listMeterSubjectsQuery{ @@ -266,8 +331,8 @@ func TestListMeterSubjects(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT DISTINCT subject FROM openmeter.om_events WHERE namespace = ? AND type = ? AND time >= ? AND time <= ? ORDER BY subject", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix(), to.Unix()}, }, } From 40c1f8a88f79e97d34ac96db9994fad55f4111e2 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 19:49:24 -0700 Subject: [PATCH 10/33] feat(server): use parsed --- app/common/openmeter.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 39790d1bf..7c388a040 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -21,7 +21,7 @@ import ( "github.com/openmeterio/openmeter/openmeter/namespace" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler/ingestnotification" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/driver/noop" "github.com/openmeterio/openmeter/openmeter/watermill/eventbus" @@ -42,12 +42,9 @@ func NewClickHouseStreamingConnector( logger *slog.Logger, ) (*clickhouse_connector.ClickhouseConnector, error) { streamingConnector, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Meters: meterRepository, - CreateOrReplaceMeter: conf.CreateOrReplaceMeter, - PopulateMeter: conf.PopulateMeter, - Logger: logger, + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, }) if err != nil { return nil, fmt.Errorf("init clickhouse streaming: %w", err) From b5ff8e1f0336551ccdc11a38de68ec04736b6421 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 20:06:46 -0700 Subject: [PATCH 11/33] fix(app): wire --- app/common/wire.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/app/common/wire.go b/app/common/wire.go index 9c4e79f1e..3192340fc 100644 --- a/app/common/wire.go +++ b/app/common/wire.go @@ -16,7 +16,7 @@ import ( "github.com/openmeterio/openmeter/openmeter/meter" registrybuilder "github.com/openmeterio/openmeter/openmeter/registry/builder" "github.com/openmeterio/openmeter/openmeter/streaming" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/router" ) From b1b7ade301c59ef947147607250caaa50c04ce27 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 20:19:57 -0700 Subject: [PATCH 12/33] fix(connector): query without window --- .../clickhouse_connector_parse/meter_query.go | 2 +- .../meter_query_test.go | 20 +++++++++---------- .../clickhouse_connector_raw/meter_query.go | 2 +- .../meter_query_test.go | 20 +++++++++---------- 4 files changed, 22 insertions(+), 22 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_query.go index 194041a3b..75ef8975b 100644 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query.go @@ -68,7 +68,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { groupByColumns = append(groupByColumns, "windowstart", "windowend") } else { - selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") + selectColumns = append(selectColumns, "min(time) as windowstart", "max(time) as windowend") } switch d.Meter.Aggregation { diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go index db491ea29..3f844a8da 100644 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go @@ -47,7 +47,7 @@ func TestQueryMeter(t *testing.T) { Aggregation: models.MeterAggregationSum, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation @@ -59,7 +59,7 @@ func TestQueryMeter(t *testing.T) { Aggregation: models.MeterAggregationCount, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start @@ -72,7 +72,7 @@ func TestQueryMeter(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period @@ -86,7 +86,7 @@ func TestQueryMeter(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size @@ -131,7 +131,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields @@ -145,7 +145,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects @@ -159,7 +159,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value @@ -172,7 +172,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, }, { // Aggregate data with filtering for a single group and multiple values @@ -185,7 +185,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, }, { // Aggregate data with filtering for multiple groups and multiple values @@ -198,7 +198,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, }, } diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse_connector_raw/meter_query.go index 7e6af13a3..c2bd36420 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query.go @@ -70,7 +70,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { groupByColumns = append(groupByColumns, "windowstart", "windowend") } else { - selectColumns = append(selectColumns, "min(windowstart)", "max(windowend)") + selectColumns = append(selectColumns, "min(time) as windowstart", "max(time) as windowend") } // Select Value diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go index 744a107d7..32e150d58 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go @@ -57,7 +57,7 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation @@ -74,7 +74,7 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT min(windowstart), max(windowend), count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start @@ -92,7 +92,7 @@ func TestQueryMeter(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period @@ -111,7 +111,7 @@ func TestQueryMeter(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size @@ -171,7 +171,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields @@ -190,7 +190,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects @@ -209,7 +209,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value @@ -227,7 +227,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for a single group and multiple values @@ -245,7 +245,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for multiple groups and multiple values @@ -263,7 +263,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(windowstart), max(windowend), sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", + wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, } From 8406261d0121b49ff661cd6a0f9f680f9a976e47 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 20:56:06 -0700 Subject: [PATCH 13/33] fix(meter): parse string number --- openmeter/meter/parse.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/openmeter/meter/parse.go b/openmeter/meter/parse.go index 15f3eecae..59b3b3eb7 100644 --- a/openmeter/meter/parse.go +++ b/openmeter/meter/parse.go @@ -52,12 +52,14 @@ func ParseEvent(meter Meter, ev event.Event) (*float64, *string, map[string]stri case MeterAggregationSum, MeterAggregationAvg, MeterAggregationMin, MeterAggregationMax: switch value := rawValue.(type) { case string: - _, err = strconv.ParseFloat(value, 64) + val, err := strconv.ParseFloat(value, 64) if err != nil { // TODO: omit value or make sure it's length is not too long return nil, nil, groupBy, fmt.Errorf("event data value cannot be parsed as float64: %s", value) } + return &val, nil, groupBy, nil + case float64: return &value, nil, groupBy, nil From 7d3b8a1cec22f67c5f5a69571300fa3beee166ff Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 21:05:11 -0700 Subject: [PATCH 14/33] test(meter): parse numeric string value --- openmeter/meter/parse_test.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/openmeter/meter/parse_test.go b/openmeter/meter/parse_test.go index e68d686c3..dbc1c3293 100644 --- a/openmeter/meter/parse_test.go +++ b/openmeter/meter/parse_test.go @@ -71,6 +71,24 @@ func TestParseEvent(t *testing.T) { "path": "/api/v1", }, }, + { + description: "should parse event with numeric string value", + meter: meterSum, + event: func(t *testing.T) event.Event { + ev := event.New() + ev.SetType("api-calls") + + err := ev.SetData(event.ApplicationJSON, []byte(`{"duration_ms": "100", "method": "GET", "path": "/api/v1"}`)) + require.NoError(t, err) + + return ev + }, + value: lo.ToPtr(100.0), + groupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, + }, { description: "should parse count as value one", meter: meterCount, From be77d888d5f2528a28bf8e6a5f1abda0435a3e34 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Sun, 20 Oct 2024 21:13:13 -0700 Subject: [PATCH 15/33] fix(connector): round windows --- .../clickhouse_connector_parse/meter_query.go | 6 +++++- .../meter_query_test.go | 20 +++++++++---------- .../clickhouse_connector_raw/meter_query.go | 6 +++++- .../meter_query_test.go | 20 +++++++++---------- 4 files changed, 30 insertions(+), 22 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_query.go index 75ef8975b..2e8aff75d 100644 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query.go @@ -68,7 +68,11 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { groupByColumns = append(groupByColumns, "windowstart", "windowend") } else { - selectColumns = append(selectColumns, "min(time) as windowstart", "max(time) as windowend") + // TODO: remove this when we don't round to the nearest minute anymore + // We round them to the nearest minute to ensure the result is the same as with + // streaming connector using materialized views with per minute windows + selectColumn := fmt.Sprintf("tumbleStart(min(%s), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(%s), toIntervalMinute(1)) AS windowend", timeColumn, timeColumn) + selectColumns = append(selectColumns, selectColumn) } switch d.Meter.Aggregation { diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go index 3f844a8da..54dd4dcb2 100644 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go @@ -47,7 +47,7 @@ func TestQueryMeter(t *testing.T) { Aggregation: models.MeterAggregationSum, }, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation @@ -59,7 +59,7 @@ func TestQueryMeter(t *testing.T) { Aggregation: models.MeterAggregationCount, }, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start @@ -72,7 +72,7 @@ func TestQueryMeter(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period @@ -86,7 +86,7 @@ func TestQueryMeter(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size @@ -131,7 +131,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields @@ -145,7 +145,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects @@ -159,7 +159,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value @@ -172,7 +172,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, }, { // Aggregate data with filtering for a single group and multiple values @@ -185,7 +185,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, }, { // Aggregate data with filtering for multiple groups and multiple values @@ -198,7 +198,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", + wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, }, } diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse_connector_raw/meter_query.go index c2bd36420..695b8adb0 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query.go @@ -70,7 +70,11 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { groupByColumns = append(groupByColumns, "windowstart", "windowend") } else { - selectColumns = append(selectColumns, "min(time) as windowstart", "max(time) as windowend") + // TODO: remove this when we don't round to the nearest minute anymore + // We round them to the nearest minute to ensure the result is the same as with + // streaming connector using materialized views with per minute windows + selectColumn := fmt.Sprintf("tumbleStart(min(%s), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(%s), toIntervalMinute(1)) AS windowend", timeColumn, timeColumn) + selectColumns = append(selectColumns, selectColumn) } // Select Value diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go index 32e150d58..d919bfcac 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go @@ -57,7 +57,7 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate with count aggregation @@ -74,7 +74,7 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data from start @@ -92,7 +92,7 @@ func TestQueryMeter(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, }, { // Aggregate data between period @@ -111,7 +111,7 @@ func TestQueryMeter(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size @@ -171,7 +171,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields @@ -190,7 +190,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, }, { // Aggregate data for a multiple subjects @@ -209,7 +209,7 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value @@ -227,7 +227,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for a single group and multiple values @@ -245,7 +245,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, { // Aggregate data with filtering for multiple groups and multiple values @@ -263,7 +263,7 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT min(time) as windowstart, max(time) as windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", wantArgs: []interface{}{"my_namespace", "meter1"}, }, } From f8c0b5bcc2c9db674d0f7aced112241a749df1b7 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 09:16:39 -0700 Subject: [PATCH 16/33] refactor(streaming): reuse raw event connctor --- .../clickhouse_connector/connector.go | 240 +++------------ .../clickhouse_connector/event_query.go | 168 ----------- .../clickhouse_connector/event_query_test.go | 187 ------------ .../clickhouse_connector/meter_query.go | 4 +- .../clickhouse_connector_parse/connector.go | 277 ++++-------------- .../clickhouse_connector_parse/event_query.go | 168 ----------- .../event_query_test.go | 187 ------------ .../clickhouse_connector_parse/model.go | 9 - 8 files changed, 104 insertions(+), 1136 deletions(-) delete mode 100644 openmeter/streaming/clickhouse_connector/event_query.go delete mode 100644 openmeter/streaming/clickhouse_connector/event_query_test.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/event_query.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/event_query_test.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/model.go diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index facf7e9a6..fc67da840 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -2,18 +2,17 @@ package clickhouse_connector import ( "context" - "encoding/json" "fmt" "log/slog" "strings" "time" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/cloudevents/sdk-go/v2/event" "github.com/openmeterio/openmeter/api" "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" + raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" "github.com/openmeterio/openmeter/pkg/models" ) @@ -21,7 +20,8 @@ var _ streaming.Connector = (*ClickhouseConnector)(nil) // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. type ClickhouseConnector struct { - config ClickhouseConnectorConfig + config ClickhouseConnectorConfig + rawEventConnector *raw_event_connector.ClickhouseConnector } type ClickhouseConnectorConfig struct { @@ -61,39 +61,57 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return nil, fmt.Errorf("validate config: %w", err) } - connector := &ClickhouseConnector{ - config: config, + rawEventConnector, err := raw_event_connector.NewClickhouseConnector(ctx, raw_event_connector.ClickhouseConnectorConfig{ + Logger: config.Logger, + ClickHouse: config.ClickHouse, + Database: config.Database, + AsyncInsert: config.AsyncInsert, + AsyncInsertWait: config.AsyncInsertWait, + InsertQuerySettings: config.InsertQuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("create raw event connector: %w", err) } - err := connector.createEventsTable(ctx) - if err != nil { - return nil, fmt.Errorf("create events table in clickhouse: %w", err) + connector := &ClickhouseConnector{ + config: config, + rawEventConnector: rawEventConnector, } return connector, nil } -func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } +func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { + return nil +} - events, err := c.queryEventsTable(ctx, namespace, params) +func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { + err := c.deleteNamespace(ctx, namespace) if err != nil { - if _, ok := err.(*models.NamespaceNotFoundError); ok { - return nil, err - } - - return nil, fmt.Errorf("query events: %w", err) + return fmt.Errorf("delete namespace in clickhouse: %w", err) } + return nil +} + +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + return c.rawEventConnector.BatchInsert(ctx, rawEvents, meterEvents) +} - return events, nil +func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + return c.rawEventConnector.CountEvents(ctx, namespace, params) +} + +func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + return c.rawEventConnector.ListEvents(ctx, namespace, params) } func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { if namespace == "" { return fmt.Errorf("namespace is required") } + if meter.Slug == "" { + return fmt.Errorf("meter is required") + } err := c.createMeterView(ctx, namespace, meter) if err != nil { @@ -127,6 +145,9 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, if namespace == "" { return nil, fmt.Errorf("namespace is required") } + if meter.Slug == "" { + return nil, fmt.Errorf("meter is required") + } values, err := c.queryMeterView(ctx, namespace, meter, params) if err != nil { @@ -174,18 +195,6 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s return subjects, nil } -func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - return nil -} - -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { - err := c.deleteNamespace(ctx, namespace) - if err != nil { - return fmt.Errorf("delete namespace in clickhouse: %w", err) - } - return nil -} - // DeleteNamespace deletes the namespace related resources from Clickhouse // We don't delete the events table as it it reused between namespaces // We only delete the materialized views for the meters @@ -210,175 +219,6 @@ func (c *ClickhouseConnector) deleteNamespace(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } - - rows, err := c.queryCountEvents(ctx, namespace, params) - if err != nil { - if _, ok := err.(*models.NamespaceNotFoundError); ok { - return nil, err - } - - return nil, fmt.Errorf("query count events: %w", err) - } - - return rows, nil -} - -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { - var err error - - // Insert raw events - query := InsertEventsQuery{ - Database: c.config.Database, - Events: rawEvents, - QuerySettings: c.config.InsertQuerySettings, - } - sql, args := query.ToSQL() - - // By default, ClickHouse is writing data synchronously. - // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts - if c.config.AsyncInsert { - // With the `wait_for_async_insert` setting, you can configure - // if you want an insert statement to return with an acknowledgment - // either immediately after the data got inserted into the buffer. - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert raw events: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { - table := createEventsTable{ - Database: c.config.Database, - } - - err := c.config.ClickHouse.Exec(ctx, table.toSQL()) - if err != nil { - return fmt.Errorf("create events table: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { - table := queryEventsTable{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, - To: params.To, - IngestedAtFrom: params.IngestedAtFrom, - IngestedAtTo: params.IngestedAtTo, - ID: params.ID, - Subject: params.Subject, - HasError: params.HasError, - Limit: params.Limit, - } - - sql, args := table.toSQL() - - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.NamespaceNotFoundError{Namespace: namespace} - } - - return nil, fmt.Errorf("query events table query: %w", err) - } - - events := []api.IngestedEvent{} - - for rows.Next() { - var id string - var eventType string - var subject string - var source string - var eventTime time.Time - var dataStr string - var validationError string - var ingestedAt time.Time - var storedAt time.Time - - if err = rows.Scan(&id, &eventType, &subject, &source, &eventTime, &dataStr, &validationError, &ingestedAt, &storedAt); err != nil { - return nil, err - } - - // Parse data - var data interface{} - err := json.Unmarshal([]byte(dataStr), &data) - if err != nil { - return nil, fmt.Errorf("query events parse data: %w", err) - } - - event := event.New() - event.SetID(id) - event.SetType(eventType) - event.SetSubject(subject) - event.SetSource(source) - event.SetTime(eventTime) - err = event.SetData("application/json", data) - if err != nil { - return nil, fmt.Errorf("query events set data: %w", err) - } - - ingestedEvent := api.IngestedEvent{ - Event: event, - } - - if validationError != "" { - ingestedEvent.ValidationError = &validationError - } - - ingestedEvent.IngestedAt = ingestedAt - ingestedEvent.StoredAt = storedAt - - events = append(events, ingestedEvent) - } - - return events, nil -} - -func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { - table := queryCountEvents{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, - } - - sql, args := table.toSQL() - - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.NamespaceNotFoundError{Namespace: namespace} - } - - return nil, fmt.Errorf("query events count query: %w", err) - } - - results := []streaming.CountEventRow{} - - for rows.Next() { - result := streaming.CountEventRow{} - - if err = rows.Scan(&result.Count, &result.Subject, &result.IsError); err != nil { - return nil, err - } - - results = append(results, result) - } - - return results, nil -} - func (c *ClickhouseConnector) createMeterView(ctx context.Context, namespace string, meter models.Meter) error { // CreateOrReplace is used to force the recreation of the materialized view // This is not safe to use in production as it will drop the existing views diff --git a/openmeter/streaming/clickhouse_connector/event_query.go b/openmeter/streaming/clickhouse_connector/event_query.go deleted file mode 100644 index 4e0e562fc..000000000 --- a/openmeter/streaming/clickhouse_connector/event_query.go +++ /dev/null @@ -1,168 +0,0 @@ -package clickhouse_connector - -import ( - _ "embed" - "fmt" - "strings" - "time" - - "github.com/huandu/go-sqlbuilder" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -const EventsTableName = "om_events" - -// Create Events Table -type createEventsTable struct { - Database string -} - -func (d createEventsTable) toSQL() string { - tableName := GetEventsTableName(d.Database) - - sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() - sb.CreateTable(tableName) - sb.IfNotExists() - sb.Define("namespace", "String") - sb.Define("validation_error", "String") - sb.Define("id", "String") - sb.Define("type", "LowCardinality(String)") - sb.Define("subject", "String") - sb.Define("source", "String") - sb.Define("time", "DateTime") - sb.Define("data", "String") - sb.Define("ingested_at", "DateTime") - sb.Define("stored_at", "DateTime") - sb.SQL("ENGINE = MergeTree") - sb.SQL("PARTITION BY toYYYYMM(time)") - sb.SQL("ORDER BY (namespace, time, type, subject)") - - sql, _ := sb.Build() - return sql -} - -// Query Events Table -type queryEventsTable struct { - Database string - Namespace string - From *time.Time - To *time.Time - IngestedAtFrom *time.Time - IngestedAtTo *time.Time - ID *string - Subject *string - HasError *bool - Limit int -} - -func (d queryEventsTable) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - where := []string{} - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") - query.From(tableName) - - where = append(where, query.Equal("namespace", d.Namespace)) - if d.From != nil { - where = append(where, query.GreaterEqualThan("time", d.From.Unix())) - } - if d.To != nil { - where = append(where, query.LessEqualThan("time", d.To.Unix())) - } - if d.IngestedAtFrom != nil { - where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) - } - if d.IngestedAtTo != nil { - where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) - } - if d.ID != nil { - where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) - } - if d.Subject != nil { - where = append(where, query.Equal("subject", *d.Subject)) - } - if d.HasError != nil { - if *d.HasError { - where = append(where, "notEmpty(validation_error) = 1") - } else { - where = append(where, "empty(validation_error) = 1") - } - } - query.Where(where...) - - query.Desc().OrderBy("time") - query.Limit(d.Limit) - - sql, args := query.Build() - return sql, args -} - -type queryCountEvents struct { - Database string - Namespace string - From time.Time -} - -func (d queryCountEvents) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") - query.From(tableName) - - query.Where(query.Equal("namespace", d.Namespace)) - query.Where(query.GreaterEqualThan("time", d.From.Unix())) - query.GroupBy("subject", "is_error") - - sql, args := query.Build() - return sql, args -} - -// Insert Events Query -type InsertEventsQuery struct { - Database string - Events []streaming.RawEvent - QuerySettings map[string]string -} - -func (q InsertEventsQuery) ToSQL() (string, []interface{}) { - tableName := GetEventsTableName(q.Database) - - query := sqlbuilder.ClickHouse.NewInsertBuilder() - query.InsertInto(tableName) - query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") - - // Add settings - var settings []string - for key, value := range q.QuerySettings { - settings = append(settings, fmt.Sprintf("%s = %s", key, value)) - } - - if len(settings) > 0 { - query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) - } - - for _, event := range q.Events { - query.Values( - event.Namespace, - event.ValidationError, - event.ID, - event.Type, - event.Source, - event.Subject, - event.Time, - event.Data, - event.IngestedAt, - event.StoredAt, - ) - } - - sql, args := query.Build() - return sql, args -} - -func GetEventsTableName(database string) string { - return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) -} diff --git a/openmeter/streaming/clickhouse_connector/event_query_test.go b/openmeter/streaming/clickhouse_connector/event_query_test.go deleted file mode 100644 index 24b28d6e4..000000000 --- a/openmeter/streaming/clickhouse_connector/event_query_test.go +++ /dev/null @@ -1,187 +0,0 @@ -package clickhouse_connector - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -func TestCreateEventsTable(t *testing.T) { - tests := []struct { - data createEventsTable - want string - }{ - { - data: createEventsTable{ - Database: "openmeter", - }, - want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - got := tt.data.toSQL() - assert.Equal(t, tt.want, got) - }) - } -} - -func TestQueryEventsTable(t *testing.T) { - subjectFilter := "customer-1" - idFilter := "event-id-1" - hasErrorTrue := true - hasErrorFalse := false - - tests := []struct { - query queryEventsTable - wantSQL string - wantArgs []interface{} - }{ - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - Subject: &subjectFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", subjectFilter}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - ID: &idFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorTrue, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorFalse, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - -func TestQueryEventsCount(t *testing.T) { - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - tests := []struct { - query queryCountEvents - wantSQL string - wantArgs []interface{} - }{ - { - query: queryCountEvents{ - Database: "openmeter", - Namespace: "my_namespace", - From: from, - }, - wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", - wantArgs: []interface{}{"my_namespace", from.Unix()}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - -func TestInsertEventsQuery(t *testing.T) { - now := time.Now() - - query := InsertEventsQuery{ - Database: "database", - Events: []streaming.RawEvent{ - { - Namespace: "my_namespace", - ID: "1", - Source: "source", - Subject: "subject-1", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, - }, - { - Namespace: "my_namespace", - ID: "2", - Source: "source", - Subject: "subject-2", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, - }, - { - Namespace: "my_namespace", - ValidationError: "event data value cannot be parsed as float64: not a number", - ID: "3", - Source: "source", - Subject: "subject-2", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, - }, - }, - } - - sql, args := query.ToSQL() - - assert.Equal(t, []interface{}{ - "my_namespace", "", "1", "api-calls", "source", "subject-1", now, `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "", "2", "api-calls", "source", "subject-2", now, `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now, `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, - }, args) - assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) -} diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index 52203ce50..aa63d7aef 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -9,6 +9,8 @@ import ( "github.com/huandu/go-sqlbuilder" + + raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" "github.com/openmeterio/openmeter/pkg/models" "github.com/openmeterio/openmeter/pkg/slicesx" ) @@ -102,7 +104,7 @@ func (d createMeterView) toSQL() (string, []interface{}, error) { } func (d createMeterView) toSelectSQL() (string, error) { - eventsTableName := GetEventsTableName(d.Database) + eventsTableName := raw_event_connector.GetEventsTableName(d.Database) aggStateFn := "" switch d.Aggregation { diff --git a/openmeter/streaming/clickhouse_connector_parse/connector.go b/openmeter/streaming/clickhouse_connector_parse/connector.go index a642a8f5e..fa02a061a 100644 --- a/openmeter/streaming/clickhouse_connector_parse/connector.go +++ b/openmeter/streaming/clickhouse_connector_parse/connector.go @@ -2,18 +2,17 @@ package clickhouse_connector_parse import ( "context" - "encoding/json" "fmt" "log/slog" "strings" "time" "github.com/ClickHouse/clickhouse-go/v2" - "github.com/cloudevents/sdk-go/v2/event" "github.com/shopspring/decimal" "github.com/openmeterio/openmeter/api" "github.com/openmeterio/openmeter/openmeter/streaming" + raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" "github.com/openmeterio/openmeter/pkg/models" ) @@ -21,7 +20,8 @@ var _ streaming.Connector = (*ClickhouseConnector)(nil) // ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. type ClickhouseConnector struct { - config ClickhouseConnectorConfig + config ClickhouseConnectorConfig + rawEventConnector *raw_event_connector.ClickhouseConnector } type ClickhouseConnectorConfig struct { @@ -54,13 +54,21 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return nil, fmt.Errorf("validate config: %w", err) } - connector := &ClickhouseConnector{ - config: config, + rawEventConnector, err := raw_event_connector.NewClickhouseConnector(ctx, raw_event_connector.ClickhouseConnectorConfig{ + Logger: config.Logger, + ClickHouse: config.ClickHouse, + Database: config.Database, + AsyncInsert: config.AsyncInsert, + AsyncInsertWait: config.AsyncInsertWait, + InsertQuerySettings: config.InsertQuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("create raw event connector: %w", err) } - err := connector.createEventsTable(ctx) - if err != nil { - return nil, fmt.Errorf("create events table in clickhouse: %w", err) + connector := &ClickhouseConnector{ + config: config, + rawEventConnector: rawEventConnector, } err = connector.createMeterEventTable(ctx) @@ -71,21 +79,56 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return connector, nil } -func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } +func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { + return nil +} - events, err := c.queryEventsTable(ctx, namespace, params) +func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { + // We don't delete the event tables as it it reused between namespaces + return nil +} + +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { + // Insert raw events + err := c.rawEventConnector.BatchInsert(ctx, rawEvents, meterEvents) if err != nil { - if _, ok := err.(*models.NamespaceNotFoundError); ok { - return nil, err - } + return fmt.Errorf("failed to batch insert raw events: %w", err) + } + + // NOTE: The two inserts are not atomic. + // If the second insert fails, the first insert will not be rolled back. + + // Insert meter events + if len(meterEvents) == 0 { + return nil + } + + query := InsertMeterEventsQuery{ + Database: c.config.Database, + MeterEvents: meterEvents, + QuerySettings: c.config.InsertQuerySettings, + } + sql, args := query.ToSQL() - return nil, fmt.Errorf("query events: %w", err) + if c.config.AsyncInsert { + err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) + } else { + err = c.config.ClickHouse.Exec(ctx, sql, args...) + } + + if err != nil { + return fmt.Errorf("failed to batch insert meter events: %w", err) } - return events, nil + return nil +} + +func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { + return c.rawEventConnector.CountEvents(ctx, namespace, params) +} + +func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { + return c.rawEventConnector.ListEvents(ctx, namespace, params) } func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { @@ -149,94 +192,6 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s return subjects, nil } -func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - return nil -} - -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { - // We don't delete the event tables as it it reused between namespaces - return nil -} - -func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } - - rows, err := c.queryCountEvents(ctx, namespace, params) - if err != nil { - if _, ok := err.(*models.NamespaceNotFoundError); ok { - return nil, err - } - - return nil, fmt.Errorf("query count events: %w", err) - } - - return rows, nil -} - -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { - var err error - - // Insert raw events - query := InsertEventsQuery{ - Database: c.config.Database, - Events: rawEvents, - QuerySettings: c.config.InsertQuerySettings, - } - sql, args := query.ToSQL() - - // By default, ClickHouse is writing data synchronously. - // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts - if c.config.AsyncInsert { - // With the `wait_for_async_insert` setting, you can configure - // if you want an insert statement to return with an acknowledgment - // either immediately after the data got inserted into the buffer. - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert raw events: %w", err) - } - - // Insert meter events - if len(meterEvents) > 0 { - query := InsertMeterEventsQuery{ - Database: c.config.Database, - MeterEvents: meterEvents, - QuerySettings: c.config.InsertQuerySettings, - } - sql, args := query.ToSQL() - - if c.config.AsyncInsert { - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert meter events: %w", err) - } - } - - return nil -} - -func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { - table := createEventsTable{ - Database: c.config.Database, - } - - err := c.config.ClickHouse.Exec(ctx, table.toSQL()) - if err != nil { - return fmt.Errorf("create events table: %w", err) - } - - return nil -} - func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { table := createMeterEventTable{ Database: c.config.Database, @@ -250,116 +205,6 @@ func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { return nil } -func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { - table := queryEventsTable{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, - To: params.To, - IngestedAtFrom: params.IngestedAtFrom, - IngestedAtTo: params.IngestedAtTo, - ID: params.ID, - Subject: params.Subject, - HasError: params.HasError, - Limit: params.Limit, - } - - sql, args := table.toSQL() - - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.NamespaceNotFoundError{Namespace: namespace} - } - - return nil, fmt.Errorf("query events table query: %w", err) - } - - events := []api.IngestedEvent{} - - for rows.Next() { - var id string - var eventType string - var subject string - var source string - var eventTime time.Time - var dataStr string - var validationError string - var ingestedAt time.Time - var storedAt time.Time - - if err = rows.Scan(&id, &eventType, &subject, &source, &eventTime, &dataStr, &validationError, &ingestedAt, &storedAt); err != nil { - return nil, err - } - - // Parse data - var data interface{} - err := json.Unmarshal([]byte(dataStr), &data) - if err != nil { - return nil, fmt.Errorf("query events parse data: %w", err) - } - - event := event.New() - event.SetID(id) - event.SetType(eventType) - event.SetSubject(subject) - event.SetSource(source) - event.SetTime(eventTime) - err = event.SetData("application/json", data) - if err != nil { - return nil, fmt.Errorf("query events set data: %w", err) - } - - ingestedEvent := api.IngestedEvent{ - Event: event, - } - - if validationError != "" { - ingestedEvent.ValidationError = &validationError - } - - ingestedEvent.IngestedAt = ingestedAt - ingestedEvent.StoredAt = storedAt - - events = append(events, ingestedEvent) - } - - return events, nil -} - -func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { - table := queryCountEvents{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, - } - - sql, args := table.toSQL() - - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.NamespaceNotFoundError{Namespace: namespace} - } - - return nil, fmt.Errorf("query events count query: %w", err) - } - - results := []streaming.CountEventRow{} - - for rows.Next() { - result := streaming.CountEventRow{} - - if err = rows.Scan(&result.Count, &result.Subject, &result.IsError); err != nil { - return nil, err - } - - results = append(results, result) - } - - return results, nil -} - func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeter{ Database: c.config.Database, diff --git a/openmeter/streaming/clickhouse_connector_parse/event_query.go b/openmeter/streaming/clickhouse_connector_parse/event_query.go deleted file mode 100644 index 50e4264b1..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/event_query.go +++ /dev/null @@ -1,168 +0,0 @@ -package clickhouse_connector_parse - -import ( - _ "embed" - "fmt" - "strings" - "time" - - "github.com/huandu/go-sqlbuilder" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -const EventsTableName = "om_events" - -// Create Events Table -type createEventsTable struct { - Database string -} - -func (d createEventsTable) toSQL() string { - tableName := GetEventsTableName(d.Database) - - sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() - sb.CreateTable(tableName) - sb.IfNotExists() - sb.Define("namespace", "String") - sb.Define("validation_error", "String") - sb.Define("id", "String") - sb.Define("type", "LowCardinality(String)") - sb.Define("subject", "String") - sb.Define("source", "String") - sb.Define("time", "DateTime") - sb.Define("data", "String") - sb.Define("ingested_at", "DateTime") - sb.Define("stored_at", "DateTime") - sb.SQL("ENGINE = MergeTree") - sb.SQL("PARTITION BY toYYYYMM(time)") - sb.SQL("ORDER BY (namespace, time, type, subject)") - - sql, _ := sb.Build() - return sql -} - -// Query Events Table -type queryEventsTable struct { - Database string - Namespace string - From *time.Time - To *time.Time - IngestedAtFrom *time.Time - IngestedAtTo *time.Time - ID *string - Subject *string - HasError *bool - Limit int -} - -func (d queryEventsTable) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - where := []string{} - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("id", "type", "subject", "source", "time", "data", "validation_error", "ingested_at", "stored_at") - query.From(tableName) - - where = append(where, query.Equal("namespace", d.Namespace)) - if d.From != nil { - where = append(where, query.GreaterEqualThan("time", d.From.Unix())) - } - if d.To != nil { - where = append(where, query.LessEqualThan("time", d.To.Unix())) - } - if d.IngestedAtFrom != nil { - where = append(where, query.GreaterEqualThan("ingested_at", d.IngestedAtFrom.Unix())) - } - if d.IngestedAtTo != nil { - where = append(where, query.LessEqualThan("ingested_at", d.IngestedAtTo.Unix())) - } - if d.ID != nil { - where = append(where, query.Like("id", fmt.Sprintf("%%%s%%", *d.ID))) - } - if d.Subject != nil { - where = append(where, query.Equal("subject", *d.Subject)) - } - if d.HasError != nil { - if *d.HasError { - where = append(where, "notEmpty(validation_error) = 1") - } else { - where = append(where, "empty(validation_error) = 1") - } - } - query.Where(where...) - - query.Desc().OrderBy("time") - query.Limit(d.Limit) - - sql, args := query.Build() - return sql, args -} - -type queryCountEvents struct { - Database string - Namespace string - From time.Time -} - -func (d queryCountEvents) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") - query.From(tableName) - - query.Where(query.Equal("namespace", d.Namespace)) - query.Where(query.GreaterEqualThan("time", d.From.Unix())) - query.GroupBy("subject", "is_error") - - sql, args := query.Build() - return sql, args -} - -// Insert Events Query -type InsertEventsQuery struct { - Database string - Events []streaming.RawEvent - QuerySettings map[string]string -} - -func (q InsertEventsQuery) ToSQL() (string, []interface{}) { - tableName := GetEventsTableName(q.Database) - - query := sqlbuilder.ClickHouse.NewInsertBuilder() - query.InsertInto(tableName) - query.Cols("namespace", "validation_error", "id", "type", "source", "subject", "time", "data", "ingested_at", "stored_at") - - // Add settings - var settings []string - for key, value := range q.QuerySettings { - settings = append(settings, fmt.Sprintf("%s = %s", key, value)) - } - - if len(settings) > 0 { - query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) - } - - for _, event := range q.Events { - query.Values( - event.Namespace, - event.ValidationError, - event.ID, - event.Type, - event.Source, - event.Subject, - event.Time, - event.Data, - event.IngestedAt, - event.StoredAt, - ) - } - - sql, args := query.Build() - return sql, args -} - -func GetEventsTableName(database string) string { - return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) -} diff --git a/openmeter/streaming/clickhouse_connector_parse/event_query_test.go b/openmeter/streaming/clickhouse_connector_parse/event_query_test.go deleted file mode 100644 index 3f75f431e..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/event_query_test.go +++ /dev/null @@ -1,187 +0,0 @@ -package clickhouse_connector_parse - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -func TestCreateEventsTable(t *testing.T) { - tests := []struct { - data createEventsTable - want string - }{ - { - data: createEventsTable{ - Database: "openmeter", - }, - want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - got := tt.data.toSQL() - assert.Equal(t, tt.want, got) - }) - } -} - -func TestQueryEventsTable(t *testing.T) { - subjectFilter := "customer-1" - idFilter := "event-id-1" - hasErrorTrue := true - hasErrorFalse := false - - tests := []struct { - query queryEventsTable - wantSQL string - wantArgs []interface{} - }{ - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - Subject: &subjectFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", subjectFilter}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - ID: &idFilter, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorTrue, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - { - query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorFalse, - }, - wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", - wantArgs: []interface{}{"my_namespace"}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - -func TestQueryEventsCount(t *testing.T) { - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - tests := []struct { - query queryCountEvents - wantSQL string - wantArgs []interface{} - }{ - { - query: queryCountEvents{ - Database: "openmeter", - Namespace: "my_namespace", - From: from, - }, - wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", - wantArgs: []interface{}{"my_namespace", from.Unix()}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} - -func TestInsertEventsQuery(t *testing.T) { - now := time.Now() - - query := InsertEventsQuery{ - Database: "database", - Events: []streaming.RawEvent{ - { - Namespace: "my_namespace", - ID: "1", - Source: "source", - Subject: "subject-1", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, - }, - { - Namespace: "my_namespace", - ID: "2", - Source: "source", - Subject: "subject-2", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, - }, - { - Namespace: "my_namespace", - ValidationError: "event data value cannot be parsed as float64: not a number", - ID: "3", - Source: "source", - Subject: "subject-2", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, - }, - }, - } - - sql, args := query.ToSQL() - - assert.Equal(t, []interface{}{ - "my_namespace", "", "1", "api-calls", "source", "subject-1", now, `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "", "2", "api-calls", "source", "subject-2", now, `{"duration_ms": 80, "method": "GET", "path": "/api/v1"}`, now, now, - "my_namespace", "event data value cannot be parsed as float64: not a number", "3", "api-calls", "source", "subject-2", now, `{"duration_ms": "foo", "method": "GET", "path": "/api/v1"}`, now, now, - }, args) - assert.Equal(t, `INSERT INTO database.om_events (namespace, validation_error, id, type, source, subject, time, data, ingested_at, stored_at) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) -} diff --git a/openmeter/streaming/clickhouse_connector_parse/model.go b/openmeter/streaming/clickhouse_connector_parse/model.go deleted file mode 100644 index 308838dfc..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/model.go +++ /dev/null @@ -1,9 +0,0 @@ -package clickhouse_connector_parse - -import "github.com/openmeterio/openmeter/pkg/models" - -type MeterView struct { - Slug string - Aggregation models.MeterAggregation - GroupBy []string -} From 39741ae3bd45e9498befb11fa4f9d7fc1bc6c51e Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 09:19:24 -0700 Subject: [PATCH 17/33] feat(streaming): add query raw events config --- openmeter/streaming/clickhouse_connector/connector.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector/connector.go index fc67da840..c2e798da2 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector/connector.go @@ -34,6 +34,7 @@ type ClickhouseConnectorConfig struct { AsyncInsert bool AsyncInsertWait bool InsertQuerySettings map[string]string + QueryRawEvents bool } func (c ClickhouseConnectorConfig) Validate() error { @@ -149,6 +150,11 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return nil, fmt.Errorf("meter is required") } + // Query raw events if the flag is set + if c.config.QueryRawEvents { + return c.rawEventConnector.QueryMeter(ctx, namespace, meter, params) + } + values, err := c.queryMeterView(ctx, namespace, meter, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { @@ -183,6 +189,11 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s return nil, fmt.Errorf("meter is required") } + // Query raw events if the flag is set + if c.config.QueryRawEvents { + return c.rawEventConnector.ListMeterSubjects(ctx, namespace, meter, params) + } + subjects, err := c.listMeterViewSubjects(ctx, namespace, meter, params) if err != nil { if _, ok := err.(*models.MeterNotFoundError); ok { From d56a0c63a66ecd124452fe8e0e8130c2a92d1c7d Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 11:49:25 -0700 Subject: [PATCH 18/33] feat(config): streaming engine --- app/common/openmeter.go | 72 ++++++++++++++--- app/common/wire.go | 3 - app/config/aggregation.go | 52 +++++++++++++ cmd/balance-worker/wire_gen.go | 4 +- cmd/notification-service/wire_gen.go | 4 +- cmd/server/wire_gen.go | 6 +- cmd/sink-worker/main.go | 90 ++++++++++++++++------ openmeter/server/server_test.go | 6 +- openmeter/streaming/connector.go | 3 + openmeter/streaming/testutils/streaming.go | 6 +- 10 files changed, 198 insertions(+), 48 deletions(-) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 7c388a040..eb30816b3 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -21,7 +21,10 @@ import ( "github.com/openmeterio/openmeter/openmeter/namespace" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler/ingestnotification" - clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" + "github.com/openmeterio/openmeter/openmeter/streaming" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/driver/noop" "github.com/openmeterio/openmeter/openmeter/watermill/eventbus" @@ -40,17 +43,64 @@ func NewClickHouseStreamingConnector( clickHouse clickhouse.Conn, meterRepository meter.Repository, logger *slog.Logger, -) (*clickhouse_connector.ClickhouseConnector, error) { - streamingConnector, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Logger: logger, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse streaming: %w", err) +) (streaming.Connector, error) { + var ( + connector streaming.Connector + err error + ) + + switch conf.Engine { + case config.AggregationEngineClickHouseRaw: + connector, err = clickhouse_connector_raw.NewClickhouseConnector(ctx, clickhouse_connector_raw.ClickhouseConnectorConfig{ + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + + // TODO: add insert related config after moved from sink config + // AsyncInsert: conf.Aggregation.AsyncInsert, + // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, + // InsertQuerySettings: conf.Aggregation.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse raw engine: %w", err) + } + + case config.AggregationEngineClickHouseMV: + connector, err = clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + PopulateMeter: conf.PopulateMeter, + CreateOrReplaceMeter: conf.CreateOrReplaceMeter, + QueryRawEvents: conf.QueryRawEvents, + + // TODO: add insert related config after moved from sink config + // AsyncInsert: conf.Aggregation.AsyncInsert, + // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, + // InsertQuerySettings: conf.Aggregation.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse mv engine: %w", err) + } + case config.AggregationEngineClickHouseParse: + connector, err = clickhouse_connector_parse.NewClickhouseConnector(ctx, clickhouse_connector_parse.ClickhouseConnectorConfig{ + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + + // TODO: add insert related config after moved from sink config + // AsyncInsert: conf.Aggregation.AsyncInsert, + // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, + // InsertQuerySettings: conf.Aggregation.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse parse engine: %w", err) + } + default: + return nil, fmt.Errorf("invalid aggregation engine: %s", conf.Engine) } - return streamingConnector, nil + return connector, nil } func NewNamespacedTopicResolver(config config.Configuration) (*topicresolver.NamespacedTopicResolver, error) { @@ -133,7 +183,7 @@ func NewKafkaNamespaceHandler( func NewNamespaceHandlers( kafkaHandler *kafkaingest.NamespaceHandler, - clickHouseHandler *clickhouse_connector.ClickhouseConnector, + clickHouseHandler streaming.Connector, ) []namespace.Handler { return []namespace.Handler{ kafkaHandler, diff --git a/app/common/wire.go b/app/common/wire.go index 3192340fc..54b323899 100644 --- a/app/common/wire.go +++ b/app/common/wire.go @@ -15,8 +15,6 @@ import ( "github.com/openmeterio/openmeter/openmeter/ingest/kafkaingest/topicresolver" "github.com/openmeterio/openmeter/openmeter/meter" registrybuilder "github.com/openmeterio/openmeter/openmeter/registry/builder" - "github.com/openmeterio/openmeter/openmeter/streaming" - clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/router" ) @@ -103,7 +101,6 @@ var OpenMeter = wire.NewSet( wire.Bind(new(meter.Repository), new(*meter.InMemoryRepository)), NewClickHouseStreamingConnector, - wire.Bind(new(streaming.Connector), new(*clickhouse_connector.ClickhouseConnector)), NewNamespacedTopicResolver, wire.Bind(new(topicresolver.Resolver), new(*topicresolver.NamespacedTopicResolver)), diff --git a/app/config/aggregation.go b/app/config/aggregation.go index dca483983..519b73a06 100644 --- a/app/config/aggregation.go +++ b/app/config/aggregation.go @@ -4,13 +4,38 @@ import ( "crypto/tls" "errors" "fmt" + "slices" "time" "github.com/ClickHouse/clickhouse-go/v2" "github.com/spf13/viper" ) +type AggregationEngine string + +const ( + // Raw engine queries the raw events table + AggregationEngineClickHouseRaw AggregationEngine = "clickhouse_raw" + // MV engine maintains and queries materialized views + AggregationEngineClickHouseMV AggregationEngine = "clickhouse_mv" + // Parse engine parses events by meters and stores them in a separate unified meter events table + AggregationEngineClickHouseParse AggregationEngine = "clickhouse_parse" +) + +func (e AggregationEngine) Values() []AggregationEngine { + return []AggregationEngine{AggregationEngineClickHouseRaw, AggregationEngineClickHouseMV, AggregationEngineClickHouseParse} +} + +func (e AggregationEngine) Validate() error { + if !slices.Contains(e.Values(), e) { + return fmt.Errorf("invalid value") + } + return nil +} + type AggregationConfiguration struct { + // Engine is the aggregation engine to use + Engine AggregationEngine ClickHouse ClickHouseAggregationConfiguration // Populate creates the materialized view with data from the events table // This is not safe to use in production as requires to stop ingestion @@ -18,6 +43,8 @@ type AggregationConfiguration struct { // CreateOrReplace is used to force the recreation of the materialized view // This is not safe to use in production as it will drop the existing views CreateOrReplaceMeter bool + // QueryRawEvents is used to query the raw events table instead of the materialized view + QueryRawEvents bool } // Validate validates the configuration. @@ -26,6 +53,29 @@ func (c AggregationConfiguration) Validate() error { return fmt.Errorf("clickhouse: %w", err) } + if c.Engine == "" { + return errors.New("engine is required") + } + + if err := c.Engine.Validate(); err != nil { + return fmt.Errorf("engine: %w", err) + } + + // Validate engine specific options + if c.Engine != AggregationEngineClickHouseMV { + if c.PopulateMeter { + return errors.New("populate meter is only supported with materialized view engine") + } + + if c.CreateOrReplaceMeter { + return errors.New("create or replace meter is only with materialized view engine") + } + + if c.QueryRawEvents { + return errors.New("query raw events is only with materialized view engine") + } + } + return nil } @@ -100,6 +150,8 @@ func (c ClickHouseAggregationConfiguration) GetClientOptions() *clickhouse.Optio // ConfigureAggregation configures some defaults in the Viper instance. func ConfigureAggregation(v *viper.Viper) { + v.SetDefault("aggregation.engine", AggregationEngineClickHouseMV) + v.SetDefault("aggregation.clickhouse.address", "127.0.0.1:9000") v.SetDefault("aggregation.clickhouse.tls", false) v.SetDefault("aggregation.clickhouse.database", "openmeter") diff --git a/cmd/balance-worker/wire_gen.go b/cmd/balance-worker/wire_gen.go index 99b49dfb5..e1e1215e2 100644 --- a/cmd/balance-worker/wire_gen.go +++ b/cmd/balance-worker/wire_gen.go @@ -134,7 +134,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v3 := conf.Meters inMemoryRepository := common.NewMeterRepository(v3) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v2, inMemoryRepository, logger) + connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v2, inMemoryRepository, logger) if err != nil { cleanup5() cleanup4() @@ -145,7 +145,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } entitlementOptions := registrybuilder.EntitlementOptions{ DatabaseClient: client, - StreamingConnector: clickhouseConnector, + StreamingConnector: connector, Logger: logger, MeterRepository: inMemoryRepository, Publisher: eventbusPublisher, diff --git a/cmd/notification-service/wire_gen.go b/cmd/notification-service/wire_gen.go index 8185b92ab..d452f03e2 100644 --- a/cmd/notification-service/wire_gen.go +++ b/cmd/notification-service/wire_gen.go @@ -74,7 +74,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) + connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() @@ -140,7 +140,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl GlobalInitializer: globalInitializer, Migrator: migrator, Metadata: commonMetadata, - StreamingConnector: clickhouseConnector, + StreamingConnector: connector, MeterRepository: inMemoryRepository, EntClient: client, TelemetryServer: v3, diff --git a/cmd/server/wire_gen.go b/cmd/server/wire_gen.go index 83fd3776d..c9867f1e5 100644 --- a/cmd/server/wire_gen.go +++ b/cmd/server/wire_gen.go @@ -78,7 +78,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - clickhouseConnector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) + connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() @@ -198,7 +198,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl cleanup() return Application{}, nil, err } - v5 := common.NewNamespaceHandlers(namespaceHandler, clickhouseConnector) + v5 := common.NewNamespaceHandlers(namespaceHandler, connector) namespaceConfiguration := conf.Namespace manager, err := common.NewNamespaceManager(v5, namespaceConfiguration) if err != nil { @@ -215,7 +215,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl application := Application{ GlobalInitializer: globalInitializer, Migrator: migrator, - StreamingConnector: clickhouseConnector, + StreamingConnector: connector, MeterRepository: inMemoryRepository, EntClient: client, TelemetryServer: v3, diff --git a/cmd/sink-worker/main.go b/cmd/sink-worker/main.go index f4bfae910..c4653cc5c 100644 --- a/cmd/sink-worker/main.go +++ b/cmd/sink-worker/main.go @@ -23,7 +23,10 @@ import ( "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/sink" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" - clickhouse_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" + "github.com/openmeterio/openmeter/openmeter/streaming" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" pkgkafka "github.com/openmeterio/openmeter/pkg/kafka" ) @@ -136,30 +139,67 @@ func main() { } } -func initSink(ctx context.Context, config config.Configuration, logger *slog.Logger, metricMeter metric.Meter, tracer trace.Tracer, meterRepository meter.Repository, flushHandler flushhandler.FlushEventHandler) (*sink.Sink, error) { +func initSink(ctx context.Context, conf config.Configuration, logger *slog.Logger, metricMeter metric.Meter, tracer trace.Tracer, meterRepository meter.Repository, flushHandler flushhandler.FlushEventHandler) (*sink.Sink, error) { // Initialize ClickHouse client - clickhouseClient, err := clickhouse.Open(config.Aggregation.ClickHouse.GetClientOptions()) + clickhouseClient, err := clickhouse.Open(conf.Aggregation.ClickHouse.GetClientOptions()) if err != nil { return nil, fmt.Errorf("init clickhouse client: %w", err) } // Initialize streaming connector - streaming, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - Logger: logger, - ClickHouse: clickhouseClient, - Database: config.Aggregation.ClickHouse.Database, - AsyncInsert: config.Sink.Storage.AsyncInsert, - AsyncInsertWait: config.Sink.Storage.AsyncInsertWait, - InsertQuerySettings: config.Sink.Storage.QuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse streaming connector: %w", err) + var ( + streaming streaming.Connector + ) + + switch conf.Aggregation.Engine { + case config.AggregationEngineClickHouseRaw: + streaming, err = clickhouse_connector_raw.NewClickhouseConnector(ctx, clickhouse_connector_raw.ClickhouseConnectorConfig{ + ClickHouse: clickhouseClient, + Database: conf.Aggregation.ClickHouse.Database, + Logger: logger, + AsyncInsert: conf.Sink.Storage.AsyncInsert, + AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, + InsertQuerySettings: conf.Sink.Storage.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse raw engine: %w", err) + } + + case config.AggregationEngineClickHouseMV: + streaming, err = clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ + ClickHouse: clickhouseClient, + Database: conf.Aggregation.ClickHouse.Database, + Logger: logger, + PopulateMeter: conf.Aggregation.PopulateMeter, + CreateOrReplaceMeter: conf.Aggregation.CreateOrReplaceMeter, + QueryRawEvents: conf.Aggregation.QueryRawEvents, + AsyncInsert: conf.Sink.Storage.AsyncInsert, + AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, + InsertQuerySettings: conf.Sink.Storage.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse mv engine: %w", err) + } + case config.AggregationEngineClickHouseParse: + streaming, err = clickhouse_connector_parse.NewClickhouseConnector(ctx, clickhouse_connector_parse.ClickhouseConnectorConfig{ + ClickHouse: clickhouseClient, + Database: conf.Aggregation.ClickHouse.Database, + Logger: logger, + AsyncInsert: conf.Sink.Storage.AsyncInsert, + AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, + InsertQuerySettings: conf.Sink.Storage.QuerySettings, + }) + if err != nil { + return nil, fmt.Errorf("init clickhouse parse engine: %w", err) + } + default: + return nil, fmt.Errorf("invalid aggregation engine: %s", conf.Aggregation.Engine) } // Initialize deduplicator if enabled var deduplicator dedupe.Deduplicator - if config.Sink.Dedupe.Enabled { - deduplicator, err = config.Sink.Dedupe.NewDeduplicator() + if conf.Sink.Dedupe.Enabled { + deduplicator, err = conf.Sink.Dedupe.NewDeduplicator() if err != nil { return nil, fmt.Errorf("failed to initialize deduplicator: %w", err) } @@ -175,7 +215,7 @@ func initSink(ctx context.Context, config config.Configuration, logger *slog.Log // Initialize Kafka consumer - consumerConfig := config.Sink.Kafka.AsConsumerConfig() + consumerConfig := conf.Sink.Kafka.AsConsumerConfig() // Override following Kafka consumer configuration parameters with hardcoded values as the Sink implementation relies on // these to be set to a specific value. @@ -210,7 +250,7 @@ func initSink(ctx context.Context, config config.Configuration, logger *slog.Log // Enable Kafka client logging go pkgkafka.ConsumeLogChannel(consumer, logger.WithGroup("kafka").WithGroup("consumer")) - topicResolver, err := topicresolver.NewNamespacedTopicResolver(config.Ingest.Kafka.EventsTopicTemplate) + topicResolver, err := topicresolver.NewNamespacedTopicResolver(conf.Ingest.Kafka.EventsTopicTemplate) if err != nil { return nil, fmt.Errorf("failed to create topic name resolver: %w", err) } @@ -223,16 +263,16 @@ func initSink(ctx context.Context, config config.Configuration, logger *slog.Log Storage: storage, Deduplicator: deduplicator, Consumer: consumer, - MinCommitCount: config.Sink.MinCommitCount, - MaxCommitWait: config.Sink.MaxCommitWait, - MaxPollTimeout: config.Sink.MaxPollTimeout, - FlushSuccessTimeout: config.Sink.FlushSuccessTimeout, - DrainTimeout: config.Sink.DrainTimeout, - NamespaceRefetch: config.Sink.NamespaceRefetch, + MinCommitCount: conf.Sink.MinCommitCount, + MaxCommitWait: conf.Sink.MaxCommitWait, + MaxPollTimeout: conf.Sink.MaxPollTimeout, + FlushSuccessTimeout: conf.Sink.FlushSuccessTimeout, + DrainTimeout: conf.Sink.DrainTimeout, + NamespaceRefetch: conf.Sink.NamespaceRefetch, FlushEventHandler: flushHandler, TopicResolver: topicResolver, - NamespaceRefetchTimeout: config.Sink.NamespaceRefetchTimeout, - NamespaceTopicRegexp: config.Sink.NamespaceTopicRegexp, + NamespaceRefetchTimeout: conf.Sink.NamespaceRefetchTimeout, + NamespaceTopicRegexp: conf.Sink.NamespaceTopicRegexp, } return sink.NewSink(sinkConfig) diff --git a/openmeter/server/server_test.go b/openmeter/server/server_test.go index a520d55eb..42f7cfa1f 100644 --- a/openmeter/server/server_test.go +++ b/openmeter/server/server_test.go @@ -75,7 +75,11 @@ var ( type MockStreamingConnector struct{} -func (c *MockStreamingConnector) Init(ctx context.Context) error { +func (c *MockStreamingConnector) CreateNamespace(ctx context.Context, namespace string) error { + return nil +} + +func (c *MockStreamingConnector) DeleteNamespace(ctx context.Context, namespace string) error { return nil } diff --git a/openmeter/streaming/connector.go b/openmeter/streaming/connector.go index 6710b8bd9..3fd810873 100644 --- a/openmeter/streaming/connector.go +++ b/openmeter/streaming/connector.go @@ -5,6 +5,7 @@ import ( "time" "github.com/openmeterio/openmeter/api" + "github.com/openmeterio/openmeter/openmeter/namespace" "github.com/openmeterio/openmeter/pkg/models" ) @@ -59,6 +60,8 @@ type MeterEvent struct { } type Connector interface { + namespace.Handler + CountEvents(ctx context.Context, namespace string, params CountEventsParams) ([]CountEventRow, error) ListEvents(ctx context.Context, namespace string, params ListEventsParams) ([]api.IngestedEvent, error) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error diff --git a/openmeter/streaming/testutils/streaming.go b/openmeter/streaming/testutils/streaming.go index d396d700a..01483c9a6 100644 --- a/openmeter/streaming/testutils/streaming.go +++ b/openmeter/streaming/testutils/streaming.go @@ -51,7 +51,11 @@ func (m *MockStreamingConnector) AddRow(meterSlug string, row models.MeterQueryR m.rows[meterSlug] = append(m.rows[meterSlug], row) } -func (m *MockStreamingConnector) Init(ctx context.Context) error { +func (c *MockStreamingConnector) CreateNamespace(ctx context.Context, namespace string) error { + return nil +} + +func (c *MockStreamingConnector) DeleteNamespace(ctx context.Context, namespace string) error { return nil } From ef3d896cb9e7ae3680d11c6e0da2268dd35293f4 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 12:00:20 -0700 Subject: [PATCH 19/33] feat(config): unify streaming --- app/common/openmeter.go | 44 ++++++++++++------------- app/config/aggregation.go | 23 ++++++++++++++ app/config/sink.go | 2 ++ cmd/sink-worker/main.go | 67 ++++++++++----------------------------- 4 files changed, 61 insertions(+), 75 deletions(-) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index eb30816b3..57ab3b0c8 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -52,14 +52,12 @@ func NewClickHouseStreamingConnector( switch conf.Engine { case config.AggregationEngineClickHouseRaw: connector, err = clickhouse_connector_raw.NewClickhouseConnector(ctx, clickhouse_connector_raw.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Logger: logger, - - // TODO: add insert related config after moved from sink config - // AsyncInsert: conf.Aggregation.AsyncInsert, - // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, - // InsertQuerySettings: conf.Aggregation.QuerySettings, + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + AsyncInsert: conf.AsyncInsert, + AsyncInsertWait: conf.AsyncInsertWait, + InsertQuerySettings: conf.InsertQuerySettings, }) if err != nil { return nil, fmt.Errorf("init clickhouse raw engine: %w", err) @@ -67,31 +65,29 @@ func NewClickHouseStreamingConnector( case config.AggregationEngineClickHouseMV: connector, err = clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Logger: logger, + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + AsyncInsert: conf.AsyncInsert, + AsyncInsertWait: conf.AsyncInsertWait, + InsertQuerySettings: conf.InsertQuerySettings, + + Meters: meterRepository, PopulateMeter: conf.PopulateMeter, CreateOrReplaceMeter: conf.CreateOrReplaceMeter, QueryRawEvents: conf.QueryRawEvents, - - // TODO: add insert related config after moved from sink config - // AsyncInsert: conf.Aggregation.AsyncInsert, - // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, - // InsertQuerySettings: conf.Aggregation.QuerySettings, }) if err != nil { return nil, fmt.Errorf("init clickhouse mv engine: %w", err) } case config.AggregationEngineClickHouseParse: connector, err = clickhouse_connector_parse.NewClickhouseConnector(ctx, clickhouse_connector_parse.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Logger: logger, - - // TODO: add insert related config after moved from sink config - // AsyncInsert: conf.Aggregation.AsyncInsert, - // AsyncInsertWait: conf.Aggregation.AsyncInsertWait, - // InsertQuerySettings: conf.Aggregation.QuerySettings, + ClickHouse: clickHouse, + Database: conf.ClickHouse.Database, + Logger: logger, + AsyncInsert: conf.AsyncInsert, + AsyncInsertWait: conf.AsyncInsertWait, + InsertQuerySettings: conf.InsertQuerySettings, }) if err != nil { return nil, fmt.Errorf("init clickhouse parse engine: %w", err) diff --git a/app/config/aggregation.go b/app/config/aggregation.go index 519b73a06..a23c6256e 100644 --- a/app/config/aggregation.go +++ b/app/config/aggregation.go @@ -37,6 +37,23 @@ type AggregationConfiguration struct { // Engine is the aggregation engine to use Engine AggregationEngine ClickHouse ClickHouseAggregationConfiguration + + // Set true for ClickHouse first store the incoming inserts into an in-memory buffer + // before flushing them regularly to disk. + // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts + AsyncInsert bool + // Set true if you want an insert statement to return with an acknowledgment immediatelyy + // without waiting for the data got inserted into the buffer. + // Setting true can cause silent errors that you need to monitor separately. + AsyncInsertWait bool + + // See https://clickhouse.com/docs/en/operations/settings/settings + // For example, you can set the `max_insert_threads` setting to control the number of threads + // or the `parallel_view_processing` setting to enable pushing to attached views concurrently. + InsertQuerySettings map[string]string + + // Engine specific options + // Populate creates the materialized view with data from the events table // This is not safe to use in production as requires to stop ingestion PopulateMeter bool @@ -61,6 +78,10 @@ func (c AggregationConfiguration) Validate() error { return fmt.Errorf("engine: %w", err) } + if c.AsyncInsertWait && !c.AsyncInsert { + return errors.New("async insert wait is set but async insert is not") + } + // Validate engine specific options if c.Engine != AggregationEngineClickHouseMV { if c.PopulateMeter { @@ -151,6 +172,8 @@ func (c ClickHouseAggregationConfiguration) GetClientOptions() *clickhouse.Optio // ConfigureAggregation configures some defaults in the Viper instance. func ConfigureAggregation(v *viper.Viper) { v.SetDefault("aggregation.engine", AggregationEngineClickHouseMV) + v.SetDefault("aggregation.asyncInsert", false) + v.SetDefault("aggregation.asyncInsertWait", false) v.SetDefault("aggregation.clickhouse.address", "127.0.0.1:9000") v.SetDefault("aggregation.clickhouse.tls", false) diff --git a/app/config/sink.go b/app/config/sink.go index ff0e29c72..e342d5ec6 100644 --- a/app/config/sink.go +++ b/app/config/sink.go @@ -22,6 +22,7 @@ type SinkConfiguration struct { IngestNotifications IngestNotificationsConfiguration // Kafka client/Consumer configuration Kafka KafkaConfig + // TODO: remove, config moved to aggregation config // Storage configuration Storage StorageConfiguration @@ -154,6 +155,7 @@ func ConfigureSink(v *viper.Viper) { v.SetDefault("sink.namespaceRefetchTimeout", "10s") v.SetDefault("sink.namespaceTopicRegexp", "^om_([A-Za-z0-9]+(?:_[A-Za-z0-9]+)*)_events$") + // TODO: remove, config moved to aggregation config // Sink Storage v.SetDefault("sink.storage.asyncInsert", false) v.SetDefault("sink.storage.asyncInsertWait", false) diff --git a/cmd/sink-worker/main.go b/cmd/sink-worker/main.go index c4653cc5c..e3d8f7292 100644 --- a/cmd/sink-worker/main.go +++ b/cmd/sink-worker/main.go @@ -17,16 +17,13 @@ import ( "go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/trace" + "github.com/openmeterio/openmeter/app/common" "github.com/openmeterio/openmeter/app/config" "github.com/openmeterio/openmeter/openmeter/dedupe" "github.com/openmeterio/openmeter/openmeter/ingest/kafkaingest/topicresolver" "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/sink" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" - "github.com/openmeterio/openmeter/openmeter/streaming" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" pkgkafka "github.com/openmeterio/openmeter/pkg/kafka" ) @@ -146,54 +143,22 @@ func initSink(ctx context.Context, conf config.Configuration, logger *slog.Logge return nil, fmt.Errorf("init clickhouse client: %w", err) } - // Initialize streaming connector - var ( - streaming streaming.Connector - ) - - switch conf.Aggregation.Engine { - case config.AggregationEngineClickHouseRaw: - streaming, err = clickhouse_connector_raw.NewClickhouseConnector(ctx, clickhouse_connector_raw.ClickhouseConnectorConfig{ - ClickHouse: clickhouseClient, - Database: conf.Aggregation.ClickHouse.Database, - Logger: logger, - AsyncInsert: conf.Sink.Storage.AsyncInsert, - AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, - InsertQuerySettings: conf.Sink.Storage.QuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse raw engine: %w", err) - } + // Temporary: copy over sink storage settings + // TODO: remove after config migration is over + if conf.Sink.Storage.AsyncInsert { + conf.Aggregation.AsyncInsert = conf.Sink.Storage.AsyncInsert + } + if conf.Sink.Storage.AsyncInsertWait { + conf.Aggregation.AsyncInsertWait = conf.Sink.Storage.AsyncInsertWait + } + if conf.Sink.Storage.QuerySettings != nil { + conf.Aggregation.InsertQuerySettings = conf.Sink.Storage.QuerySettings + } - case config.AggregationEngineClickHouseMV: - streaming, err = clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - ClickHouse: clickhouseClient, - Database: conf.Aggregation.ClickHouse.Database, - Logger: logger, - PopulateMeter: conf.Aggregation.PopulateMeter, - CreateOrReplaceMeter: conf.Aggregation.CreateOrReplaceMeter, - QueryRawEvents: conf.Aggregation.QueryRawEvents, - AsyncInsert: conf.Sink.Storage.AsyncInsert, - AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, - InsertQuerySettings: conf.Sink.Storage.QuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse mv engine: %w", err) - } - case config.AggregationEngineClickHouseParse: - streaming, err = clickhouse_connector_parse.NewClickhouseConnector(ctx, clickhouse_connector_parse.ClickhouseConnectorConfig{ - ClickHouse: clickhouseClient, - Database: conf.Aggregation.ClickHouse.Database, - Logger: logger, - AsyncInsert: conf.Sink.Storage.AsyncInsert, - AsyncInsertWait: conf.Sink.Storage.AsyncInsertWait, - InsertQuerySettings: conf.Sink.Storage.QuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse parse engine: %w", err) - } - default: - return nil, fmt.Errorf("invalid aggregation engine: %s", conf.Aggregation.Engine) + // Initialize streaming connector + streaming, err := common.NewClickHouseStreamingConnector(ctx, conf.Aggregation, clickhouseClient, meterRepository, logger) + if err != nil { + return nil, fmt.Errorf("init clickhouse streaming connector: %w", err) } // Initialize deduplicator if enabled From 9ca22ab1eee29486c1495aec8ae5053ff36ea080 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 12:05:05 -0700 Subject: [PATCH 20/33] refactor(wire): rename factory --- app/common/openmeter.go | 2 +- app/common/wire.go | 2 +- cmd/balance-worker/wire_gen.go | 2 +- cmd/notification-service/wire_gen.go | 2 +- cmd/server/wire_gen.go | 2 +- cmd/sink-worker/main.go | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 57ab3b0c8..40af42e9f 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -37,7 +37,7 @@ func NewMeterRepository(meters []*models.Meter) *meter.InMemoryRepository { return meter.NewInMemoryRepository(slicesx.Map(meters, lo.FromPtr[models.Meter])) } -func NewClickHouseStreamingConnector( +func NewStreamingConnector( ctx context.Context, conf config.AggregationConfiguration, clickHouse clickhouse.Conn, diff --git a/app/common/wire.go b/app/common/wire.go index 54b323899..b12fdad29 100644 --- a/app/common/wire.go +++ b/app/common/wire.go @@ -100,7 +100,7 @@ var OpenMeter = wire.NewSet( NewMeterRepository, wire.Bind(new(meter.Repository), new(*meter.InMemoryRepository)), - NewClickHouseStreamingConnector, + NewStreamingConnector, NewNamespacedTopicResolver, wire.Bind(new(topicresolver.Resolver), new(*topicresolver.NamespacedTopicResolver)), diff --git a/cmd/balance-worker/wire_gen.go b/cmd/balance-worker/wire_gen.go index e1e1215e2..e5238a13b 100644 --- a/cmd/balance-worker/wire_gen.go +++ b/cmd/balance-worker/wire_gen.go @@ -134,7 +134,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v3 := conf.Meters inMemoryRepository := common.NewMeterRepository(v3) - connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v2, inMemoryRepository, logger) + connector, err := common.NewStreamingConnector(ctx, aggregationConfiguration, v2, inMemoryRepository, logger) if err != nil { cleanup5() cleanup4() diff --git a/cmd/notification-service/wire_gen.go b/cmd/notification-service/wire_gen.go index d452f03e2..7551d8a92 100644 --- a/cmd/notification-service/wire_gen.go +++ b/cmd/notification-service/wire_gen.go @@ -74,7 +74,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) + connector, err := common.NewStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() diff --git a/cmd/server/wire_gen.go b/cmd/server/wire_gen.go index c9867f1e5..2c4b5f045 100644 --- a/cmd/server/wire_gen.go +++ b/cmd/server/wire_gen.go @@ -78,7 +78,7 @@ func initializeApplication(ctx context.Context, conf config.Configuration) (Appl } v2 := conf.Meters inMemoryRepository := common.NewMeterRepository(v2) - connector, err := common.NewClickHouseStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) + connector, err := common.NewStreamingConnector(ctx, aggregationConfiguration, v, inMemoryRepository, logger) if err != nil { cleanup4() cleanup3() diff --git a/cmd/sink-worker/main.go b/cmd/sink-worker/main.go index e3d8f7292..e7c346a19 100644 --- a/cmd/sink-worker/main.go +++ b/cmd/sink-worker/main.go @@ -156,7 +156,7 @@ func initSink(ctx context.Context, conf config.Configuration, logger *slog.Logge } // Initialize streaming connector - streaming, err := common.NewClickHouseStreamingConnector(ctx, conf.Aggregation, clickhouseClient, meterRepository, logger) + streaming, err := common.NewStreamingConnector(ctx, conf.Aggregation, clickhouseClient, meterRepository, logger) if err != nil { return nil, fmt.Errorf("init clickhouse streaming connector: %w", err) } From d121fa26dda3cbd128b683103eb0db1023799caa Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 12:14:51 -0700 Subject: [PATCH 21/33] test(config): fix --- app/config/config_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/app/config/config_test.go b/app/config/config_test.go index f7138841e..106b062c0 100644 --- a/app/config/config_test.go +++ b/app/config/config_test.go @@ -124,6 +124,9 @@ func TestComplete(t *testing.T) { ConnMaxLifetime: 10 * time.Minute, BlockBufferSize: 10, }, + Engine: AggregationEngineClickHouseMV, + AsyncInsert: false, + AsyncInsertWait: false, }, Sink: SinkConfiguration{ GroupId: "openmeter-sink-worker", From 080da752269a9f2847a3c6097b46b416f6e79ee9 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 12:25:11 -0700 Subject: [PATCH 22/33] refactor(connector): gci --- openmeter/streaming/clickhouse_connector/meter_query.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index aa63d7aef..9b722e6b2 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -8,8 +8,6 @@ import ( "time" "github.com/huandu/go-sqlbuilder" - - raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" "github.com/openmeterio/openmeter/pkg/models" "github.com/openmeterio/openmeter/pkg/slicesx" From 96ce765aa5b4c27489e242ba248cf47bf21af3b6 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 12:31:29 -0700 Subject: [PATCH 23/33] refactor(connector): gci --- openmeter/streaming/clickhouse_connector/meter_query.go | 1 + 1 file changed, 1 insertion(+) diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector/meter_query.go index 9b722e6b2..86bc52d51 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector/meter_query.go @@ -8,6 +8,7 @@ import ( "time" "github.com/huandu/go-sqlbuilder" + raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" "github.com/openmeterio/openmeter/pkg/models" "github.com/openmeterio/openmeter/pkg/slicesx" From 6c00d3fffd0461720f6472f94539783f3a674531 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 14:24:03 -0700 Subject: [PATCH 24/33] test(connector): insert meter event --- .../meter_event_query_test.go | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go new file mode 100644 index 000000000..0c3be2d8b --- /dev/null +++ b/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go @@ -0,0 +1,68 @@ +package clickhouse_connector_parse + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/openmeterio/openmeter/openmeter/streaming" +) + +func TestInsertMeterEventsQuery(t *testing.T) { + now := time.Now() + + rawEvent := streaming.RawEvent{ + Namespace: "my_namespace", + ID: "1", + Source: "source", + Subject: "subject-1", + Time: now, + StoredAt: now, + IngestedAt: now, + Type: "api-calls", + Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, + } + + query := InsertMeterEventsQuery{ + Database: "database", + QuerySettings: map[string]string{ + "parallel_view_processing": "1", + "max_insert_threads": "2", + }, + MeterEvents: []streaming.MeterEvent{ + { + RawEvent: rawEvent, + Meter: "api_request_duration", + Value: 100.0, + GroupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, + }, + { + RawEvent: rawEvent, + Meter: "api_request_total", + Value: 1.0, + GroupBy: map[string]string{ + "method": "GET", + "path": "/api/v1", + }, + }, + }, + } + + sql, args := query.ToSQL() + + assert.Equal(t, []interface{}{ + // First Meter Event + "my_namespace", now, "api_request_duration", "subject-1", 100.0, "", + map[string]string{"method": "GET", "path": "/api/v1"}, + "1", "source", "api-calls", now, now, + // Second Meter Event + "my_namespace", now, "api_request_total", "subject-1", 1.0, "", + map[string]string{"method": "GET", "path": "/api/v1"}, + "1", "source", "api-calls", now, now, + }, args) + assert.Equal(t, `INSERT INTO database.om_meter_events (namespace, time, meter, subject, value, value_str, group_by, event_id, event_source, event_type, ingested_at, stored_at) SETTINGS parallel_view_processing = 1, max_insert_threads = 2 VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) +} From c125900e097e860c40a1b59f83173130c95dfdab Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 16:43:30 -0700 Subject: [PATCH 25/33] feat(entitlement): use streaming connector factiry --- app/common/openmeter.go | 4 ++-- cmd/jobs/entitlement/init.go | 11 ++--------- .../connector.go | 2 +- .../meter_query.go | 2 +- .../meter_query_test.go | 2 +- 5 files changed, 7 insertions(+), 14 deletions(-) rename openmeter/streaming/{clickhouse_connector => clickhouse_connector_mv}/connector.go (99%) rename openmeter/streaming/{clickhouse_connector => clickhouse_connector_mv}/meter_query.go (99%) rename openmeter/streaming/{clickhouse_connector => clickhouse_connector_mv}/meter_query_test.go (99%) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 40af42e9f..e56ac6c22 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -22,7 +22,7 @@ import ( "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler/ingestnotification" "github.com/openmeterio/openmeter/openmeter/streaming" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_mv" "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" @@ -64,7 +64,7 @@ func NewStreamingConnector( } case config.AggregationEngineClickHouseMV: - connector, err = clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ + connector, err = clickhouse_connector_mv.NewClickhouseConnector(ctx, clickhouse_connector_mv.ClickhouseConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, Logger: logger, diff --git a/cmd/jobs/entitlement/init.go b/cmd/jobs/entitlement/init.go index fb119dc6c..2a77c375f 100644 --- a/cmd/jobs/entitlement/init.go +++ b/cmd/jobs/entitlement/init.go @@ -8,11 +8,11 @@ import ( "github.com/ClickHouse/clickhouse-go/v2" "go.opentelemetry.io/otel/metric" + "github.com/openmeterio/openmeter/app/common" "github.com/openmeterio/openmeter/app/config" "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/registry" registrybuilder "github.com/openmeterio/openmeter/openmeter/registry/builder" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/eventbus" entdriver "github.com/openmeterio/openmeter/pkg/framework/entutils/entdriver" @@ -50,14 +50,7 @@ func initEntitlements(ctx context.Context, conf config.Configuration, logger *sl return nil, fmt.Errorf("failed to initialize clickhouse client: %w", err) } - streamingConnector, err := clickhouse_connector.NewClickhouseConnector(ctx, clickhouse_connector.ClickhouseConnectorConfig{ - Logger: logger, - ClickHouse: clickHouseClient, - Database: conf.Aggregation.ClickHouse.Database, - Meters: meterRepository, - CreateOrReplaceMeter: conf.Aggregation.CreateOrReplaceMeter, - PopulateMeter: conf.Aggregation.PopulateMeter, - }) + streamingConnector, err := common.NewStreamingConnector(ctx, conf.Aggregation, clickHouseClient, meterRepository, logger) if err != nil { return nil, fmt.Errorf("init clickhouse streaming: %w", err) } diff --git a/openmeter/streaming/clickhouse_connector/connector.go b/openmeter/streaming/clickhouse_connector_mv/connector.go similarity index 99% rename from openmeter/streaming/clickhouse_connector/connector.go rename to openmeter/streaming/clickhouse_connector_mv/connector.go index c2e798da2..9a860953e 100644 --- a/openmeter/streaming/clickhouse_connector/connector.go +++ b/openmeter/streaming/clickhouse_connector_mv/connector.go @@ -1,4 +1,4 @@ -package clickhouse_connector +package clickhouse_connector_mv import ( "context" diff --git a/openmeter/streaming/clickhouse_connector/meter_query.go b/openmeter/streaming/clickhouse_connector_mv/meter_query.go similarity index 99% rename from openmeter/streaming/clickhouse_connector/meter_query.go rename to openmeter/streaming/clickhouse_connector_mv/meter_query.go index 86bc52d51..45b016c47 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_mv/meter_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector +package clickhouse_connector_mv import ( _ "embed" diff --git a/openmeter/streaming/clickhouse_connector/meter_query_test.go b/openmeter/streaming/clickhouse_connector_mv/meter_query_test.go similarity index 99% rename from openmeter/streaming/clickhouse_connector/meter_query_test.go rename to openmeter/streaming/clickhouse_connector_mv/meter_query_test.go index 9a3bee96c..f38f0b4f0 100644 --- a/openmeter/streaming/clickhouse_connector/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_mv/meter_query_test.go @@ -1,4 +1,4 @@ -package clickhouse_connector +package clickhouse_connector_mv import ( "testing" From b82e021f4918480d320102e55fe38c281eb551cd Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Mon, 21 Oct 2024 20:00:42 -0700 Subject: [PATCH 26/33] fix(streaming): query raw --- .../clickhouse_connector_raw/meter_query.go | 2 +- .../meter_query_test.go | 65 +++++++++++-------- 2 files changed, 40 insertions(+), 27 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse_connector_raw/meter_query.go index 695b8adb0..c1b4bad0e 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query.go @@ -132,7 +132,7 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { query.Select(selectColumns...) query.From(tableName) query.Where(query.Equal(getColumn("namespace"), d.Namespace)) - query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) + query.Where(query.Equal(getColumn("type"), d.Meter.EventType)) if len(d.Subject) > 0 { mapFunc := func(subject string) string { diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go index d919bfcac..63fe89e99 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go @@ -27,6 +27,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -40,8 +41,8 @@ func TestQueryMeter(t *testing.T) { GroupBy: []string{"subject", "group1", "group2"}, WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (om_events.subject = ?) AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "event1", "subject1", from.Unix(), to.Unix()}, }, { // Aggregate all available data query: queryMeter{ @@ -49,6 +50,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -57,8 +59,8 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ?", + wantArgs: []interface{}{"my_namespace", "event1"}, }, { // Aggregate with count aggregation query: queryMeter{ @@ -66,6 +68,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationCount, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -74,8 +77,8 @@ func TestQueryMeter(t *testing.T) { }, }, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, count(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ?", + wantArgs: []interface{}{"my_namespace", "event1"}, }, { // Aggregate data from start query: queryMeter{ @@ -83,6 +86,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -92,8 +96,8 @@ func TestQueryMeter(t *testing.T) { }, From: &from, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND om_events.time >= ?", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix()}, }, { // Aggregate data between period query: queryMeter{ @@ -101,6 +105,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -111,8 +116,8 @@ func TestQueryMeter(t *testing.T) { From: &from, To: &to, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND om_events.time >= ? AND om_events.time <= ?", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix(), to.Unix()}, }, { // Aggregate data between period, groupped by window size query: queryMeter{ @@ -120,6 +125,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -131,8 +137,8 @@ func TestQueryMeter(t *testing.T) { To: &to, WindowSize: &windowSize, }, - wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix(), to.Unix()}, }, { // Aggregate data between period in a different timezone, groupped by window size query: queryMeter{ @@ -140,6 +146,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -152,8 +159,8 @@ func TestQueryMeter(t *testing.T) { WindowSize: &windowSize, WindowTimeZone: tz, }, - wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, + wantSQL: "SELECT tumbleStart(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND om_events.time >= ? AND om_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", + wantArgs: []interface{}{"my_namespace", "event1", from.Unix(), to.Unix()}, }, { // Aggregate data for a single subject query: queryMeter{ @@ -161,6 +168,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -171,8 +179,8 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (om_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "event1", "subject1"}, }, { // Aggregate data for a single subject and group by additional fields query: queryMeter{ @@ -180,6 +188,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -190,8 +199,8 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject}, GroupBy: []string{"subject", "group1", "group2"}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject, JSON_VALUE(om_events.data, '$.group1') as group1, JSON_VALUE(om_events.data, '$.group2') as group2 FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (om_events.subject = ?) GROUP BY subject, group1, group2", + wantArgs: []interface{}{"my_namespace", "event1", "subject1"}, }, { // Aggregate data for a multiple subjects query: queryMeter{ @@ -199,6 +208,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -209,8 +219,8 @@ func TestQueryMeter(t *testing.T) { Subject: []string{subject, "subject2"}, GroupBy: []string{"subject"}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value, om_events.subject FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (om_events.subject = ? OR om_events.subject = ?) GROUP BY subject", + wantArgs: []interface{}{"my_namespace", "event1", "subject1", "subject2"}, }, { // Aggregate data with filtering for a single group and single value query: queryMeter{ @@ -218,6 +228,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -227,8 +238,8 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1')", + wantArgs: []interface{}{"my_namespace", "event1"}, }, { // Aggregate data with filtering for a single group and multiple values query: queryMeter{ @@ -236,6 +247,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -245,8 +257,8 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2')", + wantArgs: []interface{}{"my_namespace", "event1"}, }, { // Aggregate data with filtering for multiple groups and multiple values query: queryMeter{ @@ -254,6 +266,7 @@ func TestQueryMeter(t *testing.T) { Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", + EventType: "event1", Aggregation: models.MeterAggregationSum, ValueProperty: "$.value", GroupBy: map[string]string{ @@ -263,8 +276,8 @@ func TestQueryMeter(t *testing.T) { }, FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, }, - wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.meter = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", - wantArgs: []interface{}{"my_namespace", "meter1"}, + wantSQL: "SELECT tumbleStart(min(om_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_events.time), toIntervalMinute(1)) AS windowend, sum(cast(JSON_VALUE(om_events.data, '$.value'), 'Float64')) AS value FROM openmeter.om_events WHERE om_events.namespace = ? AND om_events.type = ? AND (JSON_VALUE(om_events.data, '$.group1') = 'g1v1' OR JSON_VALUE(om_events.data, '$.group1') = 'g1v2') AND (JSON_VALUE(om_events.data, '$.group2') = 'g2v1' OR JSON_VALUE(om_events.data, '$.group2') = 'g2v2')", + wantArgs: []interface{}{"my_namespace", "event1"}, }, } From a9866d506c87dfe82439fc669e986a9c11d59788 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Tue, 22 Oct 2024 13:27:15 -0700 Subject: [PATCH 27/33] fix(connector): table order --- .../clickhouse_connector_parse/meter_event_query.go | 8 +++++++- .../streaming/clickhouse_connector_raw/event_query.go | 8 +++++++- .../clickhouse_connector_raw/event_query_test.go | 2 +- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go index 7bb5f5d83..4eec35aee 100644 --- a/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go +++ b/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go @@ -46,7 +46,13 @@ func (d createMeterEventTable) toSQL() string { sb.Define("stored_at", "DateTime") sb.SQL("ENGINE = MergeTree") sb.SQL("PARTITION BY toYYYYMM(time)") - sb.SQL("ORDER BY (namespace, time, meter, subject)") + // Lowest cardinality columns we always filter on goes to the most left. + // ClickHouse always picks partition first so we always filter time by month. + // Theoritically we could add toStartOfHour(time) to the order sooner than subject + // but we bet on that a typical namespace has more subjects than hours in a month. + // Subject is an optional filter so it won't always help to reduce number of rows scanned. + // Finally we add time not just to speed up queries but also to keep data on the disk together. + sb.SQL("ORDER BY (namespace, meter, subject, toStartOfHour(time), group_by)") sql, _ := sb.Build() return sql diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query.go b/openmeter/streaming/clickhouse_connector_raw/event_query.go index 4c3bc353e..2daae40c2 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query.go +++ b/openmeter/streaming/clickhouse_connector_raw/event_query.go @@ -36,7 +36,13 @@ func (d createEventsTable) toSQL() string { sb.Define("stored_at", "DateTime") sb.SQL("ENGINE = MergeTree") sb.SQL("PARTITION BY toYYYYMM(time)") - sb.SQL("ORDER BY (namespace, time, type, subject)") + // Lowest cardinality columns we always filter on goes to the most left. + // ClickHouse always picks partition first so we always filter time by month. + // Theoritically we could add toStartOfHour(time) to the order sooner than subject + // but we bet on that a typical namespace has more subjects than hours in a month. + // Subject is an optional filter so it won't always help to reduce number of rows scanned. + // Finally we add time not just to speed up queries but also to keep data on the disk together. + sb.SQL("ORDER BY (namespace, type, subject, toStartOfHour(time))") sql, _ := sb.Build() return sql diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go index 231b6ef1e..848677132 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go +++ b/openmeter/streaming/clickhouse_connector_raw/event_query_test.go @@ -18,7 +18,7 @@ func TestCreateEventsTable(t *testing.T) { data: createEventsTable{ Database: "openmeter", }, - want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, time, type, subject)", + want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, type, subject, toStartOfHour(time))", }, } From cbf6404653639448bb4dcbb24ce7e3c4a71d006e Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Tue, 22 Oct 2024 18:22:38 -0700 Subject: [PATCH 28/33] feat(connector): delete meter parse connector --- app/common/openmeter.go | 13 - app/config/aggregation.go | 6 +- app/config/sink.go | 2 +- cmd/server/main.go | 5 - openmeter/server/server_test.go | 2 +- openmeter/sink/models/models.go | 10 +- openmeter/sink/namespaces.go | 20 +- openmeter/sink/storage.go | 16 +- .../clickhouse_connector_mv/connector.go | 4 +- .../clickhouse_connector_parse/connector.go | 325 ------------------ .../meter_event_query.go | 122 ------- .../meter_event_query_test.go | 68 ---- .../clickhouse_connector_parse/meter_query.go | 206 ----------- .../meter_query_test.go | 271 --------------- .../clickhouse_connector_raw/connector.go | 2 +- openmeter/streaming/connector.go | 11 +- openmeter/streaming/testutils/streaming.go | 2 +- 17 files changed, 12 insertions(+), 1073 deletions(-) delete mode 100644 openmeter/streaming/clickhouse_connector_parse/connector.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/meter_event_query.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/meter_query.go delete mode 100644 openmeter/streaming/clickhouse_connector_parse/meter_query_test.go diff --git a/app/common/openmeter.go b/app/common/openmeter.go index e56ac6c22..3128e5cea 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -23,7 +23,6 @@ import ( "github.com/openmeterio/openmeter/openmeter/sink/flushhandler/ingestnotification" "github.com/openmeterio/openmeter/openmeter/streaming" "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_mv" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_parse" "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/driver/noop" @@ -80,18 +79,6 @@ func NewStreamingConnector( if err != nil { return nil, fmt.Errorf("init clickhouse mv engine: %w", err) } - case config.AggregationEngineClickHouseParse: - connector, err = clickhouse_connector_parse.NewClickhouseConnector(ctx, clickhouse_connector_parse.ClickhouseConnectorConfig{ - ClickHouse: clickHouse, - Database: conf.ClickHouse.Database, - Logger: logger, - AsyncInsert: conf.AsyncInsert, - AsyncInsertWait: conf.AsyncInsertWait, - InsertQuerySettings: conf.InsertQuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("init clickhouse parse engine: %w", err) - } default: return nil, fmt.Errorf("invalid aggregation engine: %s", conf.Engine) } diff --git a/app/config/aggregation.go b/app/config/aggregation.go index a23c6256e..4cf3e49ae 100644 --- a/app/config/aggregation.go +++ b/app/config/aggregation.go @@ -18,12 +18,10 @@ const ( AggregationEngineClickHouseRaw AggregationEngine = "clickhouse_raw" // MV engine maintains and queries materialized views AggregationEngineClickHouseMV AggregationEngine = "clickhouse_mv" - // Parse engine parses events by meters and stores them in a separate unified meter events table - AggregationEngineClickHouseParse AggregationEngine = "clickhouse_parse" ) func (e AggregationEngine) Values() []AggregationEngine { - return []AggregationEngine{AggregationEngineClickHouseRaw, AggregationEngineClickHouseMV, AggregationEngineClickHouseParse} + return []AggregationEngine{AggregationEngineClickHouseRaw, AggregationEngineClickHouseMV} } func (e AggregationEngine) Validate() error { @@ -42,7 +40,7 @@ type AggregationConfiguration struct { // before flushing them regularly to disk. // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts AsyncInsert bool - // Set true if you want an insert statement to return with an acknowledgment immediatelyy + // Set true if you want an insert statement to return with an acknowledgment immediately // without waiting for the data got inserted into the buffer. // Setting true can cause silent errors that you need to monitor separately. AsyncInsertWait bool diff --git a/app/config/sink.go b/app/config/sink.go index e342d5ec6..0a89f08c0 100644 --- a/app/config/sink.go +++ b/app/config/sink.go @@ -103,7 +103,7 @@ type StorageConfiguration struct { // before flushing them regularly to disk. // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts AsyncInsert bool - // Set true if you want an insert statement to return with an acknowledgment immediatelyy + // Set true if you want an insert statement to return with an acknowledgment immediately // without waiting for the data got inserted into the buffer. // Setting true can cause silent errors that you need to monitor separately. AsyncInsertWait bool diff --git a/cmd/server/main.go b/cmd/server/main.go index 8eef91969..ea66b1497 100644 --- a/cmd/server/main.go +++ b/cmd/server/main.go @@ -336,11 +336,6 @@ func main() { }) for _, meter := range conf.Meters { - if meter == nil { - logger.Error("meter configuration is nil") - os.Exit(1) - } - err := app.StreamingConnector.CreateMeter(ctx, app.NamespaceManager.GetDefaultNamespace(), *meter) if err != nil { slog.Warn("failed to initialize meter", "error", err) diff --git a/openmeter/server/server_test.go b/openmeter/server/server_test.go index 42f7cfa1f..6c29cd5eb 100644 --- a/openmeter/server/server_test.go +++ b/openmeter/server/server_test.go @@ -118,7 +118,7 @@ func (c *MockStreamingConnector) ListMeterSubjects(ctx context.Context, namespac return []string{"s1"}, nil } -func (c *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { +func (c *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent) error { return nil } diff --git a/openmeter/sink/models/models.go b/openmeter/sink/models/models.go index 3d0d372e2..3db49cd3b 100644 --- a/openmeter/sink/models/models.go +++ b/openmeter/sink/models/models.go @@ -15,15 +15,7 @@ type SinkMessage struct { Serialized *serializer.CloudEventsKafkaPayload Status ProcessingStatus // Meters contains the list of meters this message affects - Meters []models.Meter - MeterEvents []MeterEvent -} - -type MeterEvent struct { - Meter *models.Meter - Value float64 - ValueString string - GroupBy map[string]string + Meters []models.Meter } type ProcessingState int8 diff --git a/openmeter/sink/namespaces.go b/openmeter/sink/namespaces.go index 7108f7a54..5e1c78bbc 100644 --- a/openmeter/sink/namespaces.go +++ b/openmeter/sink/namespaces.go @@ -73,7 +73,7 @@ func (n *NamespaceStore) ValidateEvent(_ context.Context, m *sinkmodels.SinkMess } // Parse event with meter - value, valueString, groupBy, err := ommeter.ParseEvent(meter, event) + _, _, _, err = ommeter.ParseEvent(meter, event) if err != nil { m.Status = sinkmodels.ProcessingStatus{ State: sinkmodels.INVALID, @@ -82,24 +82,6 @@ func (n *NamespaceStore) ValidateEvent(_ context.Context, m *sinkmodels.SinkMess return } - - // Create meter event - meterEvent := sinkmodels.MeterEvent{ - Meter: &meter, - GroupBy: groupBy, - } - - // Meterring numeric value - if value != nil { - meterEvent.Value = *value - } - - // Meterring string value - if valueString != nil { - meterEvent.ValueString = *valueString - } - - m.MeterEvents = append(m.MeterEvents, meterEvent) } } diff --git a/openmeter/sink/storage.go b/openmeter/sink/storage.go index 41626e37a..a33650a9d 100644 --- a/openmeter/sink/storage.go +++ b/openmeter/sink/storage.go @@ -42,7 +42,6 @@ type ClickHouseStorage struct { // BatchInsert inserts multiple messages into ClickHouse. func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmodels.SinkMessage) error { var rawEvents []streaming.RawEvent - var meterEvents []streaming.MeterEvent for _, message := range messages { var eventErr string @@ -81,22 +80,9 @@ func (c *ClickHouseStorage) BatchInsert(ctx context.Context, messages []sinkmode } rawEvents = append(rawEvents, rawEvent) - - // Meter events per meter - for _, meterEvent := range message.MeterEvents { - meterEvent := streaming.MeterEvent{ - RawEvent: rawEvent, - Meter: meterEvent.Meter.GetID(), - Value: meterEvent.Value, - ValueString: meterEvent.ValueString, - GroupBy: meterEvent.GroupBy, - } - - meterEvents = append(meterEvents, meterEvent) - } } - if err := c.config.Streaming.BatchInsert(ctx, rawEvents, meterEvents); err != nil { + if err := c.config.Streaming.BatchInsert(ctx, rawEvents); err != nil { return fmt.Errorf("failed to store events: %w", err) } diff --git a/openmeter/streaming/clickhouse_connector_mv/connector.go b/openmeter/streaming/clickhouse_connector_mv/connector.go index 9a860953e..3bf6a8d70 100644 --- a/openmeter/streaming/clickhouse_connector_mv/connector.go +++ b/openmeter/streaming/clickhouse_connector_mv/connector.go @@ -94,8 +94,8 @@ func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { - return c.rawEventConnector.BatchInsert(ctx, rawEvents, meterEvents) +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { + return c.rawEventConnector.BatchInsert(ctx, rawEvents) } func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { diff --git a/openmeter/streaming/clickhouse_connector_parse/connector.go b/openmeter/streaming/clickhouse_connector_parse/connector.go deleted file mode 100644 index fa02a061a..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/connector.go +++ /dev/null @@ -1,325 +0,0 @@ -package clickhouse_connector_parse - -import ( - "context" - "fmt" - "log/slog" - "strings" - "time" - - "github.com/ClickHouse/clickhouse-go/v2" - "github.com/shopspring/decimal" - - "github.com/openmeterio/openmeter/api" - "github.com/openmeterio/openmeter/openmeter/streaming" - raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" - "github.com/openmeterio/openmeter/pkg/models" -) - -var _ streaming.Connector = (*ClickhouseConnector)(nil) - -// ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. -type ClickhouseConnector struct { - config ClickhouseConnectorConfig - rawEventConnector *raw_event_connector.ClickhouseConnector -} - -type ClickhouseConnectorConfig struct { - Logger *slog.Logger - ClickHouse clickhouse.Conn - Database string - AsyncInsert bool - AsyncInsertWait bool - InsertQuerySettings map[string]string -} - -func (c ClickhouseConnectorConfig) Validate() error { - if c.Logger == nil { - return fmt.Errorf("logger is required") - } - - if c.ClickHouse == nil { - return fmt.Errorf("clickhouse connection is required") - } - - if c.Database == "" { - return fmt.Errorf("database is required") - } - - return nil -} - -func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { - if err := config.Validate(); err != nil { - return nil, fmt.Errorf("validate config: %w", err) - } - - rawEventConnector, err := raw_event_connector.NewClickhouseConnector(ctx, raw_event_connector.ClickhouseConnectorConfig{ - Logger: config.Logger, - ClickHouse: config.ClickHouse, - Database: config.Database, - AsyncInsert: config.AsyncInsert, - AsyncInsertWait: config.AsyncInsertWait, - InsertQuerySettings: config.InsertQuerySettings, - }) - if err != nil { - return nil, fmt.Errorf("create raw event connector: %w", err) - } - - connector := &ClickhouseConnector{ - config: config, - rawEventConnector: rawEventConnector, - } - - err = connector.createMeterEventTable(ctx) - if err != nil { - return nil, fmt.Errorf("create meter events table in clickhouse: %w", err) - } - - return connector, nil -} - -func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { - return nil -} - -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { - // We don't delete the event tables as it it reused between namespaces - return nil -} - -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { - // Insert raw events - err := c.rawEventConnector.BatchInsert(ctx, rawEvents, meterEvents) - if err != nil { - return fmt.Errorf("failed to batch insert raw events: %w", err) - } - - // NOTE: The two inserts are not atomic. - // If the second insert fails, the first insert will not be rolled back. - - // Insert meter events - if len(meterEvents) == 0 { - return nil - } - - query := InsertMeterEventsQuery{ - Database: c.config.Database, - MeterEvents: meterEvents, - QuerySettings: c.config.InsertQuerySettings, - } - sql, args := query.ToSQL() - - if c.config.AsyncInsert { - err = c.config.ClickHouse.AsyncInsert(ctx, sql, c.config.AsyncInsertWait, args...) - } else { - err = c.config.ClickHouse.Exec(ctx, sql, args...) - } - - if err != nil { - return fmt.Errorf("failed to batch insert meter events: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { - return c.rawEventConnector.CountEvents(ctx, namespace, params) -} - -func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { - return c.rawEventConnector.ListEvents(ctx, namespace, params) -} - -func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { - // Do nothing - return nil -} - -func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { - // Do nothing - return nil -} - -func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } - - values, err := c.queryMeter(ctx, namespace, meter, params) - if err != nil { - if _, ok := err.(*models.MeterNotFoundError); ok { - return nil, err - } - - return nil, fmt.Errorf("get values: %w", err) - } - - // If the total usage is queried for a single period (no window size), - // replace the window start and end with the period for each row. - // We can still have multiple rows for a single period due to group bys. - if params.WindowSize == nil { - for i := range values { - if params.From != nil { - values[i].WindowStart = *params.From - } - if params.To != nil { - values[i].WindowEnd = *params.To - } - } - } - - return values, nil -} - -func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { - if namespace == "" { - return nil, fmt.Errorf("namespace is required") - } - if meter.Slug == "" { - return nil, fmt.Errorf("meter is required") - } - - subjects, err := c.listMeterViewSubjects(ctx, namespace, meter.Slug, params.From, params.To) - if err != nil { - if _, ok := err.(*models.MeterNotFoundError); ok { - return nil, err - } - - return nil, fmt.Errorf("list meter subjects: %w", err) - } - - return subjects, nil -} - -func (c *ClickhouseConnector) createMeterEventTable(ctx context.Context) error { - table := createMeterEventTable{ - Database: c.config.Database, - } - - err := c.config.ClickHouse.Exec(ctx, table.toSQL()) - if err != nil { - return fmt.Errorf("create meter event table: %w", err) - } - - return nil -} - -func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { - queryMeter := queryMeter{ - Database: c.config.Database, - Namespace: namespace, - Meter: meter, - From: params.From, - To: params.To, - Subject: params.FilterSubject, - FilterGroupBy: params.FilterGroupBy, - GroupBy: params.GroupBy, - WindowSize: params.WindowSize, - WindowTimeZone: params.WindowTimeZone, - } - - values := []models.MeterQueryRow{} - - sql, args, err := queryMeter.toSQL() - if err != nil { - return values, fmt.Errorf("query meter view: %w", err) - } - - start := time.Now() - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.MeterNotFoundError{MeterSlug: meter.Slug} - } - - return values, fmt.Errorf("query meter view query: %w", err) - } - elapsed := time.Since(start) - slog.Debug("query meter view", "elapsed", elapsed.String(), "sql", sql, "args", args) - - for rows.Next() { - row := models.MeterQueryRow{ - GroupBy: map[string]*string{}, - } - - var value decimal.Decimal - args := []interface{}{&row.WindowStart, &row.WindowEnd, &value} - argCount := len(args) - - for range queryMeter.GroupBy { - tmp := "" - args = append(args, &tmp) - } - - if err := rows.Scan(args...); err != nil { - return values, fmt.Errorf("query meter view row scan: %w", err) - } - - // TODO: should we use decima all the way? - row.Value, _ = value.Float64() - - for i, key := range queryMeter.GroupBy { - if s, ok := args[i+argCount].(*string); ok { - if key == "subject" { - row.Subject = s - continue - } - - // We treat empty string as nil - if s != nil && *s == "" { - row.GroupBy[key] = nil - } else { - row.GroupBy[key] = s - } - } - } - - // an empty row is returned when there are no values for the meter - if row.WindowStart.IsZero() && row.WindowEnd.IsZero() && row.Value == 0 { - continue - } - - values = append(values, row) - } - rows.Close() - err = rows.Err() - if err != nil { - return values, fmt.Errorf("query meter rows error: %w", err) - } - - return values, nil -} - -func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meterSlug string, from *time.Time, to *time.Time) ([]string, error) { - query := listMeterSubjectsQuery{ - Database: c.config.Database, - Namespace: namespace, - MeterSlug: meterSlug, - From: from, - To: to, - } - - sql, args := query.toSQL() - - rows, err := c.config.ClickHouse.Query(ctx, sql, args...) - if err != nil { - if strings.Contains(err.Error(), "code: 60") { - return nil, &models.MeterNotFoundError{MeterSlug: meterSlug} - } - - return nil, fmt.Errorf("list meter view subjects: %w", err) - } - - subjects := []string{} - for rows.Next() { - var subject string - if err = rows.Scan(&subject); err != nil { - return nil, err - } - - subjects = append(subjects, subject) - } - - return subjects, nil -} diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go deleted file mode 100644 index 4eec35aee..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/meter_event_query.go +++ /dev/null @@ -1,122 +0,0 @@ -package clickhouse_connector_parse - -import ( - _ "embed" - "fmt" - "strings" - - "github.com/huandu/go-sqlbuilder" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -const ( - MeterEventTableName = "om_meter_events" -) - -// Create Meter Event Table -type createMeterEventTable struct { - Database string -} - -func (d createMeterEventTable) toSQL() string { - tableName := GetMeterEventsTableName(d.Database) - - sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() - sb.CreateTable(tableName) - sb.IfNotExists() - - // Identifiers - sb.Define("namespace", "String") - sb.Define("time", "DateTime") - sb.Define("meter", "LowCardinality(String)") - sb.Define("subject", "String") - - // Usage - sb.Define("value", "Decimal(14, 4)") - // For unique aggregation we need to store the value as a string - sb.Define("value_str", "String") - sb.Define("group_by", "Map(String, String)") - - // Metadata - sb.Define("event_id", "String") - sb.Define("event_type", "LowCardinality(String)") - sb.Define("event_source", "String") - sb.Define("ingested_at", "DateTime") - sb.Define("stored_at", "DateTime") - sb.SQL("ENGINE = MergeTree") - sb.SQL("PARTITION BY toYYYYMM(time)") - // Lowest cardinality columns we always filter on goes to the most left. - // ClickHouse always picks partition first so we always filter time by month. - // Theoritically we could add toStartOfHour(time) to the order sooner than subject - // but we bet on that a typical namespace has more subjects than hours in a month. - // Subject is an optional filter so it won't always help to reduce number of rows scanned. - // Finally we add time not just to speed up queries but also to keep data on the disk together. - sb.SQL("ORDER BY (namespace, meter, subject, toStartOfHour(time), group_by)") - - sql, _ := sb.Build() - return sql -} - -// Insert Meter Events Query -type InsertMeterEventsQuery struct { - Database string - MeterEvents []streaming.MeterEvent - QuerySettings map[string]string -} - -func (q InsertMeterEventsQuery) ToSQL() (string, []interface{}) { - tableName := GetMeterEventsTableName(q.Database) - - query := sqlbuilder.ClickHouse.NewInsertBuilder() - query.InsertInto(tableName) - query.Cols( - "namespace", - "time", - "meter", - "subject", - "value", - "value_str", - "group_by", - "event_id", - "event_source", - "event_type", - "ingested_at", - "stored_at", - ) - - // Add settings - var settings []string - for key, value := range q.QuerySettings { - settings = append(settings, fmt.Sprintf("%s = %s", key, value)) - } - - if len(settings) > 0 { - query.SQL(fmt.Sprintf("SETTINGS %s", strings.Join(settings, ", "))) - } - - for _, meterEvent := range q.MeterEvents { - query.Values( - meterEvent.Namespace, - meterEvent.Time, - meterEvent.Meter, - meterEvent.Subject, - meterEvent.Value, - meterEvent.ValueString, - meterEvent.GroupBy, - meterEvent.RawEvent.ID, - meterEvent.RawEvent.Source, - meterEvent.RawEvent.Type, - meterEvent.IngestedAt, - meterEvent.StoredAt, - ) - } - - sql, args := query.Build() - return sql, args -} - -// Get Meter Events Table Name -func GetMeterEventsTableName(database string) string { - return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), MeterEventTableName) -} diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go deleted file mode 100644 index 0c3be2d8b..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/meter_event_query_test.go +++ /dev/null @@ -1,68 +0,0 @@ -package clickhouse_connector_parse - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/openmeter/streaming" -) - -func TestInsertMeterEventsQuery(t *testing.T) { - now := time.Now() - - rawEvent := streaming.RawEvent{ - Namespace: "my_namespace", - ID: "1", - Source: "source", - Subject: "subject-1", - Time: now, - StoredAt: now, - IngestedAt: now, - Type: "api-calls", - Data: `{"duration_ms": 100, "method": "GET", "path": "/api/v1"}`, - } - - query := InsertMeterEventsQuery{ - Database: "database", - QuerySettings: map[string]string{ - "parallel_view_processing": "1", - "max_insert_threads": "2", - }, - MeterEvents: []streaming.MeterEvent{ - { - RawEvent: rawEvent, - Meter: "api_request_duration", - Value: 100.0, - GroupBy: map[string]string{ - "method": "GET", - "path": "/api/v1", - }, - }, - { - RawEvent: rawEvent, - Meter: "api_request_total", - Value: 1.0, - GroupBy: map[string]string{ - "method": "GET", - "path": "/api/v1", - }, - }, - }, - } - - sql, args := query.ToSQL() - - assert.Equal(t, []interface{}{ - // First Meter Event - "my_namespace", now, "api_request_duration", "subject-1", 100.0, "", - map[string]string{"method": "GET", "path": "/api/v1"}, - "1", "source", "api-calls", now, now, - // Second Meter Event - "my_namespace", now, "api_request_total", "subject-1", 1.0, "", - map[string]string{"method": "GET", "path": "/api/v1"}, - "1", "source", "api-calls", now, now, - }, args) - assert.Equal(t, `INSERT INTO database.om_meter_events (namespace, time, meter, subject, value, value_str, group_by, event_id, event_source, event_type, ingested_at, stored_at) SETTINGS parallel_view_processing = 1, max_insert_threads = 2 VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?), (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`, sql) -} diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query.go b/openmeter/streaming/clickhouse_connector_parse/meter_query.go deleted file mode 100644 index 2e8aff75d..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query.go +++ /dev/null @@ -1,206 +0,0 @@ -package clickhouse_connector_parse - -import ( - _ "embed" - "fmt" - "sort" - "time" - - "github.com/huandu/go-sqlbuilder" - - "github.com/openmeterio/openmeter/pkg/models" - "github.com/openmeterio/openmeter/pkg/slicesx" -) - -type queryMeter struct { - Database string - Namespace string - Meter models.Meter - Subject []string - FilterGroupBy map[string][]string - From *time.Time - To *time.Time - GroupBy []string - WindowSize *models.WindowSize - WindowTimeZone *time.Location -} - -func (d queryMeter) toSQL() (string, []interface{}, error) { - tableName := GetMeterEventsTableName(d.Database) - getColumn := columnFactory(MeterEventTableName) - timeColumn := getColumn("time") - - var selectColumns, groupByColumns, where []string - - groupByWindowSize := d.WindowSize != nil - - tz := "UTC" - if d.WindowTimeZone != nil { - tz = d.WindowTimeZone.String() - } - - if groupByWindowSize { - switch *d.WindowSize { - case models.WindowSizeMinute: - selectColumns = append( - selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalMinute(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalMinute(1), '%s') AS windowend", timeColumn, tz), - ) - - case models.WindowSizeHour: - selectColumns = append( - selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalHour(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalHour(1), '%s') AS windowend", timeColumn, tz), - ) - - case models.WindowSizeDay: - selectColumns = append( - selectColumns, - fmt.Sprintf("tumbleStart(%s, toIntervalDay(1), '%s') AS windowstart", timeColumn, tz), - fmt.Sprintf("tumbleEnd(%s, toIntervalDay(1), '%s') AS windowend", timeColumn, tz), - ) - - default: - return "", nil, fmt.Errorf("invalid window size type: %s", *d.WindowSize) - } - - groupByColumns = append(groupByColumns, "windowstart", "windowend") - } else { - // TODO: remove this when we don't round to the nearest minute anymore - // We round them to the nearest minute to ensure the result is the same as with - // streaming connector using materialized views with per minute windows - selectColumn := fmt.Sprintf("tumbleStart(min(%s), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(%s), toIntervalMinute(1)) AS windowend", timeColumn, timeColumn) - selectColumns = append(selectColumns, selectColumn) - } - - switch d.Meter.Aggregation { - case models.MeterAggregationSum: - selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) - case models.MeterAggregationAvg: - selectColumns = append(selectColumns, fmt.Sprintf("avg(%s) AS value", getColumn("value"))) - case models.MeterAggregationMin: - selectColumns = append(selectColumns, fmt.Sprintf("min(%s) AS value", getColumn("value"))) - case models.MeterAggregationMax: - selectColumns = append(selectColumns, fmt.Sprintf("max(%s) AS value", getColumn("value"))) - case models.MeterAggregationUniqueCount: - selectColumns = append(selectColumns, fmt.Sprintf("toDecimal(uniq(%s)) AS value", getColumn("value_str"))) - case models.MeterAggregationCount: - selectColumns = append(selectColumns, fmt.Sprintf("sum(%s) AS value", getColumn("value"))) - default: - return "", nil, fmt.Errorf("invalid aggregation type: %s", d.Meter.Aggregation) - } - - for _, groupByKey := range d.GroupBy { - c := sqlbuilder.Escape(groupByKey) - selectColumn := fmt.Sprintf("%s['%s'] as %s", getColumn("group_by"), c, c) - - // Subject is a special case - if groupByKey == "subject" { - selectColumn = getColumn("subject") - } - - selectColumns = append(selectColumns, selectColumn) - groupByColumns = append(groupByColumns, c) - } - - query := sqlbuilder.ClickHouse.NewSelectBuilder() - query.Select(selectColumns...) - query.From(tableName) - query.Where(query.Equal(getColumn("namespace"), d.Namespace)) - query.Where(query.Equal(getColumn("meter"), d.Meter.GetID())) - - if len(d.Subject) > 0 { - mapFunc := func(subject string) string { - return query.Equal(getColumn("subject"), subject) - } - - where = append(where, query.Or(slicesx.Map(d.Subject, mapFunc)...)) - } - - if len(d.FilterGroupBy) > 0 { - // We sort the group by s to ensure the query is deterministic - groupByKeys := make([]string, 0, len(d.FilterGroupBy)) - for k := range d.FilterGroupBy { - groupByKeys = append(groupByKeys, k) - } - sort.Strings(groupByKeys) - - for _, groupByKey := range groupByKeys { - values := d.FilterGroupBy[groupByKey] - if len(values) == 0 { - return "", nil, fmt.Errorf("empty filter for group by: %s", groupByKey) - } - mapFunc := func(value string) string { - column := sqlbuilder.Escape(fmt.Sprintf("%s['%s']", getColumn("group_by"), groupByKey)) - - // Subject is a special case - if groupByKey == "subject" { - column = "subject" - } - - return query.Equal(column, value) - } - - where = append(where, query.Or(slicesx.Map(values, mapFunc)...)) - } - } - - if d.From != nil { - where = append(where, query.GreaterEqualThan(getColumn("time"), d.From.Unix())) - } - - if d.To != nil { - where = append(where, query.LessEqualThan(getColumn("time"), d.To.Unix())) - } - - if len(where) > 0 { - query.Where(where...) - } - - query.GroupBy(groupByColumns...) - - if groupByWindowSize { - query.OrderBy("windowstart") - } - - sql, args := query.Build() - return sql, args, nil -} - -type listMeterSubjectsQuery struct { - Database string - Namespace string - MeterSlug string - From *time.Time - To *time.Time -} - -func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { - tableName := GetMeterEventsTableName(d.Database) - - sb := sqlbuilder.ClickHouse.NewSelectBuilder() - sb.Select("DISTINCT subject") - sb.Where(sb.Equal("namespace", d.Namespace)) - sb.Where(sb.Equal("meter", d.MeterSlug)) - sb.From(tableName) - sb.OrderBy("subject") - - if d.From != nil { - sb.Where(sb.GreaterEqualThan("time", d.From.Unix())) - } - - if d.To != nil { - sb.Where(sb.LessEqualThan("time", d.To.Unix())) - } - - sql, args := sb.Build() - return sql, args -} - -func columnFactory(alias string) func(string) string { - return func(column string) string { - return fmt.Sprintf("%s.%s", alias, column) - } -} diff --git a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go b/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go deleted file mode 100644 index 54dd4dcb2..000000000 --- a/openmeter/streaming/clickhouse_connector_parse/meter_query_test.go +++ /dev/null @@ -1,271 +0,0 @@ -package clickhouse_connector_parse - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - - "github.com/openmeterio/openmeter/pkg/models" -) - -func TestQueryMeter(t *testing.T) { - subject := "subject1" - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") - tz, _ := time.LoadLocation("Asia/Shanghai") - windowSize := models.WindowSizeHour - - tests := []struct { - query queryMeter - wantSQL string - wantArgs []interface{} - }{ - { - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - From: &from, - To: &to, - GroupBy: []string{"subject", "group1", "group2"}, - WindowSize: &windowSize, - }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend, subject, group1, group2 ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", from.Unix(), to.Unix()}, - }, - { // Aggregate all available data - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, - }, - { // Aggregate with count aggregation - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationCount, - }, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ?", - wantArgs: []interface{}{"my_namespace", "meter1"}, - }, - { // Aggregate data from start - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, - }, - { // Aggregate data between period - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - To: &to, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ?", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, - }, - { // Aggregate data between period, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - To: &to, - WindowSize: &windowSize, - }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'UTC') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, - }, - { // Aggregate data between period in a different timezone, groupped by window size - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - From: &from, - To: &to, - WindowSize: &windowSize, - WindowTimeZone: tz, - }, - wantSQL: "SELECT tumbleStart(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowstart, tumbleEnd(om_meter_events.time, toIntervalHour(1), 'Asia/Shanghai') AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND om_meter_events.time >= ? AND om_meter_events.time <= ? GROUP BY windowstart, windowend ORDER BY windowstart", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, - }, - { // Aggregate data for a single subject - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - GroupBy: []string{"subject"}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, - }, - { // Aggregate data for a single subject and group by additional fields - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject}, - GroupBy: []string{"subject", "group1", "group2"}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject, om_meter_events.group_by['group1'] as group1, om_meter_events.group_by['group2'] as group2 FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ?) GROUP BY subject, group1, group2", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1"}, - }, - { // Aggregate data for a multiple subjects - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - Subject: []string{subject, "subject2"}, - GroupBy: []string{"subject"}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value, om_meter_events.subject FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.subject = ? OR om_meter_events.subject = ?) GROUP BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", "subject1", "subject2"}, - }, - { // Aggregate data with filtering for a single group and single value - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - FilterGroupBy: map[string][]string{"g1": {"g1v1"}}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1"}, - }, - { // Aggregate data with filtering for a single group and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2"}, - }, - { // Aggregate data with filtering for multiple groups and multiple values - query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", - Meter: models.Meter{ - Slug: "meter1", - Aggregation: models.MeterAggregationSum, - }, - FilterGroupBy: map[string][]string{"g1": {"g1v1", "g1v2"}, "g2": {"g2v1", "g2v2"}}, - }, - wantSQL: "SELECT tumbleStart(min(om_meter_events.time), toIntervalMinute(1)) AS windowstart, tumbleEnd(max(om_meter_events.time), toIntervalMinute(1)) AS windowend, sum(om_meter_events.value) AS value FROM openmeter.om_meter_events WHERE om_meter_events.namespace = ? AND om_meter_events.meter = ? AND (om_meter_events.group_by['g1'] = ? OR om_meter_events.group_by['g1'] = ?) AND (om_meter_events.group_by['g2'] = ? OR om_meter_events.group_by['g2'] = ?)", - wantArgs: []interface{}{"my_namespace", "meter1", "g1v1", "g1v2", "g2v1", "g2v2"}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs, err := tt.query.toSQL() - if err != nil { - t.Error(err) - return - } - - assert.Equal(t, tt.wantSQL, gotSql) - assert.Equal(t, tt.wantArgs, gotArgs) - }) - } -} - -func TestListMeterSubjects(t *testing.T) { - from, _ := time.Parse(time.RFC3339, "2023-01-01T00:00:00.001Z") - to, _ := time.Parse(time.RFC3339, "2023-01-02T00:00:00Z") - - tests := []struct { - query listMeterSubjectsQuery - wantSQL string - wantArgs []interface{} - }{ - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1"}, - }, - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - From: &from, - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix()}, - }, - { - query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - From: &from, - To: &to, - }, - wantSQL: "SELECT DISTINCT subject FROM openmeter.om_meter_events WHERE namespace = ? AND meter = ? AND time >= ? AND time <= ? ORDER BY subject", - wantArgs: []interface{}{"my_namespace", "meter1", from.Unix(), to.Unix()}, - }, - } - - for _, tt := range tests { - tt := tt - t.Run("", func(t *testing.T) { - gotSql, gotArgs := tt.query.toSQL() - - assert.Equal(t, tt.wantArgs, gotArgs) - assert.Equal(t, tt.wantSQL, gotSql) - }) - } -} diff --git a/openmeter/streaming/clickhouse_connector_raw/connector.go b/openmeter/streaming/clickhouse_connector_raw/connector.go index fcee96fc4..a63881ef6 100644 --- a/openmeter/streaming/clickhouse_connector_raw/connector.go +++ b/openmeter/streaming/clickhouse_connector_raw/connector.go @@ -170,7 +170,7 @@ func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, return rows, nil } -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { +func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { var err error // Insert raw events diff --git a/openmeter/streaming/connector.go b/openmeter/streaming/connector.go index 3fd810873..29cffdc71 100644 --- a/openmeter/streaming/connector.go +++ b/openmeter/streaming/connector.go @@ -50,15 +50,6 @@ type RawEvent struct { StoredAt time.Time } -// Meter Event represents a single event related to a meter -type MeterEvent struct { - RawEvent - Meter string - Value float64 - ValueString string - GroupBy map[string]string -} - type Connector interface { namespace.Handler @@ -68,6 +59,6 @@ type Connector interface { DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error QueryMeter(ctx context.Context, namespace string, meter models.Meter, params QueryParams) ([]models.MeterQueryRow, error) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params ListMeterSubjectsParams) ([]string, error) - BatchInsert(ctx context.Context, events []RawEvent, meterEvents []MeterEvent) error + BatchInsert(ctx context.Context, events []RawEvent) error // Add more methods as needed ... } diff --git a/openmeter/streaming/testutils/streaming.go b/openmeter/streaming/testutils/streaming.go index 01483c9a6..cf737ce0e 100644 --- a/openmeter/streaming/testutils/streaming.go +++ b/openmeter/streaming/testutils/streaming.go @@ -98,7 +98,7 @@ func (m *MockStreamingConnector) QueryMeter(ctx context.Context, namespace strin return rows, nil } -func (m *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent, meterEvents []streaming.MeterEvent) error { +func (m *MockStreamingConnector) BatchInsert(ctx context.Context, events []streaming.RawEvent) error { return nil } From 10df0e4f3f4a1ebf6d09e4472025896c72f96ea7 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Tue, 22 Oct 2024 18:28:36 -0700 Subject: [PATCH 29/33] feat(connector): remove meter change --- pkg/models/meter.go | 8 -------- 1 file changed, 8 deletions(-) diff --git a/pkg/models/meter.go b/pkg/models/meter.go index 59154ae08..7f9e6356c 100644 --- a/pkg/models/meter.go +++ b/pkg/models/meter.go @@ -114,14 +114,6 @@ type Meter struct { WindowSize WindowSize `json:"windowSize,omitempty" yaml:"windowSize,omitempty"` } -func (m *Meter) GetID() string { - if m.ID == "" { - return m.Slug - } - - return m.ID -} - type MeterOptions struct { ID string Description string From 7a1e5cca0a6438c8f97d2c4bc87e22f8adb58b09 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Tue, 22 Oct 2024 18:36:25 -0700 Subject: [PATCH 30/33] refactor(connector): clickhouse pkg --- app/common/openmeter.go | 8 +-- .../materialized_view}/connector.go | 52 +++++++++---------- .../materialized_view}/meter_query.go | 6 +-- .../materialized_view}/meter_query_test.go | 2 +- .../raw_events}/connector.go | 46 ++++++++-------- .../raw_events}/event_query.go | 4 +- .../raw_events}/event_query_test.go | 2 +- .../raw_events}/meter_query.go | 2 +- .../raw_events}/meter_query_test.go | 2 +- 9 files changed, 62 insertions(+), 62 deletions(-) rename openmeter/streaming/{clickhouse_connector_mv => clickhouse/materialized_view}/connector.go (77%) rename openmeter/streaming/{clickhouse_connector_mv => clickhouse/materialized_view}/meter_query.go (98%) rename openmeter/streaming/{clickhouse_connector_mv => clickhouse/materialized_view}/meter_query_test.go (99%) rename openmeter/streaming/{clickhouse_connector_raw => clickhouse/raw_events}/connector.go (79%) rename openmeter/streaming/{clickhouse_connector_raw => clickhouse/raw_events}/event_query.go (97%) rename openmeter/streaming/{clickhouse_connector_raw => clickhouse/raw_events}/event_query_test.go (99%) rename openmeter/streaming/{clickhouse_connector_raw => clickhouse/raw_events}/meter_query.go (99%) rename openmeter/streaming/{clickhouse_connector_raw => clickhouse/raw_events}/meter_query_test.go (99%) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 3128e5cea..213939184 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -22,8 +22,8 @@ import ( "github.com/openmeterio/openmeter/openmeter/sink/flushhandler" "github.com/openmeterio/openmeter/openmeter/sink/flushhandler/ingestnotification" "github.com/openmeterio/openmeter/openmeter/streaming" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_mv" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse/materialized_view" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse/raw_events" watermillkafka "github.com/openmeterio/openmeter/openmeter/watermill/driver/kafka" "github.com/openmeterio/openmeter/openmeter/watermill/driver/noop" "github.com/openmeterio/openmeter/openmeter/watermill/eventbus" @@ -50,7 +50,7 @@ func NewStreamingConnector( switch conf.Engine { case config.AggregationEngineClickHouseRaw: - connector, err = clickhouse_connector_raw.NewClickhouseConnector(ctx, clickhouse_connector_raw.ClickhouseConnectorConfig{ + connector, err = raw_events.NewConnector(ctx, raw_events.ConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, Logger: logger, @@ -63,7 +63,7 @@ func NewStreamingConnector( } case config.AggregationEngineClickHouseMV: - connector, err = clickhouse_connector_mv.NewClickhouseConnector(ctx, clickhouse_connector_mv.ClickhouseConnectorConfig{ + connector, err = materialized_view.NewConnector(ctx, materialized_view.ConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, Logger: logger, diff --git a/openmeter/streaming/clickhouse_connector_mv/connector.go b/openmeter/streaming/clickhouse/materialized_view/connector.go similarity index 77% rename from openmeter/streaming/clickhouse_connector_mv/connector.go rename to openmeter/streaming/clickhouse/materialized_view/connector.go index 3bf6a8d70..4904d8861 100644 --- a/openmeter/streaming/clickhouse_connector_mv/connector.go +++ b/openmeter/streaming/clickhouse/materialized_view/connector.go @@ -1,4 +1,4 @@ -package clickhouse_connector_mv +package materialized_view import ( "context" @@ -12,19 +12,19 @@ import ( "github.com/openmeterio/openmeter/api" "github.com/openmeterio/openmeter/openmeter/meter" "github.com/openmeterio/openmeter/openmeter/streaming" - raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" + raw_events "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse/raw_events" "github.com/openmeterio/openmeter/pkg/models" ) -var _ streaming.Connector = (*ClickhouseConnector)(nil) +var _ streaming.Connector = (*Connector)(nil) -// ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. -type ClickhouseConnector struct { - config ClickhouseConnectorConfig - rawEventConnector *raw_event_connector.ClickhouseConnector +// Connector implements `ingest.Connector“ and `namespace.Handler interfaces. +type Connector struct { + config ConnectorConfig + rawEventConnector *raw_events.Connector } -type ClickhouseConnectorConfig struct { +type ConnectorConfig struct { Logger *slog.Logger ClickHouse clickhouse.Conn Database string @@ -37,7 +37,7 @@ type ClickhouseConnectorConfig struct { QueryRawEvents bool } -func (c ClickhouseConnectorConfig) Validate() error { +func (c ConnectorConfig) Validate() error { if c.Logger == nil { return fmt.Errorf("logger is required") } @@ -57,12 +57,12 @@ func (c ClickhouseConnectorConfig) Validate() error { return nil } -func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { +func NewConnector(ctx context.Context, config ConnectorConfig) (*Connector, error) { if err := config.Validate(); err != nil { return nil, fmt.Errorf("validate config: %w", err) } - rawEventConnector, err := raw_event_connector.NewClickhouseConnector(ctx, raw_event_connector.ClickhouseConnectorConfig{ + rawEventConnector, err := raw_events.NewConnector(ctx, raw_events.ConnectorConfig{ Logger: config.Logger, ClickHouse: config.ClickHouse, Database: config.Database, @@ -74,7 +74,7 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return nil, fmt.Errorf("create raw event connector: %w", err) } - connector := &ClickhouseConnector{ + connector := &Connector{ config: config, rawEventConnector: rawEventConnector, } @@ -82,11 +82,11 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return connector, nil } -func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { +func (c *Connector) CreateNamespace(ctx context.Context, namespace string) error { return nil } -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { +func (c *Connector) DeleteNamespace(ctx context.Context, namespace string) error { err := c.deleteNamespace(ctx, namespace) if err != nil { return fmt.Errorf("delete namespace in clickhouse: %w", err) @@ -94,19 +94,19 @@ func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { +func (c *Connector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { return c.rawEventConnector.BatchInsert(ctx, rawEvents) } -func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { +func (c *Connector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { return c.rawEventConnector.CountEvents(ctx, namespace, params) } -func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { +func (c *Connector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { return c.rawEventConnector.ListEvents(ctx, namespace, params) } -func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { if namespace == "" { return fmt.Errorf("namespace is required") } @@ -122,7 +122,7 @@ func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, return nil } -func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { if namespace == "" { return fmt.Errorf("namespace is required") } @@ -142,7 +142,7 @@ func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, return nil } -func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *Connector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -181,7 +181,7 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return values, nil } -func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { +func (c *Connector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -209,7 +209,7 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s // DeleteNamespace deletes the namespace related resources from Clickhouse // We don't delete the events table as it it reused between namespaces // We only delete the materialized views for the meters -func (c *ClickhouseConnector) deleteNamespace(ctx context.Context, namespace string) error { +func (c *Connector) deleteNamespace(ctx context.Context, namespace string) error { // Retrieve meters belonging to the namespace meters, err := c.config.Meters.ListMeters(ctx, namespace) if err != nil { @@ -230,7 +230,7 @@ func (c *ClickhouseConnector) deleteNamespace(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) createMeterView(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) createMeterView(ctx context.Context, namespace string, meter models.Meter) error { // CreateOrReplace is used to force the recreation of the materialized view // This is not safe to use in production as it will drop the existing views if c.config.CreateOrReplaceMeter { @@ -262,7 +262,7 @@ func (c *ClickhouseConnector) createMeterView(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) deleteMeterView(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) deleteMeterView(ctx context.Context, namespace string, meter models.Meter) error { query := deleteMeterView{ Database: c.config.Database, Namespace: namespace, @@ -283,7 +283,7 @@ func (c *ClickhouseConnector) deleteMeterView(ctx context.Context, namespace str return nil } -func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *Connector) queryMeterView(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeterView{ Database: c.config.Database, Namespace: namespace, @@ -366,7 +366,7 @@ func (c *ClickhouseConnector) queryMeterView(ctx context.Context, namespace stri return values, nil } -func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { +func (c *Connector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { query := listMeterViewSubjects{ Database: c.config.Database, Namespace: namespace, diff --git a/openmeter/streaming/clickhouse_connector_mv/meter_query.go b/openmeter/streaming/clickhouse/materialized_view/meter_query.go similarity index 98% rename from openmeter/streaming/clickhouse_connector_mv/meter_query.go rename to openmeter/streaming/clickhouse/materialized_view/meter_query.go index 45b016c47..19413c192 100644 --- a/openmeter/streaming/clickhouse_connector_mv/meter_query.go +++ b/openmeter/streaming/clickhouse/materialized_view/meter_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector_mv +package materialized_view import ( _ "embed" @@ -9,7 +9,7 @@ import ( "github.com/huandu/go-sqlbuilder" - raw_event_connector "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse_connector_raw" + "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse/raw_events" "github.com/openmeterio/openmeter/pkg/models" "github.com/openmeterio/openmeter/pkg/slicesx" ) @@ -103,7 +103,7 @@ func (d createMeterView) toSQL() (string, []interface{}, error) { } func (d createMeterView) toSelectSQL() (string, error) { - eventsTableName := raw_event_connector.GetEventsTableName(d.Database) + eventsTableName := raw_events.GetEventsTableName(d.Database) aggStateFn := "" switch d.Aggregation { diff --git a/openmeter/streaming/clickhouse_connector_mv/meter_query_test.go b/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_mv/meter_query_test.go rename to openmeter/streaming/clickhouse/materialized_view/meter_query_test.go index f38f0b4f0..8f2eae7de 100644 --- a/openmeter/streaming/clickhouse_connector_mv/meter_query_test.go +++ b/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go @@ -1,4 +1,4 @@ -package clickhouse_connector_mv +package materialized_view import ( "testing" diff --git a/openmeter/streaming/clickhouse_connector_raw/connector.go b/openmeter/streaming/clickhouse/raw_events/connector.go similarity index 79% rename from openmeter/streaming/clickhouse_connector_raw/connector.go rename to openmeter/streaming/clickhouse/raw_events/connector.go index a63881ef6..c9dfa5261 100644 --- a/openmeter/streaming/clickhouse_connector_raw/connector.go +++ b/openmeter/streaming/clickhouse/raw_events/connector.go @@ -1,4 +1,4 @@ -package clickhouse_connector_raw +package raw_events import ( "context" @@ -17,14 +17,14 @@ import ( "github.com/openmeterio/openmeter/pkg/models" ) -var _ streaming.Connector = (*ClickhouseConnector)(nil) +var _ streaming.Connector = (*Connector)(nil) -// ClickhouseConnector implements `ingest.Connector“ and `namespace.Handler interfaces. -type ClickhouseConnector struct { - config ClickhouseConnectorConfig +// Connector implements `ingest.Connector“ and `namespace.Handler interfaces. +type Connector struct { + config ConnectorConfig } -type ClickhouseConnectorConfig struct { +type ConnectorConfig struct { Logger *slog.Logger ClickHouse clickhouse.Conn Database string @@ -33,7 +33,7 @@ type ClickhouseConnectorConfig struct { InsertQuerySettings map[string]string } -func (c ClickhouseConnectorConfig) Validate() error { +func (c ConnectorConfig) Validate() error { if c.Logger == nil { return fmt.Errorf("logger is required") } @@ -49,12 +49,12 @@ func (c ClickhouseConnectorConfig) Validate() error { return nil } -func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfig) (*ClickhouseConnector, error) { +func NewConnector(ctx context.Context, config ConnectorConfig) (*Connector, error) { if err := config.Validate(); err != nil { return nil, fmt.Errorf("validate config: %w", err) } - connector := &ClickhouseConnector{ + connector := &Connector{ config: config, } @@ -66,7 +66,7 @@ func NewClickhouseConnector(ctx context.Context, config ClickhouseConnectorConfi return connector, nil } -func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { +func (c *Connector) ListEvents(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -83,17 +83,17 @@ func (c *ClickhouseConnector) ListEvents(ctx context.Context, namespace string, return events, nil } -func (c *ClickhouseConnector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) CreateMeter(ctx context.Context, namespace string, meter models.Meter) error { // Do nothing return nil } -func (c *ClickhouseConnector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { +func (c *Connector) DeleteMeter(ctx context.Context, namespace string, meter models.Meter) error { // Do nothing return nil } -func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *Connector) QueryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -124,7 +124,7 @@ func (c *ClickhouseConnector) QueryMeter(ctx context.Context, namespace string, return values, nil } -func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { +func (c *Connector) ListMeterSubjects(ctx context.Context, namespace string, meter models.Meter, params streaming.ListMeterSubjectsParams) ([]string, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -144,16 +144,16 @@ func (c *ClickhouseConnector) ListMeterSubjects(ctx context.Context, namespace s return subjects, nil } -func (c *ClickhouseConnector) CreateNamespace(ctx context.Context, namespace string) error { +func (c *Connector) CreateNamespace(ctx context.Context, namespace string) error { return nil } -func (c *ClickhouseConnector) DeleteNamespace(ctx context.Context, namespace string) error { +func (c *Connector) DeleteNamespace(ctx context.Context, namespace string) error { // We don't delete the event tables as it it reused between namespaces return nil } -func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { +func (c *Connector) CountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { if namespace == "" { return nil, fmt.Errorf("namespace is required") } @@ -170,7 +170,7 @@ func (c *ClickhouseConnector) CountEvents(ctx context.Context, namespace string, return rows, nil } -func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { +func (c *Connector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEvent) error { var err error // Insert raw events @@ -199,7 +199,7 @@ func (c *ClickhouseConnector) BatchInsert(ctx context.Context, rawEvents []strea return nil } -func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { +func (c *Connector) createEventsTable(ctx context.Context) error { table := createEventsTable{ Database: c.config.Database, } @@ -212,7 +212,7 @@ func (c *ClickhouseConnector) createEventsTable(ctx context.Context) error { return nil } -func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { +func (c *Connector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { table := queryEventsTable{ Database: c.config.Database, Namespace: namespace, @@ -289,7 +289,7 @@ func (c *ClickhouseConnector) queryEventsTable(ctx context.Context, namespace st return events, nil } -func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { +func (c *Connector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { table := queryCountEvents{ Database: c.config.Database, Namespace: namespace, @@ -322,7 +322,7 @@ func (c *ClickhouseConnector) queryCountEvents(ctx context.Context, namespace st return results, nil } -func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { +func (c *Connector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeter{ Database: c.config.Database, Namespace: namespace, @@ -408,7 +408,7 @@ func (c *ClickhouseConnector) queryMeter(ctx context.Context, namespace string, return values, nil } -func (c *ClickhouseConnector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, from *time.Time, to *time.Time) ([]string, error) { +func (c *Connector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, from *time.Time, to *time.Time) ([]string, error) { query := listMeterSubjectsQuery{ Database: c.config.Database, Namespace: namespace, diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query.go b/openmeter/streaming/clickhouse/raw_events/event_query.go similarity index 97% rename from openmeter/streaming/clickhouse_connector_raw/event_query.go rename to openmeter/streaming/clickhouse/raw_events/event_query.go index 2daae40c2..8dcd3988a 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query.go +++ b/openmeter/streaming/clickhouse/raw_events/event_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector_raw +package raw_events import ( _ "embed" @@ -38,7 +38,7 @@ func (d createEventsTable) toSQL() string { sb.SQL("PARTITION BY toYYYYMM(time)") // Lowest cardinality columns we always filter on goes to the most left. // ClickHouse always picks partition first so we always filter time by month. - // Theoritically we could add toStartOfHour(time) to the order sooner than subject + // Theoretically we could add toStartOfHour(time) to the order sooner than subject // but we bet on that a typical namespace has more subjects than hours in a month. // Subject is an optional filter so it won't always help to reduce number of rows scanned. // Finally we add time not just to speed up queries but also to keep data on the disk together. diff --git a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go b/openmeter/streaming/clickhouse/raw_events/event_query_test.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_raw/event_query_test.go rename to openmeter/streaming/clickhouse/raw_events/event_query_test.go index 848677132..f0c6710f1 100644 --- a/openmeter/streaming/clickhouse_connector_raw/event_query_test.go +++ b/openmeter/streaming/clickhouse/raw_events/event_query_test.go @@ -1,4 +1,4 @@ -package clickhouse_connector_raw +package raw_events import ( "testing" diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query.go b/openmeter/streaming/clickhouse/raw_events/meter_query.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_raw/meter_query.go rename to openmeter/streaming/clickhouse/raw_events/meter_query.go index c1b4bad0e..a0348a0e3 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query.go +++ b/openmeter/streaming/clickhouse/raw_events/meter_query.go @@ -1,4 +1,4 @@ -package clickhouse_connector_raw +package raw_events import ( _ "embed" diff --git a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go b/openmeter/streaming/clickhouse/raw_events/meter_query_test.go similarity index 99% rename from openmeter/streaming/clickhouse_connector_raw/meter_query_test.go rename to openmeter/streaming/clickhouse/raw_events/meter_query_test.go index 63fe89e99..b7180d185 100644 --- a/openmeter/streaming/clickhouse_connector_raw/meter_query_test.go +++ b/openmeter/streaming/clickhouse/raw_events/meter_query_test.go @@ -1,4 +1,4 @@ -package clickhouse_connector_raw +package raw_events import ( "testing" From 8317009393e5b42b1039371d8a0b4ba0d18a2511 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Wed, 23 Oct 2024 11:11:25 -0700 Subject: [PATCH 31/33] feat(streaming): make event table name configurable --- app/common/openmeter.go | 2 + app/config/aggregation.go | 7 ++ .../clickhouse/materialized_view/connector.go | 23 ++++-- .../materialized_view/meter_query.go | 22 ++--- .../materialized_view/meter_query_test.go | 75 +++++++++-------- .../clickhouse/raw_events/connector.go | 78 ++++++++++-------- .../clickhouse/raw_events/event_query.go | 52 ++++++------ .../clickhouse/raw_events/event_query_test.go | 56 +++++++------ .../clickhouse/raw_events/meter_query.go | 38 ++++----- .../clickhouse/raw_events/meter_query_test.go | 80 +++++++++++-------- 10 files changed, 249 insertions(+), 184 deletions(-) diff --git a/app/common/openmeter.go b/app/common/openmeter.go index 213939184..bf60d27ad 100644 --- a/app/common/openmeter.go +++ b/app/common/openmeter.go @@ -53,6 +53,7 @@ func NewStreamingConnector( connector, err = raw_events.NewConnector(ctx, raw_events.ConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, + EventsTableName: conf.EventsTableName, Logger: logger, AsyncInsert: conf.AsyncInsert, AsyncInsertWait: conf.AsyncInsertWait, @@ -66,6 +67,7 @@ func NewStreamingConnector( connector, err = materialized_view.NewConnector(ctx, materialized_view.ConnectorConfig{ ClickHouse: clickHouse, Database: conf.ClickHouse.Database, + EventsTableName: conf.EventsTableName, Logger: logger, AsyncInsert: conf.AsyncInsert, AsyncInsertWait: conf.AsyncInsertWait, diff --git a/app/config/aggregation.go b/app/config/aggregation.go index 4cf3e49ae..2c8d17cdb 100644 --- a/app/config/aggregation.go +++ b/app/config/aggregation.go @@ -36,6 +36,8 @@ type AggregationConfiguration struct { Engine AggregationEngine ClickHouse ClickHouseAggregationConfiguration + EventsTableName string + // Set true for ClickHouse first store the incoming inserts into an in-memory buffer // before flushing them regularly to disk. // See https://clickhouse.com/docs/en/cloud/bestpractices/asynchronous-inserts @@ -76,6 +78,10 @@ func (c AggregationConfiguration) Validate() error { return fmt.Errorf("engine: %w", err) } + if c.EventsTableName == "" { + return errors.New("events table is required") + } + if c.AsyncInsertWait && !c.AsyncInsert { return errors.New("async insert wait is set but async insert is not") } @@ -170,6 +176,7 @@ func (c ClickHouseAggregationConfiguration) GetClientOptions() *clickhouse.Optio // ConfigureAggregation configures some defaults in the Viper instance. func ConfigureAggregation(v *viper.Viper) { v.SetDefault("aggregation.engine", AggregationEngineClickHouseMV) + v.SetDefault("aggregation.eventsTableName", "om_events") v.SetDefault("aggregation.asyncInsert", false) v.SetDefault("aggregation.asyncInsertWait", false) diff --git a/openmeter/streaming/clickhouse/materialized_view/connector.go b/openmeter/streaming/clickhouse/materialized_view/connector.go index 4904d8861..9bf3be3d7 100644 --- a/openmeter/streaming/clickhouse/materialized_view/connector.go +++ b/openmeter/streaming/clickhouse/materialized_view/connector.go @@ -28,6 +28,7 @@ type ConnectorConfig struct { Logger *slog.Logger ClickHouse clickhouse.Conn Database string + EventsTableName string Meters meter.Repository CreateOrReplaceMeter bool PopulateMeter bool @@ -50,6 +51,10 @@ func (c ConnectorConfig) Validate() error { return fmt.Errorf("database is required") } + if c.EventsTableName == "" { + return fmt.Errorf("events table name is required") + } + if c.Meters == nil { return fmt.Errorf("meters repository is required") } @@ -66,6 +71,7 @@ func NewConnector(ctx context.Context, config ConnectorConfig) (*Connector, erro Logger: config.Logger, ClickHouse: config.ClickHouse, Database: config.Database, + EventsTableName: config.EventsTableName, AsyncInsert: config.AsyncInsert, AsyncInsertWait: config.AsyncInsertWait, InsertQuerySettings: config.InsertQuerySettings, @@ -241,14 +247,15 @@ func (c *Connector) createMeterView(ctx context.Context, namespace string, meter } view := createMeterView{ - Populate: c.config.PopulateMeter, - Database: c.config.Database, - Namespace: namespace, - MeterSlug: meter.Slug, - Aggregation: meter.Aggregation, - EventType: meter.EventType, - ValueProperty: meter.ValueProperty, - GroupBy: meter.GroupBy, + Populate: c.config.PopulateMeter, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Namespace: namespace, + MeterSlug: meter.Slug, + Aggregation: meter.Aggregation, + EventType: meter.EventType, + ValueProperty: meter.ValueProperty, + GroupBy: meter.GroupBy, } sql, args, err := view.toSQL() if err != nil { diff --git a/openmeter/streaming/clickhouse/materialized_view/meter_query.go b/openmeter/streaming/clickhouse/materialized_view/meter_query.go index 19413c192..1618ed73a 100644 --- a/openmeter/streaming/clickhouse/materialized_view/meter_query.go +++ b/openmeter/streaming/clickhouse/materialized_view/meter_query.go @@ -9,7 +9,6 @@ import ( "github.com/huandu/go-sqlbuilder" - "github.com/openmeterio/openmeter/openmeter/streaming/clickhouse/raw_events" "github.com/openmeterio/openmeter/pkg/models" "github.com/openmeterio/openmeter/pkg/slicesx" ) @@ -20,13 +19,14 @@ type column struct { } type createMeterView struct { - Database string - Aggregation models.MeterAggregation - Namespace string - MeterSlug string - EventType string - ValueProperty string - GroupBy map[string]string + Database string + EventsTableName string + Aggregation models.MeterAggregation + Namespace string + MeterSlug string + EventType string + ValueProperty string + GroupBy map[string]string // Populate creates the materialized view with data from the events table // This is not safe to use in production as requires to stop ingestion Populate bool @@ -103,7 +103,7 @@ func (d createMeterView) toSQL() (string, []interface{}, error) { } func (d createMeterView) toSelectSQL() (string, error) { - eventsTableName := raw_events.GetEventsTableName(d.Database) + eventsTableName := getTableName(d.Database, d.EventsTableName) aggStateFn := "" switch d.Aggregation { @@ -364,3 +364,7 @@ func columnFactory(alias string) func(string) string { return fmt.Sprintf("%s.%s", alias, column) } } + +func getTableName(database string, tableName string) string { + return fmt.Sprintf("%s.%s", database, tableName) +} diff --git a/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go b/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go index 8f2eae7de..573fb6f37 100644 --- a/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go +++ b/openmeter/streaming/clickhouse/materialized_view/meter_query_test.go @@ -17,65 +17,70 @@ func TestCreateMeterView(t *testing.T) { }{ { query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationSum, - EventType: "myevent", - ValueProperty: "$.duration_ms", - GroupBy: map[string]string{"group1": "$.group1", "group2": "$.group2"}, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationSum, + EventType: "myevent", + ValueProperty: "$.duration_ms", + GroupBy: map[string]string{"group1": "$.group1", "group2": "$.group2"}, }, wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(sum, Float64), group1 String, group2 String) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject, group1, group2) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, sumState(cast(JSON_VALUE(data, '$.duration_ms'), 'Float64')) AS value, JSON_VALUE(data, '$.group1') as group1, JSON_VALUE(data, '$.group2') as group2 FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject, group1, group2", wantArgs: nil, }, { query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationAvg, - EventType: "myevent", - ValueProperty: "$.token_count", - GroupBy: map[string]string{}, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationAvg, + EventType: "myevent", + ValueProperty: "$.token_count", + GroupBy: map[string]string{}, }, wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(avg, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, avgState(cast(JSON_VALUE(data, '$.token_count'), 'Float64')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", wantArgs: nil, }, { query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, - EventType: "myevent", - ValueProperty: "", - GroupBy: map[string]string{}, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, + EventType: "myevent", + ValueProperty: "", + GroupBy: map[string]string{}, }, wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", wantArgs: nil, }, { query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationCount, - EventType: "myevent", - ValueProperty: "", - GroupBy: map[string]string{}, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationCount, + EventType: "myevent", + ValueProperty: "", + GroupBy: map[string]string{}, }, wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(count, Float64)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, countState(*) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", wantArgs: nil, }, { query: createMeterView{ - Database: "openmeter", - Namespace: "my_namespace", - MeterSlug: "meter1", - Aggregation: models.MeterAggregationUniqueCount, - EventType: "myevent", - ValueProperty: "$.trace_id", - GroupBy: map[string]string{}, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + MeterSlug: "meter1", + Aggregation: models.MeterAggregationUniqueCount, + EventType: "myevent", + ValueProperty: "$.trace_id", + GroupBy: map[string]string{}, }, wantSQL: "CREATE MATERIALIZED VIEW IF NOT EXISTS openmeter.om_my_namespace_meter1 (subject String, windowstart DateTime, windowend DateTime, value AggregateFunction(uniq, String)) ENGINE = AggregatingMergeTree() ORDER BY (windowstart, windowend, subject) AS SELECT subject, tumbleStart(time, toIntervalMinute(1)) AS windowstart, tumbleEnd(time, toIntervalMinute(1)) AS windowend, uniqState(JSON_VALUE(data, '$.trace_id')) AS value FROM openmeter.om_events WHERE openmeter.om_events.namespace = 'my_namespace' AND empty(openmeter.om_events.validation_error) = 1 AND openmeter.om_events.type = 'myevent' GROUP BY windowstart, windowend, subject", wantArgs: nil, diff --git a/openmeter/streaming/clickhouse/raw_events/connector.go b/openmeter/streaming/clickhouse/raw_events/connector.go index c9dfa5261..9eb095da0 100644 --- a/openmeter/streaming/clickhouse/raw_events/connector.go +++ b/openmeter/streaming/clickhouse/raw_events/connector.go @@ -21,13 +21,15 @@ var _ streaming.Connector = (*Connector)(nil) // Connector implements `ingest.Connector“ and `namespace.Handler interfaces. type Connector struct { - config ConnectorConfig + config ConnectorConfig + eventsTable string } type ConnectorConfig struct { Logger *slog.Logger ClickHouse clickhouse.Conn Database string + EventsTableName string AsyncInsert bool AsyncInsertWait bool InsertQuerySettings map[string]string @@ -46,6 +48,10 @@ func (c ConnectorConfig) Validate() error { return fmt.Errorf("database is required") } + if c.EventsTableName == "" { + return fmt.Errorf("events table is required") + } + return nil } @@ -175,9 +181,10 @@ func (c *Connector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEv // Insert raw events query := InsertEventsQuery{ - Database: c.config.Database, - Events: rawEvents, - QuerySettings: c.config.InsertQuerySettings, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Events: rawEvents, + QuerySettings: c.config.InsertQuerySettings, } sql, args := query.ToSQL() @@ -201,7 +208,8 @@ func (c *Connector) BatchInsert(ctx context.Context, rawEvents []streaming.RawEv func (c *Connector) createEventsTable(ctx context.Context) error { table := createEventsTable{ - Database: c.config.Database, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, } err := c.config.ClickHouse.Exec(ctx, table.toSQL()) @@ -214,16 +222,17 @@ func (c *Connector) createEventsTable(ctx context.Context) error { func (c *Connector) queryEventsTable(ctx context.Context, namespace string, params streaming.ListEventsParams) ([]api.IngestedEvent, error) { table := queryEventsTable{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, - To: params.To, - IngestedAtFrom: params.IngestedAtFrom, - IngestedAtTo: params.IngestedAtTo, - ID: params.ID, - Subject: params.Subject, - HasError: params.HasError, - Limit: params.Limit, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Namespace: namespace, + From: params.From, + To: params.To, + IngestedAtFrom: params.IngestedAtFrom, + IngestedAtTo: params.IngestedAtTo, + ID: params.ID, + Subject: params.Subject, + HasError: params.HasError, + Limit: params.Limit, } sql, args := table.toSQL() @@ -291,9 +300,10 @@ func (c *Connector) queryEventsTable(ctx context.Context, namespace string, para func (c *Connector) queryCountEvents(ctx context.Context, namespace string, params streaming.CountEventsParams) ([]streaming.CountEventRow, error) { table := queryCountEvents{ - Database: c.config.Database, - Namespace: namespace, - From: params.From, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Namespace: namespace, + From: params.From, } sql, args := table.toSQL() @@ -324,16 +334,17 @@ func (c *Connector) queryCountEvents(ctx context.Context, namespace string, para func (c *Connector) queryMeter(ctx context.Context, namespace string, meter models.Meter, params streaming.QueryParams) ([]models.MeterQueryRow, error) { queryMeter := queryMeter{ - Database: c.config.Database, - Namespace: namespace, - Meter: meter, - From: params.From, - To: params.To, - Subject: params.FilterSubject, - FilterGroupBy: params.FilterGroupBy, - GroupBy: params.GroupBy, - WindowSize: params.WindowSize, - WindowTimeZone: params.WindowTimeZone, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Namespace: namespace, + Meter: meter, + From: params.From, + To: params.To, + Subject: params.FilterSubject, + FilterGroupBy: params.FilterGroupBy, + GroupBy: params.GroupBy, + WindowSize: params.WindowSize, + WindowTimeZone: params.WindowTimeZone, } values := []models.MeterQueryRow{} @@ -410,11 +421,12 @@ func (c *Connector) queryMeter(ctx context.Context, namespace string, meter mode func (c *Connector) listMeterViewSubjects(ctx context.Context, namespace string, meter models.Meter, from *time.Time, to *time.Time) ([]string, error) { query := listMeterSubjectsQuery{ - Database: c.config.Database, - Namespace: namespace, - Meter: meter, - From: from, - To: to, + Database: c.config.Database, + EventsTableName: c.config.EventsTableName, + Namespace: namespace, + Meter: meter, + From: from, + To: to, } sql, args := query.toSQL() diff --git a/openmeter/streaming/clickhouse/raw_events/event_query.go b/openmeter/streaming/clickhouse/raw_events/event_query.go index 8dcd3988a..afde09936 100644 --- a/openmeter/streaming/clickhouse/raw_events/event_query.go +++ b/openmeter/streaming/clickhouse/raw_events/event_query.go @@ -11,15 +11,14 @@ import ( "github.com/openmeterio/openmeter/openmeter/streaming" ) -const EventsTableName = "om_events" - // Create Events Table type createEventsTable struct { - Database string + Database string + EventsTableName string } func (d createEventsTable) toSQL() string { - tableName := GetEventsTableName(d.Database) + tableName := getTableName(d.Database, d.EventsTableName) sb := sqlbuilder.ClickHouse.NewCreateTableBuilder() sb.CreateTable(tableName) @@ -50,20 +49,21 @@ func (d createEventsTable) toSQL() string { // Query Events Table type queryEventsTable struct { - Database string - Namespace string - From *time.Time - To *time.Time - IngestedAtFrom *time.Time - IngestedAtTo *time.Time - ID *string - Subject *string - HasError *bool - Limit int + Database string + EventsTableName string + Namespace string + From *time.Time + To *time.Time + IngestedAtFrom *time.Time + IngestedAtTo *time.Time + ID *string + Subject *string + HasError *bool + Limit int } func (d queryEventsTable) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) + tableName := getTableName(d.Database, d.EventsTableName) where := []string{} query := sqlbuilder.ClickHouse.NewSelectBuilder() @@ -106,13 +106,14 @@ func (d queryEventsTable) toSQL() (string, []interface{}) { } type queryCountEvents struct { - Database string - Namespace string - From time.Time + Database string + EventsTableName string + Namespace string + From time.Time } func (d queryCountEvents) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) + tableName := getTableName(d.Database, d.EventsTableName) query := sqlbuilder.ClickHouse.NewSelectBuilder() query.Select("count() as count", "subject", "notEmpty(validation_error) as is_error") @@ -128,13 +129,14 @@ func (d queryCountEvents) toSQL() (string, []interface{}) { // Insert Events Query type InsertEventsQuery struct { - Database string - Events []streaming.RawEvent - QuerySettings map[string]string + Database string + EventsTableName string + Events []streaming.RawEvent + QuerySettings map[string]string } func (q InsertEventsQuery) ToSQL() (string, []interface{}) { - tableName := GetEventsTableName(q.Database) + tableName := getTableName(q.Database, q.EventsTableName) query := sqlbuilder.ClickHouse.NewInsertBuilder() query.InsertInto(tableName) @@ -169,6 +171,6 @@ func (q InsertEventsQuery) ToSQL() (string, []interface{}) { return sql, args } -func GetEventsTableName(database string) string { - return fmt.Sprintf("%s.%s", sqlbuilder.Escape(database), EventsTableName) +func getTableName(database string, tableName string) string { + return fmt.Sprintf("%s.%s", database, tableName) } diff --git a/openmeter/streaming/clickhouse/raw_events/event_query_test.go b/openmeter/streaming/clickhouse/raw_events/event_query_test.go index f0c6710f1..278593f63 100644 --- a/openmeter/streaming/clickhouse/raw_events/event_query_test.go +++ b/openmeter/streaming/clickhouse/raw_events/event_query_test.go @@ -16,7 +16,8 @@ func TestCreateEventsTable(t *testing.T) { }{ { data: createEventsTable{ - Database: "openmeter", + Database: "openmeter", + EventsTableName: "om_events", }, want: "CREATE TABLE IF NOT EXISTS openmeter.om_events (namespace String, validation_error String, id String, type LowCardinality(String), subject String, source String, time DateTime, data String, ingested_at DateTime, stored_at DateTime) ENGINE = MergeTree PARTITION BY toYYYYMM(time) ORDER BY (namespace, type, subject, toStartOfHour(time))", }, @@ -44,49 +45,54 @@ func TestQueryEventsTable(t *testing.T) { }{ { query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + Limit: 100, }, wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? ORDER BY time DESC LIMIT 100", wantArgs: []interface{}{"my_namespace"}, }, { query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - Subject: &subjectFilter, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + Limit: 100, + Subject: &subjectFilter, }, wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND subject = ? ORDER BY time DESC LIMIT 100", wantArgs: []interface{}{"my_namespace", subjectFilter}, }, { query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - ID: &idFilter, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + Limit: 100, + ID: &idFilter, }, wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND id LIKE ? ORDER BY time DESC LIMIT 100", wantArgs: []interface{}{"my_namespace", "%event-id-1%"}, }, { query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorTrue, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorTrue, }, wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND notEmpty(validation_error) = 1 ORDER BY time DESC LIMIT 100", wantArgs: []interface{}{"my_namespace"}, }, { query: queryEventsTable{ - Database: "openmeter", - Namespace: "my_namespace", - Limit: 100, - HasError: &hasErrorFalse, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + Limit: 100, + HasError: &hasErrorFalse, }, wantSQL: "SELECT id, type, subject, source, time, data, validation_error, ingested_at, stored_at FROM openmeter.om_events WHERE namespace = ? AND empty(validation_error) = 1 ORDER BY time DESC LIMIT 100", wantArgs: []interface{}{"my_namespace"}, @@ -113,9 +119,10 @@ func TestQueryEventsCount(t *testing.T) { }{ { query: queryCountEvents{ - Database: "openmeter", - Namespace: "my_namespace", - From: from, + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", + From: from, }, wantSQL: "SELECT count() as count, subject, notEmpty(validation_error) as is_error FROM openmeter.om_events WHERE namespace = ? AND time >= ? GROUP BY subject, is_error", wantArgs: []interface{}{"my_namespace", from.Unix()}, @@ -137,7 +144,8 @@ func TestInsertEventsQuery(t *testing.T) { now := time.Now() query := InsertEventsQuery{ - Database: "database", + Database: "database", + EventsTableName: "om_events", Events: []streaming.RawEvent{ { Namespace: "my_namespace", diff --git a/openmeter/streaming/clickhouse/raw_events/meter_query.go b/openmeter/streaming/clickhouse/raw_events/meter_query.go index a0348a0e3..8ba046530 100644 --- a/openmeter/streaming/clickhouse/raw_events/meter_query.go +++ b/openmeter/streaming/clickhouse/raw_events/meter_query.go @@ -14,21 +14,22 @@ import ( ) type queryMeter struct { - Database string - Namespace string - Meter models.Meter - Subject []string - FilterGroupBy map[string][]string - From *time.Time - To *time.Time - GroupBy []string - WindowSize *models.WindowSize - WindowTimeZone *time.Location + Database string + EventsTableName string + Namespace string + Meter models.Meter + Subject []string + FilterGroupBy map[string][]string + From *time.Time + To *time.Time + GroupBy []string + WindowSize *models.WindowSize + WindowTimeZone *time.Location } func (d queryMeter) toSQL() (string, []interface{}, error) { - tableName := GetEventsTableName(d.Database) - getColumn := columnFactory(EventsTableName) + tableName := getTableName(d.Database, d.EventsTableName) + getColumn := columnFactory(d.EventsTableName) timeColumn := getColumn("time") var selectColumns, groupByColumns, where []string @@ -199,15 +200,16 @@ func (d queryMeter) toSQL() (string, []interface{}, error) { } type listMeterSubjectsQuery struct { - Database string - Namespace string - Meter models.Meter - From *time.Time - To *time.Time + Database string + EventsTableName string + Namespace string + Meter models.Meter + From *time.Time + To *time.Time } func (d listMeterSubjectsQuery) toSQL() (string, []interface{}) { - tableName := GetEventsTableName(d.Database) + tableName := getTableName(d.Database, d.EventsTableName) sb := sqlbuilder.ClickHouse.NewSelectBuilder() sb.Select("DISTINCT subject") diff --git a/openmeter/streaming/clickhouse/raw_events/meter_query_test.go b/openmeter/streaming/clickhouse/raw_events/meter_query_test.go index b7180d185..3fedc7db5 100644 --- a/openmeter/streaming/clickhouse/raw_events/meter_query_test.go +++ b/openmeter/streaming/clickhouse/raw_events/meter_query_test.go @@ -23,8 +23,9 @@ func TestQueryMeter(t *testing.T) { }{ { query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -46,8 +47,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate all available data query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -64,8 +66,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate with count aggregation query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -82,8 +85,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data from start query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -101,8 +105,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data between period query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -121,8 +126,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data between period, groupped by window size query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -142,8 +148,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data between period in a different timezone, groupped by window size query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -164,8 +171,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data for a single subject query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -184,8 +192,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data for a single subject and group by additional fields query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -204,8 +213,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data for a multiple subjects query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -224,8 +234,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data with filtering for a single group and single value query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -243,8 +254,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data with filtering for a single group and multiple values query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -262,8 +274,9 @@ func TestQueryMeter(t *testing.T) { }, { // Aggregate data with filtering for multiple groups and multiple values query: queryMeter{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -307,8 +320,9 @@ func TestListMeterSubjects(t *testing.T) { }{ { query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -320,8 +334,9 @@ func TestListMeterSubjects(t *testing.T) { }, { query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", @@ -334,8 +349,9 @@ func TestListMeterSubjects(t *testing.T) { }, { query: listMeterSubjectsQuery{ - Database: "openmeter", - Namespace: "my_namespace", + Database: "openmeter", + EventsTableName: "om_events", + Namespace: "my_namespace", Meter: models.Meter{ Slug: "meter1", EventType: "event1", From 7a5fd36c6dd1abc2359267a395561de70c31bf68 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Wed, 23 Oct 2024 11:17:39 -0700 Subject: [PATCH 32/33] test(config): fix --- app/config/config_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/app/config/config_test.go b/app/config/config_test.go index 106b062c0..4909c5a42 100644 --- a/app/config/config_test.go +++ b/app/config/config_test.go @@ -125,6 +125,7 @@ func TestComplete(t *testing.T) { BlockBufferSize: 10, }, Engine: AggregationEngineClickHouseMV, + EventsTableName: "om_events", AsyncInsert: false, AsyncInsertWait: false, }, From 279cf6da3f3cc240c0f72f732f415c7781f28728 Mon Sep 17 00:00:00 2001 From: Peter Marton Date: Wed, 23 Oct 2024 11:35:01 -0700 Subject: [PATCH 33/33] refactor(connector): remove unused field --- openmeter/streaming/clickhouse/raw_events/connector.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/openmeter/streaming/clickhouse/raw_events/connector.go b/openmeter/streaming/clickhouse/raw_events/connector.go index 9eb095da0..0127cb601 100644 --- a/openmeter/streaming/clickhouse/raw_events/connector.go +++ b/openmeter/streaming/clickhouse/raw_events/connector.go @@ -21,8 +21,7 @@ var _ streaming.Connector = (*Connector)(nil) // Connector implements `ingest.Connector“ and `namespace.Handler interfaces. type Connector struct { - config ConnectorConfig - eventsTable string + config ConnectorConfig } type ConnectorConfig struct {