From 22107421eb184765ae596c48d85bf357a2949e52 Mon Sep 17 00:00:00 2001 From: Aliaksandr Valialkin Date: Wed, 22 May 2024 21:01:20 +0200 Subject: [PATCH] lib/logstorage: work-in-progress --- app/vlinsert/elasticsearch/elasticsearch.go | 2 +- app/vlinsert/jsonline/jsonline.go | 2 +- app/vlselect/logsql/logsql.go | 105 ++++- .../logsql/stream_label_names_response.qtpl | 17 + .../stream_label_names_response.qtpl.go | 69 +++ .../logsql/stream_label_values_response.qtpl | 17 + .../stream_label_values_response.qtpl.go | 69 +++ app/vlselect/logsql/streams_response.qtpl | 17 + app/vlselect/logsql/streams_response.qtpl.go | 69 +++ app/vlselect/main.go | 55 ++- app/vlstorage/main.go | 19 + docs/VictoriaLogs/CHANGELOG.md | 13 + docs/VictoriaLogs/LogsQL.md | 366 +++++++++++---- docs/VictoriaLogs/querying/README.md | 147 +++++- lib/logstorage/bitmap.go | 7 + lib/logstorage/block_result.go | 161 ++++--- lib/logstorage/filter_exact.go | 4 +- lib/logstorage/filter_exact_prefix.go | 2 +- lib/logstorage/filter_regexp.go | 2 +- lib/logstorage/if_filter.go | 75 ++++ lib/logstorage/json_parser.go | 35 +- lib/logstorage/json_parser_test.go | 44 +- lib/logstorage/logfmt_parser.go | 77 ++++ lib/logstorage/logfmt_parser_test.go | 30 ++ lib/logstorage/parser.go | 87 ++-- lib/logstorage/parser_test.go | 180 ++++++-- lib/logstorage/pattern.go | 229 ++++++++++ lib/logstorage/pattern_test.go | 239 ++++++++++ ..._timing_test.go => pattern_timing_test.go} | 38 +- lib/logstorage/pipe.go | 22 +- lib/logstorage/pipe_copy.go | 28 +- lib/logstorage/pipe_copy_test.go | 181 ++++++++ lib/logstorage/pipe_delete_test.go | 105 +++++ lib/logstorage/pipe_extract.go | 339 +++----------- lib/logstorage/pipe_extract_test.go | 398 ++++++++++------- lib/logstorage/pipe_field_names.go | 8 +- lib/logstorage/pipe_field_names_test.go | 58 +++ lib/logstorage/pipe_fields.go | 2 +- lib/logstorage/pipe_fields_test.go | 129 ++++++ lib/logstorage/pipe_filter_test.go | 97 ++++ lib/logstorage/pipe_format.go | 200 +++++++++ lib/logstorage/pipe_format_test.go | 201 +++++++++ lib/logstorage/pipe_limit_test.go | 92 ++++ lib/logstorage/pipe_offset_test.go | 86 ++++ lib/logstorage/pipe_rename.go | 32 +- lib/logstorage/pipe_rename_test.go | 170 +++++++ lib/logstorage/pipe_sort.go | 18 +- lib/logstorage/pipe_sort_test.go | 248 +++++++++++ lib/logstorage/pipe_stats.go | 136 +++--- lib/logstorage/pipe_stats_test.go | 370 ++++++++++++++++ lib/logstorage/pipe_topk.go | 23 +- lib/logstorage/pipe_uniq.go | 16 +- lib/logstorage/pipe_uniq_test.go | 170 +++++++ lib/logstorage/pipe_unpack.go | 258 +++++++++++ lib/logstorage/pipe_unpack_json.go | 188 ++++---- lib/logstorage/pipe_unpack_json_test.go | 159 ++++++- lib/logstorage/pipe_unpack_logfmt.go | 285 +++--------- lib/logstorage/pipe_unpack_logfmt_test.go | 98 +++- lib/logstorage/rows.go | 33 +- lib/logstorage/stats_avg.go | 61 ++- lib/logstorage/stats_avg_test.go | 412 +++++++++++++++++ lib/logstorage/stats_count.go | 16 +- lib/logstorage/stats_count_empty.go | 19 +- lib/logstorage/stats_count_empty_test.go | 322 ++++++++++++++ lib/logstorage/stats_count_test.go | 326 ++++++++++++++ lib/logstorage/stats_count_uniq.go | 107 ++--- lib/logstorage/stats_count_uniq_test.go | 373 ++++++++++++++++ lib/logstorage/stats_fields_max.go | 241 ++++++++++ lib/logstorage/stats_fields_max_test.go | 286 ++++++++++++ lib/logstorage/stats_fields_min.go | 241 ++++++++++ lib/logstorage/stats_fields_min_test.go | 285 ++++++++++++ lib/logstorage/stats_max.go | 34 +- lib/logstorage/stats_max_test.go | 366 +++++++++++++++ lib/logstorage/stats_median.go | 18 +- lib/logstorage/stats_median_test.go | 364 +++++++++++++++ lib/logstorage/stats_min.go | 40 +- lib/logstorage/stats_min_test.go | 366 +++++++++++++++ lib/logstorage/stats_quantile.go | 44 +- lib/logstorage/stats_quantile_test.go | 362 +++++++++++++++ lib/logstorage/stats_sum.go | 27 +- lib/logstorage/stats_sum_len.go | 23 +- lib/logstorage/stats_sum_len_test.go | 364 +++++++++++++++ lib/logstorage/stats_sum_test.go | 364 +++++++++++++++ lib/logstorage/stats_uniq_values.go | 90 ++-- lib/logstorage/stats_uniq_values_test.go | 417 ++++++++++++++++++ lib/logstorage/stats_values.go | 25 +- lib/logstorage/stats_values_test.go | 30 ++ lib/logstorage/storage_search.go | 280 ++++++++++-- lib/logstorage/storage_search_test.go | 20 + lib/stringsutil/less_natural_test.go | 10 + 90 files changed, 10672 insertions(+), 1588 deletions(-) create mode 100644 app/vlselect/logsql/stream_label_names_response.qtpl create mode 100644 app/vlselect/logsql/stream_label_names_response.qtpl.go create mode 100644 app/vlselect/logsql/stream_label_values_response.qtpl create mode 100644 app/vlselect/logsql/stream_label_values_response.qtpl.go create mode 100644 app/vlselect/logsql/streams_response.qtpl create mode 100644 app/vlselect/logsql/streams_response.qtpl.go create mode 100644 lib/logstorage/if_filter.go create mode 100644 lib/logstorage/logfmt_parser.go create mode 100644 lib/logstorage/logfmt_parser_test.go create mode 100644 lib/logstorage/pattern.go create mode 100644 lib/logstorage/pattern_test.go rename lib/logstorage/{pipe_extract_timing_test.go => pattern_timing_test.go} (81%) create mode 100644 lib/logstorage/pipe_format.go create mode 100644 lib/logstorage/pipe_format_test.go create mode 100644 lib/logstorage/pipe_unpack.go create mode 100644 lib/logstorage/stats_avg_test.go create mode 100644 lib/logstorage/stats_count_empty_test.go create mode 100644 lib/logstorage/stats_count_test.go create mode 100644 lib/logstorage/stats_count_uniq_test.go create mode 100644 lib/logstorage/stats_fields_max.go create mode 100644 lib/logstorage/stats_fields_max_test.go create mode 100644 lib/logstorage/stats_fields_min.go create mode 100644 lib/logstorage/stats_fields_min_test.go create mode 100644 lib/logstorage/stats_max_test.go create mode 100644 lib/logstorage/stats_median_test.go create mode 100644 lib/logstorage/stats_min_test.go create mode 100644 lib/logstorage/stats_sum_len_test.go create mode 100644 lib/logstorage/stats_sum_test.go create mode 100644 lib/logstorage/stats_uniq_values_test.go create mode 100644 lib/logstorage/stats_values_test.go diff --git a/app/vlinsert/elasticsearch/elasticsearch.go b/app/vlinsert/elasticsearch/elasticsearch.go index de0051109..3d103791c 100644 --- a/app/vlinsert/elasticsearch/elasticsearch.go +++ b/app/vlinsert/elasticsearch/elasticsearch.go @@ -210,7 +210,7 @@ func readBulkLine(sc *bufio.Scanner, timeField, msgField string, } line = sc.Bytes() p := logstorage.GetJSONParser() - if err := p.ParseLogMessage(line, ""); err != nil { + if err := p.ParseLogMessage(line); err != nil { return false, fmt.Errorf("cannot parse json-encoded log entry: %w", err) } diff --git a/app/vlinsert/jsonline/jsonline.go b/app/vlinsert/jsonline/jsonline.go index 9d1e7ebf8..33d832789 100644 --- a/app/vlinsert/jsonline/jsonline.go +++ b/app/vlinsert/jsonline/jsonline.go @@ -105,7 +105,7 @@ func readLine(sc *bufio.Scanner, timeField, msgField string, processLogMessage f } p := logstorage.GetJSONParser() - if err := p.ParseLogMessage(line, ""); err != nil { + if err := p.ParseLogMessage(line); err != nil { return false, fmt.Errorf("cannot parse json-encoded log entry: %w", err) } ts, err := extractTimestampFromFields(timeField, p.Fields) diff --git a/app/vlselect/logsql/logsql.go b/app/vlselect/logsql/logsql.go index bb82402cd..a0d669756 100644 --- a/app/vlselect/logsql/logsql.go +++ b/app/vlselect/logsql/logsql.go @@ -5,7 +5,6 @@ import ( "fmt" "math" "net/http" - "slices" "sort" "strings" "sync" @@ -145,8 +144,6 @@ func ProcessFieldNamesRequest(ctx context.Context, w http.ResponseWriter, r *htt return } - slices.Sort(fieldNames) - // Write results w.Header().Set("Content-Type", "application/json") WriteFieldNamesResponse(w, fieldNames) @@ -163,9 +160,9 @@ func ProcessFieldValuesRequest(ctx context.Context, w http.ResponseWriter, r *ht } // Parse fieldName query arg - fieldName := r.FormValue("field_name") + fieldName := r.FormValue("field") if fieldName == "" { - httpserver.Errorf(w, r, "missing 'field_name' query arg") + httpserver.Errorf(w, r, "missing 'field' query arg") return } @@ -187,16 +184,102 @@ func ProcessFieldValuesRequest(ctx context.Context, w http.ResponseWriter, r *ht return } - if limit == 0 || len(values) < limit { - // Sort values only if their number is below the limit. - // Otherwise there is little sense in sorting, since the query may return - // different subset of values on every execution. - slices.Sort(values) + // Write results + w.Header().Set("Content-Type", "application/json") + WriteFieldValuesResponse(w, values) +} + +// ProcessStreamLabelNamesRequest processes /select/logsql/stream_label_names request. +// +// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-names +func ProcessStreamLabelNamesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) { + q, tenantIDs, err := parseCommonArgs(r) + if err != nil { + httpserver.Errorf(w, r, "%s", err) + return + } + + // Obtain stream label names for the given query + q.Optimize() + names, err := vlstorage.GetStreamLabelNames(ctx, tenantIDs, q) + if err != nil { + httpserver.Errorf(w, r, "cannot obtain stream label names: %s", err) } // Write results w.Header().Set("Content-Type", "application/json") - WriteFieldValuesResponse(w, values) + WriteStreamLabelNamesResponse(w, names) +} + +// ProcessStreamLabelValuesRequest processes /select/logsql/stream_label_values request. +// +// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-values +func ProcessStreamLabelValuesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) { + q, tenantIDs, err := parseCommonArgs(r) + if err != nil { + httpserver.Errorf(w, r, "%s", err) + return + } + + // Parse labelName query arg + labelName := r.FormValue("label") + if labelName == "" { + httpserver.Errorf(w, r, "missing 'label' query arg") + return + } + + // Parse limit query arg + limit, err := httputils.GetInt(r, "limit") + if err != nil { + httpserver.Errorf(w, r, "%s", err) + return + } + if limit < 0 { + limit = 0 + } + + // Obtain stream label names for the given query + q.Optimize() + values, err := vlstorage.GetStreamLabelValues(ctx, tenantIDs, q, labelName, uint64(limit)) + if err != nil { + httpserver.Errorf(w, r, "cannot obtain stream label values: %s", err) + } + + // Write results + w.Header().Set("Content-Type", "application/json") + WriteStreamLabelValuesResponse(w, values) +} + +// ProcessStreamsRequest processes /select/logsql/streams request. +// +// See https://docs.victoriametrics.com/victorialogs/querying/#querying-streams +func ProcessStreamsRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) { + q, tenantIDs, err := parseCommonArgs(r) + if err != nil { + httpserver.Errorf(w, r, "%s", err) + return + } + + // Parse limit query arg + limit, err := httputils.GetInt(r, "limit") + if err != nil { + httpserver.Errorf(w, r, "%s", err) + return + } + if limit < 0 { + limit = 0 + } + + // Obtain streams for the given query + q.Optimize() + streams, err := vlstorage.GetStreams(ctx, tenantIDs, q, uint64(limit)) + if err != nil { + httpserver.Errorf(w, r, "cannot obtain streams: %s", err) + } + + // Write results + w.Header().Set("Content-Type", "application/json") + WriteStreamsResponse(w, streams) } // ProcessQueryRequest handles /select/logsql/query request. diff --git a/app/vlselect/logsql/stream_label_names_response.qtpl b/app/vlselect/logsql/stream_label_names_response.qtpl new file mode 100644 index 000000000..2e476a79a --- /dev/null +++ b/app/vlselect/logsql/stream_label_names_response.qtpl @@ -0,0 +1,17 @@ +{% stripspace %} + +// StreamLabelNamesResponse formats /select/logsql/stream_label_names response +{% func StreamLabelNamesResponse(names []string) %} +{ + "names":[ + {% if len(names) > 0 %} + {%q= names[0] %} + {% for _, v := range names[1:] %} + ,{%q= v %} + {% endfor %} + {% endif %} + ] +} +{% endfunc %} + +{% endstripspace %} diff --git a/app/vlselect/logsql/stream_label_names_response.qtpl.go b/app/vlselect/logsql/stream_label_names_response.qtpl.go new file mode 100644 index 000000000..fa7555656 --- /dev/null +++ b/app/vlselect/logsql/stream_label_names_response.qtpl.go @@ -0,0 +1,69 @@ +// Code generated by qtc from "stream_label_names_response.qtpl". DO NOT EDIT. +// See https://github.com/valyala/quicktemplate for details. + +// StreamLabelNamesResponse formats /select/logsql/stream_label_names response + +//line app/vlselect/logsql/stream_label_names_response.qtpl:4 +package logsql + +//line app/vlselect/logsql/stream_label_names_response.qtpl:4 +import ( + qtio422016 "io" + + qt422016 "github.com/valyala/quicktemplate" +) + +//line app/vlselect/logsql/stream_label_names_response.qtpl:4 +var ( + _ = qtio422016.Copy + _ = qt422016.AcquireByteBuffer +) + +//line app/vlselect/logsql/stream_label_names_response.qtpl:4 +func StreamStreamLabelNamesResponse(qw422016 *qt422016.Writer, names []string) { +//line app/vlselect/logsql/stream_label_names_response.qtpl:4 + qw422016.N().S(`{"names":[`) +//line app/vlselect/logsql/stream_label_names_response.qtpl:7 + if len(names) > 0 { +//line app/vlselect/logsql/stream_label_names_response.qtpl:8 + qw422016.N().Q(names[0]) +//line app/vlselect/logsql/stream_label_names_response.qtpl:9 + for _, v := range names[1:] { +//line app/vlselect/logsql/stream_label_names_response.qtpl:9 + qw422016.N().S(`,`) +//line app/vlselect/logsql/stream_label_names_response.qtpl:10 + qw422016.N().Q(v) +//line app/vlselect/logsql/stream_label_names_response.qtpl:11 + } +//line app/vlselect/logsql/stream_label_names_response.qtpl:12 + } +//line app/vlselect/logsql/stream_label_names_response.qtpl:12 + qw422016.N().S(`]}`) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 +} + +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 +func WriteStreamLabelNamesResponse(qq422016 qtio422016.Writer, names []string) { +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + qw422016 := qt422016.AcquireWriter(qq422016) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + StreamStreamLabelNamesResponse(qw422016, names) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + qt422016.ReleaseWriter(qw422016) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 +} + +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 +func StreamLabelNamesResponse(names []string) string { +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + qb422016 := qt422016.AcquireByteBuffer() +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + WriteStreamLabelNamesResponse(qb422016, names) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + qs422016 := string(qb422016.B) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + qt422016.ReleaseByteBuffer(qb422016) +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 + return qs422016 +//line app/vlselect/logsql/stream_label_names_response.qtpl:15 +} diff --git a/app/vlselect/logsql/stream_label_values_response.qtpl b/app/vlselect/logsql/stream_label_values_response.qtpl new file mode 100644 index 000000000..49c1695df --- /dev/null +++ b/app/vlselect/logsql/stream_label_values_response.qtpl @@ -0,0 +1,17 @@ +{% stripspace %} + +// StreamLabelValuesResponse formats /select/logsql/stream_label_values response +{% func StreamLabelValuesResponse(values []string) %} +{ + "values":[ + {% if len(values) > 0 %} + {%q= values[0] %} + {% for _, v := range values[1:] %} + ,{%q= v %} + {% endfor %} + {% endif %} + ] +} +{% endfunc %} + +{% endstripspace %} diff --git a/app/vlselect/logsql/stream_label_values_response.qtpl.go b/app/vlselect/logsql/stream_label_values_response.qtpl.go new file mode 100644 index 000000000..7e385c60f --- /dev/null +++ b/app/vlselect/logsql/stream_label_values_response.qtpl.go @@ -0,0 +1,69 @@ +// Code generated by qtc from "stream_label_values_response.qtpl". DO NOT EDIT. +// See https://github.com/valyala/quicktemplate for details. + +// StreamLabelValuesResponse formats /select/logsql/stream_label_values response + +//line app/vlselect/logsql/stream_label_values_response.qtpl:4 +package logsql + +//line app/vlselect/logsql/stream_label_values_response.qtpl:4 +import ( + qtio422016 "io" + + qt422016 "github.com/valyala/quicktemplate" +) + +//line app/vlselect/logsql/stream_label_values_response.qtpl:4 +var ( + _ = qtio422016.Copy + _ = qt422016.AcquireByteBuffer +) + +//line app/vlselect/logsql/stream_label_values_response.qtpl:4 +func StreamStreamLabelValuesResponse(qw422016 *qt422016.Writer, values []string) { +//line app/vlselect/logsql/stream_label_values_response.qtpl:4 + qw422016.N().S(`{"values":[`) +//line app/vlselect/logsql/stream_label_values_response.qtpl:7 + if len(values) > 0 { +//line app/vlselect/logsql/stream_label_values_response.qtpl:8 + qw422016.N().Q(values[0]) +//line app/vlselect/logsql/stream_label_values_response.qtpl:9 + for _, v := range values[1:] { +//line app/vlselect/logsql/stream_label_values_response.qtpl:9 + qw422016.N().S(`,`) +//line app/vlselect/logsql/stream_label_values_response.qtpl:10 + qw422016.N().Q(v) +//line app/vlselect/logsql/stream_label_values_response.qtpl:11 + } +//line app/vlselect/logsql/stream_label_values_response.qtpl:12 + } +//line app/vlselect/logsql/stream_label_values_response.qtpl:12 + qw422016.N().S(`]}`) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 +} + +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 +func WriteStreamLabelValuesResponse(qq422016 qtio422016.Writer, values []string) { +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + qw422016 := qt422016.AcquireWriter(qq422016) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + StreamStreamLabelValuesResponse(qw422016, values) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + qt422016.ReleaseWriter(qw422016) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 +} + +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 +func StreamLabelValuesResponse(values []string) string { +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + qb422016 := qt422016.AcquireByteBuffer() +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + WriteStreamLabelValuesResponse(qb422016, values) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + qs422016 := string(qb422016.B) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + qt422016.ReleaseByteBuffer(qb422016) +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 + return qs422016 +//line app/vlselect/logsql/stream_label_values_response.qtpl:15 +} diff --git a/app/vlselect/logsql/streams_response.qtpl b/app/vlselect/logsql/streams_response.qtpl new file mode 100644 index 000000000..3242aa798 --- /dev/null +++ b/app/vlselect/logsql/streams_response.qtpl @@ -0,0 +1,17 @@ +{% stripspace %} + +// StreamsResponse formats /select/logsql/streams response +{% func StreamsResponse(streams []string) %} +{ + "streams":[ + {% if len(streams) > 0 %} + {%q= streams[0] %} + {% for _, v := range streams[1:] %} + ,{%q= v %} + {% endfor %} + {% endif %} + ] +} +{% endfunc %} + +{% endstripspace %} diff --git a/app/vlselect/logsql/streams_response.qtpl.go b/app/vlselect/logsql/streams_response.qtpl.go new file mode 100644 index 000000000..e1a1b8feb --- /dev/null +++ b/app/vlselect/logsql/streams_response.qtpl.go @@ -0,0 +1,69 @@ +// Code generated by qtc from "streams_response.qtpl". DO NOT EDIT. +// See https://github.com/valyala/quicktemplate for details. + +// StreamsResponse formats /select/logsql/streams response + +//line app/vlselect/logsql/streams_response.qtpl:4 +package logsql + +//line app/vlselect/logsql/streams_response.qtpl:4 +import ( + qtio422016 "io" + + qt422016 "github.com/valyala/quicktemplate" +) + +//line app/vlselect/logsql/streams_response.qtpl:4 +var ( + _ = qtio422016.Copy + _ = qt422016.AcquireByteBuffer +) + +//line app/vlselect/logsql/streams_response.qtpl:4 +func StreamStreamsResponse(qw422016 *qt422016.Writer, streams []string) { +//line app/vlselect/logsql/streams_response.qtpl:4 + qw422016.N().S(`{"streams":[`) +//line app/vlselect/logsql/streams_response.qtpl:7 + if len(streams) > 0 { +//line app/vlselect/logsql/streams_response.qtpl:8 + qw422016.N().Q(streams[0]) +//line app/vlselect/logsql/streams_response.qtpl:9 + for _, v := range streams[1:] { +//line app/vlselect/logsql/streams_response.qtpl:9 + qw422016.N().S(`,`) +//line app/vlselect/logsql/streams_response.qtpl:10 + qw422016.N().Q(v) +//line app/vlselect/logsql/streams_response.qtpl:11 + } +//line app/vlselect/logsql/streams_response.qtpl:12 + } +//line app/vlselect/logsql/streams_response.qtpl:12 + qw422016.N().S(`]}`) +//line app/vlselect/logsql/streams_response.qtpl:15 +} + +//line app/vlselect/logsql/streams_response.qtpl:15 +func WriteStreamsResponse(qq422016 qtio422016.Writer, streams []string) { +//line app/vlselect/logsql/streams_response.qtpl:15 + qw422016 := qt422016.AcquireWriter(qq422016) +//line app/vlselect/logsql/streams_response.qtpl:15 + StreamStreamsResponse(qw422016, streams) +//line app/vlselect/logsql/streams_response.qtpl:15 + qt422016.ReleaseWriter(qw422016) +//line app/vlselect/logsql/streams_response.qtpl:15 +} + +//line app/vlselect/logsql/streams_response.qtpl:15 +func StreamsResponse(streams []string) string { +//line app/vlselect/logsql/streams_response.qtpl:15 + qb422016 := qt422016.AcquireByteBuffer() +//line app/vlselect/logsql/streams_response.qtpl:15 + WriteStreamsResponse(qb422016, streams) +//line app/vlselect/logsql/streams_response.qtpl:15 + qs422016 := string(qb422016.B) +//line app/vlselect/logsql/streams_response.qtpl:15 + qt422016.ReleaseByteBuffer(qb422016) +//line app/vlselect/logsql/streams_response.qtpl:15 + return qs422016 +//line app/vlselect/logsql/streams_response.qtpl:15 +} diff --git a/app/vlselect/main.go b/app/vlselect/main.go index 94b366f9e..a45cf26c3 100644 --- a/app/vlselect/main.go +++ b/app/vlselect/main.go @@ -75,10 +75,9 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool { // Skip requests, which do not start with /select/, since these aren't our requests. return false } - path = strings.TrimPrefix(path, "/select") path = strings.ReplaceAll(path, "//", "/") - if path == "/vmui" { + if path == "/select/vmui" { // VMUI access via incomplete url without `/` in the end. Redirect to complete url. // Use relative redirect, since the hostname and path prefix may be incorrect if VictoriaMetrics // is hidden behind vmauth or similar proxy. @@ -87,8 +86,8 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool { httpserver.Redirect(w, newURL) return true } - if strings.HasPrefix(path, "/vmui/") { - if strings.HasPrefix(path, "/vmui/static/") { + if strings.HasPrefix(path, "/select/vmui/") { + if strings.HasPrefix(path, "/select/vmui/static/") { // Allow clients caching static contents for long period of time, since it shouldn't change over time. // Path to static contents (such as js and css) must be changed whenever its contents is changed. // See https://developer.chrome.com/docs/lighthouse/performance/uses-long-cache-ttl/ @@ -140,27 +139,36 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool { } } + httpserver.EnableCORS(w, r) switch path { - case "/logsql/query": - logsqlQueryRequests.Inc() - httpserver.EnableCORS(w, r) - logsql.ProcessQueryRequest(ctx, w, r) - return true - case "/logsql/field_values": - logsqlFieldValuesRequests.Inc() - httpserver.EnableCORS(w, r) - logsql.ProcessFieldValuesRequest(ctx, w, r) - return true - case "/logsql/field_names": + case "/select/logsql/field_names": logsqlFieldNamesRequests.Inc() - httpserver.EnableCORS(w, r) logsql.ProcessFieldNamesRequest(ctx, w, r) return true - case "/logsql/hits": + case "/select/logsql/field_values": + logsqlFieldValuesRequests.Inc() + logsql.ProcessFieldValuesRequest(ctx, w, r) + return true + case "/select/logsql/hits": logsqlHitsRequests.Inc() - httpserver.EnableCORS(w, r) logsql.ProcessHitsRequest(ctx, w, r) return true + case "/select/logsql/query": + logsqlQueryRequests.Inc() + logsql.ProcessQueryRequest(ctx, w, r) + return true + case "/select/logsql/stream_label_names": + logsqlStreamLabelNamesRequests.Inc() + logsql.ProcessStreamLabelNamesRequest(ctx, w, r) + return true + case "/select/logsql/stream_label_values": + logsqlStreamLabelValuesRequests.Inc() + logsql.ProcessStreamLabelValuesRequest(ctx, w, r) + return true + case "/select/logsql/streams": + logsqlStreamsRequests.Inc() + logsql.ProcessStreamsRequest(ctx, w, r) + return true default: return false } @@ -180,8 +188,11 @@ func getMaxQueryDuration(r *http.Request) time.Duration { } var ( - logsqlQueryRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/query"}`) - logsqlFieldValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_values"}`) - logsqlFieldNamesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_names"}`) - logsqlHitsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/hits"}`) + logsqlFieldNamesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_names"}`) + logsqlFieldValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_values"}`) + logsqlHitsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/hits"}`) + logsqlQueryRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/query"}`) + logsqlStreamLabelNamesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_label_names"}`) + logsqlStreamLabelValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_label_values"}`) + logsqlStreamsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/streams"}`) ) diff --git a/app/vlstorage/main.go b/app/vlstorage/main.go index 1d46a3ed3..91a1aa2c3 100644 --- a/app/vlstorage/main.go +++ b/app/vlstorage/main.go @@ -123,6 +123,25 @@ func GetFieldValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *log return strg.GetFieldValues(ctx, tenantIDs, q, fieldName, limit) } +// GetStreamLabelNames executes q and returns stream labels names seen in results. +func GetStreamLabelNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]string, error) { + return strg.GetStreamLabelNames(ctx, tenantIDs, q) +} + +// GetStreamLabelValues executes q and returns stream label values for the given labelName seen in results. +// +// If limit > 0, then up to limit unique stream label values are returned. +func GetStreamLabelValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, labelName string, limit uint64) ([]string, error) { + return strg.GetStreamLabelValues(ctx, tenantIDs, q, labelName, limit) +} + +// GetStreams executes q and returns streams seen in query results. +// +// If limit > 0, then up to limit unique streams are returned. +func GetStreams(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, limit uint64) ([]string, error) { + return strg.GetStreams(ctx, tenantIDs, q, limit) +} + func writeStorageMetrics(w io.Writer, strg *logstorage.Storage) { var ss logstorage.StorageStats strg.UpdateStats(&ss) diff --git a/docs/VictoriaLogs/CHANGELOG.md b/docs/VictoriaLogs/CHANGELOG.md index e4e86afe0..9f0a8f49f 100644 --- a/docs/VictoriaLogs/CHANGELOG.md +++ b/docs/VictoriaLogs/CHANGELOG.md @@ -19,8 +19,21 @@ according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/QuickSta ## tip +* FEATURE: allow using `~"some_regexp"` [regexp filter](https://docs.victoriametrics.com/victorialogs/logsql/#regexp-filter) instead of `re("some_regexp")`. +* FEATURE: allow using `="some phrase"` [exact filter](https://docs.victoriametrics.com/victorialogs/logsql/#exact-filter) instead of `exact("some phrase")`. +* FEATURE: allow using `="some prefix"*` [exact prefix filter](https://docs.victoriametrics.com/victorialogs/logsql/#exact-prefix-filter) instead of `exact("some prefix"*)`. +* FEATURE: add ability to generate output fields according to the provided format string. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#format-pipe). +* FEATURE: add ability to extract fields with [`extract` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#extract-pipe) only if the given condition is met. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#conditional-extract). +* FEATURE: add ability to unpack JSON fields with [`unpack_json` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#unpack_json-pipe) only if the given condition is met. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#conditional-unpack_json). +* FEATURE: add ability to unpack [logfmt](https://brandur.org/logfmt) fields with [`unpack_logfmt` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#unpack_logfmt-pipe) only if the given condition is met. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#conditional-unpack_logfmt). +* FEATURE: add [`fields_min`](https://docs.victoriametrics.com/victorialogs/logsql/#fields_min-stats) and [`fields_max`](https://docs.victoriametrics.com/victorialogs/logsql/#fields_max-stats) functions for [`stats` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#stats-pipe), which allow returning all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) for the log entry with the minimum / maximum value at the given field. +* FEATURE: add `/select/logsql/streams` HTTP endpoint for returning [streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) from results of the given query. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#querying-streams) for details. +* FEATURE: add `/select/logsql/stream_label_names` HTTP endpoint for returning [stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label names from results of the given query. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-names) for details. +* FEATURE: add `/select/logsql/stream_label_values` HTTP endpoint for returning [stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label values for the given label from results of the given query. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-values) for details. * FEATURE: [web UI](https://docs.victoriametrics.com/VictoriaLogs/querying/#web-ui): change time range limitation from `_time` in the expression to `start` and `end` query args. +* BUGFIX: fix `invalid memory address or nil pointer dereference` panic when using [`extract`](https://docs.victoriametrics.com/victorialogs/logsql/#extract-pipe), [`unpack_json`](https://docs.victoriametrics.com/victorialogs/logsql/#unpack_json-pipe) or [`unpack_logfmt`](https://docs.victoriametrics.com/victorialogs/logsql/#unpack_logfmt-pipe) pipes. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6306). + ## [v0.8.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.8.0-victorialogs) Released at 2024-05-20 diff --git a/docs/VictoriaLogs/LogsQL.md b/docs/VictoriaLogs/LogsQL.md index d9d6ad219..81ca6807a 100644 --- a/docs/VictoriaLogs/LogsQL.md +++ b/docs/VictoriaLogs/LogsQL.md @@ -306,10 +306,10 @@ with `app` field equal to `nginx`: _stream:{app="nginx"} ``` -This query is equivalent to the following [exact()](#exact-filter) query, but the upper query usually works much faster: +This query is equivalent to the following [`exact` filter](#exact-filter) query, but the upper query usually works much faster: ```logsql -app:exact("nginx") +app:="nginx" ``` Performance tips: @@ -449,7 +449,7 @@ This query matches the following [log messages](https://docs.victoriametrics.com This query doesn't match the following log messages: - `Error: foobar`, since the `Error` [word](#word) starts with capital letter. Use `i(err*)` for this case. See [these docs](#case-insensitive-filter) for details. -- `fooerror`, since the `fooerror` [word](#word) doesn't start with `err`. Use `re("err")` for this case. See [these docs](#regexp-filter) for details. +- `fooerror`, since the `fooerror` [word](#word) doesn't start with `err`. Use `~"err"` for this case. See [these docs](#regexp-filter) for details. Prefix filter can be applied to [phrases](#phrase-filter). For example, the following query matches [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field) containing phrases with `unexpected fail` prefix: @@ -537,40 +537,40 @@ See also: The [word filter](#word-filter) and [phrase filter](#phrase-filter) return [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field), which contain the given word or phrase inside them. The message may contain additional text other than the requested word or phrase. If you need searching for log messages -or [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field) with the exact value, then use the `exact(...)` filter. +or [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field) with the exact value, then use the `exact` filter. For example, the following query returns log messages wih the exact value `fatal error: cannot find /foo/bar`: ```logsql -exact("fatal error: cannot find /foo/bar") +="fatal error: cannot find /foo/bar" ``` The query doesn't match the following log messages: - `fatal error: cannot find /foo/bar/baz` or `some-text fatal error: cannot find /foo/bar`, since they contain an additional text - other than the specified in the `exact()` filter. Use `"fatal error: cannot find /foo/bar"` query in this case. See [these docs](#phrase-filter) for details. + other than the specified in the `exact` filter. Use `"fatal error: cannot find /foo/bar"` query in this case. See [these docs](#phrase-filter) for details. -- `FATAL ERROR: cannot find /foo/bar`, since the `exact()` filter is case-sensitive. Use `i("fatal error: cannot find /foo/bar")` in this case. +- `FATAL ERROR: cannot find /foo/bar`, since the `exact` filter is case-sensitive. Use `i("fatal error: cannot find /foo/bar")` in this case. See [these docs](#case-insensitive-filter) for details. -By default the `exact()` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). -Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the `exact()` filter and put a colon after it +By default the `exact` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). +Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the `exact` filter and put a colon after it if it must be searched in the given field. For example, the following query returns log entries with the exact `error` value at `log.level` field: ```logsql -log.level:exact("error") +log.level:="error" ``` Both the field name and the phrase can contain arbitrary [utf-8](https://en.wikipedia.org/wiki/UTF-8)-encoded chars. For example: ```logsql -log.уровень:exact("ошибка") +log.уровень:="ошибка" ``` The field name can be put inside quotes if it contains special chars, which may clash with the query syntax. For example, the following query matches the `error` value in the field `log:level`: ```logsql -"log:level":exact("error") +"log:level":="error" ``` See also: @@ -585,11 +585,11 @@ See also: ### Exact prefix filter -Sometimes it is needed to find log messages starting with some prefix. This can be done with the `exact("prefix"*)` filter. +Sometimes it is needed to find log messages starting with some prefix. This can be done with the `="prefix"*` filter. For example, the following query matches log messages, which start from `Processing request` prefix: ```logsql -exact("Processing request"*) +="Processing request"* ``` This filter matches the following [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field): @@ -599,30 +599,30 @@ This filter matches the following [log messages](https://docs.victoriametrics.co It doesn't match the following log messages: -- `processing request foobar`, since the log message starts with lowercase `p`. Use `exact("processing request"*) OR exact("Processing request"*)` +- `processing request foobar`, since the log message starts with lowercase `p`. Use `="processing request"* OR ="Processing request"*` query in this case. See [these docs](#logical-filter) for details. - `start: Processing request`, since the log message doesn't start with `Processing request`. Use `"Processing request"` query in this case. See [these docs](#phrase-filter) for details. -By default the `exact()` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). -Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the `exact()` filter and put a colon after it +By default the `exact` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). +Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the `exact` filter and put a colon after it if it must be searched in the given field. For example, the following query returns log entries with `log.level` field, which starts with `err` prefix: ```logsql -log.level:exact("err"*) +log.level:="err"* ``` Both the field name and the phrase can contain arbitrary [utf-8](https://en.wikipedia.org/wiki/UTF-8)-encoded chars. For example: ```logsql -log.уровень:exact("ошиб"*) +log.уровень:="ошиб"* ``` The field name can be put inside quotes if it contains special chars, which may clash with the query syntax. For example, the following query matches `log:level` values starting with `err` prefix: ```logsql -"log:level":exact("err"*) +"log:level":="err"* ``` See also: @@ -641,7 +641,7 @@ combined into a single [logical filter](#logical-filter). For example, the follo containing either `error` or `fatal` exact values: ```logsql -log.level:(exact("error") OR exact("fatal")) +log.level:(="error" OR ="fatal") ``` While this solution works OK, LogsQL provides simpler and faster solution for this case - the `in()` filter. @@ -690,7 +690,7 @@ The query matches the following [log messages](https://docs.victoriametrics.com/ The query doesn't match the following log messages: -- `FooError`, since the `FooError` [word](#word) has superflouos prefix `Foo`. Use `re("(?i)error")` for this case. See [these docs](#regexp-filter) for details. +- `FooError`, since the `FooError` [word](#word) has superflouos prefix `Foo`. Use `~"(?i)error"` for this case. See [these docs](#regexp-filter) for details. - `too many Errors`, since the `Errors` [word](#word) has superflouos suffix `s`. Use `i(error*)` for this case. By default the `i()` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). @@ -764,11 +764,11 @@ See also: ### Regexp filter -LogsQL supports regular expression filter with [re2 syntax](https://github.com/google/re2/wiki/Syntax) via `re(...)` expression. +LogsQL supports regular expression filter with [re2 syntax](https://github.com/google/re2/wiki/Syntax) via `~"regex"` syntax. For example, the following query returns all the log messages containing `err` or `warn` susbstrings: ```logsql -re("err|warn") +~"err|warn" ``` The query matches the following [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field), which contain either `err` or `warn` substrings: @@ -779,33 +779,33 @@ The query matches the following [log messages](https://docs.victoriametrics.com/ The query doesn't match the following log messages: -- `ERROR: cannot open file`, since the `ERROR` word is in uppercase letters. Use `re("(?i)(err|warn)")` query for case-insensitive regexp search. +- `ERROR: cannot open file`, since the `ERROR` word is in uppercase letters. Use `~"(?i)(err|warn)"` query for case-insensitive regexp search. See [these docs](https://github.com/google/re2/wiki/Syntax) for details. See also [case-insenstive filter docs](#case-insensitive-filter). - `it is warmer than usual`, since it doesn't contain neither `err` nor `warn` substrings. -By default the `re()` filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). +By default the regexp filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field). Specify the needed [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the filter in order to apply it to the given field. For example, the following query matches `event.original` field containing either `err` or `warn` substrings: ```logsql -event.original:re("err|warn") +event.original:~"err|warn" ``` If the field name contains special chars, which may clash with the query syntax, then it may be put into quotes in the query. For example, the following query matches `event:original` field containing either `err` or `warn` substrings: ```logsql -"event:original":re("err|warn") +"event:original":~"err|warn" ``` Performance tips: - Prefer combining simple [word filter](#word-filter) with [logical filter](#logical-filter) instead of using regexp filter. - For example, the `re("error|warning")` query can be substituted with `error OR warning` query, which usually works much faster. - Note that the `re("error|warning")` matches `errors` as well as `warnings` [words](#word), while `error OR warning` matches + For example, the `~"error|warning"` query can be substituted with `error OR warning` query, which usually works much faster. + Note that the `~"error|warning"` matches `errors` as well as `warnings` [words](#word), while `error OR warning` matches only the specified [words](#word). See also [multi-exact filter](#multi-exact-filter). - Prefer moving the regexp filter to the end of the [logical filter](#logical-filter), so lightweighter filters are executed first. -- Prefer using `exact("some prefix"*)` instead of `re("^some prefix")`, since the [exact()](#exact-prefix-filter) works much faster than the `re()` filter. +- Prefer using `="some prefix"*` instead of `~"^some prefix"`, since the [`exact` filter](#exact-prefix-filter) works much faster than the regexp filter. - See [other performance tips](#performance-tips). See also: @@ -1031,7 +1031,7 @@ Performance tips: while moving less specific and the slowest filters (such as [regexp filter](#regexp-filter) and [case-insensitive filter](#case-insensitive-filter)) to the right. For example, if you need to find [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field) with the `error` word, which match some `/foo/(bar|baz)` regexp, - it is better from performance PoV to use the query `error re("/foo/(bar|baz)")` instead of `re("/foo/(bar|baz)") error`. + it is better from performance PoV to use the query `error ~"/foo/(bar|baz)"` instead of `~"/foo/(bar|baz)" error`. The most specific filter means that it matches the lowest number of log entries comparing to other filters. @@ -1056,6 +1056,7 @@ LogsQL supports the following pipes: - [`field_names`](#field_names-pipe) returns all the names of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). - [`fields`](#fields-pipe) selects the given set of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). - [`filter`](#filter-pipe) applies additional [filters](#filters) to results. +- [`format`](#format-pipe) formats ouptut field from input [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). - [`limit`](#limit-pipe) limits the number selected logs. - [`offset`](#offset-pipe) skips the given number of selected logs. - [`rename`](#rename-pipe) renames [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). @@ -1110,21 +1111,21 @@ See also: ### extract pipe -`| extract from field_name "pattern"` [pipe](#pipes) allows extracting additional fields specified in the `pattern` from the given -`field_name` [log field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). Existing log fields remain unchanged -after the `| extract ...` pipe. +`| extract "pattern" from field_name` [pipe](#pipes) allows extracting abitrary text into output fields according to the [`pattern`](#format-for-extract-pipe-pattern) from the given +[`field_name`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). Existing log fields remain unchanged after the `| extract ...` pipe. -`| extract ...` pipe can be useful for extracting additional fields needed for further data processing with other pipes such as [`stats` pipe](#stats-pipe) or [`sort` pipe](#sort-pipe). +`| extract ...` can be useful for extracting additional fields needed for further data processing with other pipes such as [`stats` pipe](#stats-pipe) or [`sort` pipe](#sort-pipe). For example, the following query selects logs with the `error` [word](#word) for the last day, extracts ip address from [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) into `ip` field and then calculates top 10 ip addresses with the biggest number of logs: ```logsql -_time:1d error | extract from _msg "ip= " | stats by (ip) count() logs | sort by (logs) desc limit 10 +_time:1d error | extract "ip= " from _msg | stats by (ip) count() logs | sort by (logs) desc limit 10 ``` -It is expected that `_msg` field contains `ip=...` substring, which ends with space. For example, `error ip=1.2.3.4 from user_id=42`. +It is expected that `_msg` field contains `ip=...` substring ending with space. For example, `error ip=1.2.3.4 from user_id=42`. +If there is no such substring in the current `_msg` field, then the `ip` output field will be empty. If the `| extract ...` pipe is applied to [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field), then the `from _msg` part can be omitted. For example, the following query is equivalent to the previous one: @@ -1133,19 +1134,40 @@ For example, the following query is equivalent to the previous one: _time:1d error | extract "ip= " | stats by (ip) count() logs | sort by (logs) desc limit 10 ``` +If the `pattern` contains double quotes, then it can be quoted into single quotes. For example, the following query extracts `ip` from the corresponding JSON field: + +```logsql +_time:5m | extract '"ip":""' +``` + See also: -- [format for extract pipe pattern](#format-for-extract-pipe-pattern) +- [Format for extract pipe pattern](#format-for-extract-pipe-pattern) +- [Conditional extract](#conditional-extract) - [`unpack_json` pipe](#unpack_json-pipe) - [`unpack_logfmt` pipe](#unpack_logfmt-pipe) #### Format for extract pipe pattern -The `pattern` part from [`| extract from src_field "pattern"` pipe](#extract-pipes) may contain arbitrary text, which matches as is to the `src_field` value. -Additionally to arbitrary text, the `pattern` may contain placeholders in the form `<...>`, which match any strings, including empty strings. -Placeholders may be named, such as ``, or anonymous, such as `<_>`. Named placeholders extract the matching text into -the corresponding [log field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model). -Anonymous placeholders are useful for skipping arbitrary text during pattern matching. +The `pattern` part from [`extract ` pipe](#extract-pipe) has the following format: + +``` +text1text2...textNtextN+1 +``` + +Where `text1`, ... `textN+1` is arbitrary non-empty text, which matches as is to the input text. + +The `field1`, ... `fieldN` are placeholders, which match a substring of any length (including zero length) in the input text until the next `textX`. +Placeholders can be anonymous and named. Anonymous placeholders are written as `<_>`. They are used for convenience when some input text +must be skipped until the next `textX`. Named palceholders are written as ``, where `some_name` is the name of the log field to store +the corresponding matching substring to. + +The matching starts from the first occurence of the `text1` in the input text. If the `pattern` starts with `` and doesn't contain `text1`, +then the matching starts from the beginning of the input text. Matching is performed sequentially according to the `pattern`. If some `textX` isn't found +in the remaining input text, then the remaining named placeholders receive empty string values and the matching finishes prematurely. + +Matching finishes successfully when `textN+1` is found in the input text. +If the `pattern` ends with `` and doesn't contain `textN+1`, then the `` matches the remaining input text. For example, if [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) contains the following text: @@ -1153,34 +1175,44 @@ For example, if [`_msg` field](https://docs.victoriametrics.com/victorialogs/key 1.2.3.4 GET /foo/bar?baz 404 "Mozilla foo bar baz" some tail here ``` -Then the following `| extract ...` [pipe](#pipes) can be used for extracting `ip`, `path` and `user_agent` fields from it: +Then the following `pattern` can be used for extracting `ip`, `path` and `user_agent` fields from it: ``` -| extract ' <_> <_> ""' + <_> <_> "" ``` Note that the user-agent part of the log message is in double quotes. This means that it may contain special chars, including escaped double quote, e.g. `\"`. This may break proper matching of the string in double quotes. -VictoriaLogs automatically detects the whole string in quotes and automatically decodes it if the first char in the placeholder is double quote or backtick. -So it is better to use the following `pattern` for proper matching of quoted strings: +VictoriaLogs automatically detects quoted strings and automatically unquotes them if the first matching char in the placeholder is double quote or backtick. +So it is better to use the following `pattern` for proper matching of quoted `user_agent` string: ``` -| extract " <_> <_> " + <_> <_> ``` -Note that the `user_agent` now matches double quotes, but VictoriaLogs automatically unquotes the matching string before storing it in the `user_agent` field. -This is useful for extracting JSON strings. For example, the following `pattern` properly extracts the `message` JSON string into `msg` field: +This is useful for extracting JSON strings. For example, the following `pattern` properly extracts the `message` JSON string into `msg` field, even if it contains special chars: ``` -| extract '"message":' +"message": ``` If some special chars such as `<` must be matched by the `pattern`, then they can be [html-escaped](https://en.wikipedia.org/wiki/List_of_XML_and_HTML_character_entity_references). -For example, the following `pattern` properly matches `a < 123.456` text: +For example, the following `pattern` properly matches `a < b` text by extracting `a` into `left` field and `b` into `right` field: ``` -| extract " < " + < +``` + +#### Conditional extract + +If some log entries must be skipped from [`extract` pipe](#extract-pipe), then add `if ()` filter after the `extract` word. +The `` can contain arbitrary [filters](#filters). For example, the following query extracts `ip` field +from [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) only +if the input [log entry](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) doesn't contain `ip` field or this field is empty: + +```logsql +_time:5m | extract if (ip:"") "ip= " ``` ### field_names pipe @@ -1209,6 +1241,12 @@ and [`_msg`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message- _time:5m | fields host, _msg ``` +`keep` can be used instead of `fields` for convenience. For example, the following query is equivalent to the previous one: + +```logsql +_time:5m | keep host, _msg +``` + See also: - [`copy` pipe](#copy-pipe) @@ -1232,6 +1270,50 @@ See also: - [`stats` pipe](#stats-pipe) - [`sort` pipe](#sort-pipe) +### format pipe + +`| format "pattern" as result_field` [pipe](#format-pipe) combines [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +according to the `pattern` and stores it to the `result_field`. All the other fields remain unchanged after the `| format ...` pipe. + +For example, the following query stores `request from :` text into [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field), +by substituting `` and `` with the corresponding [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) names: + +```logsql +_time:5m | format "request from :" as _msg +``` + +If the result of the `format` pattern is stored into [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field), +then `as _msg` part can be omitted. The following query is equivalent to the previous one: + +```logsql +_time:5m | format "request from :" +``` + +If some field values must be put into double quotes before formatting, then add `q:` in front of the corresponding field name. +For example, the following command generates properly encoded JSON object from `_msg` and `stacktrace` [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +and stores it into `my_json` output field: + +```logsql +_time:5m | format '{"_msg":,"stacktrace":}' as my_json +``` + +See also: + +- [Conditional format](#conditional-format) +- [`extract` pipe](#extract-pipe) + + +#### Conditional format + +If the [`format` pipe](#format-pipe) musn't be applied to every [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model), +then add `if ()` just after the `format` word. +The `` can contain arbitrary [filters](#filters). For example, the following query stores the formatted result to `message` field +only if `ip` and `host` [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) aren't empty: + +```logsql +_time:5m | format if (ip:* and host:*) "request from :" as message +``` + ### limit pipe If only a subset of selected logs must be processed, then `| limit N` [pipe](#pipes) can be used, where `N` can contain any [supported integer numeric value](#numeric-values). @@ -1548,8 +1630,8 @@ See also: ### unpack_json pipe -`| unpack_json from field_name` pipe unpacks `{"k1":"v1", ..., "kN":"vN"}` JSON from the given `field_name` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) -into `k1`, ... `kN` field names with the corresponding `v1`, ..., `vN` values. It overrides existing fields with names from the `k1`, ..., `kN` list. Other fields remain untouched. +`| unpack_json from field_name` pipe unpacks `{"k1":"v1", ..., "kN":"vN"}` JSON from the given input [`field_name`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +into `k1`, ... `kN` output field names with the corresponding `v1`, ..., `vN` values. It overrides existing fields with names from the `k1`, ..., `kN` list. Other fields remain untouched. Nested JSON is unpacked according to the rules defined [here](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). @@ -1566,30 +1648,48 @@ The following query is equivalent to the previous one: _time:5m | unpack_json ``` -If you want to make sure that the unpacked JSON fields do not clash with the existing fields, then specify common prefix for all the fields extracted from JSON, -by adding `result_prefix "prefix_name"` to `unpack_json`. For example, the following query adds `foo_` prefix for all the unpacked fields -form [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field): +If only some fields must be extracted from JSON, then they can be enumerated inside `fields (...)`. For example, the following query unpacks only `foo` and `bar` +fields from JSON value stored in `my_json` [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model): ```logsql -_time:5m | unpack_json result_prefix "foo_" +_time:5m | unpack_json from my_json fields (foo, bar) ``` Performance tip: if you need extracting a single field from long JSON, it is faster to use [`extract` pipe](#extract-pipe). For example, the following query extracts `"ip"` field from JSON -stored in [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field): +stored in [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) at the maximum speed: ``` _time:5m | extract '"ip":' ``` +If you want to make sure that the unpacked JSON fields do not clash with the existing fields, then specify common prefix for all the fields extracted from JSON, +by adding `result_prefix "prefix_name"` to `unpack_json`. For example, the following query adds `foo_` prefix for all the unpacked fields +form `foo`: + +```logsql +_time:5m | unpack_json from foo result_prefix "foo_" +``` + See also: +- [Conditional `unpack_json`](#conditional-unpack_json) - [`unpack_logfmt` pipe](#unpack_logfmt-pipe) - [`extract` pipe](#extract-pipe) +#### Conditional unpack_json + +If the [`unpack_json` pipe](#unpack_json-pipe) musn't be applied to every [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model), +then add `if ()` after `unpack_json`. +The `` can contain arbitrary [filters](#filters). For example, the following query unpacks JSON fields from `foo` field only if `ip` field in the current log entry isn't set or empty: + +```logsql +_time:5m | unpack_json if (ip:"") from foo +``` + ### unpack_logfmt pipe `| unpack_logfmt from field_name` pipe unpacks `k1=v1 ... kN=vN` [logfmt](https://brandur.org/logfmt) fields -from the given `field_name` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into `k1`, ... `kN` field names +from the given [`field_name`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into `k1`, ... `kN` field names with the corresponding `v1`, ..., `vN` values. It overrides existing fields with names from the `k1`, ..., `kN` list. Other fields remain untouched. For example, the following query unpacks [logfmt](https://brandur.org/logfmt) fields from the [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) @@ -1606,12 +1706,11 @@ The following query is equivalent to the previous one: _time:5m | unpack_logfmt ``` -If you want to make sure that the unpacked [logfmt](https://brandur.org/logfmt) fields do not clash with the existing fields, then specify common prefix for all the fields extracted from JSON, -by adding `result_prefix "prefix_name"` to `unpack_logfmt`. For example, the following query adds `foo_` prefix for all the unpacked fields -from [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field): +If only some fields must be unpacked from logfmt, then they can be enumerated inside `fields (...)`. For example, the following query extracts only `foo` and `bar` fields +from logfmt stored in the `my_logfmt` field: ```logsql -_time:5m | unpack_logfmt result_prefix "foo_" +_time:5m | unpack_logfmt from my_logfmt fields (foo, bar) ``` Performance tip: if you need extracting a single field from long [logfmt](https://brandur.org/logfmt) line, it is faster to use [`extract` pipe](#extract-pipe). @@ -1622,31 +1721,53 @@ in [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#mes _time:5m | extract ' ip=' ``` +If you want to make sure that the unpacked [logfmt](https://brandur.org/logfmt) fields do not clash with the existing fields, then specify common prefix for all the fields extracted from JSON, +by adding `result_prefix "prefix_name"` to `unpack_logfmt`. For example, the following query adds `foo_` prefix for all the unpacked fields +from `foo` field: + +```logsql +_time:5m | unpack_logfmt from foo result_prefix "foo_" +``` + See also: +- [Conditional unpack_logfmt](#conditional-unpack_logfmt) - [`unpack_json` pipe](#unpack_json-pipe) - [`extract` pipe](#extract-pipe) +#### Conditional unpack_logfmt + +If the [`unpack_logfmt` pipe](#unpack_logfmt-pipe) musn't be applied to every [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model), +then add `if ()` after `unpack_logfmt`. +The `` can contain arbitrary [filters](#filters). For example, the following query unpacks logfmt fields from `foo` field +only if `ip` field in the current log entry isn't set or empty: + +```logsql +_time:5m | unpack_logfmt if (ip:"") from foo +``` + ## stats pipe functions LogsQL supports the following functions for [`stats` pipe](#stats-pipe): -- [`avg`](#avg-stats) calculates the average value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`count`](#count-stats) calculates the number of log entries. -- [`count_empty`](#count_empty-stats) calculates the number logs with empty [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`count_uniq`](#count_uniq-stats) calculates the number of unique non-empty values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`max`](#max-stats) calcualtes the maximum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`median`](#median-stats) calcualtes the [median](https://en.wikipedia.org/wiki/Median) value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`min`](#min-stats) calculates the minumum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`quantile`](#quantile-stats) calculates the given quantile for the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`sum`](#sum-stats) calculates the sum for the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -- [`sum_len`](#sum_len-stats) calculates the sum of lengths for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`avg`](#avg-stats) returns the average value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`count`](#count-stats) returns the number of log entries. +- [`count_empty`](#count_empty-stats) returns the number logs with empty [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`count_uniq`](#count_uniq-stats) returns the number of unique non-empty values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`fields_max`](#fields_max-stats) returns the [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with the minimum value at the given field. +- [`fields_min`](#fields_min-stats) returns the [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with the maximum value at the given field. +- [`max`](#max-stats) returns the maximum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`median`](#median-stats) returns the [median](https://en.wikipedia.org/wiki/Median) value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`min`](#min-stats) returns the minumum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`quantile`](#quantile-stats) returns the given quantile for the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`sum`](#sum-stats) returns the sum for the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). +- [`sum_len`](#sum_len-stats) returns the sum of lengths for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). - [`uniq_values`](#uniq_values-stats) returns unique non-empty values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). - [`values`](#values-stats) returns all the values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). ### avg stats -`avg(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the average value across +`avg(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the average value across all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). Non-numeric values are ignored. @@ -1668,7 +1789,7 @@ See also: ### count stats -`count()` calculates the number of selected logs. +`count()` [stats pipe function](#stats-pipe-functions) calculates the number of selected logs. For example, the following query returns the number of logs over the last 5 minutes: @@ -1700,7 +1821,7 @@ See also: ### count_empty stats -`count_empty(field1, ..., fieldN)` calculates the number of logs with empty `(field1, ..., fieldN)` tuples. +`count_empty(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the number of logs with empty `(field1, ..., fieldN)` tuples. For example, the following query calculates the number of logs with empty `username` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) during the last 5 minutes: @@ -1716,7 +1837,7 @@ See also: ### count_uniq stats -`count_uniq(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the number of unique non-empty `(field1, ..., fieldN)` tuples. +`count_uniq(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the number of unique non-empty `(field1, ..., fieldN)` tuples. For example, the following query returns the number of unique non-empty values for `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) over the last 5 minutes: @@ -1746,9 +1867,62 @@ See also: - [`uniq_values`](#uniq_values-stats) - [`count`](#count-stats) +### fields_max stats + +`fields_max(field)` [stats pipe function](#stats-pipe-functions) returns [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +with the maximum value for the given `field`. Log entry is returned as JSON-encoded dictionary with all the fields from the original log. + +For example, the following query returns log entry with the maximum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +across logs for the last 5 minutes: + +```logsql +_time:5m | stats fields_max(duration) as log_with_max_duration +``` + +Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract) pipes. + +If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`. +For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the maximum `duration` over the last 5 minutes: + +```logsql +_time:5m | stats fields_max(duration, _time, path, duration) as time_and_ip_with_max_duration +``` + +See also: + +- [`max`](#max-stats) +- [`fields_min`](#fields_min-stats) + + +### fields_min stats + +`fields_min(field)` [stats pipe function](#stats-pipe-functions) returns [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +with the minimum value for the given `field`. Log entry is returned as JSON-encoded dictionary with all the fields from the original log. + +For example, the following query returns log entry with the minimum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) +across logs for the last 5 minutes: + +```logsql +_time:5m | stats fields_min(duration) as log_with_min_duration +``` + +Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract) pipes. + +If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`. +For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the minimum `duration` over the last 5 minutes: + +```logsql +_time:5m | stats fields_min(duration, _time, path, duration) as time_and_ip_with_min_duration +``` + +See also: + +- [`min`](#min-stats) +- [`fields_max`](#fields_max-stats) + ### max stats -`max(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns the maximum value across +`max(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) returns the maximum value across all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). For example, the following query returns the maximum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) @@ -1758,17 +1932,18 @@ over logs for the last 5 minutes: _time:5m | stats max(duration) max_duration ``` +[`fields_max`](#fields_max-stats) function can be used for obtaining other fields with the maximum duration. + See also: +- [`fields_max`](#fields_max-stats) - [`min`](#min-stats) - [`quantile`](#quantile-stats) - [`avg`](#avg-stats) -- [`sum`](#sum-stats) -- [`count`](#count-stats) ### median stats -`median(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the [median](https://en.wikipedia.org/wiki/Median) value across +`median(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the [median](https://en.wikipedia.org/wiki/Median) value across the give numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). For example, the following query return median for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) @@ -1785,7 +1960,7 @@ See also: ### min stats -`min(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns the minimum value across +`min(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) returns the minimum value across all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). For example, the following query returns the minimum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) @@ -1795,17 +1970,18 @@ over logs for the last 5 minutes: _time:5m | stats min(duration) min_duration ``` +[`fields_min`](#fields_min-stats) function can be used for obtaining other fields with the minimum duration. + See also: +- [`fields_min`](#fields_min-stats) - [`max`](#max-stats) - [`quantile`](#quantile-stats) - [`avg`](#avg-stats) -- [`sum`](#sum-stats) -- [`count`](#count-stats) ### quantile stats -`quantile(phi, field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates `phi` [percentile](https://en.wikipedia.org/wiki/Percentile) over numeric values +`quantile(phi, field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates `phi` [percentile](https://en.wikipedia.org/wiki/Percentile) over numeric values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). The `phi` must be in the range `0 ... 1`, where `0` means `0th` percentile, while `1` means `100th` percentile. @@ -1828,7 +2004,7 @@ See also: ### sum stats -`sum(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the sum of numeric values across +`sum(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the sum of numeric values across all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). For example, the following query returns the sum of numeric values for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) @@ -1847,7 +2023,7 @@ See also: ### sum_len stats -`sum_len(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the sum of lengths of all the values +`sum_len(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) calculates the sum of lengths of all the values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). For example, the following query returns the sum of lengths of [`_msg` fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) @@ -1863,9 +2039,9 @@ See also: ### uniq_values stats -`uniq_values(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns the unique non-empty values across +`uniq_values(field1, ..., fieldN)` [stats pipe function](#stats-pipe-functions) returns the unique non-empty values across the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). -The returned values are encoded in JSON array. The order of the returned values is arbitrary. +The returned values are encoded in sorted JSON array. For example, the following query returns unique non-empty values for the `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) over logs for the last 5 minutes: @@ -1878,12 +2054,14 @@ Every unique value is stored in memory during query execution. Big number of uni only a subset of unique values. In this case add `limit N` after `uniq_values(...)` in order to limit the number of returned unique values to `N`, while limiting the maximum memory usage. For example, the following query returns up to `100` unique values for the `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) -over the logs for the last 5 minutes. Note that arbitrary subset of unique `ip` values is returned every time: +over the logs for the last 5 minutes: ```logsql _time:5m | stats uniq_values(ip) limit 100 as unique_ips_100 ``` +Arbitrary subset of unique `ip` values is returned every time if the `limit` is reached. + See also: - [`uniq` pipe](#uniq-pipe) @@ -1893,7 +2071,7 @@ See also: ### values stats -`values(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns all the values (including empty values) +`values(field1, ..., fieldN)` [stats pipe fuction](#stats-pipe-functions) returns all the values (including empty values) for the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). The returned values are encoded in JSON array. diff --git a/docs/VictoriaLogs/querying/README.md b/docs/VictoriaLogs/querying/README.md index 42f45b487..44d52fc64 100644 --- a/docs/VictoriaLogs/querying/README.md +++ b/docs/VictoriaLogs/querying/README.md @@ -23,7 +23,19 @@ via the following ways: ## HTTP API -VictoriaLogs can be queried at the `/select/logsql/query` HTTP endpoint. +VictoriaLogs provides the following HTTP endpoints: + +- [`/select/logsql/query`](#querying-logs) for querying logs +- [`/select/logsql/hits`](#querying-hits-stats) for querying log hits stats over the given time range +- [`/select/logsql/streams`](#querying-streams) for querying [log streams](#https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) +- [`/select/logsql/stream_label_names`](#querying-stream-label-names) for querying [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label names +- [`/select/logsql/stream_label_values`](#querying-stream-label-values) for querying [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label values +- [`/select/logsql/field_names`](#querying-field-names) for querying [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) names. +- [`/select/logsql/field_values`](#querying-field-values) for querying [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) values. + +### Querying logs + +Logs stored in VictoriaLogs can be queried at the `/select/logsql/query` HTTP endpoint. The [LogsQL](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html) query must be passed via `query` argument. For example, the following query returns all the log entries with the `error` word: @@ -88,6 +100,10 @@ curl http://localhost:9428/select/logsql/query -H 'AccountID: 12' -H 'ProjectID: The number of requests to `/select/logsql/query` can be [monitored](https://docs.victoriametrics.com/VictoriaLogs/#monitoring) with `vl_http_requests_total{path="/select/logsql/query"}` metric. +- [Querying hits stats](#querying-hits-stats) +- [Querying streams](#querying-streams) +- [HTTP API](#http-api) + ### Querying hits stats VictoriaMetrics provides `/select/logsql/hits?query=&start=&end=&step=` HTTP endpoint, which returns the number @@ -187,10 +203,125 @@ The grouped fields are put inside `"fields"` object: See also: -- [Querying field names](#querying-field-names) -- [Querying field values](#querying-field-values) +- [Querying logs](#querying-logs) +- [Querying streams](#querying-streams) - [HTTP API](#http-api) +### Querying streams + +VictoriaLogs provides `/select/logsql/streams?query=&start=&end=` HTTP endpoint, which returns [streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) +from results of the given `` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[ ... ]` time range. + +The `` and `` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats). +If `` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs. +If `` is missing, then it equals to the maximum timestamp across logs stored in VictoriaLogs. + +For example, the following command returns streams across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word) +for the last 5 minutes: + +```sh +curl http://localhost:9428/select/logsql/streams -d 'query=error' -d 'start=5m' +``` + +Below is an example JSON output returned from this endpoint: + +```json +{ + "streams": [ + "{host=\"1.2.3.4\",app=\"foo\"}", + "{host=\"1.2.3.4\",app=\"bar\"}", + "{host=\"10.2.3.4\",app=\"foo\"}", + "{host=\"10.2.3.5\",app=\"baz\"}" + ] +} +``` + +The `/select/logsql/streams` endpoint supports optional `limit=N` query arg, which allows limiting the number of returned streams to `N`. +The endpoint returns arbitrary subset of values if their number exceeds `N`, so `limit=N` cannot be used for pagination over big number of streams. + +See also: + +- [Querying logs](#querying-logs) +- [Querying hits stats](#querying-hits-stats) +- [HTTP API](#http-api) + +### Querying stream label names + +VictoriaLogs provides `/select/logsql/stream_label_names?query=&start=&end=` HTTP endpoint, which returns +[log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label names from results +of the given `` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[ ... ]` time range. + +The `` and `` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats). +If `` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs. +If `` is missing, then it equals to the maximum timestamp across logs stored in VictoriaLogs. + +For example, the following command returns stream label names across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word) +for the last 5 minutes: + +```sh +curl http://localhost:9428/select/logsql/stream_label_names -d 'query=error' -d 'start=5m' +``` + +Below is an example JSON output returned from this endpoint: + +```json +{ + "names": [ + "app", + "container", + "datacenter", + "host", + "namespace" + ] +} +``` + +See also: + +- [Querying stream label names](#querying-stream-label-names) +- [Querying field values](#querying-field-values) +- [Querying streams](#querying-streams) +- [HTTP API](#http-api) + +### Querying stream label values + +VictoriaLogs provides `/select/logsql/stream_label_values?query=&start=&&label=` HTTP endpoint, +which returns [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label values for the label with the given `` name +from results of the given `` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[ ... ]` time range. + +The `` and `` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats). +If `` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs. +If `` is missing, then it equals to the maximum timestamp across logs stored in VictoriaLogs. + +For example, the following command returns values for the stream label `host` across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word) +for the last 5 minutes: + +```sh +curl http://localhost:9428/select/logsql/stream_label_values -d 'query=error' -d 'start=5m' -d 'label=host' +``` + +Below is an example JSON output returned from this endpoint: + +```json +{ + "values": [ + "host-0", + "host-1", + "host-2", + "host-3" + ] +} +``` + +The `/select/logsql/stream_label_names` endpoint supports optional `limit=N` query arg, which allows limiting the number of returned values to `N`. +The endpoint returns arbitrary subset of values if their number exceeds `N`, so `limit=N` cannot be used for pagination over big number of field values. + +See also: + +- [Querying stream label values](#querying-stream-label-values) +- [Querying field names](#querying-field-names) +- [Querying streams](#querying-streams) +- [HTTP API](#http-api) ### Querying field names @@ -225,13 +356,14 @@ Below is an example JSON output returned from this endpoint: See also: +- [Querying stream label names](#querying-stream-label-names) - [Querying field values](#querying-field-values) -- [Querying hits stats](#querying-hits-stats) +- [Querying streams](#querying-streams) - [HTTP API](#http-api) ### Querying field values -VictoriaLogs provides `/select/logsql/field_values?query=&field_name=&start=&end=` HTTP endpoint, which returns +VictoriaLogs provides `/select/logsql/field_values?query=&field=&start=&end=` HTTP endpoint, which returns unique values for the given `` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) from results of the given `` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[ ... ]` time range. @@ -243,7 +375,7 @@ For example, the following command returns unique values for `host` [field](http across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word) for the last 5 minutes: ```sh -curl http://localhost:9428/select/logsql/field_values -d 'query=error' -d 'field_name=host' -d 'start=5m' +curl http://localhost:9428/select/logsql/field_values -d 'query=error' -d 'field=host' -d 'start=5m' ``` Below is an example JSON output returned from this endpoint: @@ -265,8 +397,9 @@ The endpoint returns arbitrary subset of values if their number exceeds `N`, so See also: +- [Querying stream label values](#querying-stream-label-values) - [Querying field names](#querying-field-names) -- [Querying hits stats](#querying-hits-stats) +- [Querying streams](#querying-streams) - [HTTP API](#http-api) diff --git a/lib/logstorage/bitmap.go b/lib/logstorage/bitmap.go index b12680848..ac1c10695 100644 --- a/lib/logstorage/bitmap.go +++ b/lib/logstorage/bitmap.go @@ -98,6 +98,13 @@ func (bm *bitmap) areAllBitsSet() bool { return true } +func (bm *bitmap) isSetBit(i int) bool { + wordIdx := uint(i) / 64 + wordOffset := uint(i) % 64 + word := bm.a[wordIdx] + return (word & (1 << wordOffset)) != 0 +} + func (bm *bitmap) andNot(x *bitmap) { if bm.bitsLen != x.bitsLen { logger.Panicf("BUG: cannot merge bitmaps with distinct lengths; %d vs %d", bm.bitsLen, x.bitsLen) diff --git a/lib/logstorage/block_result.go b/lib/logstorage/block_result.go index 552cffd47..5162c5f57 100644 --- a/lib/logstorage/block_result.go +++ b/lib/logstorage/block_result.go @@ -28,12 +28,6 @@ type blockResult struct { // timestamps contain timestamps for the selected log entries in the block. timestamps []int64 - // csBufOffset contains csBuf offset for the requested columns. - // - // columns with indexes below csBufOffset are ignored. - // This is needed for simplifying data transformations at pipe stages. - csBufOffset int - // csBuf contains requested columns. csBuf []blockResultColumn @@ -52,8 +46,6 @@ func (br *blockResult) reset() { br.timestamps = br.timestamps[:0] - br.csBufOffset = 0 - clear(br.csBuf) br.csBuf = br.csBuf[:0] @@ -208,30 +200,13 @@ func (br *blockResult) sizeBytes() int { return n } -// addResultColumns adds the given rcs to br. -// -// The br is valid only until rcs are modified. -func (br *blockResult) addResultColumns(rcs []resultColumn) { - if len(rcs) == 0 || len(rcs[0].values) == 0 { - return - } - - for i := range rcs { - br.addResultColumn(&rcs[i]) - } -} - // setResultColumns sets the given rcs as br columns. // // The br is valid only until rcs are modified. -func (br *blockResult) setResultColumns(rcs []resultColumn) { +func (br *blockResult) setResultColumns(rcs []resultColumn, rowsCount int) { br.reset() - if len(rcs) == 0 || len(rcs[0].values) == 0 { - return - } - - br.timestamps = fastnum.AppendInt64Zeros(br.timestamps[:0], len(rcs[0].values)) + br.timestamps = fastnum.AppendInt64Zeros(br.timestamps[:0], rowsCount) for i := range rcs { br.addResultColumn(&rcs[i]) @@ -1227,56 +1202,59 @@ func (br *blockResult) getBucketedValue(s string, bf *byStatsField) string { // copyColumns copies columns from srcColumnNames to dstColumnNames. func (br *blockResult) copyColumns(srcColumnNames, dstColumnNames []string) { - if len(srcColumnNames) == 0 { - return + for i, srcName := range srcColumnNames { + br.copySingleColumn(srcName, dstColumnNames[i]) } +} - csBuf := br.csBuf - csBufOffset := len(csBuf) - for _, c := range br.getColumns() { - if idx := slices.Index(srcColumnNames, c.name); idx >= 0 { - c.name = dstColumnNames[idx] - csBuf = append(csBuf, *c) - // continue is skipped intentionally in order to leave the original column in the columns list. +func (br *blockResult) copySingleColumn(srcName, dstName string) { + found := false + cs := br.getColumns() + csBufLen := len(br.csBuf) + for _, c := range cs { + if c.name != dstName { + br.csBuf = append(br.csBuf, *c) } - if !slices.Contains(dstColumnNames, c.name) { - csBuf = append(csBuf, *c) + if c.name == srcName { + cCopy := *c + cCopy.name = dstName + br.csBuf = append(br.csBuf, cCopy) + found = true } } - br.csBufOffset = csBufOffset - br.csBuf = csBuf + if !found { + br.addConstColumn(dstName, "") + } + br.csBuf = append(br.csBuf[:0], br.csBuf[csBufLen:]...) br.csInitialized = false - - for _, dstColumnName := range dstColumnNames { - br.createMissingColumnByName(dstColumnName) - } } // renameColumns renames columns from srcColumnNames to dstColumnNames. func (br *blockResult) renameColumns(srcColumnNames, dstColumnNames []string) { - if len(srcColumnNames) == 0 { - return + for i, srcName := range srcColumnNames { + br.renameSingleColumn(srcName, dstColumnNames[i]) } +} - csBuf := br.csBuf - csBufOffset := len(csBuf) - for _, c := range br.getColumns() { - if idx := slices.Index(srcColumnNames, c.name); idx >= 0 { - c.name = dstColumnNames[idx] - csBuf = append(csBuf, *c) - continue - } - if !slices.Contains(dstColumnNames, c.name) { - csBuf = append(csBuf, *c) +func (br *blockResult) renameSingleColumn(srcName, dstName string) { + found := false + cs := br.getColumns() + csBufLen := len(br.csBuf) + for _, c := range cs { + if c.name == srcName { + cCopy := *c + cCopy.name = dstName + br.csBuf = append(br.csBuf, cCopy) + found = true + } else if c.name != dstName { + br.csBuf = append(br.csBuf, *c) } } - br.csBufOffset = csBufOffset - br.csBuf = csBuf + if !found { + br.addConstColumn(dstName, "") + } + br.csBuf = append(br.csBuf[:0], br.csBuf[csBufLen:]...) br.csInitialized = false - - for _, dstColumnName := range dstColumnNames { - br.createMissingColumnByName(dstColumnName) - } } // deleteColumns deletes columns with the given columnNames. @@ -1285,15 +1263,15 @@ func (br *blockResult) deleteColumns(columnNames []string) { return } - csBuf := br.csBuf - csBufOffset := len(csBuf) - for _, c := range br.getColumns() { + cs := br.getColumns() + csBufLen := len(br.csBuf) + for _, c := range cs { if !slices.Contains(columnNames, c.name) { - csBuf = append(csBuf, *c) + br.csBuf = append(br.csBuf, *c) } } - br.csBufOffset = csBufOffset - br.csBuf = csBuf + + br.csBuf = append(br.csBuf[:0], br.csBuf[csBufLen:]...) br.csInitialized = false } @@ -1305,14 +1283,21 @@ func (br *blockResult) setColumns(columnNames []string) { } // Slow path - construct the requested columns - csBuf := br.csBuf - csBufOffset := len(csBuf) - for _, columnName := range columnNames { - c := br.getColumnByName(columnName) - csBuf = append(csBuf, *c) + cs := br.getColumns() + csBufLen := len(br.csBuf) + for _, c := range cs { + if slices.Contains(columnNames, c.name) { + br.csBuf = append(br.csBuf, *c) + } } - br.csBufOffset = csBufOffset - br.csBuf = csBuf + + for _, columnName := range columnNames { + if idx := getBlockResultColumnIdxByName(cs, columnName); idx < 0 { + br.addConstColumn(columnName, "") + } + } + + br.csBuf = append(br.csBuf[:0], br.csBuf[csBufLen:]...) br.csInitialized = false } @@ -1344,22 +1329,12 @@ func (br *blockResult) getColumnByName(columnName string) *blockResultColumn { return &br.csBuf[len(br.csBuf)-1] } -func (br *blockResult) createMissingColumnByName(columnName string) { - for _, c := range br.getColumns() { - if c.name == columnName { - return - } - } - - br.addConstColumn(columnName, "") -} - func (br *blockResult) getColumns() []*blockResultColumn { if br.csInitialized { return br.cs } - csBuf := br.csBuf[br.csBufOffset:] + csBuf := br.csBuf clear(br.cs) cs := br.cs[:0] for i := range csBuf { @@ -1810,6 +1785,11 @@ type resultColumn struct { values []string } +func (rc *resultColumn) reset() { + rc.name = "" + rc.resetValues() +} + func (rc *resultColumn) resetValues() { clear(rc.values) rc.values = rc.values[:0] @@ -1818,8 +1798,8 @@ func (rc *resultColumn) resetValues() { func appendResultColumnWithName(dst []resultColumn, name string) []resultColumn { dst = slicesutil.SetLength(dst, len(dst)+1) rc := &dst[len(dst)-1] - rc.resetValues() rc.name = name + rc.resetValues() return dst } @@ -1862,5 +1842,12 @@ func visitValuesReadonly(bs *blockSearch, ch *columnHeader, bm *bitmap, f func(v }) } +func getCanonicalColumnName(columnName string) string { + if columnName == "" { + return "_msg" + } + return columnName +} + var nan = math.NaN() var inf = math.Inf(1) diff --git a/lib/logstorage/filter_exact.go b/lib/logstorage/filter_exact.go index fbd74d17d..98e9ea1ee 100644 --- a/lib/logstorage/filter_exact.go +++ b/lib/logstorage/filter_exact.go @@ -11,7 +11,7 @@ import ( // filterExact matches the exact value. // -// Example LogsQL: `fieldName:exact("foo bar")` +// Example LogsQL: `fieldName:exact("foo bar")` of `fieldName:="foo bar" type filterExact struct { fieldName string value string @@ -21,7 +21,7 @@ type filterExact struct { } func (fe *filterExact) String() string { - return fmt.Sprintf("%sexact(%s)", quoteFieldNameIfNeeded(fe.fieldName), quoteTokenIfNeeded(fe.value)) + return fmt.Sprintf("%s=%s", quoteFieldNameIfNeeded(fe.fieldName), quoteTokenIfNeeded(fe.value)) } func (fe *filterExact) updateNeededFields(neededFields fieldsSet) { diff --git a/lib/logstorage/filter_exact_prefix.go b/lib/logstorage/filter_exact_prefix.go index 653bdfc4b..e0b3b8648 100644 --- a/lib/logstorage/filter_exact_prefix.go +++ b/lib/logstorage/filter_exact_prefix.go @@ -20,7 +20,7 @@ type filterExactPrefix struct { } func (fep *filterExactPrefix) String() string { - return fmt.Sprintf("%sexact(%s*)", quoteFieldNameIfNeeded(fep.fieldName), quoteTokenIfNeeded(fep.prefix)) + return fmt.Sprintf("%s=%s*", quoteFieldNameIfNeeded(fep.fieldName), quoteTokenIfNeeded(fep.prefix)) } func (fep *filterExactPrefix) updateNeededFields(neededFields fieldsSet) { diff --git a/lib/logstorage/filter_regexp.go b/lib/logstorage/filter_regexp.go index 31b383826..4d8ad0245 100644 --- a/lib/logstorage/filter_regexp.go +++ b/lib/logstorage/filter_regexp.go @@ -16,7 +16,7 @@ type filterRegexp struct { } func (fr *filterRegexp) String() string { - return fmt.Sprintf("%sre(%q)", quoteFieldNameIfNeeded(fr.fieldName), fr.re.String()) + return fmt.Sprintf("%s~%q", quoteFieldNameIfNeeded(fr.fieldName), fr.re.String()) } func (fr *filterRegexp) updateNeededFields(neededFields fieldsSet) { diff --git a/lib/logstorage/if_filter.go b/lib/logstorage/if_filter.go new file mode 100644 index 000000000..f6852e604 --- /dev/null +++ b/lib/logstorage/if_filter.go @@ -0,0 +1,75 @@ +package logstorage + +import ( + "fmt" +) + +type ifFilter struct { + f filter + neededFields []string +} + +func (iff *ifFilter) String() string { + return "if (" + iff.f.String() + ")" +} + +func parseIfFilter(lex *lexer) (*ifFilter, error) { + if !lex.isKeyword("if") { + return nil, fmt.Errorf("unexpected keyword %q; expecting 'if'", lex.token) + } + lex.nextToken() + if !lex.isKeyword("(") { + return nil, fmt.Errorf("unexpected token %q after 'if'; expecting '('", lex.token) + } + lex.nextToken() + + if lex.isKeyword(")") { + lex.nextToken() + iff := &ifFilter{ + f: &filterNoop{}, + } + return iff, nil + } + + f, err := parseFilter(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'if' filter: %w", err) + } + if !lex.isKeyword(")") { + return nil, fmt.Errorf("unexpected token %q after 'if' filter; expecting ')'", lex.token) + } + lex.nextToken() + + neededFields := newFieldsSet() + f.updateNeededFields(neededFields) + + iff := &ifFilter{ + f: f, + neededFields: neededFields.getAll(), + } + + return iff, nil +} + +func (iff *ifFilter) optimizeFilterIn() { + if iff == nil { + return + } + + optimizeFilterIn(iff.f) +} + +func optimizeFilterIn(f filter) { + if f == nil { + return + } + + visitFunc := func(f filter) bool { + fi, ok := f.(*filterIn) + if ok && fi.q != nil { + fi.q.Optimize() + } + return false + } + _ = visitFilter(f, visitFunc) +} diff --git a/lib/logstorage/json_parser.go b/lib/logstorage/json_parser.go index 767d636a8..ccffcf315 100644 --- a/lib/logstorage/json_parser.go +++ b/lib/logstorage/json_parser.go @@ -33,16 +33,10 @@ type JSONParser struct { } func (p *JSONParser) reset() { - p.resetNobuf() - - p.buf = p.buf[:0] -} - -func (p *JSONParser) resetNobuf() { clear(p.Fields) p.Fields = p.Fields[:0] - p.prefixBuf = p.prefixBuf[:0] + p.buf = p.buf[:0] } // GetJSONParser returns JSONParser ready to parse JSON lines. @@ -66,39 +60,20 @@ func PutJSONParser(p *JSONParser) { var parserPool sync.Pool -// ParseLogMessageNoResetBuf parses the given JSON log message msg into p.Fields. -// -// It adds the given prefix to all the parsed field names. -// -// The p.Fields remains valid until the next call to PutJSONParser(). -func (p *JSONParser) ParseLogMessageNoResetBuf(msg, prefix string) error { - return p.parseLogMessage(msg, prefix, false) -} - // ParseLogMessage parses the given JSON log message msg into p.Fields. // -// It adds the given prefix to all the parsed field names. -// // The p.Fields remains valid until the next call to ParseLogMessage() or PutJSONParser(). -func (p *JSONParser) ParseLogMessage(msg []byte, prefix string) error { - msgStr := bytesutil.ToUnsafeString(msg) - return p.parseLogMessage(msgStr, prefix, true) -} +func (p *JSONParser) ParseLogMessage(msg []byte) error { + p.reset() -func (p *JSONParser) parseLogMessage(msg, prefix string, resetBuf bool) error { - v, err := p.p.Parse(msg) + msgStr := bytesutil.ToUnsafeString(msg) + v, err := p.p.Parse(msgStr) if err != nil { return fmt.Errorf("cannot parse json: %w", err) } if t := v.Type(); t != fastjson.TypeObject { return fmt.Errorf("expecting json dictionary; got %s", t) } - if resetBuf { - p.reset() - } else { - p.resetNobuf() - } - p.prefixBuf = append(p.prefixBuf[:0], prefix...) p.Fields, p.buf, p.prefixBuf = appendLogFields(p.Fields, p.buf, p.prefixBuf, v) return nil } diff --git a/lib/logstorage/json_parser_test.go b/lib/logstorage/json_parser_test.go index 4c294615f..818fd4976 100644 --- a/lib/logstorage/json_parser_test.go +++ b/lib/logstorage/json_parser_test.go @@ -10,7 +10,7 @@ func TestJSONParserFailure(t *testing.T) { t.Helper() p := GetJSONParser() - err := p.ParseLogMessage([]byte(data), "") + err := p.ParseLogMessage([]byte(data)) if err == nil { t.Fatalf("expecting non-nil error") } @@ -23,11 +23,11 @@ func TestJSONParserFailure(t *testing.T) { } func TestJSONParserSuccess(t *testing.T) { - f := func(data, prefix string, fieldsExpected []Field) { + f := func(data string, fieldsExpected []Field) { t.Helper() p := GetJSONParser() - err := p.ParseLogMessage([]byte(data), prefix) + err := p.ParseLogMessage([]byte(data)) if err != nil { t.Fatalf("unexpected error: %s", err) } @@ -37,23 +37,21 @@ func TestJSONParserSuccess(t *testing.T) { PutJSONParser(p) } - f("{}", "", nil) - f(`{"foo":"bar"}`, "", []Field{ + f("{}", nil) + f(`{"foo":"bar"}`, []Field{ { Name: "foo", Value: "bar", }, }) - f(`{"foo":"bar"}`, "prefix_", []Field{ + f(`{"foo":{"bar":{"x":"y","z":["foo"]}},"a":1,"b":true,"c":[1,2],"d":false}`, []Field{ { - Name: "prefix_foo", - Value: "bar", + Name: "foo.bar.x", + Value: "y", }, - }) - f(`{"foo":{"bar":"baz"},"a":1,"b":true,"c":[1,2],"d":false}`, "", []Field{ { - Name: "foo.bar", - Value: "baz", + Name: "foo.bar.z", + Value: `["foo"]`, }, { Name: "a", @@ -72,26 +70,4 @@ func TestJSONParserSuccess(t *testing.T) { Value: "false", }, }) - f(`{"foo":{"bar":"baz"},"a":1,"b":true,"c":[1,2],"d":false}`, "prefix_", []Field{ - { - Name: "prefix_foo.bar", - Value: "baz", - }, - { - Name: "prefix_a", - Value: "1", - }, - { - Name: "prefix_b", - Value: "true", - }, - { - Name: "prefix_c", - Value: "[1,2]", - }, - { - Name: "prefix_d", - Value: "false", - }, - }) } diff --git a/lib/logstorage/logfmt_parser.go b/lib/logstorage/logfmt_parser.go new file mode 100644 index 000000000..5eede8bbb --- /dev/null +++ b/lib/logstorage/logfmt_parser.go @@ -0,0 +1,77 @@ +package logstorage + +import ( + "strings" + "sync" +) + +type logfmtParser struct { + fields []Field +} + +func (p *logfmtParser) reset() { + clear(p.fields) + p.fields = p.fields[:0] +} + +func (p *logfmtParser) addField(name, value string) { + p.fields = append(p.fields, Field{ + Name: name, + Value: value, + }) +} + +func (p *logfmtParser) parse(s string) { + for { + // Search for field name + n := strings.IndexByte(s, '=') + if n < 0 { + // field name couldn't be read + return + } + + name := strings.TrimSpace(s[:n]) + s = s[n+1:] + if len(s) == 0 { + p.addField(name, "") + return + } + + // Search for field value + value, nOffset := tryUnquoteString(s) + if nOffset >= 0 { + p.addField(name, value) + s = s[nOffset:] + if len(s) == 0 { + return + } + if s[0] != ' ' { + return + } + s = s[1:] + } else { + n := strings.IndexByte(s, ' ') + if n < 0 { + p.addField(name, s) + return + } + p.addField(name, s[:n]) + s = s[n+1:] + } + } +} + +func getLogfmtParser() *logfmtParser { + v := logfmtParserPool.Get() + if v == nil { + return &logfmtParser{} + } + return v.(*logfmtParser) +} + +func putLogfmtParser(p *logfmtParser) { + p.reset() + logfmtParserPool.Put(p) +} + +var logfmtParserPool sync.Pool diff --git a/lib/logstorage/logfmt_parser_test.go b/lib/logstorage/logfmt_parser_test.go new file mode 100644 index 000000000..60161271e --- /dev/null +++ b/lib/logstorage/logfmt_parser_test.go @@ -0,0 +1,30 @@ +package logstorage + +import ( + "testing" +) + +func TestLogfmtParser(t *testing.T) { + f := func(s, resultExpected string) { + t.Helper() + + p := getLogfmtParser() + defer putLogfmtParser(p) + + p.parse(s) + result := marshalFieldsToJSON(nil, p.fields) + if string(result) != resultExpected { + t.Fatalf("unexpected result when parsing [%s]; got\n%s\nwant\n%s\n", s, result, resultExpected) + } + } + + f(``, `{}`) + f(`foo=bar`, `{"foo":"bar"}`) + f(`foo="bar=baz x=y"`, `{"foo":"bar=baz x=y"}`) + f(`foo=`, `{"foo":""}`) + f(`foo=bar baz="x y" a=b`, `{"foo":"bar","baz":"x y","a":"b"}`) + + // errors + f(`foo`, `{}`) + f(`foo=bar baz=x z qwe`, `{"foo":"bar","baz":"x"}`) +} diff --git a/lib/logstorage/parser.go b/lib/logstorage/parser.go index ef0ce27db..536b0016e 100644 --- a/lib/logstorage/parser.go +++ b/lib/logstorage/parser.go @@ -232,10 +232,15 @@ func (q *Query) AddCountByTimePipe(step, off int64, fields []string) { } s := fmt.Sprintf("stats by (%s) count() hits", byFieldsStr) lex := newLexer(s) + ps, err := parsePipeStats(lex) if err != nil { - logger.Panicf("BUG: unexpected error when parsing %q: %s", s, err) + logger.Panicf("BUG: unexpected error when parsing [%s]: %s", s, err) } + if !lex.isEnd() { + logger.Panicf("BUG: unexpected tail left after parsing [%s]: %q", s, lex.s) + } + q.pipes = append(q.pipes, ps) } @@ -320,10 +325,16 @@ func (q *Query) Optimize() { switch t := p.(type) { case *pipeStats: for _, f := range t.funcs { - if f.iff != nil { - optimizeFilterIn(f.iff) - } + f.iff.optimizeFilterIn() } + case *pipeFormat: + t.iff.optimizeFilterIn() + case *pipeExtract: + t.iff.optimizeFilterIn() + case *pipeUnpackJSON: + t.iff.optimizeFilterIn() + case *pipeUnpackLogfmt: + t.iff.optimizeFilterIn() } } } @@ -344,17 +355,6 @@ func removeStarFilters(f filter) filter { return f } -func optimizeFilterIn(f filter) { - visitFunc := func(f filter) bool { - fi, ok := f.(*filterIn) - if ok && fi.q != nil { - fi.q.Optimize() - } - return false - } - _ = visitFilter(f, visitFunc) -} - func optimizeSortOffsetPipes(pipes []pipe) []pipe { // Merge 'sort ... | offset ...' into 'sort ... offset ...' i := 1 @@ -498,11 +498,14 @@ func parseQuery(lex *lexer) (*Query, error) { f: f, } - pipes, err := parsePipes(lex) - if err != nil { - return nil, fmt.Errorf("%w; context: [%s]", err, lex.context()) + if lex.isKeyword("|") { + lex.nextToken() + pipes, err := parsePipes(lex) + if err != nil { + return nil, fmt.Errorf("%w; context: [%s]", err, lex.context()) + } + q.pipes = pipes } - q.pipes = pipes return q, nil } @@ -592,6 +595,10 @@ func parseGenericFilter(lex *lexer, fieldName string) (filter, error) { return parseFilterGT(lex, fieldName) case lex.isKeyword("<"): return parseFilterLT(lex, fieldName) + case lex.isKeyword("="): + return parseFilterEQ(lex, fieldName) + case lex.isKeyword("~"): + return parseFilterTilda(lex, fieldName) case lex.isKeyword("not", "!"): return parseFilterNot(lex, fieldName) case lex.isKeyword("exact"): @@ -1012,10 +1019,39 @@ func parseFilterRegexp(lex *lexer, fieldName string) (filter, error) { }) } -func parseFilterGT(lex *lexer, fieldName string) (filter, error) { - if fieldName == "" { - return nil, fmt.Errorf("'>' and '>=' must be prefixed with the field name") +func parseFilterTilda(lex *lexer, fieldName string) (filter, error) { + lex.nextToken() + arg := getCompoundFuncArg(lex) + re, err := regexp.Compile(arg) + if err != nil { + return nil, fmt.Errorf("invalid regexp %q: %w", arg, err) } + fr := &filterRegexp{ + fieldName: fieldName, + re: re, + } + return fr, nil +} + +func parseFilterEQ(lex *lexer, fieldName string) (filter, error) { + lex.nextToken() + phrase := getCompoundFuncArg(lex) + if lex.isKeyword("*") && !lex.isSkippedSpace { + lex.nextToken() + f := &filterExactPrefix{ + fieldName: fieldName, + prefix: phrase, + } + return f, nil + } + f := &filterExact{ + fieldName: fieldName, + value: phrase, + } + return f, nil +} + +func parseFilterGT(lex *lexer, fieldName string) (filter, error) { lex.nextToken() includeMinValue := false @@ -1045,9 +1081,6 @@ func parseFilterGT(lex *lexer, fieldName string) (filter, error) { } func parseFilterLT(lex *lexer, fieldName string) (filter, error) { - if fieldName == "" { - return nil, fmt.Errorf("'<' and '<=' must be prefixed with the field name") - } lex.nextToken() includeMaxValue := false @@ -1151,7 +1184,7 @@ func parseFilterRange(lex *lexer, fieldName string) (filter, error) { func parseFloat64(lex *lexer) (float64, string, error) { s, err := getCompoundToken(lex) if err != nil { - return 0, "", fmt.Errorf("cannot parse float64: %w", err) + return 0, "", fmt.Errorf("cannot parse float64 from %q: %w", s, err) } f, err := strconv.ParseFloat(s, 64) if err == nil { @@ -1164,7 +1197,7 @@ func parseFloat64(lex *lexer) (float64, string, error) { if err == nil { return float64(n), s, nil } - return 0, "", fmt.Errorf("cannot parse %q as float64: %w", lex.token, err) + return 0, "", fmt.Errorf("cannot parse %q as float64: %w", s, err) } func parseFuncArg(lex *lexer, fieldName string, callback func(args string) (filter, error)) (filter, error) { diff --git a/lib/logstorage/parser_test.go b/lib/logstorage/parser_test.go index bd45658d8..6e8c458a6 100644 --- a/lib/logstorage/parser_test.go +++ b/lib/logstorage/parser_test.go @@ -505,6 +505,8 @@ func TestParseRangeFilter(t *testing.T) { f(`duration:range[100ns, 1y2w2.5m3s5ms]`, `duration`, 100, 1*nsecsPerYear+2*nsecsPerWeek+2.5*nsecsPerMinute+3*nsecsPerSecond+5*nsecsPerMillisecond) + f(`>=10`, ``, 10, inf) + f(`<=10`, ``, -inf, 10) f(`foo:>10.43`, `foo`, nextafter(10.43, inf), inf) f(`foo: > -10.43`, `foo`, nextafter(-10.43, inf), inf) f(`foo:>=10.43`, `foo`, 10.43, inf) @@ -569,7 +571,7 @@ func TestParseQuerySuccess(t *testing.T) { f(`foo:(bar baz or not :xxx)`, `foo:bar foo:baz or !foo:xxx`) f(`(foo:bar and (foo:baz or aa:bb) and xx) and y`, `foo:bar (foo:baz or aa:bb) xx y`) f("level:error and _msg:(a or b)", "level:error (a or b)") - f("level: ( ((error or warn*) and re(foo))) (not (bar))", `(level:error or level:warn*) level:re("foo") !bar`) + f("level: ( ((error or warn*) and re(foo))) (not (bar))", `(level:error or level:warn*) level:~"foo" !bar`) f("!(foo bar or baz and not aa*)", `!(foo bar or baz !aa*)`) // prefix search @@ -687,12 +689,17 @@ func TestParseQuerySuccess(t *testing.T) { f("string_range-a:x", `string_range-a:x`) // exact filter - f("exact(foo)", `exact(foo)`) - f("exact(foo*)", `exact(foo*)`) - f("exact('foo bar),|baz')", `exact("foo bar),|baz")`) - f("exact('foo bar),|baz'*)", `exact("foo bar),|baz"*)`) - f(`exact(foo/b:ar)`, `exact("foo/b:ar")`) - f(`foo:exact(foo/b:ar*)`, `foo:exact("foo/b:ar"*)`) + f("exact(foo)", `=foo`) + f("exact(foo*)", `=foo*`) + f("exact('foo bar),|baz')", `="foo bar),|baz"`) + f("exact('foo bar),|baz'*)", `="foo bar),|baz"*`) + f(`exact(foo/b:ar)`, `="foo/b:ar"`) + f(`foo:exact(foo/b:ar*)`, `foo:="foo/b:ar"*`) + f(`exact("foo/bar")`, `="foo/bar"`) + f(`exact('foo/bar')`, `="foo/bar"`) + f(`="foo/bar"`, `="foo/bar"`) + f("=foo=bar =b<=a>z ='abc'*", `="foo=bar" ="b<=a>z" =abc*`) + f("==foo =>=bar x : ( = =a=20)", `="=foo" =">=bar" x:="=a=20`) // i filter f("i(foo)", `i(foo)`) @@ -750,11 +757,16 @@ func TestParseQuerySuccess(t *testing.T) { f(`foo: >= 10.5M`, `foo:>=10.5M`) f(`foo: < 10.5M`, `foo:<10.5M`) f(`foo: <= 10.5M`, `foo:<=10.5M`) + f(`foo:(>10 <=20)`, `foo:>10 foo:<=20`) + f(`>=10 <20`, `>=10 <20`) // re filter - f("re('foo|ba(r.+)')", `re("foo|ba(r.+)")`) - f("re(foo)", `re("foo")`) - f(`foo:re(foo-bar/baz.)`, `foo:re("foo-bar/baz.")`) + f("re('foo|ba(r.+)')", `~"foo|ba(r.+)"`) + f("re(foo)", `~"foo"`) + f(`foo:re(foo-bar/baz.)`, `foo:~"foo-bar/baz."`) + f(`~foo.bar.baz`, `~"foo.bar.baz"`) + f(`foo:~~foo~ba/ba>z`, `foo:~"~foo~ba/ba>z"`) + f(`foo:~'.*'`, `foo:~".*"`) // seq filter f(`seq()`, `seq()`) @@ -882,6 +894,10 @@ func TestParseQuerySuccess(t *testing.T) { f(`* | stats min(*) x`, `* | stats min(*) as x`) f(`* | stats min(foo,*,bar) x`, `* | stats min(*) as x`) + // stats pipe fields_min + f(`* | stats fields_Min(foo) bar`, `* | stats fields_min(foo) as bar`) + f(`* | stats BY(x, y, ) fields_MIN(foo,bar,) bar`, `* | stats by (x, y) fields_min(foo, bar) as bar`) + // stats pipe avg f(`* | stats Avg(foo) bar`, `* | stats avg(foo) as bar`) f(`* | stats BY(x, y, ) AVG(foo,bar,) bar`, `* | stats by (x, y) avg(foo, bar) as bar`) @@ -923,8 +939,8 @@ func TestParseQuerySuccess(t *testing.T) { f(`* | stats quantile(0, foo) bar`, `* | stats quantile(0, foo) as bar`) f(`* | stats quantile(1, foo) bar`, `* | stats quantile(1, foo) as bar`) f(`* | stats quantile(0.5, a, b, c) bar`, `* | stats quantile(0.5, a, b, c) as bar`) - f(`* | stats quantile(0.99, *) bar`, `* | stats quantile(0.99, *) as bar`) - f(`* | stats quantile(0.99, a, *, b) bar`, `* | stats quantile(0.99, *) as bar`) + f(`* | stats quantile(0.99) bar`, `* | stats quantile(0.99) as bar`) + f(`* | stats quantile(0.99, a, *, b) bar`, `* | stats quantile(0.99) as bar`) // stats pipe median f(`* | stats Median(foo) bar`, `* | stats median(foo) as bar`) @@ -997,10 +1013,11 @@ func TestParseQuerySuccess(t *testing.T) { // extract pipe f(`* | extract "foobaz"`, `* | extract "foobaz"`) - f(`* | extract from _msg "foobaz"`, `* | extract "foobaz"`) - f(`* | extract from '' 'foobaz'`, `* | extract "foobaz"`) - f("* | extract from x `foobaz`", `* | extract from x "foobaz"`) - f("* | extract from x foobaz", `* | extract from x "foobaz"`) + f(`* | extract "foobaz" from _msg`, `* | extract "foobaz"`) + f(`* | extract 'foobaz' from ''`, `* | extract "foobaz"`) + f("* | extract `foobaz` from x", `* | extract "foobaz" from x`) + f("* | extract foobaz from x", `* | extract "foobaz" from x`) + f("* | extract if (a:b) foobaz from x", `* | extract if (a:b) "foobaz" from x`) // unpack_json pipe f(`* | unpack_json`, `* | unpack_json`) @@ -1314,6 +1331,10 @@ func TestParseQueryFailure(t *testing.T) { f(`foo | stats min`) f(`foo | stats min()`) + // invalid stats min + f(`foo | stats fields_min`) + f(`foo | stats fields_min()`) + // invalid stats avg f(`foo | stats avg`) f(`foo | stats avg()`) @@ -1350,7 +1371,6 @@ func TestParseQueryFailure(t *testing.T) { f(`foo | stats quantile`) f(`foo | stats quantile() foo`) f(`foo | stats quantile(bar, baz) foo`) - f(`foo | stats quantile(0.5) foo`) f(`foo | stats quantile(-1, x) foo`) f(`foo | stats quantile(10, x) foo`) @@ -1540,6 +1560,49 @@ func TestQueryGetNeededColumns(t *testing.T) { f(`* | stats by(f3,f4) count(f1,f2) r1 | stats count(f2) r1, count(r1) r2 | fields r2`, `f1,f2,f3,f4`, ``) f(`* | stats by(f3,f4) count(f1,f2) r1 | stats count(f3) r1, count(r1) r2 | fields r1`, `f3,f4`, ``) + f(`* | stats avg() q`, `*`, ``) + f(`* | stats avg(*) q`, `*`, ``) + f(`* | stats avg(x) q`, `x`, ``) + f(`* | stats count_empty() q`, `*`, ``) + f(`* | stats count_empty(*) q`, `*`, ``) + f(`* | stats count_empty(x) q`, `x`, ``) + f(`* | stats count() q`, ``, ``) + f(`* | stats count(*) q`, ``, ``) + f(`* | stats count(x) q`, `x`, ``) + f(`* | stats count_uniq() q`, `*`, ``) + f(`* | stats count_uniq(*) q`, `*`, ``) + f(`* | stats count_uniq(x) q`, `x`, ``) + f(`* | stats fields_max(a) q`, `*`, ``) + f(`* | stats fields_max(a, *) q`, `*`, ``) + f(`* | stats fields_max(a, x) q`, `a,x`, ``) + f(`* | stats fields_min(a) q`, `*`, ``) + f(`* | stats fields_min(a, *) q`, `*`, ``) + f(`* | stats fields_min(a, x) q`, `a,x`, ``) + f(`* | stats min() q`, `*`, ``) + f(`* | stats min(*) q`, `*`, ``) + f(`* | stats min(x) q`, `x`, ``) + f(`* | stats median() q`, `*`, ``) + f(`* | stats median(*) q`, `*`, ``) + f(`* | stats median(x) q`, `x`, ``) + f(`* | stats max() q`, `*`, ``) + f(`* | stats max(*) q`, `*`, ``) + f(`* | stats max(x) q`, `x`, ``) + f(`* | stats quantile(0.5) q`, `*`, ``) + f(`* | stats quantile(0.5, *) q`, `*`, ``) + f(`* | stats quantile(0.5, x) q`, `x`, ``) + f(`* | stats sum() q`, `*`, ``) + f(`* | stats sum(*) q`, `*`, ``) + f(`* | stats sum(x) q`, `x`, ``) + f(`* | stats sum_len() q`, `*`, ``) + f(`* | stats sum_len(*) q`, `*`, ``) + f(`* | stats sum_len(x) q`, `x`, ``) + f(`* | stats uniq_values() q`, `*`, ``) + f(`* | stats uniq_values(*) q`, `*`, ``) + f(`* | stats uniq_values(x) q`, `x`, ``) + f(`* | stats values() q`, `*`, ``) + f(`* | stats values(*) q`, `*`, ``) + f(`* | stats values(x) q`, `x`, ``) + f(`_time:5m | stats by(_time:day) count() r1 | stats values(_time) r2`, `_time`, ``) f(`_time:1y | stats (_time:1w) count() r1 | stats count() r2`, `_time`, ``) @@ -1570,15 +1633,80 @@ func TestQueryGetNeededColumns(t *testing.T) { f(`* | fields x,y | field_names as bar | fields baz`, `x,y`, ``) f(`* | rm x,y | field_names as bar | fields baz`, `*`, `x,y`) - f(`* | extract from s1 "x"`, `*`, `f1,f2`) - f(`* | extract from s1 "x" | fields foo`, `foo`, ``) - f(`* | extract from s1 "x" | fields foo,s1`, `foo,s1`, ``) - f(`* | extract from s1 "x" | fields foo,f1`, `foo,s1`, ``) - f(`* | extract from s1 "x" | fields foo,f1,f2`, `foo,s1`, ``) - f(`* | extract from s1 "x" | rm foo`, `*`, `f1,f2,foo`) - f(`* | extract from s1 "x" | rm foo,s1`, `*`, `f1,f2,foo`) - f(`* | extract from s1 "x" | rm foo,f1`, `*`, `f1,f2,foo`) - f(`* | extract from s1 "x" | rm foo,f1,f2`, `*`, `f1,f2,foo,s1`) + f(`* | format "foo" as s1`, `*`, `s1`) + f(`* | format "foo" as s1`, `*`, `s1`) + f(`* | format "foo" as s1`, `*`, ``) + + f(`* | format if (x1:y) "foo" as s1`, `*`, `s1`) + f(`* | format if (x1:y) "foo" as s1`, `*`, `s1`) + f(`* | format if (s1:y) "foo" as s1`, `*`, ``) + f(`* | format if (x1:y) "foo" as s1`, `*`, ``) + + f(`* | format "foo" as s1 | fields f1`, `f1`, ``) + f(`* | format "foo" as s1 | fields s1`, ``, ``) + f(`* | format "foo" as s1 | fields f2`, `f2`, ``) + f(`* | format "foo" as s1 | fields f1`, `f1`, ``) + f(`* | format "foo" as s1 | fields s1`, `f1`, ``) + f(`* | format "foo" as s1 | fields f1`, `f1`, ``) + f(`* | format "foo" as s1 | fields s1`, `s1`, ``) + + f(`* | format if (f1:x) "foo" as s1 | fields s1`, `f1`, ``) + f(`* | format if (f1:x) "foo" as s1 | fields s2`, `s2`, ``) + + f(`* | format "foo" as s1 | rm f1`, `*`, `f1,s1`) + f(`* | format "foo" as s1 | rm s1`, `*`, `s1`) + f(`* | format "foo" as s1 | rm f2`, `*`, `f2,s1`) + f(`* | format "foo" as s1 | rm f1`, `*`, `s1`) + f(`* | format "foo" as s1 | rm s1`, `*`, `s1`) + f(`* | format "foo" as s1 | rm f1`, `*`, `f1`) + f(`* | format "foo" as s1 | rm s1`, `*`, `s1`) + + f(`* | format if (f1:x) "foo" as s1 | rm s1`, `*`, `s1`) + f(`* | format if (f1:x) "foo" as s1 | rm f1`, `*`, `s1`) + f(`* | format if (f1:x) "foo" as s1 | rm f2`, `*`, `f2,s1`) + + f(`* | extract "x" from s1`, `*`, `f1,f2`) + f(`* | extract if (f3:foo) "x" from s1`, `*`, `f1,f2`) + f(`* | extract if (f1:foo) "x" from s1`, `*`, `f2`) + f(`* | extract "x" from s1 | fields foo`, `foo`, ``) + f(`* | extract if (x:bar) "x" from s1 | fields foo`, `foo`, ``) + f(`* | extract "x" from s1| fields foo,s1`, `foo,s1`, ``) + f(`* | extract if (x:bar) "x" from s1 | fields foo,s1`, `foo,s1`, ``) + f(`* | extract "x" from s1 | fields foo,f1`, `foo,s1`, ``) + f(`* | extract if (x:bar) "x" from s1 | fields foo,f1`, `foo,s1,x`, ``) + f(`* | extract "x" from s1 | fields foo,f1,f2`, `foo,s1`, ``) + f(`* | extract if (x:bar) "x" from s1 | fields foo,f1,f2`, `foo,s1,x`, ``) + f(`* | extract "x" from s1 | rm foo`, `*`, `f1,f2,foo`) + f(`* | extract if (x:bar) "x" from s1 | rm foo`, `*`, `f1,f2,foo`) + f(`* | extract "x" from s1 | rm foo,s1`, `*`, `f1,f2,foo`) + f(`* | extract if (x:bar) "x" from s1 | rm foo,s1`, `*`, `f1,f2,foo`) + f(`* | extract "x" from s1 | rm foo,f1`, `*`, `f1,f2,foo`) + f(`* | extract if (x:bar) "x" from s1 | rm foo,f1`, `*`, `f1,f2,foo`) + f(`* | extract "x" from s1 | rm foo,f1,f2`, `*`, `f1,f2,foo,s1`) + f(`* | extract if (x:bar) "x" from s1 | rm foo,f1,f2`, `*`, `f1,f2,foo,s1`) + + f(`* | extract "xy" from s1 `, `*`, ``) + f(`* | extract if (x:foo) "xy" from s1`, `*`, ``) + f(`* | extract if (s1:foo) "xy" from s1`, `*`, ``) + f(`* | extract if (s1:foo) "xy" from s1`, `*`, `f1`) + + f(`* | extract "xy" from s1 | fields s2`, `s2`, ``) + f(`* | extract "xy" from s1 | fields s1`, `s1`, ``) + f(`* | extract if (x:foo) "xy" from s1 | fields s1`, `s1,x`, ``) + f(`* | extract if (x:foo) "xy" from s1 | fields s2`, `s2`, ``) + f(`* | extract if (s1:foo) "xy" from s1 | fields s1`, `s1`, ``) + f(`* | extract if (s1:foo) "xy" from s1 | fields s2`, `s2`, ``) + f(`* | extract if (s1:foo) "xy" from s1 | fields s1`, `s1`, ``) + f(`* | extract if (s1:foo) "xy" from s1 | fields s2`, `s2`, ``) + + f(`* | extract "xy" from s1 | rm s2`, `*`, `s2`) + f(`* | extract "xy" from s1 | rm s1`, `*`, `s1`) + f(`* | extract if (x:foo) "xy" from s1 | rm s1`, `*`, `s1`) + f(`* | extract if (x:foo) "xy" from s1 | rm s2`, `*`, `s2`) + f(`* | extract if (s1:foo) "xy" from s1 | rm s1`, `*`, `s1`) + f(`* | extract if (s1:foo) "xy" from s1 | rm s2`, `*`, `s2`) + f(`* | extract if (s1:foo) "xy" from s1 | rm s1`, `*`, `f1`) + f(`* | extract if (s1:foo) "xy" from s1 | rm s2`, `*`, `f1,s2`) f(`* | unpack_json`, `*`, ``) f(`* | unpack_json from s1`, `*`, ``) diff --git a/lib/logstorage/pattern.go b/lib/logstorage/pattern.go new file mode 100644 index 000000000..ed1f7758f --- /dev/null +++ b/lib/logstorage/pattern.go @@ -0,0 +1,229 @@ +package logstorage + +import ( + "fmt" + "html" + "strconv" + "strings" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" +) + +// pattern represents text pattern in the form 'some_textother_text...' +type pattern struct { + // steps contains steps for extracting fields from string + steps []patternStep + + // matches contains matches for every step in steps + matches []string + + // fields contains matches for non-empty fields + fields []patternField +} + +type patternField struct { + name string + value *string +} + +type patternStep struct { + prefix string + field string + opt string +} + +func (ptn *pattern) clone() *pattern { + steps := ptn.steps + fields, matches := newFieldsAndMatchesFromPatternSteps(steps) + if len(fields) == 0 { + logger.Panicf("BUG: fields cannot be empty for steps=%v", steps) + } + return &pattern{ + steps: steps, + matches: matches, + fields: fields, + } +} + +func parsePattern(s string) (*pattern, error) { + steps, err := parsePatternSteps(s) + if err != nil { + return nil, err + } + + // Verify that prefixes are non-empty between fields. The first prefix may be empty. + for i := 1; i < len(steps); i++ { + if steps[i].prefix == "" { + return nil, fmt.Errorf("missing delimiter between <%s> and <%s>", steps[i-1].field, steps[i].field) + } + } + + // Build pattern struct + fields, matches := newFieldsAndMatchesFromPatternSteps(steps) + if len(fields) == 0 { + return nil, fmt.Errorf("pattern %q must contain at least a single named field in the form ", s) + } + + ptn := &pattern{ + steps: steps, + matches: matches, + fields: fields, + } + return ptn, nil +} + +func newFieldsAndMatchesFromPatternSteps(steps []patternStep) ([]patternField, []string) { + matches := make([]string, len(steps)) + + var fields []patternField + for i, step := range steps { + if step.field != "" { + fields = append(fields, patternField{ + name: step.field, + value: &matches[i], + }) + } + } + + return fields, matches +} + +func (ptn *pattern) apply(s string) { + clear(ptn.matches) + + steps := ptn.steps + + if prefix := steps[0].prefix; prefix != "" { + n := strings.Index(s, prefix) + if n < 0 { + // Mismatch + return + } + s = s[n+len(prefix):] + } + + matches := ptn.matches + for i := range steps { + nextPrefix := "" + if i+1 < len(steps) { + nextPrefix = steps[i+1].prefix + } + + us, nOffset := tryUnquoteString(s) + if nOffset >= 0 { + // Matched quoted string + matches[i] = us + s = s[nOffset:] + if !strings.HasPrefix(s, nextPrefix) { + // Mismatch + return + } + s = s[len(nextPrefix):] + } else { + // Match unquoted string until the nextPrefix + if nextPrefix == "" { + matches[i] = s + return + } + n := strings.Index(s, nextPrefix) + if n < 0 { + // Mismatch + return + } + matches[i] = s[:n] + s = s[n+len(nextPrefix):] + } + } +} + +func tryUnquoteString(s string) (string, int) { + if len(s) == 0 { + return s, -1 + } + if s[0] != '"' && s[0] != '`' { + return s, -1 + } + qp, err := strconv.QuotedPrefix(s) + if err != nil { + return s, -1 + } + us, err := strconv.Unquote(qp) + if err != nil { + return s, -1 + } + return us, len(qp) +} + +func parsePatternSteps(s string) ([]patternStep, error) { + steps, err := parsePatternStepsInternal(s) + if err != nil { + return nil, err + } + + // Unescape prefixes + for i := range steps { + step := &steps[i] + step.prefix = html.UnescapeString(step.prefix) + } + + // extract options part from fields + for i := range steps { + step := &steps[i] + field := step.field + if n := strings.IndexByte(field, ':'); n >= 0 { + step.opt = field[:n] + step.field = field[n+1:] + } + } + + return steps, nil +} + +func parsePatternStepsInternal(s string) ([]patternStep, error) { + if len(s) == 0 { + return nil, nil + } + + var steps []patternStep + + n := strings.IndexByte(s, '<') + if n < 0 { + steps = append(steps, patternStep{ + prefix: s, + }) + return steps, nil + } + prefix := s[:n] + s = s[n+1:] + for { + n := strings.IndexByte(s, '>') + if n < 0 { + return nil, fmt.Errorf("missing '>' for <%s", s) + } + field := s[:n] + s = s[n+1:] + + if field == "_" || field == "*" { + field = "" + } + steps = append(steps, patternStep{ + prefix: prefix, + field: field, + }) + if len(s) == 0 { + break + } + + n = strings.IndexByte(s, '<') + if n < 0 { + steps = append(steps, patternStep{ + prefix: s, + }) + break + } + prefix = s[:n] + s = s[n+1:] + } + + return steps, nil +} diff --git a/lib/logstorage/pattern_test.go b/lib/logstorage/pattern_test.go new file mode 100644 index 000000000..eee5a1a26 --- /dev/null +++ b/lib/logstorage/pattern_test.go @@ -0,0 +1,239 @@ +package logstorage + +import ( + "reflect" + "testing" +) + +func TestPatternApply(t *testing.T) { + f := func(patternStr, s string, resultsExpected []string) { + t.Helper() + + checkFields := func(ptn *pattern) { + t.Helper() + if len(ptn.fields) != len(resultsExpected) { + t.Fatalf("unexpected number of results; got %d; want %d", len(ptn.fields), len(resultsExpected)) + } + for i, f := range ptn.fields { + if v := *f.value; v != resultsExpected[i] { + t.Fatalf("unexpected value for field %q; got %q; want %q", f.name, v, resultsExpected[i]) + } + } + } + + ptn, err := parsePattern(patternStr) + if err != nil { + t.Fatalf("cannot parse %q: %s", patternStr, err) + } + ptn.apply(s) + checkFields(ptn) + + // clone pattern and check fields again + ptnCopy := ptn.clone() + ptnCopy.apply(s) + checkFields(ptn) + } + + f("", "", []string{""}) + f("", "abc", []string{"abc"}) + f("bar", "", []string{""}) + f("bar", "bar", []string{""}) + f("bar", "bazbar", []string{"baz"}) + f("bar", "a bazbar xdsf", []string{"a baz"}) + f("bar<>", "a bazbar xdsf", []string{"a baz"}) + f("bar<>x", "a bazbar xdsf", []string{"a baz"}) + f("foo", "", []string{""}) + f("foo", "foo", []string{""}) + f("foo", "a foo xdf sdf", []string{" xdf sdf"}) + f("foo", "a foo foobar", []string{" foobar"}) + f("foobaz", "a foo foobar", []string{""}) + f("foobaz", "a foobaz bar", []string{""}) + f("foobaz", "a foo foobar baz", []string{" foobar "}) + f("foobaz", "a foo foobar bazabc", []string{" foobar "}) + + f("ip= <> path= ", "x=a, ip=1.2.3.4 method=GET host='abc' path=/foo/bar some tail here", []string{"1.2.3.4", "/foo/bar"}) + + // escaped pattern + f("ip=<>", "foo ip=<1.2.3.4> bar", []string{"1.2.3.4"}) + f("ip=<>", "foo ip= bar", []string{"foo&bar"}) + + // quoted fields + f(`"msg":,`, `{"foo":"bar","msg":"foo,b\"ar\n\t","baz":"x"}`, []string{`foo,b"ar` + "\n\t"}) + f(`foo=`, "foo=`bar baz,abc` def", []string{"bar baz,abc"}) + f(`foo= `, "foo=`bar baz,abc` def", []string{"bar baz,abc"}) + f(``, `"foo,\"bar"`, []string{`foo,"bar`}) + f(`,"bar`, `"foo,\"bar"`, []string{`foo,"bar`}) +} + +func TestParsePatternFailure(t *testing.T) { + f := func(patternStr string) { + t.Helper() + + ptn, err := parsePattern(patternStr) + if err == nil { + t.Fatalf("expecting error when parsing %q; got %v", patternStr, ptn) + } + } + + // Missing named fields + f("") + f("foobar") + f("<>") + f("<>foo<>bar") + + // Missing delimiter between fields + f("") + f("abcdef") + f("abc") + f("abc<_>") + f("abc<_><_>") +} + +func TestParsePatternStepsSuccess(t *testing.T) { + f := func(s string, stepsExpected []patternStep) { + t.Helper() + + steps, err := parsePatternSteps(s) + if err != nil { + t.Fatalf("unexpected error when parsing %q: %s", s, err) + } + if !reflect.DeepEqual(steps, stepsExpected) { + t.Fatalf("unexpected steps for [%s]; got %v; want %v", s, steps, stepsExpected) + } + } + + f("", nil) + + f("foobar", []patternStep{ + { + prefix: "foobar", + }, + }) + + f("<>", []patternStep{ + {}, + }) + + f("foo<>", []patternStep{ + { + prefix: "foo", + }, + }) + + f("", []patternStep{ + { + field: "foo", + }, + { + field: "bar", + }, + }) + + f("", []patternStep{ + { + field: "foo", + }, + }) + f("bar", []patternStep{ + { + field: "foo", + }, + { + prefix: "bar", + }, + }) + f("<>bar", []patternStep{ + {}, + { + prefix: "bar", + field: "foo", + }, + }) + f("bar", []patternStep{ + { + prefix: "bar", + field: "foo", + }, + }) + f("barabc", []patternStep{ + { + prefix: "bar", + field: "foo", + }, + { + prefix: "abc", + }, + }) + f("barabc<_>", []patternStep{ + { + prefix: "bar", + field: "foo", + }, + { + prefix: "abc", + }, + }) + f("bar", []patternStep{ + { + field: "foo", + }, + { + prefix: "bar", + field: "baz", + }, + }) + f("barbaz", []patternStep{ + { + prefix: "bar", + field: "foo", + }, + { + prefix: "baz", + }, + }) + f("<&>", []patternStep{ + { + prefix: "<&>", + }, + }) + f("<&gt;", []patternStep{ + { + prefix: "<", + field: "foo", + }, + { + prefix: ">", + }, + }) + f("barf<:foo:bar:baz>", []patternStep{ + { + field: "foo", + opt: "q", + }, + { + prefix: "bar", + field: "baz:c:y", + opt: "abc", + }, + { + prefix: "f", + field: "foo:bar:baz", + }, + }) + +} + +func TestParsePatternStepsFailure(t *testing.T) { + f := func(s string) { + t.Helper() + + steps, err := parsePatternSteps(s) + if err == nil { + t.Fatalf("expecting non-nil error when parsing %q; got steps: %v", s, steps) + } + } + + // missing > + f(""`, a) + benchmarkPatternApply(b, `"level":""`, a) }) b.Run("single-small-field-at-start-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"level":`, a) + benchmarkPatternApply(b, `"level":`, a) }) b.Run("single-small-field-at-end", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"foo":""`, a) + benchmarkPatternApply(b, `"foo":""`, a) }) b.Run("single-small-field-at-end-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"foo":`, a) + benchmarkPatternApply(b, `"foo":`, a) }) b.Run("single-medium-field", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"msg":""`, a) + benchmarkPatternApply(b, `"msg":""`, a) }) b.Run("single-medium-field-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"msg":`, a) + benchmarkPatternApply(b, `"msg":`, a) }) b.Run("single-large-field", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"stacktrace":""`, a) + benchmarkPatternApply(b, `"stacktrace":""`, a) }) b.Run("single-large-field-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"stacktrace":`, a) + benchmarkPatternApply(b, `"stacktrace":`, a) }) b.Run("two-fields", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"level":"",<_>"msg":""`, a) + benchmarkPatternApply(b, `"level":"",<_>"msg":""`, a) }) b.Run("two-fields-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"level":,<_>"msg":`, a) + benchmarkPatternApply(b, `"level":,<_>"msg":`, a) }) b.Run("many-fields", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"level":"","ts":"","caller":"","msg":"","error":""`, a) + benchmarkPatternApply(b, `"level":"","ts":"","caller":"","msg":"","error":""`, a) }) b.Run("many-fields-unquote", func(b *testing.B) { - benchmarkExtractFormatApply(b, `"level":,"ts":,"caller":,"msg":,"error":`, a) + benchmarkPatternApply(b, `"level":,"ts":,"caller":,"msg":,"error":`, a) }) } -func benchmarkExtractFormatApply(b *testing.B, pattern string, a []string) { - steps, err := parseExtractFormatSteps(pattern) +func benchmarkPatternApply(b *testing.B, patternStr string, a []string) { + ptnMain, err := parsePattern(patternStr) if err != nil { - b.Fatalf("unexpected error: %s", err) + b.Fatalf("cannot parse pattern %q: %s", patternStr, err) } n := 0 @@ -65,12 +65,12 @@ func benchmarkExtractFormatApply(b *testing.B, pattern string, a []string) { b.ReportAllocs() b.SetBytes(int64(n)) b.RunParallel(func(pb *testing.PB) { + ptn := ptnMain.clone() sink := 0 - ef := newExtractFormat(steps) for pb.Next() { for _, s := range a { - ef.apply(s) - for _, v := range ef.matches { + ptn.apply(s) + for _, v := range ptn.matches { sink += len(v) } } diff --git a/lib/logstorage/pipe.go b/lib/logstorage/pipe.go index 85323cc7e..ccc4dbeff 100644 --- a/lib/logstorage/pipe.go +++ b/lib/logstorage/pipe.go @@ -63,18 +63,20 @@ func (dpp defaultPipeProcessor) flush() error { func parsePipes(lex *lexer) ([]pipe, error) { var pipes []pipe - for !lex.isKeyword(")", "") { - if !lex.isKeyword("|") { - return nil, fmt.Errorf("expecting '|'; got %q", lex.token) - } - lex.nextToken() + for { p, err := parsePipe(lex) if err != nil { return nil, err } pipes = append(pipes, p) + + switch { + case lex.isKeyword("|"): + lex.nextToken() + case lex.isKeyword(")", ""): + return pipes, nil + } } - return pipes, nil } func parsePipe(lex *lexer) (pipe, error) { @@ -103,7 +105,7 @@ func parsePipe(lex *lexer) (pipe, error) { return nil, fmt.Errorf("cannot parse 'field_names' pipe: %w", err) } return pf, nil - case lex.isKeyword("fields"): + case lex.isKeyword("fields", "keep"): pf, err := parsePipeFields(lex) if err != nil { return nil, fmt.Errorf("cannot parse 'fields' pipe: %w", err) @@ -115,6 +117,12 @@ func parsePipe(lex *lexer) (pipe, error) { return nil, fmt.Errorf("cannot parse 'filter' pipe: %w", err) } return pf, nil + case lex.isKeyword("format"): + pf, err := parsePipeFormat(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'format' pipe: %w", err) + } + return pf, nil case lex.isKeyword("limit", "head"): pl, err := parsePipeLimit(lex) if err != nil { diff --git a/lib/logstorage/pipe_copy.go b/lib/logstorage/pipe_copy.go index d512bb0d5..8868b9c90 100644 --- a/lib/logstorage/pipe_copy.go +++ b/lib/logstorage/pipe_copy.go @@ -32,29 +32,21 @@ func (pc *pipeCopy) String() string { } func (pc *pipeCopy) updateNeededFields(neededFields, unneededFields fieldsSet) { - neededSrcFields := make([]bool, len(pc.srcFields)) - for i, dstField := range pc.dstFields { - if neededFields.contains(dstField) && !unneededFields.contains(dstField) { - neededSrcFields[i] = true - } - } - if neededFields.contains("*") { - // update only unneeded fields - unneededFields.addFields(pc.dstFields) - for i, srcField := range pc.srcFields { - if neededSrcFields[i] { + for i := len(pc.srcFields) - 1; i >= 0; i-- { + srcField := pc.srcFields[i] + dstField := pc.dstFields[i] + + if neededFields.contains("*") { + if !unneededFields.contains(dstField) { + unneededFields.add(dstField) unneededFields.remove(srcField) } - } - } else { - // update only needed fields and reset unneeded fields - neededFields.removeFields(pc.dstFields) - for i, srcField := range pc.srcFields { - if neededSrcFields[i] { + } else { + if neededFields.contains(dstField) { + neededFields.remove(dstField) neededFields.add(srcField) } } - unneededFields.reset() } } diff --git a/lib/logstorage/pipe_copy_test.go b/lib/logstorage/pipe_copy_test.go index c9e30ae22..327825ea4 100644 --- a/lib/logstorage/pipe_copy_test.go +++ b/lib/logstorage/pipe_copy_test.go @@ -5,6 +5,186 @@ import ( "testing" ) +func TestParsePipeCopySuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`copy foo as bar`) + f(`copy foo as bar, a as b`) +} + +func TestParsePipeCopyFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`copy`) + f(`copy x`) + f(`copy x as`) + f(`copy x y z`) +} + +func TestPipeCopy(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // single row, copy from existing field + f("copy a as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", `test`}, + }, + }) + + // single row, copy from existing field to multiple fields + f("copy a as b, a as c, _msg as d", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", `test`}, + {"c", `test`}, + {"d", `{"foo":"bar"}`}, + }, + }) + + // single row, copy from non-exsiting field + f("copy x as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", ``}, + }, + }) + + // copy to existing field + f("copy _msg as a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `{"foo":"bar"}`}, + }, + }) + + // copy to itself + f("copy a as a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + // swap copy + f("copy a as b, _msg as a, b as _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `test`}, + {"a", `{"foo":"bar"}`}, + {"b", `test`}, + }, + }) + + // copy to the same field multiple times + f("copy a as b, _msg as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", `{"foo":"bar"}`}, + }, + }) + + // chain copy + f("copy a as b, b as c", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", `test`}, + {"c", `test`}, + }, + }) + + // Multiple rows + f("copy a as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"a", `foobar`}, + }, + { + {"b", `baz`}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", `test`}, + }, + { + {"a", `foobar`}, + {"b", `foobar`}, + }, + { + {"b", ``}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + {"b", ""}, + }, + }) +} + func TestPipeCopyUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() @@ -13,6 +193,7 @@ func TestPipeCopyUpdateNeededFields(t *testing.T) { // all the needed fields f("copy s1 d1, s2 d2", "*", "", "*", "d1,d2") + f("copy a a", "*", "", "*", "") // all the needed fields, unneeded fields do not intersect with src and dst f("copy s1 d1 ,s2 d2", "*", "f1,f2", "*", "d1,d2,f1,f2") diff --git a/lib/logstorage/pipe_delete_test.go b/lib/logstorage/pipe_delete_test.go index 05c718dea..d423c23fd 100644 --- a/lib/logstorage/pipe_delete_test.go +++ b/lib/logstorage/pipe_delete_test.go @@ -4,6 +4,111 @@ import ( "testing" ) +func TestParsePipeDeleteSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`delete f1`) + f(`delete f1, f2`) +} + +func TestParsePipeDeleteFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`delete`) + f(`delete x y`) +} + +func TestPipeDelete(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // single row, drop existing field + f("delete _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `test`}, + }, + }) + + // single row, drop existing field multiple times + f("delete _msg, _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `test`}, + }, + }) + + // single row, drop all the fields + f("delete a, _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + {}, + }) + + // delete non-existing fields + f("delete foo, _msg, bar", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `test`}, + }, + }) + + // Multiple rows + f("delete _msg, a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"a", `foobar`}, + }, + { + {"b", `baz`}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + {"b", "df"}, + }, + }, [][]Field{ + {}, + {}, + { + {"b", `baz`}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + {"b", "df"}, + }, + }) +} + func TestPipeDeleteUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_extract.go b/lib/logstorage/pipe_extract.go index 854c51138..b9b4c3ae1 100644 --- a/lib/logstorage/pipe_extract.go +++ b/lib/logstorage/pipe_extract.go @@ -2,30 +2,30 @@ package logstorage import ( "fmt" - "html" - "strconv" - "strings" - "unsafe" - - "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" ) -// pipeExtract processes '| extract from ' pipe. +// pipeExtract processes '| extract ...' pipe. // // See https://docs.victoriametrics.com/victorialogs/logsql/#extract-pipe type pipeExtract struct { fromField string - steps []extractFormatStep + ptn *pattern - pattern string + patternStr string + + // iff is an optional filter for skipping the extract func + iff *ifFilter } func (pe *pipeExtract) String() string { s := "extract" + if pe.iff != nil { + s += " " + pe.iff.String() + } + s += " " + quoteTokenIfNeeded(pe.patternStr) if !isMsgFieldName(pe.fromField) { s += " from " + quoteTokenIfNeeded(pe.fromField) } - s += " " + quoteTokenIfNeeded(pe.pattern) return s } @@ -33,7 +33,7 @@ func (pe *pipeExtract) updateNeededFields(neededFields, unneededFields fieldsSet if neededFields.contains("*") { unneededFieldsOrig := unneededFields.clone() needFromField := false - for _, step := range pe.steps { + for _, step := range pe.ptn.steps { if step.field != "" { if !unneededFieldsOrig.contains(step.field) { needFromField = true @@ -43,110 +43,45 @@ func (pe *pipeExtract) updateNeededFields(neededFields, unneededFields fieldsSet } if needFromField { unneededFields.remove(pe.fromField) + if pe.iff != nil { + unneededFields.removeFields(pe.iff.neededFields) + } } else { unneededFields.add(pe.fromField) } } else { + neededFieldsOrig := neededFields.clone() needFromField := false - for _, step := range pe.steps { - if step.field != "" && neededFields.contains(step.field) { + for _, step := range pe.ptn.steps { + if step.field != "" && neededFieldsOrig.contains(step.field) { needFromField = true neededFields.remove(step.field) } } if needFromField { neededFields.add(pe.fromField) + if pe.iff != nil { + neededFields.addFields(pe.iff.neededFields) + } } } } func (pe *pipeExtract) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor { - shards := make([]pipeExtractProcessorShard, workersCount) - for i := range shards { - ef := newExtractFormat(pe.steps) - rcs := make([]resultColumn, len(ef.fields)) - for j := range rcs { - rcs[j].name = ef.fields[j].name - } - shards[i] = pipeExtractProcessorShard{ - pipeExtractProcessorShardNopad: pipeExtractProcessorShardNopad{ - ef: ef, - rcs: rcs, - }, + patterns := make([]*pattern, workersCount) + for i := range patterns { + patterns[i] = pe.ptn.clone() + } + + unpackFunc := func(uctx *fieldsUnpackerContext, s string) { + ptn := patterns[uctx.workerID] + ptn.apply(s) + for _, f := range ptn.fields { + uctx.addField(f.name, *f.value) } } - pep := &pipeExtractProcessor{ - pe: pe, - ppBase: ppBase, - - shards: shards, - } - return pep -} - -type pipeExtractProcessor struct { - pe *pipeExtract - ppBase pipeProcessor - - shards []pipeExtractProcessorShard -} - -type pipeExtractProcessorShard struct { - pipeExtractProcessorShardNopad - - // The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 . - _ [128 - unsafe.Sizeof(pipeExtractProcessorShardNopad{})%128]byte -} - -type pipeExtractProcessorShardNopad struct { - ef *extractFormat - - rcs []resultColumn -} - -func (pep *pipeExtractProcessor) writeBlock(workerID uint, br *blockResult) { - if len(br.timestamps) == 0 { - return - } - - shard := &pep.shards[workerID] - ef := shard.ef - rcs := shard.rcs - - c := br.getColumnByName(pep.pe.fromField) - if c.isConst { - v := c.valuesEncoded[0] - ef.apply(v) - for i, f := range ef.fields { - fieldValue := *f.value - rc := &rcs[i] - for range br.timestamps { - rc.addValue(fieldValue) - } - } - } else { - values := c.getValues(br) - for i, v := range values { - if i == 0 || values[i-1] != v { - ef.apply(v) - } - for j, f := range ef.fields { - rcs[j].addValue(*f.value) - } - } - } - - br.addResultColumns(rcs) - pep.ppBase.writeBlock(workerID, br) - - for i := range rcs { - rcs[i].resetValues() - } -} - -func (pep *pipeExtractProcessor) flush() error { - return nil + return newPipeUnpackProcessor(workersCount, unpackFunc, ppBase, pe.fromField, "", pe.iff) } func parsePipeExtract(lex *lexer) (*pipeExtract, error) { @@ -155,6 +90,27 @@ func parsePipeExtract(lex *lexer) (*pipeExtract, error) { } lex.nextToken() + // parse optional if (...) + var iff *ifFilter + if lex.isKeyword("if") { + f, err := parseIfFilter(lex) + if err != nil { + return nil, err + } + iff = f + } + + // parse pattern + patternStr, err := getCompoundToken(lex) + if err != nil { + return nil, fmt.Errorf("cannot read 'pattern': %w", err) + } + ptn, err := parsePattern(patternStr) + if err != nil { + return nil, fmt.Errorf("cannot parse 'pattern' %q: %w", patternStr, err) + } + + // parse optional 'from ...' part fromField := "_msg" if lex.isKeyword("from") { lex.nextToken() @@ -165,193 +121,12 @@ func parsePipeExtract(lex *lexer) (*pipeExtract, error) { fromField = f } - pattern, err := getCompoundToken(lex) - if err != nil { - return nil, fmt.Errorf("cannot read 'pattern': %w", err) - } - steps, err := parseExtractFormatSteps(pattern) - if err != nil { - return nil, fmt.Errorf("cannot parse 'pattern' %q: %w", pattern, err) + pe := &pipeExtract{ + fromField: fromField, + ptn: ptn, + patternStr: patternStr, + iff: iff, } - pe := &pipeExtract{ - fromField: fromField, - steps: steps, - pattern: pattern, - } return pe, nil } - -type extractFormat struct { - // steps contains steps for extracting fields from string - steps []extractFormatStep - - // matches contains matches for every step in steps - matches []string - - // fields contains matches for non-empty fields - fields []extractField -} - -type extractField struct { - name string - value *string -} - -type extractFormatStep struct { - prefix string - field string -} - -func newExtractFormat(steps []extractFormatStep) *extractFormat { - if len(steps) == 0 { - logger.Panicf("BUG: steps cannot be empty") - } - - matches := make([]string, len(steps)) - - var fields []extractField - for i, step := range steps { - if step.field != "" { - fields = append(fields, extractField{ - name: step.field, - value: &matches[i], - }) - } - } - if len(fields) == 0 { - logger.Panicf("BUG: fields cannot be empty") - } - - ef := &extractFormat{ - steps: steps, - matches: matches, - fields: fields, - } - return ef -} - -func (ef *extractFormat) apply(s string) { - clear(ef.matches) - - steps := ef.steps - - if prefix := steps[0].prefix; prefix != "" { - n := strings.Index(s, prefix) - if n < 0 { - // Mismatch - return - } - s = s[n+len(prefix):] - } - - matches := ef.matches - for i := range steps { - nextPrefix := "" - if i+1 < len(steps) { - nextPrefix = steps[i+1].prefix - } - - us, nOffset := tryUnquoteString(s) - if nOffset >= 0 { - // Matched quoted string - matches[i] = us - s = s[nOffset:] - if !strings.HasPrefix(s, nextPrefix) { - // Mismatch - return - } - s = s[len(nextPrefix):] - } else { - // Match unquoted string until the nextPrefix - if nextPrefix == "" { - matches[i] = s - return - } - n := strings.Index(s, nextPrefix) - if n < 0 { - // Mismatch - return - } - matches[i] = s[:n] - s = s[n+len(nextPrefix):] - } - } -} - -func tryUnquoteString(s string) (string, int) { - if len(s) == 0 { - return s, -1 - } - if s[0] != '"' && s[0] != '`' { - return s, -1 - } - qp, err := strconv.QuotedPrefix(s) - if err != nil { - return s, -1 - } - us, err := strconv.Unquote(qp) - if err != nil { - return s, -1 - } - return us, len(qp) -} - -func parseExtractFormatSteps(s string) ([]extractFormatStep, error) { - var steps []extractFormatStep - - hasNamedField := false - - n := strings.IndexByte(s, '<') - if n < 0 { - return nil, fmt.Errorf("missing <...> fields") - } - prefix := s[:n] - s = s[n+1:] - for { - n := strings.IndexByte(s, '>') - if n < 0 { - return nil, fmt.Errorf("missing '>' for <%s", s) - } - field := s[:n] - s = s[n+1:] - - if field == "_" || field == "*" { - field = "" - } - steps = append(steps, extractFormatStep{ - prefix: prefix, - field: field, - }) - if !hasNamedField && field != "" { - hasNamedField = true - } - if len(s) == 0 { - break - } - - n = strings.IndexByte(s, '<') - if n < 0 { - steps = append(steps, extractFormatStep{ - prefix: s, - }) - break - } - if n == 0 { - return nil, fmt.Errorf("missing delimiter after <%s>", field) - } - prefix = s[:n] - s = s[n+1:] - } - - if !hasNamedField { - return nil, fmt.Errorf("missing named fields like ") - } - - for i := range steps { - step := &steps[i] - step.prefix = html.UnescapeString(step.prefix) - } - - return steps, nil -} diff --git a/lib/logstorage/pipe_extract_test.go b/lib/logstorage/pipe_extract_test.go index b8c21ff0a..7a2d39f99 100644 --- a/lib/logstorage/pipe_extract_test.go +++ b/lib/logstorage/pipe_extract_test.go @@ -1,181 +1,196 @@ package logstorage import ( - "reflect" "testing" ) -func TestExtractFormatApply(t *testing.T) { - f := func(pattern, s string, resultsExpected []string) { +func TestParsePipeExtractSuccess(t *testing.T) { + f := func(pipeStr string) { t.Helper() - - steps, err := parseExtractFormatSteps(pattern) - if err != nil { - t.Fatalf("unexpected error: %s", err) - } - ef := newExtractFormat(steps) - ef.apply(s) - - if len(ef.fields) != len(resultsExpected) { - t.Fatalf("unexpected number of results; got %d; want %d", len(ef.fields), len(resultsExpected)) - } - for i, f := range ef.fields { - if v := *f.value; v != resultsExpected[i] { - t.Fatalf("unexpected value for field %q; got %q; want %q", f.name, v, resultsExpected[i]) - } - } + expectParsePipeSuccess(t, pipeStr) } - f("", "", []string{""}) - f("", "abc", []string{"abc"}) - f("bar", "", []string{""}) - f("bar", "bar", []string{""}) - f("bar", "bazbar", []string{"baz"}) - f("bar", "a bazbar xdsf", []string{"a baz"}) - f("bar<>", "a bazbar xdsf", []string{"a baz"}) - f("bar<>x", "a bazbar xdsf", []string{"a baz"}) - f("foo", "", []string{""}) - f("foo", "foo", []string{""}) - f("foo", "a foo xdf sdf", []string{" xdf sdf"}) - f("foo", "a foo foobar", []string{" foobar"}) - f("foobaz", "a foo foobar", []string{""}) - f("foobaz", "a foobaz bar", []string{""}) - f("foobaz", "a foo foobar baz", []string{" foobar "}) - f("foobaz", "a foo foobar bazabc", []string{" foobar "}) - - f("ip= <> path= ", "x=a, ip=1.2.3.4 method=GET host='abc' path=/foo/bar some tail here", []string{"1.2.3.4", "/foo/bar"}) - - // escaped pattern - f("ip=<>", "foo ip=<1.2.3.4> bar", []string{"1.2.3.4"}) - f("ip=<>", "foo ip= bar", []string{"foo&bar"}) - - // quoted fields - f(`"msg":,`, `{"foo":"bar","msg":"foo,b\"ar\n\t","baz":"x"}`, []string{`foo,b"ar` + "\n\t"}) - f(`foo=`, "foo=`bar baz,abc` def", []string{"bar baz,abc"}) - f(`foo= `, "foo=`bar baz,abc` def", []string{"bar baz,abc"}) - f(``, `"foo,\"bar"`, []string{`foo,"bar`}) - f(`,"bar`, `"foo,\"bar"`, []string{`foo,"bar`}) + f(`extract "foo"`) + f(`extract "foo" from x`) + f(`extract if (x:y) "foo" from baz`) } -func TestParseExtractFormatStepsSuccess(t *testing.T) { - f := func(s string, stepsExpected []extractFormatStep) { +func TestParsePipeExtractFailure(t *testing.T) { + f := func(pipeStr string) { t.Helper() - - steps, err := parseExtractFormatSteps(s) - if err != nil { - t.Fatalf("unexpected error when parsing %q: %s", s, err) - } - if !reflect.DeepEqual(steps, stepsExpected) { - t.Fatalf("unexpected steps for [%s]; got %v; want %v", s, steps, stepsExpected) - } + expectParsePipeFailure(t, pipeStr) } - f("", []extractFormatStep{ - { - field: "foo", - }, - }) - f("bar", []extractFormatStep{ - { - field: "foo", - }, - { - prefix: "bar", - }, - }) - f("<>bar", []extractFormatStep{ - {}, - { - prefix: "bar", - field: "foo", - }, - }) - f("bar", []extractFormatStep{ - { - prefix: "bar", - field: "foo", - }, - }) - f("barabc", []extractFormatStep{ - { - prefix: "bar", - field: "foo", - }, - { - prefix: "abc", - }, - }) - f("barabc<_>", []extractFormatStep{ - { - prefix: "bar", - field: "foo", - }, - { - prefix: "abc", - }, - }) - f("bar", []extractFormatStep{ - { - field: "foo", - }, - { - prefix: "bar", - field: "baz", - }, - }) - f("barbaz", []extractFormatStep{ - { - prefix: "bar", - field: "foo", - }, - { - prefix: "baz", - }, - }) - f("<&gt;", []extractFormatStep{ - { - prefix: "<", - field: "foo", - }, - { - prefix: ">", - }, - }) + f(`extract`) + f(`extract from`) + f(`extract from x`) + f(`extract from x "y"`) + f(`extract if (x:y)`) + f(`extract "a" if (x:y)`) + f(`extract "a"`) + f(`extract ""`) + f(`extract "<*>foo<_>bar"`) } -func TestParseExtractFormatStepFailure(t *testing.T) { - f := func(s string) { +func TestPipeExtract(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { t.Helper() - - _, err := parseExtractFormatSteps(s) - if err == nil { - t.Fatalf("expecting non-nil error when parsing %q", s) - } + expectPipeResults(t, pipeStr, rows, rowsExpected) } - // empty string - f("") + // single row, extract from _msg + f(`extract "baz= a="`, [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + }, + }, [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + {"abc", "x y=z"}, + {"aa", "b"}, + }, + }) - // zero fields - f("foobar") + // single row, extract from _msg into _msg + f(`extract "msg=<_msg>"`, [][]Field{ + { + {"_msg", `msg=bar`}, + }, + }, [][]Field{ + { + {"_msg", "bar"}, + }, + }) - // Zero named fields - f("<>") - f("foo<>") - f("<>foo") - f("foo<_>bar<*>baz<>xxx") + // single row, extract from non-existing field + f(`extract "foo=" from x`, [][]Field{ + { + {"_msg", `foo=bar`}, + }, + }, [][]Field{ + { + {"_msg", `foo=bar`}, + {"bar", ""}, + }, + }) - // missing delimiter between fields - f("") - f("<>") - f("<>") - f("bb<>aa") - f("aa") - f("aabb") + // single row, pattern mismatch + f(`extract "foo=" from x`, [][]Field{ + { + {"x", `foobar`}, + }, + }, [][]Field{ + { + {"x", `foobar`}, + {"bar", ""}, + }, + }) - // missing > - f(" baz=" from x`, [][]Field{ + { + {"x", `a foo="a\"b\\c" cde baz=aa`}, + }, + }, [][]Field{ + { + {"x", `a foo="a\"b\\c" cde baz=aa`}, + {"bar", `a"b\c`}, + {"xx", ""}, + }, + }) + + // single row, overwirte existing column + f(`extract "foo= baz=" from x`, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", "abc"}, + }, + }, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", `cc`}, + {"xx", `aa b`}, + }, + }) + + // single row, if match + f(`extract if (x:baz) "foo= baz=" from "x"`, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", "abc"}, + }, + }, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", `cc`}, + {"xx", `aa b`}, + }, + }) + + // single row, if mismatch + f(`extract if (bar:"") "foo= baz=" from 'x'`, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", "abc"}, + }, + }, [][]Field{ + { + {"x", `a foo=cc baz=aa b`}, + {"bar", `abc`}, + }, + }) + + // multiple rows with distinct set of labels + f(`extract if (!ip:keep) "ip= "`, [][]Field{ + { + {"foo", "bar"}, + {"_msg", "request from ip=1.2.3.4 xxx"}, + {"f3", "y"}, + }, + { + {"foo", "aaa"}, + {"_msg", "ip=5.4.3.1 abcd"}, + {"ip", "keep"}, + {"a", "b"}, + }, + { + {"foo", "aaa"}, + {"_msg", "ip=34.32.11.94 abcd"}, + {"ip", "ppp"}, + {"a", "b"}, + }, + { + {"foo", "klkfs"}, + {"_msg", "sdfdsfds dsf fd fdsa ip=123 abcd"}, + {"ip", "bbbsd"}, + {"a", "klo2i"}, + }, + }, [][]Field{ + { + {"foo", "bar"}, + {"_msg", "request from ip=1.2.3.4 xxx"}, + {"f3", "y"}, + {"ip", "1.2.3.4"}, + }, + { + {"foo", "aaa"}, + {"_msg", "ip=5.4.3.1 abcd"}, + {"ip", "keep"}, + {"a", "b"}, + }, + { + {"foo", "aaa"}, + {"_msg", "ip=34.32.11.94 abcd"}, + {"ip", "34.32.11.94"}, + {"a", "b"}, + }, + { + {"foo", "klkfs"}, + {"_msg", "sdfdsfds dsf fd fdsa ip=123 abcd"}, + {"ip", "123"}, + {"a", "klo2i"}, + }, + }) } func TestPipeExtractUpdateNeededFields(t *testing.T) { @@ -185,29 +200,70 @@ func TestPipeExtractUpdateNeededFields(t *testing.T) { } // all the needed fields - f("extract from x ''", "*", "", "*", "foo") + f("extract '' from x", "*", "", "*", "foo") + f("extract if (foo:bar) '' from x", "*", "", "*", "") - // all the needed fields, unneeded fields do not intersect with fromField and output fields - f("extract from x ''", "*", "f1,f2", "*", "f1,f2,foo") + // unneeded fields do not intersect with pattern and output fields + f("extract '' from x", "*", "f1,f2", "*", "f1,f2,foo") + f("extract if (f1:x) '' from x", "*", "f1,f2", "*", "f2,foo") + f("extract if (foo:bar f1:x) '' from x", "*", "f1,f2", "*", "f2") - // all the needed fields, unneeded fields intersect with fromField - f("extract from x ''", "*", "f2,x", "*", "f2,foo") + // unneeded fields intersect with pattern + f("extract '' from x", "*", "f2,x", "*", "f2,foo") + f("extract if (f1:abc) '' from x", "*", "f2,x", "*", "f2,foo") + f("extract if (f2:abc) '' from x", "*", "f2,x", "*", "foo") - // all the needed fields, unneeded fields intersect with output fields - f("extract from x 'x'", "*", "f2,foo", "*", "bar,f2,foo") + // unneeded fields intersect with output fields + f("extract 'x' from x", "*", "f2,foo", "*", "bar,f2,foo") + f("extract if (f1:abc) 'x' from x", "*", "f2,foo", "*", "bar,f2,foo") + f("extract if (f2:abc foo:w) 'x' from x", "*", "f2,foo", "*", "bar") - // all the needed fields, unneeded fields intersect with all the output fields - f("extract from x 'x'", "*", "f2,foo,bar", "*", "bar,f2,foo,x") + // unneeded fields intersect with all the output fields + f("extract 'x' from x", "*", "f2,foo,bar", "*", "bar,f2,foo,x") + f("extract if (a:b f2:q x:y foo:w) 'x' from x", "*", "f2,foo,bar", "*", "bar,f2,foo,x") - // needed fields do not intersect with fromField and output fields - f("extract from x 'x'", "f1,f2", "", "f1,f2", "") + // needed fields do not intersect with pattern and output fields + f("extract 'x' from x", "f1,f2", "", "f1,f2", "") + f("extract if (a:b) 'x' from x", "f1,f2", "", "f1,f2", "") + f("extract if (f1:b) 'x' from x", "f1,f2", "", "f1,f2", "") - // needed fields intersect with fromField - f("extract from x 'x'", "f2,x", "", "f2,x", "") + // needed fields intersect with pattern field + f("extract 'x' from x", "f2,x", "", "f2,x", "") + f("extract if (a:b) 'x' from x", "f2,x", "", "f2,x", "") // needed fields intersect with output fields - f("extract from x 'x'", "f2,foo", "", "f2,x", "") + f("extract 'x' from x", "f2,foo", "", "f2,x", "") + f("extract if (a:b) 'x' from x", "f2,foo", "", "a,f2,x", "") - // needed fields intersect with fromField and output fields - f("extract from x 'x'", "f2,foo,x,y", "", "f2,x,y", "") + // needed fields intersect with pattern and output fields + f("extract 'x' from x", "f2,foo,x,y", "", "f2,x,y", "") + f("extract if (a:b foo:q) 'x' from x", "f2,foo,x,y", "", "a,f2,foo,x,y", "") +} + +func expectParsePipeFailure(t *testing.T, pipeStr string) { + t.Helper() + + lex := newLexer(pipeStr) + p, err := parsePipe(lex) + if err == nil && lex.isEnd() { + t.Fatalf("expecting error when parsing [%s]; parsed result: [%s]", pipeStr, p) + } +} + +func expectParsePipeSuccess(t *testing.T, pipeStr string) { + t.Helper() + + lex := newLexer(pipeStr) + p, err := parsePipe(lex) + if err != nil { + t.Fatalf("cannot parse [%s]: %s", pipeStr, err) + } + if !lex.isEnd() { + t.Fatalf("unexpected tail after parsing [%s]: [%s]", pipeStr, lex.s) + } + + pipeStrResult := p.String() + if pipeStrResult != pipeStr { + t.Fatalf("unexpected string representation of pipe; got\n%s\nwant\n%s", pipeStrResult, pipeStr) + } } diff --git a/lib/logstorage/pipe_field_names.go b/lib/logstorage/pipe_field_names.go index 9ac1a206d..284855c8d 100644 --- a/lib/logstorage/pipe_field_names.go +++ b/lib/logstorage/pipe_field_names.go @@ -123,12 +123,17 @@ type pipeFieldNamesWriteContext struct { rcs [1]resultColumn br blockResult + // rowsCount is the number of rows in the current block + rowsCount int + + // valuesLen is the total length of values in the current block valuesLen int } func (wctx *pipeFieldNamesWriteContext) writeRow(v string) { wctx.rcs[0].addValue(v) wctx.valuesLen += len(v) + wctx.rowsCount++ if wctx.valuesLen >= 1_000_000 { wctx.flush() } @@ -140,7 +145,8 @@ func (wctx *pipeFieldNamesWriteContext) flush() { wctx.valuesLen = 0 // Flush rcs to ppBase - br.setResultColumns(wctx.rcs[:1]) + br.setResultColumns(wctx.rcs[:1], wctx.rowsCount) + wctx.rowsCount = 0 wctx.pfp.ppBase.writeBlock(0, br) br.reset() wctx.rcs[0].resetValues() diff --git a/lib/logstorage/pipe_field_names_test.go b/lib/logstorage/pipe_field_names_test.go index 4b19f2e34..72e5d4adc 100644 --- a/lib/logstorage/pipe_field_names_test.go +++ b/lib/logstorage/pipe_field_names_test.go @@ -4,6 +4,64 @@ import ( "testing" ) +func TestParsePipeFieldNamesSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`field_names as x`) +} + +func TestParsePipeFieldNamesFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`field_names`) + f(`field_names(foo)`) + f(`field_names a b`) + f(`field_names as`) +} + +func TestPipeFieldNames(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // single row, result column doesn't clash with original columns + f("field_names as x", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"x", "_msg"}, + }, + { + {"x", "a"}, + }, + }) + + // single row, result column do clashes with original columns + f("field_names as _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", "_msg"}, + }, + { + {"_msg", "a"}, + }, + }) +} + func TestPipeFieldNamesUpdateNeededFields(t *testing.T) { f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_fields.go b/lib/logstorage/pipe_fields.go index 07298c6dd..a391cbd0e 100644 --- a/lib/logstorage/pipe_fields.go +++ b/lib/logstorage/pipe_fields.go @@ -77,7 +77,7 @@ func (pfp *pipeFieldsProcessor) flush() error { } func parsePipeFields(lex *lexer) (*pipeFields, error) { - if !lex.isKeyword("fields") { + if !lex.isKeyword("fields", "keep") { return nil, fmt.Errorf("expecting 'fields'; got %q", lex.token) } diff --git a/lib/logstorage/pipe_fields_test.go b/lib/logstorage/pipe_fields_test.go index a7b1e792b..7df2ecced 100644 --- a/lib/logstorage/pipe_fields_test.go +++ b/lib/logstorage/pipe_fields_test.go @@ -4,6 +4,135 @@ import ( "testing" ) +func TestParsePipeFieldsSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`fields *`) + f(`fields f1`) + f(`fields f1, f2, f3`) +} + +func TestParsePipeFieldsFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`fields`) + f(`fields x y`) +} + +func TestPipeFields(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // single row, star + f("fields *", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + // single row, leave existing field + f("fields a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `test`}, + }, + }) + + // single row, no existing fields + f("fields x, y", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"x", ``}, + {"y", ``}, + }, + }) + + // single row, mention existing field multiple times + f("fields a, a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `test`}, + }, + }) + + // mention non-existing fields + f("fields foo, a, bar", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"foo", ""}, + {"bar", ""}, + {"a", `test`}, + }, + }) + + // Multiple rows + f("fields a, b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"a", `foobar`}, + }, + { + {"b", `baz`}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + {"d", "df"}, + }, + }, [][]Field{ + { + {"a", `test`}, + {"b", ``}, + }, + { + {"a", `foobar`}, + {"b", ""}, + }, + { + {"a", ""}, + {"b", "baz"}, + }, + { + {"a", ""}, + {"b", ""}, + }, + }) +} + func TestPipeFieldsUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_filter_test.go b/lib/logstorage/pipe_filter_test.go index 73da2c8bb..0c3183019 100644 --- a/lib/logstorage/pipe_filter_test.go +++ b/lib/logstorage/pipe_filter_test.go @@ -4,6 +4,103 @@ import ( "testing" ) +func TestParsePipeFilterSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`filter *`) + f(`filter foo bar`) + f(`filter a:b or c:d in(x,y) z:>343`) +} + +func TestParsePipeFilterFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`filter`) + f(`filter |`) + f(`filter ()`) +} + +func TestPipeFilter(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // filter mismatch + f("filter abc", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{}) + + // filter match + f("filter foo", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + // multiple rows + f("filter x:foo y:bar", [][]Field{ + { + {"a", "f1"}, + {"x", "foo"}, + {"y", "bar"}, + }, + { + {"a", "f2"}, + {"x", "x foo bar"}, + {"y", "aa bar bbb"}, + {"z", "iwert"}, + }, + { + {"a", "f3"}, + {"x", "x fo bar"}, + {"y", "aa bar bbb"}, + {"z", "it"}, + }, + { + {"a", "f4"}, + {"x", "x foo bar"}, + {"y", "aa ba bbb"}, + {"z", "t"}, + }, + { + {"x", "x foo"}, + {"y", "aa bar"}, + }, + }, [][]Field{ + { + {"a", "f1"}, + {"x", "foo"}, + {"y", "bar"}, + }, + { + {"a", "f2"}, + {"x", "x foo bar"}, + {"y", "aa bar bbb"}, + {"z", "iwert"}, + }, + { + {"x", "x foo"}, + {"y", "aa bar"}, + }, + }) +} + func TestPipeFilterUpdateNeededFields(t *testing.T) { f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_format.go b/lib/logstorage/pipe_format.go new file mode 100644 index 000000000..459400c4d --- /dev/null +++ b/lib/logstorage/pipe_format.go @@ -0,0 +1,200 @@ +package logstorage + +import ( + "fmt" + "strconv" + "unsafe" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" +) + +// pipeFormat processes '| format ...' pipe. +// +// See https://docs.victoriametrics.com/victorialogs/logsql/#format-pipe +type pipeFormat struct { + formatStr string + steps []patternStep + + resultField string + + // iff is an optional filter for skipping the format func + iff *ifFilter +} + +func (pf *pipeFormat) String() string { + s := "format" + if pf.iff != nil { + s += " " + pf.iff.String() + } + s += " " + quoteTokenIfNeeded(pf.formatStr) + if !isMsgFieldName(pf.resultField) { + s += " as " + quoteTokenIfNeeded(pf.resultField) + } + return s +} + +func (pf *pipeFormat) updateNeededFields(neededFields, unneededFields fieldsSet) { + if neededFields.contains("*") { + if !unneededFields.contains(pf.resultField) { + unneededFields.add(pf.resultField) + if pf.iff != nil { + unneededFields.removeFields(pf.iff.neededFields) + } + for _, step := range pf.steps { + if step.field != "" { + unneededFields.remove(step.field) + } + } + } + } else { + if neededFields.contains(pf.resultField) { + neededFields.remove(pf.resultField) + if pf.iff != nil { + neededFields.addFields(pf.iff.neededFields) + } + for _, step := range pf.steps { + if step.field != "" { + neededFields.add(step.field) + } + } + } + } +} + +func (pf *pipeFormat) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor { + return &pipeFormatProcessor{ + pf: pf, + ppBase: ppBase, + + shards: make([]pipeFormatProcessorShard, workersCount), + } +} + +type pipeFormatProcessor struct { + pf *pipeFormat + ppBase pipeProcessor + + shards []pipeFormatProcessorShard +} + +type pipeFormatProcessorShard struct { + pipeFormatProcessorShardNopad + + // The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 . + _ [128 - unsafe.Sizeof(pipeFormatProcessorShardNopad{})%128]byte +} + +type pipeFormatProcessorShardNopad struct { + bm bitmap + + uctx fieldsUnpackerContext + wctx pipeUnpackWriteContext +} + +func (pfp *pipeFormatProcessor) writeBlock(workerID uint, br *blockResult) { + if len(br.timestamps) == 0 { + return + } + + shard := &pfp.shards[workerID] + shard.wctx.init(workerID, pfp.ppBase, br) + shard.uctx.init(workerID, "") + + bm := &shard.bm + bm.init(len(br.timestamps)) + bm.setBits() + if iff := pfp.pf.iff; iff != nil { + iff.f.applyToBlockResult(br, bm) + if bm.isZero() { + pfp.ppBase.writeBlock(workerID, br) + return + } + } + + for rowIdx := range br.timestamps { + if bm.isSetBit(rowIdx) { + shard.formatRow(pfp.pf, br, rowIdx) + shard.wctx.writeRow(rowIdx, shard.uctx.fields) + } else { + shard.wctx.writeRow(rowIdx, nil) + } + } + + shard.wctx.flush() + shard.wctx.reset() + shard.uctx.reset() +} + +func (pfp *pipeFormatProcessor) flush() error { + return nil +} + +func (shard *pipeFormatProcessorShard) formatRow(pf *pipeFormat, br *blockResult, rowIdx int) { + bb := bbPool.Get() + b := bb.B + for _, step := range pf.steps { + b = append(b, step.prefix...) + if step.field != "" { + c := br.getColumnByName(step.field) + v := c.getValueAtRow(br, rowIdx) + if step.opt == "q" { + b = strconv.AppendQuote(b, v) + } else { + b = append(b, v...) + } + } + } + bb.B = b + + s := bytesutil.ToUnsafeString(b) + shard.uctx.resetFields() + shard.uctx.addField(pf.resultField, s) + bbPool.Put(bb) +} + +func parsePipeFormat(lex *lexer) (*pipeFormat, error) { + if !lex.isKeyword("format") { + return nil, fmt.Errorf("unexpected token: %q; want %q", lex.token, "format") + } + lex.nextToken() + + // parse optional if (...) + var iff *ifFilter + if lex.isKeyword("if") { + f, err := parseIfFilter(lex) + if err != nil { + return nil, err + } + iff = f + } + + // parse format + formatStr, err := getCompoundToken(lex) + if err != nil { + return nil, fmt.Errorf("cannot read 'format': %w", err) + } + steps, err := parsePatternSteps(formatStr) + if err != nil { + return nil, fmt.Errorf("cannot parse 'pattern' %q: %w", formatStr, err) + } + + // parse optional 'as ...` part + resultField := "_msg" + if lex.isKeyword("as") { + lex.nextToken() + field, err := parseFieldName(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse result field after 'format %q as': %w", formatStr, err) + } + resultField = field + } + + pf := &pipeFormat{ + formatStr: formatStr, + steps: steps, + resultField: resultField, + iff: iff, + } + + return pf, nil +} diff --git a/lib/logstorage/pipe_format_test.go b/lib/logstorage/pipe_format_test.go new file mode 100644 index 000000000..bda91e588 --- /dev/null +++ b/lib/logstorage/pipe_format_test.go @@ -0,0 +1,201 @@ +package logstorage + +import ( + "testing" +) + +func TestParsePipeFormatSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`format "foo"`) + f(`format "" as x`) + f(`format "<>" as x`) + f(`format foo as x`) + f(`format ""`) + f(`format "bar"`) + f(`format "barbac"`) + f(`format if (x:y) "barbac"`) +} + +func TestParsePipeFormatFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`format`) + f(`format if`) + f(`format foo bar`) + f(`format foo if`) + f(`format foo as x if (x:y)`) +} + +func TestPipeFormat(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // plain string into a single field + f(`format '{"foo":,"bar":""}' as x`, [][]Field{ + { + {"foo", `"abc"`}, + {"bar", `cde`}, + }, + }, [][]Field{ + { + {"foo", `"abc"`}, + {"bar", `cde`}, + {"x", `{"foo":"\"abc\"","bar":"cde"}`}, + }, + }) + + // plain string into a single field + f(`format foo as x`, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "x"}, + }, + }, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "x"}, + {"x", `foo`}, + }, + }) + + // plain string with html escaping into a single field + f(`format "<foo>" as x`, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "x"}, + }, + }, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "x"}, + {"x", ``}, + }, + }) + + // format with empty placeholders into existing field + f(`format "<_>foo<_>" as _msg`, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "x"}, + }, + }, [][]Field{ + { + {"_msg", `foo`}, + {"a", "x"}, + }, + }) + + // format with various placeholders into new field + f(`format "aaa<_msg>xxx" as x`, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "b"}, + }, + }, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "b"}, + {"x", `aaafoobarxxbx`}, + }, + }) + + // format into existing field + f(`format "aaa<_msg>xxx"`, [][]Field{ + { + {"_msg", `foobar`}, + {"a", "b"}, + }, + }, [][]Field{ + { + {"_msg", `aaafoobarxxbx`}, + {"a", "b"}, + }, + }) + + // conditional format over multiple rows + f(`format if (!c:*) "a: , b: , x: " as c`, [][]Field{ + { + {"b", "bar"}, + {"a", "foo"}, + {"c", "keep-me"}, + }, + { + {"c", ""}, + {"a", "f"}, + }, + { + {"b", "x"}, + }, + }, [][]Field{ + { + {"b", "bar"}, + {"a", "foo"}, + {"c", "keep-me"}, + }, + { + {"a", "f"}, + {"c", "a: f, b: , x: f"}, + }, + { + {"b", "x"}, + {"c", "a: , b: x, x: "}, + }, + }) +} + +func TestPipeFormatUpdateNeededFields(t *testing.T) { + f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { + t.Helper() + expectPipeNeededFields(t, s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected) + } + + // all the needed fields + f(`format "foo" as x`, "*", "", "*", "x") + f(`format "foo" as x`, "*", "", "*", "x") + f(`format if (f2:z) "foo" as x`, "*", "", "*", "x") + + // unneeded fields do not intersect with pattern and output field + f(`format "foo" as x`, "*", "f1,f2", "*", "f1,f2,x") + f(`format "foo" as x`, "*", "f1,f2", "*", "f1,f2,x") + f(`format if (f4:z) "foo" as x`, "*", "f1,f2", "*", "f1,f2,x") + f(`format if (f1:z) "foo" as x`, "*", "f1,f2", "*", "f2,x") + + // unneeded fields intersect with pattern + f(`format "foo" as x`, "*", "f1,f2", "*", "f2,x") + f(`format if (f4:z) "foo" as x`, "*", "f1,f2", "*", "f2,x") + f(`format if (f2:z) "foo" as x`, "*", "f1,f2", "*", "x") + + // unneeded fields intersect with output field + f(`format "foo" as x`, "*", "x,y", "*", "x,y") + f(`format if (f2:z) "foo" as x`, "*", "x,y", "*", "x,y") + f(`format if (y:z) "foo" as x`, "*", "x,y", "*", "x,y") + + // needed fields do not intersect with pattern and output field + f(`format "foo" as f2`, "x,y", "", "x,y", "") + f(`format if (f3:z) "foo" as f2`, "x,y", "", "x,y", "") + f(`format if (x:z) "foo" as f2`, "x,y", "", "x,y", "") + + // needed fields intersect with pattern field + f(`format "foo" as f2`, "f1,y", "", "f1,y", "") + f(`format if (f3:z) "foo" as f2`, "f1,y", "", "f1,y", "") + f(`format if (x:z) "foo" as f2`, "f1,y", "", "f1,y", "") + + // needed fields intersect with output field + f(`format "foo" as f2`, "f2,y", "", "f1,y", "") + f(`format if (f3:z) "foo" as f2`, "f2,y", "", "f1,f3,y", "") + f(`format if (x:z or y:w) "foo" as f2`, "f2,y", "", "f1,x,y", "") + + // needed fields intersect with pattern and output fields + f(`format "foo" as f2`, "f1,f2,y", "", "f1,y", "") + f(`format if (f3:z) "foo" as f2`, "f1,f2,y", "", "f1,f3,y", "") + f(`format if (x:z or y:w) "foo" as f2`, "f1,f2,y", "", "f1,x,y", "") +} diff --git a/lib/logstorage/pipe_limit_test.go b/lib/logstorage/pipe_limit_test.go index 49e79d5d2..bc7afa4fd 100644 --- a/lib/logstorage/pipe_limit_test.go +++ b/lib/logstorage/pipe_limit_test.go @@ -4,6 +4,98 @@ import ( "testing" ) +func TestParsePipeLimitSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`limit 10`) + f(`limit 10000`) +} + +func TestParsePipeLimitFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`limit`) + f(`limit -10`) + f(`limit foo`) +} + +func TestPipeLimit(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("limit 100", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + f("limit 1", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + f("limit 0", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{}) + + f("limit 1", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"_msg", `abc`}, + {"a", `aiewr`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + f("limit 1", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"_msg", `abc`}, + {"a", `aiewr`}, + {"asdf", "fsf"}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) +} + func TestPipeLimitUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_offset_test.go b/lib/logstorage/pipe_offset_test.go index 53b22af61..57f62a719 100644 --- a/lib/logstorage/pipe_offset_test.go +++ b/lib/logstorage/pipe_offset_test.go @@ -4,6 +4,92 @@ import ( "testing" ) +func TestParsePipeOffsetSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`offset 10`) + f(`offset 10000`) +} + +func TestParsePipeOffsetFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`offset`) + f(`offset -10`) + f(`offset foo`) +} + +func TestPipeOffset(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("offset 100", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{}) + + f("offset 0", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + f("offset 1", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"_msg", `abc`}, + {"a", `aiewr`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `aiewr`}, + }, + }) + + f("offset 2", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"_msg", `sdfsd`}, + {"adffd", `aiewr`}, + {"assdff", "fsf"}, + }, + { + {"_msg", `abc`}, + {"a", `aiewr`}, + {"asdf", "fsf"}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `aiewr`}, + {"asdf", "fsf"}, + }, + }) +} + func TestPipeOffsetUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_rename.go b/lib/logstorage/pipe_rename.go index 99a774ad4..76814c3e3 100644 --- a/lib/logstorage/pipe_rename.go +++ b/lib/logstorage/pipe_rename.go @@ -32,33 +32,25 @@ func (pr *pipeRename) String() string { } func (pr *pipeRename) updateNeededFields(neededFields, unneededFields fieldsSet) { - neededSrcFields := make([]bool, len(pr.srcFields)) - for i, dstField := range pr.dstFields { - if neededFields.contains(dstField) && !unneededFields.contains(dstField) { - neededSrcFields[i] = true - } - } - if neededFields.contains("*") { - // update only unneeded fields - unneededFields.addFields(pr.dstFields) - for i, srcField := range pr.srcFields { - if neededSrcFields[i] { - unneededFields.remove(srcField) - } else { + for i := len(pr.srcFields) - 1; i >= 0; i-- { + srcField := pr.srcFields[i] + dstField := pr.dstFields[i] + + if neededFields.contains("*") { + if unneededFields.contains(dstField) { unneededFields.add(srcField) + } else { + unneededFields.add(dstField) + unneededFields.remove(srcField) } - } - } else { - // update only needed fields and reset unneeded fields - neededFields.removeFields(pr.dstFields) - for i, srcField := range pr.srcFields { - if neededSrcFields[i] { + } else { + if neededFields.contains(dstField) { + neededFields.remove(dstField) neededFields.add(srcField) } else { neededFields.remove(srcField) } } - unneededFields.reset() } } diff --git a/lib/logstorage/pipe_rename_test.go b/lib/logstorage/pipe_rename_test.go index 506a2f5fa..4b44e5ee7 100644 --- a/lib/logstorage/pipe_rename_test.go +++ b/lib/logstorage/pipe_rename_test.go @@ -4,6 +4,175 @@ import ( "testing" ) +func TestParsePipeRenameSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`rename foo as bar`) + f(`rename foo as bar, a as b`) +} + +func TestParsePipeRenameFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`rename`) + f(`rename x`) + f(`rename x as`) + f(`rename x y z`) +} + +func TestPipeRename(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // single row, rename from existing field + f("rename a as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"b", `test`}, + }, + }) + + // single row, rename from existing field to multiple fields + f("rename a as b, a as c, _msg as d", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"b", `test`}, + {"c", ``}, + {"d", `{"foo":"bar"}`}, + }, + }) + + // single row, rename from non-exsiting field + f("rename x as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + {"b", ``}, + }, + }) + + // rename to existing field + f("rename _msg as a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"a", `{"foo":"bar"}`}, + }, + }) + + // rename to itself + f("rename a as a", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }) + + // swap rename + f("rename a as b, _msg as a, b as _msg", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `test`}, + {"a", `{"foo":"bar"}`}, + }, + }) + + // rename to the same field multiple times + f("rename a as b, _msg as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"b", `{"foo":"bar"}`}, + }, + }) + + // chain rename (shouldn't work - otherwise swap rename will break) + f("rename a as b, b as c", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"c", `test`}, + }, + }) + + // Multiple rows + f("rename a as b", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"a", `test`}, + }, + { + {"a", `foobar`}, + }, + { + {"b", `baz`}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"b", `test`}, + }, + { + {"b", `foobar`}, + }, + { + {"b", ``}, + {"c", "d"}, + {"e", "afdf"}, + }, + { + {"c", "dss"}, + {"b", ""}, + }, + }) +} + func TestPipeRenameUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() @@ -12,6 +181,7 @@ func TestPipeRenameUpdateNeededFields(t *testing.T) { // all the needed fields f("rename s1 d1, s2 d2", "*", "", "*", "d1,d2") + f("rename a a", "*", "", "*", "") // all the needed fields, unneeded fields do not intersect with src and dst f("rename s1 d1, s2 d2", "*", "f1,f2", "*", "d1,d2,f1,f2") diff --git a/lib/logstorage/pipe_sort.go b/lib/logstorage/pipe_sort.go index a29da7aa4..cdc294644 100644 --- a/lib/logstorage/pipe_sort.go +++ b/lib/logstorage/pipe_sort.go @@ -485,8 +485,14 @@ type pipeSortWriteContext struct { rcs []resultColumn br blockResult + // rowsWritten is the total number of rows passed to writeNextRow. rowsWritten uint64 - valuesLen int + + // rowsCount is the number of rows in the current block + rowsCount int + + // valuesLen is the length of all the values in the current block + valuesLen int } func (wctx *pipeSortWriteContext) writeNextRow(shard *pipeSortProcessorShard) { @@ -516,7 +522,7 @@ func (wctx *pipeSortWriteContext) writeNextRow(shard *pipeSortProcessorShard) { } } if !areEqualColumns { - // send the current block to bbBase and construct a block with new set of columns + // send the current block to ppBase and construct a block with new set of columns wctx.flush() rcs = wctx.rcs[:0] @@ -543,6 +549,7 @@ func (wctx *pipeSortWriteContext) writeNextRow(shard *pipeSortProcessorShard) { wctx.valuesLen += len(v) } + wctx.rowsCount++ if wctx.valuesLen >= 1_000_000 { wctx.flush() } @@ -554,12 +561,9 @@ func (wctx *pipeSortWriteContext) flush() { wctx.valuesLen = 0 - if len(rcs) == 0 { - return - } - // Flush rcs to ppBase - br.setResultColumns(rcs) + br.setResultColumns(rcs, wctx.rowsCount) + wctx.rowsCount = 0 wctx.psp.ppBase.writeBlock(0, br) br.reset() for i := range rcs { diff --git a/lib/logstorage/pipe_sort_test.go b/lib/logstorage/pipe_sort_test.go index b2bb0aef8..10b6564fb 100644 --- a/lib/logstorage/pipe_sort_test.go +++ b/lib/logstorage/pipe_sort_test.go @@ -4,6 +4,254 @@ import ( "testing" ) +func TestParsePipeSortSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`sort`) + f(`sort by (x)`) + f(`sort by (x) limit 10`) + f(`sort by (x) offset 20 limit 10`) + f(`sort by (x desc, y) desc`) +} + +func TestParsePipeSortFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`sort a`) + f(`sort by`) + f(`sort by(x) foo`) + f(`sort by(x) limit`) + f(`sort by(x) limit N`) + f(`sort by(x) offset`) + f(`sort by(x) offset N`) +} + +func TestPipeSort(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + // Sort by all fields + f("sort", [][]Field{ + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"_msg", `abc`}, + {"a", `2`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + }) + + // Sort by a single field + f("sort by (a)", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + }, [][]Field{ + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"_msg", `abc`}, + {"a", `2`}, + }, + }) + + // Sort by a in descending order + f("sort by (a) desc", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + }) + + // Sort by multiple fields + f("sort by (a, b desc) desc", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", ""}, + }, + }) + + // Sort by multiple fields with limit + f("sort by (a, b) limit 1", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"_msg", `def`}, + {"a", `1`}, + {"b", ""}, + }, + }) + + // Sort by multiple fields with limit desc + f("sort by (a, b) desc limit 1", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"a", `2`}, + {"b", `54`}, + }, + }) + + // Sort by multiple fields with offset + f("sort by (a, b) offset 1", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }) + + // Sort by multiple fields with offset and limit + f("sort by (a, b) offset 1 limit 1", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + }) + + // Sort by multiple fields with offset and limit + f("sort by (a, b) desc offset 2 limit 100", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"_msg", `def`}, + {"a", `1`}, + {"b", ""}, + }, + }) +} + func TestPipeSortUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_stats.go b/lib/logstorage/pipe_stats.go index 1397a1e60..4643017ae 100644 --- a/lib/logstorage/pipe_stats.go +++ b/lib/logstorage/pipe_stats.go @@ -2,7 +2,6 @@ package logstorage import ( "fmt" - "slices" "strings" "sync/atomic" "unsafe" @@ -28,11 +27,8 @@ type pipeStatsFunc struct { // f is stats function to execute f statsFunc - // neededFieldsForFunc contains needed fields for f execution - neededFieldsForFunc []string - // iff is an additional filter, which is applied to results before executing f on them - iff filter + iff *ifFilter // resultName is the name of the output generated by f resultName string @@ -90,7 +86,7 @@ func (ps *pipeStats) String() string { for i, f := range ps.funcs { line := f.f.String() if f.iff != nil { - line += " if (" + f.iff.String() + ")" + line += " " + f.iff.String() } line += " as " + quoteTokenIfNeeded(f.resultName) a[i] = line @@ -112,7 +108,7 @@ func (ps *pipeStats) updateNeededFields(neededFields, unneededFields fieldsSet) if neededFieldsOrig.contains(f.resultName) && !unneededFields.contains(f.resultName) { f.f.updateNeededFields(neededFields) if f.iff != nil { - f.iff.updateNeededFields(neededFields) + neededFields.addFields(f.iff.neededFields) } } } @@ -311,7 +307,7 @@ func (shard *pipeStatsProcessorShard) applyPerFunctionFilters(brSrc *blockResult bm := &shard.bms[i] bm.init(len(brSrc.timestamps)) bm.setBits() - iff.applyToBlockResult(brSrc, bm) + iff.f.applyToBlockResult(brSrc, bm) if bm.areAllBitsSet() { // Fast path - per-function filter doesn't filter out rows brs[i] = brSrc @@ -323,7 +319,7 @@ func (shard *pipeStatsProcessorShard) applyPerFunctionFilters(brSrc *blockResult if bm.isZero() { brDst.reset() } else { - brDst.initFromFilterNeededColumns(brSrc, bm, funcs[i].neededFieldsForFunc) + brDst.initFromFilterNeededColumns(brSrc, bm, iff.neededFields) } brs[i] = brDst } @@ -442,6 +438,7 @@ func (psp *pipeStatsProcessor) flush() error { var br blockResult var values []string + rowsCount := 0 valuesLen := 0 for key, psg := range m { // m may be quite big, so this loop can take a lot of time and CPU. @@ -478,8 +475,11 @@ func (psp *pipeStatsProcessor) flush() error { rcs[i].addValue(v) valuesLen += len(v) } + + rowsCount++ if valuesLen >= 1_000_000 { - br.setResultColumns(rcs) + br.setResultColumns(rcs, rowsCount) + rowsCount = 0 psp.ppBase.writeBlock(0, &br) br.reset() for i := range rcs { @@ -489,7 +489,7 @@ func (psp *pipeStatsProcessor) flush() error { } } - br.setResultColumns(rcs) + br.setResultColumns(rcs, rowsCount) psp.ppBase.writeBlock(0, &br) return nil @@ -526,18 +526,14 @@ func parsePipeStats(lex *lexer) (*pipeStats, error) { if lex.isKeyword("if") { iff, err := parseIfFilter(lex) if err != nil { - return nil, fmt.Errorf("cannot parse 'if' filter for %s: %w", sf, err) + return nil, err } f.iff = iff - - neededFields := newFieldsSet() - iff.updateNeededFields(neededFields) - f.neededFieldsForFunc = neededFields.getAll() } resultName, err := parseResultName(lex) if err != nil { - return nil, fmt.Errorf("cannot parse result name for %s: %w", sf, err) + return nil, fmt.Errorf("cannot parse result name for [%s]: %w", sf, err) } f.resultName = resultName @@ -554,32 +550,14 @@ func parsePipeStats(lex *lexer) (*pipeStats, error) { } } -func parseIfFilter(lex *lexer) (filter, error) { - if !lex.isKeyword("if") { - return nil, fmt.Errorf("unexpected keyword %q; expecting 'if'", lex.token) - } - lex.nextToken() - if !lex.isKeyword("(") { - return nil, fmt.Errorf("unexpected token %q after 'if'; expecting '('", lex.token) - } - lex.nextToken() - if lex.isKeyword(")") { - lex.nextToken() - return &filterNoop{}, nil - } - f, err := parseFilter(lex) - if err != nil { - return nil, fmt.Errorf("cannot parse 'if' filter: %w", err) - } - if !lex.isKeyword(")") { - return nil, fmt.Errorf("unexpected token %q after 'if' filter; expecting ')'", lex.token) - } - lex.nextToken() - return f, nil -} - func parseStatsFunc(lex *lexer) (statsFunc, error) { switch { + case lex.isKeyword("avg"): + sas, err := parseStatsAvg(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'avg' func: %w", err) + } + return sas, nil case lex.isKeyword("count"): scs, err := parseStatsCount(lex) if err != nil { @@ -598,30 +576,54 @@ func parseStatsFunc(lex *lexer) (statsFunc, error) { return nil, fmt.Errorf("cannot parse 'count_uniq' func: %w", err) } return sus, nil - case lex.isKeyword("sum"): - sss, err := parseStatsSum(lex) + case lex.isKeyword("fields_max"): + sms, err := parseStatsFieldsMax(lex) if err != nil { - return nil, fmt.Errorf("cannot parse 'sum' func: %w", err) + return nil, fmt.Errorf("cannot parse 'fields_max' func: %w", err) } - return sss, nil + return sms, nil + case lex.isKeyword("fields_min"): + sms, err := parseStatsFieldsMin(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'fields_min' func: %w", err) + } + return sms, nil case lex.isKeyword("max"): sms, err := parseStatsMax(lex) if err != nil { return nil, fmt.Errorf("cannot parse 'max' func: %w", err) } return sms, nil + case lex.isKeyword("median"): + sms, err := parseStatsMedian(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'median' func: %w", err) + } + return sms, nil case lex.isKeyword("min"): sms, err := parseStatsMin(lex) if err != nil { return nil, fmt.Errorf("cannot parse 'min' func: %w", err) } return sms, nil - case lex.isKeyword("avg"): - sas, err := parseStatsAvg(lex) + case lex.isKeyword("quantile"): + sqs, err := parseStatsQuantile(lex) if err != nil { - return nil, fmt.Errorf("cannot parse 'avg' func: %w", err) + return nil, fmt.Errorf("cannot parse 'quantile' func: %w", err) } - return sas, nil + return sqs, nil + case lex.isKeyword("sum"): + sss, err := parseStatsSum(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'sum' func: %w", err) + } + return sss, nil + case lex.isKeyword("sum_len"): + sss, err := parseStatsSumLen(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'sum_len' func: %w", err) + } + return sss, nil case lex.isKeyword("uniq_values"): sus, err := parseStatsUniqValues(lex) if err != nil { @@ -634,24 +636,6 @@ func parseStatsFunc(lex *lexer) (statsFunc, error) { return nil, fmt.Errorf("cannot parse 'values' func: %w", err) } return svs, nil - case lex.isKeyword("sum_len"): - sss, err := parseStatsSumLen(lex) - if err != nil { - return nil, fmt.Errorf("cannot parse 'sum_len' func: %w", err) - } - return sss, nil - case lex.isKeyword("quantile"): - sqs, err := parseStatsQuantile(lex) - if err != nil { - return nil, fmt.Errorf("cannot parse 'quantile' func: %w", err) - } - return sqs, nil - case lex.isKeyword("median"): - sms, err := parseStatsMedian(lex) - if err != nil { - return nil, fmt.Errorf("cannot parse 'median' func: %w", err) - } - return sms, nil default: return nil, fmt.Errorf("unknown stats func %q", lex.token) } @@ -846,24 +830,6 @@ func tryParseBucketSize(s string) (float64, bool) { return 0, false } -// parseFieldNamesForStatsFunc parses field names for statsFunc. -// -// It returns ["*"] if the fields names list is empty or if it contains "*" field. -func parseFieldNamesForStatsFunc(lex *lexer, funcName string) ([]string, error) { - if !lex.isKeyword(funcName) { - return nil, fmt.Errorf("unexpected func; got %q; want %q", lex.token, funcName) - } - lex.nextToken() - fields, err := parseFieldNamesInParens(lex) - if err != nil { - return nil, fmt.Errorf("cannot parse %q args: %w", funcName, err) - } - if len(fields) == 0 || slices.Contains(fields, "*") { - fields = []string{"*"} - } - return fields, nil -} - func parseFieldNamesInParens(lex *lexer) ([]string, error) { if !lex.isKeyword("(") { return nil, fmt.Errorf("missing `(`") diff --git a/lib/logstorage/pipe_stats_test.go b/lib/logstorage/pipe_stats_test.go index 51837eba6..0d2cdd4c9 100644 --- a/lib/logstorage/pipe_stats_test.go +++ b/lib/logstorage/pipe_stats_test.go @@ -4,6 +4,376 @@ import ( "testing" ) +func TestParsePipeStatsSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`stats count(*) as rows`) + f(`stats by (x) count(*) as rows, count_uniq(x) as uniqs`) + f(`stats by (_time:month offset 6.5h, y) count(*) as rows, count_uniq(x) as uniqs`) + f(`stats by (_time:month offset 6.5h, y) count(*) if (q:w) as rows, count_uniq(x) as uniqs`) +} + +func TestParsePipeStatsFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`stats`) + f(`stats by`) + f(`stats foo`) + f(`stats count`) + f(`stats if (x:y)`) + f(`stats by(x) foo`) + f(`stats by(x:abc) count() rows`) + f(`stats by(x:1h offset) count () rows`) + f(`stats by(x:1h offset foo) count() rows`) +} + +func TestPipeStats(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats count(*) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"rows", "3"}, + }, + }) + + f("stats count(*) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `2`}, + {"b", `54`}, + }, + {}, + }, [][]Field{ + { + {"rows", "5"}, + }, + }) + + f("stats count(b) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"rows", "2"}, + }, + }) + + f("stats count(x) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"rows", "0"}, + }, + }) + + f("stats count(x, _msg, b) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"rows", "3"}, + }, + }) + + f("stats by (a) count(*) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"rows", "1"}, + }, + { + {"a", "2"}, + {"rows", "2"}, + }, + }) + + f("stats by (a) count(*) if (b:54) as rows", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"rows", "0"}, + }, + { + {"a", "2"}, + {"rows", "1"}, + }, + }) + + f("stats by (a, x) count(*) if (b:54) as rows_b54, count(*) as rows_total", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"x", "123"}, + }, + { + {"a", `2`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "123"}, + {"rows_b54", "0"}, + {"rows_total", "1"}, + }, + { + {"a", "2"}, + {"x", ""}, + {"rows_b54", "1"}, + {"rows_total", "2"}, + }, + }) + + f("stats by (x:1KiB) count(*) as rows", [][]Field{ + { + {"x", "1023"}, + {"_msg", "foo"}, + }, + { + {"x", "1024"}, + {"_msg", "bar"}, + }, + { + {"x", "2047"}, + {"_msg", "baz"}, + }, + }, [][]Field{ + { + {"x", "0"}, + {"rows", "1"}, + }, + { + {"x", "1024"}, + {"rows", "2"}, + }, + }) + + f("stats by (ip:/24) count(*) as rows", [][]Field{ + { + {"ip", "1.2.3.4"}, + }, + { + {"ip", "1.2.3.255"}, + }, + { + {"ip", "127.2.3.4"}, + }, + { + {"ip", "1.2.4.0"}, + }, + }, [][]Field{ + { + {"ip", "1.2.3.0"}, + {"rows", "2"}, + }, + { + {"ip", "1.2.4.0"}, + {"rows", "1"}, + }, + { + {"ip", "127.2.3.0"}, + {"rows", "1"}, + }, + }) + + f("stats by (_time:1d) count(*) as rows", [][]Field{ + { + {"_time", "2024-04-01T10:20:30Z"}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "1"}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "2"}, + {"b", `54`}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "2"}, + {"c", `xyz`}, + }, + }, [][]Field{ + { + {"_time", "2024-04-01T00:00:00Z"}, + {"rows", "1"}, + }, + { + {"_time", "2024-04-02T00:00:00Z"}, + {"rows", "3"}, + }, + }) + + f("stats by (_time:1d offset 2h) count(*) as rows", [][]Field{ + { + {"_time", "2024-04-01T00:20:30Z"}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_time", "2024-04-02T22:20:30Z"}, + {"a", "1"}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "2"}, + {"b", `54`}, + }, + { + {"_time", "2024-04-03T01:59:59.999999999Z"}, + {"a", "2"}, + {"c", `xyz`}, + }, + }, [][]Field{ + { + {"_time", "2024-03-31T02:00:00Z"}, + {"rows", "1"}, + }, + { + {"_time", "2024-04-02T02:00:00Z"}, + {"rows", "3"}, + }, + }) + + f("stats by (a, _time:1d) count(*) as rows", [][]Field{ + { + {"_time", "2024-04-01T10:20:30Z"}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "1"}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "2"}, + {"b", `54`}, + }, + { + {"_time", "2024-04-02T10:20:30Z"}, + {"a", "2"}, + {"c", `xyz`}, + }, + }, [][]Field{ + { + {"a", "2"}, + {"_time", "2024-04-01T00:00:00Z"}, + {"rows", "1"}, + }, + { + {"a", "1"}, + {"_time", "2024-04-02T00:00:00Z"}, + {"rows", "1"}, + }, + { + {"a", "2"}, + {"_time", "2024-04-02T00:00:00Z"}, + {"rows", "2"}, + }, + }) +} + func TestPipeStatsUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_topk.go b/lib/logstorage/pipe_topk.go index ea77370e1..4aa3d5bdf 100644 --- a/lib/logstorage/pipe_topk.go +++ b/lib/logstorage/pipe_topk.go @@ -258,7 +258,8 @@ func (shard *pipeTopkProcessorShard) addRow(br *blockResult, byColumns []string, r.timestamp = timestamp rows := shard.rows - if len(rows) > 0 && !topkLess(shard.ps, r, rows[0]) { + maxRows := shard.ps.offset + shard.ps.limit + if uint64(len(rows)) >= maxRows && !topkLess(shard.ps, r, rows[0]) { // Fast path - nothing to add. return } @@ -282,7 +283,7 @@ func (shard *pipeTopkProcessorShard) addRow(br *blockResult, byColumns []string, shard.stateSizeBudget -= r.sizeBytes() // Push r to shard.rows. - if uint64(len(rows)) < shard.ps.offset+shard.ps.limit { + if uint64(len(rows)) < maxRows { heap.Push(shard, r) shard.stateSizeBudget -= int(unsafe.Sizeof(r)) } else { @@ -424,8 +425,14 @@ type pipeTopkWriteContext struct { rcs []resultColumn br blockResult + // rowsWritten is the total number of rows passed to writeNextRow. rowsWritten uint64 - valuesLen int + + // rowsCount is the number of rows in the current block + rowsCount int + + // valuesLen is the total length of values in the current block + valuesLen int } func (wctx *pipeTopkWriteContext) writeNextRow(shard *pipeTopkProcessorShard) bool { @@ -457,7 +464,7 @@ func (wctx *pipeTopkWriteContext) writeNextRow(shard *pipeTopkProcessorShard) bo } } if !areEqualColumns { - // send the current block to bbBase and construct a block with new set of columns + // send the current block to ppBase and construct a block with new set of columns wctx.flush() rcs = wctx.rcs[:0] @@ -489,6 +496,7 @@ func (wctx *pipeTopkWriteContext) writeNextRow(shard *pipeTopkProcessorShard) bo wctx.valuesLen += len(v) } + wctx.rowsCount++ if wctx.valuesLen >= 1_000_000 { wctx.flush() } @@ -502,12 +510,9 @@ func (wctx *pipeTopkWriteContext) flush() { wctx.valuesLen = 0 - if len(rcs) == 0 { - return - } - // Flush rcs to ppBase - br.setResultColumns(rcs) + br.setResultColumns(rcs, wctx.rowsCount) + wctx.rowsCount = 0 wctx.ptp.ppBase.writeBlock(0, br) br.reset() for i := range rcs { diff --git a/lib/logstorage/pipe_uniq.go b/lib/logstorage/pipe_uniq.go index 6b43fb442..32b9e8f28 100644 --- a/lib/logstorage/pipe_uniq.go +++ b/lib/logstorage/pipe_uniq.go @@ -331,8 +331,13 @@ type pipeUniqWriteContext struct { rcs []resultColumn br blockResult + // rowsWritten is the total number of rows passed to writeRow. rowsWritten uint64 + // rowsCount is the number of rows in the current block + rowsCount int + + // valuesLen is the total length of values in the current block valuesLen int } @@ -354,7 +359,7 @@ func (wctx *pipeUniqWriteContext) writeRow(rowFields []Field) { } } if !areEqualColumns { - // send the current block to bbBase and construct a block with new set of columns + // send the current block to ppBase and construct a block with new set of columns wctx.flush() rcs = wctx.rcs[:0] @@ -369,6 +374,8 @@ func (wctx *pipeUniqWriteContext) writeRow(rowFields []Field) { rcs[i].addValue(v) wctx.valuesLen += len(v) } + + wctx.rowsCount++ if wctx.valuesLen >= 1_000_000 { wctx.flush() } @@ -380,12 +387,9 @@ func (wctx *pipeUniqWriteContext) flush() { wctx.valuesLen = 0 - if len(rcs) == 0 { - return - } - // Flush rcs to ppBase - br.setResultColumns(rcs) + br.setResultColumns(rcs, wctx.rowsCount) + wctx.rowsCount = 0 wctx.pup.ppBase.writeBlock(0, br) br.reset() for i := range rcs { diff --git a/lib/logstorage/pipe_uniq_test.go b/lib/logstorage/pipe_uniq_test.go index b88a2697e..1b59ca88c 100644 --- a/lib/logstorage/pipe_uniq_test.go +++ b/lib/logstorage/pipe_uniq_test.go @@ -4,6 +4,176 @@ import ( "testing" ) +func TestParsePipeUniqSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`uniq`) + f(`uniq limit 10`) + f(`uniq by (x)`) + f(`uniq by (x) limit 10`) + f(`uniq by (x, y)`) + f(`uniq by (x, y) limit 10`) +} + +func TestParsePipeUniqFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`uniq foo`) + f(`uniq by`) + f(`uniq by(x) limit`) + f(`uniq by(x) limit foo`) +} + +func TestPipeUniq(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("uniq", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }) + + f("uniq by (a)", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"a", "2"}, + }, + }) + + f("uniq by (b)", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"b", "3"}, + }, + { + {"b", "54"}, + }, + }) + + f("uniq by (c)", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"c", ""}, + }, + { + {"c", "d"}, + }, + }) + + f("uniq by (d)", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"d", ""}, + }, + }) + + f("uniq by (a, b)", [][]Field{ + { + {"a", `2`}, + {"b", `3`}, + }, + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", `2`}, + {"b", `54`}, + {"c", "d"}, + }, + }, [][]Field{ + { + {"a", "2"}, + {"b", "3"}, + }, + { + {"a", "2"}, + {"b", "54"}, + }, + }) +} + func TestPipeUniqUpdateNeededFields(t *testing.T) { f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) { t.Helper() diff --git a/lib/logstorage/pipe_unpack.go b/lib/logstorage/pipe_unpack.go new file mode 100644 index 000000000..5b4333452 --- /dev/null +++ b/lib/logstorage/pipe_unpack.go @@ -0,0 +1,258 @@ +package logstorage + +import ( + "unsafe" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" +) + +type fieldsUnpackerContext struct { + workerID uint + fieldPrefix string + + fields []Field + a arena +} + +func (uctx *fieldsUnpackerContext) reset() { + uctx.workerID = 0 + uctx.fieldPrefix = "" + uctx.resetFields() + uctx.a.reset() +} + +func (uctx *fieldsUnpackerContext) resetFields() { + clear(uctx.fields) + uctx.fields = uctx.fields[:0] +} + +func (uctx *fieldsUnpackerContext) init(workerID uint, fieldPrefix string) { + uctx.reset() + + uctx.workerID = workerID + uctx.fieldPrefix = fieldPrefix +} + +func (uctx *fieldsUnpackerContext) addField(name, value string) { + nameCopy := "" + fieldPrefix := uctx.fieldPrefix + if fieldPrefix != "" { + nameBuf := uctx.a.newBytes(len(fieldPrefix) + len(name)) + copy(nameBuf, fieldPrefix) + copy(nameBuf[len(fieldPrefix):], name) + nameCopy = bytesutil.ToUnsafeString(nameBuf) + } else { + nameCopy = uctx.a.copyString(name) + } + + valueCopy := uctx.a.copyString(value) + + uctx.fields = append(uctx.fields, Field{ + Name: nameCopy, + Value: valueCopy, + }) +} + +func newPipeUnpackProcessor(workersCount int, unpackFunc func(uctx *fieldsUnpackerContext, s string), ppBase pipeProcessor, + fromField, fieldPrefix string, iff *ifFilter) *pipeUnpackProcessor { + + return &pipeUnpackProcessor{ + unpackFunc: unpackFunc, + ppBase: ppBase, + + shards: make([]pipeUnpackProcessorShard, workersCount), + + fromField: fromField, + fieldPrefix: fieldPrefix, + iff: iff, + } +} + +type pipeUnpackProcessor struct { + unpackFunc func(uctx *fieldsUnpackerContext, s string) + ppBase pipeProcessor + + shards []pipeUnpackProcessorShard + + fromField string + fieldPrefix string + + iff *ifFilter +} + +type pipeUnpackProcessorShard struct { + pipeUnpackProcessorShardNopad + + // The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 . + _ [128 - unsafe.Sizeof(pipeUnpackProcessorShardNopad{})%128]byte +} + +type pipeUnpackProcessorShardNopad struct { + bm bitmap + + uctx fieldsUnpackerContext + wctx pipeUnpackWriteContext +} + +func (pup *pipeUnpackProcessor) writeBlock(workerID uint, br *blockResult) { + if len(br.timestamps) == 0 { + return + } + + shard := &pup.shards[workerID] + shard.wctx.init(workerID, pup.ppBase, br) + shard.uctx.init(workerID, pup.fieldPrefix) + + bm := &shard.bm + bm.init(len(br.timestamps)) + bm.setBits() + if pup.iff != nil { + pup.iff.f.applyToBlockResult(br, bm) + if bm.isZero() { + pup.ppBase.writeBlock(workerID, br) + return + } + } + + c := br.getColumnByName(pup.fromField) + if c.isConst { + v := c.valuesEncoded[0] + shard.uctx.resetFields() + pup.unpackFunc(&shard.uctx, v) + for rowIdx := range br.timestamps { + if bm.isSetBit(rowIdx) { + shard.wctx.writeRow(rowIdx, shard.uctx.fields) + } else { + shard.wctx.writeRow(rowIdx, nil) + } + } + } else { + values := c.getValues(br) + vPrevApplied := "" + for i, v := range values { + if bm.isSetBit(i) { + if vPrevApplied != v { + shard.uctx.resetFields() + pup.unpackFunc(&shard.uctx, v) + vPrevApplied = v + } + shard.wctx.writeRow(i, shard.uctx.fields) + } else { + shard.wctx.writeRow(i, nil) + } + } + } + + shard.wctx.flush() + shard.wctx.reset() + shard.uctx.reset() +} + +func (pup *pipeUnpackProcessor) flush() error { + return nil +} + +type pipeUnpackWriteContext struct { + workerID uint + ppBase pipeProcessor + + brSrc *blockResult + csSrc []*blockResultColumn + + rcs []resultColumn + br blockResult + + // rowsCount is the number of rows in the current block + rowsCount int + + // valuesLen is the total length of values in the current block + valuesLen int +} + +func (wctx *pipeUnpackWriteContext) reset() { + wctx.workerID = 0 + wctx.ppBase = nil + + wctx.brSrc = nil + wctx.csSrc = nil + + rcs := wctx.rcs + for i := range rcs { + rcs[i].reset() + } + wctx.rcs = rcs[:0] + + wctx.rowsCount = 0 + wctx.valuesLen = 0 +} + +func (wctx *pipeUnpackWriteContext) init(workerID uint, ppBase pipeProcessor, brSrc *blockResult) { + wctx.reset() + + wctx.workerID = workerID + wctx.ppBase = ppBase + + wctx.brSrc = brSrc + wctx.csSrc = brSrc.getColumns() +} + +func (wctx *pipeUnpackWriteContext) writeRow(rowIdx int, extraFields []Field) { + csSrc := wctx.csSrc + rcs := wctx.rcs + + areEqualColumns := len(rcs) == len(csSrc)+len(extraFields) + if areEqualColumns { + for i, f := range extraFields { + if rcs[len(csSrc)+i].name != f.Name { + areEqualColumns = false + break + } + } + } + if !areEqualColumns { + // send the current block to ppBase and construct a block with new set of columns + wctx.flush() + + rcs = wctx.rcs[:0] + for _, c := range csSrc { + rcs = appendResultColumnWithName(rcs, c.name) + } + for _, f := range extraFields { + rcs = appendResultColumnWithName(rcs, f.Name) + } + wctx.rcs = rcs + } + + brSrc := wctx.brSrc + for i, c := range csSrc { + v := c.getValueAtRow(brSrc, rowIdx) + rcs[i].addValue(v) + wctx.valuesLen += len(v) + } + for i, f := range extraFields { + v := f.Value + rcs[len(csSrc)+i].addValue(v) + wctx.valuesLen += len(v) + } + + wctx.rowsCount++ + if wctx.valuesLen >= 1_000_000 { + wctx.flush() + } +} + +func (wctx *pipeUnpackWriteContext) flush() { + rcs := wctx.rcs + + wctx.valuesLen = 0 + + // Flush rcs to ppBase + br := &wctx.br + br.setResultColumns(rcs, wctx.rowsCount) + wctx.rowsCount = 0 + wctx.ppBase.writeBlock(wctx.workerID, br) + br.reset() + for i := range rcs { + rcs[i].resetValues() + } +} diff --git a/lib/logstorage/pipe_unpack_json.go b/lib/logstorage/pipe_unpack_json.go index f9a44556c..790950c45 100644 --- a/lib/logstorage/pipe_unpack_json.go +++ b/lib/logstorage/pipe_unpack_json.go @@ -2,23 +2,41 @@ package logstorage import ( "fmt" - "unsafe" + "slices" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" ) // pipeUnpackJSON processes '| unpack_json ...' pipe. // // See https://docs.victoriametrics.com/victorialogs/logsql/#unpack_json-pipe type pipeUnpackJSON struct { + // fromField is the field to unpack json fields from fromField string + // fields is an optional list of fields to extract from json. + // + // if it is empty, then all the fields are extracted. + fields []string + + // resultPrefix is prefix to add to unpacked field names resultPrefix string + + // iff is an optional filter for skipping unpacking json + iff *ifFilter } func (pu *pipeUnpackJSON) String() string { s := "unpack_json" + if pu.iff != nil { + s += " " + pu.iff.String() + } if !isMsgFieldName(pu.fromField) { s += " from " + quoteTokenIfNeeded(pu.fromField) } + if len(pu.fields) > 0 { + s += " fields (" + fieldsToString(pu.fields) + ")" + } if pu.resultPrefix != "" { s += " result_prefix " + quoteTokenIfNeeded(pu.resultPrefix) } @@ -26,91 +44,84 @@ func (pu *pipeUnpackJSON) String() string { } func (pu *pipeUnpackJSON) updateNeededFields(neededFields, unneededFields fieldsSet) { + updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.iff, neededFields, unneededFields) +} + +func updateNeededFieldsForUnpackPipe(fromField string, outFields []string, iff *ifFilter, neededFields, unneededFields fieldsSet) { if neededFields.contains("*") { - unneededFields.remove(pu.fromField) + unneededFieldsOrig := unneededFields.clone() + unneededFieldsCount := 0 + if len(outFields) > 0 { + for _, f := range outFields { + if unneededFieldsOrig.contains(f) { + unneededFieldsCount++ + } + unneededFields.add(f) + } + } + if len(outFields) == 0 || unneededFieldsCount < len(outFields) { + unneededFields.remove(fromField) + if iff != nil { + unneededFields.removeFields(iff.neededFields) + } + } } else { - neededFields.add(pu.fromField) + neededFieldsOrig := neededFields.clone() + needFromField := len(outFields) == 0 + if len(outFields) > 0 { + needFromField = false + for _, f := range outFields { + if neededFieldsOrig.contains(f) { + needFromField = true + } + neededFields.remove(f) + } + } + if needFromField { + neededFields.add(fromField) + if iff != nil { + neededFields.addFields(iff.neededFields) + } + } } } func (pu *pipeUnpackJSON) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor { - shards := make([]pipeUnpackJSONProcessorShard, workersCount) - - pup := &pipeUnpackJSONProcessor{ - pu: pu, - ppBase: ppBase, - - shards: shards, - } - return pup -} - -type pipeUnpackJSONProcessor struct { - pu *pipeUnpackJSON - ppBase pipeProcessor - - shards []pipeUnpackJSONProcessorShard -} - -type pipeUnpackJSONProcessorShard struct { - pipeUnpackJSONProcessorShardNopad - - // The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 . - _ [128 - unsafe.Sizeof(pipeUnpackJSONProcessorShardNopad{})%128]byte -} - -type pipeUnpackJSONProcessorShardNopad struct { - p JSONParser - - wctx pipeUnpackWriteContext -} - -func (shard *pipeUnpackJSONProcessorShard) parseJSON(v, resultPrefix string) []Field { - if len(v) == 0 || v[0] != '{' { - // This isn't a JSON object - return nil - } - if err := shard.p.ParseLogMessageNoResetBuf(v, resultPrefix); err != nil { - // Cannot parse v - return nil - } - return shard.p.Fields -} - -func (pup *pipeUnpackJSONProcessor) writeBlock(workerID uint, br *blockResult) { - if len(br.timestamps) == 0 { - return - } - - resultPrefix := pup.pu.resultPrefix - shard := &pup.shards[workerID] - wctx := &shard.wctx - wctx.init(br, pup.ppBase) - - c := br.getColumnByName(pup.pu.fromField) - if c.isConst { - v := c.valuesEncoded[0] - extraFields := shard.parseJSON(v, resultPrefix) - for rowIdx := range br.timestamps { - wctx.writeRow(rowIdx, extraFields) + unpackJSON := func(uctx *fieldsUnpackerContext, s string) { + if len(s) == 0 || s[0] != '{' { + // This isn't a JSON object + return } - } else { - values := c.getValues(br) - var extraFields []Field - for i, v := range values { - if i == 0 || values[i-1] != v { - extraFields = shard.parseJSON(v, resultPrefix) + p := GetJSONParser() + err := p.ParseLogMessage(bytesutil.ToUnsafeBytes(s)) + if err != nil { + for _, fieldName := range pu.fields { + uctx.addField(fieldName, "") + } + } else { + if len(pu.fields) == 0 { + for _, f := range p.Fields { + uctx.addField(f.Name, f.Value) + } + } else { + for _, fieldName := range pu.fields { + addedField := false + for _, f := range p.Fields { + if f.Name == fieldName { + uctx.addField(f.Name, f.Value) + addedField = true + break + } + } + if !addedField { + uctx.addField(fieldName, "") + } + } } - wctx.writeRow(i, extraFields) } + PutJSONParser(p) } - - wctx.flush() - shard.p.reset() -} - -func (pup *pipeUnpackJSONProcessor) flush() error { - return nil + return newPipeUnpackProcessor(workersCount, unpackJSON, ppBase, pu.fromField, pu.resultPrefix, pu.iff) } func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) { @@ -119,6 +130,15 @@ func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) { } lex.nextToken() + var iff *ifFilter + if lex.isKeyword("if") { + f, err := parseIfFilter(lex) + if err != nil { + return nil, err + } + iff = f + } + fromField := "_msg" if lex.isKeyword("from") { lex.nextToken() @@ -129,6 +149,19 @@ func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) { fromField = f } + var fields []string + if lex.isKeyword("fields") { + lex.nextToken() + fs, err := parseFieldNamesInParens(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'fields': %w", err) + } + fields = fs + if slices.Contains(fields, "*") { + fields = nil + } + } + resultPrefix := "" if lex.isKeyword("result_prefix") { lex.nextToken() @@ -141,7 +174,10 @@ func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) { pu := &pipeUnpackJSON{ fromField: fromField, + fields: fields, resultPrefix: resultPrefix, + iff: iff, } + return pu, nil } diff --git a/lib/logstorage/pipe_unpack_json_test.go b/lib/logstorage/pipe_unpack_json_test.go index 2f04d5737..53650f071 100644 --- a/lib/logstorage/pipe_unpack_json_test.go +++ b/lib/logstorage/pipe_unpack_json_test.go @@ -8,12 +8,66 @@ import ( "testing" ) +func TestParsePipeUnpackJSONSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`unpack_json`) + f(`unpack_json fields (a)`) + f(`unpack_json fields (a, b, c)`) + f(`unpack_json if (a:x)`) + f(`unpack_json from x`) + f(`unpack_json from x fields (a, b)`) + f(`unpack_json if (a:x) from x fields (a, b)`) + f(`unpack_json from x result_prefix abc`) + f(`unpack_json if (a:x) from x fields (a, b) result_prefix abc`) + f(`unpack_json result_prefix abc`) + f(`unpack_json if (a:x) fields (a, b) result_prefix abc`) +} + +func TestParsePipeUnpackJSONFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`unpack_json foo`) + f(`unpack_json if`) + f(`unpack_json fields`) + f(`unpack_json fields x`) + f(`unpack_json if (x:y) foobar`) + f(`unpack_json from`) + f(`unpack_json from x y`) + f(`unpack_json from x if`) + f(`unpack_json from x result_prefix`) + f(`unpack_json from x result_prefix a b`) + f(`unpack_json from x result_prefix a if`) + f(`unpack_json result_prefix`) + f(`unpack_json result_prefix a b`) + f(`unpack_json result_prefix a if`) +} + func TestPipeUnpackJSON(t *testing.T) { f := func(pipeStr string, rows, rowsExpected [][]Field) { t.Helper() expectPipeResults(t, pipeStr, rows, rowsExpected) } + // unpack only the requested fields + f("unpack_json fields (foo, b)", [][]Field{ + { + {"_msg", `{"foo":"bar","z":"q","a":"b"}`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar","z":"q","a":"b"}`}, + {"foo", "bar"}, + {"b", ""}, + }, + }) + // single row, unpack from _msg f("unpack_json", [][]Field{ { @@ -26,6 +80,30 @@ func TestPipeUnpackJSON(t *testing.T) { }, }) + // failed if condition + f("unpack_json if (x:foo)", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"x", ""}, + }, + }) + + // matched if condition + f("unpack_json if (foo)", [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + }, + }, [][]Field{ + { + {"_msg", `{"foo":"bar"}`}, + {"foo", "bar"}, + }, + }) + // single row, unpack from _msg into _msg f("unpack_json", [][]Field{ { @@ -133,8 +211,8 @@ func TestPipeUnpackJSON(t *testing.T) { }, }) - // multiple rows with distinct number of fields with result_prefix - f("unpack_json from x result_prefix qwe_", [][]Field{ + // multiple rows with distinct number of fields with result_prefix and if condition + f("unpack_json if (y:abc) from x result_prefix qwe_", [][]Field{ { {"x", `{"foo":"bar","baz":"xyz"}`}, {"y", `abc`}, @@ -157,9 +235,9 @@ func TestPipeUnpackJSON(t *testing.T) { {"y", `abc`}, }, { + {"y", ""}, {"z", `foobar`}, {"x", `{"z":["bar",123]}`}, - {"qwe_z", `["bar",123]`}, }, }) } @@ -184,6 +262,7 @@ func expectPipeResults(t *testing.T, pipeStr string, rows, rowsExpected [][]Fiel brw.writeRow(row) } brw.flush() + pp.flush() ppTest.expectRows(t, rowsExpected) } @@ -200,6 +279,8 @@ type testBlockResultWriter struct { ppBase pipeProcessor rcs []resultColumn br blockResult + + rowsCount int } func (brw *testBlockResultWriter) writeRow(row []Field) { @@ -215,6 +296,7 @@ func (brw *testBlockResultWriter) writeRow(row []Field) { for i, field := range row { brw.rcs[i].addValue(field.Value) } + brw.rowsCount++ if rand.Intn(5) == 0 { brw.flush() } @@ -233,7 +315,8 @@ func (brw *testBlockResultWriter) areSameFields(row []Field) bool { } func (brw *testBlockResultWriter) flush() { - brw.br.setResultColumns(brw.rcs) + brw.br.setResultColumns(brw.rcs, brw.rowsCount) + brw.rowsCount = 0 workerID := rand.Intn(brw.workersCount) brw.ppBase.writeBlock(uint(workerID), &brw.br) brw.br.reset() @@ -308,35 +391,56 @@ func (pp *testPipeProcessor) expectRows(t *testing.T, expectedRows [][]Field) { } func sortTestRows(rows [][]Field) { + for _, row := range rows { + sortTestFields(row) + } slices.SortFunc(rows, func(a, b []Field) int { - reverse := -1 + reverse := false if len(a) > len(b) { - reverse = 1 + reverse = true a, b = b, a } for i, fA := range a { fB := b[i] - if fA.Name == fB.Name { - if fA.Value == fB.Value { - continue - } - if fA.Value < fB.Value { - return reverse - } - return -reverse + result := cmpTestFields(fA, fB) + if result == 0 { + continue } - if fA.Name < fB.Name { - return reverse + if reverse { + result = -result } - return -reverse + return result } if len(a) == len(b) { return 0 } - return reverse + if reverse { + return 1 + } + return -1 }) } +func sortTestFields(fields []Field) { + slices.SortFunc(fields, cmpTestFields) +} + +func cmpTestFields(a, b Field) int { + if a.Name == b.Name { + if a.Value == b.Value { + return 0 + } + if a.Value < b.Value { + return -1 + } + return 1 + } + if a.Name < b.Name { + return -1 + } + return 1 +} + func rowsToString(rows [][]Field) string { a := make([]string, len(rows)) for i, row := range rows { @@ -361,16 +465,35 @@ func TestPipeUnpackJSONUpdateNeededFields(t *testing.T) { // all the needed fields f("unpack_json from x", "*", "", "*", "") + f("unpack_json if (y:z) from x", "*", "", "*", "") + f("unpack_json if (y:z) from x fields (a, b)", "*", "", "*", "a,b") // all the needed fields, unneeded fields do not intersect with src f("unpack_json from x", "*", "f1,f2", "*", "f1,f2") + f("unpack_json if (y:z) from x", "*", "f1,f2", "*", "f1,f2") + f("unpack_json if (f1:z) from x", "*", "f1,f2", "*", "f2") + f("unpack_json if (y:z) from x fields (f3)", "*", "f1,f2", "*", "f1,f2,f3") + f("unpack_json if (y:z) from x fields (f1)", "*", "f1,f2", "*", "f1,f2") // all the needed fields, unneeded fields intersect with src f("unpack_json from x", "*", "f2,x", "*", "f2") + f("unpack_json if (y:z) from x", "*", "f2,x", "*", "f2") + f("unpack_json if (f2:z) from x", "*", "f1,f2,x", "*", "f1") + f("unpack_json if (f2:z) from x fields (f3)", "*", "f1,f2,x", "*", "f1,f3") // needed fields do not intersect with src f("unpack_json from x", "f1,f2", "", "f1,f2,x", "") + f("unpack_json if (y:z) from x", "f1,f2", "", "f1,f2,x,y", "") + f("unpack_json if (f1:z) from x", "f1,f2", "", "f1,f2,x", "") + f("unpack_json if (y:z) from x fields (f3)", "f1,f2", "", "f1,f2", "") + f("unpack_json if (y:z) from x fields (f2)", "f1,f2", "", "f1,x,y", "") + f("unpack_json if (f2:z) from x fields (f2)", "f1,f2", "", "f1,f2,x", "") // needed fields intersect with src f("unpack_json from x", "f2,x", "", "f2,x", "") + f("unpack_json if (y:z) from x", "f2,x", "", "f2,x,y", "") + f("unpack_json if (f2:z y:qwe) from x", "f2,x", "", "f2,x,y", "") + f("unpack_json if (y:z) from x fields (f1)", "f2,x", "", "f2,x", "") + f("unpack_json if (y:z) from x fields (f2)", "f2,x", "", "x,y", "") + f("unpack_json if (y:z) from x fields (x)", "f2,x", "", "f2,x,y", "") } diff --git a/lib/logstorage/pipe_unpack_logfmt.go b/lib/logstorage/pipe_unpack_logfmt.go index 37f508839..bc294a9b5 100644 --- a/lib/logstorage/pipe_unpack_logfmt.go +++ b/lib/logstorage/pipe_unpack_logfmt.go @@ -2,26 +2,39 @@ package logstorage import ( "fmt" - "strings" - "unsafe" - - "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" + "slices" ) // pipeUnpackLogfmt processes '| unpack_logfmt ...' pipe. // // See https://docs.victoriametrics.com/victorialogs/logsql/#unpack_logfmt-pipe type pipeUnpackLogfmt struct { + // fromField is the field to unpack logfmt fields from fromField string + // fields is an optional list of fields to extract from logfmt. + // + // if it is empty, then all the fields are extracted. + fields []string + + // resultPrefix is prefix to add to unpacked field names resultPrefix string + + // iff is an optional filter for skipping unpacking logfmt + iff *ifFilter } func (pu *pipeUnpackLogfmt) String() string { s := "unpack_logfmt" + if pu.iff != nil { + s += " " + pu.iff.String() + } if !isMsgFieldName(pu.fromField) { s += " from " + quoteTokenIfNeeded(pu.fromField) } + if len(pu.fields) > 0 { + s += " fields (" + fieldsToString(pu.fields) + ")" + } if pu.resultPrefix != "" { s += " result_prefix " + quoteTokenIfNeeded(pu.resultPrefix) } @@ -29,79 +42,39 @@ func (pu *pipeUnpackLogfmt) String() string { } func (pu *pipeUnpackLogfmt) updateNeededFields(neededFields, unneededFields fieldsSet) { - if neededFields.contains("*") { - unneededFields.remove(pu.fromField) - } else { - neededFields.add(pu.fromField) - } + updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.iff, neededFields, unneededFields) } func (pu *pipeUnpackLogfmt) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor { - shards := make([]pipeUnpackLogfmtProcessorShard, workersCount) + unpackLogfmt := func(uctx *fieldsUnpackerContext, s string) { + p := getLogfmtParser() - pup := &pipeUnpackLogfmtProcessor{ - pu: pu, - ppBase: ppBase, - - shards: shards, - } - return pup -} - -type pipeUnpackLogfmtProcessor struct { - pu *pipeUnpackLogfmt - ppBase pipeProcessor - - shards []pipeUnpackLogfmtProcessorShard -} - -type pipeUnpackLogfmtProcessorShard struct { - pipeUnpackLogfmtProcessorShardNopad - - // The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 . - _ [128 - unsafe.Sizeof(pipeUnpackLogfmtProcessorShardNopad{})%128]byte -} - -type pipeUnpackLogfmtProcessorShardNopad struct { - p logfmtParser - - wctx pipeUnpackWriteContext -} - -func (pup *pipeUnpackLogfmtProcessor) writeBlock(workerID uint, br *blockResult) { - if len(br.timestamps) == 0 { - return - } - - resultPrefix := pup.pu.resultPrefix - shard := &pup.shards[workerID] - wctx := &shard.wctx - wctx.init(br, pup.ppBase) - - c := br.getColumnByName(pup.pu.fromField) - if c.isConst { - v := c.valuesEncoded[0] - extraFields := shard.p.parse(v, resultPrefix) - for rowIdx := range br.timestamps { - wctx.writeRow(rowIdx, extraFields) - } - } else { - values := c.getValues(br) - var extraFields []Field - for i, v := range values { - if i == 0 || values[i-1] != v { - extraFields = shard.p.parse(v, resultPrefix) + p.parse(s) + if len(pu.fields) == 0 { + for _, f := range p.fields { + uctx.addField(f.Name, f.Value) + } + } else { + for _, fieldName := range pu.fields { + addedField := false + for _, f := range p.fields { + if f.Name == fieldName { + uctx.addField(f.Name, f.Value) + addedField = true + break + } + } + if !addedField { + uctx.addField(fieldName, "") + } } - wctx.writeRow(i, extraFields) } + + putLogfmtParser(p) } - wctx.flush() - shard.p.reset() -} + return newPipeUnpackProcessor(workersCount, unpackLogfmt, ppBase, pu.fromField, pu.resultPrefix, pu.iff) -func (pup *pipeUnpackLogfmtProcessor) flush() error { - return nil } func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) { @@ -110,6 +83,15 @@ func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) { } lex.nextToken() + var iff *ifFilter + if lex.isKeyword("if") { + f, err := parseIfFilter(lex) + if err != nil { + return nil, err + } + iff = f + } + fromField := "_msg" if lex.isKeyword("from") { lex.nextToken() @@ -120,6 +102,19 @@ func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) { fromField = f } + var fields []string + if lex.isKeyword("fields") { + lex.nextToken() + fs, err := parseFieldNamesInParens(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'fields': %w", err) + } + fields = fs + if slices.Contains(fields, "*") { + fields = nil + } + } + resultPrefix := "" if lex.isKeyword("result_prefix") { lex.nextToken() @@ -132,158 +127,10 @@ func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) { pu := &pipeUnpackLogfmt{ fromField: fromField, + fields: fields, resultPrefix: resultPrefix, + iff: iff, } + return pu, nil } - -type pipeUnpackWriteContext struct { - brSrc *blockResult - csSrc []*blockResultColumn - ppBase pipeProcessor - - rcs []resultColumn - br blockResult - - valuesLen int -} - -func (wctx *pipeUnpackWriteContext) init(brSrc *blockResult, ppBase pipeProcessor) { - wctx.brSrc = brSrc - wctx.csSrc = brSrc.getColumns() - wctx.ppBase = ppBase -} - -func (wctx *pipeUnpackWriteContext) writeRow(rowIdx int, extraFields []Field) { - csSrc := wctx.csSrc - rcs := wctx.rcs - - areEqualColumns := len(rcs) == len(csSrc)+len(extraFields) - if areEqualColumns { - for i, f := range extraFields { - if rcs[len(csSrc)+i].name != f.Name { - areEqualColumns = false - break - } - } - } - if !areEqualColumns { - // send the current block to bbBase and construct a block with new set of columns - wctx.flush() - - rcs = wctx.rcs[:0] - for _, c := range csSrc { - rcs = appendResultColumnWithName(rcs, c.name) - } - for _, f := range extraFields { - rcs = appendResultColumnWithName(rcs, f.Name) - } - wctx.rcs = rcs - } - - brSrc := wctx.brSrc - for i, c := range csSrc { - v := c.getValueAtRow(brSrc, rowIdx) - rcs[i].addValue(v) - wctx.valuesLen += len(v) - } - for i, f := range extraFields { - v := f.Value - rcs[len(csSrc)+i].addValue(v) - wctx.valuesLen += len(v) - } - if wctx.valuesLen >= 1_000_000 { - wctx.flush() - } -} - -func (wctx *pipeUnpackWriteContext) flush() { - rcs := wctx.rcs - - wctx.valuesLen = 0 - - if len(rcs) == 0 { - return - } - - // Flush rcs to ppBase - br := &wctx.br - br.setResultColumns(rcs) - wctx.ppBase.writeBlock(0, br) - br.reset() - for i := range rcs { - rcs[i].resetValues() - } -} - -type logfmtParser struct { - Fields []Field - - buf []byte -} - -func (p *logfmtParser) reset() { - clear(p.Fields) - p.Fields = p.Fields[:0] - - p.buf = p.buf[:0] -} - -func (p *logfmtParser) parse(s, resultPrefix string) []Field { - clear(p.Fields) - p.Fields = p.Fields[:0] - - for { - // Search for field name - n := strings.IndexByte(s, '=') - if n < 0 { - // field name couldn't be read - return p.Fields - } - - name := strings.TrimSpace(s[:n]) - s = s[n+1:] - if len(s) == 0 { - p.addField(name, "", resultPrefix) - return p.Fields - } - - // Search for field value - value, nOffset := tryUnquoteString(s) - if nOffset >= 0 { - p.addField(name, value, resultPrefix) - s = s[nOffset:] - if len(s) == 0 { - return p.Fields - } - if s[0] != ' ' { - return p.Fields - } - s = s[1:] - } else { - n := strings.IndexByte(s, ' ') - if n < 0 { - p.addField(name, s, resultPrefix) - return p.Fields - } - p.addField(name, s[:n], resultPrefix) - s = s[n+1:] - } - } -} - -func (p *logfmtParser) addField(name, value, resultPrefix string) { - if resultPrefix != "" { - buf := p.buf - bufLen := len(buf) - buf = append(buf, resultPrefix...) - buf = append(buf, name...) - p.buf = buf - - name = bytesutil.ToUnsafeString(buf[bufLen:]) - } - p.Fields = append(p.Fields, Field{ - Name: name, - Value: value, - }) -} diff --git a/lib/logstorage/pipe_unpack_logfmt_test.go b/lib/logstorage/pipe_unpack_logfmt_test.go index d5a267adf..a5c7426f8 100644 --- a/lib/logstorage/pipe_unpack_logfmt_test.go +++ b/lib/logstorage/pipe_unpack_logfmt_test.go @@ -4,12 +4,69 @@ import ( "testing" ) +func TestParsePipeUnpackLogfmtSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeSuccess(t, pipeStr) + } + + f(`unpack_logfmt`) + f(`unpack_logfmt fields (a, b)`) + f(`unpack_logfmt if (a:x)`) + f(`unpack_logfmt if (a:x) fields (a, b)`) + f(`unpack_logfmt from x`) + f(`unpack_logfmt from x fields (a, b)`) + f(`unpack_logfmt if (a:x) from x`) + f(`unpack_logfmt if (a:x) from x fields (a, b)`) + f(`unpack_logfmt from x result_prefix abc`) + f(`unpack_logfmt if (a:x) from x result_prefix abc`) + f(`unpack_logfmt if (a:x) from x fields (a, b) result_prefix abc`) + f(`unpack_logfmt result_prefix abc`) + f(`unpack_logfmt if (a:x) result_prefix abc`) + f(`unpack_logfmt if (a:x) fields (a, b) result_prefix abc`) +} + +func TestParsePipeUnpackLogfmtFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParsePipeFailure(t, pipeStr) + } + + f(`unpack_logfmt foo`) + f(`unpack_logfmt fields`) + f(`unpack_logfmt if`) + f(`unpack_logfmt if (x:y) foobar`) + f(`unpack_logfmt from`) + f(`unpack_logfmt from x y`) + f(`unpack_logfmt from x if`) + f(`unpack_logfmt from x result_prefix`) + f(`unpack_logfmt from x result_prefix a b`) + f(`unpack_logfmt from x result_prefix a if`) + f(`unpack_logfmt result_prefix`) + f(`unpack_logfmt result_prefix a b`) + f(`unpack_logfmt result_prefix a if`) +} + func TestPipeUnpackLogfmt(t *testing.T) { f := func(pipeStr string, rows, rowsExpected [][]Field) { t.Helper() expectPipeResults(t, pipeStr, rows, rowsExpected) } + // unpack a subset of fields + f("unpack_logfmt fields (foo, a, b)", [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + }, + }, [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + {"foo", "bar"}, + {"a", "b"}, + {"b", ""}, + }, + }) + // single row, unpack from _msg f("unpack_logfmt", [][]Field{ { @@ -24,6 +81,32 @@ func TestPipeUnpackLogfmt(t *testing.T) { }, }) + // failed if condition + f("unpack_logfmt if (foo:bar)", [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + }, + }, [][]Field{ + { + {"foo", ""}, + {"_msg", `foo=bar baz="x y=z" a=b`}, + }, + }) + + // matched if condition + f("unpack_logfmt if (foo)", [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + }, + }, [][]Field{ + { + {"_msg", `foo=bar baz="x y=z" a=b`}, + {"foo", "bar"}, + {"baz", "x y=z"}, + {"a", "b"}, + }, + }) + // single row, unpack from _msg into _msg f("unpack_logfmt", [][]Field{ { @@ -121,8 +204,8 @@ func TestPipeUnpackLogfmt(t *testing.T) { }, }) - // multiple rows with distinct number of fields, with result_prefix - f("unpack_logfmt from x result_prefix qwe_", [][]Field{ + // multiple rows with distinct number of fields, with result_prefix and if condition + f("unpack_logfmt if (y:abc) from x result_prefix qwe_", [][]Field{ { {"x", `foo=bar baz=xyz`}, {"y", `abc`}, @@ -145,9 +228,9 @@ func TestPipeUnpackLogfmt(t *testing.T) { {"y", `abc`}, }, { + {"y", ""}, {"z", `foobar`}, {"x", `z=bar`}, - {"qwe_z", `bar`}, }, }) } @@ -160,16 +243,25 @@ func TestPipeUnpackLogfmtUpdateNeededFields(t *testing.T) { // all the needed fields f("unpack_logfmt from x", "*", "", "*", "") + f("unpack_logfmt if (y:z) from x", "*", "", "*", "") // all the needed fields, unneeded fields do not intersect with src f("unpack_logfmt from x", "*", "f1,f2", "*", "f1,f2") + f("unpack_logfmt if (y:z) from x", "*", "f1,f2", "*", "f1,f2") + f("unpack_logfmt if (f1:z) from x", "*", "f1,f2", "*", "f2") // all the needed fields, unneeded fields intersect with src f("unpack_logfmt from x", "*", "f2,x", "*", "f2") + f("unpack_logfmt if (y:z) from x", "*", "f2,x", "*", "f2") + f("unpack_logfmt if (f2:z) from x", "*", "f1,f2,x", "*", "f1") // needed fields do not intersect with src f("unpack_logfmt from x", "f1,f2", "", "f1,f2,x", "") + f("unpack_logfmt if (y:z) from x", "f1,f2", "", "f1,f2,x,y", "") + f("unpack_logfmt if (f1:z) from x", "f1,f2", "", "f1,f2,x", "") // needed fields intersect with src f("unpack_logfmt from x", "f2,x", "", "f2,x", "") + f("unpack_logfmt if (y:z) from x", "f2,x", "", "f2,x,y", "") + f("unpack_logfmt if (f2:z y:qwe) from x", "f2,x", "", "f2,x,y", "") } diff --git a/lib/logstorage/rows.go b/lib/logstorage/rows.go index fe7c268fd..192205f9c 100644 --- a/lib/logstorage/rows.go +++ b/lib/logstorage/rows.go @@ -2,6 +2,7 @@ package logstorage import ( "fmt" + "strconv" "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding" @@ -24,8 +25,8 @@ func (f *Field) Reset() { // String returns string representation of f. func (f *Field) String() string { - name := getCanonicalColumnName(f.Name) - return fmt.Sprintf("%q:%q", name, f.Value) + x := f.marshalToJSON(nil) + return string(x) } func (f *Field) marshal(dst []byte) []byte { @@ -56,6 +57,27 @@ func (f *Field) unmarshal(a *arena, src []byte) ([]byte, error) { return src, nil } +func (f *Field) marshalToJSON(dst []byte) []byte { + dst = strconv.AppendQuote(dst, f.Name) + dst = append(dst, ':') + dst = strconv.AppendQuote(dst, f.Value) + return dst +} + +func marshalFieldsToJSON(dst []byte, fields []Field) []byte { + dst = append(dst, '{') + if len(fields) > 0 { + dst = fields[0].marshalToJSON(dst) + fields = fields[1:] + for i := range fields { + dst = append(dst, ',') + dst = fields[i].marshalToJSON(dst) + } + } + dst = append(dst, '}') + return dst +} + func appendFields(a *arena, dst, src []Field) []Field { for _, f := range src { dst = append(dst, Field{ @@ -126,10 +148,3 @@ func (rs *rows) mergeRows(timestampsA, timestampsB []int64, fieldsA, fieldsB [][ rs.appendRows(timestampsA, fieldsA) } } - -func getCanonicalColumnName(columnName string) string { - if columnName == "" { - return "_msg" - } - return columnName -} diff --git a/lib/logstorage/stats_avg.go b/lib/logstorage/stats_avg.go index cdf7d6477..9fcb6b1f5 100644 --- a/lib/logstorage/stats_avg.go +++ b/lib/logstorage/stats_avg.go @@ -1,22 +1,23 @@ package logstorage import ( + "fmt" "slices" "strconv" + "strings" "unsafe" ) type statsAvg struct { - fields []string - containsStar bool + fields []string } func (sa *statsAvg) String() string { - return "avg(" + fieldNamesString(sa.fields) + ")" + return "avg(" + statsFuncFieldsToString(sa.fields) + ")" } func (sa *statsAvg) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sa.fields) + updateNeededFieldsForStatsFunc(neededFields, sa.fields) } func (sa *statsAvg) newStatsProcessor() (statsProcessor, int) { @@ -34,7 +35,8 @@ type statsAvgProcessor struct { } func (sap *statsAvgProcessor) updateStatsForAllRows(br *blockResult) int { - if sap.sa.containsStar { + fields := sap.sa.fields + if len(fields) == 0 { // Scan all the columns for _, c := range br.getColumns() { f, count := c.sumValues(br) @@ -43,7 +45,7 @@ func (sap *statsAvgProcessor) updateStatsForAllRows(br *blockResult) int { } } else { // Scan the requested columns - for _, field := range sap.sa.fields { + for _, field := range fields { c := br.getColumnByName(field) f, count := c.sumValues(br) sap.sum += f @@ -54,7 +56,8 @@ func (sap *statsAvgProcessor) updateStatsForAllRows(br *blockResult) int { } func (sap *statsAvgProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { - if sap.sa.containsStar { + fields := sap.sa.fields + if len(fields) == 0 { // Scan all the fields for the given row for _, c := range br.getColumns() { f, ok := c.getFloatValueAtRow(br, rowIdx) @@ -65,7 +68,7 @@ func (sap *statsAvgProcessor) updateStatsForRow(br *blockResult, rowIdx int) int } } else { // Scan only the given fields for the given row - for _, field := range sap.sa.fields { + for _, field := range fields { c := br.getColumnByName(field) f, ok := c.getFloatValueAtRow(br, rowIdx) if ok { @@ -89,13 +92,49 @@ func (sap *statsAvgProcessor) finalizeStats() string { } func parseStatsAvg(lex *lexer) (*statsAvg, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "avg") + fields, err := parseStatsFuncFields(lex, "avg") if err != nil { return nil, err } sa := &statsAvg{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sa, nil } + +func parseStatsFuncFields(lex *lexer, funcName string) ([]string, error) { + if !lex.isKeyword(funcName) { + return nil, fmt.Errorf("unexpected func; got %q; want %q", lex.token, funcName) + } + lex.nextToken() + fields, err := parseFieldNamesInParens(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse %q args: %w", funcName, err) + } + if len(fields) == 0 || slices.Contains(fields, "*") { + fields = nil + } + return fields, nil +} + +func statsFuncFieldsToString(fields []string) string { + if len(fields) == 0 { + return "*" + } + return fieldsToString(fields) +} + +func fieldsToString(fields []string) string { + a := make([]string, len(fields)) + for i, f := range fields { + a[i] = quoteTokenIfNeeded(f) + } + return strings.Join(a, ", ") +} + +func updateNeededFieldsForStatsFunc(neededFields fieldsSet, fields []string) { + if len(fields) == 0 { + neededFields.add("*") + } + neededFields.addFields(fields) +} diff --git a/lib/logstorage/stats_avg_test.go b/lib/logstorage/stats_avg_test.go new file mode 100644 index 000000000..b4aaf5d2d --- /dev/null +++ b/lib/logstorage/stats_avg_test.go @@ -0,0 +1,412 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsAvgSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`avg(*)`) + f(`avg(a)`) + f(`avg(a, b)`) +} + +func TestParseStatsAvgFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`avg`) + f(`avg(a b)`) + f(`avg(x) y`) +} + +func TestStatsAvg(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats avg(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "12.6"}, + }, + }) + + f("stats avg(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats avg(a) as a", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"a", "2"}, + }, + }) + + f("stats avg(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "12.6"}, + }, + }) + + f("stats avg(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "28.5"}, + }, + }) + + f("stats avg(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "NaN"}, + }, + }) + + f("stats avg(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2.5"}, + }, + }) + + f("stats by (b) avg(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "1.5"}, + }, + { + {"b", ""}, + {"x", "NaN"}, + }, + }) + + f("stats by (a) avg(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "6"}, + }, + }) + + f("stats by (a) avg(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", "3"}, + {"x", "4.5"}, + }, + }) + + f("stats by (a) avg(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "NaN"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) avg(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", "3"}, + {"x", "4.5"}, + }, + }) + + f("stats by (a, b) avg(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) avg(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "NaN"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "NaN"}, + }, + }) +} + +func expectParseStatsFuncFailure(t *testing.T, s string) { + t.Helper() + + lex := newLexer(s) + sf, err := parseStatsFunc(lex) + if err == nil && lex.isEnd() { + t.Fatalf("expecting error when parsing [%s]; parsed result: [%s]", s, sf) + } +} + +func expectParseStatsFuncSuccess(t *testing.T, s string) { + t.Helper() + + lex := newLexer(s) + p, err := parseStatsFunc(lex) + if err != nil { + t.Fatalf("cannot parse [%s]: %s", s, err) + } + if !lex.isEnd() { + t.Fatalf("unexpected tail after parsing [%s]: [%s]", s, lex.s) + } + + sResult := p.String() + if sResult != s { + t.Fatalf("unexpected string representation of stats func; got\n%s\nwant\n%s", sResult, s) + } +} diff --git a/lib/logstorage/stats_count.go b/lib/logstorage/stats_count.go index 1de39031b..8b852ef0e 100644 --- a/lib/logstorage/stats_count.go +++ b/lib/logstorage/stats_count.go @@ -9,16 +9,15 @@ import ( ) type statsCount struct { - fields []string - containsStar bool + fields []string } func (sc *statsCount) String() string { - return "count(" + fieldNamesString(sc.fields) + ")" + return "count(" + statsFuncFieldsToString(sc.fields) + ")" } func (sc *statsCount) updateNeededFields(neededFields fieldsSet) { - if sc.containsStar { + if len(sc.fields) == 0 { // There is no need in fetching any columns for count(*) - the number of matching rows can be calculated as len(blockResult.timestamps) return } @@ -40,7 +39,7 @@ type statsCountProcessor struct { func (scp *statsCountProcessor) updateStatsForAllRows(br *blockResult) int { fields := scp.sc.fields - if scp.sc.containsStar { + if len(fields) == 0 { // Fast path - unconditionally count all the columns. scp.rowsCount += uint64(len(br.timestamps)) return 0 @@ -138,7 +137,7 @@ func (scp *statsCountProcessor) updateStatsForAllRows(br *blockResult) int { func (scp *statsCountProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { fields := scp.sc.fields - if scp.sc.containsStar { + if len(fields) == 0 { // Fast path - unconditionally count the given column scp.rowsCount++ return 0 @@ -200,13 +199,12 @@ func (scp *statsCountProcessor) finalizeStats() string { } func parseStatsCount(lex *lexer) (*statsCount, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "count") + fields, err := parseStatsFuncFields(lex, "count") if err != nil { return nil, err } sc := &statsCount{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sc, nil } diff --git a/lib/logstorage/stats_count_empty.go b/lib/logstorage/stats_count_empty.go index b9916ef24..69c180911 100644 --- a/lib/logstorage/stats_count_empty.go +++ b/lib/logstorage/stats_count_empty.go @@ -9,16 +9,15 @@ import ( ) type statsCountEmpty struct { - fields []string - containsStar bool + fields []string } func (sc *statsCountEmpty) String() string { - return "count_empty(" + fieldNamesString(sc.fields) + ")" + return "count_empty(" + statsFuncFieldsToString(sc.fields) + ")" } func (sc *statsCountEmpty) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sc.fields) + updateNeededFieldsForStatsFunc(neededFields, sc.fields) } func (sc *statsCountEmpty) newStatsProcessor() (statsProcessor, int) { @@ -36,7 +35,7 @@ type statsCountEmptyProcessor struct { func (scp *statsCountEmptyProcessor) updateStatsForAllRows(br *blockResult) int { fields := scp.sc.fields - if scp.sc.containsStar { + if len(fields) == 0 { bm := getBitmap(len(br.timestamps)) bm.setBits() for _, c := range br.getColumns() { @@ -96,8 +95,7 @@ func (scp *statsCountEmptyProcessor) updateStatsForAllRows(br *blockResult) int for _, f := range fields { c := br.getColumnByName(f) if c.isConst { - if c.valuesEncoded[0] == "" { - scp.rowsCount += uint64(len(br.timestamps)) + if c.valuesEncoded[0] != "" { return 0 } continue @@ -134,7 +132,7 @@ func (scp *statsCountEmptyProcessor) updateStatsForAllRows(br *blockResult) int func (scp *statsCountEmptyProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { fields := scp.sc.fields - if scp.sc.containsStar { + if len(fields) == 0 { for _, c := range br.getColumns() { if v := c.getValueAtRow(br, rowIdx); v != "" { return 0 @@ -198,13 +196,12 @@ func (scp *statsCountEmptyProcessor) finalizeStats() string { } func parseStatsCountEmpty(lex *lexer) (*statsCountEmpty, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "count_empty") + fields, err := parseStatsFuncFields(lex, "count_empty") if err != nil { return nil, err } sc := &statsCountEmpty{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sc, nil } diff --git a/lib/logstorage/stats_count_empty_test.go b/lib/logstorage/stats_count_empty_test.go new file mode 100644 index 000000000..bd9963620 --- /dev/null +++ b/lib/logstorage/stats_count_empty_test.go @@ -0,0 +1,322 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsCountEmptySuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`count_empty(*)`) + f(`count_empty(a)`) + f(`count_empty(a, b)`) +} + +func TestParseStatsCountEmptyFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`count_empty`) + f(`count_empty(a b)`) + f(`count_empty(x) y`) +} + +func TestStatsCountEmpty(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats count_empty(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats count_empty(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count_empty(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"aa", `3`}, + {"bb", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count_empty(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats count_empty(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats by (a) count_empty(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "0"}, + }, + }) + + f("stats by (a) count_empty(b) if (!c:foo) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "bar"}, + }, + { + {"a", `3`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count_empty(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + {}, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", ""}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "0"}, + }, + }) + + f("stats by (a) count_empty(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count_empty(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "0"}, + }, + }) + + f("stats by (a, b) count_empty(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"c", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "0"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "0"}, + }, + { + {"a", ""}, + {"b", "5"}, + {"x", "1"}, + }, + }) +} diff --git a/lib/logstorage/stats_count_test.go b/lib/logstorage/stats_count_test.go new file mode 100644 index 000000000..c8728e23e --- /dev/null +++ b/lib/logstorage/stats_count_test.go @@ -0,0 +1,326 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsCountSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`count(*)`) + f(`count(a)`) + f(`count(a, b)`) +} + +func TestParseStatsCountFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`count`) + f(`count(a b)`) + f(`count(x) y`) +} + +func TestStatsCount(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats count(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "4"}, + }, + }) + + f("stats count(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"aa", `3`}, + {"bb", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "0"}, + }, + }) + + f("stats count(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats by (a) count(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "2"}, + }, + }) + + f("stats by (a) count(b) if (!c:foo) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "aadf"}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "bar"}, + }, + { + {"a", `3`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + {}, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", ""}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"foo", "bar"}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", ""}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "2"}, + }, + }) + + f("stats by (a, b) count(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"c", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", ""}, + {"b", "5"}, + {"x", "0"}, + }, + }) +} diff --git a/lib/logstorage/stats_count_uniq.go b/lib/logstorage/stats_count_uniq.go index 7ad3af47f..f61d461db 100644 --- a/lib/logstorage/stats_count_uniq.go +++ b/lib/logstorage/stats_count_uniq.go @@ -2,7 +2,6 @@ package logstorage import ( "fmt" - "slices" "strconv" "unsafe" @@ -11,13 +10,12 @@ import ( ) type statsCountUniq struct { - fields []string - containsStar bool - limit uint64 + fields []string + limit uint64 } func (su *statsCountUniq) String() string { - s := "count_uniq(" + fieldNamesString(su.fields) + ")" + s := "count_uniq(" + statsFuncFieldsToString(su.fields) + ")" if su.limit > 0 { s += fmt.Sprintf(" limit %d", su.limit) } @@ -25,7 +23,7 @@ func (su *statsCountUniq) String() string { } func (su *statsCountUniq) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(su.fields) + updateNeededFieldsForStatsFunc(neededFields, su.fields) } func (su *statsCountUniq) newStatsProcessor() (statsProcessor, int) { @@ -52,17 +50,23 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { } fields := sup.su.fields - m := sup.m stateSizeIncrease := 0 - if sup.su.containsStar { + if len(fields) == 0 { // Count unique rows cs := br.getColumns() + + columnValues := sup.columnValues[:0] + for _, c := range cs { + values := c.getValues(br) + columnValues = append(columnValues, values) + } + sup.columnValues = columnValues + keyBuf := sup.keyBuf[:0] for i := range br.timestamps { seenKey := true - for _, c := range cs { - values := c.getValues(br) + for _, values := range columnValues { if i == 0 || values[i-1] != values[i] { seenKey = false break @@ -75,23 +79,20 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { allEmptyValues := true keyBuf = keyBuf[:0] - for _, c := range cs { - v := c.getValueAtRow(br, i) + for j, values := range columnValues { + v := values[i] if v != "" { allEmptyValues = false } // Put column name into key, since every block can contain different set of columns for '*' selector. - keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(c.name)) + keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(cs[j].name)) keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(v)) } if allEmptyValues { // Do not count empty values continue } - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) } sup.keyBuf = keyBuf return stateSizeIncrease @@ -112,10 +113,7 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { } keyBuf = append(keyBuf[:0], 1) keyBuf = encoding.MarshalInt64(keyBuf, timestamp) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) } sup.keyBuf = keyBuf return stateSizeIncrease @@ -130,10 +128,7 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { keyBuf := sup.keyBuf[:0] keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) sup.keyBuf = keyBuf return stateSizeIncrease } @@ -147,10 +142,7 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { } keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) } sup.keyBuf = keyBuf return stateSizeIncrease @@ -170,10 +162,7 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { } keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) } sup.keyBuf = keyBuf return stateSizeIncrease @@ -216,10 +205,7 @@ func (sup *statsCountUniqProcessor) updateStatsForAllRows(br *blockResult) int { // Do not count empty values continue } - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) } sup.keyBuf = keyBuf return stateSizeIncrease @@ -231,10 +217,9 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in } fields := sup.su.fields - m := sup.m stateSizeIncrease := 0 - if sup.su.containsStar { + if len(fields) == 0 { // Count unique rows allEmptyValues := true keyBuf := sup.keyBuf[:0] @@ -253,10 +238,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in // Do not count empty values return stateSizeIncrease } - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) return stateSizeIncrease } if len(fields) == 1 { @@ -269,10 +251,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in keyBuf := sup.keyBuf[:0] keyBuf = append(keyBuf[:0], 1) keyBuf = encoding.MarshalInt64(keyBuf, br.timestamps[rowIdx]) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) sup.keyBuf = keyBuf return stateSizeIncrease } @@ -286,10 +265,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in keyBuf := sup.keyBuf[:0] keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) sup.keyBuf = keyBuf return stateSizeIncrease } @@ -305,10 +281,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in keyBuf := sup.keyBuf[:0] keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) sup.keyBuf = keyBuf return stateSizeIncrease } @@ -322,10 +295,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in keyBuf := sup.keyBuf[:0] keyBuf = append(keyBuf[:0], 0) keyBuf = append(keyBuf, v...) - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) sup.keyBuf = keyBuf return stateSizeIncrease } @@ -347,10 +317,7 @@ func (sup *statsCountUniqProcessor) updateStatsForRow(br *blockResult, rowIdx in // Do not count empty values return stateSizeIncrease } - if _, ok := m[string(keyBuf)]; !ok { - m[string(keyBuf)] = struct{}{} - stateSizeIncrease += len(keyBuf) + int(unsafe.Sizeof("")) - } + stateSizeIncrease += sup.updateState(keyBuf) return stateSizeIncrease } @@ -376,19 +343,27 @@ func (sup *statsCountUniqProcessor) finalizeStats() string { return strconv.FormatUint(n, 10) } +func (sup *statsCountUniqProcessor) updateState(v []byte) int { + stateSizeIncrease := 0 + if _, ok := sup.m[string(v)]; !ok { + sup.m[string(v)] = struct{}{} + stateSizeIncrease += len(v) + int(unsafe.Sizeof("")) + } + return stateSizeIncrease +} + func (sup *statsCountUniqProcessor) limitReached() bool { limit := sup.su.limit return limit > 0 && uint64(len(sup.m)) >= limit } func parseStatsCountUniq(lex *lexer) (*statsCountUniq, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "count_uniq") + fields, err := parseStatsFuncFields(lex, "count_uniq") if err != nil { return nil, err } su := &statsCountUniq{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } if lex.isKeyword("limit") { lex.nextToken() diff --git a/lib/logstorage/stats_count_uniq_test.go b/lib/logstorage/stats_count_uniq_test.go new file mode 100644 index 000000000..a237d9cf5 --- /dev/null +++ b/lib/logstorage/stats_count_uniq_test.go @@ -0,0 +1,373 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsCountUniqSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`count_uniq(*)`) + f(`count_uniq(a)`) + f(`count_uniq(a, b)`) + f(`count_uniq(*) limit 10`) + f(`count_uniq(a) limit 20`) + f(`count_uniq(a, b) limit 5`) +} + +func TestParseStatsCountUniqFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`count_uniq`) + f(`count_uniq(a b)`) + f(`count_uniq(x) y`) + f(`count_uniq(x) limit`) + f(`count_uniq(x) limit N`) +} + +func TestStatsCountUniq(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats count_uniq(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats count_uniq(*) limit 2 as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count_uniq(*) limit 10 as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats count_uniq(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count_uniq(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + {}, + { + {"aa", `3`}, + {"bb", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats count_uniq(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "0"}, + }, + }) + + f("stats count_uniq(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats by (a) count_uniq(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "2"}, + }, + }) + + f("stats by (a) count_uniq(b) if (!c:foo) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "aadf"}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "bar"}, + }, + { + {"a", `3`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count_uniq(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + {}, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", ""}, + {"x", "0"}, + }, + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count_uniq(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) count_uniq(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"foo", "bar"}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "2"}, + }, + { + {"a", ""}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "2"}, + }, + }) + + f("stats by (a, b) count_uniq(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"c", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", ""}, + {"b", "5"}, + {"x", "0"}, + }, + }) +} diff --git a/lib/logstorage/stats_fields_max.go b/lib/logstorage/stats_fields_max.go new file mode 100644 index 000000000..599cf48b0 --- /dev/null +++ b/lib/logstorage/stats_fields_max.go @@ -0,0 +1,241 @@ +package logstorage + +import ( + "fmt" + "math" + "slices" + "strings" + "unsafe" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" +) + +type statsFieldsMax struct { + srcField string + + fetchFields []string +} + +func (sm *statsFieldsMax) String() string { + s := "fields_max(" + quoteTokenIfNeeded(sm.srcField) + if len(sm.fetchFields) > 0 { + s += ", " + fieldNamesString(sm.fetchFields) + } + s += ")" + return s +} + +func (sm *statsFieldsMax) updateNeededFields(neededFields fieldsSet) { + if len(sm.fetchFields) == 0 { + neededFields.add("*") + } else { + neededFields.addFields(sm.fetchFields) + } + neededFields.add(sm.srcField) +} + +func (sm *statsFieldsMax) newStatsProcessor() (statsProcessor, int) { + smp := &statsFieldsMaxProcessor{ + sm: sm, + } + return smp, int(unsafe.Sizeof(*smp)) +} + +type statsFieldsMaxProcessor struct { + sm *statsFieldsMax + + max string + + fields []Field +} + +func (smp *statsFieldsMaxProcessor) updateStatsForAllRows(br *blockResult) int { + stateSizeIncrease := 0 + + c := br.getColumnByName(smp.sm.srcField) + if c.isConst { + v := c.valuesEncoded[0] + stateSizeIncrease += smp.updateState(v, br, 0) + return stateSizeIncrease + } + if c.isTime { + bb := bbPool.Get() + bb.B = marshalTimestampRFC3339NanoString(bb.B[:0], br.timestamps[0]) + v := bytesutil.ToUnsafeString(bb.B) + stateSizeIncrease += smp.updateState(v, br, 0) + bbPool.Put(bb) + return stateSizeIncrease + } + + needUpdateState := false + switch c.valueType { + case valueTypeString: + needUpdateState = true + case valueTypeDict: + for _, v := range c.dictValues { + if smp.needUpdateStateString(v) { + needUpdateState = true + break + } + } + case valueTypeUint8, valueTypeUint16, valueTypeUint32, valueTypeUint64: + bb := bbPool.Get() + bb.B = marshalUint64String(bb.B[:0], c.maxValue) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeFloat64: + f := math.Float64frombits(c.maxValue) + bb := bbPool.Get() + bb.B = marshalFloat64String(bb.B[:0], f) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeIPv4: + bb := bbPool.Get() + bb.B = marshalIPv4String(bb.B[:0], uint32(c.maxValue)) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeTimestampISO8601: + bb := bbPool.Get() + bb.B = marshalTimestampISO8601String(bb.B[:0], int64(c.maxValue)) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + default: + logger.Panicf("BUG: unknown valueType=%d", c.valueType) + } + + if needUpdateState { + values := c.getValues(br) + for i, v := range values { + stateSizeIncrease += smp.updateState(v, br, i) + } + } + + return stateSizeIncrease +} + +func (smp *statsFieldsMaxProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { + stateSizeIncrease := 0 + + c := br.getColumnByName(smp.sm.srcField) + if c.isConst { + v := c.valuesEncoded[0] + stateSizeIncrease += smp.updateState(v, br, rowIdx) + return stateSizeIncrease + } + if c.isTime { + bb := bbPool.Get() + bb.B = marshalTimestampRFC3339NanoString(bb.B[:0], br.timestamps[rowIdx]) + v := bytesutil.ToUnsafeString(bb.B) + stateSizeIncrease += smp.updateState(v, br, rowIdx) + bbPool.Put(bb) + return stateSizeIncrease + } + + v := c.getValueAtRow(br, rowIdx) + stateSizeIncrease += smp.updateState(v, br, rowIdx) + + return stateSizeIncrease +} + +func (smp *statsFieldsMaxProcessor) mergeState(sfp statsProcessor) { + src := sfp.(*statsFieldsMaxProcessor) + if smp.needUpdateStateString(src.max) { + smp.max = src.max + smp.fields = src.fields + } +} + +func (smp *statsFieldsMaxProcessor) needUpdateStateBytes(b []byte) bool { + v := bytesutil.ToUnsafeString(b) + return smp.needUpdateStateString(v) +} + +func (smp *statsFieldsMaxProcessor) needUpdateStateString(v string) bool { + if v == "" { + return false + } + return smp.max == "" || lessString(smp.max, v) +} + +func (smp *statsFieldsMaxProcessor) updateState(v string, br *blockResult, rowIdx int) int { + stateSizeIncrease := 0 + + if !smp.needUpdateStateString(v) { + // There is no need in updating state + return stateSizeIncrease + } + + stateSizeIncrease -= len(smp.max) + stateSizeIncrease += len(v) + smp.max = strings.Clone(v) + + fields := smp.fields + for _, f := range fields { + stateSizeIncrease -= len(f.Name) + len(f.Value) + } + + clear(fields) + fields = fields[:0] + fetchFields := smp.sm.fetchFields + if len(fetchFields) == 0 { + cs := br.getColumns() + for _, c := range cs { + v := c.getValueAtRow(br, rowIdx) + fields = append(fields, Field{ + Name: strings.Clone(c.name), + Value: strings.Clone(v), + }) + stateSizeIncrease += len(c.name) + len(v) + } + } else { + for _, field := range fetchFields { + c := br.getColumnByName(field) + v := c.getValueAtRow(br, rowIdx) + fields = append(fields, Field{ + Name: strings.Clone(c.name), + Value: strings.Clone(v), + }) + stateSizeIncrease += len(c.name) + len(v) + } + } + smp.fields = fields + + return stateSizeIncrease +} + +func (smp *statsFieldsMaxProcessor) finalizeStats() string { + bb := bbPool.Get() + bb.B = marshalFieldsToJSON(bb.B, smp.fields) + result := string(bb.B) + bbPool.Put(bb) + + return result +} + +func parseStatsFieldsMax(lex *lexer) (*statsFieldsMax, error) { + if !lex.isKeyword("fields_max") { + return nil, fmt.Errorf("unexpected func; got %q; want 'fields_max'", lex.token) + } + lex.nextToken() + fields, err := parseFieldNamesInParens(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'fields_max' args: %w", err) + } + + if len(fields) == 0 { + return nil, fmt.Errorf("missing first arg for 'fields_max' func - source field") + } + + srcField := fields[0] + fetchFields := fields[1:] + if slices.Contains(fetchFields, "*") { + fetchFields = nil + } + + sm := &statsFieldsMax{ + srcField: srcField, + fetchFields: fetchFields, + } + return sm, nil +} diff --git a/lib/logstorage/stats_fields_max_test.go b/lib/logstorage/stats_fields_max_test.go new file mode 100644 index 000000000..6f1a59ce5 --- /dev/null +++ b/lib/logstorage/stats_fields_max_test.go @@ -0,0 +1,286 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsFieldsMaxSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`fields_max(foo)`) + f(`fields_max(foo, bar)`) + f(`fields_max(foo, bar, baz)`) +} + +func TestParseStatsFieldsMaxFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`fields_max`) + f(`fields_max()`) + f(`fields_max(x) bar`) +} + +func TestStatsFieldsMax(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats fields_max(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{"a":"3","b":"54"}`}, + }, + }) + + f("stats fields_max(foo) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{}`}, + }, + }) + + f("stats fields_max(b, a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", `{"a":"3"}`}, + }, + }) + + f("stats fields_max(b, a, x, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", `{"a":"3","x":"","b":"54"}`}, + }, + }) + + f("stats fields_max(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{"a":"3","b":"54"}`}, + }, + }) + + f("stats by (b) fields_max(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `-12.34`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", `{"_msg":"abc","a":"2","b":"3"}`}, + }, + { + {"b", ""}, + {"x", `{}`}, + }, + }) + + f("stats by (a) fields_max(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{"_msg":"abc","a":"1","b":"3"}`}, + }, + { + {"a", "3"}, + {"x", `{"a":"3","b":"7"}`}, + }, + }) + + f("stats by (a) fields_max(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `foo`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{}`}, + }, + { + {"a", "3"}, + {"x", `{"a":"3","c":"foo"}`}, + }, + }) + + f("stats by (a) fields_max(b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `34`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `7`}, + {"c", "bar"}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{"c":""}`}, + }, + { + {"a", "3"}, + {"x", `{"c":"bar"}`}, + }, + }) + + f("stats by (a, b) fields_max(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "4"}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", `{}`}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", `{"_msg":"def","a":"1","c":"foo","b":""}`}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", `{"a":"3","b":"5","c":"4"}`}, + }, + }) +} diff --git a/lib/logstorage/stats_fields_min.go b/lib/logstorage/stats_fields_min.go new file mode 100644 index 000000000..e57b466e8 --- /dev/null +++ b/lib/logstorage/stats_fields_min.go @@ -0,0 +1,241 @@ +package logstorage + +import ( + "fmt" + "math" + "slices" + "strings" + "unsafe" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" +) + +type statsFieldsMin struct { + srcField string + + fetchFields []string +} + +func (sm *statsFieldsMin) String() string { + s := "fields_min(" + quoteTokenIfNeeded(sm.srcField) + if len(sm.fetchFields) > 0 { + s += ", " + fieldNamesString(sm.fetchFields) + } + s += ")" + return s +} + +func (sm *statsFieldsMin) updateNeededFields(neededFields fieldsSet) { + if len(sm.fetchFields) == 0 { + neededFields.add("*") + } else { + neededFields.addFields(sm.fetchFields) + } + neededFields.add(sm.srcField) +} + +func (sm *statsFieldsMin) newStatsProcessor() (statsProcessor, int) { + smp := &statsFieldsMinProcessor{ + sm: sm, + } + return smp, int(unsafe.Sizeof(*smp)) +} + +type statsFieldsMinProcessor struct { + sm *statsFieldsMin + + min string + + fields []Field +} + +func (smp *statsFieldsMinProcessor) updateStatsForAllRows(br *blockResult) int { + stateSizeIncrease := 0 + + c := br.getColumnByName(smp.sm.srcField) + if c.isConst { + v := c.valuesEncoded[0] + stateSizeIncrease += smp.updateState(v, br, 0) + return stateSizeIncrease + } + if c.isTime { + bb := bbPool.Get() + bb.B = marshalTimestampRFC3339NanoString(bb.B[:0], br.timestamps[0]) + v := bytesutil.ToUnsafeString(bb.B) + stateSizeIncrease += smp.updateState(v, br, 0) + bbPool.Put(bb) + return stateSizeIncrease + } + + needUpdateState := false + switch c.valueType { + case valueTypeString: + needUpdateState = true + case valueTypeDict: + for _, v := range c.dictValues { + if smp.needUpdateStateString(v) { + needUpdateState = true + break + } + } + case valueTypeUint8, valueTypeUint16, valueTypeUint32, valueTypeUint64: + bb := bbPool.Get() + bb.B = marshalUint64String(bb.B[:0], c.minValue) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeFloat64: + f := math.Float64frombits(c.minValue) + bb := bbPool.Get() + bb.B = marshalFloat64String(bb.B[:0], f) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeIPv4: + bb := bbPool.Get() + bb.B = marshalIPv4String(bb.B[:0], uint32(c.minValue)) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + case valueTypeTimestampISO8601: + bb := bbPool.Get() + bb.B = marshalTimestampISO8601String(bb.B[:0], int64(c.minValue)) + needUpdateState = smp.needUpdateStateBytes(bb.B) + bbPool.Put(bb) + default: + logger.Panicf("BUG: unknown valueType=%d", c.valueType) + } + + if needUpdateState { + values := c.getValues(br) + for i, v := range values { + stateSizeIncrease += smp.updateState(v, br, i) + } + } + + return stateSizeIncrease +} + +func (smp *statsFieldsMinProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { + stateSizeIncrease := 0 + + c := br.getColumnByName(smp.sm.srcField) + if c.isConst { + v := c.valuesEncoded[0] + stateSizeIncrease += smp.updateState(v, br, rowIdx) + return stateSizeIncrease + } + if c.isTime { + bb := bbPool.Get() + bb.B = marshalTimestampRFC3339NanoString(bb.B[:0], br.timestamps[rowIdx]) + v := bytesutil.ToUnsafeString(bb.B) + stateSizeIncrease += smp.updateState(v, br, rowIdx) + bbPool.Put(bb) + return stateSizeIncrease + } + + v := c.getValueAtRow(br, rowIdx) + stateSizeIncrease += smp.updateState(v, br, rowIdx) + + return stateSizeIncrease +} + +func (smp *statsFieldsMinProcessor) mergeState(sfp statsProcessor) { + src := sfp.(*statsFieldsMinProcessor) + if smp.needUpdateStateString(src.min) { + smp.min = src.min + smp.fields = src.fields + } +} + +func (smp *statsFieldsMinProcessor) needUpdateStateBytes(b []byte) bool { + v := bytesutil.ToUnsafeString(b) + return smp.needUpdateStateString(v) +} + +func (smp *statsFieldsMinProcessor) needUpdateStateString(v string) bool { + if v == "" { + return false + } + return smp.min == "" || lessString(v, smp.min) +} + +func (smp *statsFieldsMinProcessor) updateState(v string, br *blockResult, rowIdx int) int { + stateSizeIncrease := 0 + + if !smp.needUpdateStateString(v) { + // There is no need in updating state + return stateSizeIncrease + } + + stateSizeIncrease -= len(smp.min) + stateSizeIncrease += len(v) + smp.min = strings.Clone(v) + + fields := smp.fields + for _, f := range fields { + stateSizeIncrease -= len(f.Name) + len(f.Value) + } + + clear(fields) + fields = fields[:0] + fetchFields := smp.sm.fetchFields + if len(fetchFields) == 0 { + cs := br.getColumns() + for _, c := range cs { + v := c.getValueAtRow(br, rowIdx) + fields = append(fields, Field{ + Name: strings.Clone(c.name), + Value: strings.Clone(v), + }) + stateSizeIncrease += len(c.name) + len(v) + } + } else { + for _, field := range fetchFields { + c := br.getColumnByName(field) + v := c.getValueAtRow(br, rowIdx) + fields = append(fields, Field{ + Name: strings.Clone(c.name), + Value: strings.Clone(v), + }) + stateSizeIncrease += len(c.name) + len(v) + } + } + smp.fields = fields + + return stateSizeIncrease +} + +func (smp *statsFieldsMinProcessor) finalizeStats() string { + bb := bbPool.Get() + bb.B = marshalFieldsToJSON(bb.B, smp.fields) + result := string(bb.B) + bbPool.Put(bb) + + return result +} + +func parseStatsFieldsMin(lex *lexer) (*statsFieldsMin, error) { + if !lex.isKeyword("fields_min") { + return nil, fmt.Errorf("unexpected func; got %q; want 'fields_min'", lex.token) + } + lex.nextToken() + fields, err := parseFieldNamesInParens(lex) + if err != nil { + return nil, fmt.Errorf("cannot parse 'fields_min' args: %w", err) + } + + if len(fields) == 0 { + return nil, fmt.Errorf("missing first arg for 'fields_min' func - source field") + } + + srcField := fields[0] + fetchFields := fields[1:] + if slices.Contains(fetchFields, "*") { + fetchFields = nil + } + + sm := &statsFieldsMin{ + srcField: srcField, + fetchFields: fetchFields, + } + return sm, nil +} diff --git a/lib/logstorage/stats_fields_min_test.go b/lib/logstorage/stats_fields_min_test.go new file mode 100644 index 000000000..f45d3a139 --- /dev/null +++ b/lib/logstorage/stats_fields_min_test.go @@ -0,0 +1,285 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsFieldsMinSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`fields_min(foo)`) + f(`fields_min(foo, bar)`) + f(`fields_min(foo, bar, baz)`) +} + +func TestParseStatsFieldsMinFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`fields_min`) + f(`fields_min()`) + f(`fields_min(x) bar`) +} + +func TestStatsFieldsMin(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats fields_min(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{"_msg":"def","a":"1"}`}, + }, + }) + + f("stats fields_min(foo) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{}`}, + }, + }) + + f("stats fields_min(b, a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", `{"a":"2"}`}, + }, + }) + + f("stats fields_min(b, a, x, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", `{"a":"2","x":"","b":"3"}`}, + }, + }) + + f("stats fields_min(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `{"_msg":"abc","a":"2","b":"3"}`}, + }, + }) + + f("stats by (b) fields_min(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `-12.34`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", `{"_msg":"def","a":"-12.34","b":"3"}`}, + }, + { + {"b", ""}, + {"x", `{}`}, + }, + }) + + f("stats by (a) fields_min(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{"_msg":"abc","a":"1","b":"3"}`}, + }, + { + {"a", "3"}, + {"x", `{"a":"3","b":"5"}`}, + }, + }) + + f("stats by (a) fields_min(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `foo`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{}`}, + }, + { + {"a", "3"}, + {"x", `{"a":"3","c":"foo"}`}, + }, + }) + + f("stats by (a) fields_min(b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `34`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `{"c":""}`}, + }, + { + {"a", "3"}, + {"x", `{"c":"foo"}`}, + }, + }) + + f("stats by (a, b) fields_min(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "4"}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", `{}`}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", `{"_msg":"def","a":"1","c":"foo","b":""}`}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", `{"a":"3","b":"5","c":"4"}`}, + }, + }) +} diff --git a/lib/logstorage/stats_max.go b/lib/logstorage/stats_max.go index 5eed2e748..c92cd7a75 100644 --- a/lib/logstorage/stats_max.go +++ b/lib/logstorage/stats_max.go @@ -2,7 +2,6 @@ package logstorage import ( "math" - "slices" "strings" "unsafe" @@ -11,16 +10,15 @@ import ( ) type statsMax struct { - fields []string - containsStar bool + fields []string } func (sm *statsMax) String() string { - return "max(" + fieldNamesString(sm.fields) + ")" + return "max(" + statsFuncFieldsToString(sm.fields) + ")" } func (sm *statsMax) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sm.fields) + updateNeededFieldsForStatsFunc(neededFields, sm.fields) } func (sm *statsMax) newStatsProcessor() (statsProcessor, int) { @@ -33,14 +31,13 @@ func (sm *statsMax) newStatsProcessor() (statsProcessor, int) { type statsMaxProcessor struct { sm *statsMax - max string - hasMax bool + max string } func (smp *statsMaxProcessor) updateStatsForAllRows(br *blockResult) int { maxLen := len(smp.max) - if smp.sm.containsStar { + if len(smp.sm.fields) == 0 { // Find the minimum value across all the columns for _, c := range br.getColumns() { smp.updateStateForColumn(br, c) @@ -59,7 +56,7 @@ func (smp *statsMaxProcessor) updateStatsForAllRows(br *blockResult) int { func (smp *statsMaxProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { maxLen := len(smp.max) - if smp.sm.containsStar { + if len(smp.sm.fields) == 0 { // Find the minimum value across all the fields for the given row for _, c := range br.getColumns() { v := c.getValueAtRow(br, rowIdx) @@ -79,9 +76,7 @@ func (smp *statsMaxProcessor) updateStatsForRow(br *blockResult, rowIdx int) int func (smp *statsMaxProcessor) mergeState(sfp statsProcessor) { src := sfp.(*statsMaxProcessor) - if src.hasMax { - smp.updateStateString(src.max) - } + smp.updateStateString(src.max) } func (smp *statsMaxProcessor) updateStateForColumn(br *blockResult, c *blockResultColumn) { @@ -154,28 +149,27 @@ func (smp *statsMaxProcessor) updateStateBytes(b []byte) { } func (smp *statsMaxProcessor) updateStateString(v string) { - if smp.hasMax && !lessString(smp.max, v) { + if v == "" { + // Skip empty strings + return + } + if smp.max != "" && !lessString(smp.max, v) { return } smp.max = strings.Clone(v) - smp.hasMax = true } func (smp *statsMaxProcessor) finalizeStats() string { - if !smp.hasMax { - return "NaN" - } return smp.max } func parseStatsMax(lex *lexer) (*statsMax, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "max") + fields, err := parseStatsFuncFields(lex, "max") if err != nil { return nil, err } sm := &statsMax{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sm, nil } diff --git a/lib/logstorage/stats_max_test.go b/lib/logstorage/stats_max_test.go new file mode 100644 index 000000000..b4e45a8f5 --- /dev/null +++ b/lib/logstorage/stats_max_test.go @@ -0,0 +1,366 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsMaxSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`max(*)`) + f(`max(a)`) + f(`max(a, b)`) +} + +func TestParseStatsMaxFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`max`) + f(`max(a b)`) + f(`max(x) y`) +} + +func TestStatsMax(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats max(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "def"}, + }, + }) + + f("stats max(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats max(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats max(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats max(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", ""}, + }, + }) + + f("stats max(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `3432`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats by (b) max(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "2"}, + }, + { + {"b", ""}, + {"x", ""}, + }, + }) + + f("stats by (a) max(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a) max(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "10"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "def"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a) max(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `foo`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", ""}, + }, + { + {"a", "3"}, + {"x", "foo"}, + }, + }) + + f("stats by (a) max(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `34`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "34"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a, b) max(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) max(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "4"}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", ""}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "foo"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "4"}, + }, + }) +} diff --git a/lib/logstorage/stats_median.go b/lib/logstorage/stats_median.go index baeb240e3..92b95fc02 100644 --- a/lib/logstorage/stats_median.go +++ b/lib/logstorage/stats_median.go @@ -1,30 +1,27 @@ package logstorage import ( - "slices" "unsafe" ) type statsMedian struct { - fields []string - containsStar bool + fields []string } func (sm *statsMedian) String() string { - return "median(" + fieldNamesString(sm.fields) + ")" + return "median(" + statsFuncFieldsToString(sm.fields) + ")" } func (sm *statsMedian) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sm.fields) + updateNeededFieldsForStatsFunc(neededFields, sm.fields) } func (sm *statsMedian) newStatsProcessor() (statsProcessor, int) { smp := &statsMedianProcessor{ sqp: &statsQuantileProcessor{ sq: &statsQuantile{ - fields: sm.fields, - containsStar: sm.containsStar, - phi: 0.5, + fields: sm.fields, + phi: 0.5, }, }, } @@ -53,13 +50,12 @@ func (smp *statsMedianProcessor) finalizeStats() string { } func parseStatsMedian(lex *lexer) (*statsMedian, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "median") + fields, err := parseStatsFuncFields(lex, "median") if err != nil { return nil, err } sm := &statsMedian{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sm, nil } diff --git a/lib/logstorage/stats_median_test.go b/lib/logstorage/stats_median_test.go new file mode 100644 index 000000000..93c8f3197 --- /dev/null +++ b/lib/logstorage/stats_median_test.go @@ -0,0 +1,364 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsMedianSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`median(*)`) + f(`median(a)`) + f(`median(a, b)`) +} + +func TestParseStatsMedianFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`median`) + f(`median(a b)`) + f(`median(x) y`) +} + +func TestStatsMedian(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats median(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats median(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats median(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats median(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats median(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "NaN"}, + }, + }) + + f("stats median(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats by (b) median(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "2"}, + }, + { + {"b", ""}, + {"x", "NaN"}, + }, + }) + + f("stats by (a) median(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a) median(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) median(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "NaN"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) median(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a, b) median(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) median(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "NaN"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "NaN"}, + }, + }) +} diff --git a/lib/logstorage/stats_min.go b/lib/logstorage/stats_min.go index f82d85e6b..fe9890dab 100644 --- a/lib/logstorage/stats_min.go +++ b/lib/logstorage/stats_min.go @@ -2,7 +2,6 @@ package logstorage import ( "math" - "slices" "strings" "unsafe" @@ -11,16 +10,15 @@ import ( ) type statsMin struct { - fields []string - containsStar bool + fields []string } func (sm *statsMin) String() string { - return "min(" + fieldNamesString(sm.fields) + ")" + return "min(" + statsFuncFieldsToString(sm.fields) + ")" } func (sm *statsMin) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sm.fields) + updateNeededFieldsForStatsFunc(neededFields, sm.fields) } func (sm *statsMin) newStatsProcessor() (statsProcessor, int) { @@ -33,21 +31,21 @@ func (sm *statsMin) newStatsProcessor() (statsProcessor, int) { type statsMinProcessor struct { sm *statsMin - min string - hasMin bool + min string } func (smp *statsMinProcessor) updateStatsForAllRows(br *blockResult) int { minLen := len(smp.min) - if smp.sm.containsStar { + fields := smp.sm.fields + if len(fields) == 0 { // Find the minimum value across all the columns for _, c := range br.getColumns() { smp.updateStateForColumn(br, c) } } else { // Find the minimum value across the requested columns - for _, field := range smp.sm.fields { + for _, field := range fields { c := br.getColumnByName(field) smp.updateStateForColumn(br, c) } @@ -59,7 +57,8 @@ func (smp *statsMinProcessor) updateStatsForAllRows(br *blockResult) int { func (smp *statsMinProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { minLen := len(smp.min) - if smp.sm.containsStar { + fields := smp.sm.fields + if len(fields) == 0 { // Find the minimum value across all the fields for the given row for _, c := range br.getColumns() { v := c.getValueAtRow(br, rowIdx) @@ -67,7 +66,7 @@ func (smp *statsMinProcessor) updateStatsForRow(br *blockResult, rowIdx int) int } } else { // Find the minimum value across the requested fields for the given row - for _, field := range smp.sm.fields { + for _, field := range fields { c := br.getColumnByName(field) v := c.getValueAtRow(br, rowIdx) smp.updateStateString(v) @@ -79,9 +78,7 @@ func (smp *statsMinProcessor) updateStatsForRow(br *blockResult, rowIdx int) int func (smp *statsMinProcessor) mergeState(sfp statsProcessor) { src := sfp.(*statsMinProcessor) - if src.hasMin { - smp.updateStateString(src.min) - } + smp.updateStateString(src.min) } func (smp *statsMinProcessor) updateStateForColumn(br *blockResult, c *blockResultColumn) { @@ -154,28 +151,27 @@ func (smp *statsMinProcessor) updateStateBytes(b []byte) { } func (smp *statsMinProcessor) updateStateString(v string) { - if smp.hasMin && !lessString(v, smp.min) { + if v == "" { + // Skip empty strings + return + } + if smp.min != "" && !lessString(v, smp.min) { return } smp.min = strings.Clone(v) - smp.hasMin = true } func (smp *statsMinProcessor) finalizeStats() string { - if !smp.hasMin { - return "NaN" - } return smp.min } func parseStatsMin(lex *lexer) (*statsMin, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "min") + fields, err := parseStatsFuncFields(lex, "min") if err != nil { return nil, err } sm := &statsMin{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return sm, nil } diff --git a/lib/logstorage/stats_min_test.go b/lib/logstorage/stats_min_test.go new file mode 100644 index 000000000..adda694ff --- /dev/null +++ b/lib/logstorage/stats_min_test.go @@ -0,0 +1,366 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsMinSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`min(*)`) + f(`min(a)`) + f(`min(a, b)`) +} + +func TestParseStatsMinFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`min`) + f(`min(a b)`) + f(`min(x) y`) +} + +func TestStatsMin(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats min(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats min(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats min(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + {"c", "1232"}, + }, + }, [][]Field{ + { + {"x", "1"}, + }, + }) + + f("stats min(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats min(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", ""}, + }, + }) + + f("stats min(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats by (b) min(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `-12.34`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "-12.34"}, + }, + { + {"b", ""}, + {"x", ""}, + }, + }) + + f("stats by (a) min(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) min(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "-34"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "-34"}, + }, + { + {"a", "3"}, + {"x", "3"}, + }, + }) + + f("stats by (a) min(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `foo`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", ""}, + }, + { + {"a", "3"}, + {"x", "foo"}, + }, + }) + + f("stats by (a) min(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `34`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) min(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) min(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "foo"}, + }, + { + {"a", `3`}, + {"b", `5`}, + {"c", "4"}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", ""}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "foo"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "4"}, + }, + }) +} diff --git a/lib/logstorage/stats_quantile.go b/lib/logstorage/stats_quantile.go index d7ac2352e..4bf988f28 100644 --- a/lib/logstorage/stats_quantile.go +++ b/lib/logstorage/stats_quantile.go @@ -14,18 +14,23 @@ import ( ) type statsQuantile struct { - fields []string - containsStar bool + fields []string - phi float64 + phi float64 + phiStr string } func (sq *statsQuantile) String() string { - return fmt.Sprintf("quantile(%g, %s)", sq.phi, fieldNamesString(sq.fields)) + s := "quantile(" + sq.phiStr + if len(sq.fields) > 0 { + s += ", " + fieldNamesString(sq.fields) + } + s += ")" + return s } func (sq *statsQuantile) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sq.fields) + updateNeededFieldsForStatsFunc(neededFields, sq.fields) } func (sq *statsQuantile) newStatsProcessor() (statsProcessor, int) { @@ -44,12 +49,13 @@ type statsQuantileProcessor struct { func (sqp *statsQuantileProcessor) updateStatsForAllRows(br *blockResult) int { stateSizeIncrease := 0 - if sqp.sq.containsStar { + fields := sqp.sq.fields + if len(fields) == 0 { for _, c := range br.getColumns() { stateSizeIncrease += sqp.updateStateForColumn(br, c) } } else { - for _, field := range sqp.sq.fields { + for _, field := range fields { c := br.getColumnByName(field) stateSizeIncrease += sqp.updateStateForColumn(br, c) } @@ -62,7 +68,8 @@ func (sqp *statsQuantileProcessor) updateStatsForRow(br *blockResult, rowIdx int h := &sqp.h stateSizeIncrease := 0 - if sqp.sq.containsStar { + fields := sqp.sq.fields + if len(fields) == 0 { for _, c := range br.getColumns() { f, ok := c.getFloatValueAtRow(br, rowIdx) if ok { @@ -70,7 +77,7 @@ func (sqp *statsQuantileProcessor) updateStatsForRow(br *blockResult, rowIdx int } } } else { - for _, field := range sqp.sq.fields { + for _, field := range fields { c := br.getColumnByName(field) f, ok := c.getFloatValueAtRow(br, rowIdx) if ok { @@ -181,30 +188,31 @@ func parseStatsQuantile(lex *lexer) (*statsQuantile, error) { if err != nil { return nil, fmt.Errorf("cannot parse 'quantile' args: %w", err) } - if len(fields) < 2 { - return nil, fmt.Errorf("'quantile' must have at least two args: phi and field name") + if len(fields) < 1 { + return nil, fmt.Errorf("'quantile' must have at least phi arg") } // Parse phi - phi, ok := tryParseFloat64(fields[0]) + phiStr := fields[0] + phi, ok := tryParseFloat64(phiStr) if !ok { - return nil, fmt.Errorf("phi arg in 'quantile' must be floating point number; got %q", fields[0]) + return nil, fmt.Errorf("phi arg in 'quantile' must be floating point number; got %q", phiStr) } if phi < 0 || phi > 1 { - return nil, fmt.Errorf("phi arg in 'quantile' must be in the range [0..1]; got %q", fields[0]) + return nil, fmt.Errorf("phi arg in 'quantile' must be in the range [0..1]; got %q", phiStr) } // Parse fields fields = fields[1:] if slices.Contains(fields, "*") { - fields = []string{"*"} + fields = nil } sq := &statsQuantile{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, - phi: phi, + phi: phi, + phiStr: phiStr, } return sq, nil } diff --git a/lib/logstorage/stats_quantile_test.go b/lib/logstorage/stats_quantile_test.go index f497258ad..580b6c6fb 100644 --- a/lib/logstorage/stats_quantile_test.go +++ b/lib/logstorage/stats_quantile_test.go @@ -5,6 +5,368 @@ import ( "testing" ) +func TestParseStatsQuantileSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`quantile(0.3)`) + f(`quantile(1, a)`) + f(`quantile(0.99, a, b)`) +} + +func TestParseStatsQuantileFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`quantile`) + f(`quantile(a)`) + f(`quantile(a, b)`) + f(`quantile(10, b)`) + f(`quantile(-1, b)`) + f(`quantile(0.5, b) c`) +} + +func TestStatsQuantile(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats quantile(0.9) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats quantile(0.9, a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats quantile(0.9, a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats quantile(0.9, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "54"}, + }, + }) + + f("stats quantile(0.9, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "NaN"}, + }, + }) + + f("stats quantile(0.9, a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats by (b) quantile(0.9, a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "2"}, + }, + { + {"b", ""}, + {"x", "NaN"}, + }, + }) + + f("stats by (a) quantile(0.9, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a) quantile(0.9) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a) quantile(0.9, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "NaN"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) quantile(0.9, a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "7"}, + }, + }) + + f("stats by (a, b) quantile(0.9, a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) quantile(0.9, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "NaN"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "NaN"}, + }, + }) +} + func TestHistogramQuantile(t *testing.T) { f := func(a []float64, phi, qExpected float64) { t.Helper() diff --git a/lib/logstorage/stats_sum.go b/lib/logstorage/stats_sum.go index 5ca7b082f..abab7fbc8 100644 --- a/lib/logstorage/stats_sum.go +++ b/lib/logstorage/stats_sum.go @@ -2,22 +2,20 @@ package logstorage import ( "math" - "slices" "strconv" "unsafe" ) type statsSum struct { - fields []string - containsStar bool + fields []string } func (ss *statsSum) String() string { - return "sum(" + fieldNamesString(ss.fields) + ")" + return "sum(" + statsFuncFieldsToString(ss.fields) + ")" } func (ss *statsSum) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(ss.fields) + updateNeededFieldsForStatsFunc(neededFields, ss.fields) } func (ss *statsSum) newStatsProcessor() (statsProcessor, int) { @@ -35,14 +33,15 @@ type statsSumProcessor struct { } func (ssp *statsSumProcessor) updateStatsForAllRows(br *blockResult) int { - if ssp.ss.containsStar { + fields := ssp.ss.fields + if len(fields) == 0 { // Sum all the columns for _, c := range br.getColumns() { ssp.updateStateForColumn(br, c) } } else { // Sum the requested columns - for _, field := range ssp.ss.fields { + for _, field := range fields { c := br.getColumnByName(field) ssp.updateStateForColumn(br, c) } @@ -51,7 +50,8 @@ func (ssp *statsSumProcessor) updateStatsForAllRows(br *blockResult) int { } func (ssp *statsSumProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { - if ssp.ss.containsStar { + fields := ssp.ss.fields + if len(fields) == 0 { // Sum all the fields for the given row for _, c := range br.getColumns() { f, ok := c.getFloatValueAtRow(br, rowIdx) @@ -61,7 +61,7 @@ func (ssp *statsSumProcessor) updateStatsForRow(br *blockResult, rowIdx int) int } } else { // Sum only the given fields for the given row - for _, field := range ssp.ss.fields { + for _, field := range fields { c := br.getColumnByName(field) f, ok := c.getFloatValueAtRow(br, rowIdx) if ok { @@ -89,7 +89,9 @@ func (ssp *statsSumProcessor) updateState(f float64) { func (ssp *statsSumProcessor) mergeState(sfp statsProcessor) { src := sfp.(*statsSumProcessor) - ssp.sum += src.sum + if !math.IsNaN(src.sum) { + ssp.updateState(src.sum) + } } func (ssp *statsSumProcessor) finalizeStats() string { @@ -97,13 +99,12 @@ func (ssp *statsSumProcessor) finalizeStats() string { } func parseStatsSum(lex *lexer) (*statsSum, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "sum") + fields, err := parseStatsFuncFields(lex, "sum") if err != nil { return nil, err } ss := &statsSum{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return ss, nil } diff --git a/lib/logstorage/stats_sum_len.go b/lib/logstorage/stats_sum_len.go index 0b27f45c2..fd83bc30e 100644 --- a/lib/logstorage/stats_sum_len.go +++ b/lib/logstorage/stats_sum_len.go @@ -1,22 +1,20 @@ package logstorage import ( - "slices" "strconv" "unsafe" ) type statsSumLen struct { - fields []string - containsStar bool + fields []string } func (ss *statsSumLen) String() string { - return "sum_len(" + fieldNamesString(ss.fields) + ")" + return "sum_len(" + statsFuncFieldsToString(ss.fields) + ")" } func (ss *statsSumLen) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(ss.fields) + updateNeededFieldsForStatsFunc(neededFields, ss.fields) } func (ss *statsSumLen) newStatsProcessor() (statsProcessor, int) { @@ -34,14 +32,15 @@ type statsSumLenProcessor struct { } func (ssp *statsSumLenProcessor) updateStatsForAllRows(br *blockResult) int { - if ssp.ss.containsStar { + fields := ssp.ss.fields + if len(fields) == 0 { // Sum all the columns for _, c := range br.getColumns() { ssp.sumLen += c.sumLenValues(br) } } else { // Sum the requested columns - for _, field := range ssp.ss.fields { + for _, field := range fields { c := br.getColumnByName(field) ssp.sumLen += c.sumLenValues(br) } @@ -50,7 +49,8 @@ func (ssp *statsSumLenProcessor) updateStatsForAllRows(br *blockResult) int { } func (ssp *statsSumLenProcessor) updateStatsForRow(br *blockResult, rowIdx int) int { - if ssp.ss.containsStar { + fields := ssp.ss.fields + if len(fields) == 0 { // Sum all the fields for the given row for _, c := range br.getColumns() { v := c.getValueAtRow(br, rowIdx) @@ -58,7 +58,7 @@ func (ssp *statsSumLenProcessor) updateStatsForRow(br *blockResult, rowIdx int) } } else { // Sum only the given fields for the given row - for _, field := range ssp.ss.fields { + for _, field := range fields { c := br.getColumnByName(field) v := c.getValueAtRow(br, rowIdx) ssp.sumLen += uint64(len(v)) @@ -77,13 +77,12 @@ func (ssp *statsSumLenProcessor) finalizeStats() string { } func parseStatsSumLen(lex *lexer) (*statsSumLen, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "sum_len") + fields, err := parseStatsFuncFields(lex, "sum_len") if err != nil { return nil, err } ss := &statsSumLen{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } return ss, nil } diff --git a/lib/logstorage/stats_sum_len_test.go b/lib/logstorage/stats_sum_len_test.go new file mode 100644 index 000000000..ddbbc94e7 --- /dev/null +++ b/lib/logstorage/stats_sum_len_test.go @@ -0,0 +1,364 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsSumLenSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`sum_len(*)`) + f(`sum_len(a)`) + f(`sum_len(a, b)`) +} + +func TestParseStatsSumLenFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`sum_len`) + f(`sum_len(a b)`) + f(`sum_len(x) y`) +} + +func TestStatsSumLen(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats sum_len(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `-3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "13"}, + }, + }) + + f("stats sum_len(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats sum_len(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "6"}, + }, + }) + + f("stats sum_len(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "3"}, + }, + }) + + f("stats sum_len(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "0"}, + }, + }) + + f("stats sum_len(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "2"}, + }, + }) + + f("stats by (b) sum_len(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "2"}, + }, + { + {"b", ""}, + {"x", "0"}, + }, + }) + + f("stats by (a) sum_len(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"x", "2"}, + }, + }) + + f("stats by (a) sum_len(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "10"}, + }, + { + {"a", "3"}, + {"x", "4"}, + }, + }) + + f("stats by (a) sum_len(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "0"}, + }, + { + {"a", "3"}, + {"x", "1"}, + }, + }) + + f("stats by (a) sum_len(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "4"}, + }, + { + {"a", "3"}, + {"x", "4"}, + }, + }) + + f("stats by (a, b) sum_len(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "1"}, + }, + }) + + f("stats by (a, b) sum_len(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "0"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "0"}, + }, + }) +} diff --git a/lib/logstorage/stats_sum_test.go b/lib/logstorage/stats_sum_test.go new file mode 100644 index 000000000..e22345968 --- /dev/null +++ b/lib/logstorage/stats_sum_test.go @@ -0,0 +1,364 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsSumSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`sum(*)`) + f(`sum(a)`) + f(`sum(a, b)`) +} + +func TestParseStatsSumFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`sum`) + f(`sum(a b)`) + f(`sum(x) y`) +} + +func TestStatsSum(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats sum(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `-3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "57"}, + }, + }) + + f("stats sum(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "6"}, + }, + }) + + f("stats sum(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "63"}, + }, + }) + + f("stats sum(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "57"}, + }, + }) + + f("stats sum(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "NaN"}, + }, + }) + + f("stats sum(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", "5"}, + }, + }) + + f("stats by (b) sum(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", "3"}, + }, + { + {"b", ""}, + {"x", "NaN"}, + }, + }) + + f("stats by (a) sum(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"x", "12"}, + }, + }) + + f("stats by (a) sum(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "8"}, + }, + { + {"a", "3"}, + {"x", "18"}, + }, + }) + + f("stats by (a) sum(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "NaN"}, + }, + { + {"a", "3"}, + {"x", "5"}, + }, + }) + + f("stats by (a) sum(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", "8"}, + }, + { + {"a", "3"}, + {"x", "18"}, + }, + }) + + f("stats by (a, b) sum(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "1"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "1"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "3"}, + }, + }) + + f("stats by (a, b) sum(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", "NaN"}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", "3"}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", "NaN"}, + }, + }) +} diff --git a/lib/logstorage/stats_uniq_values.go b/lib/logstorage/stats_uniq_values.go index 4942f1213..4e07ab4fb 100644 --- a/lib/logstorage/stats_uniq_values.go +++ b/lib/logstorage/stats_uniq_values.go @@ -11,13 +11,12 @@ import ( ) type statsUniqValues struct { - fields []string - containsStar bool - limit uint64 + fields []string + limit uint64 } func (su *statsUniqValues) String() string { - s := "uniq_values(" + fieldNamesString(su.fields) + ")" + s := "uniq_values(" + statsFuncFieldsToString(su.fields) + ")" if su.limit > 0 { s += fmt.Sprintf(" limit %d", su.limit) } @@ -25,7 +24,7 @@ func (su *statsUniqValues) String() string { } func (su *statsUniqValues) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(su.fields) + updateNeededFieldsForStatsFunc(neededFields, su.fields) } func (su *statsUniqValues) newStatsProcessor() (statsProcessor, int) { @@ -50,12 +49,13 @@ func (sup *statsUniqValuesProcessor) updateStatsForAllRows(br *blockResult) int } stateSizeIncrease := 0 - if sup.su.containsStar { + fields := sup.su.fields + if len(fields) == 0 { for _, c := range br.getColumns() { stateSizeIncrease += sup.updateStatsForAllRowsColumn(c, br) } } else { - for _, field := range sup.su.fields { + for _, field := range fields { c := br.getColumnByName(field) stateSizeIncrease += sup.updateStatsForAllRowsColumn(c, br) } @@ -64,7 +64,6 @@ func (sup *statsUniqValuesProcessor) updateStatsForAllRows(br *blockResult) int } func (sup *statsUniqValuesProcessor) updateStatsForAllRowsColumn(c *blockResultColumn, br *blockResult) int { - m := sup.m stateSizeIncrease := 0 if c.isConst { // collect unique const values @@ -73,11 +72,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForAllRowsColumn(c *blockResultC // skip empty values return stateSizeIncrease } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) return stateSizeIncrease } if c.valueType == valueTypeDict { @@ -87,11 +82,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForAllRowsColumn(c *blockResultC // skip empty values continue } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) } return stateSizeIncrease } @@ -107,11 +98,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForAllRowsColumn(c *blockResultC // This value has been already counted. continue } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) } return stateSizeIncrease } @@ -123,12 +110,13 @@ func (sup *statsUniqValuesProcessor) updateStatsForRow(br *blockResult, rowIdx i } stateSizeIncrease := 0 - if sup.su.containsStar { + fields := sup.su.fields + if len(fields) == 0 { for _, c := range br.getColumns() { stateSizeIncrease += sup.updateStatsForRowColumn(c, br, rowIdx) } } else { - for _, field := range sup.su.fields { + for _, field := range fields { c := br.getColumnByName(field) stateSizeIncrease += sup.updateStatsForRowColumn(c, br, rowIdx) } @@ -137,7 +125,6 @@ func (sup *statsUniqValuesProcessor) updateStatsForRow(br *blockResult, rowIdx i } func (sup *statsUniqValuesProcessor) updateStatsForRowColumn(c *blockResultColumn, br *blockResult, rowIdx int) int { - m := sup.m stateSizeIncrease := 0 if c.isConst { // collect unique const values @@ -146,11 +133,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForRowColumn(c *blockResultColum // skip empty values return stateSizeIncrease } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) return stateSizeIncrease } if c.valueType == valueTypeDict { @@ -162,11 +145,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForRowColumn(c *blockResultColum // skip empty values return stateSizeIncrease } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) return stateSizeIncrease } @@ -176,11 +155,7 @@ func (sup *statsUniqValuesProcessor) updateStatsForRowColumn(c *blockResultColum // skip empty values return stateSizeIncrease } - if _, ok := m[v]; !ok { - vCopy := strings.Clone(v) - m[vCopy] = struct{}{} - stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) - } + stateSizeIncrease += sup.updateState(v) return stateSizeIncrease } @@ -190,10 +165,9 @@ func (sup *statsUniqValuesProcessor) mergeState(sfp statsProcessor) { } src := sfp.(*statsUniqValuesProcessor) - m := sup.m for k := range src.m { - if _, ok := m[k]; !ok { - m[k] = struct{}{} + if _, ok := sup.m[k]; !ok { + sup.m[k] = struct{}{} } } } @@ -207,6 +181,7 @@ func (sup *statsUniqValuesProcessor) finalizeStats() string { for k := range sup.m { items = append(items, k) } + sortStrings(items) if limit := sup.su.limit; limit > 0 && uint64(len(items)) > limit { items = items[:limit] @@ -215,6 +190,28 @@ func (sup *statsUniqValuesProcessor) finalizeStats() string { return marshalJSONArray(items) } +func sortStrings(a []string) { + slices.SortFunc(a, func(x, y string) int { + if x == y { + return 0 + } + if lessString(x, y) { + return -1 + } + return 1 + }) +} + +func (sup *statsUniqValuesProcessor) updateState(v string) int { + stateSizeIncrease := 0 + if _, ok := sup.m[v]; !ok { + vCopy := strings.Clone(v) + sup.m[vCopy] = struct{}{} + stateSizeIncrease += len(vCopy) + int(unsafe.Sizeof(vCopy)) + } + return stateSizeIncrease +} + func (sup *statsUniqValuesProcessor) limitReached() bool { limit := sup.su.limit return limit > 0 && uint64(len(sup.m)) >= limit @@ -242,13 +239,12 @@ func marshalJSONArray(items []string) string { } func parseStatsUniqValues(lex *lexer) (*statsUniqValues, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "uniq_values") + fields, err := parseStatsFuncFields(lex, "uniq_values") if err != nil { return nil, err } su := &statsUniqValues{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } if lex.isKeyword("limit") { lex.nextToken() diff --git a/lib/logstorage/stats_uniq_values_test.go b/lib/logstorage/stats_uniq_values_test.go new file mode 100644 index 000000000..70d25d200 --- /dev/null +++ b/lib/logstorage/stats_uniq_values_test.go @@ -0,0 +1,417 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsUniqValuesSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`uniq_values(*)`) + f(`uniq_values(a)`) + f(`uniq_values(a, b)`) + f(`uniq_values(a, b) limit 10`) +} + +func TestParseStatsUniqValuesFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`uniq_values`) + f(`uniq_values(a b)`) + f(`uniq_values(x) y`) + f(`uniq_values(x) limit`) + f(`uniq_values(x) limit N`) +} + +func TestStatsUniqValues(t *testing.T) { + f := func(pipeStr string, rows, rowsExpected [][]Field) { + t.Helper() + expectPipeResults(t, pipeStr, rows, rowsExpected) + } + + f("stats uniq_values(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `-3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["-3","1","2","3","54","abc","def"]`}, + }, + }) + + f("stats uniq_values(*) limit 1999 as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `-3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["-3","1","2","3","54","abc","def"]`}, + }, + }) + + f("stats uniq_values(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["1","2","3"]`}, + }, + }) + + f("stats uniq_values(a, b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["1","2","3","54"]`}, + }, + }) + + f("stats uniq_values(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["3","54"]`}, + }, + }) + + f("stats uniq_values(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `[]`}, + }, + }) + + f("stats uniq_values(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `54`}, + }, + }, [][]Field{ + { + {"x", `["2","3"]`}, + }, + }) + + f("stats by (b) uniq_values(a) if (b:*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `2`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"b", "3"}, + }, + { + {"a", `3`}, + {"c", `54`}, + }, + }, [][]Field{ + { + {"b", "3"}, + {"x", `["1","2"]`}, + }, + { + {"b", ""}, + {"x", `[]`}, + }, + }) + + f("stats by (a) uniq_values(b) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `["3"]`}, + }, + { + {"a", "3"}, + {"x", `["5","7"]`}, + }, + }) + + f("stats by (a) uniq_values(*) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `["1","3","abc","def"]`}, + }, + { + {"a", "3"}, + {"x", `["3","5","7"]`}, + }, + }) + + f("stats by (a) uniq_values(*) limit 100 as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `["1","3","abc","def"]`}, + }, + { + {"a", "3"}, + {"x", `["3","5","7"]`}, + }, + }) + + f("stats by (a) uniq_values(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + }, + { + {"a", `3`}, + {"c", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `[]`}, + }, + { + {"a", "3"}, + {"x", `["5"]`}, + }, + }) + + f("stats by (a) uniq_values(a, b, c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + { + {"a", `3`}, + {"b", `7`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"x", `["1","3"]`}, + }, + { + {"a", "3"}, + {"x", `["3","5","7"]`}, + }, + }) + + f("stats by (a, b) uniq_values(a) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", `["1"]`}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", `["1"]`}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", `["3"]`}, + }, + }) + + f("stats by (a, b) uniq_values(c) as x", [][]Field{ + { + {"_msg", `abc`}, + {"a", `1`}, + {"b", `3`}, + }, + { + {"_msg", `def`}, + {"a", `1`}, + {"c", "3"}, + }, + { + {"a", `3`}, + {"b", `5`}, + }, + }, [][]Field{ + { + {"a", "1"}, + {"b", "3"}, + {"x", `[]`}, + }, + { + {"a", "1"}, + {"b", ""}, + {"x", `["3"]`}, + }, + { + {"a", "3"}, + {"b", "5"}, + {"x", `[]`}, + }, + }) +} diff --git a/lib/logstorage/stats_values.go b/lib/logstorage/stats_values.go index dd57a6ddc..ceab48998 100644 --- a/lib/logstorage/stats_values.go +++ b/lib/logstorage/stats_values.go @@ -2,19 +2,17 @@ package logstorage import ( "fmt" - "slices" "strings" "unsafe" ) type statsValues struct { - fields []string - containsStar bool - limit uint64 + fields []string + limit uint64 } func (sv *statsValues) String() string { - s := "values(" + fieldNamesString(sv.fields) + ")" + s := "values(" + statsFuncFieldsToString(sv.fields) + ")" if sv.limit > 0 { s += fmt.Sprintf(" limit %d", sv.limit) } @@ -22,7 +20,7 @@ func (sv *statsValues) String() string { } func (sv *statsValues) updateNeededFields(neededFields fieldsSet) { - neededFields.addFields(sv.fields) + updateNeededFieldsForStatsFunc(neededFields, sv.fields) } func (sv *statsValues) newStatsProcessor() (statsProcessor, int) { @@ -45,12 +43,13 @@ func (svp *statsValuesProcessor) updateStatsForAllRows(br *blockResult) int { } stateSizeIncrease := 0 - if svp.sv.containsStar { + fields := svp.sv.fields + if len(fields) == 0 { for _, c := range br.getColumns() { stateSizeIncrease += svp.updateStatsForAllRowsColumn(c, br) } } else { - for _, field := range svp.sv.fields { + for _, field := range fields { c := br.getColumnByName(field) stateSizeIncrease += svp.updateStatsForAllRowsColumn(c, br) } @@ -112,12 +111,13 @@ func (svp *statsValuesProcessor) updateStatsForRow(br *blockResult, rowIdx int) } stateSizeIncrease := 0 - if svp.sv.containsStar { + fields := svp.sv.fields + if len(fields) == 0 { for _, c := range br.getColumns() { stateSizeIncrease += svp.updateStatsForRowColumn(c, br, rowIdx) } } else { - for _, field := range svp.sv.fields { + for _, field := range fields { c := br.getColumnByName(field) stateSizeIncrease += svp.updateStatsForRowColumn(c, br, rowIdx) } @@ -188,13 +188,12 @@ func (svp *statsValuesProcessor) limitReached() bool { } func parseStatsValues(lex *lexer) (*statsValues, error) { - fields, err := parseFieldNamesForStatsFunc(lex, "values") + fields, err := parseStatsFuncFields(lex, "values") if err != nil { return nil, err } sv := &statsValues{ - fields: fields, - containsStar: slices.Contains(fields, "*"), + fields: fields, } if lex.isKeyword("limit") { lex.nextToken() diff --git a/lib/logstorage/stats_values_test.go b/lib/logstorage/stats_values_test.go new file mode 100644 index 000000000..34b621db3 --- /dev/null +++ b/lib/logstorage/stats_values_test.go @@ -0,0 +1,30 @@ +package logstorage + +import ( + "testing" +) + +func TestParseStatsValuesSuccess(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncSuccess(t, pipeStr) + } + + f(`values(*)`) + f(`values(a)`) + f(`values(a, b)`) + f(`values(a, b) limit 10`) +} + +func TestParseStatsValuesFailure(t *testing.T) { + f := func(pipeStr string) { + t.Helper() + expectParseStatsFuncFailure(t, pipeStr) + } + + f(`values`) + f(`values(a b)`) + f(`values(x) y`) + f(`values(a, b) limit`) + f(`values(a, b) limit foo`) +} diff --git a/lib/logstorage/storage_search.go b/lib/logstorage/storage_search.go index 894c738f2..69be30c46 100644 --- a/lib/logstorage/storage_search.go +++ b/lib/logstorage/storage_search.go @@ -146,17 +146,30 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query, // GetFieldNames returns field names from q results for the given tenantIDs. func (s *Storage) GetFieldNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) { - // add `field_names ...` to the end of q.pipes pipes := append([]pipe{}, q.pipes...) - - pipeStr := "field_names as names" + pipeStr := "field_names as names | sort by (names)" lex := newLexer(pipeStr) + pf, err := parsePipeFieldNames(lex) if err != nil { - logger.Panicf("BUG: unexpected error when parsing 'field_names' pipe: %s", err) + logger.Panicf("BUG: unexpected error when parsing 'field_names' pipe at [%s]: %s", pipeStr, err) } pf.isFirstPipe = len(pipes) == 0 - pipes = append(pipes, pf) + + if !lex.isKeyword("|") { + logger.Panicf("BUG: unexpected token after 'field_names' pipe at [%s]: %q", pipeStr, lex.token) + } + lex.nextToken() + + ps, err := parsePipeSort(lex) + if err != nil { + logger.Panicf("BUG: unexpected error when parsing 'sort' pipe at [%s]: %s", pipeStr, err) + } + if !lex.isEnd() { + logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s) + } + + pipes = append(pipes, pf, ps) q = &Query{ f: q.f, @@ -168,39 +181,96 @@ func (s *Storage) GetFieldNames(ctx context.Context, tenantIDs []TenantID, q *Qu // GetFieldValues returns unique values for the given fieldName returned by q for the given tenantIDs. // -// If limit > 0, then up to limit unique values are returned. The values are returned in arbitrary order because of performance reasons. -// The caller may sort the returned values if needed. +// If limit > 0, then up to limit unique values are returned. func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]string, error) { - // add 'uniq fieldName' to the end of q.pipes - if !endsWithPipeUniqSingleField(q.pipes, fieldName) { - pipes := append([]pipe{}, q.pipes...) + pipes := append([]pipe{}, q.pipes...) + quotedFieldName := quoteTokenIfNeeded(fieldName) + pipeStr := fmt.Sprintf("uniq by (%s) limit %d | sort by (%s)", quotedFieldName, limit, quotedFieldName) + lex := newLexer(pipeStr) - pipeStr := fmt.Sprintf("uniq by (%s) limit %d", quoteTokenIfNeeded(fieldName), limit) - lex := newLexer(pipeStr) - pu, err := parsePipeUniq(lex) - if err != nil { - logger.Panicf("BUG: unexpected error when parsing 'uniq' pipe: %s", err) - } - pipes = append(pipes, pu) + pu, err := parsePipeUniq(lex) + if err != nil { + logger.Panicf("BUG: unexpected error when parsing 'uniq' pipe at [%s]: %s", pipeStr, err) + } - q = &Query{ - f: q.f, - pipes: pipes, - } + if !lex.isKeyword("|") { + logger.Panicf("BUG: unexpected token after 'uniq' pipe at [%s]: %q", pipeStr, lex.token) + } + lex.nextToken() + + ps, err := parsePipeSort(lex) + if err != nil { + logger.Panicf("BUG: unexpected error when parsing 'sort' pipe at [%s]: %s", pipeStr, err) + } + if !lex.isEnd() { + logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s) + } + + pipes = append(pipes, pu, ps) + + q = &Query{ + f: q.f, + pipes: pipes, } return s.runSingleColumnQuery(ctx, tenantIDs, q) } -func endsWithPipeUniqSingleField(pipes []pipe, fieldName string) bool { - if len(pipes) == 0 { - return false +// GetStreamLabelNames returns stream label names from q results for the given tenantIDs. +func (s *Storage) GetStreamLabelNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) { + streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64) + if err != nil { + return nil, err } - pu, ok := pipes[len(pipes)-1].(*pipeUniq) - if !ok { - return false + + var names []string + m := make(map[string]struct{}) + forEachStreamLabel(streams, func(label Field) { + if _, ok := m[label.Name]; !ok { + nameCopy := strings.Clone(label.Name) + names = append(names, nameCopy) + m[nameCopy] = struct{}{} + } + }) + sortStrings(names) + + return names, nil +} + +// GetStreamLabelValues returns stream label values for the given labelName from q results for the given tenantIDs. +// +// If limit > 9, then up to limit unique label values are returned. +func (s *Storage) GetStreamLabelValues(ctx context.Context, tenantIDs []TenantID, q *Query, labelName string, limit uint64) ([]string, error) { + streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64) + if err != nil { + return nil, err } - return len(pu.byFields) == 1 && pu.byFields[0] == fieldName + + var values []string + m := make(map[string]struct{}) + forEachStreamLabel(streams, func(label Field) { + if label.Name != labelName { + return + } + if _, ok := m[label.Value]; !ok { + valueCopy := strings.Clone(label.Value) + values = append(values, valueCopy) + m[valueCopy] = struct{}{} + } + }) + if uint64(len(values)) > limit { + values = values[:limit] + } + sortStrings(values) + + return values, nil +} + +// GetStreams returns streams from q results for the given tenantIDs. +// +// If limit > 0, then up to limit unique streams are returned. +func (s *Storage) GetStreams(ctx context.Context, tenantIDs []TenantID, q *Query, limit uint64) ([]string, error) { + return s.GetFieldValues(ctx, tenantIDs, q, "_stream", limit) } func (s *Storage) runSingleColumnQuery(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) { @@ -259,7 +329,17 @@ func (s *Storage) initFilterInValues(ctx context.Context, tenantIDs []TenantID, return qNew, nil } +func (iff *ifFilter) hasFilterInWithQuery() bool { + if iff == nil { + return false + } + return hasFilterInWithQueryForFilter(iff.f) +} + func hasFilterInWithQueryForFilter(f filter) bool { + if f == nil { + return false + } visitFunc := func(f filter) bool { fi, ok := f.(*filterIn) return ok && fi.needExecuteQuery @@ -269,12 +349,27 @@ func hasFilterInWithQueryForFilter(f filter) bool { func hasFilterInWithQueryForPipes(pipes []pipe) bool { for _, p := range pipes { - ps, ok := p.(*pipeStats) - if !ok { - continue - } - for _, f := range ps.funcs { - if f.iff != nil && hasFilterInWithQueryForFilter(f.iff) { + switch t := p.(type) { + case *pipeStats: + for _, f := range t.funcs { + if f.iff.hasFilterInWithQuery() { + return true + } + } + case *pipeFormat: + if t.iff.hasFilterInWithQuery() { + return true + } + case *pipeExtract: + if t.iff.hasFilterInWithQuery() { + return true + } + case *pipeUnpackJSON: + if t.iff.hasFilterInWithQuery() { + return true + } + case *pipeUnpackLogfmt: + if t.iff.hasFilterInWithQuery() { return true } } @@ -284,7 +379,26 @@ func hasFilterInWithQueryForPipes(pipes []pipe) bool { type getFieldValuesFunc func(q *Query, fieldName string) ([]string, error) +func (iff *ifFilter) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (*ifFilter, error) { + if iff == nil { + return nil, nil + } + + f, err := initFilterInValuesForFilter(cache, iff.f, getFieldValuesFunc) + if err != nil { + return nil, err + } + + iffNew := *iff + iffNew.f = f + return &iffNew, nil +} + func initFilterInValuesForFilter(cache map[string][]string, f filter, getFieldValuesFunc getFieldValuesFunc) (filter, error) { + if f == nil { + return nil, nil + } + visitFunc := func(f filter) bool { fi, ok := f.(*filterIn) return ok && fi.needExecuteQuery @@ -320,19 +434,49 @@ func initFilterInValuesForPipes(cache map[string][]string, pipes []pipe, getFiel case *pipeStats: funcsNew := make([]pipeStatsFunc, len(t.funcs)) for j, f := range t.funcs { - if f.iff != nil { - fNew, err := initFilterInValuesForFilter(cache, f.iff, getFieldValuesFunc) - if err != nil { - return nil, err - } - f.iff = fNew + iffNew, err := f.iff.initFilterInValues(cache, getFieldValuesFunc) + if err != nil { + return nil, err } + f.iff = iffNew funcsNew[j] = f } pipesNew[i] = &pipeStats{ byFields: t.byFields, funcs: funcsNew, } + case *pipeFormat: + iffNew, err := t.iff.initFilterInValues(cache, getFieldValuesFunc) + if err != nil { + return nil, err + } + pf := *t + pf.iff = iffNew + pipesNew[i] = &pf + case *pipeExtract: + iffNew, err := t.iff.initFilterInValues(cache, getFieldValuesFunc) + if err != nil { + return nil, err + } + pe := *t + pe.iff = iffNew + pipesNew[i] = &pe + case *pipeUnpackJSON: + iffNew, err := t.iff.initFilterInValues(cache, getFieldValuesFunc) + if err != nil { + return nil, err + } + pu := *t + pu.iff = iffNew + pipesNew[i] = &pu + case *pipeUnpackLogfmt: + iffNew, err := t.iff.initFilterInValues(cache, getFieldValuesFunc) + if err != nil { + return nil, err + } + pu := *t + pu.iff = iffNew + pipesNew[i] = &pu default: pipesNew[i] = p } @@ -862,3 +1006,59 @@ func getFilterTimeRange(f filter) (int64, int64) { } return math.MinInt64, math.MaxInt64 } + +func forEachStreamLabel(streams []string, f func(label Field)) { + var labels []Field + for _, stream := range streams { + var err error + labels, err = parseStreamLabels(labels[:0], stream) + if err != nil { + continue + } + for i := range labels { + f(labels[i]) + } + } +} + +func parseStreamLabels(dst []Field, s string) ([]Field, error) { + if len(s) == 0 || s[0] != '{' { + return dst, fmt.Errorf("missing '{' at the beginning of stream name") + } + s = s[1:] + if len(s) == 0 || s[len(s)-1] != '}' { + return dst, fmt.Errorf("missing '}' at the end of stream name") + } + s = s[:len(s)-1] + if len(s) == 0 { + return dst, nil + } + + for { + n := strings.Index(s, `="`) + if n < 0 { + return dst, fmt.Errorf("cannot find label value in double quotes at [%s]", s) + } + name := s[:n] + s = s[n+1:] + + value, nOffset := tryUnquoteString(s) + if nOffset < 0 { + return dst, fmt.Errorf("cannot find parse label value in double quotes at [%s]", s) + } + s = s[nOffset:] + + dst = append(dst, Field{ + Name: name, + Value: value, + }) + + if len(s) == 0 { + return dst, nil + } + if s[0] != ',' { + return dst, fmt.Errorf("missing ',' after %s=%q", name, value) + } + s = s[1:] + } +} diff --git a/lib/logstorage/storage_search_test.go b/lib/logstorage/storage_search_test.go index 7e2dd9461..fe1cb5392 100644 --- a/lib/logstorage/storage_search_test.go +++ b/lib/logstorage/storage_search_test.go @@ -650,3 +650,23 @@ func TestStorageSearch(t *testing.T) { s.MustClose() fs.MustRemoveAll(path) } + +func TestParseStreamLabelsSuccess(t *testing.T) { + f := func(s, resultExpected string) { + t.Helper() + + labels, err := parseStreamLabels(nil, s) + if err != nil { + t.Fatalf("unexpected error: %s", err) + } + result := marshalFieldsToJSON(nil, labels) + if string(result) != resultExpected { + t.Fatalf("unexpected result\ngot\n%s\nwant\n%s", result, resultExpected) + } + } + + f(`{}`, `{}`) + f(`{foo="bar"}`, `{"foo":"bar"}`) + f(`{a="b",c="d"}`, `{"a":"b","c":"d"}`) + f(`{a="a=,b\"c}",b="d"}`, `{"a":"a=,b\"c}","b":"d"}`) +} diff --git a/lib/stringsutil/less_natural_test.go b/lib/stringsutil/less_natural_test.go index 1de93e2e9..4e4b1e100 100644 --- a/lib/stringsutil/less_natural_test.go +++ b/lib/stringsutil/less_natural_test.go @@ -39,6 +39,16 @@ func TestLessNatural(t *testing.T) { f("123", "99", false) f("99", "123", true) + // negative numbers (works unexpectedly - this is OK for natural sort order) + f("-93", "5", false) + f("5", "-93", true) + f("-9", "-5", false) + f("-5", "-9", true) + f("-93", "foo", true) + f("foo", "-93", false) + f("foo-9", "foo-10", true) + f("foo-10", "foo-9", false) + // floating-point comparsion (works unexpectedly - this is OK for natural sort order) f("1.23", "1.123", true) f("1.123", "1.23", false)