From cf0fd3fc63e5db20a7b4242110a3ebba200830fd Mon Sep 17 00:00:00 2001 From: Krzysztof Kiewicz Date: Mon, 23 Dec 2024 10:20:11 +0100 Subject: [PATCH] Fix `ids` query DSL (#1135) By accident I noticed our SQL generated for `ids` looks quite nonsensical, e.g. that's what's generated for my new very simple request: ``` WHERE "@timestamp" IN toDateTime64('2024-12-21 07:29:03.367', '2024-12-21 07:29:02.992', 3) ``` Also noticed we have 0 tests for this query. Fixing those 2 things here. (adding `TupleExpr` for just this 1 usecase might seem unnecessary, but it's needed in my 2 other open PRs, so I'd do that) Also, minor: changed `return model.NewSimpleQuery(nil, false)` to more descriptive `return model.NewSimpleQueryInvalid()` everywhere. --- quesma/logger/log_with_throttling.go | 13 ++- quesma/model/base_visitor.go | 9 ++ quesma/model/expr.go | 14 +++ quesma/model/expr_string_renderer.go | 17 +++ quesma/model/simple_query.go | 4 + quesma/queryparser/query_parser.go | 144 ++++++++++++------------ quesma/queryparser/query_parser_test.go | 2 + quesma/testdata/requests.go | 63 +++++++++++ 8 files changed, 196 insertions(+), 70 deletions(-) diff --git a/quesma/logger/log_with_throttling.go b/quesma/logger/log_with_throttling.go index 8335a026d..9d29309b3 100644 --- a/quesma/logger/log_with_throttling.go +++ b/quesma/logger/log_with_throttling.go @@ -8,7 +8,7 @@ import ( "time" ) -var throttleMap = util.SyncMap[string, time.Time]{} +var throttleMap = util.NewSyncMap[string, time.Time]() const throttleDuration = 30 * time.Minute @@ -25,3 +25,14 @@ func WarnWithCtxAndThrottling(ctx context.Context, aggrName, paramName, format s throttleMap.Store(mapKey, time.Now()) } } + +// WarnWithThrottling - logs a warning message with throttling. +// We only log once per throttleDuration for each warnName, so that we don't spam the logs. +func WarnWithThrottling(warnName, format string, v ...any) { + timestamp, ok := throttleMap.Load(warnName) + weThrottle := ok && time.Since(timestamp) < throttleDuration + if !weThrottle { + Warn().Msgf(format, v...) + throttleMap.Store(warnName, time.Now()) + } +} diff --git a/quesma/model/base_visitor.go b/quesma/model/base_visitor.go index d12173835..71d68d04b 100644 --- a/quesma/model/base_visitor.go +++ b/quesma/model/base_visitor.go @@ -5,6 +5,7 @@ package model type BaseExprVisitor struct { OverrideVisitFunction func(b *BaseExprVisitor, e FunctionExpr) interface{} OverrideVisitLiteral func(b *BaseExprVisitor, l LiteralExpr) interface{} + OverrideVisitTuple func(b *BaseExprVisitor, t TupleExpr) interface{} OverrideVisitInfix func(b *BaseExprVisitor, e InfixExpr) interface{} OverrideVisitColumnRef func(b *BaseExprVisitor, e ColumnRef) interface{} OverrideVisitPrefixExpr func(b *BaseExprVisitor, e PrefixExpr) interface{} @@ -43,6 +44,14 @@ func (v *BaseExprVisitor) VisitLiteral(e LiteralExpr) interface{} { return NewLiteral(e.Value) } + +func (v *BaseExprVisitor) VisitTuple(e TupleExpr) interface{} { + if v.OverrideVisitTuple != nil { + return v.OverrideVisitTuple(v, e) + } + return NewTupleExpr(v.VisitChildren(e.Exprs)...) +} + func (v *BaseExprVisitor) VisitInfix(e InfixExpr) interface{} { if v.OverrideVisitInfix != nil { return v.OverrideVisitInfix(v, e) diff --git a/quesma/model/expr.go b/quesma/model/expr.go index 7be75377d..28dbcc757 100644 --- a/quesma/model/expr.go +++ b/quesma/model/expr.go @@ -12,6 +12,7 @@ type Expr interface { var ( InvalidExpr = Expr(nil) TrueExpr = NewLiteral(true) + FalseExpr = NewLiteral(false) ) // ColumnRef is a reference to a column in a table, we can enrich it with more information (e.g. type used) as we go @@ -86,6 +87,18 @@ func (e LiteralExpr) Accept(v ExprVisitor) interface{} { return v.VisitLiteral(e) } +type TupleExpr struct { + Exprs []Expr +} + +func NewTupleExpr(exprs ...Expr) TupleExpr { + return TupleExpr{Exprs: exprs} +} + +func (e TupleExpr) Accept(v ExprVisitor) interface{} { + return v.VisitTuple(e) +} + type InfixExpr struct { Left Expr Op string @@ -278,6 +291,7 @@ func (e CTE) Accept(v ExprVisitor) interface{} { type ExprVisitor interface { VisitFunction(e FunctionExpr) interface{} VisitLiteral(l LiteralExpr) interface{} + VisitTuple(t TupleExpr) interface{} VisitInfix(e InfixExpr) interface{} VisitColumnRef(e ColumnRef) interface{} VisitPrefixExpr(e PrefixExpr) interface{} diff --git a/quesma/model/expr_string_renderer.go b/quesma/model/expr_string_renderer.go index 38941e8a6..7ab4adc28 100644 --- a/quesma/model/expr_string_renderer.go +++ b/quesma/model/expr_string_renderer.go @@ -4,6 +4,7 @@ package model import ( "fmt" + "quesma/logger" "quesma/quesma/types" "regexp" "sort" @@ -68,6 +69,22 @@ func (v *renderer) VisitLiteral(l LiteralExpr) interface{} { return fmt.Sprintf("%v", l.Value) } +func (v *renderer) VisitTuple(t TupleExpr) interface{} { + switch len(t.Exprs) { + case 0: + logger.WarnWithThrottling("VisitTuple", "TupleExpr with no expressions") + return "()" + case 1: + return t.Exprs[0].Accept(v) + default: + args := make([]string, len(t.Exprs)) + for i, arg := range t.Exprs { + args[i] = arg.Accept(v).(string) + } + return fmt.Sprintf("tuple(%s)", strings.Join(args, ", ")) // can omit "tuple", but I think SQL's more readable with it + } +} + func (v *renderer) VisitInfix(e InfixExpr) interface{} { var lhs, rhs interface{} // TODO FOR NOW LITTLE PARANOID BUT HELPS ME NOT SEE MANY PANICS WHEN TESTING if e.Left != nil { diff --git a/quesma/model/simple_query.go b/quesma/model/simple_query.go index 829759941..0265fa0f9 100644 --- a/quesma/model/simple_query.go +++ b/quesma/model/simple_query.go @@ -28,6 +28,10 @@ func NewSimpleQuery(whereClause Expr, canParse bool) SimpleQuery { return SimpleQuery{WhereClause: whereClause, CanParse: canParse} } +func NewSimpleQueryInvalid() SimpleQuery { + return SimpleQuery{CanParse: false} +} + // LimitForCount returns (limit, true) if we need count(*) with limit, // (not-important, false) if we don't need count/limit func (s *SimpleQuery) LimitForCount() (limit int, doWeNeedLimit bool) { diff --git a/quesma/queryparser/query_parser.go b/quesma/queryparser/query_parser.go index f202fec27..0722c088b 100644 --- a/quesma/queryparser/query_parser.go +++ b/quesma/queryparser/query_parser.go @@ -302,7 +302,7 @@ func (cw *ClickhouseQueryTranslator) parseQueryMap(queryMap QueryMap) model.Simp unparsedQuery = string(prettyMarshal) } logger.Error().Msgf("can't parse query: %s", unparsedQuery) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // `constant_score` query is just a wrapper for filter query which returns constant relevance score, which we ignore anyway @@ -311,30 +311,27 @@ func (cw *ClickhouseQueryTranslator) parseConstantScore(queryMap QueryMap) model return cw.parseBool(queryMap) } else { logger.Error().Msgf("parsing error: `constant_score` needs to wrap `filter` query") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } func (cw *ClickhouseQueryTranslator) parseIds(queryMap QueryMap) model.SimpleQuery { - var ids, finalIds []string - if val, ok := queryMap["values"]; ok { - if values, ok := val.([]interface{}); ok { - for _, id := range values { - ids = append(ids, id.(string)) - } + idsRaw, err := cw.parseArrayField(queryMap, "values") + if err != nil { + logger.ErrorWithCtx(cw.Ctx).Msgf("parsing error: %v", err) + return model.NewSimpleQueryInvalid() + } + ids := make([]string, 0, len(idsRaw)) + for _, id := range idsRaw { + if idAsString, ok := id.(string); ok { + ids = append(ids, idAsString) + } else { + logger.ErrorWithCtx(cw.Ctx).Msgf("invalid id format, id value: %v type: %T", id, id) + return model.NewSimpleQueryInvalid() } - } else { - logger.Error().Msgf("parsing error: missing mandatory `values` field") - return model.NewSimpleQuery(nil, false) } - logger.Warn().Msgf("unsupported id query executed, requested ids of [%s]", strings.Join(ids, "','")) - timestampColumnName := model.TimestampFieldName - - if len(ids) == 0 { - logger.Warn().Msgf("parsing error: empty _id array") - return model.NewSimpleQuery(nil, false) - } + logger.Warn().Msgf("unsupported id query executed, requested ids of [%s]", strings.Join(ids, "','")) // when our generated ID appears in query looks like this: `1d0b8q1` // therefore we need to strip the hex part (before `q`) and convert it to decimal @@ -343,39 +340,48 @@ func (cw *ClickhouseQueryTranslator) parseIds(queryMap QueryMap) model.SimpleQue idInHex := strings.Split(id, "q")[0] if idAsStr, err := hex.DecodeString(idInHex); err != nil { logger.Error().Msgf("error parsing document id %s: %v", id, err) - return model.NewSimpleQuery(nil, true) + return model.NewSimpleQueryInvalid() } else { tsWithoutTZ := strings.TrimSuffix(string(idAsStr), " +0000 UTC") ids[i] = fmt.Sprintf("'%s'", tsWithoutTZ) } } - var whereStmt model.Expr // TODO replace with cw.Schema - if v, ok := cw.Table.Cols[timestampColumnName]; ok { - switch v.Type.String() { + var idToSql func(string) model.Expr + timestampColumnName := model.TimestampFieldName + if column, ok := cw.Table.Cols[timestampColumnName]; ok { + switch column.Type.String() { case clickhouse.DateTime64.String(): - for _, id := range ids { - finalIds = append(finalIds, fmt.Sprintf("toDateTime64(%s,3)", id)) - } - if len(finalIds) == 1 { - whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " = ", model.NewFunction("toDateTime64", model.NewLiteral(ids[0]), model.NewLiteral("3"))) - } else { - whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " IN ", model.NewFunction("toDateTime64", model.NewLiteral(strings.Join(ids, ",")), model.NewLiteral("3"))) + idToSql = func(id string) model.Expr { + return model.NewFunction("toDateTime64", model.NewLiteral(id), model.NewLiteral(3)) } case clickhouse.DateTime.String(): - for _, id := range ids { - finalIds = append(finalIds, fmt.Sprintf("toDateTime(%s)", id)) - } - if len(finalIds) == 1 { - whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " = ", model.NewFunction("toDateTime", model.NewLiteral(finalIds[0]))) - } else { - whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " IN ", model.NewFunction("toDateTime", model.NewLiteral(strings.Join(ids, ",")))) + idToSql = func(id string) model.Expr { + return model.NewFunction("toDateTime", model.NewLiteral(id)) } default: - logger.Warn().Msgf("timestamp field of unsupported type %s", v.Type.String()) - return model.NewSimpleQuery(nil, true) + logger.ErrorWithCtx(cw.Ctx).Msgf("timestamp field of unsupported type %s", column.Type.String()) + return model.NewSimpleQueryInvalid() + } + } else { + logger.ErrorWithCtx(cw.Ctx).Msgf("timestamp field %s not found in schema", timestampColumnName) + return model.NewSimpleQueryInvalid() + } + + var whereStmt model.Expr + switch len(ids) { + case 0: + whereStmt = model.FalseExpr // timestamp IN [] <=> false + case 1: + whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " = ", idToSql(ids[0])) + default: + idsAsExprs := make([]model.Expr, len(ids)) + for i, id := range ids { + idsAsExprs[i] = idToSql(id) } + idsTuple := model.NewTupleExpr(idsAsExprs...) + whereStmt = model.NewInfixExpr(model.NewColumnRef(timestampColumnName), " IN ", idsTuple) } return model.NewSimpleQuery(whereStmt, true) } @@ -481,14 +487,14 @@ func (cw *ClickhouseQueryTranslator) parseTerm(queryMap QueryMap) model.SimpleQu } } logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 term, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // TODO remove optional parameters like boost func (cw *ClickhouseQueryTranslator) parseTerms(queryMap QueryMap) model.SimpleQuery { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 term, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } for k, v := range queryMap { @@ -500,7 +506,7 @@ func (cw *ClickhouseQueryTranslator) parseTerms(queryMap QueryMap) model.SimpleQ vAsArray, ok := v.([]interface{}) if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid terms type: %T, value: %v", v, v) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } if len(vAsArray) == 1 { simpleStatement := model.NewInfixExpr(model.NewColumnRef(k), "=", model.NewLiteral(sprint(vAsArray[0]))) @@ -517,7 +523,7 @@ func (cw *ClickhouseQueryTranslator) parseTerms(queryMap QueryMap) model.SimpleQ // unreachable unless something really weird happens logger.ErrorWithCtx(cw.Ctx).Msg("theoretically unreachable code") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } func (cw *ClickhouseQueryTranslator) parseMatchAll(_ QueryMap) model.SimpleQuery { @@ -537,7 +543,7 @@ func (cw *ClickhouseQueryTranslator) parseMatchAll(_ QueryMap) model.SimpleQuery func (cw *ClickhouseQueryTranslator) parseMatch(queryMap QueryMap, matchPhrase bool) model.SimpleQuery { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 match, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } for fieldName, v := range queryMap { @@ -575,7 +581,7 @@ func (cw *ClickhouseQueryTranslator) parseMatch(queryMap QueryMap, matchPhrase b // unreachable unless something really weird happens logger.ErrorWithCtx(cw.Ctx).Msg("theoretically unreachable code") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } func (cw *ClickhouseQueryTranslator) parseMultiMatch(queryMap QueryMap) model.SimpleQuery { @@ -586,25 +592,25 @@ func (cw *ClickhouseQueryTranslator) parseMultiMatch(queryMap QueryMap) model.Si fields = cw.extractFields(fieldsAsArray) } else { logger.ErrorWithCtx(cw.Ctx).Msgf("invalid fields type: %T, value: %v", fieldsAsInterface, fieldsAsInterface) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } else { fields = []string{model.FullTextFieldNamePlaceHolder} } - alwaysFalseStmt := model.NewLiteral("false") + if len(fields) == 0 { - return model.NewSimpleQuery(alwaysFalseStmt, true) + return model.NewSimpleQuery(model.FalseExpr, true) } query, ok := queryMap["query"] if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("no query in multi_match query: %v", queryMap) - return model.NewSimpleQuery(alwaysFalseStmt, false) + return model.NewSimpleQueryInvalid() } queryAsString, ok := query.(string) if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid query type: %T, value: %v", query, query) - return model.NewSimpleQuery(alwaysFalseStmt, false) + return model.NewSimpleQueryInvalid() } var subQueries []string wereDone := false @@ -637,7 +643,7 @@ func (cw *ClickhouseQueryTranslator) parseMultiMatch(queryMap QueryMap) model.Si func (cw *ClickhouseQueryTranslator) parsePrefix(queryMap QueryMap) model.SimpleQuery { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 prefix, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } for fieldName, v := range queryMap { @@ -652,13 +658,13 @@ func (cw *ClickhouseQueryTranslator) parsePrefix(queryMap QueryMap) model.Simple return model.NewSimpleQuery(simpleStat, true) default: logger.WarnWithCtx(cw.Ctx).Msgf("unsupported prefix type: %T, value: %v", v, v) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } // unreachable unless something really weird happens logger.ErrorWithCtx(cw.Ctx).Msg("theoretically unreachable code") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // Not supporting 'case_insensitive' (optional) @@ -667,7 +673,7 @@ func (cw *ClickhouseQueryTranslator) parsePrefix(queryMap QueryMap) model.Simple func (cw *ClickhouseQueryTranslator) parseWildcard(queryMap QueryMap) model.SimpleQuery { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 wildcard, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } for fieldName, v := range queryMap { @@ -679,21 +685,21 @@ func (cw *ClickhouseQueryTranslator) parseWildcard(queryMap QueryMap) model.Simp return model.NewSimpleQuery(whereStatement, true) } else { logger.WarnWithCtx(cw.Ctx).Msgf("invalid value type: %T, value: %v", value, value) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } else { logger.WarnWithCtx(cw.Ctx).Msgf("no value in wildcard query: %v", queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } else { logger.WarnWithCtx(cw.Ctx).Msgf("invalid wildcard type: %T, value: %v", v, v) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } // unreachable unless something really weird happens logger.ErrorWithCtx(cw.Ctx).Msg("theoretically unreachable code") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // This one is really complicated (https://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl-query-string-query.html) @@ -719,12 +725,12 @@ func (cw *ClickhouseQueryTranslator) parseNested(queryMap QueryMap) model.Simple return cw.parseQueryMap(queryAsMap) } else { logger.WarnWithCtx(cw.Ctx).Msgf("invalid nested query type: %T, value: %v", query, query) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } } logger.WarnWithCtx(cw.Ctx).Msgf("no query in nested query: %v", queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } func (cw *ClickhouseQueryTranslator) parseDateMathExpression(expr string) (string, error) { @@ -755,7 +761,7 @@ func (cw *ClickhouseQueryTranslator) parseDateMathExpression(expr string) (strin func (cw *ClickhouseQueryTranslator) parseRange(queryMap QueryMap) model.SimpleQuery { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 range, got: %d. value: %v", len(queryMap), queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // Maybe change to false if numeric fields exist. @@ -848,7 +854,7 @@ func (cw *ClickhouseQueryTranslator) parseRange(queryMap QueryMap) model.SimpleQ // unreachable unless something really weird happens logger.ErrorWithCtx(cw.Ctx).Msg("theoretically unreachable code") - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } // TODO: not supported: @@ -862,7 +868,7 @@ func (cw *ClickhouseQueryTranslator) parseExists(queryMap QueryMap) model.Simple fieldName, ok := v.(string) if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid exists type: %T, value: %v", v, v) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } sql = model.NewInfixExpr(model.NewColumnRef(fieldName), "IS", model.NewLiteral("NOT NULL")) @@ -877,7 +883,7 @@ func (cw *ClickhouseQueryTranslator) parseExists(queryMap QueryMap) model.Simple func (cw *ClickhouseQueryTranslator) parseRegexp(queryMap QueryMap) (result model.SimpleQuery) { if len(queryMap) != 1 { logger.WarnWithCtx(cw.Ctx).Msgf("we expect only 1 regexp, got: %d. value: %v", len(queryMap), queryMap) - return + return model.NewSimpleQueryInvalid() } // really simple == (out of all special characters, only . and .* may be present) @@ -899,17 +905,17 @@ func (cw *ClickhouseQueryTranslator) parseRegexp(queryMap QueryMap) (result mode parameters, ok := parametersRaw.(QueryMap) if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid regexp parameters type: %T, value: %v", parametersRaw, parametersRaw) - return + return model.NewSimpleQueryInvalid() } patternRaw, exists := parameters["value"] if !exists { logger.WarnWithCtx(cw.Ctx).Msgf("no value in regexp query: %v", queryMap) - return + return model.NewSimpleQueryInvalid() } pattern, ok := patternRaw.(string) if !ok { logger.WarnWithCtx(cw.Ctx).Msgf("invalid pattern type: %T, value: %v", patternRaw, patternRaw) - return + return model.NewSimpleQueryInvalid() } if len(parameters) > 1 { @@ -930,7 +936,7 @@ func (cw *ClickhouseQueryTranslator) parseRegexp(queryMap QueryMap) (result mode } logger.ErrorWithCtx(cw.Ctx).Msg("parseRegexp: theoretically unreachable code") - return + return model.NewSimpleQueryInvalid() } func (cw *ClickhouseQueryTranslator) extractFields(fields []interface{}) []string { @@ -1246,7 +1252,7 @@ func (cw *ClickhouseQueryTranslator) parseGeoBoundingBox(queryMap QueryMap) mode } } else { logger.WarnWithCtx(cw.Ctx).Msgf("no bottom_right in geo_bounding_box query: %v", queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } if topLeft, ok := v.(QueryMap)["top_left"]; ok { if topLeftCornerAsArray, ok := topLeft.([]interface{}); ok { @@ -1255,7 +1261,7 @@ func (cw *ClickhouseQueryTranslator) parseGeoBoundingBox(queryMap QueryMap) mode } } else { logger.WarnWithCtx(cw.Ctx).Msgf("no top_left in geo_bounding_box query: %v", queryMap) - return model.NewSimpleQuery(nil, false) + return model.NewSimpleQueryInvalid() } args := make([]model.Expr, 0) args = append(args, model.NewColumnRef(field)) diff --git a/quesma/queryparser/query_parser_test.go b/quesma/queryparser/query_parser_test.go index b857a8574..bd6322898 100644 --- a/quesma/queryparser/query_parser_test.go +++ b/quesma/queryparser/query_parser_test.go @@ -6,6 +6,7 @@ import ( "context" "fmt" "quesma/clickhouse" + "quesma/logger" "quesma/model" "quesma/model/typical_queries" "quesma/persistence" @@ -27,6 +28,7 @@ import ( // what should be? According to docs, I think so... Maybe test in Kibana? // OK, Kibana disagrees, it is indeed wrong. func TestQueryParserStringAttrConfig(t *testing.T) { + logger.InitSimpleLoggerForTestsWarnLevel() tableName := "logs-generic-default" table, err := clickhouse.NewTable(`CREATE TABLE `+tableName+` ( "message" String, "@timestamp" DateTime64(3, 'UTC'), "attributes_values" Map(String,String)) diff --git a/quesma/testdata/requests.go b/quesma/testdata/requests.go index 6ac614cbb..a1fc2bd0f 100644 --- a/quesma/testdata/requests.go +++ b/quesma/testdata/requests.go @@ -2320,6 +2320,69 @@ var TestsSearch = []SearchTestCase{ }, []string{}, }, + { // [40] + "ids, 0 values", + `{ + "query": { + "ids": { + "values": [] + } + }, + "track_total_hits": false + }`, + []string{`false`}, + model.ListAllFields, + []string{ + `SELECT "message" ` + + `FROM ` + TableName + ` ` + + `WHERE false ` + + `LIMIT 10`, + }, + []string{}, + }, + { // [41] + "ids, 1 value", + `{ + "query": { + "ids": { + "values": ["323032342d31322d32312030373a32393a30332e333637202b3030303020555443q1"] + } + }, + "track_total_hits": false + }`, + []string{`"@timestamp" = toDateTime64('2024-12-21 07:29:03.367',3)`}, + model.ListAllFields, + []string{ + `SELECT "message" ` + + `FROM ` + TableName + ` ` + + `WHERE "@timestamp" = toDateTime64('2024-12-21 07:29:03.367',3) ` + + `LIMIT 10`, + }, + []string{}, + }, + { // [42] + "ids, 2+ values", + `{ + "query": { + "ids": { + "values": [ + "323032342d31322d32312030373a32393a30332e333637202b3030303020555443q1", + "323032342d31322d32312030373a32393a30322e393932202b3030303020555443q3" + ] + } + }, + "track_total_hits": false + }`, + []string{`"@timestamp" IN tuple(toDateTime64('2024-12-21 07:29:03.367',3), toDateTime64('2024-12-21 07:29:02.992',3))`}, + model.ListAllFields, + []string{ + `SELECT "message" ` + + `FROM ` + TableName + ` ` + + `WHERE "@timestamp" IN tuple(toDateTime64('2024-12-21 07:29:03.367',3), toDateTime64('2024-12-21 07:29:02.992',3)) ` + + `LIMIT 10`, + }, + []string{}, + }, } var TestSearchRuntimeMappings = []SearchTestCase{