diff --git a/pkg/querybuilder/resourcefilter/statement_builder.go b/pkg/querybuilder/resourcefilter/statement_builder.go index 315d227b2701..977e2d36d682 100644 --- a/pkg/querybuilder/resourcefilter/statement_builder.go +++ b/pkg/querybuilder/resourcefilter/statement_builder.go @@ -164,6 +164,7 @@ func (b *resourceFilterStatementBuilder[T]) addConditions( FullTextColumn: b.fullTextColumn, JsonBodyPrefix: b.jsonBodyPrefix, JsonKeyToKey: b.jsonKeyToKey, + OnlyResourceFilter: true, // Only process resource terms SkipFullTextFilter: true, SkipFunctionCalls: true, // there is no need for "key" not found error for resource filtering diff --git a/pkg/querybuilder/where_clause_visitor.go b/pkg/querybuilder/where_clause_visitor.go index ba4ad3e36824..2c271a5650b4 100644 --- a/pkg/querybuilder/where_clause_visitor.go +++ b/pkg/querybuilder/where_clause_visitor.go @@ -18,6 +18,199 @@ import ( var searchTroubleshootingGuideURL = "https://signoz.io/docs/userguide/search-troubleshooting/" +// BooleanExpression represents a boolean expression with proper evaluation context +type BooleanExpression struct { + SQL string + IsTrue bool + IsEmpty bool +} + +// NewBooleanExpression creates a BooleanExpression from SQL +func NewBooleanExpression(sql string) BooleanExpression { + return BooleanExpression{ + SQL: sql, + IsTrue: sql == "true", + IsEmpty: sql == "", + } +} + +// booleanEvaluatingVisitor is a specialized visitor for resource filter context +// that properly applies boolean algebra during tree traversal +type booleanEvaluatingVisitor struct { + *filterExpressionVisitor +} + +func newBooleanEvaluatingVisitor(opts FilterExprVisitorOpts) *booleanEvaluatingVisitor { + return &booleanEvaluatingVisitor{ + filterExpressionVisitor: newFilterExpressionVisitor(opts), + } +} + +// Visit dispatches to boolean-aware visit methods +func (v *booleanEvaluatingVisitor) Visit(tree antlr.ParseTree) any { + if tree == nil { + return NewBooleanExpression("") + } + + switch t := tree.(type) { + case *grammar.QueryContext: + return v.VisitQuery(t) + case *grammar.ExpressionContext: + return v.VisitExpression(t) + case *grammar.OrExpressionContext: + return v.VisitOrExpression(t) + case *grammar.AndExpressionContext: + return v.VisitAndExpression(t) + case *grammar.UnaryExpressionContext: + return v.VisitUnaryExpression(t) + case *grammar.PrimaryContext: + return v.VisitPrimary(t) + default: + // For leaf nodes, delegate to original visitor and wrap result + result := v.filterExpressionVisitor.Visit(tree) + if sql, ok := result.(string); ok { + return NewBooleanExpression(sql) + } + return NewBooleanExpression("") + } +} + +func (v *booleanEvaluatingVisitor) VisitQuery(ctx *grammar.QueryContext) any { + return v.Visit(ctx.Expression()) +} + +func (v *booleanEvaluatingVisitor) VisitExpression(ctx *grammar.ExpressionContext) any { + return v.Visit(ctx.OrExpression()) +} + +func (v *booleanEvaluatingVisitor) VisitOrExpression(ctx *grammar.OrExpressionContext) any { + andExpressions := ctx.AllAndExpression() + + var result BooleanExpression + hasTrue := false + hasEmpty := false + + for i, expr := range andExpressions { + exprResult := v.Visit(expr).(BooleanExpression) + if exprResult.IsTrue { + hasTrue = true + } + if exprResult.IsEmpty { + hasEmpty = true + } + + if i == 0 { + result = exprResult + } else { + if result.IsEmpty { + result = exprResult + } else if !exprResult.IsEmpty { + sql := v.builder.Or(result.SQL, exprResult.SQL) + result = NewBooleanExpression(sql) + } + } + } + + // In resource filter context, if any operand is empty (meaning "include all resources"), + // the entire OR should be empty (include all resources) + if hasEmpty && v.onlyResourceFilter { + result.IsEmpty = true + result.IsTrue = true + result.SQL = "" + } else if hasTrue { + // Mark as always true if any operand is true, but preserve the SQL structure + result.IsTrue = true + } + + return result +} + +func (v *booleanEvaluatingVisitor) VisitAndExpression(ctx *grammar.AndExpressionContext) any { + unaryExpressions := ctx.AllUnaryExpression() + + var result BooleanExpression + allTrue := true + + for i, expr := range unaryExpressions { + exprResult := v.Visit(expr).(BooleanExpression) + if !exprResult.IsTrue && !exprResult.IsEmpty { + allTrue = false + } + + if i == 0 { + result = exprResult + } else { + // Apply boolean AND logic + if exprResult.IsTrue { + // A AND true = A, continue with result + continue + } + if result.IsTrue { + result = exprResult + } else if result.IsEmpty { + result = exprResult + } else if !exprResult.IsEmpty { + sql := v.builder.And(result.SQL, exprResult.SQL) + result = NewBooleanExpression(sql) + } + } + } + + // If all terms were "true", mark the result as always true + if allTrue && len(unaryExpressions) > 0 { + result.IsTrue = true + if result.SQL == "" { + result.SQL = "true" + } + } + + return result +} + +func (v *booleanEvaluatingVisitor) VisitUnaryExpression(ctx *grammar.UnaryExpressionContext) any { + result := v.Visit(ctx.Primary()).(BooleanExpression) + + if ctx.NOT() != nil { + // Apply NOT logic with resource filter context awareness + if v.onlyResourceFilter { + if result.IsTrue { + return NewBooleanExpression("") // NOT(true) = include all resources + } + if result.IsEmpty { + return NewBooleanExpression("") // NOT(empty) = include all resources + } + } + + sql := fmt.Sprintf("NOT (%s)", result.SQL) + return NewBooleanExpression(sql) + } + + return result +} + +func (v *booleanEvaluatingVisitor) VisitPrimary(ctx *grammar.PrimaryContext) any { + if ctx.OrExpression() != nil { + result := v.Visit(ctx.OrExpression()).(BooleanExpression) + // If no boolean simplification happened, preserve original parentheses structure + if !result.IsEmpty && !result.IsTrue { + // Use original visitor to get proper parentheses structure + originalSQL := v.filterExpressionVisitor.Visit(ctx) + if sql, ok := originalSQL.(string); ok && sql != "" { + return NewBooleanExpression(sql) + } + result.SQL = fmt.Sprintf("(%s)", result.SQL) + } + return result + } + + // For other cases, delegate to original visitor + sqlResult := v.filterExpressionVisitor.Visit(ctx) + if sql, ok := sqlResult.(string); ok { + return NewBooleanExpression(sql) + } + return NewBooleanExpression("") +} + // filterExpressionVisitor implements the FilterQueryVisitor interface // to convert the parsed filter expressions into ClickHouse WHERE clause type filterExpressionVisitor struct { @@ -34,6 +227,7 @@ type filterExpressionVisitor struct { jsonBodyPrefix string jsonKeyToKey qbtypes.JsonKeyToFieldFunc skipResourceFilter bool + onlyResourceFilter bool skipFullTextFilter bool skipFunctionCalls bool ignoreNotFoundKeys bool @@ -52,6 +246,7 @@ type FilterExprVisitorOpts struct { JsonBodyPrefix string JsonKeyToKey qbtypes.JsonKeyToFieldFunc SkipResourceFilter bool + OnlyResourceFilter bool // Only process resource terms, skip non-resource terms SkipFullTextFilter bool SkipFunctionCalls bool IgnoreNotFoundKeys bool @@ -70,6 +265,7 @@ func newFilterExpressionVisitor(opts FilterExprVisitorOpts) *filterExpressionVis jsonBodyPrefix: opts.JsonBodyPrefix, jsonKeyToKey: opts.JsonKeyToKey, skipResourceFilter: opts.SkipResourceFilter, + onlyResourceFilter: opts.OnlyResourceFilter, skipFullTextFilter: opts.SkipFullTextFilter, skipFunctionCalls: opts.SkipFunctionCalls, ignoreNotFoundKeys: opts.IgnoreNotFoundKeys, @@ -160,6 +356,31 @@ func PrepareWhereClause(query string, opts FilterExprVisitorOpts) (*PreparedWher cond = "true" } + // In resource filter context, apply robust boolean evaluation only when needed + if opts.OnlyResourceFilter { + // Check if the condition contains patterns that need boolean simplification + // We need boolean evaluation when: + // 1. Expression contains " true" (indicating simplified non-resource terms) + // 2. Expression is exactly "true" + // 3. Expression contains "NOT" with true values that need simplification + needsBooleanEval := strings.Contains(cond, " true") || + cond == "true" || + (strings.Contains(cond, "NOT") && strings.Contains(cond, "true")) + + if needsBooleanEval { + // Re-parse and evaluate with boolean algebra + boolVisitor := newBooleanEvaluatingVisitor(opts) + boolResult := boolVisitor.Visit(tree) + if boolExpr, ok := boolResult.(BooleanExpression); ok { + if boolExpr.IsEmpty { + cond = "true" // Empty means include all resources + } else { + cond = boolExpr.SQL + } + } + } + } + whereClause := sqlbuilder.NewWhereClause().AddWhereExpr(visitor.builder.Args, cond) return &PreparedWhereClause{whereClause, visitor.warnings, visitor.mainWarnURL}, nil @@ -226,22 +447,23 @@ func (v *filterExpressionVisitor) VisitExpression(ctx *grammar.ExpressionContext func (v *filterExpressionVisitor) VisitOrExpression(ctx *grammar.OrExpressionContext) any { andExpressions := ctx.AllAndExpression() - andExpressionConditions := make([]string, len(andExpressions)) - for i, expr := range andExpressions { + validConditions := []string{} + + for _, expr := range andExpressions { if condExpr, ok := v.Visit(expr).(string); ok && condExpr != "" { - andExpressionConditions[i] = condExpr + validConditions = append(validConditions, condExpr) } } - if len(andExpressionConditions) == 0 { + if len(validConditions) == 0 { return "" } - if len(andExpressionConditions) == 1 { - return andExpressionConditions[0] + if len(validConditions) == 1 { + return validConditions[0] } - return v.builder.Or(andExpressionConditions...) + return v.builder.Or(validConditions...) } // VisitAndExpression handles AND expressions @@ -272,6 +494,17 @@ func (v *filterExpressionVisitor) VisitUnaryExpression(ctx *grammar.UnaryExpress // Check if this is a NOT expression if ctx.NOT() != nil { + // In resource filter context, handle NOT specially + if v.onlyResourceFilter { + // NOT(true) means NOT(all non-resource terms) which means "include all resources" + if result == "true" { + return "" // No filtering = include all resources + } + // NOT(empty) should return empty (no filtering) + if result == "" { + return "" + } + } return fmt.Sprintf("NOT (%s)", result) } @@ -283,7 +516,7 @@ func (v *filterExpressionVisitor) VisitPrimary(ctx *grammar.PrimaryContext) any if ctx.OrExpression() != nil { // This is a parenthesized expression if condExpr, ok := v.Visit(ctx.OrExpression()).(string); ok && condExpr != "" { - return fmt.Sprintf("(%s)", v.Visit(ctx.OrExpression()).(string)) + return fmt.Sprintf("(%s)", condExpr) } return "" } else if ctx.Comparison() != nil { @@ -365,6 +598,22 @@ func (v *filterExpressionVisitor) VisitComparison(ctx *grammar.ComparisonContext } } + // this is used to only process resource terms in resource filter context + if v.onlyResourceFilter { + filteredKeys := []*telemetrytypes.TelemetryFieldKey{} + for _, key := range keys { + if key.FieldContext == telemetrytypes.FieldContextResource { + filteredKeys = append(filteredKeys, key) + } + } + keys = filteredKeys + if len(keys) == 0 { + // For non-resource terms in resource filter context, return "true" + // This ensures OR expressions work correctly (e.g., resource OR non-resource) + return "true" + } + } + // Handle EXISTS specially if ctx.EXISTS() != nil { op := qbtypes.FilterOperatorExists diff --git a/pkg/telemetrylogs/stmt_builder_test.go b/pkg/telemetrylogs/stmt_builder_test.go index ed23fafc183c..54f72df501ca 100644 --- a/pkg/telemetrylogs/stmt_builder_test.go +++ b/pkg/telemetrylogs/stmt_builder_test.go @@ -142,6 +142,111 @@ func TestStatementBuilderTimeSeries(t *testing.T) { }, expectedErr: nil, }, + { + name: "Time series with NOT predicate containing only non-resource terms", + requestType: qbtypes.RequestTypeTimeSeries, + query: qbtypes.QueryBuilderQuery[qbtypes.LogAggregation]{ + Signal: telemetrytypes.SignalLogs, + StepInterval: qbtypes.Step{Duration: 60 * time.Second}, + Aggregations: []qbtypes.LogAggregation{ + { + Expression: "count()", + }, + }, + Filter: &qbtypes.Filter{ + Expression: "NOT (message CONTAINS 'foo' AND hasToken(body, 'bar'))", + }, + }, + expected: qbtypes.Statement{ + Query: "WITH __resource_filter AS (SELECT fingerprint FROM signoz_logs.distributed_logs_v2_resource WHERE true AND seen_at_ts_bucket_start >= ? AND seen_at_ts_bucket_start <= ?) SELECT toStartOfInterval(fromUnixTimestamp64Nano(timestamp), INTERVAL 60 SECOND) AS ts, count() AS __result_0 FROM signoz_logs.distributed_logs_v2 WHERE resource_fingerprint GLOBAL IN (SELECT fingerprint FROM __resource_filter) AND NOT ((((LOWER(attributes_string['message']) LIKE LOWER(?) AND mapContains(attributes_string, 'message') = ?) AND hasToken(LOWER(body), LOWER(?))))) AND timestamp >= ? AND ts_bucket_start >= ? AND timestamp < ? AND ts_bucket_start <= ? GROUP BY ts", + Args: []any{uint64(1747945619), uint64(1747983448), "%foo%", true, "bar", "1747947419000000000", uint64(1747945619), "1747983448000000000", uint64(1747983448)}, + }, + expectedErr: nil, + }, + { + name: "Time series with NOT OR mixed resource/non-resource terms", + requestType: qbtypes.RequestTypeTimeSeries, + query: qbtypes.QueryBuilderQuery[qbtypes.LogAggregation]{ + Signal: telemetrytypes.SignalLogs, + StepInterval: qbtypes.Step{Duration: 60 * time.Second}, + Aggregations: []qbtypes.LogAggregation{ + { + Expression: "count()", + }, + }, + Filter: &qbtypes.Filter{ + Expression: "NOT (service.name = 'redis-manual' OR http.method = 'GET')", + }, + }, + expected: qbtypes.Statement{ + Query: "WITH __resource_filter AS (SELECT fingerprint FROM signoz_logs.distributed_logs_v2_resource WHERE true AND seen_at_ts_bucket_start >= ? AND seen_at_ts_bucket_start <= ?) SELECT toStartOfInterval(fromUnixTimestamp64Nano(timestamp), INTERVAL 60 SECOND) AS ts, count() AS __result_0 FROM signoz_logs.distributed_logs_v2 WHERE resource_fingerprint GLOBAL IN (SELECT fingerprint FROM __resource_filter) AND NOT ((((resources_string['service.name'] = ? AND mapContains(resources_string, 'service.name') = ?) OR (attributes_string['http.method'] = ? AND mapContains(attributes_string, 'http.method') = ?)))) AND timestamp >= ? AND ts_bucket_start >= ? AND timestamp < ? AND ts_bucket_start <= ? GROUP BY ts", + Args: []any{uint64(1747945619), uint64(1747983448), "redis-manual", true, "GET", true, "1747947419000000000", uint64(1747945619), "1747983448000000000", uint64(1747983448)}, + }, + expectedErr: nil, + }, + { + name: "Time series with complex NOT expression and nested OR conditions", + requestType: qbtypes.RequestTypeTimeSeries, + query: qbtypes.QueryBuilderQuery[qbtypes.LogAggregation]{ + Signal: telemetrytypes.SignalLogs, + StepInterval: qbtypes.Step{Duration: 60 * time.Second}, + Aggregations: []qbtypes.LogAggregation{ + { + Expression: "count()", + }, + }, + Filter: &qbtypes.Filter{ + Expression: "service.name IN 'redis' AND request.type = 'External' AND http.status_code < 500 AND http.status_code >= 400 AND NOT ((http.request.header.tenant_id = '[\"tenant-1\"]' AND http.status_code = 401) OR (http.request.header.tenant_id = '[\"tenant-2\"]' AND http.status_code = 404 AND http.route = '/tenants/{tenant_id}'))", + }, + }, + expected: qbtypes.Statement{ + Query: "WITH __resource_filter AS (SELECT fingerprint FROM signoz_logs.distributed_logs_v2_resource WHERE ((simpleJSONExtractString(labels, 'service.name') = ?) AND labels LIKE ? AND (labels LIKE ?)) AND seen_at_ts_bucket_start >= ? AND seen_at_ts_bucket_start <= ?) SELECT toStartOfInterval(fromUnixTimestamp64Nano(timestamp), INTERVAL 60 SECOND) AS ts, count() AS __result_0 FROM signoz_logs.distributed_logs_v2 WHERE resource_fingerprint GLOBAL IN (SELECT fingerprint FROM __resource_filter) AND (((resources_string['service.name'] = ?) AND mapContains(resources_string, 'service.name') = ?) AND (attributes_string['request.type'] = ? AND mapContains(attributes_string, 'request.type') = ?) AND (toFloat64(attributes_number['http.status_code']) < ? AND mapContains(attributes_number, 'http.status_code') = ?) AND (toFloat64(attributes_number['http.status_code']) >= ? AND mapContains(attributes_number, 'http.status_code') = ?) AND NOT ((((((attributes_string['http.request.header.tenant_id'] = ? AND mapContains(attributes_string, 'http.request.header.tenant_id') = ?) AND (toFloat64(attributes_number['http.status_code']) = ? AND mapContains(attributes_number, 'http.status_code') = ?))) OR (((attributes_string['http.request.header.tenant_id'] = ? AND mapContains(attributes_string, 'http.request.header.tenant_id') = ?) AND (toFloat64(attributes_number['http.status_code']) = ? AND mapContains(attributes_number, 'http.status_code') = ?) AND (attributes_string['http.route'] = ? AND mapContains(attributes_string, 'http.route') = ?))))))) AND timestamp >= ? AND ts_bucket_start >= ? AND timestamp < ? AND ts_bucket_start <= ? GROUP BY ts", + Args: []any{"redis", "%service.name%", "%service.name\":\"redis%", uint64(1747945619), uint64(1747983448), "redis", true, "External", true, float64(500), true, float64(400), true, "[\"tenant-1\"]", true, float64(401), true, "[\"tenant-2\"]", true, float64(404), true, "/tenants/{tenant_id}", true, "1747947419000000000", uint64(1747945619), "1747983448000000000", uint64(1747983448)}, + }, + expectedErr: nil, + }, + { + name: "Time series with complex OR expression containing NOT with nested conditions", + requestType: qbtypes.RequestTypeTimeSeries, + query: qbtypes.QueryBuilderQuery[qbtypes.LogAggregation]{ + Signal: telemetrytypes.SignalLogs, + StepInterval: qbtypes.Step{Duration: 60 * time.Second}, + Aggregations: []qbtypes.LogAggregation{ + { + Expression: "count()", + }, + }, + Filter: &qbtypes.Filter{ + Expression: "(service.name IN 'redis' AND request.type = 'External' AND http.status_code < 500 AND http.status_code >= 400 OR NOT ((http.request.header.tenant_id = '[\"tenant-1\"]' AND http.status_code = 401) OR (http.request.header.tenant_id = '[\"tenant-2\"]' AND http.status_code = 404 AND http.route = '/tenants/{tenant_id}')))", + }, + }, + expected: qbtypes.Statement{ + Query: "WITH __resource_filter AS (SELECT fingerprint FROM signoz_logs.distributed_logs_v2_resource WHERE true AND seen_at_ts_bucket_start >= ? AND seen_at_ts_bucket_start <= ?) SELECT toStartOfInterval(fromUnixTimestamp64Nano(timestamp), INTERVAL 60 SECOND) AS ts, count() AS __result_0 FROM signoz_logs.distributed_logs_v2 WHERE resource_fingerprint GLOBAL IN (SELECT fingerprint FROM __resource_filter) AND (((((resources_string['service.name'] = ?) AND mapContains(resources_string, 'service.name') = ?) AND (attributes_string['request.type'] = ? AND mapContains(attributes_string, 'request.type') = ?) AND (toFloat64(attributes_number['http.status_code']) < ? AND mapContains(attributes_number, 'http.status_code') = ?) AND (toFloat64(attributes_number['http.status_code']) >= ? AND mapContains(attributes_number, 'http.status_code') = ?)) OR NOT ((((((attributes_string['http.request.header.tenant_id'] = ? AND mapContains(attributes_string, 'http.request.header.tenant_id') = ?) AND (toFloat64(attributes_number['http.status_code']) = ? AND mapContains(attributes_number, 'http.status_code') = ?))) OR (((attributes_string['http.request.header.tenant_id'] = ? AND mapContains(attributes_string, 'http.request.header.tenant_id') = ?) AND (toFloat64(attributes_number['http.status_code']) = ? AND mapContains(attributes_number, 'http.status_code') = ?) AND (attributes_string['http.route'] = ? AND mapContains(attributes_string, 'http.route') = ?)))))))) AND timestamp >= ? AND ts_bucket_start >= ? AND timestamp < ? AND ts_bucket_start <= ? GROUP BY ts", + Args: []any{uint64(1747945619), uint64(1747983448), "redis", true, "External", true, float64(500), true, float64(400), true, "[\"tenant-1\"]", true, float64(401), true, "[\"tenant-2\"]", true, float64(404), true, "/tenants/{tenant_id}", true, "1747947419000000000", uint64(1747945619), "1747983448000000000", uint64(1747983448)}, + }, + expectedErr: nil, + }, + { + name: "Time series with OR between multiple resource conditions", + requestType: qbtypes.RequestTypeTimeSeries, + query: qbtypes.QueryBuilderQuery[qbtypes.LogAggregation]{ + Signal: telemetrytypes.SignalLogs, + StepInterval: qbtypes.Step{Duration: 60 * time.Second}, + Aggregations: []qbtypes.LogAggregation{ + { + Expression: "count()", + }, + }, + Filter: &qbtypes.Filter{ + Expression: "service.name = 'redis' OR service.name = 'driver'", + }, + }, + expected: qbtypes.Statement{ + Query: "WITH __resource_filter AS (SELECT fingerprint FROM signoz_logs.distributed_logs_v2_resource WHERE ((simpleJSONExtractString(labels, 'service.name') = ? AND labels LIKE ? AND labels LIKE ?) OR (simpleJSONExtractString(labels, 'service.name') = ? AND labels LIKE ? AND labels LIKE ?)) AND seen_at_ts_bucket_start >= ? AND seen_at_ts_bucket_start <= ?) SELECT toStartOfInterval(fromUnixTimestamp64Nano(timestamp), INTERVAL 60 SECOND) AS ts, count() AS __result_0 FROM signoz_logs.distributed_logs_v2 WHERE resource_fingerprint GLOBAL IN (SELECT fingerprint FROM __resource_filter) AND ((resources_string['service.name'] = ? AND mapContains(resources_string, 'service.name') = ?) OR (resources_string['service.name'] = ? AND mapContains(resources_string, 'service.name') = ?)) AND timestamp >= ? AND ts_bucket_start >= ? AND timestamp < ? AND ts_bucket_start <= ? GROUP BY ts", + Args: []any{"redis", "%service.name%", "%service.name\":\"redis%", "driver", "%service.name%", "%service.name\":\"driver%", uint64(1747945619), uint64(1747983448), "redis", true, "driver", true, "1747947419000000000", uint64(1747945619), "1747983448000000000", uint64(1747983448)}, + }, + expectedErr: nil, + }, } fm := NewFieldMapper() diff --git a/pkg/telemetrylogs/test_data.go b/pkg/telemetrylogs/test_data.go index 74714361bbd7..4332f989f309 100644 --- a/pkg/telemetrylogs/test_data.go +++ b/pkg/telemetrylogs/test_data.go @@ -862,6 +862,27 @@ func buildCompleteFieldKeyMap() map[string][]*telemetrytypes.TelemetryFieldKey { Materialized: true, }, }, + "request.type": { + { + Name: "request.type", + FieldContext: telemetrytypes.FieldContextAttribute, + FieldDataType: telemetrytypes.FieldDataTypeString, + }, + }, + "http.request.header.tenant_id": { + { + Name: "http.request.header.tenant_id", + FieldContext: telemetrytypes.FieldContextAttribute, + FieldDataType: telemetrytypes.FieldDataTypeString, + }, + }, + "http.route": { + { + Name: "http.route", + FieldContext: telemetrytypes.FieldContextAttribute, + FieldDataType: telemetrytypes.FieldDataTypeString, + }, + }, } for _, keys := range keysMap {