mirror of
https://github.com/VictoriaMetrics/VictoriaMetrics.git
synced 2024-11-21 14:44:00 +00:00
lib/logstorage: work-in-progress
This commit is contained in:
parent
e2590f0485
commit
dc55146752
46 changed files with 2615 additions and 808 deletions
|
@ -6,8 +6,8 @@
|
||||||
|
|
||||||
{% stripspace %}
|
{% stripspace %}
|
||||||
|
|
||||||
// LabelsForHits formats labels for /select/logsql/hits response
|
// FieldsForHits formats labels for /select/logsql/hits response
|
||||||
{% func LabelsForHits(columns []logstorage.BlockColumn, rowIdx int) %}
|
{% func FieldsForHits(columns []logstorage.BlockColumn, rowIdx int) %}
|
||||||
{
|
{
|
||||||
{% if len(columns) > 0 %}
|
{% if len(columns) > 0 %}
|
||||||
{%q= columns[0].Name %}:{%q= columns[0].Values[rowIdx] %}
|
{%q= columns[0].Name %}:{%q= columns[0].Values[rowIdx] %}
|
||||||
|
|
|
@ -11,7 +11,7 @@ import (
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logstorage"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logstorage"
|
||||||
)
|
)
|
||||||
|
|
||||||
// LabelsForHits formats labels for /select/logsql/hits response
|
// FieldsForHits formats labels for /select/logsql/hits response
|
||||||
|
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:10
|
//line app/vlselect/logsql/hits_response.qtpl:10
|
||||||
import (
|
import (
|
||||||
|
@ -27,7 +27,7 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:10
|
//line app/vlselect/logsql/hits_response.qtpl:10
|
||||||
func StreamLabelsForHits(qw422016 *qt422016.Writer, columns []logstorage.BlockColumn, rowIdx int) {
|
func StreamFieldsForHits(qw422016 *qt422016.Writer, columns []logstorage.BlockColumn, rowIdx int) {
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:10
|
//line app/vlselect/logsql/hits_response.qtpl:10
|
||||||
qw422016.N().S(`{`)
|
qw422016.N().S(`{`)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:12
|
//line app/vlselect/logsql/hits_response.qtpl:12
|
||||||
|
@ -58,22 +58,22 @@ func StreamLabelsForHits(qw422016 *qt422016.Writer, columns []logstorage.BlockCo
|
||||||
}
|
}
|
||||||
|
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
func WriteLabelsForHits(qq422016 qtio422016.Writer, columns []logstorage.BlockColumn, rowIdx int) {
|
func WriteFieldsForHits(qq422016 qtio422016.Writer, columns []logstorage.BlockColumn, rowIdx int) {
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
qw422016 := qt422016.AcquireWriter(qq422016)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
StreamLabelsForHits(qw422016, columns, rowIdx)
|
StreamFieldsForHits(qw422016, columns, rowIdx)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
qt422016.ReleaseWriter(qw422016)
|
qt422016.ReleaseWriter(qw422016)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
}
|
}
|
||||||
|
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
func LabelsForHits(columns []logstorage.BlockColumn, rowIdx int) string {
|
func FieldsForHits(columns []logstorage.BlockColumn, rowIdx int) string {
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
qb422016 := qt422016.AcquireByteBuffer()
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
WriteLabelsForHits(qb422016, columns, rowIdx)
|
WriteFieldsForHits(qb422016, columns, rowIdx)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
qs422016 := string(qb422016.B)
|
qs422016 := string(qb422016.B)
|
||||||
//line app/vlselect/logsql/hits_response.qtpl:19
|
//line app/vlselect/logsql/hits_response.qtpl:19
|
||||||
|
|
|
@ -77,7 +77,7 @@ func ProcessHitsRequest(ctx context.Context, w http.ResponseWriter, r *http.Requ
|
||||||
hitsStr := strings.Clone(hitsValues[i])
|
hitsStr := strings.Clone(hitsValues[i])
|
||||||
|
|
||||||
bb.Reset()
|
bb.Reset()
|
||||||
WriteLabelsForHits(bb, columns, i)
|
WriteFieldsForHits(bb, columns, i)
|
||||||
|
|
||||||
mLock.Lock()
|
mLock.Lock()
|
||||||
hs, ok := m[string(bb.B)]
|
hs, ok := m[string(bb.B)]
|
||||||
|
@ -189,21 +189,21 @@ func ProcessFieldValuesRequest(ctx context.Context, w http.ResponseWriter, r *ht
|
||||||
WriteValuesWithHitsJSON(w, values)
|
WriteValuesWithHitsJSON(w, values)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessStreamLabelNamesRequest processes /select/logsql/stream_label_names request.
|
// ProcessStreamFieldNamesRequest processes /select/logsql/stream_field_names request.
|
||||||
//
|
//
|
||||||
// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-names
|
// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-field-names
|
||||||
func ProcessStreamLabelNamesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) {
|
func ProcessStreamFieldNamesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) {
|
||||||
q, tenantIDs, err := parseCommonArgs(r)
|
q, tenantIDs, err := parseCommonArgs(r)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
httpserver.Errorf(w, r, "%s", err)
|
httpserver.Errorf(w, r, "%s", err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// Obtain stream label names for the given query
|
// Obtain stream field names for the given query
|
||||||
q.Optimize()
|
q.Optimize()
|
||||||
names, err := vlstorage.GetStreamLabelNames(ctx, tenantIDs, q)
|
names, err := vlstorage.GetStreamFieldNames(ctx, tenantIDs, q)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
httpserver.Errorf(w, r, "cannot obtain stream label names: %s", err)
|
httpserver.Errorf(w, r, "cannot obtain stream field names: %s", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
|
@ -211,20 +211,20 @@ func ProcessStreamLabelNamesRequest(ctx context.Context, w http.ResponseWriter,
|
||||||
WriteValuesWithHitsJSON(w, names)
|
WriteValuesWithHitsJSON(w, names)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessStreamLabelValuesRequest processes /select/logsql/stream_label_values request.
|
// ProcessStreamFieldValuesRequest processes /select/logsql/stream_field_values request.
|
||||||
//
|
//
|
||||||
// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-label-values
|
// See https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-field-values
|
||||||
func ProcessStreamLabelValuesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) {
|
func ProcessStreamFieldValuesRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) {
|
||||||
q, tenantIDs, err := parseCommonArgs(r)
|
q, tenantIDs, err := parseCommonArgs(r)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
httpserver.Errorf(w, r, "%s", err)
|
httpserver.Errorf(w, r, "%s", err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// Parse labelName query arg
|
// Parse fieldName query arg
|
||||||
labelName := r.FormValue("label")
|
fieldName := r.FormValue("field")
|
||||||
if labelName == "" {
|
if fieldName == "" {
|
||||||
httpserver.Errorf(w, r, "missing 'label' query arg")
|
httpserver.Errorf(w, r, "missing 'field' query arg")
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -238,11 +238,11 @@ func ProcessStreamLabelValuesRequest(ctx context.Context, w http.ResponseWriter,
|
||||||
limit = 0
|
limit = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
// Obtain stream label names for the given query
|
// Obtain stream field values for the given query and the given fieldName
|
||||||
q.Optimize()
|
q.Optimize()
|
||||||
values, err := vlstorage.GetStreamLabelValues(ctx, tenantIDs, q, labelName, uint64(limit))
|
values, err := vlstorage.GetStreamFieldValues(ctx, tenantIDs, q, fieldName, uint64(limit))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
httpserver.Errorf(w, r, "cannot obtain stream label values: %s", err)
|
httpserver.Errorf(w, r, "cannot obtain stream field values: %s", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
|
|
|
@ -157,13 +157,13 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool {
|
||||||
logsqlQueryRequests.Inc()
|
logsqlQueryRequests.Inc()
|
||||||
logsql.ProcessQueryRequest(ctx, w, r)
|
logsql.ProcessQueryRequest(ctx, w, r)
|
||||||
return true
|
return true
|
||||||
case "/select/logsql/stream_label_names":
|
case "/select/logsql/stream_field_names":
|
||||||
logsqlStreamLabelNamesRequests.Inc()
|
logsqlStreamFieldNamesRequests.Inc()
|
||||||
logsql.ProcessStreamLabelNamesRequest(ctx, w, r)
|
logsql.ProcessStreamFieldNamesRequest(ctx, w, r)
|
||||||
return true
|
return true
|
||||||
case "/select/logsql/stream_label_values":
|
case "/select/logsql/stream_field_values":
|
||||||
logsqlStreamLabelValuesRequests.Inc()
|
logsqlStreamFieldValuesRequests.Inc()
|
||||||
logsql.ProcessStreamLabelValuesRequest(ctx, w, r)
|
logsql.ProcessStreamFieldValuesRequest(ctx, w, r)
|
||||||
return true
|
return true
|
||||||
case "/select/logsql/streams":
|
case "/select/logsql/streams":
|
||||||
logsqlStreamsRequests.Inc()
|
logsqlStreamsRequests.Inc()
|
||||||
|
@ -192,7 +192,7 @@ var (
|
||||||
logsqlFieldValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_values"}`)
|
logsqlFieldValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/field_values"}`)
|
||||||
logsqlHitsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/hits"}`)
|
logsqlHitsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/hits"}`)
|
||||||
logsqlQueryRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/query"}`)
|
logsqlQueryRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/query"}`)
|
||||||
logsqlStreamLabelNamesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_label_names"}`)
|
logsqlStreamFieldNamesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_field_names"}`)
|
||||||
logsqlStreamLabelValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_label_values"}`)
|
logsqlStreamFieldValuesRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/stream_field_values"}`)
|
||||||
logsqlStreamsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/streams"}`)
|
logsqlStreamsRequests = metrics.NewCounter(`vl_http_requests_total{path="/select/logsql/streams"}`)
|
||||||
)
|
)
|
||||||
|
|
|
@ -123,16 +123,16 @@ func GetFieldValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *log
|
||||||
return strg.GetFieldValues(ctx, tenantIDs, q, fieldName, limit)
|
return strg.GetFieldValues(ctx, tenantIDs, q, fieldName, limit)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreamLabelNames executes q and returns stream labels names seen in results.
|
// GetStreamFieldNames executes q and returns stream field names seen in results.
|
||||||
func GetStreamLabelNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]logstorage.ValueWithHits, error) {
|
func GetStreamFieldNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetStreamLabelNames(ctx, tenantIDs, q)
|
return strg.GetStreamFieldNames(ctx, tenantIDs, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreamLabelValues executes q and returns stream label values for the given labelName seen in results.
|
// GetStreamFieldValues executes q and returns stream field values for the given fieldName seen in results.
|
||||||
//
|
//
|
||||||
// If limit > 0, then up to limit unique stream label values are returned.
|
// If limit > 0, then up to limit unique stream field values are returned.
|
||||||
func GetStreamLabelValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, labelName string, limit uint64) ([]logstorage.ValueWithHits, error) {
|
func GetStreamFieldValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, fieldName string, limit uint64) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetStreamLabelValues(ctx, tenantIDs, q, labelName, limit)
|
return strg.GetStreamFieldValues(ctx, tenantIDs, q, fieldName, limit)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreams executes q and returns streams seen in query results.
|
// GetStreams executes q and returns streams seen in query results.
|
||||||
|
|
|
@ -19,6 +19,16 @@ according to [these docs](https://docs.victoriametrics.com/victorialogs/quicksta
|
||||||
|
|
||||||
## tip
|
## tip
|
||||||
|
|
||||||
|
* FEATURE: add [`pack_json` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#pack_json-pipe), which packs all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into a JSON object and stores it into the given field.
|
||||||
|
* FEATURE: add [`unroll` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#unroll-pipe), which can be used for unrolling JSON arrays stored in [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
* FEATURE: add [`replace_regexp` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#replace_regexp-pipe), which allows updating [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with regular expressions.
|
||||||
|
* FEATURE: improve performance for [`format`](https://docs.victoriametrics.com/victorialogs/logsql/#format-pipe) and [`extract`](https://docs.victoriametrics.com/victorialogs/logsql/#extract-pipe) pipes.
|
||||||
|
* FEATURE: improve performance for [`/select/logsql/field_names` HTTP API](https://docs.victoriametrics.com/victorialogs/querying/#querying-field-names).
|
||||||
|
|
||||||
|
* BUGFIX: prevent from panic in [`sort` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe) when VictoriaLogs runs on a system with one CPU core.
|
||||||
|
* BUGFIX: do not return referenced fields if they weren't present in the original logs. For example, `_time:5m | format if (non_existing_field:"") "abc"` could return empty `non_exiting_field`, while it shuldn't be returned because it is missing in the original logs.
|
||||||
|
* BUGFIX: properly initialize values for [`in(...)` filter](https://docs.victoriametrics.com/victorialogs/logsql/#exact-filter) inside [`filter` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#filter-pipe) if the `in(...)` contains other [filters](https://docs.victoriametrics.com/victorialogs/logsql/#filters). For example, `_time:5m | filter ip:in(user_type:admin | fields ip)` now works correctly.
|
||||||
|
|
||||||
## [v0.11.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.11.0-victorialogs)
|
## [v0.11.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.11.0-victorialogs)
|
||||||
|
|
||||||
Released at 2024-05-25
|
Released at 2024-05-25
|
||||||
|
@ -63,8 +73,8 @@ Released at 2024-05-22
|
||||||
* 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 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 [`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/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_field_names` HTTP endpoint for returning [stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field names from results of the given query. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-field-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: add `/select/logsql/stream_field_values` HTTP endpoint for returning [stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field values for the given label from results of the given query. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#querying-stream-field-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.
|
* 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).
|
* 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).
|
||||||
|
|
|
@ -37,6 +37,8 @@ For example, the following query finds all the logs with `error` word:
|
||||||
error
|
error
|
||||||
```
|
```
|
||||||
|
|
||||||
|
See [how to send queries to VictoriaLogs](https://docs.victoriametrics.com/victorialogs/querying/).
|
||||||
|
|
||||||
If the queried [word](#word) clashes with LogsQL keywords, then just wrap it into quotes.
|
If the queried [word](#word) clashes with LogsQL keywords, then just wrap it into quotes.
|
||||||
For example, the following query finds all the log messages with `and` [word](#word):
|
For example, the following query finds all the log messages with `and` [word](#word):
|
||||||
|
|
||||||
|
@ -80,11 +82,32 @@ Typical LogsQL query constists of multiple [filters](#filters) joined with `AND`
|
||||||
So LogsQL allows omitting `AND` words. For example, the following query is equivalent to the query above:
|
So LogsQL allows omitting `AND` words. For example, the following query is equivalent to the query above:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
error _time:5m
|
_time:5m error
|
||||||
```
|
```
|
||||||
|
|
||||||
The query returns all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) by default.
|
The query returns logs in arbitrary order because sorting of big amounts of logs may require non-trivial amounts of CPU and RAM.
|
||||||
See [how to query specific fields](#querying-specific-fields).
|
The number of logs with `error` word over the last 5 minutes isn't usually too big (e.g. less than a few millions), so it is OK to sort them with [`sort` pipe](#sort-pipe).
|
||||||
|
The following query sorts the selected logs by [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) field:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m error | sort by (_time)
|
||||||
|
```
|
||||||
|
|
||||||
|
It is unlikely you are going to investigate more than a few hundreds of logs returned by the query above. So you can limit the number of returned logs
|
||||||
|
with [`limit` pipe](#limit-pipe). The following query returns the last 10 logs with the `error` word over the last 5 minutes:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m error | sort by (_time) desc | limit 10
|
||||||
|
```
|
||||||
|
|
||||||
|
By default VictoriaLogs returns all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
If you need only the given set of fields, then add [`fields` pipe](#fields-pipe) to the end of the query. For example, the following query returns only
|
||||||
|
[`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field), [`_stream`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields)
|
||||||
|
and [`_msg`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) fields:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
error _time:5m | fields _time, _stream, _msg
|
||||||
|
```
|
||||||
|
|
||||||
Suppose the query above selects too many rows because some buggy app pushes invalid error logs to VictoriaLogs. Suppose the app adds `buggy_app` [word](#word) to every log line.
|
Suppose the query above selects too many rows because some buggy app pushes invalid error logs to VictoriaLogs. Suppose the app adds `buggy_app` [word](#word) to every log line.
|
||||||
Then the following query removes all the logs from the buggy app, allowing us paying attention to the real errors:
|
Then the following query removes all the logs from the buggy app, allowing us paying attention to the real errors:
|
||||||
|
@ -93,8 +116,10 @@ Then the following query removes all the logs from the buggy app, allowing us pa
|
||||||
_time:5m error NOT buggy_app
|
_time:5m error NOT buggy_app
|
||||||
```
|
```
|
||||||
|
|
||||||
This query uses `NOT` [operator](#logical-filter) for removing log lines from the buggy app. The `NOT` operator is used frequently, so it can be substituted with `!` char.
|
This query uses `NOT` [operator](#logical-filter) for removing log lines from the buggy app. The `NOT` operator is used frequently, so it can be substituted with `!` char
|
||||||
So the following query is equivalent to the previous one:
|
(the `!` char is used instead of `-` char as a shorthand for `NOT` operator becasue it nicely combines with [`=`](https://docs.victoriametrics.com/victorialogs/logsql/#exact-filter)
|
||||||
|
and [`~`](https://docs.victoriametrics.com/victorialogs/logsql/#regexp-filter) filters like `!=` and `!~`).
|
||||||
|
The following query is equivalent to the previous one:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
_time:5m error !buggy_app
|
_time:5m error !buggy_app
|
||||||
|
@ -113,17 +138,15 @@ This query can be rewritten to more clear query with the `OR` [operator](#logica
|
||||||
_time:5m error !(buggy_app OR foobar)
|
_time:5m error !(buggy_app OR foobar)
|
||||||
```
|
```
|
||||||
|
|
||||||
Note that the parentheses are required here, since otherwise the query won't return the expected results.
|
The parentheses are **required** here, since otherwise the query won't return the expected results.
|
||||||
The query `error !buggy_app OR foobar` is interpreted as `(error AND NOT buggy_app) OR foobar`. This query may return error logs
|
The query `error !buggy_app OR foobar` is interpreted as `(error AND NOT buggy_app) OR foobar` according to [priorities for AND, OR and NOT operator](#logical-filters).
|
||||||
from the buggy app if they contain `foobar` [word](#word). This query also continues returning all the error logs from the second buggy app.
|
This query returns logs with `foobar` [word](#word), even if do not contain `error` word or contain `buggy_app` word.
|
||||||
This is because of different priorities for `NOT`, `AND` and `OR` operators.
|
So it is recommended wrapping the needed query parts into explicit parentheses if you are unsure in priority rules.
|
||||||
Read [these docs](#logical-filter) for more details. There is no need in remembering all these priority rules -
|
|
||||||
just wrap the needed query parts into explicit parentheses if you aren't sure in priority rules.
|
|
||||||
As an additional bonus, explicit parentheses make queries easier to read and maintain.
|
As an additional bonus, explicit parentheses make queries easier to read and maintain.
|
||||||
|
|
||||||
Queries above assume that the `error` [word](#word) is stored in the [log message](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field).
|
Queries above assume that the `error` [word](#word) is stored in the [log message](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field).
|
||||||
This word can be stored in other [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) such as `log.level`.
|
If this word is stored in other [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) such as `log.level`, then add `log.level:` prefix
|
||||||
How to select error logs in this case? Just add the `log.level:` prefix in front of the `error` word:
|
in front of the `error` word:
|
||||||
|
|
||||||
```logsq
|
```logsq
|
||||||
_time:5m log.level:error !(buggy_app OR foobar)
|
_time:5m log.level:error !(buggy_app OR foobar)
|
||||||
|
@ -158,8 +181,16 @@ If the `app` field is associated with the log stream, then the query above can b
|
||||||
_time:5m log.level:error _stream:{app!~"buggy_app|foobar"}
|
_time:5m log.level:error _stream:{app!~"buggy_app|foobar"}
|
||||||
```
|
```
|
||||||
|
|
||||||
This query completely skips scanning for logs from `buggy_app` and `foobar` apps, thus significantly reducing disk read IO and CPU time
|
This query skips scanning for [log messages](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) from `buggy_app` and `foobar` apps.
|
||||||
needed for performing the query.
|
It inpsects only `log.level` and [`_stream`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) labels.
|
||||||
|
This significantly reduces disk read IO and CPU time needed for performing the query.
|
||||||
|
|
||||||
|
LogsQL also provides [functions for statistics calculation](#stats-pipe) over the selected logs. For example, the following query returns the number of logs
|
||||||
|
with the `error` word for the last 5 minutes:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m error | stats count() logs_with_error
|
||||||
|
```
|
||||||
|
|
||||||
Finally, it is recommended reading [performance tips](#performance-tips).
|
Finally, it is recommended reading [performance tips](#performance-tips).
|
||||||
|
|
||||||
|
@ -177,13 +208,16 @@ These words are taken into account by full-text search filters such as
|
||||||
|
|
||||||
#### Query syntax
|
#### Query syntax
|
||||||
|
|
||||||
LogsQL query must contain [filters](#filters) for selecting the matching logs. At least a single filter is required.
|
LogsQL query must contain at least a single [filter](#filters) for selecting the matching logs.
|
||||||
For example, the following query selects all the logs for the last 5 minutes by using [`_time` filter](#time-filter):
|
For example, the following query selects all the logs for the last 5 minutes by using [`_time` filter](#time-filter):
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
_time:5m
|
_time:5m
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Tip: try [`*` filter](https://docs.victoriametrics.com/victorialogs/logsql/#any-value-filter), which selects all the logs stored in VictoriaLogs.
|
||||||
|
Do not worry - this doesn't crash VictoriaLogs, even if it contains trillions of logs. In the worst case it will return
|
||||||
|
|
||||||
Additionally to filters, LogQL query may contain arbitrary mix of optional actions for processing the selected logs. These actions are delimited by `|` and are known as [`pipes`](#pipes).
|
Additionally to filters, LogQL query may contain arbitrary mix of optional actions for processing the selected logs. These actions are delimited by `|` and are known as [`pipes`](#pipes).
|
||||||
For example, the following query uses [`stats` pipe](#stats-pipe) for returning the number of [log messages](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field)
|
For example, the following query uses [`stats` pipe](#stats-pipe) for returning the number of [log messages](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field)
|
||||||
with the `error` [word](#word) for the last 5 minutes:
|
with the `error` [word](#word) for the last 5 minutes:
|
||||||
|
@ -1080,13 +1114,16 @@ LogsQL supports the following pipes:
|
||||||
- [`format`](#format-pipe) formats ouptut field from input [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`format`](#format-pipe) formats ouptut field from input [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
- [`limit`](#limit-pipe) limits the number selected logs.
|
- [`limit`](#limit-pipe) limits the number selected logs.
|
||||||
- [`offset`](#offset-pipe) skips the given number of selected logs.
|
- [`offset`](#offset-pipe) skips the given number of selected logs.
|
||||||
|
- [`pack_json`](#pack_json-pipe) packs [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into JSON object.
|
||||||
- [`rename`](#rename-pipe) renames [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`rename`](#rename-pipe) renames [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
- [`replace`](#replace-pipe) replaces substrings in the specified [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`replace`](#replace-pipe) replaces substrings in the specified [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
- [`replace_regexp`](#replace_regexp-pipe) updates [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with regular expressions.
|
||||||
- [`sort`](#sort-pipe) sorts logs by the given [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`sort`](#sort-pipe) sorts logs by the given [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
- [`stats`](#stats-pipe) calculates various stats over the selected logs.
|
- [`stats`](#stats-pipe) calculates various stats over the selected logs.
|
||||||
- [`uniq`](#uniq-pipe) returns unique log entires.
|
- [`uniq`](#uniq-pipe) returns unique log entires.
|
||||||
- [`unpack_json`](#unpack_json-pipe) unpacks JSON fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`unpack_json`](#unpack_json-pipe) unpacks JSON fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
- [`unpack_logfmt`](#unpack_logfmt-pipe) unpacks [logfmt](https://brandur.org/logfmt) fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
- [`unpack_logfmt`](#unpack_logfmt-pipe) unpacks [logfmt](https://brandur.org/logfmt) fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
- [`unroll`](#unroll-pipe) unrolls JSON arrays from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
|
||||||
### copy pipe
|
### copy pipe
|
||||||
|
|
||||||
|
@ -1178,6 +1215,9 @@ For example, the following query preserves the original `ip` field value if `foo
|
||||||
_time:5m | extract 'ip=<ip> ' from foo skip_empty_results
|
_time:5m | extract 'ip=<ip> ' from foo skip_empty_results
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Performance tip: it is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `extract`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Format for extract pipe pattern](#format-for-extract-pipe-pattern)
|
- [Format for extract pipe pattern](#format-for-extract-pipe-pattern)
|
||||||
|
@ -1363,10 +1403,14 @@ when at least `field1` or `field2` aren't empty, while preserving the original `
|
||||||
_time:5m | format "<field1><field2>" as foo skip_empty_results
|
_time:5m | format "<field1><field2>" as foo skip_empty_results
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Performance tip: it is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `format`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Conditional format](#conditional-format)
|
- [Conditional format](#conditional-format)
|
||||||
- [`replace` pipe](#replace-pipe)
|
- [`replace` pipe](#replace-pipe)
|
||||||
|
- [`replace_regexp` pipe](#replace_regexp-pipe)
|
||||||
- [`extract` pipe](#extract-pipe)
|
- [`extract` pipe](#extract-pipe)
|
||||||
|
|
||||||
|
|
||||||
|
@ -1419,6 +1463,37 @@ See also:
|
||||||
- [`limit` pipe](#limit-pipe)
|
- [`limit` pipe](#limit-pipe)
|
||||||
- [`sort` pipe](#sort-pipe)
|
- [`sort` pipe](#sort-pipe)
|
||||||
|
|
||||||
|
### pack_json pipe
|
||||||
|
|
||||||
|
`| pack_json as field_name` [pipe](#pipe) packs all [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into JSON object
|
||||||
|
and stores its as a string in the given `field_name`.
|
||||||
|
|
||||||
|
For example, the following query packs all the fields into JSON object and stores it into [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field)
|
||||||
|
for logs over the last 5 minutes:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | pack_json as _msg
|
||||||
|
```
|
||||||
|
|
||||||
|
The `as _msg` part can be omitted if packed JSON object is stored into [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field).
|
||||||
|
The following query is equivalent to the previous one:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | pack_json
|
||||||
|
```
|
||||||
|
|
||||||
|
The `pack_json` doesn't touch other labels. If you do not need them, then add [`| fields ...`](#fields-pipe) after the `pack_json` pipe. For example, the following query
|
||||||
|
leaves only the `foo` label with the original log fields packed into JSON:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | pack_json as foo | fields foo
|
||||||
|
```
|
||||||
|
|
||||||
|
See also:
|
||||||
|
|
||||||
|
- [`unpack_json` pipe](#unpack_json-pipe)
|
||||||
|
|
||||||
|
|
||||||
### rename pipe
|
### rename pipe
|
||||||
|
|
||||||
If some [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) must be renamed, then `| rename src1 as dst1, ..., srcN as dstN` [pipe](#pipes) can be used.
|
If some [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) must be renamed, then `| rename src1 as dst1, ..., srcN as dstN` [pipe](#pipes) can be used.
|
||||||
|
@ -1470,9 +1545,13 @@ at the [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#da
|
||||||
_time:5m | replace ('foo', 'bar') at baz limit 1
|
_time:5m | replace ('foo', 'bar') at baz limit 1
|
||||||
```
|
```
|
||||||
|
|
||||||
|
Performance tip: it is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `replace`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Conditional replace](#conditional-replace)
|
- [Conditional replace](#conditional-replace)
|
||||||
|
- [`replace_regexp` pipe](#replace_regexp-pipe)
|
||||||
- [`format` pipe](#format-pipe)
|
- [`format` pipe](#format-pipe)
|
||||||
- [`extract` pipe](#extract-pipe)
|
- [`extract` pipe](#extract-pipe)
|
||||||
|
|
||||||
|
@ -1487,6 +1566,58 @@ only if `user_type` field equals to `admin`:
|
||||||
_time:5m | replace if (user_type:=admin) replace ("secret", "***") at password
|
_time:5m | replace if (user_type:=admin) replace ("secret", "***") at password
|
||||||
```
|
```
|
||||||
|
|
||||||
|
### replace_regexp pipe
|
||||||
|
|
||||||
|
`| replace_regexp ("regexp", "replacement") at field` [pipe](#pipes) replaces all the substrings matching the given `regexp` with the given `replacement`
|
||||||
|
in the given [`field`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
|
||||||
|
The `regexp` must contain regular expression with [RE2 syntax](https://github.com/google/re2/wiki/Syntax).
|
||||||
|
The `replacement` may contain `$N` or `${N}` placeholders, which are substituted with the `N-th` capturing group in the `regexp`.
|
||||||
|
|
||||||
|
For example, the following query replaces all the substrings starting with `host-` and ending with `-foo` with the contents between `host-` and `-foo` in the [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) for logs over the last 5 minutes:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | replace_regexp ("host-(.+?)-foo", "$1") at _msg
|
||||||
|
```
|
||||||
|
|
||||||
|
The `at _msg` part can be omitted if the replacement occurs in the [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field).
|
||||||
|
The following query is equivalent to the previous one:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | replace_regexp ("host-(.+?)-foo", "$1")
|
||||||
|
```
|
||||||
|
|
||||||
|
The number of replacements can be limited with `limit N` at the end of `replace`. For example, the following query replaces only the first `password: ...` substring
|
||||||
|
ending with whitespace with empty substring at the [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) `baz`:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | replace_regexp ('password: [^ ]+', '') at baz limit 1
|
||||||
|
```
|
||||||
|
|
||||||
|
Performance tips:
|
||||||
|
|
||||||
|
- It is recommended using [`replace` pipe](#replace-pipe) instead of `replace_regexp` if possible, since it works faster.
|
||||||
|
- It is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `replace`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
|
See also:
|
||||||
|
|
||||||
|
- [Conditional replace_regexp](#conditional-replace_regexp)
|
||||||
|
- [`replace` pipe](#replace-pipe)
|
||||||
|
- [`format` pipe](#format-pipe)
|
||||||
|
- [`extract` pipe](#extract-pipe)
|
||||||
|
|
||||||
|
#### Conditional replace_regexp
|
||||||
|
|
||||||
|
If the [`replace_regexp` pipe](#replace-pipe) musn't be applied to every [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model),
|
||||||
|
then add `if (<filters>)` after `replace_regexp`.
|
||||||
|
The `<filters>` can contain arbitrary [filters](#filters). For example, the following query replaces `password: ...` substrings ending with whitespace
|
||||||
|
with `***` in the `foo` field only if `user_type` field equals to `admin`:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | replace_regexp if (user_type:=admin) replace ("password: [^ ]+", "") at foo
|
||||||
|
```
|
||||||
|
|
||||||
### sort pipe
|
### sort pipe
|
||||||
|
|
||||||
By default logs are selected in arbitrary order because of performance reasons. If logs must be sorted, then `| sort by (field1, ..., fieldN)` [pipe](#pipes) can be used.
|
By default logs are selected in arbitrary order because of performance reasons. If logs must be sorted, then `| sort by (field1, ..., fieldN)` [pipe](#pipes) can be used.
|
||||||
|
@ -1720,10 +1851,10 @@ _time:5m | uniq by (host, path)
|
||||||
|
|
||||||
The unique entries are returned in arbitrary order. Use [`sort` pipe](#sort-pipe) in order to sort them if needed.
|
The unique entries are returned in arbitrary order. Use [`sort` pipe](#sort-pipe) in order to sort them if needed.
|
||||||
|
|
||||||
Add `hits` after `uniq by (...)` in order to return the number of matching logs per each field value:
|
Add `with hits` after `uniq by (...)` in order to return the number of matching logs per each field value:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
_time:5m | uniq by (host) hits
|
_time:5m | uniq by (host) with hits
|
||||||
```
|
```
|
||||||
|
|
||||||
Unique entries are stored in memory during query execution. Big number of unique selected entries may require a lot of memory.
|
Unique entries are stored in memory during query execution. Big number of unique selected entries may require a lot of memory.
|
||||||
|
@ -1802,15 +1933,22 @@ form `foo`:
|
||||||
_time:5m | unpack_json from foo result_prefix "foo_"
|
_time:5m | unpack_json from foo result_prefix "foo_"
|
||||||
```
|
```
|
||||||
|
|
||||||
Performance tip: it is better from performance and resource usage PoV ingesting parsed JSON logs into VictoriaLogs
|
Performance tips:
|
||||||
according to the [supported data model](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
|
||||||
instead of ingesting unparsed JSON lines into VictoriaLogs and then parsing them at query time with [`unpack_json` pipe](#unpack_json-pipe).
|
- It is better from performance and resource usage PoV ingesting parsed JSON logs into VictoriaLogs
|
||||||
|
according to the [supported data model](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||||
|
instead of ingesting unparsed JSON lines into VictoriaLogs and then parsing them at query time with [`unpack_json` pipe](#unpack_json-pipe).
|
||||||
|
|
||||||
|
- It is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `unpack_json`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Conditional `unpack_json`](#conditional-unpack_json)
|
- [Conditional `unpack_json`](#conditional-unpack_json)
|
||||||
- [`unpack_logfmt` pipe](#unpack_logfmt-pipe)
|
- [`unpack_logfmt` pipe](#unpack_logfmt-pipe)
|
||||||
- [`extract` pipe](#extract-pipe)
|
- [`extract` pipe](#extract-pipe)
|
||||||
|
- [`unroll` pipe](#unroll-pipe)
|
||||||
|
- [`pack_json` pipe](#pack_json-pipe)
|
||||||
|
|
||||||
#### Conditional unpack_json
|
#### Conditional unpack_json
|
||||||
|
|
||||||
|
@ -1879,9 +2017,14 @@ from `foo` field:
|
||||||
_time:5m | unpack_logfmt from foo result_prefix "foo_"
|
_time:5m | unpack_logfmt from foo result_prefix "foo_"
|
||||||
```
|
```
|
||||||
|
|
||||||
Performance tip: it is better from performance and resource usage PoV ingesting parsed [logfmt](https://brandur.org/logfmt) logs into VictoriaLogs
|
Performance tips:
|
||||||
according to the [supported data model](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
|
||||||
instead of ingesting unparsed logfmt lines into VictoriaLogs and then parsing them at query time with [`unpack_logfmt` pipe](#unpack_logfmt-pipe).
|
- It is better from performance and resource usage PoV ingesting parsed [logfmt](https://brandur.org/logfmt) logs into VictoriaLogs
|
||||||
|
according to the [supported data model](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||||
|
instead of ingesting unparsed logfmt lines into VictoriaLogs and then parsing them at query time with [`unpack_logfmt` pipe](#unpack_logfmt-pipe).
|
||||||
|
|
||||||
|
- It is recommended using more specific [log filters](#filters) in order to reduce the number of log entries, which are passed to `unpack_logfmt`.
|
||||||
|
See [general performance tips](#performance-tips) for details.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
|
@ -1900,6 +2043,34 @@ only if `ip` field in the current log entry isn't set or empty:
|
||||||
_time:5m | unpack_logfmt if (ip:"") from foo
|
_time:5m | unpack_logfmt if (ip:"") from foo
|
||||||
```
|
```
|
||||||
|
|
||||||
|
### unroll pipe
|
||||||
|
|
||||||
|
`| unroll by (field1, ..., fieldN)` [pipe](#pipes) can be used for unrolling JSON arrays from `field1`, `fieldN`
|
||||||
|
[log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) into separate rows.
|
||||||
|
|
||||||
|
For example, the following query unrolls `timestamp` and `value` [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) from logs for the last 5 minutes:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | unroll (timestamp, value)
|
||||||
|
```
|
||||||
|
|
||||||
|
See also:
|
||||||
|
|
||||||
|
- [`unpack_json` pipe](#unpack_json-pipe)
|
||||||
|
- [`extract` pipe](#extract-pipe)
|
||||||
|
- [`uniq_values` stats function](#uniq_values-stats)
|
||||||
|
- [`values` stats function](#values-stats)
|
||||||
|
|
||||||
|
#### Conditional unroll
|
||||||
|
|
||||||
|
If the [`unroll` pipe](#unpack_logfmt-pipe) musn't be applied to every [log entry](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model),
|
||||||
|
then add `if (<filters>)` after `unroll`.
|
||||||
|
The `<filters>` can contain arbitrary [filters](#filters). For example, the following query unrolls `value` field only if `value_type` field equals to `json_array`:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | unroll if (value_type:="json_array") (value)
|
||||||
|
```
|
||||||
|
|
||||||
## stats pipe functions
|
## stats pipe functions
|
||||||
|
|
||||||
LogsQL supports the following functions for [`stats` pipe](#stats-pipe):
|
LogsQL supports the following functions for [`stats` pipe](#stats-pipe):
|
||||||
|
@ -2204,6 +2375,8 @@ over logs for the last 5 minutes:
|
||||||
_time:5m | stats uniq_values(ip) unique_ips
|
_time:5m | stats uniq_values(ip) unique_ips
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The returned unique ip addresses can be unrolled into distinct log entries with [`unroll` pipe](#unroll-pipe).
|
||||||
|
|
||||||
Every unique value is stored in memory during query execution. Big number of unique values may require a lot of memory. Sometimes it is enough to return
|
Every unique value is stored in memory during query execution. Big number of unique values may require a lot of memory. Sometimes it is enough to return
|
||||||
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`,
|
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.
|
while limiting the maximum memory usage.
|
||||||
|
@ -2236,6 +2409,8 @@ over logs for the last 5 minutes:
|
||||||
_time:5m | stats values(ip) ips
|
_time:5m | stats values(ip) ips
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The returned ip addresses can be unrolled into distinct log entries with [`unroll` pipe](#unroll-pipe).
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [`uniq_values`](#uniq_values-stats)
|
- [`uniq_values`](#uniq_values-stats)
|
||||||
|
@ -2257,8 +2432,9 @@ LogsQL supports the following transformations on the log entries selected with [
|
||||||
See [these docs](#extract-pipe) for details.
|
See [these docs](#extract-pipe) for details.
|
||||||
- Unpacking JSON fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](#unpack_json-pipe).
|
- Unpacking JSON fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](#unpack_json-pipe).
|
||||||
- Unpacking [logfmt](https://brandur.org/logfmt) fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](#unpack_logfmt-pipe).
|
- Unpacking [logfmt](https://brandur.org/logfmt) fields from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](#unpack_logfmt-pipe).
|
||||||
- Creating a new field from existing [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) according to the provided format. See [these docs](#format-pipe).
|
- Creating a new field from existing [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) according to the provided format. See [`format` pipe](#format-pipe).
|
||||||
- Replacing substrings in the given [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](#replace-pipe).
|
- Replacing substrings in the given [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||||
|
See [`replace` pipe](#replace-pipe) and [`replace_regexp` pipe](#replace_regexp-pipe) docs.
|
||||||
|
|
||||||
LogsQL will support the following transformations in the future:
|
LogsQL will support the following transformations in the future:
|
||||||
|
|
||||||
|
@ -2350,3 +2526,5 @@ Internally duration values are converted into nanoseconds.
|
||||||
This rule doesn't apply to [time filter](#time-filter) and [stream filter](#stream-filter), which can be put at any place of the query.
|
This rule doesn't apply to [time filter](#time-filter) and [stream filter](#stream-filter), which can be put at any place of the query.
|
||||||
- Move more specific filters, which match lower number of log entries, to the beginning of the query.
|
- Move more specific filters, which match lower number of log entries, to the beginning of the query.
|
||||||
This rule doesn't apply to [time filter](#time-filter) and [stream filter](#stream-filter), which can be put at any place of the query.
|
This rule doesn't apply to [time filter](#time-filter) and [stream filter](#stream-filter), which can be put at any place of the query.
|
||||||
|
- If the selected logs are passed to [pipes](#pipes) for further transformations and statistics' calculations, then it is recommended
|
||||||
|
reducing the number of selected logs by using more specific [filters](#filters), which return lower number of logs to process by [pipes](#pipes).
|
||||||
|
|
|
@ -74,14 +74,14 @@ during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-inges
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
Both label name and label value may contain arbitrary chars. Such chars must be encoded
|
Both field name and field value may contain arbitrary chars. Such chars must be encoded
|
||||||
during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/)
|
during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/)
|
||||||
according to [JSON string encoding](https://www.rfc-editor.org/rfc/rfc7159.html#section-7).
|
according to [JSON string encoding](https://www.rfc-editor.org/rfc/rfc7159.html#section-7).
|
||||||
Unicode chars must be encoded with [UTF-8](https://en.wikipedia.org/wiki/UTF-8) encoding:
|
Unicode chars must be encoded with [UTF-8](https://en.wikipedia.org/wiki/UTF-8) encoding:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"label with whitepsace": "value\nwith\nnewlines",
|
"field with whitepsace": "value\nwith\nnewlines",
|
||||||
"Поле": "价值",
|
"Поле": "价值",
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
@ -89,13 +89,11 @@ Unicode chars must be encoded with [UTF-8](https://en.wikipedia.org/wiki/UTF-8)
|
||||||
VictoriaLogs automatically indexes all the fields in all the [ingested](https://docs.victoriametrics.com/victorialogs/data-ingestion/) logs.
|
VictoriaLogs automatically indexes all the fields in all the [ingested](https://docs.victoriametrics.com/victorialogs/data-ingestion/) logs.
|
||||||
This enables [full-text search](https://docs.victoriametrics.com/victorialogs/logsql/) across all the fields.
|
This enables [full-text search](https://docs.victoriametrics.com/victorialogs/logsql/) across all the fields.
|
||||||
|
|
||||||
VictoriaLogs supports the following field types:
|
VictoriaLogs supports the following special fields additionally to arbitrary [other fields](#other-field):
|
||||||
|
|
||||||
* [`_msg` field](#message-field)
|
* [`_msg` field](#message-field)
|
||||||
* [`_time` field](#time-field)
|
* [`_time` field](#time-field)
|
||||||
* [`_stream` fields](#stream-fields)
|
* [`_stream` fields](#stream-fields)
|
||||||
* [other fields](#other-fields)
|
|
||||||
|
|
||||||
|
|
||||||
### Message field
|
### Message field
|
||||||
|
|
||||||
|
@ -116,7 +114,9 @@ during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-inges
|
||||||
### Time field
|
### Time field
|
||||||
|
|
||||||
The ingested [log entries](#data-model) may contain `_time` field with the timestamp of the ingested log entry.
|
The ingested [log entries](#data-model) may contain `_time` field with the timestamp of the ingested log entry.
|
||||||
For example:
|
The timestamp must be in [RFC3339](https://www.rfc-editor.org/rfc/rfc3339) format. The most commonly used subset of [ISO8601](https://en.wikipedia.org/wiki/ISO_8601)
|
||||||
|
is also supported. It is allowed specifying seconds part of the timestamp with any precision up to nanoseconds.
|
||||||
|
For example, the following [log entry](#data-model) contains valid timestamp with millisecond precision in the `_time` field:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
|
@ -132,29 +132,39 @@ during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-inges
|
||||||
|
|
||||||
If `_time` field is missing, then the data ingestion time is used as log entry timestamp.
|
If `_time` field is missing, then the data ingestion time is used as log entry timestamp.
|
||||||
|
|
||||||
The log entry timestamp allows quickly narrowing down the search to a particular time range.
|
The `_time` field is used in [time filter](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter) for quickly narrowing down
|
||||||
See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter) for details.
|
the search to a particular time range.
|
||||||
|
|
||||||
### Stream fields
|
### Stream fields
|
||||||
|
|
||||||
Some [structured logging](#data-model) fields may uniquely identify the application instance, which generates log entries.
|
Some [structured logging](#data-model) fields may uniquely identify the application instance, which generates log entries.
|
||||||
This may be either a single field such as `instance=host123:456` or a set of fields such as
|
This may be either a single field such as `instance="host123:456"` or a set of fields such as
|
||||||
`(datacenter=..., env=..., job=..., instance=...)` or
|
`{datacenter="...", env="...", job="...", instance="..."}` or
|
||||||
`(kubernetes.namespace=..., kubernetes.node.name=..., kubernetes.pod.name=..., kubernetes.container.name=...)`.
|
`{kubernetes.namespace="...", kubernetes.node.name="...", kubernetes.pod.name="...", kubernetes.container.name="..."}`.
|
||||||
|
|
||||||
Log entries received from a single application instance form a log stream in VictoriaLogs.
|
Log entries received from a single application instance form a **log stream** in VictoriaLogs.
|
||||||
VictoriaLogs optimizes storing and querying of individual log streams. This provides the following benefits:
|
VictoriaLogs optimizes storing and [querying](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter) of individual log streams.
|
||||||
|
This provides the following benefits:
|
||||||
|
|
||||||
- Reduced disk space usage, since a log stream from a single application instance is usually compressed better
|
- Reduced disk space usage, since a log stream from a single application instance is usually compressed better
|
||||||
than a mixed log stream from multiple distinct applications.
|
than a mixed log stream from multiple distinct applications.
|
||||||
|
|
||||||
- Increased query performance, since VictoriaLogs needs to scan lower amounts of data
|
- Increased query performance, since VictoriaLogs needs to scan lower amounts of data
|
||||||
when [searching by stream labels](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter).
|
when [searching by stream fields](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter).
|
||||||
|
|
||||||
VictoriaLogs cannot determine automatically, which fields uniquely identify every log stream,
|
Every ingested log entry is associated with a log stream. The name of this stream is stored in `_stream` field.
|
||||||
so it stores all the received log entries in a single default stream - `{}`.
|
This field has the format similar to [labels in Prometheus metrics](https://docs.victoriametrics.com/keyconcepts/#labels):
|
||||||
This may lead to not-so-optimal resource usage and query performance.
|
|
||||||
|
|
||||||
|
```
|
||||||
|
{field1="value1", ..., fieldN="valueN"}
|
||||||
|
```
|
||||||
|
|
||||||
|
For example, if `host` and `app` fields are associated with the stream, then the `_stream` field will have `{host="host-123",app="my-app"}` value
|
||||||
|
for the log entry with `host="host-123"` and `app="my-app"` fields. The `_stream` field can be searched
|
||||||
|
with [stream filters](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter).
|
||||||
|
|
||||||
|
By default the value of `_stream` field is `{}`, since VictoriaLogs cannot determine automatically,
|
||||||
|
which fields uniquely identify every log stream. This may lead to not-so-optimal resource usage and query performance.
|
||||||
Therefore it is recommended specifying stream-level fields via `_stream_fields` query arg
|
Therefore it is recommended specifying stream-level fields via `_stream_fields` query arg
|
||||||
during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/).
|
during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/).
|
||||||
For example, if logs from Kubernetes containers have the following fields:
|
For example, if logs from Kubernetes containers have the following fields:
|
||||||
|
@ -175,20 +185,17 @@ per-container logs into distinct streams.
|
||||||
|
|
||||||
#### How to determine which fields must be associated with log streams?
|
#### How to determine which fields must be associated with log streams?
|
||||||
|
|
||||||
[Log streams](#stream-fields) can be associated with fields, which simultaneously meet the following conditions:
|
[Log streams](#stream-fields) must contain [fields](#data-model), which uniquely identify the application instance, which generates logs.
|
||||||
|
For example, `container`, `instance` and `host` are good candidates for stream fields.
|
||||||
|
|
||||||
- Fields, which remain constant across log entries received from a single application instance.
|
Additional fields may be added to log streams if they **remain constant during application instance lifetime**.
|
||||||
- Fields, which uniquely identify the application instance. For example, `instance`, `host`, `container`, etc.
|
For example, `namespace`, `node`, `pod` and `job` are good candidates for additional stream fields. Adding such fields to log streams
|
||||||
|
makes sense if you are going to use these fields during search and want speeding up it with [stream filters](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter).
|
||||||
|
|
||||||
Sometimes a single application instance may generate multiple log streams and store them into distinct log files.
|
There is **no need to add all the constant fields to log streams**, since this may increase resource usage during data ingestion and querying.
|
||||||
In this case it is OK to associate the log stream with filepath fields such as `log.file.path` additionally to instance-specific fields.
|
|
||||||
|
|
||||||
Structured logs may contain big number of fields, which do not change across log entries received from a single application instance.
|
**Never add non-nonstant fields to streams if these fields may change with every log entry of the same stream**.
|
||||||
There is no need in associating all these fields with log stream - it is enough to associate only those fields, which uniquely identify
|
For example, `ip`, `user_id` and `trace_id` **must never be associated with log streams**, since this may lead to [high cardinality issues](#high-cardinality).
|
||||||
the application instance across all the ingested logs. Additionally, some fields such as `datacenter`, `environment`, `namespace`, `job` or `app`,
|
|
||||||
can be associated with log stream in order to optimize searching by these fields with [stream filtering](https://docs.victoriametrics.com/victorialogs/logsql/#stream-filter).
|
|
||||||
|
|
||||||
Never associate log streams with fields, which may change across log entries of the same application instance. See [these docs](#high-cardinality) for details.
|
|
||||||
|
|
||||||
#### High cardinality
|
#### High cardinality
|
||||||
|
|
||||||
|
@ -196,8 +203,7 @@ Some fields in the [ingested logs](#data-model) may contain big number of unique
|
||||||
For example, fields with names such as `ip`, `user_id` or `trace_id` tend to contain big number of unique values.
|
For example, fields with names such as `ip`, `user_id` or `trace_id` tend to contain big number of unique values.
|
||||||
VictoriaLogs works perfectly with such fields unless they are associated with [log streams](#stream-fields).
|
VictoriaLogs works perfectly with such fields unless they are associated with [log streams](#stream-fields).
|
||||||
|
|
||||||
Never associate high-cardinality fields with [log streams](#stream-fields), since this may result
|
**Never** associate high-cardinality fields with [log streams](#stream-fields), since this may lead to the following issues:
|
||||||
to the following issues:
|
|
||||||
|
|
||||||
- Performance degradation during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/)
|
- Performance degradation during [data ingestion](https://docs.victoriametrics.com/victorialogs/data-ingestion/)
|
||||||
and [querying](https://docs.victoriametrics.com/victorialogs/querying/)
|
and [querying](https://docs.victoriametrics.com/victorialogs/querying/)
|
||||||
|
@ -214,9 +220,9 @@ This can help narrowing down and eliminating high-cardinality fields from [log s
|
||||||
|
|
||||||
### Other fields
|
### Other fields
|
||||||
|
|
||||||
The rest of [structured logging](#data-model) fields are optional. They can be used for simplifying and optimizing search queries.
|
Every ingested log entry may contain arbitrary number of [fields](#data-model) additionally to [`_msg`](#message-field) and [`_time`](#time-field).
|
||||||
For example, it is usually faster to search over a dedicated `trace_id` field instead of searching for the `trace_id` inside long log message.
|
For example, `level`, `ip`, `user_id`, `trace_id`, etc. Such fields can be used for simplifying and optimizing [search queries](#https://docs.victoriametrics.com/victorialogs/logsql/).
|
||||||
E.g. the `trace_id:XXXX-YYYY-ZZZZ` query usually works faster than the `_msg:"trace_id=XXXX-YYYY-ZZZZ"` query.
|
It is usually faster to search over a dedicated `trace_id` field instead of searching for the `trace_id` inside long [log message](#message-field).
|
||||||
|
E.g. the `trace_id:="XXXX-YYYY-ZZZZ"` query usually works faster than the `_msg:"trace_id=XXXX-YYYY-ZZZZ"` query.
|
||||||
|
|
||||||
See [LogsQL docs](https://docs.victoriametrics.com/victorialogs/logsql/) for more details.
|
See [LogsQL docs](https://docs.victoriametrics.com/victorialogs/logsql/) for more details.
|
||||||
|
|
||||||
|
|
|
@ -28,8 +28,8 @@ VictoriaLogs provides the following HTTP endpoints:
|
||||||
- [`/select/logsql/query`](#querying-logs) for querying logs
|
- [`/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/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/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_field_names`](#querying-stream-field-names) for querying [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field 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/stream_field_values`](#querying-stream-field-values) for querying [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field values
|
||||||
- [`/select/logsql/field_names`](#querying-field-names) for querying [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) names.
|
- [`/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.
|
- [`/select/logsql/field_values`](#querying-field-values) for querying [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) values.
|
||||||
|
|
||||||
|
@ -43,8 +43,8 @@ For example, the following query returns all the log entries with the `error` wo
|
||||||
curl http://localhost:9428/select/logsql/query -d 'query=error'
|
curl http://localhost:9428/select/logsql/query -d 'query=error'
|
||||||
```
|
```
|
||||||
|
|
||||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
The response by default contains all the [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) for the selected logs.
|
||||||
See [how to query specific fields](https://docs.victoriametrics.com/victorialogs/logsql/#querying-specific-fields).
|
Use [`fields` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#fields-pipe) for selecting only the needed fields.
|
||||||
|
|
||||||
The `query` argument can be passed either in the request url itself (aka HTTP GET request) or via request body
|
The `query` argument can be passed either in the request url itself (aka HTTP GET request) or via request body
|
||||||
with the `x-www-form-urlencoded` encoding (aka HTTP POST request). The HTTP POST is useful for sending long queries
|
with the `x-www-form-urlencoded` encoding (aka HTTP POST request). The HTTP POST is useful for sending long queries
|
||||||
|
@ -56,7 +56,8 @@ or similar tools.
|
||||||
|
|
||||||
By default the `/select/logsql/query` returns all the log entries matching the given `query`. The response size can be limited in the following ways:
|
By default the `/select/logsql/query` returns all the log entries matching the given `query`. The response size can be limited in the following ways:
|
||||||
|
|
||||||
- By closing the response stream at any time. In this case VictoriaLogs stops query execution and frees all the resources occupied by the request.
|
- By closing the response stream at any time. VictoriaLogs stops query execution and frees all the resources occupied by the request as soon as it detects closed client connection.
|
||||||
|
So it is safe running [`*` query](https://docs.victoriametrics.com/victorialogs/logsql/#any-value-filter), which selects all the logs, even if trillions of logs are stored in VictoriaLogs.
|
||||||
- By specifying the maximum number of log entries, which can be returned in the response via `limit` query arg. For example, the following request returns
|
- By specifying the maximum number of log entries, which can be returned in the response via `limit` query arg. For example, the following request returns
|
||||||
up to 10 matching log entries:
|
up to 10 matching log entries:
|
||||||
```sh
|
```sh
|
||||||
|
@ -68,7 +69,7 @@ By default the `/select/logsql/query` returns all the log entries matching the g
|
||||||
```
|
```
|
||||||
- By adding [`_time` filter](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter). The time range for the query can be specified via optional
|
- By adding [`_time` filter](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter). The time range for the query can be specified via optional
|
||||||
`start` and `end` query ars formatted according to [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#timestamp-formats).
|
`start` and `end` query ars formatted according to [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#timestamp-formats).
|
||||||
- By adding other [filters](https://docs.victoriametrics.com/victorialogs/logsql/#filters) to the query.
|
- By adding more specific [filters](https://docs.victoriametrics.com/victorialogs/logsql/#filters) to the query, which select lower number of logs.
|
||||||
|
|
||||||
The `/select/logsql/query` endpoint returns [a stream of JSON lines](https://jsonlines.org/),
|
The `/select/logsql/query` endpoint returns [a stream of JSON lines](https://jsonlines.org/),
|
||||||
where each line contains JSON-encoded log entry in the form `{field1="value1",...,fieldN="valueN"}`.
|
where each line contains JSON-encoded log entry in the form `{field1="value1",...,fieldN="valueN"}`.
|
||||||
|
@ -79,18 +80,18 @@ Example response:
|
||||||
{"_msg":"some other error","_stream":"{}","_time":"2023-01-01T13:32:15Z"}
|
{"_msg":"some other error","_stream":"{}","_time":"2023-01-01T13:32:15Z"}
|
||||||
```
|
```
|
||||||
|
|
||||||
The matching lines are sent to the response stream as soon as they are found in VictoriaLogs storage.
|
Logs lines are sent to the response stream as soon as they are found in VictoriaLogs storage.
|
||||||
This means that the returned response may contain billions of lines for queries matching too many log entries.
|
This means that the returned response may contain billions of lines for queries matching too many log entries.
|
||||||
The response can be interrupted at any time by closing the connection to VictoriaLogs server.
|
The response can be interrupted at any time by closing the connection to VictoriaLogs server.
|
||||||
This allows post-processing the returned lines at the client side with the usual Unix commands such as `grep`, `jq`, `less`, `head`, etc.
|
This allows post-processing the returned lines at the client side with the usual Unix commands such as `grep`, `jq`, `less`, `head`, etc.,
|
||||||
See [these docs](#command-line) for more details.
|
without worrying about resource usage at VictoriaLogs side. See [these docs](#command-line) for more details.
|
||||||
|
|
||||||
The returned lines aren't sorted, since sorting disables the ability to send matching log entries to response stream as soon as they are found.
|
The returned lines aren't sorted by default, since sorting disables the ability to send matching log entries to response stream as soon as they are found.
|
||||||
Query results can be sorted either at VictoriaLogs side according [to these docs](https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe)
|
Query results can be sorted either at VictoriaLogs side via [`sort` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe)
|
||||||
or at client side with the usual `sort` command according to [these docs](#command-line).
|
or at client side with the usual `sort` command according to [these docs](#command-line).
|
||||||
|
|
||||||
By default the `(AccountID=0, ProjectID=0)` [tenant](https://docs.victoriametrics.com/victorialogs/#multitenancy) is queried.
|
By default the `(AccountID=0, ProjectID=0)` [tenant](https://docs.victoriametrics.com/victorialogs/#multitenancy) is queried.
|
||||||
If you need querying other tenant, then specify the needed tenant via http request headers. For example, the following query searches
|
If you need querying other tenant, then specify it via `AccounID` and `ProjectID` http request headers. For example, the following query searches
|
||||||
for log messages at `(AccountID=12, ProjectID=34)` tenant:
|
for log messages at `(AccountID=12, ProjectID=34)` tenant:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
|
@ -100,14 +101,20 @@ 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)
|
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.
|
with `vl_http_requests_total{path="/select/logsql/query"}` metric.
|
||||||
|
|
||||||
|
See also:
|
||||||
|
|
||||||
- [Querying hits stats](#querying-hits-stats)
|
- [Querying hits stats](#querying-hits-stats)
|
||||||
- [Querying streams](#querying-streams)
|
- [Querying streams](#querying-streams)
|
||||||
- [HTTP API](#http-api)
|
- [Querying stream field names](#querying-stream-field-names)
|
||||||
|
- [Querying stream field values](#querying-stream-field-values)
|
||||||
|
- [Querying field names](#querying-field-names)
|
||||||
|
- [Querying field values](#querying-field-values)
|
||||||
|
|
||||||
|
|
||||||
### Querying hits stats
|
### Querying hits stats
|
||||||
|
|
||||||
VictoriaMetrics provides `/select/logsql/hits?query=<query>&start=<start>&end=<end>&step=<step>` HTTP endpoint, which returns the number
|
VictoriaMetrics provides `/select/logsql/hits?query=<query>&start=<start>&end=<end>&step=<step>` HTTP endpoint, which returns the number
|
||||||
of matching log entries for the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]`
|
of matching log entries for the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]`
|
||||||
time range grouped by `<step>` buckets. The returned results are sorted by time.
|
time range grouped by `<step>` buckets. The returned results are sorted by time.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
|
@ -210,7 +217,7 @@ See also:
|
||||||
### Querying streams
|
### Querying streams
|
||||||
|
|
||||||
VictoriaLogs provides `/select/logsql/streams?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns [streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields)
|
VictoriaLogs provides `/select/logsql/streams?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns [streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields)
|
||||||
from results of the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
from results of the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
||||||
The response also contains the number of log results per every `stream`.
|
The response also contains the number of log results per every `stream`.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
|
@ -254,22 +261,22 @@ See also:
|
||||||
- [Querying hits stats](#querying-hits-stats)
|
- [Querying hits stats](#querying-hits-stats)
|
||||||
- [HTTP API](#http-api)
|
- [HTTP API](#http-api)
|
||||||
|
|
||||||
### Querying stream label names
|
### Querying stream field names
|
||||||
|
|
||||||
VictoriaLogs provides `/select/logsql/stream_label_names?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns
|
VictoriaLogs provides `/select/logsql/stream_field_names?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns
|
||||||
[log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label names from results
|
[log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field names from results
|
||||||
of the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
of the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
||||||
The response also contains the number of log results per every label name.
|
The response also contains the number of log results per every field name.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
If `<start>` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs.
|
If `<start>` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs.
|
||||||
If `<end>` is missing, then it equals to the maximum timestamp across logs stored in VictoriaLogs.
|
If `<end>` 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 example, the following command returns stream field names across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word)
|
||||||
for the last 5 minutes:
|
for the last 5 minutes:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
curl http://localhost:9428/select/logsql/stream_label_names -d 'query=error' -d 'start=5m'
|
curl http://localhost:9428/select/logsql/stream_field_names -d 'query=error' -d 'start=5m'
|
||||||
```
|
```
|
||||||
|
|
||||||
Below is an example JSON output returned from this endpoint:
|
Below is an example JSON output returned from this endpoint:
|
||||||
|
@ -295,27 +302,27 @@ Below is an example JSON output returned from this endpoint:
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Querying stream label names](#querying-stream-label-names)
|
- [Querying stream field names](#querying-stream-field-names)
|
||||||
- [Querying field values](#querying-field-values)
|
- [Querying field values](#querying-field-values)
|
||||||
- [Querying streams](#querying-streams)
|
- [Querying streams](#querying-streams)
|
||||||
- [HTTP API](#http-api)
|
- [HTTP API](#http-api)
|
||||||
|
|
||||||
### Querying stream label values
|
### Querying stream field values
|
||||||
|
|
||||||
VictoriaLogs provides `/select/logsql/stream_label_values?query=<query>&start=<start>&<end>&label=<labelName>` HTTP endpoint,
|
VictoriaLogs provides `/select/logsql/stream_field_values?query=<query>&start=<start>&<end>&field=<fieldName>` HTTP endpoint,
|
||||||
which returns [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) label values for the label with the given `<labelName>` name
|
which returns [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) field values for the field with the given `<fieldName>` name
|
||||||
from results of the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
from results of the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
||||||
The response also contains the number of log results per every label value.
|
The response also contains the number of log results per every field value.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
If `<start>` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs.
|
If `<start>` is missing, then it equals to the minimum timestamp across logs stored in VictoriaLogs.
|
||||||
If `<end>` is missing, then it equals to the maximum timestamp across logs stored in VictoriaLogs.
|
If `<end>` 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 example, the following command returns values for the stream field `host` across logs with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word)
|
||||||
for the last 5 minutes:
|
for the last 5 minutes:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
curl http://localhost:9428/select/logsql/stream_label_values -d 'query=error' -d 'start=5m' -d 'label=host'
|
curl http://localhost:9428/select/logsql/stream_field_values -d 'query=error' -d 'start=5m' -d 'field=host'
|
||||||
```
|
```
|
||||||
|
|
||||||
Below is an example JSON output returned from this endpoint:
|
Below is an example JSON output returned from this endpoint:
|
||||||
|
@ -335,12 +342,12 @@ Below is an example JSON output returned from this endpoint:
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
The `/select/logsql/stream_label_names` endpoint supports optional `limit=N` query arg, which allows limiting the number of returned values to `N`.
|
The `/select/logsql/stream_field_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.
|
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:
|
See also:
|
||||||
|
|
||||||
- [Querying stream label values](#querying-stream-label-values)
|
- [Querying stream field values](#querying-stream-field-values)
|
||||||
- [Querying field names](#querying-field-names)
|
- [Querying field names](#querying-field-names)
|
||||||
- [Querying streams](#querying-streams)
|
- [Querying streams](#querying-streams)
|
||||||
- [HTTP API](#http-api)
|
- [HTTP API](#http-api)
|
||||||
|
@ -348,7 +355,7 @@ See also:
|
||||||
### Querying field names
|
### Querying field names
|
||||||
|
|
||||||
VictoriaLogs provides `/select/logsql/field_names?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns field names
|
VictoriaLogs provides `/select/logsql/field_names?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns field names
|
||||||
from results of the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
from results of the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
||||||
The response also contains the number of log results per every field name.
|
The response also contains the number of log results per every field name.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
|
@ -385,7 +392,7 @@ Below is an example JSON output returned from this endpoint:
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Querying stream label names](#querying-stream-label-names)
|
- [Querying stream field names](#querying-stream-field-names)
|
||||||
- [Querying field values](#querying-field-values)
|
- [Querying field values](#querying-field-values)
|
||||||
- [Querying streams](#querying-streams)
|
- [Querying streams](#querying-streams)
|
||||||
- [HTTP API](#http-api)
|
- [HTTP API](#http-api)
|
||||||
|
@ -394,7 +401,7 @@ See also:
|
||||||
|
|
||||||
VictoriaLogs provides `/select/logsql/field_values?query=<query>&field=<fieldName>&start=<start>&end=<end>` HTTP endpoint, which returns
|
VictoriaLogs provides `/select/logsql/field_values?query=<query>&field=<fieldName>&start=<start>&end=<end>` HTTP endpoint, which returns
|
||||||
unique values for the given `<fieldName>` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
unique values for the given `<fieldName>` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||||
from results of the given `<query>` [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
from results of the given [`<query>`](https://docs.victoriametrics.com/victorialogs/logsql/) on the given `[<start> ... <end>]` time range.
|
||||||
The response also contains the number of log results per every field value.
|
The response also contains the number of log results per every field value.
|
||||||
|
|
||||||
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
The `<start>` and `<end>` args can contain values in [any supported format](https://docs.victoriametrics.com/#timestamp-formats).
|
||||||
|
@ -435,7 +442,7 @@ When the `limit` is reached, `hits` are zeroed, since they cannot be calculated
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Querying stream label values](#querying-stream-label-values)
|
- [Querying stream field values](#querying-stream-field-values)
|
||||||
- [Querying field names](#querying-field-names)
|
- [Querying field names](#querying-field-names)
|
||||||
- [Querying streams](#querying-streams)
|
- [Querying streams](#querying-streams)
|
||||||
- [HTTP API](#http-api)
|
- [HTTP API](#http-api)
|
||||||
|
@ -454,32 +461,25 @@ There are three modes of displaying query results:
|
||||||
- `Table` - displays query results as a table.
|
- `Table` - displays query results as a table.
|
||||||
- `JSON` - displays raw JSON response from [HTTP API](#http-api).
|
- `JSON` - displays raw JSON response from [HTTP API](#http-api).
|
||||||
|
|
||||||
This is the first version that has minimal functionality. It comes with the following limitations:
|
This is the first version that has minimal functionality and may contain bugs.
|
||||||
|
It is recommended trying [command line interface](#command-line), which has no known bugs :)
|
||||||
- The number of query results is always limited to 1000 lines. Iteratively add
|
|
||||||
more specific [filters](https://docs.victoriametrics.com/victorialogs/logsql/#filters) to the query
|
|
||||||
in order to get full response with less than 1000 lines.
|
|
||||||
- Queries are always executed against [tenant](https://docs.victoriametrics.com/victorialogs/#multitenancy) `0`.
|
|
||||||
|
|
||||||
These limitations will be removed in future versions.
|
|
||||||
|
|
||||||
To get around the current limitations, you can use an alternative - the [command line interface](#command-line).
|
|
||||||
|
|
||||||
## Command-line
|
## Command-line
|
||||||
|
|
||||||
VictoriaLogs integrates well with `curl` and other command-line tools during querying because of the following features:
|
VictoriaLogs integrates well with `curl` and other command-line tools during querying because of the following features:
|
||||||
|
|
||||||
- VictoriaLogs sends the matching log entries to the response stream as soon as they are found.
|
- Matching log entries are sent to the response stream as soon as they are found.
|
||||||
This allows forwarding the response stream to arbitrary [Unix pipes](https://en.wikipedia.org/wiki/Pipeline_(Unix)).
|
This allows forwarding the response stream to arbitrary [Unix pipes](https://en.wikipedia.org/wiki/Pipeline_(Unix))
|
||||||
- VictoriaLogs automatically adjusts query execution speed to the speed of the client, which reads the response stream.
|
without waiting until the response finishes.
|
||||||
|
- Query execution speed is automatically adjusted to the speed of the client, which reads the response stream.
|
||||||
For example, if the response stream is piped to `less` command, then the query is suspended
|
For example, if the response stream is piped to `less` command, then the query is suspended
|
||||||
until the `less` command reads the next block from the response stream.
|
until the `less` command reads the next block from the response stream.
|
||||||
- VictoriaLogs automatically cancels query execution when the client closes the response stream.
|
- Query is automatically canceled when the client closes the response stream.
|
||||||
For example, if the query response is piped to `head` command, then VictoriaLogs stops executing the query
|
For example, if the query response is piped to `head` command, then VictoriaLogs stops executing the query
|
||||||
when the `head` command closes the response stream.
|
when the `head` command closes the response stream.
|
||||||
|
|
||||||
These features allow executing queries at command-line interface, which potentially select billions of rows,
|
These features allow executing queries at command-line interface, which potentially select billions of rows,
|
||||||
without the risk of high resource usage (CPU, RAM, disk IO) at VictoriaLogs server.
|
without the risk of high resource usage (CPU, RAM, disk IO) at VictoriaLogs.
|
||||||
|
|
||||||
For example, the following query can return very big number of matching log entries (e.g. billions) if VictoriaLogs contains
|
For example, the following query can return very big number of matching log entries (e.g. billions) if VictoriaLogs contains
|
||||||
many log messages with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word):
|
many log messages with the `error` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word):
|
||||||
|
@ -488,8 +488,8 @@ many log messages with the `error` [word](https://docs.victoriametrics.com/victo
|
||||||
curl http://localhost:9428/select/logsql/query -d 'query=error'
|
curl http://localhost:9428/select/logsql/query -d 'query=error'
|
||||||
```
|
```
|
||||||
|
|
||||||
If the command returns "never-ending" response, then just press `ctrl+C` at any time in order to cancel the query.
|
If the command above returns "never-ending" response, then just press `ctrl+C` at any time in order to cancel the query.
|
||||||
VictoriaLogs notices that the response stream is closed, so it cancels the query and instantly stops consuming CPU, RAM and disk IO for this query.
|
VictoriaLogs notices that the response stream is closed, so it cancels the query and stops consuming CPU, RAM and disk IO for this query.
|
||||||
|
|
||||||
Then just use `head` command for investigating the returned log messages and narrowing down the query:
|
Then just use `head` command for investigating the returned log messages and narrowing down the query:
|
||||||
|
|
||||||
|
@ -500,6 +500,12 @@ curl http://localhost:9428/select/logsql/query -d 'query=error' | head -10
|
||||||
The `head -10` command reads only the first 10 log messages from the response and then closes the response stream.
|
The `head -10` command reads only the first 10 log messages from the response and then closes the response stream.
|
||||||
This automatically cancels the query at VictoriaLogs side, so it stops consuming CPU, RAM and disk IO resources.
|
This automatically cancels the query at VictoriaLogs side, so it stops consuming CPU, RAM and disk IO resources.
|
||||||
|
|
||||||
|
Alternatively, you can limit the number of returned logs at VictoriaLogs side via [`limit` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#limit-pipe):
|
||||||
|
|
||||||
|
```sh
|
||||||
|
curl http://localhost:9428/select/logsql/query -d 'query=error | limit 10'
|
||||||
|
```
|
||||||
|
|
||||||
Sometimes it may be more convenient to use `less` command instead of `head` during the investigation of the returned response:
|
Sometimes it may be more convenient to use `less` command instead of `head` during the investigation of the returned response:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
|
@ -509,7 +515,7 @@ curl http://localhost:9428/select/logsql/query -d 'query=error' | less
|
||||||
The `less` command reads the response stream on demand, when the user scrolls down the output.
|
The `less` command reads the response stream on demand, when the user scrolls down the output.
|
||||||
VictoriaLogs suspends query execution when `less` stops reading the response stream.
|
VictoriaLogs suspends query execution when `less` stops reading the response stream.
|
||||||
It doesn't consume CPU and disk IO resources during this time. It resumes query execution
|
It doesn't consume CPU and disk IO resources during this time. It resumes query execution
|
||||||
when the `less` continues reading the response stream.
|
after the `less` continues reading the response stream.
|
||||||
|
|
||||||
Suppose that the initial investigation of the returned query results helped determining that the needed log messages contain
|
Suppose that the initial investigation of the returned query results helped determining that the needed log messages contain
|
||||||
`cannot open file` [phrase](https://docs.victoriametrics.com/victorialogs/logsql/#phrase-filter).
|
`cannot open file` [phrase](https://docs.victoriametrics.com/victorialogs/logsql/#phrase-filter).
|
||||||
|
@ -543,7 +549,13 @@ See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#stream-fi
|
||||||
[these docs](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter) about `_time` filter
|
[these docs](https://docs.victoriametrics.com/victorialogs/logsql/#time-filter) about `_time` filter
|
||||||
and [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#logical-filter) about `AND` operator.
|
and [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#logical-filter) about `AND` operator.
|
||||||
|
|
||||||
The following example shows how to sort query results by the [`_time` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field):
|
Alternatively, you can count the number of matching logs at VictoriaLogs side with [`stats` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#stats-pipe):
|
||||||
|
|
||||||
|
```sh
|
||||||
|
curl http://localhost:9428/select/logsql/query -d 'query=_stream:{app="nginx"} AND _time:5m AND error | stats count() logs_with_error'
|
||||||
|
```
|
||||||
|
|
||||||
|
The following example shows how to sort query results by the [`_time` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) with traditional Unix tools:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
curl http://localhost:9428/select/logsql/query -d 'query=error' | jq -r '._time + " " + ._msg' | sort | less
|
curl http://localhost:9428/select/logsql/query -d 'query=error' | jq -r '._time + " " + ._msg' | sort | less
|
||||||
|
@ -558,8 +570,14 @@ can take non-trivial amounts of time if the `query` returns too many results. Th
|
||||||
before sorting the results. See [these tips](https://docs.victoriametrics.com/victorialogs/logsql/#performance-tips)
|
before sorting the results. See [these tips](https://docs.victoriametrics.com/victorialogs/logsql/#performance-tips)
|
||||||
on how to narrow down query results.
|
on how to narrow down query results.
|
||||||
|
|
||||||
|
Alternatively, sorting of matching logs can be performed at VictoriaLogs side via [`sort` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe):
|
||||||
|
|
||||||
|
```sh
|
||||||
|
curl http://localhost:9428/select/logsql/query -d 'query=error | sort by (_time)' | less
|
||||||
|
```
|
||||||
|
|
||||||
The following example calculates stats on the number of log messages received during the last 5 minutes
|
The following example calculates stats on the number of log messages received during the last 5 minutes
|
||||||
grouped by `log.level` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model):
|
grouped by `log.level` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with traditional Unix tools:
|
||||||
|
|
||||||
```sh
|
```sh
|
||||||
curl http://localhost:9428/select/logsql/query -d 'query=_time:5m log.level:*' | jq -r '."log.level"' | sort | uniq -c
|
curl http://localhost:9428/select/logsql/query -d 'query=_time:5m log.level:*' | jq -r '."log.level"' | sort | uniq -c
|
||||||
|
@ -569,6 +587,12 @@ The query selects all the log messages with non-empty `log.level` field via ["an
|
||||||
then pipes them to `jq` command, which extracts the `log.level` field value from the returned JSON stream, then the extracted `log.level` values
|
then pipes them to `jq` command, which extracts the `log.level` field value from the returned JSON stream, then the extracted `log.level` values
|
||||||
are sorted with `sort` command and, finally, they are passed to `uniq -c` command for calculating the needed stats.
|
are sorted with `sort` command and, finally, they are passed to `uniq -c` command for calculating the needed stats.
|
||||||
|
|
||||||
|
Alternatively, all the stats calculations above can be performed at VictoriaLogs side via [`stats by(...)`](https://docs.victoriametrics.com/victorialogs/logsql/#stats-by-fields):
|
||||||
|
|
||||||
|
```sh
|
||||||
|
curl http://localhost:9428/select/logsql/query -d 'query=_time:5m log.level:* | stats by (log.level) count() matching_logs'
|
||||||
|
```
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
- [Key concepts](https://docs.victoriametrics.com/victorialogs/keyconcepts/).
|
- [Key concepts](https://docs.victoriametrics.com/victorialogs/keyconcepts/).
|
||||||
|
|
|
@ -31,11 +31,17 @@ type blockResult struct {
|
||||||
// csBuf contains requested columns.
|
// csBuf contains requested columns.
|
||||||
csBuf []blockResultColumn
|
csBuf []blockResultColumn
|
||||||
|
|
||||||
|
// csEmpty contains non-existing columns, which were referenced via getColumnByName()
|
||||||
|
csEmpty []blockResultColumn
|
||||||
|
|
||||||
// cs contains cached pointers to requested columns returned from getColumns() if csInitialized=true.
|
// cs contains cached pointers to requested columns returned from getColumns() if csInitialized=true.
|
||||||
cs []*blockResultColumn
|
cs []*blockResultColumn
|
||||||
|
|
||||||
// csInitialized is set to true if cs is properly initialized and can be returned from getColumns().
|
// csInitialized is set to true if cs is properly initialized and can be returned from getColumns().
|
||||||
csInitialized bool
|
csInitialized bool
|
||||||
|
|
||||||
|
fvecs []filteredValuesEncodedCreator
|
||||||
|
svecs []searchValuesEncodedCreator
|
||||||
}
|
}
|
||||||
|
|
||||||
func (br *blockResult) reset() {
|
func (br *blockResult) reset() {
|
||||||
|
@ -49,10 +55,19 @@ func (br *blockResult) reset() {
|
||||||
clear(br.csBuf)
|
clear(br.csBuf)
|
||||||
br.csBuf = br.csBuf[:0]
|
br.csBuf = br.csBuf[:0]
|
||||||
|
|
||||||
|
clear(br.csEmpty)
|
||||||
|
br.csEmpty = br.csEmpty[:0]
|
||||||
|
|
||||||
clear(br.cs)
|
clear(br.cs)
|
||||||
br.cs = br.cs[:0]
|
br.cs = br.cs[:0]
|
||||||
|
|
||||||
br.csInitialized = false
|
br.csInitialized = false
|
||||||
|
|
||||||
|
clear(br.fvecs)
|
||||||
|
br.fvecs = br.fvecs[:0]
|
||||||
|
|
||||||
|
clear(br.svecs)
|
||||||
|
br.svecs = br.svecs[:0]
|
||||||
}
|
}
|
||||||
|
|
||||||
// clone returns a clone of br, which owns its own data.
|
// clone returns a clone of br, which owns its own data.
|
||||||
|
@ -88,6 +103,10 @@ func (br *blockResult) clone() *blockResult {
|
||||||
}
|
}
|
||||||
brNew.csBuf = csNew
|
brNew.csBuf = csNew
|
||||||
|
|
||||||
|
// do not clone br.csEmpty - it will be populated by the caller via getColumnByName().
|
||||||
|
|
||||||
|
// do not clone br.fvecs and br.svecs, since they may point to external data.
|
||||||
|
|
||||||
return brNew
|
return brNew
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -128,6 +147,9 @@ func (br *blockResult) initFromFilterNeededColumns(brSrc *blockResult, bm *bitma
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// appendFilteredColumn adds cSrc with the given bm filter to br.
|
||||||
|
//
|
||||||
|
// the br is valid until brSrc, cSrc or bm is updated.
|
||||||
func (br *blockResult) appendFilteredColumn(brSrc *blockResult, cSrc *blockResultColumn, bm *bitmap) {
|
func (br *blockResult) appendFilteredColumn(brSrc *blockResult, cSrc *blockResultColumn, bm *bitmap) {
|
||||||
if len(br.timestamps) == 0 {
|
if len(br.timestamps) == 0 {
|
||||||
return
|
return
|
||||||
|
@ -146,24 +168,37 @@ func (br *blockResult) appendFilteredColumn(brSrc *blockResult, cSrc *blockResul
|
||||||
cDst.minValue = cSrc.minValue
|
cDst.minValue = cSrc.minValue
|
||||||
cDst.maxValue = cSrc.maxValue
|
cDst.maxValue = cSrc.maxValue
|
||||||
cDst.dictValues = cSrc.dictValues
|
cDst.dictValues = cSrc.dictValues
|
||||||
cDst.newValuesEncodedFunc = func(br *blockResult) []string {
|
br.fvecs = append(br.fvecs, filteredValuesEncodedCreator{
|
||||||
valuesEncodedSrc := cSrc.getValuesEncoded(brSrc)
|
br: brSrc,
|
||||||
|
c: cSrc,
|
||||||
valuesBuf := br.valuesBuf
|
bm: bm,
|
||||||
valuesBufLen := len(valuesBuf)
|
})
|
||||||
bm.forEachSetBitReadonly(func(idx int) {
|
cDst.valuesEncodedCreator = &br.fvecs[len(br.fvecs)-1]
|
||||||
valuesBuf = append(valuesBuf, valuesEncodedSrc[idx])
|
|
||||||
})
|
|
||||||
br.valuesBuf = valuesBuf
|
|
||||||
|
|
||||||
return valuesBuf[valuesBufLen:]
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
br.csBuf = append(br.csBuf, cDst)
|
br.csBuf = append(br.csBuf, cDst)
|
||||||
br.csInitialized = false
|
br.csInitialized = false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type filteredValuesEncodedCreator struct {
|
||||||
|
br *blockResult
|
||||||
|
c *blockResultColumn
|
||||||
|
bm *bitmap
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fvec *filteredValuesEncodedCreator) newValuesEncoded(br *blockResult) []string {
|
||||||
|
valuesEncodedSrc := fvec.c.getValuesEncoded(fvec.br)
|
||||||
|
|
||||||
|
valuesBuf := br.valuesBuf
|
||||||
|
valuesBufLen := len(valuesBuf)
|
||||||
|
fvec.bm.forEachSetBitReadonly(func(idx int) {
|
||||||
|
valuesBuf = append(valuesBuf, valuesEncodedSrc[idx])
|
||||||
|
})
|
||||||
|
br.valuesBuf = valuesBuf
|
||||||
|
|
||||||
|
return valuesBuf[valuesBufLen:]
|
||||||
|
}
|
||||||
|
|
||||||
// cloneValues clones the given values into br and returns the cloned values.
|
// cloneValues clones the given values into br and returns the cloned values.
|
||||||
func (br *blockResult) cloneValues(values []string) []string {
|
func (br *blockResult) cloneValues(values []string) []string {
|
||||||
if values == nil {
|
if values == nil {
|
||||||
|
@ -287,6 +322,8 @@ func (br *blockResult) initAllColumns(bs *blockSearch, bm *bitmap) {
|
||||||
br.addColumn(bs, bm, ch)
|
br.addColumn(bs, bm, ch)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
br.csInitFast()
|
||||||
}
|
}
|
||||||
|
|
||||||
// initRequestedColumns initialized only requested columns in br according to bs and bm.
|
// initRequestedColumns initialized only requested columns in br according to bs and bm.
|
||||||
|
@ -314,6 +351,8 @@ func (br *blockResult) initRequestedColumns(bs *blockSearch, bm *bitmap) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
br.csInitFast()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (br *blockResult) mustInit(bs *blockSearch, bm *bitmap) {
|
func (br *blockResult) mustInit(bs *blockSearch, bm *bitmap) {
|
||||||
|
@ -433,13 +472,28 @@ func (br *blockResult) addColumn(bs *blockSearch, bm *bitmap, ch *columnHeader)
|
||||||
minValue: ch.minValue,
|
minValue: ch.minValue,
|
||||||
maxValue: ch.maxValue,
|
maxValue: ch.maxValue,
|
||||||
dictValues: ch.valuesDict.values,
|
dictValues: ch.valuesDict.values,
|
||||||
newValuesEncodedFunc: func(br *blockResult) []string {
|
|
||||||
return br.newValuesEncodedFromColumnHeader(bs, bm, ch)
|
|
||||||
},
|
|
||||||
})
|
})
|
||||||
|
c := &br.csBuf[len(br.csBuf)-1]
|
||||||
|
|
||||||
|
br.svecs = append(br.svecs, searchValuesEncodedCreator{
|
||||||
|
bs: bs,
|
||||||
|
bm: bm,
|
||||||
|
ch: ch,
|
||||||
|
})
|
||||||
|
c.valuesEncodedCreator = &br.svecs[len(br.svecs)-1]
|
||||||
br.csInitialized = false
|
br.csInitialized = false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type searchValuesEncodedCreator struct {
|
||||||
|
bs *blockSearch
|
||||||
|
bm *bitmap
|
||||||
|
ch *columnHeader
|
||||||
|
}
|
||||||
|
|
||||||
|
func (svec *searchValuesEncodedCreator) newValuesEncoded(br *blockResult) []string {
|
||||||
|
return br.newValuesEncodedFromColumnHeader(svec.bs, svec.bm, svec.ch)
|
||||||
|
}
|
||||||
|
|
||||||
func (br *blockResult) addTimeColumn() {
|
func (br *blockResult) addTimeColumn() {
|
||||||
br.csBuf = append(br.csBuf, blockResultColumn{
|
br.csBuf = append(br.csBuf, blockResultColumn{
|
||||||
name: "_time",
|
name: "_time",
|
||||||
|
@ -1325,15 +1379,31 @@ func (br *blockResult) getColumnByName(columnName string) *blockResultColumn {
|
||||||
return cs[idx]
|
return cs[idx]
|
||||||
}
|
}
|
||||||
|
|
||||||
br.addConstColumn(columnName, "")
|
// Search for empty column with the given name
|
||||||
return &br.csBuf[len(br.csBuf)-1]
|
csEmpty := br.csEmpty
|
||||||
|
for i := range csEmpty {
|
||||||
|
if csEmpty[i].name == columnName {
|
||||||
|
return &csEmpty[i]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create missing empty column
|
||||||
|
br.csEmpty = append(br.csEmpty, blockResultColumn{
|
||||||
|
name: br.a.copyString(columnName),
|
||||||
|
isConst: true,
|
||||||
|
valuesEncoded: getEmptyStrings(1),
|
||||||
|
})
|
||||||
|
return &br.csEmpty[len(br.csEmpty)-1]
|
||||||
}
|
}
|
||||||
|
|
||||||
func (br *blockResult) getColumns() []*blockResultColumn {
|
func (br *blockResult) getColumns() []*blockResultColumn {
|
||||||
if br.csInitialized {
|
if !br.csInitialized {
|
||||||
return br.cs
|
br.csInit()
|
||||||
}
|
}
|
||||||
|
return br.cs
|
||||||
|
}
|
||||||
|
|
||||||
|
func (br *blockResult) csInit() {
|
||||||
csBuf := br.csBuf
|
csBuf := br.csBuf
|
||||||
clear(br.cs)
|
clear(br.cs)
|
||||||
cs := br.cs[:0]
|
cs := br.cs[:0]
|
||||||
|
@ -1348,8 +1418,17 @@ func (br *blockResult) getColumns() []*blockResultColumn {
|
||||||
}
|
}
|
||||||
br.cs = cs
|
br.cs = cs
|
||||||
br.csInitialized = true
|
br.csInitialized = true
|
||||||
|
}
|
||||||
|
|
||||||
return br.cs
|
func (br *blockResult) csInitFast() {
|
||||||
|
csBuf := br.csBuf
|
||||||
|
clear(br.cs)
|
||||||
|
cs := slicesutil.SetLength(br.cs, len(csBuf))
|
||||||
|
for i := range csBuf {
|
||||||
|
cs[i] = &csBuf[i]
|
||||||
|
}
|
||||||
|
br.cs = cs
|
||||||
|
br.csInitialized = true
|
||||||
}
|
}
|
||||||
|
|
||||||
func getBlockResultColumnIdxByName(cs []*blockResultColumn, name string) int {
|
func getBlockResultColumnIdxByName(cs []*blockResultColumn, name string) int {
|
||||||
|
@ -1444,10 +1523,10 @@ type blockResultColumn struct {
|
||||||
// valuesBucketed contains values after getValuesBucketed() call
|
// valuesBucketed contains values after getValuesBucketed() call
|
||||||
valuesBucketed []string
|
valuesBucketed []string
|
||||||
|
|
||||||
// newValuesEncodedFunc must return valuesEncoded.
|
// valuesEncodedCreator must return valuesEncoded.
|
||||||
//
|
//
|
||||||
// This func must be set for non-const and non-time columns if valuesEncoded field isn't set.
|
// This interface must be set for non-const and non-time columns if valuesEncoded field isn't set.
|
||||||
newValuesEncodedFunc func(br *blockResult) []string
|
valuesEncodedCreator columnValuesEncodedCreator
|
||||||
|
|
||||||
// bucketSizeStr contains bucketSizeStr for valuesBucketed
|
// bucketSizeStr contains bucketSizeStr for valuesBucketed
|
||||||
bucketSizeStr string
|
bucketSizeStr string
|
||||||
|
@ -1456,6 +1535,11 @@ type blockResultColumn struct {
|
||||||
bucketOffsetStr string
|
bucketOffsetStr string
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// columnValuesEncodedCreator must return encoded values for the current column.
|
||||||
|
type columnValuesEncodedCreator interface {
|
||||||
|
newValuesEncoded(br *blockResult) []string
|
||||||
|
}
|
||||||
|
|
||||||
// clone returns a clone of c backed by data from br.
|
// clone returns a clone of c backed by data from br.
|
||||||
//
|
//
|
||||||
// It is expected that c.valuesEncoded is already initialized for non-time column.
|
// It is expected that c.valuesEncoded is already initialized for non-time column.
|
||||||
|
@ -1484,8 +1568,8 @@ func (c *blockResultColumn) clone(br *blockResult) blockResultColumn {
|
||||||
}
|
}
|
||||||
cNew.valuesBucketed = br.cloneValues(c.valuesBucketed)
|
cNew.valuesBucketed = br.cloneValues(c.valuesBucketed)
|
||||||
|
|
||||||
// Do not copy c.newValuesEncodedFunc, since it may refer to data, which may change over time.
|
// Do not copy c.valuesEncodedCreator, since it may refer to data, which may change over time.
|
||||||
// We already copied c.valuesEncoded, so cNew.newValuesEncodedFunc must be nil.
|
// We already copied c.valuesEncoded, so cNew.valuesEncodedCreator must be nil.
|
||||||
|
|
||||||
cNew.bucketSizeStr = c.bucketSizeStr
|
cNew.bucketSizeStr = c.bucketSizeStr
|
||||||
cNew.bucketOffsetStr = c.bucketOffsetStr
|
cNew.bucketOffsetStr = c.bucketOffsetStr
|
||||||
|
@ -1579,7 +1663,7 @@ func (c *blockResultColumn) getValuesEncoded(br *blockResult) []string {
|
||||||
}
|
}
|
||||||
|
|
||||||
if c.valuesEncoded == nil {
|
if c.valuesEncoded == nil {
|
||||||
c.valuesEncoded = c.newValuesEncodedFunc(br)
|
c.valuesEncoded = c.valuesEncodedCreator.newValuesEncoded(br)
|
||||||
}
|
}
|
||||||
return c.valuesEncoded
|
return c.valuesEncoded
|
||||||
}
|
}
|
||||||
|
|
|
@ -321,23 +321,10 @@ func (q *Query) Optimize() {
|
||||||
|
|
||||||
// Call Optimize for queries from 'in(query)' filters.
|
// Call Optimize for queries from 'in(query)' filters.
|
||||||
optimizeFilterIn(q.f)
|
optimizeFilterIn(q.f)
|
||||||
|
|
||||||
|
// Optimize individual pipes.
|
||||||
for _, p := range q.pipes {
|
for _, p := range q.pipes {
|
||||||
switch t := p.(type) {
|
p.optimize()
|
||||||
case *pipeStats:
|
|
||||||
for _, f := range t.funcs {
|
|
||||||
f.iff.optimizeFilterIn()
|
|
||||||
}
|
|
||||||
case *pipeReplace:
|
|
||||||
t.iff.optimizeFilterIn()
|
|
||||||
case *pipeFormat:
|
|
||||||
t.iff.optimizeFilterIn()
|
|
||||||
case *pipeExtract:
|
|
||||||
t.iff.optimizeFilterIn()
|
|
||||||
case *pipeUnpackJSON:
|
|
||||||
t.iff.optimizeFilterIn()
|
|
||||||
case *pipeUnpackLogfmt:
|
|
||||||
t.iff.optimizeFilterIn()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -11,15 +11,26 @@ type pipe interface {
|
||||||
// updateNeededFields must update neededFields and unneededFields with fields it needs and not needs at the input.
|
// updateNeededFields must update neededFields and unneededFields with fields it needs and not needs at the input.
|
||||||
updateNeededFields(neededFields, unneededFields fieldsSet)
|
updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
|
|
||||||
// newPipeProcessor must return new pipeProcessor for the given ppBase.
|
// newPipeProcessor must return new pipeProcessor, which writes data to the given ppNext.
|
||||||
//
|
//
|
||||||
// workersCount is the number of goroutine workers, which will call writeBlock() method.
|
// workersCount is the number of goroutine workers, which will call writeBlock() method.
|
||||||
//
|
//
|
||||||
// If stopCh is closed, the returned pipeProcessor must stop performing CPU-intensive tasks which take more than a few milliseconds.
|
// If stopCh is closed, the returned pipeProcessor must stop performing CPU-intensive tasks which take more than a few milliseconds.
|
||||||
// It is OK to continue processing pipeProcessor calls if they take less than a few milliseconds.
|
// It is OK to continue processing pipeProcessor calls if they take less than a few milliseconds.
|
||||||
//
|
//
|
||||||
// The returned pipeProcessor may call cancel() at any time in order to notify worker goroutines to stop sending new data to pipeProcessor.
|
// The returned pipeProcessor may call cancel() at any time in order to notify the caller to stop sending new data to it.
|
||||||
newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor
|
newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor
|
||||||
|
|
||||||
|
// optimize must optimize the pipe
|
||||||
|
optimize()
|
||||||
|
|
||||||
|
// hasFilterInWithQuery must return true of pipe contains 'in(subquery)' filter (recursively).
|
||||||
|
hasFilterInWithQuery() bool
|
||||||
|
|
||||||
|
// initFilterInValues must return new pipe with the initialized values for 'in(subquery)' filters (recursively).
|
||||||
|
//
|
||||||
|
// It is OK to return the pipe itself if it doesn't contain 'in(subquery)' filters.
|
||||||
|
initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
// pipeProcessor must process a single pipe.
|
// pipeProcessor must process a single pipe.
|
||||||
|
@ -39,7 +50,7 @@ type pipeProcessor interface {
|
||||||
// cancel() may be called also when the pipeProcessor decides to stop accepting new data, even if there is no any error.
|
// cancel() may be called also when the pipeProcessor decides to stop accepting new data, even if there is no any error.
|
||||||
writeBlock(workerID uint, br *blockResult)
|
writeBlock(workerID uint, br *blockResult)
|
||||||
|
|
||||||
// flush must flush all the data accumulated in the pipeProcessor to the base pipeProcessor.
|
// flush must flush all the data accumulated in the pipeProcessor to the next pipeProcessor.
|
||||||
//
|
//
|
||||||
// flush is called after all the worker goroutines are stopped.
|
// flush is called after all the worker goroutines are stopped.
|
||||||
//
|
//
|
||||||
|
@ -135,6 +146,12 @@ func parsePipe(lex *lexer) (pipe, error) {
|
||||||
return nil, fmt.Errorf("cannot parse 'offset' pipe: %w", err)
|
return nil, fmt.Errorf("cannot parse 'offset' pipe: %w", err)
|
||||||
}
|
}
|
||||||
return ps, nil
|
return ps, nil
|
||||||
|
case lex.isKeyword("pack_json"):
|
||||||
|
pp, err := parsePackJSON(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'pack_json' pipe: %w", err)
|
||||||
|
}
|
||||||
|
return pp, nil
|
||||||
case lex.isKeyword("rename", "mv"):
|
case lex.isKeyword("rename", "mv"):
|
||||||
pr, err := parsePipeRename(lex)
|
pr, err := parsePipeRename(lex)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -147,6 +164,12 @@ func parsePipe(lex *lexer) (pipe, error) {
|
||||||
return nil, fmt.Errorf("cannot parse 'replace' pipe: %w", err)
|
return nil, fmt.Errorf("cannot parse 'replace' pipe: %w", err)
|
||||||
}
|
}
|
||||||
return pr, nil
|
return pr, nil
|
||||||
|
case lex.isKeyword("replace_regexp"):
|
||||||
|
pr, err := parsePipeReplaceRegexp(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'replace_regexp' pipe: %w", err)
|
||||||
|
}
|
||||||
|
return pr, nil
|
||||||
case lex.isKeyword("sort"):
|
case lex.isKeyword("sort"):
|
||||||
ps, err := parsePipeSort(lex)
|
ps, err := parsePipeSort(lex)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -177,6 +200,12 @@ func parsePipe(lex *lexer) (pipe, error) {
|
||||||
return nil, fmt.Errorf("cannot parse 'unpack_logfmt' pipe: %w", err)
|
return nil, fmt.Errorf("cannot parse 'unpack_logfmt' pipe: %w", err)
|
||||||
}
|
}
|
||||||
return pu, nil
|
return pu, nil
|
||||||
|
case lex.isKeyword("unroll"):
|
||||||
|
pu, err := parsePipeUnroll(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'unroll' pipe: %w", err)
|
||||||
|
}
|
||||||
|
return pu, nil
|
||||||
default:
|
default:
|
||||||
return nil, fmt.Errorf("unexpected pipe %q", lex.token)
|
return nil, fmt.Errorf("unexpected pipe %q", lex.token)
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,16 +50,28 @@ func (pc *pipeCopy) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pc *pipeCopy) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pc *pipeCopy) optimize() {
|
||||||
|
// Nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pc *pipeCopy) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pc *pipeCopy) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pc, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pc *pipeCopy) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeCopyProcessor{
|
return &pipeCopyProcessor{
|
||||||
pc: pc,
|
pc: pc,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeCopyProcessor struct {
|
type pipeCopyProcessor struct {
|
||||||
pc *pipeCopy
|
pc *pipeCopy
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pcp *pipeCopyProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pcp *pipeCopyProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -68,7 +80,7 @@ func (pcp *pipeCopyProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
br.copyColumns(pcp.pc.srcFields, pcp.pc.dstFields)
|
br.copyColumns(pcp.pc.srcFields, pcp.pc.dstFields)
|
||||||
pcp.ppBase.writeBlock(workerID, br)
|
pcp.ppNext.writeBlock(workerID, br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pcp *pipeCopyProcessor) flush() error {
|
func (pcp *pipeCopyProcessor) flush() error {
|
||||||
|
|
|
@ -32,16 +32,28 @@ func (pd *pipeDelete) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pd *pipeDelete) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pd *pipeDelete) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pd *pipeDelete) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pd *pipeDelete) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pd, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pd *pipeDelete) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeDeleteProcessor{
|
return &pipeDeleteProcessor{
|
||||||
pd: pd,
|
pd: pd,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeDeleteProcessor struct {
|
type pipeDeleteProcessor struct {
|
||||||
pd *pipeDelete
|
pd *pipeDelete
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pdp *pipeDeleteProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pdp *pipeDeleteProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -50,7 +62,7 @@ func (pdp *pipeDeleteProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
br.deleteColumns(pdp.pd.fields)
|
br.deleteColumns(pdp.pd.fields)
|
||||||
pdp.ppBase.writeBlock(workerID, br)
|
pdp.ppNext.writeBlock(workerID, br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pdp *pipeDeleteProcessor) flush() error {
|
func (pdp *pipeDeleteProcessor) flush() error {
|
||||||
|
|
|
@ -2,6 +2,9 @@ package logstorage
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
// pipeExtract processes '| extract ...' pipe.
|
// pipeExtract processes '| extract ...' pipe.
|
||||||
|
@ -38,6 +41,24 @@ func (pe *pipeExtract) String() string {
|
||||||
return s
|
return s
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (pe *pipeExtract) optimize() {
|
||||||
|
pe.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pe *pipeExtract) hasFilterInWithQuery() bool {
|
||||||
|
return pe.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pe *pipeExtract) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pe.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
peNew := *pe
|
||||||
|
peNew.iff = iffNew
|
||||||
|
return &peNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
func (pe *pipeExtract) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
func (pe *pipeExtract) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
if neededFields.contains("*") {
|
if neededFields.contains("*") {
|
||||||
unneededFieldsOrig := unneededFields.clone()
|
unneededFieldsOrig := unneededFields.clone()
|
||||||
|
@ -80,21 +101,129 @@ func (pe *pipeExtract) updateNeededFields(neededFields, unneededFields fieldsSet
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pe *pipeExtract) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pe *pipeExtract) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
patterns := make([]*pattern, workersCount)
|
return &pipeExtractProcessor{
|
||||||
for i := range patterns {
|
pe: pe,
|
||||||
patterns[i] = pe.ptn.clone()
|
ppNext: ppNext,
|
||||||
|
|
||||||
|
shards: make([]pipeExtractProcessorShard, workersCount),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeExtractProcessor struct {
|
||||||
|
pe *pipeExtract
|
||||||
|
ppNext 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 {
|
||||||
|
bm bitmap
|
||||||
|
ptn *pattern
|
||||||
|
|
||||||
|
resultColumns []*blockResultColumn
|
||||||
|
resultValues []string
|
||||||
|
|
||||||
|
rcs []resultColumn
|
||||||
|
a arena
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pep *pipeExtractProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
if len(br.timestamps) == 0 {
|
||||||
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
unpackFunc := func(uctx *fieldsUnpackerContext, s string) {
|
pe := pep.pe
|
||||||
ptn := patterns[uctx.workerID]
|
shard := &pep.shards[workerID]
|
||||||
ptn.apply(s)
|
|
||||||
for _, f := range ptn.fields {
|
bm := &shard.bm
|
||||||
uctx.addField(f.name, *f.value)
|
bm.init(len(br.timestamps))
|
||||||
|
bm.setBits()
|
||||||
|
if iff := pe.iff; iff != nil {
|
||||||
|
iff.f.applyToBlockResult(br, bm)
|
||||||
|
if bm.isZero() {
|
||||||
|
pep.ppNext.writeBlock(workerID, br)
|
||||||
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return newPipeUnpackProcessor(workersCount, unpackFunc, ppBase, pe.fromField, "", pe.keepOriginalFields, pe.skipEmptyResults, pe.iff)
|
if shard.ptn == nil {
|
||||||
|
shard.ptn = pe.ptn.clone()
|
||||||
|
}
|
||||||
|
ptn := shard.ptn
|
||||||
|
|
||||||
|
shard.rcs = slicesutil.SetLength(shard.rcs, len(ptn.fields))
|
||||||
|
rcs := shard.rcs
|
||||||
|
for i := range ptn.fields {
|
||||||
|
rcs[i].name = ptn.fields[i].name
|
||||||
|
}
|
||||||
|
|
||||||
|
c := br.getColumnByName(pe.fromField)
|
||||||
|
values := c.getValues(br)
|
||||||
|
|
||||||
|
shard.resultColumns = slicesutil.SetLength(shard.resultColumns, len(rcs))
|
||||||
|
resultColumns := shard.resultColumns
|
||||||
|
for i := range resultColumns {
|
||||||
|
resultColumns[i] = br.getColumnByName(rcs[i].name)
|
||||||
|
}
|
||||||
|
|
||||||
|
shard.resultValues = slicesutil.SetLength(shard.resultValues, len(rcs))
|
||||||
|
resultValues := shard.resultValues
|
||||||
|
|
||||||
|
hadUpdates := false
|
||||||
|
vPrev := ""
|
||||||
|
for rowIdx, v := range values {
|
||||||
|
if bm.isSetBit(rowIdx) {
|
||||||
|
if !hadUpdates || vPrev != v {
|
||||||
|
vPrev = v
|
||||||
|
hadUpdates = true
|
||||||
|
|
||||||
|
ptn.apply(v)
|
||||||
|
|
||||||
|
for i, f := range ptn.fields {
|
||||||
|
v := *f.value
|
||||||
|
if v == "" && pe.skipEmptyResults || pe.keepOriginalFields {
|
||||||
|
c := resultColumns[i]
|
||||||
|
if vOrig := c.getValueAtRow(br, rowIdx); vOrig != "" {
|
||||||
|
v = vOrig
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
v = shard.a.copyString(v)
|
||||||
|
}
|
||||||
|
resultValues[i] = v
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
for i, c := range resultColumns {
|
||||||
|
resultValues[i] = c.getValueAtRow(br, rowIdx)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for i, v := range resultValues {
|
||||||
|
rcs[i].addValue(v)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for i := range rcs {
|
||||||
|
br.addResultColumn(&rcs[i])
|
||||||
|
}
|
||||||
|
pep.ppNext.writeBlock(workerID, br)
|
||||||
|
|
||||||
|
for i := range rcs {
|
||||||
|
rcs[i].reset()
|
||||||
|
}
|
||||||
|
shard.a.reset()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pep *pipeExtractProcessor) flush() error {
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeExtract(lex *lexer) (*pipeExtract, error) {
|
func parsePipeExtract(lex *lexer) (*pipeExtract, error) {
|
||||||
|
|
|
@ -37,13 +37,25 @@ func (pf *pipeFieldNames) updateNeededFields(neededFields, unneededFields fields
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFieldNames) newPipeProcessor(workersCount int, stopCh <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pf *pipeFieldNames) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFieldNames) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFieldNames) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pf, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFieldNames) newPipeProcessor(workersCount int, stopCh <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
shards := make([]pipeFieldNamesProcessorShard, workersCount)
|
shards := make([]pipeFieldNamesProcessorShard, workersCount)
|
||||||
|
|
||||||
pfp := &pipeFieldNamesProcessor{
|
pfp := &pipeFieldNamesProcessor{
|
||||||
pf: pf,
|
pf: pf,
|
||||||
stopCh: stopCh,
|
stopCh: stopCh,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
}
|
}
|
||||||
|
@ -53,7 +65,7 @@ func (pf *pipeFieldNames) newPipeProcessor(workersCount int, stopCh <-chan struc
|
||||||
type pipeFieldNamesProcessor struct {
|
type pipeFieldNamesProcessor struct {
|
||||||
pf *pipeFieldNames
|
pf *pipeFieldNames
|
||||||
stopCh <-chan struct{}
|
stopCh <-chan struct{}
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeFieldNamesProcessorShard
|
shards []pipeFieldNamesProcessorShard
|
||||||
}
|
}
|
||||||
|
@ -172,10 +184,10 @@ func (wctx *pipeFieldNamesWriteContext) flush() {
|
||||||
|
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppNext
|
||||||
br.setResultColumns(wctx.rcs[:], wctx.rowsCount)
|
br.setResultColumns(wctx.rcs[:], wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.pfp.ppBase.writeBlock(0, br)
|
wctx.pfp.ppNext.writeBlock(0, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
wctx.rcs[0].resetValues()
|
wctx.rcs[0].resetValues()
|
||||||
wctx.rcs[1].resetValues()
|
wctx.rcs[1].resetValues()
|
||||||
|
|
|
@ -49,16 +49,28 @@ func (pf *pipeFields) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
unneededFields.reset()
|
unneededFields.reset()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFields) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pf *pipeFields) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFields) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFields) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pf, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFields) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeFieldsProcessor{
|
return &pipeFieldsProcessor{
|
||||||
pf: pf,
|
pf: pf,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeFieldsProcessor struct {
|
type pipeFieldsProcessor struct {
|
||||||
pf *pipeFields
|
pf *pipeFields
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFieldsProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pfp *pipeFieldsProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -69,7 +81,7 @@ func (pfp *pipeFieldsProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
if !pfp.pf.containsStar {
|
if !pfp.pf.containsStar {
|
||||||
br.setColumns(pfp.pf.fields)
|
br.setColumns(pfp.pf.fields)
|
||||||
}
|
}
|
||||||
pfp.ppBase.writeBlock(workerID, br)
|
pfp.ppNext.writeBlock(workerID, br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFieldsProcessor) flush() error {
|
func (pfp *pipeFieldsProcessor) flush() error {
|
||||||
|
|
|
@ -29,12 +29,30 @@ func (pf *pipeFilter) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFilter) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pf *pipeFilter) optimize() {
|
||||||
|
optimizeFilterIn(pf.f)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFilter) hasFilterInWithQuery() bool {
|
||||||
|
return hasFilterInWithQueryForFilter(pf.f)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFilter) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
fNew, err := initFilterInValuesForFilter(cache, pf.f, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
pfNew := *pf
|
||||||
|
pf.f = fNew
|
||||||
|
return &pfNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFilter) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
shards := make([]pipeFilterProcessorShard, workersCount)
|
shards := make([]pipeFilterProcessorShard, workersCount)
|
||||||
|
|
||||||
pfp := &pipeFilterProcessor{
|
pfp := &pipeFilterProcessor{
|
||||||
pf: pf,
|
pf: pf,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
}
|
}
|
||||||
|
@ -43,7 +61,7 @@ func (pf *pipeFilter) newPipeProcessor(workersCount int, _ <-chan struct{}, _ fu
|
||||||
|
|
||||||
type pipeFilterProcessor struct {
|
type pipeFilterProcessor struct {
|
||||||
pf *pipeFilter
|
pf *pipeFilter
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeFilterProcessorShard
|
shards []pipeFilterProcessorShard
|
||||||
}
|
}
|
||||||
|
@ -72,8 +90,8 @@ func (pfp *pipeFilterProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
bm.setBits()
|
bm.setBits()
|
||||||
pfp.pf.f.applyToBlockResult(br, bm)
|
pfp.pf.f.applyToBlockResult(br, bm)
|
||||||
if bm.areAllBitsSet() {
|
if bm.areAllBitsSet() {
|
||||||
// Fast path - the filter didn't filter out anything - send br to the base pipe as is.
|
// Fast path - the filter didn't filter out anything - send br to the next pipe as is.
|
||||||
pfp.ppBase.writeBlock(workerID, br)
|
pfp.ppNext.writeBlock(workerID, br)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if bm.isZero() {
|
if bm.isZero() {
|
||||||
|
@ -81,9 +99,9 @@ func (pfp *pipeFilterProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// Slow path - copy the remaining rows from br to shard.br before sending them to base pipe.
|
// Slow path - copy the remaining rows from br to shard.br before sending them to the next pipe.
|
||||||
shard.br.initFromFilterAllColumns(br, bm)
|
shard.br.initFromFilterAllColumns(br, bm)
|
||||||
pfp.ppBase.writeBlock(workerID, &shard.br)
|
pfp.ppNext.writeBlock(workerID, &shard.br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFilterProcessor) flush() error {
|
func (pfp *pipeFilterProcessor) flush() error {
|
||||||
|
|
|
@ -4,8 +4,6 @@ import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"strconv"
|
"strconv"
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// pipeFormat processes '| format ...' pipe.
|
// pipeFormat processes '| format ...' pipe.
|
||||||
|
@ -74,10 +72,28 @@ func (pf *pipeFormat) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFormat) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pf *pipeFormat) optimize() {
|
||||||
|
pf.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFormat) hasFilterInWithQuery() bool {
|
||||||
|
return pf.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFormat) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pf.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
pfNew := *pf
|
||||||
|
pfNew.iff = iffNew
|
||||||
|
return &pfNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pf *pipeFormat) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeFormatProcessor{
|
return &pipeFormatProcessor{
|
||||||
pf: pf,
|
pf: pf,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: make([]pipeFormatProcessorShard, workersCount),
|
shards: make([]pipeFormatProcessorShard, workersCount),
|
||||||
}
|
}
|
||||||
|
@ -85,7 +101,7 @@ func (pf *pipeFormat) newPipeProcessor(workersCount int, _ <-chan struct{}, _ fu
|
||||||
|
|
||||||
type pipeFormatProcessor struct {
|
type pipeFormatProcessor struct {
|
||||||
pf *pipeFormat
|
pf *pipeFormat
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeFormatProcessorShard
|
shards []pipeFormatProcessorShard
|
||||||
}
|
}
|
||||||
|
@ -100,8 +116,8 @@ type pipeFormatProcessorShard struct {
|
||||||
type pipeFormatProcessorShardNopad struct {
|
type pipeFormatProcessorShardNopad struct {
|
||||||
bm bitmap
|
bm bitmap
|
||||||
|
|
||||||
uctx fieldsUnpackerContext
|
a arena
|
||||||
wctx pipeUnpackWriteContext
|
rc resultColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFormatProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pfp *pipeFormatProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -110,39 +126,49 @@ func (pfp *pipeFormatProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
shard := &pfp.shards[workerID]
|
shard := &pfp.shards[workerID]
|
||||||
shard.wctx.init(workerID, pfp.ppBase, pfp.pf.keepOriginalFields, pfp.pf.skipEmptyResults, br)
|
pf := pfp.pf
|
||||||
shard.uctx.init(workerID, "")
|
|
||||||
|
|
||||||
bm := &shard.bm
|
bm := &shard.bm
|
||||||
bm.init(len(br.timestamps))
|
bm.init(len(br.timestamps))
|
||||||
bm.setBits()
|
bm.setBits()
|
||||||
if iff := pfp.pf.iff; iff != nil {
|
if iff := pf.iff; iff != nil {
|
||||||
iff.f.applyToBlockResult(br, bm)
|
iff.f.applyToBlockResult(br, bm)
|
||||||
if bm.isZero() {
|
if bm.isZero() {
|
||||||
pfp.ppBase.writeBlock(workerID, br)
|
pfp.ppNext.writeBlock(workerID, br)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
shard.rc.name = pf.resultField
|
||||||
|
|
||||||
|
resultColumn := br.getColumnByName(pf.resultField)
|
||||||
for rowIdx := range br.timestamps {
|
for rowIdx := range br.timestamps {
|
||||||
|
v := ""
|
||||||
if bm.isSetBit(rowIdx) {
|
if bm.isSetBit(rowIdx) {
|
||||||
shard.formatRow(pfp.pf, br, rowIdx)
|
v = shard.formatRow(pf, br, rowIdx)
|
||||||
shard.wctx.writeRow(rowIdx, shard.uctx.fields)
|
if v == "" && pf.skipEmptyResults || pf.keepOriginalFields {
|
||||||
|
if vOrig := resultColumn.getValueAtRow(br, rowIdx); vOrig != "" {
|
||||||
|
v = vOrig
|
||||||
|
}
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
shard.wctx.writeRow(rowIdx, nil)
|
v = resultColumn.getValueAtRow(br, rowIdx)
|
||||||
}
|
}
|
||||||
|
shard.rc.addValue(v)
|
||||||
}
|
}
|
||||||
|
|
||||||
shard.wctx.flush()
|
br.addResultColumn(&shard.rc)
|
||||||
shard.wctx.reset()
|
pfp.ppNext.writeBlock(workerID, br)
|
||||||
shard.uctx.reset()
|
|
||||||
|
shard.a.reset()
|
||||||
|
shard.rc.reset()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFormatProcessor) flush() error {
|
func (pfp *pipeFormatProcessor) flush() error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (shard *pipeFormatProcessorShard) formatRow(pf *pipeFormat, br *blockResult, rowIdx int) {
|
func (shard *pipeFormatProcessorShard) formatRow(pf *pipeFormat, br *blockResult, rowIdx int) string {
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
b := bb.B
|
b := bb.B
|
||||||
for _, step := range pf.steps {
|
for _, step := range pf.steps {
|
||||||
|
@ -159,10 +185,9 @@ func (shard *pipeFormatProcessorShard) formatRow(pf *pipeFormat, br *blockResult
|
||||||
}
|
}
|
||||||
bb.B = b
|
bb.B = b
|
||||||
|
|
||||||
s := bytesutil.ToUnsafeString(b)
|
v := shard.a.copyBytesToString(b)
|
||||||
shard.uctx.resetFields()
|
|
||||||
shard.uctx.addField(pf.resultField, s)
|
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
|
return v
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeFormat(lex *lexer) (*pipeFormat, error) {
|
func parsePipeFormat(lex *lexer) (*pipeFormat, error) {
|
||||||
|
|
|
@ -17,9 +17,22 @@ func (pl *pipeLimit) String() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pl *pipeLimit) updateNeededFields(_, _ fieldsSet) {
|
func (pl *pipeLimit) updateNeededFields(_, _ fieldsSet) {
|
||||||
|
// nothing to do
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pl *pipeLimit) newPipeProcessor(_ int, _ <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func (pl *pipeLimit) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pl *pipeLimit) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pl *pipeLimit) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pl, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pl *pipeLimit) newPipeProcessor(_ int, _ <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
if pl.limit == 0 {
|
if pl.limit == 0 {
|
||||||
// Special case - notify the caller to stop writing data to the returned pipeLimitProcessor
|
// Special case - notify the caller to stop writing data to the returned pipeLimitProcessor
|
||||||
cancel()
|
cancel()
|
||||||
|
@ -27,14 +40,14 @@ func (pl *pipeLimit) newPipeProcessor(_ int, _ <-chan struct{}, cancel func(), p
|
||||||
return &pipeLimitProcessor{
|
return &pipeLimitProcessor{
|
||||||
pl: pl,
|
pl: pl,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeLimitProcessor struct {
|
type pipeLimitProcessor struct {
|
||||||
pl *pipeLimit
|
pl *pipeLimit
|
||||||
cancel func()
|
cancel func()
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
rowsProcessed atomic.Uint64
|
rowsProcessed atomic.Uint64
|
||||||
}
|
}
|
||||||
|
@ -46,8 +59,8 @@ func (plp *pipeLimitProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
|
||||||
rowsProcessed := plp.rowsProcessed.Add(uint64(len(br.timestamps)))
|
rowsProcessed := plp.rowsProcessed.Add(uint64(len(br.timestamps)))
|
||||||
if rowsProcessed <= plp.pl.limit {
|
if rowsProcessed <= plp.pl.limit {
|
||||||
// Fast path - write all the rows to ppBase.
|
// Fast path - write all the rows to ppNext.
|
||||||
plp.ppBase.writeBlock(workerID, br)
|
plp.ppNext.writeBlock(workerID, br)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -61,7 +74,7 @@ func (plp *pipeLimitProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
// Write remaining rows.
|
// Write remaining rows.
|
||||||
keepRows := plp.pl.limit - rowsProcessed
|
keepRows := plp.pl.limit - rowsProcessed
|
||||||
br.truncateRows(int(keepRows))
|
br.truncateRows(int(keepRows))
|
||||||
plp.ppBase.writeBlock(workerID, br)
|
plp.ppNext.writeBlock(workerID, br)
|
||||||
|
|
||||||
// Notify the caller that it should stop passing more data to writeBlock().
|
// Notify the caller that it should stop passing more data to writeBlock().
|
||||||
plp.cancel()
|
plp.cancel()
|
||||||
|
|
|
@ -17,18 +17,31 @@ func (po *pipeOffset) String() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (po *pipeOffset) updateNeededFields(_, _ fieldsSet) {
|
func (po *pipeOffset) updateNeededFields(_, _ fieldsSet) {
|
||||||
|
// nothing to do
|
||||||
}
|
}
|
||||||
|
|
||||||
func (po *pipeOffset) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (po *pipeOffset) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (po *pipeOffset) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (po *pipeOffset) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return po, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (po *pipeOffset) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeOffsetProcessor{
|
return &pipeOffsetProcessor{
|
||||||
po: po,
|
po: po,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeOffsetProcessor struct {
|
type pipeOffsetProcessor struct {
|
||||||
po *pipeOffset
|
po *pipeOffset
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
rowsProcessed atomic.Uint64
|
rowsProcessed atomic.Uint64
|
||||||
}
|
}
|
||||||
|
@ -45,13 +58,13 @@ func (pop *pipeOffsetProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
|
||||||
rowsProcessed -= uint64(len(br.timestamps))
|
rowsProcessed -= uint64(len(br.timestamps))
|
||||||
if rowsProcessed >= pop.po.offset {
|
if rowsProcessed >= pop.po.offset {
|
||||||
pop.ppBase.writeBlock(workerID, br)
|
pop.ppNext.writeBlock(workerID, br)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
rowsSkip := pop.po.offset - rowsProcessed
|
rowsSkip := pop.po.offset - rowsProcessed
|
||||||
br.skipRows(int(rowsSkip))
|
br.skipRows(int(rowsSkip))
|
||||||
pop.ppBase.writeBlock(workerID, br)
|
pop.ppNext.writeBlock(workerID, br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pop *pipeOffsetProcessor) flush() error {
|
func (pop *pipeOffsetProcessor) flush() error {
|
||||||
|
|
140
lib/logstorage/pipe_pack_json.go
Normal file
140
lib/logstorage/pipe_pack_json.go
Normal file
|
@ -0,0 +1,140 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||||
|
)
|
||||||
|
|
||||||
|
// pipePackJSON processes '| pack_json ...' pipe.
|
||||||
|
//
|
||||||
|
// See https://docs.victoriametrics.com/victorialogs/logsql/#pack_json-pipe
|
||||||
|
type pipePackJSON struct {
|
||||||
|
resultField string
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) String() string {
|
||||||
|
s := "pack_json"
|
||||||
|
if !isMsgFieldName(pp.resultField) {
|
||||||
|
s += " as " + quoteTokenIfNeeded(pp.resultField)
|
||||||
|
}
|
||||||
|
return s
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
|
if neededFields.contains("*") {
|
||||||
|
if !unneededFields.contains(pp.resultField) {
|
||||||
|
unneededFields.reset()
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if neededFields.contains(pp.resultField) {
|
||||||
|
neededFields.add("*")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pp, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *pipePackJSON) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
|
return &pipePackJSONProcessor{
|
||||||
|
pp: pp,
|
||||||
|
ppNext: ppNext,
|
||||||
|
|
||||||
|
shards: make([]pipePackJSONProcessorShard, workersCount),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipePackJSONProcessor struct {
|
||||||
|
pp *pipePackJSON
|
||||||
|
ppNext pipeProcessor
|
||||||
|
|
||||||
|
shards []pipePackJSONProcessorShard
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipePackJSONProcessorShard struct {
|
||||||
|
pipePackJSONProcessorShardNopad
|
||||||
|
|
||||||
|
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||||
|
_ [128 - unsafe.Sizeof(pipePackJSONProcessorShardNopad{})%128]byte
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipePackJSONProcessorShardNopad struct {
|
||||||
|
rc resultColumn
|
||||||
|
|
||||||
|
buf []byte
|
||||||
|
fields []Field
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ppp *pipePackJSONProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
if len(br.timestamps) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
shard := &ppp.shards[workerID]
|
||||||
|
|
||||||
|
shard.rc.name = ppp.pp.resultField
|
||||||
|
|
||||||
|
cs := br.getColumns()
|
||||||
|
|
||||||
|
buf := shard.buf[:0]
|
||||||
|
fields := shard.fields
|
||||||
|
for rowIdx := range br.timestamps {
|
||||||
|
fields = fields[:0]
|
||||||
|
for _, c := range cs {
|
||||||
|
v := c.getValueAtRow(br, rowIdx)
|
||||||
|
fields = append(fields, Field{
|
||||||
|
Name: c.name,
|
||||||
|
Value: v,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
bufLen := len(buf)
|
||||||
|
buf = marshalFieldsToJSON(buf, fields)
|
||||||
|
v := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||||
|
shard.rc.addValue(v)
|
||||||
|
}
|
||||||
|
|
||||||
|
br.addResultColumn(&shard.rc)
|
||||||
|
ppp.ppNext.writeBlock(workerID, br)
|
||||||
|
|
||||||
|
shard.rc.reset()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ppp *pipePackJSONProcessor) flush() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func parsePackJSON(lex *lexer) (*pipePackJSON, error) {
|
||||||
|
if !lex.isKeyword("pack_json") {
|
||||||
|
return nil, fmt.Errorf("unexpected token: %q; want %q", lex.token, "pack_json")
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
|
||||||
|
// 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 for 'pack_json': %w", err)
|
||||||
|
}
|
||||||
|
resultField = field
|
||||||
|
}
|
||||||
|
|
||||||
|
pp := &pipePackJSON{
|
||||||
|
resultField: resultField,
|
||||||
|
}
|
||||||
|
|
||||||
|
return pp, nil
|
||||||
|
}
|
101
lib/logstorage/pipe_pack_json_test.go
Normal file
101
lib/logstorage/pipe_pack_json_test.go
Normal file
|
@ -0,0 +1,101 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestParsePipePackJSONSuccess(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeSuccess(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`pack_json`)
|
||||||
|
f(`pack_json as x`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestParsePipePackJSONFailure(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeFailure(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`pack_json foo bar`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipePackJSON(t *testing.T) {
|
||||||
|
f := func(pipeStr string, rows, rowsExpected [][]Field) {
|
||||||
|
t.Helper()
|
||||||
|
expectPipeResults(t, pipeStr, rows, rowsExpected)
|
||||||
|
}
|
||||||
|
|
||||||
|
// pack into _msg
|
||||||
|
f(`pack_json`, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", "x"},
|
||||||
|
{"foo", `abc`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `{"_msg":"x","foo":"abc","bar":"cde"}`},
|
||||||
|
{"foo", `abc`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"_msg", `{"a":"b","c":"d"}`},
|
||||||
|
{"a", "b"},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// pack into other field
|
||||||
|
f(`pack_json as a`, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", "x"},
|
||||||
|
{"foo", `abc`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `x`},
|
||||||
|
{"foo", `abc`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
{"a", `{"_msg":"x","foo":"abc","bar":"cde"}`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `{"a":"b","c":"d"}`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipePackJSONUpdateNeededFields(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(`pack_json as x`, "*", "", "*", "")
|
||||||
|
|
||||||
|
// unneeded fields do not intersect with output
|
||||||
|
f(`pack_json as x`, "*", "f1,f2", "*", "")
|
||||||
|
|
||||||
|
// unneeded fields intersect with output
|
||||||
|
f(`pack_json as f1`, "*", "f1,f2", "*", "f1,f2")
|
||||||
|
|
||||||
|
// needed fields do not intersect with output
|
||||||
|
f(`pack_json f1`, "x,y", "", "x,y", "")
|
||||||
|
|
||||||
|
// needed fields intersect with output
|
||||||
|
f(`pack_json as f2`, "f2,y", "", "*", "")
|
||||||
|
}
|
|
@ -54,16 +54,28 @@ func (pr *pipeRename) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pr *pipeRename) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pr *pipeRename) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeRename) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeRename) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pr, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeRename) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeRenameProcessor{
|
return &pipeRenameProcessor{
|
||||||
pr: pr,
|
pr: pr,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeRenameProcessor struct {
|
type pipeRenameProcessor struct {
|
||||||
pr *pipeRename
|
pr *pipeRename
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
}
|
}
|
||||||
|
|
||||||
func (prp *pipeRenameProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (prp *pipeRenameProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -72,7 +84,7 @@ func (prp *pipeRenameProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
br.renameColumns(prp.pr.srcFields, prp.pr.dstFields)
|
br.renameColumns(prp.pr.srcFields, prp.pr.dstFields)
|
||||||
prp.ppBase.writeBlock(workerID, br)
|
prp.ppNext.writeBlock(workerID, br)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (prp *pipeRenameProcessor) flush() error {
|
func (prp *pipeRenameProcessor) flush() error {
|
||||||
|
|
|
@ -3,16 +3,13 @@ package logstorage
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"strings"
|
"strings"
|
||||||
"unsafe"
|
|
||||||
|
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// pipeReplace processes '| replace ...' pipe.
|
// pipeReplace processes '| replace ...' pipe.
|
||||||
//
|
//
|
||||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#replace-pipe
|
// See https://docs.victoriametrics.com/victorialogs/logsql/#replace-pipe
|
||||||
type pipeReplace struct {
|
type pipeReplace struct {
|
||||||
srcField string
|
field string
|
||||||
oldSubstr string
|
oldSubstr string
|
||||||
newSubstr string
|
newSubstr string
|
||||||
|
|
||||||
|
@ -29,8 +26,8 @@ func (pr *pipeReplace) String() string {
|
||||||
s += " " + pr.iff.String()
|
s += " " + pr.iff.String()
|
||||||
}
|
}
|
||||||
s += fmt.Sprintf(" (%s, %s)", quoteTokenIfNeeded(pr.oldSubstr), quoteTokenIfNeeded(pr.newSubstr))
|
s += fmt.Sprintf(" (%s, %s)", quoteTokenIfNeeded(pr.oldSubstr), quoteTokenIfNeeded(pr.newSubstr))
|
||||||
if pr.srcField != "_msg" {
|
if pr.field != "_msg" {
|
||||||
s += " at " + quoteTokenIfNeeded(pr.srcField)
|
s += " at " + quoteTokenIfNeeded(pr.field)
|
||||||
}
|
}
|
||||||
if pr.limit > 0 {
|
if pr.limit > 0 {
|
||||||
s += fmt.Sprintf(" limit %d", pr.limit)
|
s += fmt.Sprintf(" limit %d", pr.limit)
|
||||||
|
@ -39,97 +36,37 @@ func (pr *pipeReplace) String() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pr *pipeReplace) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
func (pr *pipeReplace) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
if neededFields.contains("*") {
|
updateNeededFieldsForUpdatePipe(neededFields, unneededFields, pr.field, pr.iff)
|
||||||
if !unneededFields.contains(pr.srcField) && pr.iff != nil {
|
}
|
||||||
unneededFields.removeFields(pr.iff.neededFields)
|
|
||||||
}
|
func (pr *pipeReplace) optimize() {
|
||||||
} else {
|
pr.iff.optimizeFilterIn()
|
||||||
if neededFields.contains(pr.srcField) && pr.iff != nil {
|
}
|
||||||
neededFields.addFields(pr.iff.neededFields)
|
|
||||||
}
|
func (pr *pipeReplace) hasFilterInWithQuery() bool {
|
||||||
|
return pr.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplace) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pr.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
}
|
}
|
||||||
|
peNew := *pr
|
||||||
|
peNew.iff = iffNew
|
||||||
|
return &peNew, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pr *pipeReplace) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pr *pipeReplace) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
return &pipeReplaceProcessor{
|
updateFunc := func(a *arena, v string) string {
|
||||||
pr: pr,
|
bb := bbPool.Get()
|
||||||
ppBase: ppBase,
|
bb.B = appendReplace(bb.B[:0], v, pr.oldSubstr, pr.newSubstr, pr.limit)
|
||||||
|
result := a.copyBytesToString(bb.B)
|
||||||
shards: make([]pipeReplaceProcessorShard, workersCount),
|
bbPool.Put(bb)
|
||||||
}
|
return result
|
||||||
}
|
|
||||||
|
|
||||||
type pipeReplaceProcessor struct {
|
|
||||||
pr *pipeReplace
|
|
||||||
ppBase pipeProcessor
|
|
||||||
|
|
||||||
shards []pipeReplaceProcessorShard
|
|
||||||
}
|
|
||||||
|
|
||||||
type pipeReplaceProcessorShard struct {
|
|
||||||
pipeReplaceProcessorShardNopad
|
|
||||||
|
|
||||||
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
|
||||||
_ [128 - unsafe.Sizeof(pipeReplaceProcessorShardNopad{})%128]byte
|
|
||||||
}
|
|
||||||
|
|
||||||
type pipeReplaceProcessorShardNopad struct {
|
|
||||||
bm bitmap
|
|
||||||
|
|
||||||
uctx fieldsUnpackerContext
|
|
||||||
wctx pipeUnpackWriteContext
|
|
||||||
}
|
|
||||||
|
|
||||||
func (prp *pipeReplaceProcessor) writeBlock(workerID uint, br *blockResult) {
|
|
||||||
if len(br.timestamps) == 0 {
|
|
||||||
return
|
|
||||||
}
|
}
|
||||||
|
|
||||||
shard := &prp.shards[workerID]
|
return newPipeUpdateProcessor(workersCount, updateFunc, ppNext, pr.field, pr.iff)
|
||||||
shard.wctx.init(workerID, prp.ppBase, false, false, br)
|
|
||||||
shard.uctx.init(workerID, "")
|
|
||||||
|
|
||||||
pr := prp.pr
|
|
||||||
bm := &shard.bm
|
|
||||||
bm.init(len(br.timestamps))
|
|
||||||
bm.setBits()
|
|
||||||
if iff := pr.iff; iff != nil {
|
|
||||||
iff.f.applyToBlockResult(br, bm)
|
|
||||||
if bm.isZero() {
|
|
||||||
prp.ppBase.writeBlock(workerID, br)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
c := br.getColumnByName(pr.srcField)
|
|
||||||
values := c.getValues(br)
|
|
||||||
|
|
||||||
bb := bbPool.Get()
|
|
||||||
vPrev := ""
|
|
||||||
shard.uctx.addField(pr.srcField, "")
|
|
||||||
for rowIdx, v := range values {
|
|
||||||
if bm.isSetBit(rowIdx) {
|
|
||||||
if vPrev != v {
|
|
||||||
bb.B = appendReplace(bb.B[:0], v, pr.oldSubstr, pr.newSubstr, pr.limit)
|
|
||||||
s := bytesutil.ToUnsafeString(bb.B)
|
|
||||||
shard.uctx.resetFields()
|
|
||||||
shard.uctx.addField(pr.srcField, s)
|
|
||||||
vPrev = v
|
|
||||||
}
|
|
||||||
shard.wctx.writeRow(rowIdx, shard.uctx.fields)
|
|
||||||
} else {
|
|
||||||
shard.wctx.writeRow(rowIdx, nil)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
bbPool.Put(bb)
|
|
||||||
|
|
||||||
shard.wctx.flush()
|
|
||||||
shard.wctx.reset()
|
|
||||||
shard.uctx.reset()
|
|
||||||
}
|
|
||||||
|
|
||||||
func (prp *pipeReplaceProcessor) flush() error {
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeReplace(lex *lexer) (*pipeReplace, error) {
|
func parsePipeReplace(lex *lexer) (*pipeReplace, error) {
|
||||||
|
@ -164,7 +101,7 @@ func parsePipeReplace(lex *lexer) (*pipeReplace, error) {
|
||||||
|
|
||||||
newSubstr, err := getCompoundToken(lex)
|
newSubstr, err := getCompoundToken(lex)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, fmt.Errorf("cannot parse newSubstr in 'replace': %w", err)
|
return nil, fmt.Errorf("cannot parse newSubstr in 'replace(%q': %w", oldSubstr, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
if !lex.isKeyword(")") {
|
if !lex.isKeyword(")") {
|
||||||
|
@ -172,14 +109,14 @@ func parsePipeReplace(lex *lexer) (*pipeReplace, error) {
|
||||||
}
|
}
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
|
|
||||||
srcField := "_msg"
|
field := "_msg"
|
||||||
if lex.isKeyword("at") {
|
if lex.isKeyword("at") {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
f, err := parseFieldName(lex)
|
f, err := parseFieldName(lex)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, fmt.Errorf("cannot parse 'at' field after 'replace(%q, %q)': %w", oldSubstr, newSubstr, err)
|
return nil, fmt.Errorf("cannot parse 'at' field after 'replace(%q, %q)': %w", oldSubstr, newSubstr, err)
|
||||||
}
|
}
|
||||||
srcField = f
|
field = f
|
||||||
}
|
}
|
||||||
|
|
||||||
limit := uint64(0)
|
limit := uint64(0)
|
||||||
|
@ -194,7 +131,7 @@ func parsePipeReplace(lex *lexer) (*pipeReplace, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
pr := &pipeReplace{
|
pr := &pipeReplace{
|
||||||
srcField: srcField,
|
field: field,
|
||||||
oldSubstr: oldSubstr,
|
oldSubstr: oldSubstr,
|
||||||
newSubstr: newSubstr,
|
newSubstr: newSubstr,
|
||||||
limit: limit,
|
limit: limit,
|
||||||
|
|
170
lib/logstorage/pipe_replace_regexp.go
Normal file
170
lib/logstorage/pipe_replace_regexp.go
Normal file
|
@ -0,0 +1,170 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"regexp"
|
||||||
|
)
|
||||||
|
|
||||||
|
// pipeReplaceRegexp processes '| replace_regexp ...' pipe.
|
||||||
|
//
|
||||||
|
// See https://docs.victoriametrics.com/victorialogs/logsql/#replace_regexp-pipe
|
||||||
|
type pipeReplaceRegexp struct {
|
||||||
|
field string
|
||||||
|
re *regexp.Regexp
|
||||||
|
replacement string
|
||||||
|
|
||||||
|
// limit limits the number of replacements, which can be performed
|
||||||
|
limit uint64
|
||||||
|
|
||||||
|
// iff is an optional filter for skipping the replace_regexp operation
|
||||||
|
iff *ifFilter
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) String() string {
|
||||||
|
s := "replace_regexp"
|
||||||
|
if pr.iff != nil {
|
||||||
|
s += " " + pr.iff.String()
|
||||||
|
}
|
||||||
|
s += fmt.Sprintf(" (%s, %s)", quoteTokenIfNeeded(pr.re.String()), quoteTokenIfNeeded(pr.replacement))
|
||||||
|
if pr.field != "_msg" {
|
||||||
|
s += " at " + quoteTokenIfNeeded(pr.field)
|
||||||
|
}
|
||||||
|
if pr.limit > 0 {
|
||||||
|
s += fmt.Sprintf(" limit %d", pr.limit)
|
||||||
|
}
|
||||||
|
return s
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
|
updateNeededFieldsForUpdatePipe(neededFields, unneededFields, pr.field, pr.iff)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) optimize() {
|
||||||
|
pr.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) hasFilterInWithQuery() bool {
|
||||||
|
return pr.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pr.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
peNew := *pr
|
||||||
|
peNew.iff = iffNew
|
||||||
|
return &peNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pr *pipeReplaceRegexp) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
|
updateFunc := func(a *arena, v string) string {
|
||||||
|
bb := bbPool.Get()
|
||||||
|
bb.B = appendReplaceRegexp(bb.B[:0], v, pr.re, pr.replacement, pr.limit)
|
||||||
|
result := a.copyBytesToString(bb.B)
|
||||||
|
bbPool.Put(bb)
|
||||||
|
return result
|
||||||
|
}
|
||||||
|
|
||||||
|
return newPipeUpdateProcessor(workersCount, updateFunc, ppNext, pr.field, pr.iff)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
func parsePipeReplaceRegexp(lex *lexer) (*pipeReplaceRegexp, error) {
|
||||||
|
if !lex.isKeyword("replace_regexp") {
|
||||||
|
return nil, fmt.Errorf("unexpected token: %q; want %q", lex.token, "replace_regexp")
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
|
||||||
|
// parse optional if (...)
|
||||||
|
var iff *ifFilter
|
||||||
|
if lex.isKeyword("if") {
|
||||||
|
f, err := parseIfFilter(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
iff = f
|
||||||
|
}
|
||||||
|
|
||||||
|
if !lex.isKeyword("(") {
|
||||||
|
return nil, fmt.Errorf("missing '(' after 'replace_regexp'")
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
|
||||||
|
reStr, err := getCompoundToken(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse reStr in 'replace_regexp': %w", err)
|
||||||
|
}
|
||||||
|
re, err := regexp.Compile(reStr)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse regexp %q in 'replace_regexp': %w", reStr, err)
|
||||||
|
}
|
||||||
|
if !lex.isKeyword(",") {
|
||||||
|
return nil, fmt.Errorf("missing ',' after 'replace_regexp(%q'", reStr)
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
|
||||||
|
replacement, err := getCompoundToken(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse replacement in 'replace_regexp(%q': %w", reStr, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if !lex.isKeyword(")") {
|
||||||
|
return nil, fmt.Errorf("missing ')' after 'replace_regexp(%q, %q'", reStr, replacement)
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
|
||||||
|
field := "_msg"
|
||||||
|
if lex.isKeyword("at") {
|
||||||
|
lex.nextToken()
|
||||||
|
f, err := parseFieldName(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'at' field after 'replace_regexp(%q, %q)': %w", reStr, replacement, err)
|
||||||
|
}
|
||||||
|
field = f
|
||||||
|
}
|
||||||
|
|
||||||
|
limit := uint64(0)
|
||||||
|
if lex.isKeyword("limit") {
|
||||||
|
lex.nextToken()
|
||||||
|
n, ok := tryParseUint64(lex.token)
|
||||||
|
if !ok {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'limit %s' in 'replace_regexp'", lex.token)
|
||||||
|
}
|
||||||
|
lex.nextToken()
|
||||||
|
limit = n
|
||||||
|
}
|
||||||
|
|
||||||
|
pr := &pipeReplaceRegexp{
|
||||||
|
field: field,
|
||||||
|
re: re,
|
||||||
|
replacement: replacement,
|
||||||
|
limit: limit,
|
||||||
|
iff: iff,
|
||||||
|
}
|
||||||
|
|
||||||
|
return pr, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func appendReplaceRegexp(dst []byte, s string, re *regexp.Regexp, replacement string, limit uint64) []byte {
|
||||||
|
if len(s) == 0 {
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
|
||||||
|
replacements := uint64(0)
|
||||||
|
for {
|
||||||
|
locs := re.FindStringSubmatchIndex(s)
|
||||||
|
if locs == nil {
|
||||||
|
return append(dst, s...)
|
||||||
|
}
|
||||||
|
start := locs[0]
|
||||||
|
dst = append(dst, s[:start]...)
|
||||||
|
end := locs[1]
|
||||||
|
dst = re.ExpandString(dst, replacement, s, locs)
|
||||||
|
s = s[end:]
|
||||||
|
replacements++
|
||||||
|
if limit > 0 && replacements >= limit {
|
||||||
|
return append(dst, s...)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
200
lib/logstorage/pipe_replace_regexp_test.go
Normal file
200
lib/logstorage/pipe_replace_regexp_test.go
Normal file
|
@ -0,0 +1,200 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"regexp"
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestParsePipeReplaceRegexpSuccess(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeSuccess(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`replace_regexp (foo, bar)`)
|
||||||
|
f(`replace_regexp ("foo[^ ]+bar|baz", "bar${1}x$0")`)
|
||||||
|
f(`replace_regexp (" ", "") at x`)
|
||||||
|
f(`replace_regexp if (x:y) ("-", ":") at a`)
|
||||||
|
f(`replace_regexp (" ", "") at x limit 10`)
|
||||||
|
f(`replace_regexp if (x:y) (" ", "") at foo limit 10`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestParsePipeReplaceRegexpFailure(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeFailure(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`replace_regexp`)
|
||||||
|
f(`replace_regexp if`)
|
||||||
|
f(`replace_regexp foo`)
|
||||||
|
f(`replace_regexp (`)
|
||||||
|
f(`replace_regexp (foo`)
|
||||||
|
f(`replace_regexp (foo,`)
|
||||||
|
f(`replace_regexp(foo,bar`)
|
||||||
|
f(`replace_regexp(foo,bar,baz)`)
|
||||||
|
f(`replace_regexp(foo,bar) abc`)
|
||||||
|
f(`replace_regexp(bar,baz) limit`)
|
||||||
|
f(`replace_regexp(bar,baz) limit N`)
|
||||||
|
f(`replace_regexp ("foo[", "bar")`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipeReplaceRegexp(t *testing.T) {
|
||||||
|
f := func(pipeStr string, rows, rowsExpected [][]Field) {
|
||||||
|
t.Helper()
|
||||||
|
expectPipeResults(t, pipeStr, rows, rowsExpected)
|
||||||
|
}
|
||||||
|
|
||||||
|
// replace_regexp with placeholders
|
||||||
|
f(`replace_regexp ("foo(.+?)bar", "q-$1-x")`, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `abc foo a bar foobar foo b bar`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"_msg", `1234`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `abc q- a -x q-bar foo b -x`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"_msg", `1234`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// replace_regexp without limits at _msg
|
||||||
|
f(`replace_regexp ("[_/]", "-")`, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `a_bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"_msg", `1234`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"_msg", `a-bc-d-ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"_msg", `1234`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// replace_regexp with limit 1 at foo
|
||||||
|
f(`replace_regexp ("[_/]", "-") at foo limit 1`, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a_bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `1234`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a-bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `1234`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// replace_regexp with limit 100 at foo
|
||||||
|
f(`replace_regexp ("[_/]", "-") at foo limit 100`, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a_bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `1234`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a-bc-d-ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `1234`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// conditional replace_regexp at foo
|
||||||
|
f(`replace_regexp if (bar:abc) ("[_/]", "") at foo`, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a_bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `123_45/6`},
|
||||||
|
{"bar", "abc"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"foo", `a_bc_d/ef`},
|
||||||
|
{"bar", `cde`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"foo", `123456`},
|
||||||
|
{"bar", "abc"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipeReplaceRegexpUpdateNeededFields(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(`replace_regexp ("a", "b") at x`, "*", "", "*", "")
|
||||||
|
f(`replace_regexp if (f1:q) ("a", "b") at x`, "*", "", "*", "")
|
||||||
|
|
||||||
|
// unneeded fields do not intersect with at field
|
||||||
|
f(`replace_regexp ("a", "b") at x`, "*", "f1,f2", "*", "f1,f2")
|
||||||
|
f(`replace_regexp if (f3:q) ("a", "b") at x`, "*", "f1,f2", "*", "f1,f2")
|
||||||
|
f(`replace_regexp if (f2:q) ("a", "b") at x`, "*", "f1,f2", "*", "f1")
|
||||||
|
|
||||||
|
// unneeded fields intersect with at field
|
||||||
|
f(`replace_regexp ("a", "b") at x`, "*", "x,y", "*", "x,y")
|
||||||
|
f(`replace_regexp if (f1:q) ("a", "b") at x`, "*", "x,y", "*", "x,y")
|
||||||
|
f(`replace_regexp if (x:q) ("a", "b") at x`, "*", "x,y", "*", "x,y")
|
||||||
|
f(`replace_regexp if (y:q) ("a", "b") at x`, "*", "x,y", "*", "x,y")
|
||||||
|
|
||||||
|
// needed fields do not intersect with at field
|
||||||
|
f(`replace_regexp ("a", "b") at x`, "f2,y", "", "f2,y", "")
|
||||||
|
f(`replace_regexp if (f1:q) ("a", "b") at x`, "f2,y", "", "f2,y", "")
|
||||||
|
|
||||||
|
// needed fields intersect with at field
|
||||||
|
f(`replace_regexp ("a", "b") at y`, "f2,y", "", "f2,y", "")
|
||||||
|
f(`replace_regexp if (f1:q) ("a", "b") at y`, "f2,y", "", "f1,f2,y", "")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestAppendReplaceRegexp(t *testing.T) {
|
||||||
|
f := func(s, reStr, replacement string, limit int, resultExpected string) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
re := regexp.MustCompile(reStr)
|
||||||
|
result := appendReplaceRegexp(nil, s, re, replacement, uint64(limit))
|
||||||
|
if string(result) != resultExpected {
|
||||||
|
t.Fatalf("unexpected result for appendReplaceRegexp(%q, %q, %q, %d)\ngot\n%s\nwant\n%s", s, reStr, replacement, limit, result, resultExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", "", "", 0, "")
|
||||||
|
f("", "foo", "bar", 0, "")
|
||||||
|
f("abc", "foo", "bar", 0, "abc")
|
||||||
|
f("foo", "fo+", "bar", 0, "bar")
|
||||||
|
f("foox", "fo+", "bar", 0, "barx")
|
||||||
|
f("afoo", "fo+", "bar", 0, "abar")
|
||||||
|
f("afoox", "fo+", "bar", 0, "abarx")
|
||||||
|
f("foo-bar/baz", "[-/]", "_", 0, "foo_bar_baz")
|
||||||
|
f("foo bar/ baz ", "[ /]", "", 2, "foobar baz ")
|
||||||
|
|
||||||
|
// placeholders
|
||||||
|
f("afoo abc barz", "a([^ ]+)", "b${1}x", 0, "bfoox bbcx bbrzx")
|
||||||
|
f("afoo abc barz", "a([^ ]+)", "b${1}x", 1, "bfoox abc barz")
|
||||||
|
}
|
|
@ -163,10 +163,11 @@ func TestAppendReplace(t *testing.T) {
|
||||||
|
|
||||||
f("", "", "", 0, "")
|
f("", "", "", 0, "")
|
||||||
f("", "foo", "bar", 0, "")
|
f("", "foo", "bar", 0, "")
|
||||||
|
f("abc", "foo", "bar", 0, "abc")
|
||||||
f("foo", "foo", "bar", 0, "bar")
|
f("foo", "foo", "bar", 0, "bar")
|
||||||
f("foox", "foo", "bar", 0, "barx")
|
f("foox", "foo", "bar", 0, "barx")
|
||||||
f("afoo", "foo", "bar", 0, "abar")
|
f("afoo", "foo", "bar", 0, "abar")
|
||||||
f("afoox", "foo", "bar", 0, "abarx")
|
f("afoox", "foo", "bar", 0, "abarx")
|
||||||
f("foo-bar-baz", "-", "_", 0, "foo_bar_baz")
|
f("foo-bar-baz", "-", "_", 0, "foo_bar_baz")
|
||||||
f("foo bar baz ", " ", "", 0, "foobarbaz")
|
f("foo bar baz ", " ", "", 1, "foobar baz ")
|
||||||
}
|
}
|
||||||
|
|
|
@ -67,14 +67,26 @@ func (ps *pipeSort) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ps *pipeSort) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func (ps *pipeSort) optimize() {
|
||||||
if ps.limit > 0 {
|
// nothing to do
|
||||||
return newPipeTopkProcessor(ps, workersCount, stopCh, cancel, ppBase)
|
|
||||||
}
|
|
||||||
return newPipeSortProcessor(ps, workersCount, stopCh, cancel, ppBase)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func newPipeSortProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func (ps *pipeSort) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ps *pipeSort) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return ps, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ps *pipeSort) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
|
if ps.limit > 0 {
|
||||||
|
return newPipeTopkProcessor(ps, workersCount, stopCh, cancel, ppNext)
|
||||||
|
}
|
||||||
|
return newPipeSortProcessor(ps, workersCount, stopCh, cancel, ppNext)
|
||||||
|
}
|
||||||
|
|
||||||
|
func newPipeSortProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
||||||
|
|
||||||
shards := make([]pipeSortProcessorShard, workersCount)
|
shards := make([]pipeSortProcessorShard, workersCount)
|
||||||
|
@ -92,7 +104,7 @@ func newPipeSortProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}
|
||||||
ps: ps,
|
ps: ps,
|
||||||
stopCh: stopCh,
|
stopCh: stopCh,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
|
|
||||||
|
@ -107,7 +119,7 @@ type pipeSortProcessor struct {
|
||||||
ps *pipeSort
|
ps *pipeSort
|
||||||
stopCh <-chan struct{}
|
stopCh <-chan struct{}
|
||||||
cancel func()
|
cancel func()
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeSortProcessorShard
|
shards []pipeSortProcessorShard
|
||||||
|
|
||||||
|
@ -522,7 +534,7 @@ func (wctx *pipeSortWriteContext) writeNextRow(shard *pipeSortProcessorShard) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !areEqualColumns {
|
if !areEqualColumns {
|
||||||
// send the current block to ppBase and construct a block with new set of columns
|
// send the current block to ppNext and construct a block with new set of columns
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
|
||||||
rcs = wctx.rcs[:0]
|
rcs = wctx.rcs[:0]
|
||||||
|
@ -561,10 +573,10 @@ func (wctx *pipeSortWriteContext) flush() {
|
||||||
|
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppNext
|
||||||
br.setResultColumns(rcs, wctx.rowsCount)
|
br.setResultColumns(rcs, wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.psp.ppBase.writeBlock(0, br)
|
wctx.psp.ppNext.writeBlock(0, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].resetValues()
|
rcs[i].resetValues()
|
||||||
|
|
|
@ -116,24 +116,47 @@ func (ps *pipeStats) updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||||
unneededFields.reset()
|
unneededFields.reset()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ps *pipeStats) optimize() {
|
||||||
|
for _, f := range ps.funcs {
|
||||||
|
f.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ps *pipeStats) hasFilterInWithQuery() bool {
|
||||||
|
for _, f := range ps.funcs {
|
||||||
|
if f.iff.hasFilterInWithQuery() {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ps *pipeStats) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
funcsNew := make([]pipeStatsFunc, len(ps.funcs))
|
||||||
|
for i, f := range ps.funcs {
|
||||||
|
iffNew, err := f.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
f.iff = iffNew
|
||||||
|
funcsNew[i] = f
|
||||||
|
}
|
||||||
|
psNew := *ps
|
||||||
|
ps.funcs = funcsNew
|
||||||
|
return &psNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
const stateSizeBudgetChunk = 1 << 20
|
const stateSizeBudgetChunk = 1 << 20
|
||||||
|
|
||||||
func (ps *pipeStats) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func (ps *pipeStats) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
maxStateSize := int64(float64(memory.Allowed()) * 0.3)
|
maxStateSize := int64(float64(memory.Allowed()) * 0.3)
|
||||||
|
|
||||||
shards := make([]pipeStatsProcessorShard, workersCount)
|
shards := make([]pipeStatsProcessorShard, workersCount)
|
||||||
funcsLen := len(ps.funcs)
|
|
||||||
for i := range shards {
|
for i := range shards {
|
||||||
shards[i] = pipeStatsProcessorShard{
|
shards[i] = pipeStatsProcessorShard{
|
||||||
pipeStatsProcessorShardNopad: pipeStatsProcessorShardNopad{
|
pipeStatsProcessorShardNopad: pipeStatsProcessorShardNopad{
|
||||||
ps: ps,
|
ps: ps,
|
||||||
|
|
||||||
m: make(map[string]*pipeStatsGroup),
|
|
||||||
|
|
||||||
bms: make([]bitmap, funcsLen),
|
|
||||||
brs: make([]*blockResult, funcsLen),
|
|
||||||
brsBuf: make([]blockResult, funcsLen),
|
|
||||||
|
|
||||||
stateSizeBudget: stateSizeBudgetChunk,
|
stateSizeBudget: stateSizeBudgetChunk,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
@ -144,7 +167,7 @@ func (ps *pipeStats) newPipeProcessor(workersCount int, stopCh <-chan struct{},
|
||||||
ps: ps,
|
ps: ps,
|
||||||
stopCh: stopCh,
|
stopCh: stopCh,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
|
|
||||||
|
@ -159,7 +182,7 @@ type pipeStatsProcessor struct {
|
||||||
ps *pipeStats
|
ps *pipeStats
|
||||||
stopCh <-chan struct{}
|
stopCh <-chan struct{}
|
||||||
cancel func()
|
cancel func()
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeStatsProcessorShard
|
shards []pipeStatsProcessorShard
|
||||||
|
|
||||||
|
@ -190,7 +213,22 @@ type pipeStatsProcessorShardNopad struct {
|
||||||
stateSizeBudget int
|
stateSizeBudget int
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (shard *pipeStatsProcessorShard) init() {
|
||||||
|
if shard.m != nil {
|
||||||
|
// Already initialized
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
funcsLen := len(shard.ps.funcs)
|
||||||
|
|
||||||
|
shard.m = make(map[string]*pipeStatsGroup)
|
||||||
|
shard.bms = make([]bitmap, funcsLen)
|
||||||
|
shard.brs = make([]*blockResult, funcsLen)
|
||||||
|
shard.brsBuf = make([]blockResult, funcsLen)
|
||||||
|
}
|
||||||
|
|
||||||
func (shard *pipeStatsProcessorShard) writeBlock(br *blockResult) {
|
func (shard *pipeStatsProcessorShard) writeBlock(br *blockResult) {
|
||||||
|
shard.init()
|
||||||
byFields := shard.ps.byFields
|
byFields := shard.ps.byFields
|
||||||
|
|
||||||
// Apply per-function filters
|
// Apply per-function filters
|
||||||
|
@ -398,7 +436,9 @@ func (psp *pipeStatsProcessor) flush() error {
|
||||||
|
|
||||||
// Merge states across shards
|
// Merge states across shards
|
||||||
shards := psp.shards
|
shards := psp.shards
|
||||||
m := shards[0].m
|
shardMain := &shards[0]
|
||||||
|
shardMain.init()
|
||||||
|
m := shardMain.m
|
||||||
shards = shards[1:]
|
shards = shards[1:]
|
||||||
for i := range shards {
|
for i := range shards {
|
||||||
shard := &shards[i]
|
shard := &shards[i]
|
||||||
|
@ -420,12 +460,12 @@ func (psp *pipeStatsProcessor) flush() error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Write per-group states to ppBase
|
// Write per-group states to ppNext
|
||||||
byFields := psp.ps.byFields
|
byFields := psp.ps.byFields
|
||||||
if len(byFields) == 0 && len(m) == 0 {
|
if len(byFields) == 0 && len(m) == 0 {
|
||||||
// Special case - zero matching rows.
|
// Special case - zero matching rows.
|
||||||
_ = shards[0].getPipeStatsGroup(nil)
|
_ = shardMain.getPipeStatsGroup(nil)
|
||||||
m = shards[0].m
|
m = shardMain.m
|
||||||
}
|
}
|
||||||
|
|
||||||
rcs := make([]resultColumn, 0, len(byFields)+len(psp.ps.funcs))
|
rcs := make([]resultColumn, 0, len(byFields)+len(psp.ps.funcs))
|
||||||
|
@ -480,7 +520,7 @@ func (psp *pipeStatsProcessor) flush() error {
|
||||||
if valuesLen >= 1_000_000 {
|
if valuesLen >= 1_000_000 {
|
||||||
br.setResultColumns(rcs, rowsCount)
|
br.setResultColumns(rcs, rowsCount)
|
||||||
rowsCount = 0
|
rowsCount = 0
|
||||||
psp.ppBase.writeBlock(0, &br)
|
psp.ppNext.writeBlock(0, &br)
|
||||||
br.reset()
|
br.reset()
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].resetValues()
|
rcs[i].resetValues()
|
||||||
|
@ -490,7 +530,7 @@ func (psp *pipeStatsProcessor) flush() error {
|
||||||
}
|
}
|
||||||
|
|
||||||
br.setResultColumns(rcs, rowsCount)
|
br.setResultColumns(rcs, rowsCount)
|
||||||
psp.ppBase.writeBlock(0, &br)
|
psp.ppNext.writeBlock(0, &br)
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,7 +13,7 @@ import (
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/stringsutil"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/stringsutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
func newPipeTopkProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func newPipeTopkProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
||||||
|
|
||||||
shards := make([]pipeTopkProcessorShard, workersCount)
|
shards := make([]pipeTopkProcessorShard, workersCount)
|
||||||
|
@ -31,7 +31,7 @@ func newPipeTopkProcessor(ps *pipeSort, workersCount int, stopCh <-chan struct{}
|
||||||
ps: ps,
|
ps: ps,
|
||||||
stopCh: stopCh,
|
stopCh: stopCh,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
|
|
||||||
|
@ -46,7 +46,7 @@ type pipeTopkProcessor struct {
|
||||||
ps *pipeSort
|
ps *pipeSort
|
||||||
stopCh <-chan struct{}
|
stopCh <-chan struct{}
|
||||||
cancel func()
|
cancel func()
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeTopkProcessorShard
|
shards []pipeTopkProcessorShard
|
||||||
|
|
||||||
|
@ -464,7 +464,7 @@ func (wctx *pipeTopkWriteContext) writeNextRow(shard *pipeTopkProcessorShard) bo
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !areEqualColumns {
|
if !areEqualColumns {
|
||||||
// send the current block to ppBase and construct a block with new set of columns
|
// send the current block to ppNext and construct a block with new set of columns
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
|
||||||
rcs = wctx.rcs[:0]
|
rcs = wctx.rcs[:0]
|
||||||
|
@ -508,10 +508,10 @@ func (wctx *pipeTopkWriteContext) flush() {
|
||||||
|
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppNext
|
||||||
br.setResultColumns(rcs, wctx.rowsCount)
|
br.setResultColumns(rcs, wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.ptp.ppBase.writeBlock(0, br)
|
wctx.ptp.ppNext.writeBlock(0, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].resetValues()
|
rcs[i].resetValues()
|
||||||
|
|
|
@ -32,7 +32,7 @@ func (pu *pipeUniq) String() string {
|
||||||
s += " by (" + fieldNamesString(pu.byFields) + ")"
|
s += " by (" + fieldNamesString(pu.byFields) + ")"
|
||||||
}
|
}
|
||||||
if pu.hitsFieldName != "" {
|
if pu.hitsFieldName != "" {
|
||||||
s += " hits"
|
s += " with hits"
|
||||||
}
|
}
|
||||||
if pu.limit > 0 {
|
if pu.limit > 0 {
|
||||||
s += fmt.Sprintf(" limit %d", pu.limit)
|
s += fmt.Sprintf(" limit %d", pu.limit)
|
||||||
|
@ -51,7 +51,19 @@ func (pu *pipeUniq) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pu *pipeUniq) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
func (pu *pipeUniq) optimize() {
|
||||||
|
// nothing to do
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUniq) hasFilterInWithQuery() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUniq) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
return pu, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUniq) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
||||||
|
|
||||||
shards := make([]pipeUniqProcessorShard, workersCount)
|
shards := make([]pipeUniqProcessorShard, workersCount)
|
||||||
|
@ -69,7 +81,7 @@ func (pu *pipeUniq) newPipeProcessor(workersCount int, stopCh <-chan struct{}, c
|
||||||
pu: pu,
|
pu: pu,
|
||||||
stopCh: stopCh,
|
stopCh: stopCh,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: shards,
|
shards: shards,
|
||||||
|
|
||||||
|
@ -84,7 +96,7 @@ type pipeUniqProcessor struct {
|
||||||
pu *pipeUniq
|
pu *pipeUniq
|
||||||
stopCh <-chan struct{}
|
stopCh <-chan struct{}
|
||||||
cancel func()
|
cancel func()
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeUniqProcessorShard
|
shards []pipeUniqProcessorShard
|
||||||
|
|
||||||
|
@ -418,7 +430,7 @@ func (wctx *pipeUniqWriteContext) writeRow(rowFields []Field) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !areEqualColumns {
|
if !areEqualColumns {
|
||||||
// send the current block to ppBase and construct a block with new set of columns
|
// send the current block to ppNext and construct a block with new set of columns
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
|
||||||
rcs = wctx.rcs[:0]
|
rcs = wctx.rcs[:0]
|
||||||
|
@ -446,10 +458,10 @@ func (wctx *pipeUniqWriteContext) flush() {
|
||||||
|
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppNext
|
||||||
br.setResultColumns(rcs, wctx.rowsCount)
|
br.setResultColumns(rcs, wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.pup.ppBase.writeBlock(0, br)
|
wctx.pup.ppNext.writeBlock(0, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].resetValues()
|
rcs[i].resetValues()
|
||||||
|
@ -477,6 +489,12 @@ func parsePipeUniq(lex *lexer) (*pipeUniq, error) {
|
||||||
pu.byFields = bfs
|
pu.byFields = bfs
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if lex.isKeyword("with") {
|
||||||
|
lex.nextToken()
|
||||||
|
if !lex.isKeyword("hits") {
|
||||||
|
return nil, fmt.Errorf("missing 'hits' after 'with'")
|
||||||
|
}
|
||||||
|
}
|
||||||
if lex.isKeyword("hits") {
|
if lex.isKeyword("hits") {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
hitsFieldName := "hits"
|
hitsFieldName := "hits"
|
||||||
|
|
|
@ -11,15 +11,15 @@ func TestParsePipeUniqSuccess(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
f(`uniq`)
|
f(`uniq`)
|
||||||
f(`uniq hits`)
|
f(`uniq with hits`)
|
||||||
f(`uniq limit 10`)
|
f(`uniq limit 10`)
|
||||||
f(`uniq hits limit 10`)
|
f(`uniq with hits limit 10`)
|
||||||
f(`uniq by (x)`)
|
f(`uniq by (x)`)
|
||||||
f(`uniq by (x) limit 10`)
|
f(`uniq by (x) limit 10`)
|
||||||
f(`uniq by (x, y)`)
|
f(`uniq by (x, y)`)
|
||||||
f(`uniq by (x, y) hits`)
|
f(`uniq by (x, y) with hits`)
|
||||||
f(`uniq by (x, y) limit 10`)
|
f(`uniq by (x, y) limit 10`)
|
||||||
f(`uniq by (x, y) hits limit 10`)
|
f(`uniq by (x, y) with hits limit 10`)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestParsePipeUniqFailure(t *testing.T) {
|
func TestParsePipeUniqFailure(t *testing.T) {
|
||||||
|
@ -33,6 +33,7 @@ func TestParsePipeUniqFailure(t *testing.T) {
|
||||||
f(`uniq by hits`)
|
f(`uniq by hits`)
|
||||||
f(`uniq by(x) limit`)
|
f(`uniq by(x) limit`)
|
||||||
f(`uniq by(x) limit foo`)
|
f(`uniq by(x) limit foo`)
|
||||||
|
f(`uniq by (x) with`)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestPipeUniq(t *testing.T) {
|
func TestPipeUniq(t *testing.T) {
|
||||||
|
@ -365,10 +366,12 @@ func TestPipeUniqUpdateNeededFields(t *testing.T) {
|
||||||
f("uniq by()", "*", "", "*", "")
|
f("uniq by()", "*", "", "*", "")
|
||||||
f("uniq by(*)", "*", "", "*", "")
|
f("uniq by(*)", "*", "", "*", "")
|
||||||
f("uniq by(f1,f2)", "*", "", "f1,f2", "")
|
f("uniq by(f1,f2)", "*", "", "f1,f2", "")
|
||||||
|
f("uniq by(f1,f2) with hits", "*", "", "f1,f2", "")
|
||||||
|
|
||||||
// all the needed fields, unneeded fields do not intersect with src
|
// all the needed fields, unneeded fields do not intersect with src
|
||||||
f("uniq by(s1, s2)", "*", "f1,f2", "s1,s2", "")
|
f("uniq by(s1, s2)", "*", "f1,f2", "s1,s2", "")
|
||||||
f("uniq", "*", "f1,f2", "*", "")
|
f("uniq", "*", "f1,f2", "*", "")
|
||||||
|
f("uniq with hits", "*", "f1,f2", "*", "")
|
||||||
|
|
||||||
// all the needed fields, unneeded fields intersect with src
|
// all the needed fields, unneeded fields intersect with src
|
||||||
f("uniq by(s1, s2)", "*", "s1,f1,f2", "s1,s2", "")
|
f("uniq by(s1, s2)", "*", "s1,f1,f2", "s1,s2", "")
|
||||||
|
|
|
@ -6,6 +6,49 @@ import (
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
func updateNeededFieldsForUnpackPipe(fromField string, outFields []string, keepOriginalFields, skipEmptyResults bool, iff *ifFilter, neededFields, unneededFields fieldsSet) {
|
||||||
|
if neededFields.contains("*") {
|
||||||
|
unneededFieldsOrig := unneededFields.clone()
|
||||||
|
unneededFieldsCount := 0
|
||||||
|
if len(outFields) > 0 {
|
||||||
|
for _, f := range outFields {
|
||||||
|
if unneededFieldsOrig.contains(f) {
|
||||||
|
unneededFieldsCount++
|
||||||
|
}
|
||||||
|
if !keepOriginalFields && !skipEmptyResults {
|
||||||
|
unneededFields.add(f)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if len(outFields) == 0 || unneededFieldsCount < len(outFields) {
|
||||||
|
unneededFields.remove(fromField)
|
||||||
|
if iff != nil {
|
||||||
|
unneededFields.removeFields(iff.neededFields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
neededFieldsOrig := neededFields.clone()
|
||||||
|
needFromField := len(outFields) == 0
|
||||||
|
if len(outFields) > 0 {
|
||||||
|
needFromField = false
|
||||||
|
for _, f := range outFields {
|
||||||
|
if neededFieldsOrig.contains(f) {
|
||||||
|
needFromField = true
|
||||||
|
}
|
||||||
|
if !keepOriginalFields && !skipEmptyResults {
|
||||||
|
neededFields.remove(f)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if needFromField {
|
||||||
|
neededFields.add(fromField)
|
||||||
|
if iff != nil {
|
||||||
|
neededFields.addFields(iff.neededFields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
type fieldsUnpackerContext struct {
|
type fieldsUnpackerContext struct {
|
||||||
workerID uint
|
workerID uint
|
||||||
fieldPrefix string
|
fieldPrefix string
|
||||||
|
@ -53,12 +96,12 @@ func (uctx *fieldsUnpackerContext) addField(name, value string) {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func newPipeUnpackProcessor(workersCount int, unpackFunc func(uctx *fieldsUnpackerContext, s string), ppBase pipeProcessor,
|
func newPipeUnpackProcessor(workersCount int, unpackFunc func(uctx *fieldsUnpackerContext, s string), ppNext pipeProcessor,
|
||||||
fromField string, fieldPrefix string, keepOriginalFields, skipEmptyResults bool, iff *ifFilter) *pipeUnpackProcessor {
|
fromField string, fieldPrefix string, keepOriginalFields, skipEmptyResults bool, iff *ifFilter) *pipeUnpackProcessor {
|
||||||
|
|
||||||
return &pipeUnpackProcessor{
|
return &pipeUnpackProcessor{
|
||||||
unpackFunc: unpackFunc,
|
unpackFunc: unpackFunc,
|
||||||
ppBase: ppBase,
|
ppNext: ppNext,
|
||||||
|
|
||||||
shards: make([]pipeUnpackProcessorShard, workersCount),
|
shards: make([]pipeUnpackProcessorShard, workersCount),
|
||||||
|
|
||||||
|
@ -72,7 +115,7 @@ func newPipeUnpackProcessor(workersCount int, unpackFunc func(uctx *fieldsUnpack
|
||||||
|
|
||||||
type pipeUnpackProcessor struct {
|
type pipeUnpackProcessor struct {
|
||||||
unpackFunc func(uctx *fieldsUnpackerContext, s string)
|
unpackFunc func(uctx *fieldsUnpackerContext, s string)
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
|
|
||||||
shards []pipeUnpackProcessorShard
|
shards []pipeUnpackProcessorShard
|
||||||
|
|
||||||
|
@ -104,7 +147,7 @@ func (pup *pipeUnpackProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
shard := &pup.shards[workerID]
|
shard := &pup.shards[workerID]
|
||||||
shard.wctx.init(workerID, pup.ppBase, pup.keepOriginalFields, pup.skipEmptyResults, br)
|
shard.wctx.init(workerID, pup.ppNext, pup.keepOriginalFields, pup.skipEmptyResults, br)
|
||||||
shard.uctx.init(workerID, pup.fieldPrefix)
|
shard.uctx.init(workerID, pup.fieldPrefix)
|
||||||
|
|
||||||
bm := &shard.bm
|
bm := &shard.bm
|
||||||
|
@ -113,7 +156,7 @@ func (pup *pipeUnpackProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
if pup.iff != nil {
|
if pup.iff != nil {
|
||||||
pup.iff.f.applyToBlockResult(br, bm)
|
pup.iff.f.applyToBlockResult(br, bm)
|
||||||
if bm.isZero() {
|
if bm.isZero() {
|
||||||
pup.ppBase.writeBlock(workerID, br)
|
pup.ppNext.writeBlock(workerID, br)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -132,13 +175,16 @@ func (pup *pipeUnpackProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
values := c.getValues(br)
|
values := c.getValues(br)
|
||||||
vPrevApplied := ""
|
vPrev := ""
|
||||||
|
hadUnpacks := false
|
||||||
for i, v := range values {
|
for i, v := range values {
|
||||||
if bm.isSetBit(i) {
|
if bm.isSetBit(i) {
|
||||||
if vPrevApplied != v {
|
if !hadUnpacks || vPrev != v {
|
||||||
|
vPrev = v
|
||||||
|
hadUnpacks = true
|
||||||
|
|
||||||
shard.uctx.resetFields()
|
shard.uctx.resetFields()
|
||||||
pup.unpackFunc(&shard.uctx, v)
|
pup.unpackFunc(&shard.uctx, v)
|
||||||
vPrevApplied = v
|
|
||||||
}
|
}
|
||||||
shard.wctx.writeRow(i, shard.uctx.fields)
|
shard.wctx.writeRow(i, shard.uctx.fields)
|
||||||
} else {
|
} else {
|
||||||
|
@ -158,7 +204,7 @@ func (pup *pipeUnpackProcessor) flush() error {
|
||||||
|
|
||||||
type pipeUnpackWriteContext struct {
|
type pipeUnpackWriteContext struct {
|
||||||
workerID uint
|
workerID uint
|
||||||
ppBase pipeProcessor
|
ppNext pipeProcessor
|
||||||
keepOriginalFields bool
|
keepOriginalFields bool
|
||||||
skipEmptyResults bool
|
skipEmptyResults bool
|
||||||
|
|
||||||
|
@ -177,7 +223,7 @@ type pipeUnpackWriteContext struct {
|
||||||
|
|
||||||
func (wctx *pipeUnpackWriteContext) reset() {
|
func (wctx *pipeUnpackWriteContext) reset() {
|
||||||
wctx.workerID = 0
|
wctx.workerID = 0
|
||||||
wctx.ppBase = nil
|
wctx.ppNext = nil
|
||||||
wctx.keepOriginalFields = false
|
wctx.keepOriginalFields = false
|
||||||
|
|
||||||
wctx.brSrc = nil
|
wctx.brSrc = nil
|
||||||
|
@ -193,11 +239,11 @@ func (wctx *pipeUnpackWriteContext) reset() {
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
func (wctx *pipeUnpackWriteContext) init(workerID uint, ppBase pipeProcessor, keepOriginalFields, skipEmptyResults bool, brSrc *blockResult) {
|
func (wctx *pipeUnpackWriteContext) init(workerID uint, ppNext pipeProcessor, keepOriginalFields, skipEmptyResults bool, brSrc *blockResult) {
|
||||||
wctx.reset()
|
wctx.reset()
|
||||||
|
|
||||||
wctx.workerID = workerID
|
wctx.workerID = workerID
|
||||||
wctx.ppBase = ppBase
|
wctx.ppNext = ppNext
|
||||||
wctx.keepOriginalFields = keepOriginalFields
|
wctx.keepOriginalFields = keepOriginalFields
|
||||||
wctx.skipEmptyResults = skipEmptyResults
|
wctx.skipEmptyResults = skipEmptyResults
|
||||||
|
|
||||||
|
@ -219,7 +265,7 @@ func (wctx *pipeUnpackWriteContext) writeRow(rowIdx int, extraFields []Field) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !areEqualColumns {
|
if !areEqualColumns {
|
||||||
// send the current block to ppBase and construct a block with new set of columns
|
// send the current block to ppNext and construct a block with new set of columns
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
|
||||||
rcs = wctx.rcs[:0]
|
rcs = wctx.rcs[:0]
|
||||||
|
@ -264,11 +310,11 @@ func (wctx *pipeUnpackWriteContext) flush() {
|
||||||
|
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppNext
|
||||||
br := &wctx.br
|
br := &wctx.br
|
||||||
br.setResultColumns(rcs, wctx.rowsCount)
|
br.setResultColumns(rcs, wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.ppBase.writeBlock(wctx.workerID, br)
|
wctx.ppNext.writeBlock(wctx.workerID, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].resetValues()
|
rcs[i].resetValues()
|
||||||
|
|
|
@ -56,50 +56,25 @@ func (pu *pipeUnpackJSON) updateNeededFields(neededFields, unneededFields fields
|
||||||
updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff, neededFields, unneededFields)
|
updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff, neededFields, unneededFields)
|
||||||
}
|
}
|
||||||
|
|
||||||
func updateNeededFieldsForUnpackPipe(fromField string, outFields []string, keepOriginalFields, skipEmptyResults bool, iff *ifFilter, neededFields, unneededFields fieldsSet) {
|
func (pu *pipeUnpackJSON) optimize() {
|
||||||
if neededFields.contains("*") {
|
pu.iff.optimizeFilterIn()
|
||||||
unneededFieldsOrig := unneededFields.clone()
|
|
||||||
unneededFieldsCount := 0
|
|
||||||
if len(outFields) > 0 {
|
|
||||||
for _, f := range outFields {
|
|
||||||
if unneededFieldsOrig.contains(f) {
|
|
||||||
unneededFieldsCount++
|
|
||||||
}
|
|
||||||
if !keepOriginalFields && !skipEmptyResults {
|
|
||||||
unneededFields.add(f)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if len(outFields) == 0 || unneededFieldsCount < len(outFields) {
|
|
||||||
unneededFields.remove(fromField)
|
|
||||||
if iff != nil {
|
|
||||||
unneededFields.removeFields(iff.neededFields)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
neededFieldsOrig := neededFields.clone()
|
|
||||||
needFromField := len(outFields) == 0
|
|
||||||
if len(outFields) > 0 {
|
|
||||||
needFromField = false
|
|
||||||
for _, f := range outFields {
|
|
||||||
if neededFieldsOrig.contains(f) {
|
|
||||||
needFromField = true
|
|
||||||
}
|
|
||||||
if !keepOriginalFields && !skipEmptyResults {
|
|
||||||
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 {
|
func (pu *pipeUnpackJSON) hasFilterInWithQuery() bool {
|
||||||
|
return pu.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnpackJSON) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pu.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
puNew := *pu
|
||||||
|
puNew.iff = iffNew
|
||||||
|
return &puNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnpackJSON) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
unpackJSON := func(uctx *fieldsUnpackerContext, s string) {
|
unpackJSON := func(uctx *fieldsUnpackerContext, s string) {
|
||||||
if len(s) == 0 || s[0] != '{' {
|
if len(s) == 0 || s[0] != '{' {
|
||||||
// This isn't a JSON object
|
// This isn't a JSON object
|
||||||
|
@ -134,7 +109,7 @@ func (pu *pipeUnpackJSON) newPipeProcessor(workersCount int, _ <-chan struct{},
|
||||||
}
|
}
|
||||||
PutJSONParser(p)
|
PutJSONParser(p)
|
||||||
}
|
}
|
||||||
return newPipeUnpackProcessor(workersCount, unpackJSON, ppBase, pu.fromField, pu.resultPrefix, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff)
|
return newPipeUnpackProcessor(workersCount, unpackJSON, ppNext, pu.fromField, pu.resultPrefix, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff)
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) {
|
func parsePipeUnpackJSON(lex *lexer) (*pipeUnpackJSON, error) {
|
||||||
|
|
|
@ -1,10 +1,6 @@
|
||||||
package logstorage
|
package logstorage
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"math/rand"
|
|
||||||
"slices"
|
|
||||||
"strings"
|
|
||||||
"sync"
|
|
||||||
"testing"
|
"testing"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -166,7 +162,6 @@ func TestPipeUnpackJSON(t *testing.T) {
|
||||||
}, [][]Field{
|
}, [][]Field{
|
||||||
{
|
{
|
||||||
{"_msg", `{"foo":"bar"}`},
|
{"_msg", `{"foo":"bar"}`},
|
||||||
{"x", ""},
|
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
@ -313,228 +308,12 @@ func TestPipeUnpackJSON(t *testing.T) {
|
||||||
{"y", `abc`},
|
{"y", `abc`},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"y", ""},
|
|
||||||
{"z", `foobar`},
|
{"z", `foobar`},
|
||||||
{"x", `{"z":["bar",123]}`},
|
{"x", `{"z":["bar",123]}`},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func expectPipeResults(t *testing.T, pipeStr string, rows, rowsExpected [][]Field) {
|
|
||||||
t.Helper()
|
|
||||||
|
|
||||||
lex := newLexer(pipeStr)
|
|
||||||
p, err := parsePipe(lex)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("unexpected error when parsing %q: %s", pipeStr, err)
|
|
||||||
}
|
|
||||||
|
|
||||||
workersCount := 5
|
|
||||||
stopCh := make(chan struct{})
|
|
||||||
cancel := func() {}
|
|
||||||
ppTest := newTestPipeProcessor()
|
|
||||||
pp := p.newPipeProcessor(workersCount, stopCh, cancel, ppTest)
|
|
||||||
|
|
||||||
brw := newTestBlockResultWriter(workersCount, pp)
|
|
||||||
for _, row := range rows {
|
|
||||||
brw.writeRow(row)
|
|
||||||
}
|
|
||||||
brw.flush()
|
|
||||||
pp.flush()
|
|
||||||
|
|
||||||
ppTest.expectRows(t, rowsExpected)
|
|
||||||
}
|
|
||||||
|
|
||||||
func newTestBlockResultWriter(workersCount int, ppBase pipeProcessor) *testBlockResultWriter {
|
|
||||||
return &testBlockResultWriter{
|
|
||||||
workersCount: workersCount,
|
|
||||||
ppBase: ppBase,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
type testBlockResultWriter struct {
|
|
||||||
workersCount int
|
|
||||||
ppBase pipeProcessor
|
|
||||||
rcs []resultColumn
|
|
||||||
br blockResult
|
|
||||||
|
|
||||||
rowsCount int
|
|
||||||
}
|
|
||||||
|
|
||||||
func (brw *testBlockResultWriter) writeRow(row []Field) {
|
|
||||||
if !brw.areSameFields(row) {
|
|
||||||
brw.flush()
|
|
||||||
|
|
||||||
brw.rcs = brw.rcs[:0]
|
|
||||||
for _, field := range row {
|
|
||||||
brw.rcs = appendResultColumnWithName(brw.rcs, field.Name)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for i, field := range row {
|
|
||||||
brw.rcs[i].addValue(field.Value)
|
|
||||||
}
|
|
||||||
brw.rowsCount++
|
|
||||||
if rand.Intn(5) == 0 {
|
|
||||||
brw.flush()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (brw *testBlockResultWriter) areSameFields(row []Field) bool {
|
|
||||||
if len(brw.rcs) != len(row) {
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
for i, rc := range brw.rcs {
|
|
||||||
if rc.name != row[i].Name {
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true
|
|
||||||
}
|
|
||||||
|
|
||||||
func (brw *testBlockResultWriter) flush() {
|
|
||||||
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()
|
|
||||||
for i := range brw.rcs {
|
|
||||||
brw.rcs[i].resetValues()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func newTestPipeProcessor() *testPipeProcessor {
|
|
||||||
return &testPipeProcessor{}
|
|
||||||
}
|
|
||||||
|
|
||||||
type testPipeProcessor struct {
|
|
||||||
resultRowsLock sync.Mutex
|
|
||||||
resultRows [][]Field
|
|
||||||
}
|
|
||||||
|
|
||||||
func (pp *testPipeProcessor) writeBlock(_ uint, br *blockResult) {
|
|
||||||
cs := br.getColumns()
|
|
||||||
var columnValues [][]string
|
|
||||||
for _, c := range cs {
|
|
||||||
values := c.getValues(br)
|
|
||||||
columnValues = append(columnValues, values)
|
|
||||||
}
|
|
||||||
|
|
||||||
for i := range br.timestamps {
|
|
||||||
row := make([]Field, len(columnValues))
|
|
||||||
for j, values := range columnValues {
|
|
||||||
r := &row[j]
|
|
||||||
r.Name = strings.Clone(cs[j].name)
|
|
||||||
r.Value = strings.Clone(values[i])
|
|
||||||
}
|
|
||||||
pp.resultRowsLock.Lock()
|
|
||||||
pp.resultRows = append(pp.resultRows, row)
|
|
||||||
pp.resultRowsLock.Unlock()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (pp *testPipeProcessor) flush() error {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (pp *testPipeProcessor) expectRows(t *testing.T, expectedRows [][]Field) {
|
|
||||||
t.Helper()
|
|
||||||
|
|
||||||
if len(pp.resultRows) != len(expectedRows) {
|
|
||||||
t.Fatalf("unexpected number of rows; got %d; want %d\nrows got\n%s\nrows expected\n%s",
|
|
||||||
len(pp.resultRows), len(expectedRows), rowsToString(pp.resultRows), rowsToString(expectedRows))
|
|
||||||
}
|
|
||||||
|
|
||||||
sortTestRows(pp.resultRows)
|
|
||||||
sortTestRows(expectedRows)
|
|
||||||
|
|
||||||
for i, resultRow := range pp.resultRows {
|
|
||||||
expectedRow := expectedRows[i]
|
|
||||||
if len(resultRow) != len(expectedRow) {
|
|
||||||
t.Fatalf("unexpected number of fields at row #%d; got %d; want %d\nrow got\n%s\nrow expected\n%s",
|
|
||||||
i, len(resultRow), len(expectedRow), rowToString(resultRow), rowToString(expectedRow))
|
|
||||||
}
|
|
||||||
for j, resultField := range resultRow {
|
|
||||||
expectedField := expectedRow[j]
|
|
||||||
if resultField.Name != expectedField.Name {
|
|
||||||
t.Fatalf("unexpected field name at row #%d; got %q; want %q\nrow got\n%s\nrow expected\n%s",
|
|
||||||
i, resultField.Name, expectedField.Name, rowToString(resultRow), rowToString(expectedRow))
|
|
||||||
}
|
|
||||||
if resultField.Value != expectedField.Value {
|
|
||||||
t.Fatalf("unexpected value for field %q at row #%d; got %q; want %q\nrow got\n%s\nrow expected\n%s",
|
|
||||||
resultField.Name, i, resultField.Value, expectedField.Value, rowToString(resultRow), rowToString(expectedRow))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func sortTestRows(rows [][]Field) {
|
|
||||||
for _, row := range rows {
|
|
||||||
sortTestFields(row)
|
|
||||||
}
|
|
||||||
slices.SortFunc(rows, func(a, b []Field) int {
|
|
||||||
reverse := false
|
|
||||||
if len(a) > len(b) {
|
|
||||||
reverse = true
|
|
||||||
a, b = b, a
|
|
||||||
}
|
|
||||||
for i, fA := range a {
|
|
||||||
fB := b[i]
|
|
||||||
result := cmpTestFields(fA, fB)
|
|
||||||
if result == 0 {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
if reverse {
|
|
||||||
result = -result
|
|
||||||
}
|
|
||||||
return result
|
|
||||||
}
|
|
||||||
if len(a) == len(b) {
|
|
||||||
return 0
|
|
||||||
}
|
|
||||||
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 {
|
|
||||||
a[i] = rowToString(row)
|
|
||||||
}
|
|
||||||
return strings.Join(a, "\n")
|
|
||||||
}
|
|
||||||
|
|
||||||
func rowToString(row []Field) string {
|
|
||||||
a := make([]string, len(row))
|
|
||||||
for i, f := range row {
|
|
||||||
a[i] = f.String()
|
|
||||||
}
|
|
||||||
return "{" + strings.Join(a, ",") + "}"
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestPipeUnpackJSONUpdateNeededFields(t *testing.T) {
|
func TestPipeUnpackJSONUpdateNeededFields(t *testing.T) {
|
||||||
f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
|
|
|
@ -54,7 +54,25 @@ func (pu *pipeUnpackLogfmt) updateNeededFields(neededFields, unneededFields fiel
|
||||||
updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff, neededFields, unneededFields)
|
updateNeededFieldsForUnpackPipe(pu.fromField, pu.fields, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff, neededFields, unneededFields)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pu *pipeUnpackLogfmt) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pu *pipeUnpackLogfmt) optimize() {
|
||||||
|
pu.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnpackLogfmt) hasFilterInWithQuery() bool {
|
||||||
|
return pu.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnpackLogfmt) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pu.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
puNew := *pu
|
||||||
|
puNew.iff = iffNew
|
||||||
|
return &puNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnpackLogfmt) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
unpackLogfmt := func(uctx *fieldsUnpackerContext, s string) {
|
unpackLogfmt := func(uctx *fieldsUnpackerContext, s string) {
|
||||||
p := getLogfmtParser()
|
p := getLogfmtParser()
|
||||||
|
|
||||||
|
@ -82,8 +100,7 @@ func (pu *pipeUnpackLogfmt) newPipeProcessor(workersCount int, _ <-chan struct{}
|
||||||
putLogfmtParser(p)
|
putLogfmtParser(p)
|
||||||
}
|
}
|
||||||
|
|
||||||
return newPipeUnpackProcessor(workersCount, unpackLogfmt, ppBase, pu.fromField, pu.resultPrefix, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff)
|
return newPipeUnpackProcessor(workersCount, unpackLogfmt, ppNext, pu.fromField, pu.resultPrefix, pu.keepOriginalFields, pu.skipEmptyResults, pu.iff)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) {
|
func parsePipeUnpackLogfmt(lex *lexer) (*pipeUnpackLogfmt, error) {
|
||||||
|
|
|
@ -151,7 +151,6 @@ func TestPipeUnpackLogfmt(t *testing.T) {
|
||||||
},
|
},
|
||||||
}, [][]Field{
|
}, [][]Field{
|
||||||
{
|
{
|
||||||
{"foo", ""},
|
|
||||||
{"_msg", `foo=bar baz="x y=z" a=b`},
|
{"_msg", `foo=bar baz="x y=z" a=b`},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
@ -291,7 +290,6 @@ func TestPipeUnpackLogfmt(t *testing.T) {
|
||||||
{"y", `abc`},
|
{"y", `abc`},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"y", ""},
|
|
||||||
{"z", `foobar`},
|
{"z", `foobar`},
|
||||||
{"x", `z=bar`},
|
{"x", `z=bar`},
|
||||||
},
|
},
|
||||||
|
|
284
lib/logstorage/pipe_unroll.go
Normal file
284
lib/logstorage/pipe_unroll.go
Normal file
|
@ -0,0 +1,284 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"slices"
|
||||||
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/valyala/fastjson"
|
||||||
|
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||||
|
)
|
||||||
|
|
||||||
|
// pipeUnroll processes '| unroll ...' pipe.
|
||||||
|
//
|
||||||
|
// See https://docs.victoriametrics.com/victorialogs/logsql/#unroll-pipe
|
||||||
|
type pipeUnroll struct {
|
||||||
|
// fields to unroll
|
||||||
|
fields []string
|
||||||
|
|
||||||
|
// iff is an optional filter for skipping the unroll
|
||||||
|
iff *ifFilter
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) String() string {
|
||||||
|
s := "unroll"
|
||||||
|
if pu.iff != nil {
|
||||||
|
s += " " + pu.iff.String()
|
||||||
|
}
|
||||||
|
s += " by (" + fieldNamesString(pu.fields) + ")"
|
||||||
|
return s
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) optimize() {
|
||||||
|
pu.iff.optimizeFilterIn()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) hasFilterInWithQuery() bool {
|
||||||
|
return pu.iff.hasFilterInWithQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) initFilterInValues(cache map[string][]string, getFieldValuesFunc getFieldValuesFunc) (pipe, error) {
|
||||||
|
iffNew, err := pu.iff.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
puNew := *pu
|
||||||
|
puNew.iff = iffNew
|
||||||
|
return &puNew, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
|
if neededFields.contains("*") {
|
||||||
|
unneededFieldsCount := 0
|
||||||
|
for _, f := range pu.fields {
|
||||||
|
if unneededFields.contains(f) {
|
||||||
|
unneededFieldsCount++
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if unneededFieldsCount < len(pu.fields) && pu.iff != nil {
|
||||||
|
unneededFields.removeFields(pu.iff.neededFields)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
needIfFields := false
|
||||||
|
for _, f := range pu.fields {
|
||||||
|
if neededFields.contains(f) {
|
||||||
|
needIfFields = true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if needIfFields && pu.iff != nil {
|
||||||
|
neededFields.addFields(pu.iff.neededFields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pu *pipeUnroll) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppNext pipeProcessor) pipeProcessor {
|
||||||
|
return &pipeUnrollProcessor{
|
||||||
|
pu: pu,
|
||||||
|
ppNext: ppNext,
|
||||||
|
|
||||||
|
shards: make([]pipeUnrollProcessorShard, workersCount),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUnrollProcessor struct {
|
||||||
|
pu *pipeUnroll
|
||||||
|
ppNext pipeProcessor
|
||||||
|
|
||||||
|
shards []pipeUnrollProcessorShard
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUnrollProcessorShard struct {
|
||||||
|
pipeUnrollProcessorShardNopad
|
||||||
|
|
||||||
|
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||||
|
_ [128 - unsafe.Sizeof(pipeUnrollProcessorShardNopad{})%128]byte
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUnrollProcessorShardNopad struct {
|
||||||
|
bm bitmap
|
||||||
|
|
||||||
|
wctx pipeUnpackWriteContext
|
||||||
|
a arena
|
||||||
|
|
||||||
|
columnValues [][]string
|
||||||
|
unrolledValues [][]string
|
||||||
|
valuesBuf []string
|
||||||
|
fields []Field
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pup *pipeUnrollProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
if len(br.timestamps) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
pu := pup.pu
|
||||||
|
shard := &pup.shards[workerID]
|
||||||
|
shard.wctx.init(workerID, pup.ppNext, false, false, br)
|
||||||
|
|
||||||
|
bm := &shard.bm
|
||||||
|
bm.init(len(br.timestamps))
|
||||||
|
bm.setBits()
|
||||||
|
if iff := pu.iff; iff != nil {
|
||||||
|
iff.f.applyToBlockResult(br, bm)
|
||||||
|
if bm.isZero() {
|
||||||
|
pup.ppNext.writeBlock(workerID, br)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
shard.columnValues = slicesutil.SetLength(shard.columnValues, len(pu.fields))
|
||||||
|
columnValues := shard.columnValues
|
||||||
|
for i, f := range pu.fields {
|
||||||
|
c := br.getColumnByName(f)
|
||||||
|
columnValues[i] = c.getValues(br)
|
||||||
|
}
|
||||||
|
|
||||||
|
fields := shard.fields
|
||||||
|
for rowIdx := range br.timestamps {
|
||||||
|
if bm.isSetBit(rowIdx) {
|
||||||
|
shard.writeUnrolledFields(br, pu.fields, columnValues, rowIdx)
|
||||||
|
} else {
|
||||||
|
fields = fields[:0]
|
||||||
|
for i, f := range pu.fields {
|
||||||
|
v := columnValues[i][rowIdx]
|
||||||
|
fields = append(fields, Field{
|
||||||
|
Name: f,
|
||||||
|
Value: v,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
shard.wctx.writeRow(rowIdx, fields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
shard.wctx.flush()
|
||||||
|
shard.wctx.reset()
|
||||||
|
shard.a.reset()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (shard *pipeUnrollProcessorShard) writeUnrolledFields(br *blockResult, fieldNames []string, columnValues [][]string, rowIdx int) {
|
||||||
|
// unroll values at rowIdx row
|
||||||
|
|
||||||
|
shard.unrolledValues = slicesutil.SetLength(shard.unrolledValues, len(columnValues))
|
||||||
|
unrolledValues := shard.unrolledValues
|
||||||
|
|
||||||
|
valuesBuf := shard.valuesBuf[:0]
|
||||||
|
for i, values := range columnValues {
|
||||||
|
v := values[rowIdx]
|
||||||
|
valuesBufLen := len(valuesBuf)
|
||||||
|
valuesBuf = unpackJSONArray(valuesBuf, &shard.a, v)
|
||||||
|
unrolledValues[i] = valuesBuf[valuesBufLen:]
|
||||||
|
}
|
||||||
|
shard.valuesBuf = valuesBuf
|
||||||
|
|
||||||
|
// find the number of rows across unrolled values
|
||||||
|
rows := len(unrolledValues[0])
|
||||||
|
for _, values := range unrolledValues[1:] {
|
||||||
|
if len(values) > rows {
|
||||||
|
rows = len(values)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if rows == 0 {
|
||||||
|
// Unroll too a single row with empty unrolled values.
|
||||||
|
rows = 1
|
||||||
|
}
|
||||||
|
|
||||||
|
// write unrolled values to the next pipe.
|
||||||
|
fields := shard.fields
|
||||||
|
for unrollIdx := 0; unrollIdx < rows; unrollIdx++ {
|
||||||
|
fields = fields[:0]
|
||||||
|
for i, values := range unrolledValues {
|
||||||
|
v := ""
|
||||||
|
if unrollIdx < len(values) {
|
||||||
|
v = values[unrollIdx]
|
||||||
|
}
|
||||||
|
fields = append(fields, Field{
|
||||||
|
Name: fieldNames[i],
|
||||||
|
Value: v,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
shard.wctx.writeRow(rowIdx, fields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pup *pipeUnrollProcessor) flush() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func parsePipeUnroll(lex *lexer) (*pipeUnroll, error) {
|
||||||
|
if !lex.isKeyword("unroll") {
|
||||||
|
return nil, fmt.Errorf("unexpected token: %q; want %q", lex.token, "unroll")
|
||||||
|
}
|
||||||
|
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 by (...)
|
||||||
|
if lex.isKeyword("by") {
|
||||||
|
lex.nextToken()
|
||||||
|
}
|
||||||
|
|
||||||
|
fields, err := parseFieldNamesInParens(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse 'by(...)' at 'unroll': %w", err)
|
||||||
|
}
|
||||||
|
if len(fields) == 0 {
|
||||||
|
return nil, fmt.Errorf("'by(...)' at 'unroll' must contain at least a single field")
|
||||||
|
}
|
||||||
|
if slices.Contains(fields, "*") {
|
||||||
|
return nil, fmt.Errorf("unroll by '*' isn't supported")
|
||||||
|
}
|
||||||
|
|
||||||
|
pu := &pipeUnroll{
|
||||||
|
fields: fields,
|
||||||
|
iff: iff,
|
||||||
|
}
|
||||||
|
|
||||||
|
return pu, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func unpackJSONArray(dst []string, a *arena, s string) []string {
|
||||||
|
if s == "" || s[0] != '[' {
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
|
||||||
|
p := jspp.Get()
|
||||||
|
defer jspp.Put(p)
|
||||||
|
|
||||||
|
jsv, err := p.Parse(s)
|
||||||
|
if err != nil {
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
jsa, err := jsv.Array()
|
||||||
|
if err != nil {
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
for _, jsv := range jsa {
|
||||||
|
if jsv.Type() == fastjson.TypeString {
|
||||||
|
sb, err := jsv.StringBytes()
|
||||||
|
if err != nil {
|
||||||
|
logger.Panicf("BUG: unexpected error returned from StringBytes(): %s", err)
|
||||||
|
}
|
||||||
|
v := a.copyBytesToString(sb)
|
||||||
|
dst = append(dst, v)
|
||||||
|
} else {
|
||||||
|
bLen := len(a.b)
|
||||||
|
a.b = jsv.MarshalTo(a.b)
|
||||||
|
v := bytesutil.ToUnsafeString(a.b[bLen:])
|
||||||
|
dst = append(dst, v)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
|
||||||
|
var jspp fastjson.ParserPool
|
261
lib/logstorage/pipe_unroll_test.go
Normal file
261
lib/logstorage/pipe_unroll_test.go
Normal file
|
@ -0,0 +1,261 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"reflect"
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestParsePipeUnrollSuccess(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeSuccess(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`unroll by (foo)`)
|
||||||
|
f(`unroll if (x:y) by (foo, bar)`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestParsePipeUrollFailure(t *testing.T) {
|
||||||
|
f := func(pipeStr string) {
|
||||||
|
t.Helper()
|
||||||
|
expectParsePipeFailure(t, pipeStr)
|
||||||
|
}
|
||||||
|
|
||||||
|
f(`unroll`)
|
||||||
|
f(`unroll by ()`)
|
||||||
|
f(`unroll by (*)`)
|
||||||
|
f(`unroll by (f, *)`)
|
||||||
|
f(`unroll by`)
|
||||||
|
f(`unroll (`)
|
||||||
|
f(`unroll by (foo) bar`)
|
||||||
|
f(`unroll by (x) if (a:b)`)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipeUnroll(t *testing.T) {
|
||||||
|
f := func(pipeStr string, rows, rowsExpected [][]Field) {
|
||||||
|
t.Helper()
|
||||||
|
expectPipeResults(t, pipeStr, rows, rowsExpected)
|
||||||
|
}
|
||||||
|
|
||||||
|
// unroll by missing field
|
||||||
|
f("unroll (x)", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `["foo",1,{"baz":"x"},[1,2],null,NaN]`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `["foo",1,{"baz":"x"},[1,2],null,NaN]`},
|
||||||
|
{"q", "w"},
|
||||||
|
{"x", ""},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// unroll by field without JSON array
|
||||||
|
f("unroll (q)", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `["foo",1,{"baz":"x"},[1,2],null,NaN]`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `["foo",1,{"baz":"x"},[1,2],null,NaN]`},
|
||||||
|
{"q", ""},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// unroll by a single field
|
||||||
|
f("unroll (a)", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `["foo",1,{"baz":"x"},[1,2],null,NaN]`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", "foo"},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "1"},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `{"baz":"x"}`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "[1,2]"},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "null"},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "NaN"},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", ""},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// unroll by multiple fields
|
||||||
|
f("unroll by (timestamp, value)", [][]Field{
|
||||||
|
{
|
||||||
|
{"timestamp", "[1,2,3]"},
|
||||||
|
{"value", `["foo","bar","baz"]`},
|
||||||
|
{"other", "abc"},
|
||||||
|
{"x", "y"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "[1]"},
|
||||||
|
{"value", `["foo","bar"]`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "[1]"},
|
||||||
|
{"value", `bar`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"timestamp", "1"},
|
||||||
|
{"value", "foo"},
|
||||||
|
{"other", "abc"},
|
||||||
|
{"x", "y"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "2"},
|
||||||
|
{"value", "bar"},
|
||||||
|
{"other", "abc"},
|
||||||
|
{"x", "y"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "3"},
|
||||||
|
{"value", "baz"},
|
||||||
|
{"other", "abc"},
|
||||||
|
{"x", "y"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "1"},
|
||||||
|
{"value", "foo"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", ""},
|
||||||
|
{"value", "bar"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"timestamp", "1"},
|
||||||
|
{"value", ""},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// conditional unroll by missing field
|
||||||
|
f("unroll if (q:abc) (a)", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `asd`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `["foo",123]`},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `asd`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "foo"},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "123"},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// unroll by non-existing field
|
||||||
|
f("unroll (a)", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `asd`},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `["foo",123]`},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", ``},
|
||||||
|
{"q", "w"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "foo"},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "123"},
|
||||||
|
{"q", "abc"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestPipeUnrollUpdateNeededFields(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("unroll (x)", "*", "", "*", "")
|
||||||
|
f("unroll (x, y)", "*", "", "*", "")
|
||||||
|
f("unroll if (y:z) (a, b)", "*", "", "*", "")
|
||||||
|
|
||||||
|
// all the needed fields, unneeded fields do not intersect with src
|
||||||
|
f("unroll (x)", "*", "f1,f2", "*", "f1,f2")
|
||||||
|
f("unroll if (a:b) (x)", "*", "f1,f2", "*", "f1,f2")
|
||||||
|
f("unroll if (f1:b) (x)", "*", "f1,f2", "*", "f2")
|
||||||
|
|
||||||
|
// all the needed fields, unneeded fields intersect with src
|
||||||
|
f("unroll (x)", "*", "f2,x", "*", "f2,x")
|
||||||
|
f("unroll if (a:b) (x)", "*", "f2,x", "*", "f2,x")
|
||||||
|
f("unroll if (f2:b) (x)", "*", "f2,x", "*", "f2,x")
|
||||||
|
|
||||||
|
// needed fields do not intersect with src
|
||||||
|
f("unroll (x)", "f1,f2", "", "f1,f2", "")
|
||||||
|
f("unroll if (a:b) (x)", "f1,f2", "", "f1,f2", "")
|
||||||
|
|
||||||
|
// needed fields intersect with src
|
||||||
|
f("unroll (x)", "f2,x", "", "f2,x", "")
|
||||||
|
f("unroll if (a:b) (x)", "f2,x", "", "a,f2,x", "")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestUnpackJSONArray(t *testing.T) {
|
||||||
|
f := func(s string, resultExpected []string) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
var a arena
|
||||||
|
result := unpackJSONArray(nil, &a, s)
|
||||||
|
if !reflect.DeepEqual(result, resultExpected) {
|
||||||
|
t.Fatalf("unexpected result for unpackJSONArray(%q)\ngot\n%q\nwant\n%q", s, result, resultExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", nil)
|
||||||
|
f("123", nil)
|
||||||
|
f("foo", nil)
|
||||||
|
f(`"foo"`, nil)
|
||||||
|
f(`{"foo":"bar"}`, nil)
|
||||||
|
f(`[foo`, nil)
|
||||||
|
f(`[]`, nil)
|
||||||
|
f(`[1]`, []string{"1"})
|
||||||
|
f(`[1,"foo",["bar",12],{"baz":"x"},NaN,null]`, []string{"1", "foo", `["bar",12]`, `{"baz":"x"}`, "NaN", "null"})
|
||||||
|
}
|
103
lib/logstorage/pipe_update.go
Normal file
103
lib/logstorage/pipe_update.go
Normal file
|
@ -0,0 +1,103 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"unsafe"
|
||||||
|
)
|
||||||
|
|
||||||
|
func updateNeededFieldsForUpdatePipe(neededFields, unneededFields fieldsSet, field string, iff *ifFilter) {
|
||||||
|
if neededFields.contains("*") {
|
||||||
|
if !unneededFields.contains(field) && iff != nil {
|
||||||
|
unneededFields.removeFields(iff.neededFields)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if neededFields.contains(field) && iff != nil {
|
||||||
|
neededFields.addFields(iff.neededFields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func newPipeUpdateProcessor(workersCount int, updateFunc func(a *arena, v string) string, ppNext pipeProcessor, field string, iff *ifFilter) pipeProcessor {
|
||||||
|
return &pipeUpdateProcessor{
|
||||||
|
updateFunc: updateFunc,
|
||||||
|
|
||||||
|
field: field,
|
||||||
|
iff: iff,
|
||||||
|
|
||||||
|
ppNext: ppNext,
|
||||||
|
|
||||||
|
shards: make([]pipeUpdateProcessorShard, workersCount),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUpdateProcessor struct {
|
||||||
|
updateFunc func(a *arena, v string) string
|
||||||
|
|
||||||
|
field string
|
||||||
|
iff *ifFilter
|
||||||
|
|
||||||
|
ppNext pipeProcessor
|
||||||
|
|
||||||
|
shards []pipeUpdateProcessorShard
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUpdateProcessorShard struct {
|
||||||
|
pipeUpdateProcessorShardNopad
|
||||||
|
|
||||||
|
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||||
|
_ [128 - unsafe.Sizeof(pipeUpdateProcessorShardNopad{})%128]byte
|
||||||
|
}
|
||||||
|
|
||||||
|
type pipeUpdateProcessorShardNopad struct {
|
||||||
|
bm bitmap
|
||||||
|
|
||||||
|
rc resultColumn
|
||||||
|
a arena
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pup *pipeUpdateProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
if len(br.timestamps) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
shard := &pup.shards[workerID]
|
||||||
|
|
||||||
|
bm := &shard.bm
|
||||||
|
bm.init(len(br.timestamps))
|
||||||
|
bm.setBits()
|
||||||
|
if iff := pup.iff; iff != nil {
|
||||||
|
iff.f.applyToBlockResult(br, bm)
|
||||||
|
if bm.isZero() {
|
||||||
|
pup.ppNext.writeBlock(workerID, br)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
shard.rc.name = pup.field
|
||||||
|
|
||||||
|
c := br.getColumnByName(pup.field)
|
||||||
|
values := c.getValues(br)
|
||||||
|
|
||||||
|
hadUpdates := false
|
||||||
|
vPrev := ""
|
||||||
|
for rowIdx, v := range values {
|
||||||
|
if bm.isSetBit(rowIdx) {
|
||||||
|
if !hadUpdates || vPrev != v {
|
||||||
|
vPrev = v
|
||||||
|
hadUpdates = true
|
||||||
|
|
||||||
|
v = pup.updateFunc(&shard.a, v)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
shard.rc.addValue(v)
|
||||||
|
}
|
||||||
|
|
||||||
|
br.addResultColumn(&shard.rc)
|
||||||
|
pup.ppNext.writeBlock(workerID, br)
|
||||||
|
|
||||||
|
shard.rc.reset()
|
||||||
|
shard.a.reset()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pup *pipeUpdateProcessor) flush() error {
|
||||||
|
return nil
|
||||||
|
}
|
224
lib/logstorage/pipe_utils_test.go
Normal file
224
lib/logstorage/pipe_utils_test.go
Normal file
|
@ -0,0 +1,224 @@
|
||||||
|
package logstorage
|
||||||
|
|
||||||
|
import (
|
||||||
|
"math/rand"
|
||||||
|
"slices"
|
||||||
|
"strings"
|
||||||
|
"sync"
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func expectPipeResults(t *testing.T, pipeStr string, rows, rowsExpected [][]Field) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
lex := newLexer(pipeStr)
|
||||||
|
p, err := parsePipe(lex)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("unexpected error when parsing %q: %s", pipeStr, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
workersCount := 5
|
||||||
|
stopCh := make(chan struct{})
|
||||||
|
cancel := func() {}
|
||||||
|
ppTest := newTestPipeProcessor()
|
||||||
|
pp := p.newPipeProcessor(workersCount, stopCh, cancel, ppTest)
|
||||||
|
|
||||||
|
brw := newTestBlockResultWriter(workersCount, pp)
|
||||||
|
for _, row := range rows {
|
||||||
|
brw.writeRow(row)
|
||||||
|
}
|
||||||
|
brw.flush()
|
||||||
|
pp.flush()
|
||||||
|
|
||||||
|
ppTest.expectRows(t, rowsExpected)
|
||||||
|
}
|
||||||
|
|
||||||
|
func newTestBlockResultWriter(workersCount int, ppNext pipeProcessor) *testBlockResultWriter {
|
||||||
|
return &testBlockResultWriter{
|
||||||
|
workersCount: workersCount,
|
||||||
|
ppNext: ppNext,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type testBlockResultWriter struct {
|
||||||
|
workersCount int
|
||||||
|
ppNext pipeProcessor
|
||||||
|
rcs []resultColumn
|
||||||
|
br blockResult
|
||||||
|
|
||||||
|
rowsCount int
|
||||||
|
}
|
||||||
|
|
||||||
|
func (brw *testBlockResultWriter) writeRow(row []Field) {
|
||||||
|
if !brw.areSameFields(row) {
|
||||||
|
brw.flush()
|
||||||
|
|
||||||
|
brw.rcs = brw.rcs[:0]
|
||||||
|
for _, field := range row {
|
||||||
|
brw.rcs = appendResultColumnWithName(brw.rcs, field.Name)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for i, field := range row {
|
||||||
|
brw.rcs[i].addValue(field.Value)
|
||||||
|
}
|
||||||
|
brw.rowsCount++
|
||||||
|
if rand.Intn(5) == 0 {
|
||||||
|
brw.flush()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (brw *testBlockResultWriter) areSameFields(row []Field) bool {
|
||||||
|
if len(brw.rcs) != len(row) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
for i, rc := range brw.rcs {
|
||||||
|
if rc.name != row[i].Name {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
func (brw *testBlockResultWriter) flush() {
|
||||||
|
brw.br.setResultColumns(brw.rcs, brw.rowsCount)
|
||||||
|
brw.rowsCount = 0
|
||||||
|
workerID := rand.Intn(brw.workersCount)
|
||||||
|
brw.ppNext.writeBlock(uint(workerID), &brw.br)
|
||||||
|
brw.br.reset()
|
||||||
|
for i := range brw.rcs {
|
||||||
|
brw.rcs[i].resetValues()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func newTestPipeProcessor() *testPipeProcessor {
|
||||||
|
return &testPipeProcessor{}
|
||||||
|
}
|
||||||
|
|
||||||
|
type testPipeProcessor struct {
|
||||||
|
resultRowsLock sync.Mutex
|
||||||
|
resultRows [][]Field
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *testPipeProcessor) writeBlock(_ uint, br *blockResult) {
|
||||||
|
cs := br.getColumns()
|
||||||
|
var columnValues [][]string
|
||||||
|
for _, c := range cs {
|
||||||
|
values := c.getValues(br)
|
||||||
|
columnValues = append(columnValues, values)
|
||||||
|
}
|
||||||
|
|
||||||
|
for i := range br.timestamps {
|
||||||
|
row := make([]Field, len(columnValues))
|
||||||
|
for j, values := range columnValues {
|
||||||
|
r := &row[j]
|
||||||
|
r.Name = strings.Clone(cs[j].name)
|
||||||
|
r.Value = strings.Clone(values[i])
|
||||||
|
}
|
||||||
|
pp.resultRowsLock.Lock()
|
||||||
|
pp.resultRows = append(pp.resultRows, row)
|
||||||
|
pp.resultRowsLock.Unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *testPipeProcessor) flush() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (pp *testPipeProcessor) expectRows(t *testing.T, expectedRows [][]Field) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
if len(pp.resultRows) != len(expectedRows) {
|
||||||
|
t.Fatalf("unexpected number of rows; got %d; want %d\nrows got\n%s\nrows expected\n%s",
|
||||||
|
len(pp.resultRows), len(expectedRows), rowsToString(pp.resultRows), rowsToString(expectedRows))
|
||||||
|
}
|
||||||
|
|
||||||
|
sortTestRows(pp.resultRows)
|
||||||
|
sortTestRows(expectedRows)
|
||||||
|
|
||||||
|
for i, resultRow := range pp.resultRows {
|
||||||
|
expectedRow := expectedRows[i]
|
||||||
|
if len(resultRow) != len(expectedRow) {
|
||||||
|
t.Fatalf("unexpected number of fields at row #%d; got %d; want %d\nrow got\n%s\nrow expected\n%s",
|
||||||
|
i, len(resultRow), len(expectedRow), rowToString(resultRow), rowToString(expectedRow))
|
||||||
|
}
|
||||||
|
for j, resultField := range resultRow {
|
||||||
|
expectedField := expectedRow[j]
|
||||||
|
if resultField.Name != expectedField.Name {
|
||||||
|
t.Fatalf("unexpected field name at row #%d; got %q; want %q\nrow got\n%s\nrow expected\n%s",
|
||||||
|
i, resultField.Name, expectedField.Name, rowToString(resultRow), rowToString(expectedRow))
|
||||||
|
}
|
||||||
|
if resultField.Value != expectedField.Value {
|
||||||
|
t.Fatalf("unexpected value for field %q at row #%d; got %q; want %q\nrow got\n%s\nrow expected\n%s",
|
||||||
|
resultField.Name, i, resultField.Value, expectedField.Value, rowToString(resultRow), rowToString(expectedRow))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func sortTestRows(rows [][]Field) {
|
||||||
|
for _, row := range rows {
|
||||||
|
sortTestFields(row)
|
||||||
|
}
|
||||||
|
slices.SortFunc(rows, func(a, b []Field) int {
|
||||||
|
reverse := false
|
||||||
|
if len(a) > len(b) {
|
||||||
|
reverse = true
|
||||||
|
a, b = b, a
|
||||||
|
}
|
||||||
|
for i, fA := range a {
|
||||||
|
fB := b[i]
|
||||||
|
result := cmpTestFields(fA, fB)
|
||||||
|
if result == 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if reverse {
|
||||||
|
result = -result
|
||||||
|
}
|
||||||
|
return result
|
||||||
|
}
|
||||||
|
if len(a) == len(b) {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
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 {
|
||||||
|
a[i] = rowToString(row)
|
||||||
|
}
|
||||||
|
return strings.Join(a, "\n")
|
||||||
|
}
|
||||||
|
|
||||||
|
func rowToString(row []Field) string {
|
||||||
|
a := make([]string, len(row))
|
||||||
|
for i, f := range row {
|
||||||
|
a[i] = f.String()
|
||||||
|
}
|
||||||
|
return "{" + strings.Join(a, ",") + "}"
|
||||||
|
}
|
|
@ -275,7 +275,7 @@ func TestStatsFieldsMax(t *testing.T) {
|
||||||
{
|
{
|
||||||
{"a", "1"},
|
{"a", "1"},
|
||||||
{"b", ""},
|
{"b", ""},
|
||||||
{"x", `{"_msg":"def","a":"1","c":"foo","b":""}`},
|
{"x", `{"_msg":"def","a":"1","c":"foo"}`},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"a", "3"},
|
{"a", "3"},
|
||||||
|
|
|
@ -274,7 +274,7 @@ func TestStatsFieldsMin(t *testing.T) {
|
||||||
{
|
{
|
||||||
{"a", "1"},
|
{"a", "1"},
|
||||||
{"b", ""},
|
{"b", ""},
|
||||||
{"x", `{"_msg":"def","a":"1","c":"foo","b":""}`},
|
{"x", `{"_msg":"def","a":"1","c":"foo"}`},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"a", "3"},
|
{"a", "3"},
|
||||||
|
|
|
@ -229,7 +229,7 @@ func (s *Storage) getFieldValuesNoHits(ctx context.Context, tenantIDs []TenantID
|
||||||
func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]ValueWithHits, error) {
|
func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]ValueWithHits, error) {
|
||||||
pipes := append([]pipe{}, q.pipes...)
|
pipes := append([]pipe{}, q.pipes...)
|
||||||
quotedFieldName := quoteTokenIfNeeded(fieldName)
|
quotedFieldName := quoteTokenIfNeeded(fieldName)
|
||||||
pipeStr := fmt.Sprintf("uniq by (%s) hits limit %d", quotedFieldName, limit)
|
pipeStr := fmt.Sprintf("uniq by (%s) with hits limit %d", quotedFieldName, limit)
|
||||||
lex := newLexer(pipeStr)
|
lex := newLexer(pipeStr)
|
||||||
|
|
||||||
pu, err := parsePipeUniq(lex)
|
pu, err := parsePipeUniq(lex)
|
||||||
|
@ -288,18 +288,18 @@ func sortValuesWithHits(results []ValueWithHits) {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreamLabelNames returns stream label names from q results for the given tenantIDs.
|
// GetStreamFieldNames returns stream field names from q results for the given tenantIDs.
|
||||||
func (s *Storage) GetStreamLabelNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]ValueWithHits, error) {
|
func (s *Storage) GetStreamFieldNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]ValueWithHits, error) {
|
||||||
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
m := make(map[string]*uint64)
|
m := make(map[string]*uint64)
|
||||||
forEachStreamLabel(streams, func(label Field, hits uint64) {
|
forEachStreamField(streams, func(f Field, hits uint64) {
|
||||||
pHits, ok := m[label.Name]
|
pHits, ok := m[f.Name]
|
||||||
if !ok {
|
if !ok {
|
||||||
nameCopy := strings.Clone(label.Name)
|
nameCopy := strings.Clone(f.Name)
|
||||||
hitsLocal := uint64(0)
|
hitsLocal := uint64(0)
|
||||||
pHits = &hitsLocal
|
pHits = &hitsLocal
|
||||||
m[nameCopy] = pHits
|
m[nameCopy] = pHits
|
||||||
|
@ -310,23 +310,23 @@ func (s *Storage) GetStreamLabelNames(ctx context.Context, tenantIDs []TenantID,
|
||||||
return names, nil
|
return names, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreamLabelValues returns stream label values for the given labelName from q results for the given tenantIDs.
|
// GetStreamFieldValues returns stream field values for the given fieldName from q results for the given tenantIDs.
|
||||||
//
|
//
|
||||||
// If limit > 9, then up to limit unique label values are returned.
|
// If limit > 9, then up to limit unique values are returned.
|
||||||
func (s *Storage) GetStreamLabelValues(ctx context.Context, tenantIDs []TenantID, q *Query, labelName string, limit uint64) ([]ValueWithHits, error) {
|
func (s *Storage) GetStreamFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]ValueWithHits, error) {
|
||||||
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
m := make(map[string]*uint64)
|
m := make(map[string]*uint64)
|
||||||
forEachStreamLabel(streams, func(label Field, hits uint64) {
|
forEachStreamField(streams, func(f Field, hits uint64) {
|
||||||
if label.Name != labelName {
|
if f.Name != fieldName {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
pHits, ok := m[label.Value]
|
pHits, ok := m[f.Value]
|
||||||
if !ok {
|
if !ok {
|
||||||
valueCopy := strings.Clone(label.Value)
|
valueCopy := strings.Clone(f.Value)
|
||||||
hitsLocal := uint64(0)
|
hitsLocal := uint64(0)
|
||||||
pHits = &hitsLocal
|
pHits = &hitsLocal
|
||||||
m[valueCopy] = pHits
|
m[valueCopy] = pHits
|
||||||
|
@ -429,33 +429,8 @@ func hasFilterInWithQueryForFilter(f filter) bool {
|
||||||
|
|
||||||
func hasFilterInWithQueryForPipes(pipes []pipe) bool {
|
func hasFilterInWithQueryForPipes(pipes []pipe) bool {
|
||||||
for _, p := range pipes {
|
for _, p := range pipes {
|
||||||
switch t := p.(type) {
|
if p.hasFilterInWithQuery() {
|
||||||
case *pipeStats:
|
return true
|
||||||
for _, f := range t.funcs {
|
|
||||||
if f.iff.hasFilterInWithQuery() {
|
|
||||||
return true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
case *pipeReplace:
|
|
||||||
if t.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
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return false
|
return false
|
||||||
|
@ -514,64 +489,11 @@ func initFilterInValuesForFilter(cache map[string][]string, f filter, getFieldVa
|
||||||
func initFilterInValuesForPipes(cache map[string][]string, pipes []pipe, getFieldValuesFunc getFieldValuesFunc) ([]pipe, error) {
|
func initFilterInValuesForPipes(cache map[string][]string, pipes []pipe, getFieldValuesFunc getFieldValuesFunc) ([]pipe, error) {
|
||||||
pipesNew := make([]pipe, len(pipes))
|
pipesNew := make([]pipe, len(pipes))
|
||||||
for i, p := range pipes {
|
for i, p := range pipes {
|
||||||
switch t := p.(type) {
|
pNew, err := p.initFilterInValues(cache, getFieldValuesFunc)
|
||||||
case *pipeStats:
|
if err != nil {
|
||||||
funcsNew := make([]pipeStatsFunc, len(t.funcs))
|
return nil, err
|
||||||
for j, f := range t.funcs {
|
|
||||||
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 *pipeReplace:
|
|
||||||
iffNew, err := t.iff.initFilterInValues(cache, getFieldValuesFunc)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
pr := *t
|
|
||||||
pr.iff = iffNew
|
|
||||||
pipesNew[i] = &pr
|
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
pipesNew[i] = pNew
|
||||||
}
|
}
|
||||||
return pipesNew, nil
|
return pipesNew, nil
|
||||||
}
|
}
|
||||||
|
@ -1099,22 +1021,22 @@ func getFilterTimeRange(f filter) (int64, int64) {
|
||||||
return math.MinInt64, math.MaxInt64
|
return math.MinInt64, math.MaxInt64
|
||||||
}
|
}
|
||||||
|
|
||||||
func forEachStreamLabel(streams []ValueWithHits, f func(label Field, hits uint64)) {
|
func forEachStreamField(streams []ValueWithHits, f func(f Field, hits uint64)) {
|
||||||
var labels []Field
|
var fields []Field
|
||||||
for i := range streams {
|
for i := range streams {
|
||||||
var err error
|
var err error
|
||||||
labels, err = parseStreamLabels(labels[:0], streams[i].Value)
|
fields, err = parseStreamFields(fields[:0], streams[i].Value)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
hits := streams[i].Hits
|
hits := streams[i].Hits
|
||||||
for j := range labels {
|
for j := range fields {
|
||||||
f(labels[j], hits)
|
f(fields[j], hits)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func parseStreamLabels(dst []Field, s string) ([]Field, error) {
|
func parseStreamFields(dst []Field, s string) ([]Field, error) {
|
||||||
if len(s) == 0 || s[0] != '{' {
|
if len(s) == 0 || s[0] != '{' {
|
||||||
return dst, fmt.Errorf("missing '{' at the beginning of stream name")
|
return dst, fmt.Errorf("missing '{' at the beginning of stream name")
|
||||||
}
|
}
|
||||||
|
@ -1130,14 +1052,14 @@ func parseStreamLabels(dst []Field, s string) ([]Field, error) {
|
||||||
for {
|
for {
|
||||||
n := strings.Index(s, `="`)
|
n := strings.Index(s, `="`)
|
||||||
if n < 0 {
|
if n < 0 {
|
||||||
return dst, fmt.Errorf("cannot find label value in double quotes at [%s]", s)
|
return dst, fmt.Errorf("cannot find field value in double quotes at [%s]", s)
|
||||||
}
|
}
|
||||||
name := s[:n]
|
name := s[:n]
|
||||||
s = s[n+1:]
|
s = s[n+1:]
|
||||||
|
|
||||||
value, nOffset := tryUnquoteString(s, "")
|
value, nOffset := tryUnquoteString(s, "")
|
||||||
if nOffset < 0 {
|
if nOffset < 0 {
|
||||||
return dst, fmt.Errorf("cannot find parse label value in double quotes at [%s]", s)
|
return dst, fmt.Errorf("cannot find parse field value in double quotes at [%s]", s)
|
||||||
}
|
}
|
||||||
s = s[nOffset:]
|
s = s[nOffset:]
|
||||||
|
|
||||||
|
|
|
@ -650,11 +650,11 @@ func TestStorageSearch(t *testing.T) {
|
||||||
fs.MustRemoveAll(path)
|
fs.MustRemoveAll(path)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestParseStreamLabelsSuccess(t *testing.T) {
|
func TestParseStreamFieldsSuccess(t *testing.T) {
|
||||||
f := func(s, resultExpected string) {
|
f := func(s, resultExpected string) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
|
|
||||||
labels, err := parseStreamLabels(nil, s)
|
labels, err := parseStreamFields(nil, s)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("unexpected error: %s", err)
|
t.Fatalf("unexpected error: %s", err)
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue