From 526e5c1cd10311b027193381e2031a0e566e13db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Przemys=C5=82aw=20Hejman?= Date: Tue, 17 Dec 2024 11:19:43 +0100 Subject: [PATCH] Remove `clickhouse.LogManager` from `ClickhouseQueryTranslator` (#1117) This instance is not really needed, `HandleTermsEnum` was the only place where this has been used. The biggest change is probably `terms_enum.go` and also the `ResolveField(cw.Ctx, fieldStr, cw.Schema)` - it's no longer a method, but function taking Schema as an argument. --- quesma/queryparser/aggregation_parser.go | 4 +-- .../aggregation_parser_new_logic_test.go | 4 +-- .../pancake_sql_query_generation_test.go | 8 ++--- quesma/queryparser/query_parser.go | 33 +++++++++---------- quesma/queryparser/query_parser_range_test.go | 5 +-- quesma/queryparser/query_parser_test.go | 13 +++----- quesma/queryparser/query_translator.go | 19 ----------- quesma/queryparser/query_translator_test.go | 2 +- .../functionality/terms_enum/terms_enum.go | 30 +++++++++++++---- .../terms_enum/terms_enum_test.go | 4 +-- quesma/quesma/highlight_test.go | 9 ++--- quesma/quesma/query_translator.go | 2 +- quesma/quesma/search_opensearch_test.go | 2 +- 13 files changed, 57 insertions(+), 78 deletions(-) diff --git a/quesma/queryparser/aggregation_parser.go b/quesma/queryparser/aggregation_parser.go index 1a4422f01..99c64f312 100644 --- a/quesma/queryparser/aggregation_parser.go +++ b/quesma/queryparser/aggregation_parser.go @@ -242,7 +242,7 @@ func (cw *ClickhouseQueryTranslator) parseFieldField(shouldBeMap any, aggregatio } if fieldRaw, ok := Map["field"]; ok { if field, ok := fieldRaw.(string); ok { - return model.NewColumnRef(cw.ResolveField(cw.Ctx, field)) // model.NewSelectColumnTableField(cw.Table.ResolveField(cw.Ctx, field)) // remove this resolve? we do all transforms after parsing is done? + return model.NewColumnRef(ResolveField(cw.Ctx, field, cw.Schema)) // model.NewSelectColumnTableField(cw.Table.ResolveField(cw.Ctx, field)) // remove this resolve? we do all transforms after parsing is done? } else { logger.WarnWithCtx(cw.Ctx).Msgf("field is not a string, but %T, value: %v", fieldRaw, fieldRaw) } @@ -322,7 +322,7 @@ func (cw *ClickhouseQueryTranslator) parseFieldFieldMaybeScript(shouldBeMap any, // maybe "field" field if fieldRaw, ok := Map["field"]; ok { if field, ok := fieldRaw.(string); ok { - return model.NewColumnRef(cw.ResolveField(cw.Ctx, field)), true // remove this resolve? we do all transforms after parsing is done? + return model.NewColumnRef(ResolveField(cw.Ctx, field, cw.Schema)), true // remove this resolve? we do all transforms after parsing is done? } else { logger.WarnWithCtx(cw.Ctx).Msgf("field is not a string, but %T, value: %v", fieldRaw, fieldRaw) } diff --git a/quesma/queryparser/aggregation_parser_new_logic_test.go b/quesma/queryparser/aggregation_parser_new_logic_test.go index 2dc897b59..182338751 100644 --- a/quesma/queryparser/aggregation_parser_new_logic_test.go +++ b/quesma/queryparser/aggregation_parser_new_logic_test.go @@ -8,7 +8,6 @@ import ( "github.com/stretchr/testify/assert" "quesma/clickhouse" "quesma/model" - "quesma/quesma/config" "quesma/quesma/types" "quesma/schema" "quesma/testdata" @@ -34,7 +33,6 @@ func Test3AggregationParserNewLogic(t *testing.T) { Name: tableName, Config: clickhouse.NewDefaultCHConfig(), } - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, &table), &config.QuesmaConfiguration{}) s := schema.StaticRegistry{ Tables: map[schema.IndexName]schema.Schema{ @@ -54,7 +52,7 @@ func Test3AggregationParserNewLogic(t *testing.T) { }, }, } - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: &table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(tableName)]} + cw := ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(tableName)]} for i, test := range testdata.NewLogicTestCases { t.Run(test.TestName+"("+strconv.Itoa(i)+")", func(t *testing.T) { diff --git a/quesma/queryparser/pancake_sql_query_generation_test.go b/quesma/queryparser/pancake_sql_query_generation_test.go index bc3b06352..3f99e3527 100644 --- a/quesma/queryparser/pancake_sql_query_generation_test.go +++ b/quesma/queryparser/pancake_sql_query_generation_test.go @@ -10,7 +10,6 @@ import ( "quesma/clickhouse" "quesma/model" "quesma/model/bucket_aggregations" - "quesma/quesma/config" "quesma/quesma/types" "quesma/schema" "quesma/util" @@ -38,7 +37,6 @@ func TestPancakeQueryGeneration(t *testing.T) { Config: clickhouse.NewDefaultCHConfig(), } - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, &table), &config.QuesmaConfiguration{}) currentSchema := schema.Schema{ Fields: nil, Aliases: nil, @@ -46,7 +44,7 @@ func TestPancakeQueryGeneration(t *testing.T) { DatabaseName: "", } - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: &table, Ctx: context.Background(), Schema: currentSchema} + cw := ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Schema: currentSchema} for i, test := range allAggregationTests() { t.Run(test.TestName+"("+strconv.Itoa(i)+")", func(t *testing.T) { @@ -211,11 +209,9 @@ func TestPancakeQueryGeneration_halfpancake(t *testing.T) { Config: clickhouse.NewDefaultCHConfig(), } - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, &table), &config.QuesmaConfiguration{}) - currentSchema := schema.Schema{} - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: &table, Ctx: context.Background(), Schema: currentSchema} + cw := ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Schema: currentSchema} tests := []struct { name string diff --git a/quesma/queryparser/query_parser.go b/quesma/queryparser/query_parser.go index 8880ef855..f202fec27 100644 --- a/quesma/queryparser/query_parser.go +++ b/quesma/queryparser/query_parser.go @@ -16,6 +16,7 @@ import ( "quesma/model/typical_queries" "quesma/queryparser/lucene" "quesma/quesma/types" + "quesma/schema" "quesma/util" "strconv" "strings" @@ -232,7 +233,7 @@ func (cw *ClickhouseQueryTranslator) parseMetadata(queryMap QueryMap) QueryMap { } func (cw *ClickhouseQueryTranslator) ParseAutocomplete(indexFilter *QueryMap, fieldName string, prefix *string, caseIns bool) model.SimpleQuery { - fieldName = cw.ResolveField(cw.Ctx, fieldName) + fieldName = ResolveField(cw.Ctx, fieldName, cw.Schema) canParse := true stmts := make([]model.Expr, 0) if indexFilter != nil { @@ -474,7 +475,7 @@ func (cw *ClickhouseQueryTranslator) parseTerm(queryMap QueryMap) model.SimpleQu whereClause = model.NewInfixExpr(model.NewLiteral("0"), "=", model.NewLiteral("0 /* "+k+"="+sprint(v)+" */")) return model.NewSimpleQuery(whereClause, true) } - fieldName := cw.ResolveField(cw.Ctx, k) + fieldName := ResolveField(cw.Ctx, k, cw.Schema) whereClause = model.NewInfixExpr(model.NewColumnRef(fieldName), "=", model.NewLiteral(sprint(v))) return model.NewSimpleQuery(whereClause, true) } @@ -540,7 +541,7 @@ func (cw *ClickhouseQueryTranslator) parseMatch(queryMap QueryMap, matchPhrase b } for fieldName, v := range queryMap { - fieldName = cw.ResolveField(cw.Ctx, fieldName) + fieldName = ResolveField(cw.Ctx, fieldName, cw.Schema) // (fieldName, v) = either e.g. ("message", "this is a test") // or ("message", map["query": "this is a test", ...]). Here we only care about "query" until we find a case where we need more. vUnNested := v @@ -640,7 +641,7 @@ func (cw *ClickhouseQueryTranslator) parsePrefix(queryMap QueryMap) model.Simple } for fieldName, v := range queryMap { - fieldName = cw.ResolveField(cw.Ctx, fieldName) + fieldName = ResolveField(cw.Ctx, fieldName, cw.Schema) switch vCasted := v.(type) { case string: simpleStat := model.NewInfixExpr(model.NewColumnRef(fieldName), "iLIKE", model.NewLiteral("'"+vCasted+"%'")) @@ -670,7 +671,7 @@ func (cw *ClickhouseQueryTranslator) parseWildcard(queryMap QueryMap) model.Simp } for fieldName, v := range queryMap { - fieldName = cw.ResolveField(cw.Ctx, fieldName) + fieldName = ResolveField(cw.Ctx, fieldName, cw.Schema) if vAsMap, ok := v.(QueryMap); ok { if value, ok := vAsMap["value"]; ok { if valueAsString, ok := value.(string); ok { @@ -762,9 +763,9 @@ func (cw *ClickhouseQueryTranslator) parseRange(queryMap QueryMap) model.SimpleQ const dateInSchemaExpected = true for fieldName, v := range queryMap { - fieldName = cw.ResolveField(cw.Ctx, fieldName) + fieldName = ResolveField(cw.Ctx, fieldName, cw.Schema) - fieldType := cw.Table.GetDateTimeType(cw.Ctx, cw.ResolveField(cw.Ctx, fieldName), dateInSchemaExpected) + fieldType := cw.Table.GetDateTimeType(cw.Ctx, ResolveField(cw.Ctx, fieldName, cw.Schema), dateInSchemaExpected) stmts := make([]model.Expr, 0) if _, ok := v.(QueryMap); !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid range type: %T, value: %v", v, v) @@ -943,7 +944,7 @@ func (cw *ClickhouseQueryTranslator) extractFields(fields []interface{}) []strin if fieldStr == "*" { return []string{model.FullTextFieldNamePlaceHolder} } - fieldStr = cw.ResolveField(cw.Ctx, fieldStr) + fieldStr = ResolveField(cw.Ctx, fieldStr, cw.Schema) result = append(result, fieldStr) } return result @@ -1044,7 +1045,7 @@ func (cw *ClickhouseQueryTranslator) isItListRequest(queryMap QueryMap) (model.H } } - resolvedField := cw.ResolveField(cw.Ctx, fieldName) + resolvedField := ResolveField(cw.Ctx, fieldName, cw.Schema) if resolvedField == "*" { return model.HitsCountInfo{Typ: model.ListAllFields, RequestedFields: []string{"*"}, Size: size}, true } @@ -1093,11 +1094,11 @@ func (cw *ClickhouseQueryTranslator) parseSortFields(sortMaps any) (sortColumns // sortMap has only 1 key, so we can just iterate over it for k, v := range sortMap { // TODO replace cw.Table.GetFieldInfo with schema.Field[] - if strings.HasPrefix(k, "_") && cw.Table.GetFieldInfo(cw.Ctx, cw.ResolveField(cw.Ctx, k)) == clickhouse.NotExists { + if strings.HasPrefix(k, "_") && cw.Table.GetFieldInfo(cw.Ctx, ResolveField(cw.Ctx, k, cw.Schema)) == clickhouse.NotExists { // we're skipping ELK internal fields, like "_doc", "_id", etc. continue } - fieldName := cw.ResolveField(cw.Ctx, k) + fieldName := ResolveField(cw.Ctx, k, cw.Schema) switch v := v.(type) { case QueryMap: if order, ok := v["order"]; ok { @@ -1127,7 +1128,7 @@ func (cw *ClickhouseQueryTranslator) parseSortFields(sortMaps any) (sortColumns return sortColumns case map[string]interface{}: for fieldName, fieldValue := range sortMaps { - if strings.HasPrefix(fieldName, "_") && cw.Table.GetFieldInfo(cw.Ctx, cw.ResolveField(cw.Ctx, fieldName)) == clickhouse.NotExists { + if strings.HasPrefix(fieldName, "_") && cw.Table.GetFieldInfo(cw.Ctx, ResolveField(cw.Ctx, fieldName, cw.Schema)) == clickhouse.NotExists { // TODO Elastic internal fields will need to be supported in the future continue } @@ -1144,7 +1145,7 @@ func (cw *ClickhouseQueryTranslator) parseSortFields(sortMaps any) (sortColumns case map[string]string: for fieldName, fieldValue := range sortMaps { - if strings.HasPrefix(fieldName, "_") && cw.Table.GetFieldInfo(cw.Ctx, cw.ResolveField(cw.Ctx, fieldName)) == clickhouse.NotExists { + if strings.HasPrefix(fieldName, "_") && cw.Table.GetFieldInfo(cw.Ctx, ResolveField(cw.Ctx, fieldName, cw.Schema)) == clickhouse.NotExists { // TODO Elastic internal fields will need to be supported in the future continue } @@ -1180,11 +1181,9 @@ func createSortColumn(fieldName, ordering string) (model.OrderByExpr, error) { // What prevents us from moving it to transformation pipeline now, is that // we need to anotate this field somehow in the AST, to be able // to distinguish it from other fields -func (cw *ClickhouseQueryTranslator) ResolveField(ctx context.Context, fieldName string) string { +func ResolveField(ctx context.Context, fieldName string, schemaInstance schema.Schema) string { // Alias resolution should occur *after* the query is parsed, not during the parsing - schemaInstance := cw.Schema - fieldName = strings.TrimSuffix(fieldName, ".keyword") fieldName = strings.TrimSuffix(fieldName, ".text") @@ -1221,7 +1220,7 @@ func (cw *ClickhouseQueryTranslator) parseSize(queryMap QueryMap, defaultSize in func (cw *ClickhouseQueryTranslator) GetDateTimeTypeFromSelectClause(ctx context.Context, expr model.Expr, dateInSchemaExpected bool) clickhouse.DateTimeType { if ref, ok := expr.(model.ColumnRef); ok { - return cw.Table.GetDateTimeType(ctx, cw.ResolveField(ctx, ref.ColumnName), dateInSchemaExpected) + return cw.Table.GetDateTimeType(ctx, ResolveField(ctx, ref.ColumnName, cw.Schema), dateInSchemaExpected) } return clickhouse.Invalid } diff --git a/quesma/queryparser/query_parser_range_test.go b/quesma/queryparser/query_parser_range_test.go index 8e9d015fd..1b5a1ae69 100644 --- a/quesma/queryparser/query_parser_range_test.go +++ b/quesma/queryparser/query_parser_range_test.go @@ -6,9 +6,7 @@ import ( "context" "github.com/stretchr/testify/assert" "quesma/clickhouse" - "quesma/quesma/config" "quesma/schema" - "quesma/util" "testing" ) @@ -102,8 +100,7 @@ func Test_parseRange(t *testing.T) { t.Fatal(err) } assert.NoError(t, err) - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, table), &config.QuesmaConfiguration{}) - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(tableName)]} + cw := ClickhouseQueryTranslator{Table: table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(tableName)]} simpleQuery := cw.parseRange(test.rangePartOfQuery) assert.Equal(t, test.expectedWhere, simpleQuery.WhereClauseAsString()) diff --git a/quesma/queryparser/query_parser_test.go b/quesma/queryparser/query_parser_test.go index d7bc0a097..b857a8574 100644 --- a/quesma/queryparser/query_parser_test.go +++ b/quesma/queryparser/query_parser_test.go @@ -61,7 +61,7 @@ func TestQueryParserStringAttrConfig(t *testing.T) { }, }, } - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables[schema.IndexName(tableName)]} + cw := ClickhouseQueryTranslator{Table: table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables[schema.IndexName(tableName)]} for i, tt := range testdata.TestsSearch { t.Run(fmt.Sprintf("%s(%d)", tt.Name, i), func(t *testing.T) { @@ -122,7 +122,7 @@ func TestQueryParserNoFullTextFields(t *testing.T) { }, }, } - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: &table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables[schema.IndexName(tableName)]} + cw := ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables[schema.IndexName(tableName)]} for i, tt := range testdata.TestsSearchNoFullTextFields { t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -185,8 +185,7 @@ func TestQueryParserNoAttrsConfig(t *testing.T) { }, }, } - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, table), &config.QuesmaConfiguration{}) - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables["logs-generic-default"]} + cw := ClickhouseQueryTranslator{Table: table, Ctx: context.Background(), Config: &cfg, Schema: s.Tables["logs-generic-default"]} for _, tt := range testdata.TestsSearchNoAttrs { t.Run(tt.Name, func(t *testing.T) { body, parseErr := types.ParseJSON(tt.QueryJson) @@ -269,8 +268,7 @@ func Test_parseSortFields(t *testing.T) { ENGINE = Memory`, clickhouse.NewChTableConfigNoAttrs(), ) - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, table), &config.QuesmaConfiguration{}) - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: table, Ctx: context.Background()} + cw := ClickhouseQueryTranslator{Table: table, Ctx: context.Background()} for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { assert.Equal(t, tt.sortColumns, cw.parseSortFields(tt.sortMap)) @@ -294,7 +292,6 @@ func TestInvalidQueryRequests(t *testing.T) { Config: clickhouse.NewDefaultCHConfig(), } - lm := clickhouse.NewLogManager(util.NewSyncMapWith(tableName, &table), &config.QuesmaConfiguration{}) currentSchema := schema.Schema{ Fields: nil, Aliases: nil, @@ -302,7 +299,7 @@ func TestInvalidQueryRequests(t *testing.T) { DatabaseName: "", } - cw := ClickhouseQueryTranslator{ClickhouseLM: lm, Table: &table, Ctx: context.Background(), Schema: currentSchema} + cw := ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Schema: currentSchema} for i, test := range testdata.InvalidAggregationTests { t.Run(test.TestName+"("+strconv.Itoa(i)+")", func(t *testing.T) { diff --git a/quesma/queryparser/query_translator.go b/quesma/queryparser/query_translator.go index 808de7cb5..e0caa3103 100644 --- a/quesma/queryparser/query_translator.go +++ b/quesma/queryparser/query_translator.go @@ -17,8 +17,6 @@ import ( type JsonMap = map[string]interface{} type ClickhouseQueryTranslator struct { - ClickhouseLM *clickhouse.LogManager - Schema schema.Schema Ctx context.Context @@ -312,20 +310,3 @@ func (cw *ClickhouseQueryTranslator) BuildCountQuery(whereClause model.Expr, sam func (cw *ClickhouseQueryTranslator) BuildNRowsQuery(fieldNames []string, query *model.SimpleQuery, limit int) *model.Query { return query_util.BuildHitsQuery(cw.Ctx, model.SingleTableNamePlaceHolder, fieldNames, query, limit) } - -func (cw *ClickhouseQueryTranslator) BuildAutocompleteQuery(fieldName, tableName string, whereClause model.Expr, limit int) *model.Query { - return &model.Query{ - SelectCommand: *model.NewSelectCommand( - []model.Expr{model.NewColumnRef(fieldName)}, - nil, - nil, - model.NewTableRef(tableName), - whereClause, - []model.Expr{}, - limit, - 0, - true, - nil, - ), - } -} diff --git a/quesma/queryparser/query_translator_test.go b/quesma/queryparser/query_translator_test.go index 1300cb984..12ae723d8 100644 --- a/quesma/queryparser/query_translator_test.go +++ b/quesma/queryparser/query_translator_test.go @@ -437,7 +437,7 @@ func TestMakeResponseAsyncSearchQuery(t *testing.T) { // tests MakeSearchResponse, in particular if JSON we return is a proper JSON. // used to fail before we fixed field quoting. func TestMakeResponseSearchQueryIsProperJson(t *testing.T) { - cw := ClickhouseQueryTranslator{ClickhouseLM: nil, Table: clickhouse.NewEmptyTable("@"), Ctx: context.Background()} + cw := ClickhouseQueryTranslator{Table: clickhouse.NewEmptyTable("@"), Ctx: context.Background()} const limit = 1000 queries := []*model.Query{ cw.BuildNRowsQuery([]string{"*"}, &model.SimpleQuery{}, limit), diff --git a/quesma/quesma/functionality/terms_enum/terms_enum.go b/quesma/quesma/functionality/terms_enum/terms_enum.go index c118781d6..785ef0d67 100644 --- a/quesma/quesma/functionality/terms_enum/terms_enum.go +++ b/quesma/quesma/functionality/terms_enum/terms_enum.go @@ -33,13 +33,11 @@ func HandleTermsEnum(ctx context.Context, index string, body types.JSON, lm *cli return []byte{}, end_user_errors.ErrNoSuchSchema.New(fmt.Errorf("can't load %s schema", resolvedTableName)).Details("Table: %s", resolvedTableName) } - return handleTermsEnumRequest(ctx, body, &queryparser.ClickhouseQueryTranslator{ - ClickhouseLM: lm, Table: lm.FindTable(indices[0]), Ctx: context.Background(), Schema: resolvedSchema, - }, qmc) + return handleTermsEnumRequest(ctx, body, lm, &queryparser.ClickhouseQueryTranslator{Table: lm.FindTable(indices[0]), Ctx: context.Background(), Schema: resolvedSchema}, qmc) } } -func handleTermsEnumRequest(ctx context.Context, body types.JSON, qt *queryparser.ClickhouseQueryTranslator, +func handleTermsEnumRequest(ctx context.Context, body types.JSON, lm *clickhouse.LogManager, qt *queryparser.ClickhouseQueryTranslator, qmc diag.DebugInfoCollector) (result []byte, err error) { startTime := time.Now() @@ -60,7 +58,7 @@ func handleTermsEnumRequest(ctx context.Context, body types.JSON, qt *queryparse logger.ErrorWithCtx(ctx).Msgf("error reading terms enum API request body: field is not present") return json.Marshal(emptyTermsEnumResponse()) } - field = qt.ResolveField(ctx, field) + field = queryparser.ResolveField(ctx, field, qt.Schema) size := defaultSize if sizeRaw, ok := body["size"]; ok { @@ -93,11 +91,12 @@ func handleTermsEnumRequest(ctx context.Context, body types.JSON, qt *queryparse } where := qt.ParseAutocomplete(indexFilter, field, prefixString, caseInsensitive) - selectQuery := qt.BuildAutocompleteQuery(field, qt.Table.Name, where.WhereClause, size) + selectQuery := buildAutocompleteQuery(field, qt.Table.Name, where.WhereClause, size) dbQueryCtx, cancel := context.WithCancel(ctx) // TODO this will be used to cancel goroutine that is executing the query _ = cancel - if rows, _, err2 := qt.ClickhouseLM.ProcessQuery(dbQueryCtx, qt.Table, selectQuery); err2 != nil { + + if rows, _, err2 := lm.ProcessQuery(dbQueryCtx, qt.Table, selectQuery); err2 != nil { logger.Error().Msgf("terms enum failed - error processing SQL query [%s]", err2) result, err = json.Marshal(emptyTermsEnumResponse()) } else { @@ -149,3 +148,20 @@ func emptyTermsEnumResponse() *model.TermsEnumResponse { Terms: nil, } } + +func buildAutocompleteQuery(fieldName, tableName string, whereClause model.Expr, limit int) *model.Query { + return &model.Query{ + SelectCommand: *model.NewSelectCommand( + []model.Expr{model.NewColumnRef(fieldName)}, + nil, + nil, + model.NewTableRef(tableName), + whereClause, + []model.Expr{}, + limit, + 0, + true, + nil, + ), + } +} diff --git a/quesma/quesma/functionality/terms_enum/terms_enum_test.go b/quesma/quesma/functionality/terms_enum/terms_enum_test.go index a087144cc..d7dd0eeda 100644 --- a/quesma/quesma/functionality/terms_enum/terms_enum_test.go +++ b/quesma/quesma/functionality/terms_enum/terms_enum_test.go @@ -119,7 +119,7 @@ func testHandleTermsEnumRequest(t *testing.T, requestBody []byte) { }, }, } - qt := &queryparser.ClickhouseQueryTranslator{ClickhouseLM: lm, Table: table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(testTableName)]} + qt := &queryparser.ClickhouseQueryTranslator{Table: table, Ctx: context.Background(), Schema: s.Tables[schema.IndexName(testTableName)]} // Here we additionally verify that terms for `_tier` are **NOT** included in the SQL query expectedQuery1 := `SELECT DISTINCT "client_name" FROM ` + testTableName + ` WHERE (("epoch_time">=fromUnixTimestamp(1709036700) AND "epoch_time"<=fromUnixTimestamp(1709037659)) AND ("epoch_time_datetime64">=fromUnixTimestamp64Milli(1709036700000) AND "epoch_time_datetime64"<=fromUnixTimestamp64Milli(1709037659999))) LIMIT 13` expectedQuery2 := `SELECT DISTINCT "client_name" FROM ` + testTableName + ` WHERE (("epoch_time">=fromUnixTimestamp(1709036700) AND "epoch_time"<=fromUnixTimestamp(1709037659)) AND ("epoch_time_datetime64">=fromUnixTimestamp64Milli(1709036700000) AND "epoch_time_datetime64"<=fromUnixTimestamp64Milli(1709037659999))) LIMIT 13` @@ -128,7 +128,7 @@ func testHandleTermsEnumRequest(t *testing.T, requestBody []byte) { mock.ExpectQuery(fmt.Sprintf("%s|%s", regexp.QuoteMeta(expectedQuery1), regexp.QuoteMeta(expectedQuery2))). WillReturnRows(sqlmock.NewRows([]string{"client_name"}).AddRow("client_a").AddRow("client_b")) - resp, err := handleTermsEnumRequest(ctx, types.MustJSON(string(requestBody)), qt, managementConsole) + resp, err := handleTermsEnumRequest(ctx, types.MustJSON(string(requestBody)), lm, qt, managementConsole) assert.NoError(t, err) var responseModel model.TermsEnumResponse diff --git a/quesma/quesma/highlight_test.go b/quesma/quesma/highlight_test.go index 201a752e0..ef53f69a9 100644 --- a/quesma/quesma/highlight_test.go +++ b/quesma/quesma/highlight_test.go @@ -9,8 +9,6 @@ import ( "quesma/clickhouse" "quesma/model" "quesma/queryparser" - "quesma/quesma/config" - "quesma_v2/core/diag" "testing" ) @@ -108,12 +106,9 @@ func TestParseHighLight(t *testing.T) { Config: clickhouse.NewDefaultCHConfig(), } - lm := clickhouse.NewEmptyLogManager(&config.QuesmaConfiguration{}, nil, diag.NewPhoneHomeEmptyAgent(), nil) - cw := queryparser.ClickhouseQueryTranslator{ - ClickhouseLM: lm, - Table: &table, - Ctx: context.Background(), + Table: &table, + Ctx: context.Background(), } queryAsMap := make(queryparser.QueryMap) diff --git a/quesma/quesma/query_translator.go b/quesma/quesma/query_translator.go index 2a3dd2684..76abc91fd 100644 --- a/quesma/quesma/query_translator.go +++ b/quesma/quesma/query_translator.go @@ -38,6 +38,6 @@ func NewQueryTranslator(ctx context.Context, language QueryLanguage, schema sche case QueryLanguageEQL: return &eql.ClickhouseEQLQueryTranslator{ClickhouseLM: logManager, Table: table, Ctx: ctx} default: - return &queryparser.ClickhouseQueryTranslator{ClickhouseLM: logManager, Table: table, Ctx: ctx, DateMathRenderer: dateMathRenderer, Indexes: indexes, Config: configuration, Schema: schema} + return &queryparser.ClickhouseQueryTranslator{Table: table, Ctx: ctx, DateMathRenderer: dateMathRenderer, Indexes: indexes, Config: configuration, Schema: schema} } } diff --git a/quesma/quesma/search_opensearch_test.go b/quesma/quesma/search_opensearch_test.go index d7d8b8560..3ab8b9a88 100644 --- a/quesma/quesma/search_opensearch_test.go +++ b/quesma/quesma/search_opensearch_test.go @@ -46,7 +46,7 @@ func TestSearchOpensearch(t *testing.T) { defer db.Close() queryRunner := NewQueryRunnerDefaultForTests(db, &DefaultConfig, tableName, util.NewSyncMapWith(tableName, &table), s) - cw := queryparser.ClickhouseQueryTranslator{ClickhouseLM: queryRunner.logManager, Table: &table, Ctx: context.Background(), Schema: s.Tables[tableName], Config: &DefaultConfig} + cw := queryparser.ClickhouseQueryTranslator{Table: &table, Ctx: context.Background(), Schema: s.Tables[tableName], Config: &DefaultConfig} body, parseErr := types.ParseJSON(tt.QueryJson) assert.NoError(t, parseErr)