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
c96a98731a
commit
4b458370c1
45 changed files with 1972 additions and 871 deletions
|
@ -1,17 +0,0 @@
|
||||||
{% stripspace %}
|
|
||||||
|
|
||||||
// FieldNamesResponse formats /select/logsql/field_names response
|
|
||||||
{% func FieldNamesResponse(names []string) %}
|
|
||||||
{
|
|
||||||
"names":[
|
|
||||||
{% if len(names) > 0 %}
|
|
||||||
{%q= names[0] %}
|
|
||||||
{% for _, v := range names[1:] %}
|
|
||||||
,{%q= v %}
|
|
||||||
{% endfor %}
|
|
||||||
{% endif %}
|
|
||||||
]
|
|
||||||
}
|
|
||||||
{% endfunc %}
|
|
||||||
|
|
||||||
{% endstripspace %}
|
|
|
@ -1,69 +0,0 @@
|
||||||
// Code generated by qtc from "field_names_response.qtpl". DO NOT EDIT.
|
|
||||||
// See https://github.com/valyala/quicktemplate for details.
|
|
||||||
|
|
||||||
// FieldNamesResponse formats /select/logsql/field_names response
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:4
|
|
||||||
package logsql
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:4
|
|
||||||
import (
|
|
||||||
qtio422016 "io"
|
|
||||||
|
|
||||||
qt422016 "github.com/valyala/quicktemplate"
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:4
|
|
||||||
var (
|
|
||||||
_ = qtio422016.Copy
|
|
||||||
_ = qt422016.AcquireByteBuffer
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:4
|
|
||||||
func StreamFieldNamesResponse(qw422016 *qt422016.Writer, names []string) {
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:4
|
|
||||||
qw422016.N().S(`{"names":[`)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:7
|
|
||||||
if len(names) > 0 {
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:8
|
|
||||||
qw422016.N().Q(names[0])
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:9
|
|
||||||
for _, v := range names[1:] {
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:9
|
|
||||||
qw422016.N().S(`,`)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:10
|
|
||||||
qw422016.N().Q(v)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:11
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:12
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:12
|
|
||||||
qw422016.N().S(`]}`)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
func WriteFieldNamesResponse(qq422016 qtio422016.Writer, names []string) {
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
StreamFieldNamesResponse(qw422016, names)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
qt422016.ReleaseWriter(qw422016)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
func FieldNamesResponse(names []string) string {
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
WriteFieldNamesResponse(qb422016, names)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
qs422016 := string(qb422016.B)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
qt422016.ReleaseByteBuffer(qb422016)
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
return qs422016
|
|
||||||
//line app/vlselect/logsql/field_names_response.qtpl:15
|
|
||||||
}
|
|
|
@ -1,17 +0,0 @@
|
||||||
{% stripspace %}
|
|
||||||
|
|
||||||
// FieldValuesResponse formats /select/logsql/field_values response
|
|
||||||
{% func FieldValuesResponse(values []string) %}
|
|
||||||
{
|
|
||||||
"values":[
|
|
||||||
{% if len(values) > 0 %}
|
|
||||||
{%q= values[0] %}
|
|
||||||
{% for _, v := range values[1:] %}
|
|
||||||
,{%q= v %}
|
|
||||||
{% endfor %}
|
|
||||||
{% endif %}
|
|
||||||
]
|
|
||||||
}
|
|
||||||
{% endfunc %}
|
|
||||||
|
|
||||||
{% endstripspace %}
|
|
|
@ -1,69 +0,0 @@
|
||||||
// Code generated by qtc from "field_values_response.qtpl". DO NOT EDIT.
|
|
||||||
// See https://github.com/valyala/quicktemplate for details.
|
|
||||||
|
|
||||||
// FieldValuesResponse formats /select/logsql/field_values response
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:4
|
|
||||||
package logsql
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:4
|
|
||||||
import (
|
|
||||||
qtio422016 "io"
|
|
||||||
|
|
||||||
qt422016 "github.com/valyala/quicktemplate"
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:4
|
|
||||||
var (
|
|
||||||
_ = qtio422016.Copy
|
|
||||||
_ = qt422016.AcquireByteBuffer
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:4
|
|
||||||
func StreamFieldValuesResponse(qw422016 *qt422016.Writer, values []string) {
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:4
|
|
||||||
qw422016.N().S(`{"values":[`)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:7
|
|
||||||
if len(values) > 0 {
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:8
|
|
||||||
qw422016.N().Q(values[0])
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:9
|
|
||||||
for _, v := range values[1:] {
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:9
|
|
||||||
qw422016.N().S(`,`)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:10
|
|
||||||
qw422016.N().Q(v)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:11
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:12
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:12
|
|
||||||
qw422016.N().S(`]}`)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
func WriteFieldValuesResponse(qq422016 qtio422016.Writer, values []string) {
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
StreamFieldValuesResponse(qw422016, values)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
qt422016.ReleaseWriter(qw422016)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
func FieldValuesResponse(values []string) string {
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
WriteFieldValuesResponse(qb422016, values)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
qs422016 := string(qb422016.B)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
qt422016.ReleaseByteBuffer(qb422016)
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
return qs422016
|
|
||||||
//line app/vlselect/logsql/field_values_response.qtpl:15
|
|
||||||
}
|
|
|
@ -146,7 +146,7 @@ func ProcessFieldNamesRequest(ctx context.Context, w http.ResponseWriter, r *htt
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
w.Header().Set("Content-Type", "application/json")
|
w.Header().Set("Content-Type", "application/json")
|
||||||
WriteFieldNamesResponse(w, fieldNames)
|
WriteValuesWithHitsJSON(w, fieldNames)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessFieldValuesRequest handles /select/logsql/field_values request.
|
// ProcessFieldValuesRequest handles /select/logsql/field_values request.
|
||||||
|
@ -186,7 +186,7 @@ func ProcessFieldValuesRequest(ctx context.Context, w http.ResponseWriter, r *ht
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
w.Header().Set("Content-Type", "application/json")
|
w.Header().Set("Content-Type", "application/json")
|
||||||
WriteFieldValuesResponse(w, values)
|
WriteValuesWithHitsJSON(w, values)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessStreamLabelNamesRequest processes /select/logsql/stream_label_names request.
|
// ProcessStreamLabelNamesRequest processes /select/logsql/stream_label_names request.
|
||||||
|
@ -208,7 +208,7 @@ func ProcessStreamLabelNamesRequest(ctx context.Context, w http.ResponseWriter,
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
w.Header().Set("Content-Type", "application/json")
|
w.Header().Set("Content-Type", "application/json")
|
||||||
WriteStreamLabelNamesResponse(w, names)
|
WriteValuesWithHitsJSON(w, names)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessStreamLabelValuesRequest processes /select/logsql/stream_label_values request.
|
// ProcessStreamLabelValuesRequest processes /select/logsql/stream_label_values request.
|
||||||
|
@ -247,7 +247,7 @@ func ProcessStreamLabelValuesRequest(ctx context.Context, w http.ResponseWriter,
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
w.Header().Set("Content-Type", "application/json")
|
w.Header().Set("Content-Type", "application/json")
|
||||||
WriteStreamLabelValuesResponse(w, values)
|
WriteValuesWithHitsJSON(w, values)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessStreamsRequest processes /select/logsql/streams request.
|
// ProcessStreamsRequest processes /select/logsql/streams request.
|
||||||
|
@ -279,7 +279,7 @@ func ProcessStreamsRequest(ctx context.Context, w http.ResponseWriter, r *http.R
|
||||||
|
|
||||||
// Write results
|
// Write results
|
||||||
w.Header().Set("Content-Type", "application/json")
|
w.Header().Set("Content-Type", "application/json")
|
||||||
WriteStreamsResponse(w, streams)
|
WriteValuesWithHitsJSON(w, streams)
|
||||||
}
|
}
|
||||||
|
|
||||||
// ProcessQueryRequest handles /select/logsql/query request.
|
// ProcessQueryRequest handles /select/logsql/query request.
|
||||||
|
|
32
app/vlselect/logsql/logsql.qtpl
Normal file
32
app/vlselect/logsql/logsql.qtpl
Normal file
|
@ -0,0 +1,32 @@
|
||||||
|
{% import (
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logstorage"
|
||||||
|
) %}
|
||||||
|
|
||||||
|
{% stripspace %}
|
||||||
|
|
||||||
|
// ValuesWithHitsJSON generates JSON from the given values.
|
||||||
|
{% func ValuesWithHitsJSON(values []logstorage.ValueWithHits) %}
|
||||||
|
{
|
||||||
|
"values":{%= valuesWithHitsJSONArray(values) %}
|
||||||
|
}
|
||||||
|
{% endfunc %}
|
||||||
|
|
||||||
|
{% func valuesWithHitsJSONArray(values []logstorage.ValueWithHits) %}
|
||||||
|
[
|
||||||
|
{% if len(values) > 0 %}
|
||||||
|
{%= valueWithHitsJSON(values[0]) %}
|
||||||
|
{% for _, v := range values[1:] %}
|
||||||
|
,{%= valueWithHitsJSON(v) %}
|
||||||
|
{% endfor %}
|
||||||
|
{% endif %}
|
||||||
|
]
|
||||||
|
{% endfunc %}
|
||||||
|
|
||||||
|
{% func valueWithHitsJSON(v logstorage.ValueWithHits) %}
|
||||||
|
{
|
||||||
|
"value":{%q= v.Value %},
|
||||||
|
"hits":{%dul= v.Hits %}
|
||||||
|
}
|
||||||
|
{% endfunc %}
|
||||||
|
|
||||||
|
{% endstripspace %}
|
152
app/vlselect/logsql/logsql.qtpl.go
Normal file
152
app/vlselect/logsql/logsql.qtpl.go
Normal file
|
@ -0,0 +1,152 @@
|
||||||
|
// Code generated by qtc from "logsql.qtpl". DO NOT EDIT.
|
||||||
|
// See https://github.com/valyala/quicktemplate for details.
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:1
|
||||||
|
package logsql
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:1
|
||||||
|
import (
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logstorage"
|
||||||
|
)
|
||||||
|
|
||||||
|
// ValuesWithHitsJSON generates JSON from the given values.
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:8
|
||||||
|
import (
|
||||||
|
qtio422016 "io"
|
||||||
|
|
||||||
|
qt422016 "github.com/valyala/quicktemplate"
|
||||||
|
)
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:8
|
||||||
|
var (
|
||||||
|
_ = qtio422016.Copy
|
||||||
|
_ = qt422016.AcquireByteBuffer
|
||||||
|
)
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:8
|
||||||
|
func StreamValuesWithHitsJSON(qw422016 *qt422016.Writer, values []logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:8
|
||||||
|
qw422016.N().S(`{"values":`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:10
|
||||||
|
streamvaluesWithHitsJSONArray(qw422016, values)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:10
|
||||||
|
qw422016.N().S(`}`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
func WriteValuesWithHitsJSON(qq422016 qtio422016.Writer, values []logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
qw422016 := qt422016.AcquireWriter(qq422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
StreamValuesWithHitsJSON(qw422016, values)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
qt422016.ReleaseWriter(qw422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
func ValuesWithHitsJSON(values []logstorage.ValueWithHits) string {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
qb422016 := qt422016.AcquireByteBuffer()
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
WriteValuesWithHitsJSON(qb422016, values)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
qs422016 := string(qb422016.B)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
qt422016.ReleaseByteBuffer(qb422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
return qs422016
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:12
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:14
|
||||||
|
func streamvaluesWithHitsJSONArray(qw422016 *qt422016.Writer, values []logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:14
|
||||||
|
qw422016.N().S(`[`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:16
|
||||||
|
if len(values) > 0 {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:17
|
||||||
|
streamvalueWithHitsJSON(qw422016, values[0])
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:18
|
||||||
|
for _, v := range values[1:] {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:18
|
||||||
|
qw422016.N().S(`,`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:19
|
||||||
|
streamvalueWithHitsJSON(qw422016, v)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:20
|
||||||
|
}
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:21
|
||||||
|
}
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:21
|
||||||
|
qw422016.N().S(`]`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
func writevaluesWithHitsJSONArray(qq422016 qtio422016.Writer, values []logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
qw422016 := qt422016.AcquireWriter(qq422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
streamvaluesWithHitsJSONArray(qw422016, values)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
qt422016.ReleaseWriter(qw422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
func valuesWithHitsJSONArray(values []logstorage.ValueWithHits) string {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
qb422016 := qt422016.AcquireByteBuffer()
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
writevaluesWithHitsJSONArray(qb422016, values)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
qs422016 := string(qb422016.B)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
qt422016.ReleaseByteBuffer(qb422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
return qs422016
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:23
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:25
|
||||||
|
func streamvalueWithHitsJSON(qw422016 *qt422016.Writer, v logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:25
|
||||||
|
qw422016.N().S(`{"value":`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:27
|
||||||
|
qw422016.N().Q(v.Value)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:27
|
||||||
|
qw422016.N().S(`,"hits":`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:28
|
||||||
|
qw422016.N().DUL(v.Hits)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:28
|
||||||
|
qw422016.N().S(`}`)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
func writevalueWithHitsJSON(qq422016 qtio422016.Writer, v logstorage.ValueWithHits) {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
qw422016 := qt422016.AcquireWriter(qq422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
streamvalueWithHitsJSON(qw422016, v)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
qt422016.ReleaseWriter(qw422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
}
|
||||||
|
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
func valueWithHitsJSON(v logstorage.ValueWithHits) string {
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
qb422016 := qt422016.AcquireByteBuffer()
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
writevalueWithHitsJSON(qb422016, v)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
qs422016 := string(qb422016.B)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
qt422016.ReleaseByteBuffer(qb422016)
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
return qs422016
|
||||||
|
//line app/vlselect/logsql/logsql.qtpl:30
|
||||||
|
}
|
|
@ -1,17 +0,0 @@
|
||||||
{% stripspace %}
|
|
||||||
|
|
||||||
// StreamLabelNamesResponse formats /select/logsql/stream_label_names response
|
|
||||||
{% func StreamLabelNamesResponse(names []string) %}
|
|
||||||
{
|
|
||||||
"names":[
|
|
||||||
{% if len(names) > 0 %}
|
|
||||||
{%q= names[0] %}
|
|
||||||
{% for _, v := range names[1:] %}
|
|
||||||
,{%q= v %}
|
|
||||||
{% endfor %}
|
|
||||||
{% endif %}
|
|
||||||
]
|
|
||||||
}
|
|
||||||
{% endfunc %}
|
|
||||||
|
|
||||||
{% endstripspace %}
|
|
|
@ -1,69 +0,0 @@
|
||||||
// Code generated by qtc from "stream_label_names_response.qtpl". DO NOT EDIT.
|
|
||||||
// See https://github.com/valyala/quicktemplate for details.
|
|
||||||
|
|
||||||
// StreamLabelNamesResponse formats /select/logsql/stream_label_names response
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:4
|
|
||||||
package logsql
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:4
|
|
||||||
import (
|
|
||||||
qtio422016 "io"
|
|
||||||
|
|
||||||
qt422016 "github.com/valyala/quicktemplate"
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:4
|
|
||||||
var (
|
|
||||||
_ = qtio422016.Copy
|
|
||||||
_ = qt422016.AcquireByteBuffer
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:4
|
|
||||||
func StreamStreamLabelNamesResponse(qw422016 *qt422016.Writer, names []string) {
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:4
|
|
||||||
qw422016.N().S(`{"names":[`)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:7
|
|
||||||
if len(names) > 0 {
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:8
|
|
||||||
qw422016.N().Q(names[0])
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:9
|
|
||||||
for _, v := range names[1:] {
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:9
|
|
||||||
qw422016.N().S(`,`)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:10
|
|
||||||
qw422016.N().Q(v)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:11
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:12
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:12
|
|
||||||
qw422016.N().S(`]}`)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
func WriteStreamLabelNamesResponse(qq422016 qtio422016.Writer, names []string) {
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
StreamStreamLabelNamesResponse(qw422016, names)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
qt422016.ReleaseWriter(qw422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
func StreamLabelNamesResponse(names []string) string {
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
WriteStreamLabelNamesResponse(qb422016, names)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
qs422016 := string(qb422016.B)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
qt422016.ReleaseByteBuffer(qb422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
return qs422016
|
|
||||||
//line app/vlselect/logsql/stream_label_names_response.qtpl:15
|
|
||||||
}
|
|
|
@ -1,17 +0,0 @@
|
||||||
{% stripspace %}
|
|
||||||
|
|
||||||
// StreamLabelValuesResponse formats /select/logsql/stream_label_values response
|
|
||||||
{% func StreamLabelValuesResponse(values []string) %}
|
|
||||||
{
|
|
||||||
"values":[
|
|
||||||
{% if len(values) > 0 %}
|
|
||||||
{%q= values[0] %}
|
|
||||||
{% for _, v := range values[1:] %}
|
|
||||||
,{%q= v %}
|
|
||||||
{% endfor %}
|
|
||||||
{% endif %}
|
|
||||||
]
|
|
||||||
}
|
|
||||||
{% endfunc %}
|
|
||||||
|
|
||||||
{% endstripspace %}
|
|
|
@ -1,69 +0,0 @@
|
||||||
// Code generated by qtc from "stream_label_values_response.qtpl". DO NOT EDIT.
|
|
||||||
// See https://github.com/valyala/quicktemplate for details.
|
|
||||||
|
|
||||||
// StreamLabelValuesResponse formats /select/logsql/stream_label_values response
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:4
|
|
||||||
package logsql
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:4
|
|
||||||
import (
|
|
||||||
qtio422016 "io"
|
|
||||||
|
|
||||||
qt422016 "github.com/valyala/quicktemplate"
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:4
|
|
||||||
var (
|
|
||||||
_ = qtio422016.Copy
|
|
||||||
_ = qt422016.AcquireByteBuffer
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:4
|
|
||||||
func StreamStreamLabelValuesResponse(qw422016 *qt422016.Writer, values []string) {
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:4
|
|
||||||
qw422016.N().S(`{"values":[`)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:7
|
|
||||||
if len(values) > 0 {
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:8
|
|
||||||
qw422016.N().Q(values[0])
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:9
|
|
||||||
for _, v := range values[1:] {
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:9
|
|
||||||
qw422016.N().S(`,`)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:10
|
|
||||||
qw422016.N().Q(v)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:11
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:12
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:12
|
|
||||||
qw422016.N().S(`]}`)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
func WriteStreamLabelValuesResponse(qq422016 qtio422016.Writer, values []string) {
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
StreamStreamLabelValuesResponse(qw422016, values)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
qt422016.ReleaseWriter(qw422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
func StreamLabelValuesResponse(values []string) string {
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
WriteStreamLabelValuesResponse(qb422016, values)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
qs422016 := string(qb422016.B)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
qt422016.ReleaseByteBuffer(qb422016)
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
return qs422016
|
|
||||||
//line app/vlselect/logsql/stream_label_values_response.qtpl:15
|
|
||||||
}
|
|
|
@ -1,17 +0,0 @@
|
||||||
{% stripspace %}
|
|
||||||
|
|
||||||
// StreamsResponse formats /select/logsql/streams response
|
|
||||||
{% func StreamsResponse(streams []string) %}
|
|
||||||
{
|
|
||||||
"streams":[
|
|
||||||
{% if len(streams) > 0 %}
|
|
||||||
{%q= streams[0] %}
|
|
||||||
{% for _, v := range streams[1:] %}
|
|
||||||
,{%q= v %}
|
|
||||||
{% endfor %}
|
|
||||||
{% endif %}
|
|
||||||
]
|
|
||||||
}
|
|
||||||
{% endfunc %}
|
|
||||||
|
|
||||||
{% endstripspace %}
|
|
|
@ -1,69 +0,0 @@
|
||||||
// Code generated by qtc from "streams_response.qtpl". DO NOT EDIT.
|
|
||||||
// See https://github.com/valyala/quicktemplate for details.
|
|
||||||
|
|
||||||
// StreamsResponse formats /select/logsql/streams response
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:4
|
|
||||||
package logsql
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:4
|
|
||||||
import (
|
|
||||||
qtio422016 "io"
|
|
||||||
|
|
||||||
qt422016 "github.com/valyala/quicktemplate"
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:4
|
|
||||||
var (
|
|
||||||
_ = qtio422016.Copy
|
|
||||||
_ = qt422016.AcquireByteBuffer
|
|
||||||
)
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:4
|
|
||||||
func StreamStreamsResponse(qw422016 *qt422016.Writer, streams []string) {
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:4
|
|
||||||
qw422016.N().S(`{"streams":[`)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:7
|
|
||||||
if len(streams) > 0 {
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:8
|
|
||||||
qw422016.N().Q(streams[0])
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:9
|
|
||||||
for _, v := range streams[1:] {
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:9
|
|
||||||
qw422016.N().S(`,`)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:10
|
|
||||||
qw422016.N().Q(v)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:11
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:12
|
|
||||||
}
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:12
|
|
||||||
qw422016.N().S(`]}`)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
func WriteStreamsResponse(qq422016 qtio422016.Writer, streams []string) {
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
StreamStreamsResponse(qw422016, streams)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
qt422016.ReleaseWriter(qw422016)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
}
|
|
||||||
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
func StreamsResponse(streams []string) string {
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
qb422016 := qt422016.AcquireByteBuffer()
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
WriteStreamsResponse(qb422016, streams)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
qs422016 := string(qb422016.B)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
qt422016.ReleaseByteBuffer(qb422016)
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
return qs422016
|
|
||||||
//line app/vlselect/logsql/streams_response.qtpl:15
|
|
||||||
}
|
|
|
@ -112,33 +112,33 @@ func RunQuery(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorag
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetFieldNames executes q and returns field names seen in results.
|
// GetFieldNames executes q and returns field names seen in results.
|
||||||
func GetFieldNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]string, error) {
|
func GetFieldNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetFieldNames(ctx, tenantIDs, q)
|
return strg.GetFieldNames(ctx, tenantIDs, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetFieldValues executes q and returns unique values for the fieldName seen in results.
|
// GetFieldValues executes q and returns unique values for the fieldName seen in results.
|
||||||
//
|
//
|
||||||
// If limit > 0, then up to limit unique values are returned.
|
// If limit > 0, then up to limit unique values are returned.
|
||||||
func GetFieldValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, fieldName string, limit uint64) ([]string, error) {
|
func GetFieldValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, fieldName string, limit uint64) ([]logstorage.ValueWithHits, error) {
|
||||||
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.
|
// GetStreamLabelNames executes q and returns stream labels names seen in results.
|
||||||
func GetStreamLabelNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]string, error) {
|
func GetStreamLabelNames(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetStreamLabelNames(ctx, tenantIDs, q)
|
return strg.GetStreamLabelNames(ctx, tenantIDs, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreamLabelValues executes q and returns stream label values for the given labelName seen in results.
|
// GetStreamLabelValues executes q and returns stream label values for the given labelName seen in results.
|
||||||
//
|
//
|
||||||
// If limit > 0, then up to limit unique stream label values are returned.
|
// If limit > 0, then up to limit unique stream label values are returned.
|
||||||
func GetStreamLabelValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, labelName string, limit uint64) ([]string, error) {
|
func GetStreamLabelValues(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, labelName string, limit uint64) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetStreamLabelValues(ctx, tenantIDs, q, labelName, limit)
|
return strg.GetStreamLabelValues(ctx, tenantIDs, q, labelName, limit)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetStreams executes q and returns streams seen in query results.
|
// GetStreams executes q and returns streams seen in query results.
|
||||||
//
|
//
|
||||||
// If limit > 0, then up to limit unique streams are returned.
|
// If limit > 0, then up to limit unique streams are returned.
|
||||||
func GetStreams(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, limit uint64) ([]string, error) {
|
func GetStreams(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, limit uint64) ([]logstorage.ValueWithHits, error) {
|
||||||
return strg.GetStreams(ctx, tenantIDs, q, limit)
|
return strg.GetStreams(ctx, tenantIDs, q, limit)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,17 @@ according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/QuickSta
|
||||||
|
|
||||||
## tip
|
## tip
|
||||||
|
|
||||||
|
* FAETURE: return the number of matching log entries per returned value in [HTTP API](https://docs.victoriametrics.com/victorialogs/querying/#http-api) results. This simplifies detecting [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) / [stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) values with the biggest number of logs for the given [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/).
|
||||||
|
* FEATURE: improve performance for [regexp filter](https://docs.victoriametrics.com/victorialogs/logsql/#regexp-filter) in the following cases:
|
||||||
|
- If the regexp contains just a phrase without special regular expression chars. For example, `~"foo"`.
|
||||||
|
- If the regexp starts with `.*` or ends with `.*`. For example, `~".*foo.*"`.
|
||||||
|
- If the regexp contains multiple strings delimited by `|`. For example, `~"foo|bar|baz"`.
|
||||||
|
- If the regexp contains multiple [words](https://docs.victoriametrics.com/victorialogs/logsql/#word). For example, `~"foo bar baz"`.
|
||||||
|
* FEATURE: allow disabling automatic unquoting of the matched placeholders in [`extract` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#extract-pipe). See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#format-for-extract-pipe-pattern).
|
||||||
|
|
||||||
|
* BUGFIX: properly parse `!` in front of [exact filter](https://docs.victoriametrics.com/victorialogs/logsql/#exact-filter), [exact-prefix filter](https://docs.victoriametrics.com/victorialogs/logsql/#exact-prefix-filter) and [regexp filter](https://docs.victoriametrics.com/victorialogs/logsql/#regexp-filter). For example, `!~"some regexp"` is properly parsed as `not ="some regexp"`. Previously it was incorrectly parsed as `'~="some regexp"'` [phrase filter](https://docs.victoriametrics.com/victorialogs/logsql/#phrase-filter).
|
||||||
|
* BUGFIX: properly sort results by [`_time` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) when [`limit` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#limit-pipe) is applied. For example, `_time:5m | sort by (_time) desc | limit 10` properly works now.
|
||||||
|
|
||||||
## [v0.9.1](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.9.1-victorialogs)
|
## [v0.9.1](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.9.1-victorialogs)
|
||||||
|
|
||||||
Released at 2024-05-22
|
Released at 2024-05-22
|
||||||
|
|
|
@ -403,6 +403,13 @@ This query doesn't match the following log messages:
|
||||||
- `SSH: login fail`, since the `SSH` word is in capital letters. Use `i("ssh: login fail")` for case-insensitive search.
|
- `SSH: login fail`, since the `SSH` word is in capital letters. Use `i("ssh: login fail")` for case-insensitive search.
|
||||||
See [these docs](#case-insensitive-filter) for details.
|
See [these docs](#case-insensitive-filter) for details.
|
||||||
|
|
||||||
|
If the phrase contains double quotes, then either put `\` in front of double quotes or put the phrase inside single quotes. For example, the following filter searches
|
||||||
|
logs with `"foo":"bar"` phrase:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
'"foo":"bar"'
|
||||||
|
```
|
||||||
|
|
||||||
By default the given phrase is searched in the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
By default the given phrase is searched in the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
||||||
Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the phrase and put a colon after it
|
Specify the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the phrase and put a colon after it
|
||||||
if it must be searched in the given field. For example, the following query returns log entries containing the `cannot open file` phrase in the `event.original` field:
|
if it must be searched in the given field. For example, the following query returns log entries containing the `cannot open file` phrase in the `event.original` field:
|
||||||
|
@ -470,6 +477,13 @@ This query doesn't match the following log messages:
|
||||||
- `failed to open file: unexpected EOF`, since `failed` [word](#word) occurs before the `unexpected` word. Use `unexpected AND fail*` for this case.
|
- `failed to open file: unexpected EOF`, since `failed` [word](#word) occurs before the `unexpected` word. Use `unexpected AND fail*` for this case.
|
||||||
See [these docs](#logical-filter) for details.
|
See [these docs](#logical-filter) for details.
|
||||||
|
|
||||||
|
If the prefix contains double quotes, then either put `\` in front of double quotes or put the prefix inside single quotes. For example, the following filter searches
|
||||||
|
logs with `"foo":"bar` prefix:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
'"foo":"bar'*
|
||||||
|
```
|
||||||
|
|
||||||
By default the prefix filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
By default the prefix filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
||||||
Specify the needed [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the prefix filter
|
Specify the needed [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the prefix filter
|
||||||
in order to apply it to the given field. For example, the following query matches `log.level` field containing any word with the `err` prefix:
|
in order to apply it to the given field. For example, the following query matches `log.level` field containing any word with the `err` prefix:
|
||||||
|
@ -783,6 +797,13 @@ The query doesn't match the following log messages:
|
||||||
See [these docs](https://github.com/google/re2/wiki/Syntax) for details. See also [case-insenstive filter docs](#case-insensitive-filter).
|
See [these docs](https://github.com/google/re2/wiki/Syntax) for details. See also [case-insenstive filter docs](#case-insensitive-filter).
|
||||||
- `it is warmer than usual`, since it doesn't contain neither `err` nor `warn` substrings.
|
- `it is warmer than usual`, since it doesn't contain neither `err` nor `warn` substrings.
|
||||||
|
|
||||||
|
If the regexp contains double quotes, then either put `\` in front of double quotes or put the regexp inside single quotes. For example, the following regexp searches
|
||||||
|
logs matching `"foo":"(bar|baz)"` regexp:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
'"foo":"(bar|baz)"'
|
||||||
|
```
|
||||||
|
|
||||||
By default the regexp filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
By default the regexp filter is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
||||||
Specify the needed [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the filter
|
Specify the needed [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the filter
|
||||||
in order to apply it to the given field. For example, the following query matches `event.original` field containing either `err` or `warn` substrings:
|
in order to apply it to the given field. For example, the following query matches `event.original` field containing either `err` or `warn` substrings:
|
||||||
|
@ -1134,7 +1155,8 @@ For example, the following query is equivalent to the previous one:
|
||||||
_time:1d error | extract "ip=<ip> " | stats by (ip) count() logs | sort by (logs) desc limit 10
|
_time:1d error | extract "ip=<ip> " | stats by (ip) count() logs | sort by (logs) desc limit 10
|
||||||
```
|
```
|
||||||
|
|
||||||
If the `pattern` contains double quotes, then it can be quoted into single quotes. For example, the following query extracts `ip` from the corresponding JSON field:
|
If the `pattern` contains double quotes, then either put `\` in front of double quotes or put the `pattern` inside single quotes.
|
||||||
|
For example, the following query extracts `ip` from the corresponding JSON field:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
_time:5m | extract '"ip":"<ip>"'
|
_time:5m | extract '"ip":"<ip>"'
|
||||||
|
@ -1162,7 +1184,7 @@ Placeholders can be anonymous and named. Anonymous placeholders are written as `
|
||||||
must be skipped until the next `textX`. Named palceholders are written as `<some_name>`, where `some_name` is the name of the log field to store
|
must be skipped until the next `textX`. Named palceholders are written as `<some_name>`, where `some_name` is the name of the log field to store
|
||||||
the corresponding matching substring to.
|
the corresponding matching substring to.
|
||||||
|
|
||||||
The matching starts from the first occurence of the `text1` in the input text. If the `pattern` starts with `<field1>` and doesn't contain `text1`,
|
Matching starts from the first occurence of the `text1` in the input text. If the `pattern` starts with `<field1>` and doesn't contain `text1`,
|
||||||
then the matching starts from the beginning of the input text. Matching is performed sequentially according to the `pattern`. If some `textX` isn't found
|
then the matching starts from the beginning of the input text. Matching is performed sequentially according to the `pattern`. If some `textX` isn't found
|
||||||
in the remaining input text, then the remaining named placeholders receive empty string values and the matching finishes prematurely.
|
in the remaining input text, then the remaining named placeholders receive empty string values and the matching finishes prematurely.
|
||||||
|
|
||||||
|
@ -1197,6 +1219,13 @@ This is useful for extracting JSON strings. For example, the following `pattern`
|
||||||
"message":<msg>
|
"message":<msg>
|
||||||
```
|
```
|
||||||
|
|
||||||
|
The automatic string unquoting can be disabled if needed by adding `plain:` prefix in front of the field name. For example, if some JSON array of string values must be captured
|
||||||
|
into `json_array` field, then the following `pattern` can be used:
|
||||||
|
|
||||||
|
```
|
||||||
|
some json string array: [<plain:json_array>]
|
||||||
|
```
|
||||||
|
|
||||||
If some special chars such as `<` must be matched by the `pattern`, then they can be [html-escaped](https://en.wikipedia.org/wiki/List_of_XML_and_HTML_character_entity_references).
|
If some special chars such as `<` must be matched by the `pattern`, then they can be [html-escaped](https://en.wikipedia.org/wiki/List_of_XML_and_HTML_character_entity_references).
|
||||||
For example, the following `pattern` properly matches `a < b` text by extracting `a` into `left` field and `b` into `right` field:
|
For example, the following `pattern` properly matches `a < b` text by extracting `a` into `left` field and `b` into `right` field:
|
||||||
|
|
||||||
|
@ -1217,12 +1246,12 @@ _time:5m | extract if (ip:"") "ip=<ip> "
|
||||||
|
|
||||||
### field_names pipe
|
### field_names pipe
|
||||||
|
|
||||||
Sometimes it may be needed to get all the field names for the selected results. This may be done with `| field_names ...` [pipe](#pipes).
|
`| field_names` [pipe](#pipes) returns all the names of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||||
For example, the following query returns all the names of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
with an estimated number of logs per each field name.
|
||||||
from the logs over the last 5 minutes:
|
For example, the following query returns all the field names with the number of matching logs over the last 5 minutes:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
_time:5m | field_names as names
|
_time:5m | field_names
|
||||||
```
|
```
|
||||||
|
|
||||||
Field names are returned in arbitrary order. Use [`sort` pipe](#sort-pipe) in order to sort them if needed.
|
Field names are returned in arbitrary order. Use [`sort` pipe](#sort-pipe) in order to sort them if needed.
|
||||||
|
@ -1593,7 +1622,7 @@ _time:5m | stats
|
||||||
|
|
||||||
### uniq pipe
|
### uniq pipe
|
||||||
|
|
||||||
`| uniq ...` pipe allows returning only unique results over the selected logs. For example, the following LogsQL query
|
`| uniq ...` pipe returns unique results over the selected logs. For example, the following LogsQL query
|
||||||
returns unique values for `ip` [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
returns unique values for `ip` [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||||
over logs for the last 5 minutes:
|
over logs for the last 5 minutes:
|
||||||
|
|
||||||
|
@ -1610,6 +1639,12 @@ _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:
|
||||||
|
|
||||||
|
```logsql
|
||||||
|
_time:5m | uniq by (host) 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.
|
||||||
Sometimes it is enough to return up to `N` unique entries. This can be done by adding `limit N` after `by (...)` clause.
|
Sometimes it is enough to return up to `N` unique entries. This can be done by adding `limit N` after `by (...)` clause.
|
||||||
This allows limiting memory usage. For example, the following query returns up to 100 unique `(host, path)` pairs for the logs over the last 5 minutes:
|
This allows limiting memory usage. For example, the following query returns up to 100 unique `(host, path)` pairs for the logs over the last 5 minutes:
|
||||||
|
@ -1618,6 +1653,8 @@ This allows limiting memory usage. For example, the following query returns up t
|
||||||
_time:5m | uniq by (host, path) limit 100
|
_time:5m | uniq by (host, path) limit 100
|
||||||
```
|
```
|
||||||
|
|
||||||
|
If the `limit` is reached, then arbitrary subset of unique values can be returned. The `hits` calculation doesn't work when the `limit` is reached.
|
||||||
|
|
||||||
The `by` keyword can be skipped in `uniq ...` pipe. For example, the following query is equivalent to the previous one:
|
The `by` keyword can be skipped in `uniq ...` pipe. For example, the following query is equivalent to the previous one:
|
||||||
|
|
||||||
```logsql
|
```logsql
|
||||||
|
@ -1887,7 +1924,7 @@ across logs for the last 5 minutes:
|
||||||
_time:5m | stats fields_max(duration) as log_with_max_duration
|
_time:5m | stats fields_max(duration) as log_with_max_duration
|
||||||
```
|
```
|
||||||
|
|
||||||
Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract) pipes.
|
Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract-pipe) pipes.
|
||||||
|
|
||||||
If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`.
|
If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`.
|
||||||
For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the maximum `duration` over the last 5 minutes:
|
For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the maximum `duration` over the last 5 minutes:
|
||||||
|
@ -1914,7 +1951,7 @@ across logs for the last 5 minutes:
|
||||||
_time:5m | stats fields_min(duration) as log_with_min_duration
|
_time:5m | stats fields_min(duration) as log_with_min_duration
|
||||||
```
|
```
|
||||||
|
|
||||||
Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract) pipes.
|
Fields from the returned values can be decoded with [`unpack_json`](#unpack_json-pipe) or [`extract`](#extract-pipe) pipes.
|
||||||
|
|
||||||
If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`.
|
If only the specific fields are needed from the returned log entry, then they can be enumerated inside `fields_max(...)`.
|
||||||
For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the minimum `duration` over the last 5 minutes:
|
For example, the following query returns only `_time`, `path` and `duration` fields from the log entry with the minimum `duration` over the last 5 minutes:
|
||||||
|
|
|
@ -211,6 +211,7 @@ See also:
|
||||||
|
|
||||||
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>` [LogsQL 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 `<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.
|
||||||
|
@ -227,11 +228,19 @@ Below is an example JSON output returned from this endpoint:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"streams": [
|
"values": [
|
||||||
"{host=\"1.2.3.4\",app=\"foo\"}",
|
{
|
||||||
"{host=\"1.2.3.4\",app=\"bar\"}",
|
"value": "{host=\"host-123\",app=\"foo\"}",
|
||||||
"{host=\"10.2.3.4\",app=\"foo\"}",
|
"hits": 34980
|
||||||
"{host=\"10.2.3.5\",app=\"baz\"}"
|
},
|
||||||
|
{
|
||||||
|
"value": "{host=\"host-124\",app=\"bar\"}",
|
||||||
|
"hits": 32892
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"value": "{host=\"host-125\",app=\"baz\"}",
|
||||||
|
"hits": 32877
|
||||||
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
@ -250,6 +259,7 @@ See also:
|
||||||
VictoriaLogs provides `/select/logsql/stream_label_names?query=<query>&start=<start>&end=<end>` HTTP endpoint, which returns
|
VictoriaLogs provides `/select/logsql/stream_label_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) label 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>` [LogsQL 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 `<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.
|
||||||
|
@ -266,12 +276,19 @@ Below is an example JSON output returned from this endpoint:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"names": [
|
"values": [
|
||||||
"app",
|
{
|
||||||
"container",
|
"value": "app",
|
||||||
"datacenter",
|
"hits": 1033300623
|
||||||
"host",
|
},
|
||||||
"namespace"
|
{
|
||||||
|
"value": "container",
|
||||||
|
"hits": 1033300623
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"value": "datacenter",
|
||||||
|
"hits": 1033300623
|
||||||
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
@ -288,6 +305,7 @@ See also:
|
||||||
VictoriaLogs provides `/select/logsql/stream_label_values?query=<query>&start=<start>&<end>&label=<labelName>` HTTP endpoint,
|
VictoriaLogs provides `/select/logsql/stream_label_values?query=<query>&start=<start>&<end>&label=<labelName>` 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) label values for the label with the given `<labelName>` 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>` [LogsQL 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 `<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.
|
||||||
|
@ -305,10 +323,14 @@ Below is an example JSON output returned from this endpoint:
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"values": [
|
"values": [
|
||||||
"host-0",
|
{
|
||||||
"host-1",
|
"value": "host-1",
|
||||||
"host-2",
|
"hits": 69426656
|
||||||
"host-3"
|
},
|
||||||
|
{
|
||||||
|
"value": "host-2",
|
||||||
|
"hits": 66507749
|
||||||
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
@ -327,6 +349,7 @@ See also:
|
||||||
|
|
||||||
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>` [LogsQL 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 `<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.
|
||||||
|
@ -343,13 +366,19 @@ Below is an example JSON output returned from this endpoint:
|
||||||
|
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"names": [
|
"values": [
|
||||||
"_msg",
|
{
|
||||||
"_stream",
|
"value": "_msg",
|
||||||
"_time",
|
"hits": 1033300623
|
||||||
"host",
|
},
|
||||||
"level",
|
{
|
||||||
"location"
|
"value": "_stream",
|
||||||
|
"hits": 1033300623
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"value": "_time",
|
||||||
|
"hits": 1033300623
|
||||||
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
@ -366,6 +395,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>` [LogsQL 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 `<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.
|
||||||
|
@ -383,17 +413,25 @@ Below is an example JSON output returned from this endpoint:
|
||||||
```json
|
```json
|
||||||
{
|
{
|
||||||
"values": [
|
"values": [
|
||||||
"host_0",
|
{
|
||||||
"host_1",
|
"value": "host-1",
|
||||||
"host_10",
|
"hits": 69426656
|
||||||
"host_100",
|
},
|
||||||
"host_1000"
|
{
|
||||||
|
"value": "host-2",
|
||||||
|
"hits": 66507749
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"value": "host-3",
|
||||||
|
"hits": 65454351
|
||||||
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
The `/select/logsql/field_names` endpoint supports optional `limit=N` query arg, which allows limiting the number of returned values to `N`.
|
The `/select/logsql/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.
|
||||||
|
When the `limit` is reached, `hits` are zeroed, since they cannot be calculated reliably.
|
||||||
|
|
||||||
See also:
|
See also:
|
||||||
|
|
||||||
|
|
|
@ -1804,6 +1804,8 @@ func appendResultColumnWithName(dst []resultColumn, name string) []resultColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
// addValue adds the given values v to rc.
|
// addValue adds the given values v to rc.
|
||||||
|
//
|
||||||
|
// rc is valid until v is modified.
|
||||||
func (rc *resultColumn) addValue(v string) {
|
func (rc *resultColumn) addValue(v string) {
|
||||||
rc.values = append(rc.values, v)
|
rc.values = append(rc.values, v)
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,9 +2,11 @@ package logstorage
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"regexp"
|
"sync"
|
||||||
|
"unicode/utf8"
|
||||||
|
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/regexutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
// filterRegexp matches the given regexp
|
// filterRegexp matches the given regexp
|
||||||
|
@ -12,17 +14,51 @@ import (
|
||||||
// Example LogsQL: `fieldName:re("regexp")`
|
// Example LogsQL: `fieldName:re("regexp")`
|
||||||
type filterRegexp struct {
|
type filterRegexp struct {
|
||||||
fieldName string
|
fieldName string
|
||||||
re *regexp.Regexp
|
re *regexutil.Regex
|
||||||
|
|
||||||
|
tokens []string
|
||||||
|
tokensOnce sync.Once
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fr *filterRegexp) String() string {
|
func (fr *filterRegexp) String() string {
|
||||||
return fmt.Sprintf("%s~%q", quoteFieldNameIfNeeded(fr.fieldName), fr.re.String())
|
return fmt.Sprintf("%s~%s", quoteFieldNameIfNeeded(fr.fieldName), quoteTokenIfNeeded(fr.re.String()))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fr *filterRegexp) updateNeededFields(neededFields fieldsSet) {
|
func (fr *filterRegexp) updateNeededFields(neededFields fieldsSet) {
|
||||||
neededFields.add(fr.fieldName)
|
neededFields.add(fr.fieldName)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (fr *filterRegexp) getTokens() []string {
|
||||||
|
fr.tokensOnce.Do(fr.initTokens)
|
||||||
|
return fr.tokens
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fr *filterRegexp) initTokens() {
|
||||||
|
literals := fr.re.GetLiterals()
|
||||||
|
for i, literal := range literals {
|
||||||
|
literals[i] = skipFirstLastToken(literal)
|
||||||
|
}
|
||||||
|
fr.tokens = tokenizeStrings(nil, literals)
|
||||||
|
}
|
||||||
|
|
||||||
|
func skipFirstLastToken(s string) string {
|
||||||
|
for {
|
||||||
|
r, runeSize := utf8.DecodeRuneInString(s)
|
||||||
|
if !isTokenRune(r) {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
s = s[runeSize:]
|
||||||
|
}
|
||||||
|
for {
|
||||||
|
r, runeSize := utf8.DecodeLastRuneInString(s)
|
||||||
|
if !isTokenRune(r) {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
s = s[:len(s)-runeSize]
|
||||||
|
}
|
||||||
|
return s
|
||||||
|
}
|
||||||
|
|
||||||
func (fr *filterRegexp) applyToBlockResult(br *blockResult, bm *bitmap) {
|
func (fr *filterRegexp) applyToBlockResult(br *blockResult, bm *bitmap) {
|
||||||
re := fr.re
|
re := fr.re
|
||||||
applyToBlockResultGeneric(br, bm, fr.fieldName, "", func(v, _ string) bool {
|
applyToBlockResultGeneric(br, bm, fr.fieldName, "", func(v, _ string) bool {
|
||||||
|
@ -53,31 +89,37 @@ func (fr *filterRegexp) applyToBlockSearch(bs *blockSearch, bm *bitmap) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
|
tokens := fr.getTokens()
|
||||||
|
|
||||||
switch ch.valueType {
|
switch ch.valueType {
|
||||||
case valueTypeString:
|
case valueTypeString:
|
||||||
matchStringByRegexp(bs, ch, bm, re)
|
matchStringByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeDict:
|
case valueTypeDict:
|
||||||
matchValuesDictByRegexp(bs, ch, bm, re)
|
matchValuesDictByRegexp(bs, ch, bm, re)
|
||||||
case valueTypeUint8:
|
case valueTypeUint8:
|
||||||
matchUint8ByRegexp(bs, ch, bm, re)
|
matchUint8ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeUint16:
|
case valueTypeUint16:
|
||||||
matchUint16ByRegexp(bs, ch, bm, re)
|
matchUint16ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeUint32:
|
case valueTypeUint32:
|
||||||
matchUint32ByRegexp(bs, ch, bm, re)
|
matchUint32ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeUint64:
|
case valueTypeUint64:
|
||||||
matchUint64ByRegexp(bs, ch, bm, re)
|
matchUint64ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeFloat64:
|
case valueTypeFloat64:
|
||||||
matchFloat64ByRegexp(bs, ch, bm, re)
|
matchFloat64ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeIPv4:
|
case valueTypeIPv4:
|
||||||
matchIPv4ByRegexp(bs, ch, bm, re)
|
matchIPv4ByRegexp(bs, ch, bm, re, tokens)
|
||||||
case valueTypeTimestampISO8601:
|
case valueTypeTimestampISO8601:
|
||||||
matchTimestampISO8601ByRegexp(bs, ch, bm, re)
|
matchTimestampISO8601ByRegexp(bs, ch, bm, re, tokens)
|
||||||
default:
|
default:
|
||||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchTimestampISO8601ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchTimestampISO8601ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toTimestampISO8601String(bs, bb, v)
|
s := toTimestampISO8601String(bs, bb, v)
|
||||||
|
@ -86,7 +128,11 @@ func matchTimestampISO8601ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchIPv4ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchIPv4ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toIPv4String(bs, bb, v)
|
s := toIPv4String(bs, bb, v)
|
||||||
|
@ -95,7 +141,11 @@ func matchIPv4ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchFloat64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchFloat64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toFloat64String(bs, bb, v)
|
s := toFloat64String(bs, bb, v)
|
||||||
|
@ -104,7 +154,7 @@ func matchFloat64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *reg
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchValuesDictByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchValuesDictByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex) {
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
for _, v := range ch.valuesDict.values {
|
for _, v := range ch.valuesDict.values {
|
||||||
c := byte(0)
|
c := byte(0)
|
||||||
|
@ -117,13 +167,21 @@ func matchValuesDictByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchStringByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchStringByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
return re.MatchString(v)
|
return re.MatchString(v)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchUint8ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchUint8ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toUint8String(bs, bb, v)
|
s := toUint8String(bs, bb, v)
|
||||||
|
@ -132,7 +190,11 @@ func matchUint8ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regex
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchUint16ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchUint16ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toUint16String(bs, bb, v)
|
s := toUint16String(bs, bb, v)
|
||||||
|
@ -141,7 +203,11 @@ func matchUint16ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *rege
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchUint32ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchUint32ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toUint32String(bs, bb, v)
|
s := toUint32String(bs, bb, v)
|
||||||
|
@ -150,7 +216,11 @@ func matchUint32ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *rege
|
||||||
bbPool.Put(bb)
|
bbPool.Put(bb)
|
||||||
}
|
}
|
||||||
|
|
||||||
func matchUint64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
func matchUint64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexutil.Regex, tokens []string) {
|
||||||
|
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||||
|
bm.resetBits()
|
||||||
|
return
|
||||||
|
}
|
||||||
bb := bbPool.Get()
|
bb := bbPool.Get()
|
||||||
visitValues(bs, ch, bm, func(v string) bool {
|
visitValues(bs, ch, bm, func(v string) bool {
|
||||||
s := toUint64String(bs, bb, v)
|
s := toUint64String(bs, bb, v)
|
||||||
|
|
|
@ -1,8 +1,10 @@
|
||||||
package logstorage
|
package logstorage
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"regexp"
|
"fmt"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/regexutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestFilterRegexp(t *testing.T) {
|
func TestFilterRegexp(t *testing.T) {
|
||||||
|
@ -21,32 +23,32 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("0.0"),
|
re: mustCompileRegex("0.0"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||||
|
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile(`^127\.0\.0\.1$`),
|
re: mustCompileRegex(`^127\.0\.0\.1$`),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||||
|
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "non-existing-column",
|
fieldName: "non-existing-column",
|
||||||
re: regexp.MustCompile("foo.+bar|"),
|
re: mustCompileRegex("foo.+bar|"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo.+bar"),
|
re: mustCompileRegex("foo.+bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
|
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "non-existing-column",
|
fieldName: "non-existing-column",
|
||||||
re: regexp.MustCompile("foo.+bar"),
|
re: mustCompileRegex("foo.+bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -71,20 +73,20 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar|^$"),
|
re: mustCompileRegex("foo|bar|^$"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 5, 6})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 5, 6})
|
||||||
|
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("27.0"),
|
re: mustCompileRegex("27.0"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 5, 6, 7})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 5, 6, 7})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("bar.+foo"),
|
re: mustCompileRegex("bar.+foo"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -111,14 +113,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("(?i)foo|йцу"),
|
re: mustCompileRegex("(?i)foo|йцу"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 6, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 6, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("qwe.+rty|^$"),
|
re: mustCompileRegex("qwe.+rty|^$"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -146,14 +148,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("[32][23]?"),
|
re: mustCompileRegex("[32][23]?"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar"),
|
re: mustCompileRegex("foo|bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -181,14 +183,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("[32][23]?"),
|
re: mustCompileRegex("[32][23]?"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar"),
|
re: mustCompileRegex("foo|bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -216,14 +218,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("[32][23]?"),
|
re: mustCompileRegex("[32][23]?"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar"),
|
re: mustCompileRegex("foo|bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -251,14 +253,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("[32][23]?"),
|
re: mustCompileRegex("[32][23]?"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar"),
|
re: mustCompileRegex("foo|bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -286,14 +288,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("[32][23]?"),
|
re: mustCompileRegex("[32][23]?"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 6, 7, 8})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 6, 7, 8})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar"),
|
re: mustCompileRegex("foo|bar"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -322,14 +324,14 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("127.0.[40].(1|2)"),
|
re: mustCompileRegex("127.0.[40].(1|2)"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 4, 5, 6, 7})
|
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 4, 5, 6, 7})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "foo",
|
fieldName: "foo",
|
||||||
re: regexp.MustCompile("foo|bar|834"),
|
re: mustCompileRegex("foo|bar|834"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||||
})
|
})
|
||||||
|
@ -355,15 +357,42 @@ func TestFilterRegexp(t *testing.T) {
|
||||||
// match
|
// match
|
||||||
fr := &filterRegexp{
|
fr := &filterRegexp{
|
||||||
fieldName: "_msg",
|
fieldName: "_msg",
|
||||||
re: regexp.MustCompile("2006-[0-9]{2}-.+?(2|5)Z"),
|
re: mustCompileRegex("2006-[0-9]{2}-.+?(2|5)Z"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "_msg", []int{1, 4})
|
testFilterMatchForColumns(t, columns, fr, "_msg", []int{1, 4})
|
||||||
|
|
||||||
// mismatch
|
// mismatch
|
||||||
fr = &filterRegexp{
|
fr = &filterRegexp{
|
||||||
fieldName: "_msg",
|
fieldName: "_msg",
|
||||||
re: regexp.MustCompile("^01|04$"),
|
re: mustCompileRegex("^01|04$"),
|
||||||
}
|
}
|
||||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestSkipFirstLastToken(t *testing.T) {
|
||||||
|
f := func(s, resultExpected string) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
result := skipFirstLastToken(s)
|
||||||
|
if result != resultExpected {
|
||||||
|
t.Fatalf("unexpected result in skipFirstLastToken(%q); got %q; want %q", s, result, resultExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", "")
|
||||||
|
f("foobar", "")
|
||||||
|
f("foo bar", " ")
|
||||||
|
f("foo bar baz", " bar ")
|
||||||
|
f(" foo bar baz", " foo bar ")
|
||||||
|
f(",foo bar baz!", ",foo bar baz!")
|
||||||
|
f("фыад длоа д!", " длоа д!")
|
||||||
|
}
|
||||||
|
|
||||||
|
func mustCompileRegex(expr string) *regexutil.Regex {
|
||||||
|
re, err := regexutil.NewRegex(expr)
|
||||||
|
if err != nil {
|
||||||
|
panic(fmt.Errorf("BUG: cannot compile %q: %w", expr, err))
|
||||||
|
}
|
||||||
|
return re
|
||||||
|
}
|
||||||
|
|
|
@ -22,11 +22,7 @@ type filterStream struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fs *filterStream) String() string {
|
func (fs *filterStream) String() string {
|
||||||
s := fs.f.String()
|
return "_stream:" + fs.f.String()
|
||||||
if s == "{}" {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
return "_stream:" + s
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (fs *filterStream) updateNeededFields(neededFields fieldsSet) {
|
func (fs *filterStream) updateNeededFields(neededFields fieldsSet) {
|
||||||
|
|
|
@ -38,7 +38,7 @@ func (p *logfmtParser) parse(s string) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Search for field value
|
// Search for field value
|
||||||
value, nOffset := tryUnquoteString(s)
|
value, nOffset := tryUnquoteString(s, "")
|
||||||
if nOffset >= 0 {
|
if nOffset >= 0 {
|
||||||
p.addField(name, value)
|
p.addField(name, value)
|
||||||
s = s[nOffset:]
|
s = s[nOffset:]
|
||||||
|
|
|
@ -3,7 +3,6 @@ package logstorage
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
"regexp"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
@ -12,6 +11,7 @@ import (
|
||||||
|
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
|
||||||
|
"github.com/VictoriaMetrics/VictoriaMetrics/lib/regexutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type lexer struct {
|
type lexer struct {
|
||||||
|
@ -597,8 +597,12 @@ func parseGenericFilter(lex *lexer, fieldName string) (filter, error) {
|
||||||
return parseFilterLT(lex, fieldName)
|
return parseFilterLT(lex, fieldName)
|
||||||
case lex.isKeyword("="):
|
case lex.isKeyword("="):
|
||||||
return parseFilterEQ(lex, fieldName)
|
return parseFilterEQ(lex, fieldName)
|
||||||
|
case lex.isKeyword("!="):
|
||||||
|
return parseFilterNEQ(lex, fieldName)
|
||||||
case lex.isKeyword("~"):
|
case lex.isKeyword("~"):
|
||||||
return parseFilterTilda(lex, fieldName)
|
return parseFilterTilda(lex, fieldName)
|
||||||
|
case lex.isKeyword("!~"):
|
||||||
|
return parseFilterNotTilda(lex, fieldName)
|
||||||
case lex.isKeyword("not", "!"):
|
case lex.isKeyword("not", "!"):
|
||||||
return parseFilterNot(lex, fieldName)
|
return parseFilterNot(lex, fieldName)
|
||||||
case lex.isKeyword("exact"):
|
case lex.isKeyword("exact"):
|
||||||
|
@ -1007,7 +1011,7 @@ func parseFilterExact(lex *lexer, fieldName string) (filter, error) {
|
||||||
func parseFilterRegexp(lex *lexer, fieldName string) (filter, error) {
|
func parseFilterRegexp(lex *lexer, fieldName string) (filter, error) {
|
||||||
funcName := lex.token
|
funcName := lex.token
|
||||||
return parseFuncArg(lex, fieldName, func(arg string) (filter, error) {
|
return parseFuncArg(lex, fieldName, func(arg string) (filter, error) {
|
||||||
re, err := regexp.Compile(arg)
|
re, err := regexutil.NewRegex(arg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, fmt.Errorf("invalid regexp %q for %s(): %w", arg, funcName, err)
|
return nil, fmt.Errorf("invalid regexp %q for %s(): %w", arg, funcName, err)
|
||||||
}
|
}
|
||||||
|
@ -1022,7 +1026,7 @@ func parseFilterRegexp(lex *lexer, fieldName string) (filter, error) {
|
||||||
func parseFilterTilda(lex *lexer, fieldName string) (filter, error) {
|
func parseFilterTilda(lex *lexer, fieldName string) (filter, error) {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
arg := getCompoundFuncArg(lex)
|
arg := getCompoundFuncArg(lex)
|
||||||
re, err := regexp.Compile(arg)
|
re, err := regexutil.NewRegex(arg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, fmt.Errorf("invalid regexp %q: %w", arg, err)
|
return nil, fmt.Errorf("invalid regexp %q: %w", arg, err)
|
||||||
}
|
}
|
||||||
|
@ -1033,6 +1037,17 @@ func parseFilterTilda(lex *lexer, fieldName string) (filter, error) {
|
||||||
return fr, nil
|
return fr, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func parseFilterNotTilda(lex *lexer, fieldName string) (filter, error) {
|
||||||
|
f, err := parseFilterTilda(lex, fieldName)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
fn := &filterNot{
|
||||||
|
f: f,
|
||||||
|
}
|
||||||
|
return fn, nil
|
||||||
|
}
|
||||||
|
|
||||||
func parseFilterEQ(lex *lexer, fieldName string) (filter, error) {
|
func parseFilterEQ(lex *lexer, fieldName string) (filter, error) {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
phrase := getCompoundFuncArg(lex)
|
phrase := getCompoundFuncArg(lex)
|
||||||
|
@ -1051,6 +1066,17 @@ func parseFilterEQ(lex *lexer, fieldName string) (filter, error) {
|
||||||
return f, nil
|
return f, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func parseFilterNEQ(lex *lexer, fieldName string) (filter, error) {
|
||||||
|
f, err := parseFilterEQ(lex, fieldName)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
fn := &filterNot{
|
||||||
|
f: f,
|
||||||
|
}
|
||||||
|
return fn, nil
|
||||||
|
}
|
||||||
|
|
||||||
func parseFilterGT(lex *lexer, fieldName string) (filter, error) {
|
func parseFilterGT(lex *lexer, fieldName string) (filter, error) {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
|
|
||||||
|
|
|
@ -544,6 +544,16 @@ func TestParseQuerySuccess(t *testing.T) {
|
||||||
if result != resultExpected {
|
if result != resultExpected {
|
||||||
t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", result, resultExpected)
|
t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", result, resultExpected)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// verify that the marshaled query is parsed to the same query
|
||||||
|
qParsed, err := ParseQuery(result)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("cannot parse marshaled query: %s", err)
|
||||||
|
}
|
||||||
|
qStr := qParsed.String()
|
||||||
|
if qStr != result {
|
||||||
|
t.Fatalf("unexpected marshaled query\ngot\n%s\nwant\n%s", qStr, result)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
f("foo", "foo")
|
f("foo", "foo")
|
||||||
|
@ -586,7 +596,7 @@ func TestParseQuerySuccess(t *testing.T) {
|
||||||
f(`foo:(bar baz or not :xxx)`, `foo:bar foo:baz or !foo:xxx`)
|
f(`foo:(bar baz or not :xxx)`, `foo:bar foo:baz or !foo:xxx`)
|
||||||
f(`(foo:bar and (foo:baz or aa:bb) and xx) and y`, `foo:bar (foo:baz or aa:bb) xx y`)
|
f(`(foo:bar and (foo:baz or aa:bb) and xx) and y`, `foo:bar (foo:baz or aa:bb) xx y`)
|
||||||
f("level:error and _msg:(a or b)", "level:error (a or b)")
|
f("level:error and _msg:(a or b)", "level:error (a or b)")
|
||||||
f("level: ( ((error or warn*) and re(foo))) (not (bar))", `(level:error or level:warn*) level:~"foo" !bar`)
|
f("level: ( ((error or warn*) and re(foo))) (not (bar))", `(level:error or level:warn*) level:~foo !bar`)
|
||||||
f("!(foo bar or baz and not aa*)", `!(foo bar or baz !aa*)`)
|
f("!(foo bar or baz and not aa*)", `!(foo bar or baz !aa*)`)
|
||||||
|
|
||||||
// prefix search
|
// prefix search
|
||||||
|
@ -600,7 +610,7 @@ func TestParseQuerySuccess(t *testing.T) {
|
||||||
f(`"" or foo:"" and not bar:""`, `"" or foo:"" !bar:""`)
|
f(`"" or foo:"" and not bar:""`, `"" or foo:"" !bar:""`)
|
||||||
|
|
||||||
// _stream filters
|
// _stream filters
|
||||||
f(`_stream:{}`, ``)
|
f(`_stream:{}`, `_stream:{}`)
|
||||||
f(`_stream:{foo="bar", baz=~"x" OR or!="b", "x=},"="d}{"}`, `_stream:{foo="bar",baz=~"x" or "or"!="b","x=},"="d}{"}`)
|
f(`_stream:{foo="bar", baz=~"x" OR or!="b", "x=},"="d}{"}`, `_stream:{foo="bar",baz=~"x" or "or"!="b","x=},"="d}{"}`)
|
||||||
f(`_stream:{or=a or ","="b"}`, `_stream:{"or"="a" or ","="b"}`)
|
f(`_stream:{or=a or ","="b"}`, `_stream:{"or"="a" or ","="b"}`)
|
||||||
f("_stream : { foo = bar , } ", `_stream:{foo="bar"}`)
|
f("_stream : { foo = bar , } ", `_stream:{foo="bar"}`)
|
||||||
|
@ -713,7 +723,7 @@ func TestParseQuerySuccess(t *testing.T) {
|
||||||
f(`exact("foo/bar")`, `="foo/bar"`)
|
f(`exact("foo/bar")`, `="foo/bar"`)
|
||||||
f(`exact('foo/bar')`, `="foo/bar"`)
|
f(`exact('foo/bar')`, `="foo/bar"`)
|
||||||
f(`="foo/bar"`, `="foo/bar"`)
|
f(`="foo/bar"`, `="foo/bar"`)
|
||||||
f("=foo=bar =b<=a>z ='abc'*", `="foo=bar" ="b<=a>z" =abc*`)
|
f("=foo=bar !=b<=a>z foo:!='abc'*", `="foo=bar" !="b<=a>z" !foo:=abc*`)
|
||||||
f("==foo =>=bar x : ( = =a<b*='c*' >=20)", `="=foo" =">=bar" x:="=a<b"* x:="c*" x:>=20`)
|
f("==foo =>=bar x : ( = =a<b*='c*' >=20)", `="=foo" =">=bar" x:="=a<b"* x:="c*" x:>=20`)
|
||||||
|
|
||||||
// i filter
|
// i filter
|
||||||
|
@ -772,14 +782,14 @@ func TestParseQuerySuccess(t *testing.T) {
|
||||||
f(`foo: >= 10.5M`, `foo:>=10.5M`)
|
f(`foo: >= 10.5M`, `foo:>=10.5M`)
|
||||||
f(`foo: < 10.5M`, `foo:<10.5M`)
|
f(`foo: < 10.5M`, `foo:<10.5M`)
|
||||||
f(`foo: <= 10.5M`, `foo:<=10.5M`)
|
f(`foo: <= 10.5M`, `foo:<=10.5M`)
|
||||||
f(`foo:(>10 <=20)`, `foo:>10 foo:<=20`)
|
f(`foo:(>10 !<=20)`, `foo:>10 !foo:<=20`)
|
||||||
f(`>=10 <20`, `>=10 <20`)
|
f(`>=10 !<20`, `>=10 !<20`)
|
||||||
|
|
||||||
// re filter
|
// re filter
|
||||||
f("re('foo|ba(r.+)')", `~"foo|ba(r.+)"`)
|
f("re('foo|ba(r.+)')", `~"foo|ba(r.+)"`)
|
||||||
f("re(foo)", `~"foo"`)
|
f("re(foo)", `~foo`)
|
||||||
f(`foo:re(foo-bar/baz.)`, `foo:~"foo-bar/baz."`)
|
f(`foo:re(foo-bar/baz.)`, `foo:~"foo-bar/baz."`)
|
||||||
f(`~foo.bar.baz`, `~"foo.bar.baz"`)
|
f(`~foo.bar.baz !~bar`, `~foo.bar.baz !~bar`)
|
||||||
f(`foo:~~foo~ba/ba>z`, `foo:~"~foo~ba/ba>z"`)
|
f(`foo:~~foo~ba/ba>z`, `foo:~"~foo~ba/ba>z"`)
|
||||||
f(`foo:~'.*'`, `foo:~".*"`)
|
f(`foo:~'.*'`, `foo:~".*"`)
|
||||||
|
|
||||||
|
@ -1266,7 +1276,6 @@ func TestParseQueryFailure(t *testing.T) {
|
||||||
f(`foo | fields bar,,`)
|
f(`foo | fields bar,,`)
|
||||||
|
|
||||||
// invalid field_names
|
// invalid field_names
|
||||||
f(`foo | field_names`)
|
|
||||||
f(`foo | field_names |`)
|
f(`foo | field_names |`)
|
||||||
f(`foo | field_names (`)
|
f(`foo | field_names (`)
|
||||||
f(`foo | field_names )`)
|
f(`foo | field_names )`)
|
||||||
|
|
|
@ -5,8 +5,6 @@ import (
|
||||||
"html"
|
"html"
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// pattern represents text pattern in the form 'some_text<some_field>other_text...'
|
// pattern represents text pattern in the form 'some_text<some_field>other_text...'
|
||||||
|
@ -28,18 +26,25 @@ type patternField struct {
|
||||||
|
|
||||||
type patternStep struct {
|
type patternStep struct {
|
||||||
prefix string
|
prefix string
|
||||||
|
|
||||||
field string
|
field string
|
||||||
opt string
|
fieldOpt string
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ptn *pattern) clone() *pattern {
|
func (ptn *pattern) clone() *pattern {
|
||||||
steps := ptn.steps
|
matches := make([]string, len(ptn.steps))
|
||||||
fields, matches := newFieldsAndMatchesFromPatternSteps(steps)
|
var fields []patternField
|
||||||
if len(fields) == 0 {
|
for i, step := range ptn.steps {
|
||||||
logger.Panicf("BUG: fields cannot be empty for steps=%v", steps)
|
if step.field != "" {
|
||||||
|
fields = append(fields, patternField{
|
||||||
|
name: step.field,
|
||||||
|
value: &matches[i],
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return &pattern{
|
return &pattern{
|
||||||
steps: steps,
|
steps: ptn.steps,
|
||||||
matches: matches,
|
matches: matches,
|
||||||
fields: fields,
|
fields: fields,
|
||||||
}
|
}
|
||||||
|
@ -59,7 +64,18 @@ func parsePattern(s string) (*pattern, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Build pattern struct
|
// Build pattern struct
|
||||||
fields, matches := newFieldsAndMatchesFromPatternSteps(steps)
|
|
||||||
|
matches := make([]string, len(steps))
|
||||||
|
|
||||||
|
var fields []patternField
|
||||||
|
for i, step := range steps {
|
||||||
|
if step.field != "" {
|
||||||
|
fields = append(fields, patternField{
|
||||||
|
name: step.field,
|
||||||
|
value: &matches[i],
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
if len(fields) == 0 {
|
if len(fields) == 0 {
|
||||||
return nil, fmt.Errorf("pattern %q must contain at least a single named field in the form <field_name>", s)
|
return nil, fmt.Errorf("pattern %q must contain at least a single named field in the form <field_name>", s)
|
||||||
}
|
}
|
||||||
|
@ -72,35 +88,17 @@ func parsePattern(s string) (*pattern, error) {
|
||||||
return ptn, nil
|
return ptn, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func newFieldsAndMatchesFromPatternSteps(steps []patternStep) ([]patternField, []string) {
|
|
||||||
matches := make([]string, len(steps))
|
|
||||||
|
|
||||||
var fields []patternField
|
|
||||||
for i, step := range steps {
|
|
||||||
if step.field != "" {
|
|
||||||
fields = append(fields, patternField{
|
|
||||||
name: step.field,
|
|
||||||
value: &matches[i],
|
|
||||||
})
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return fields, matches
|
|
||||||
}
|
|
||||||
|
|
||||||
func (ptn *pattern) apply(s string) {
|
func (ptn *pattern) apply(s string) {
|
||||||
clear(ptn.matches)
|
clear(ptn.matches)
|
||||||
|
|
||||||
steps := ptn.steps
|
steps := ptn.steps
|
||||||
|
|
||||||
if prefix := steps[0].prefix; prefix != "" {
|
n, prefixLen := prefixIndex(s, steps[0].prefix)
|
||||||
n := strings.Index(s, prefix)
|
|
||||||
if n < 0 {
|
if n < 0 {
|
||||||
// Mismatch
|
// Mismatch
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
s = s[n+len(prefix):]
|
s = s[n+prefixLen:]
|
||||||
}
|
|
||||||
|
|
||||||
matches := ptn.matches
|
matches := ptn.matches
|
||||||
for i := range steps {
|
for i := range steps {
|
||||||
|
@ -109,7 +107,7 @@ func (ptn *pattern) apply(s string) {
|
||||||
nextPrefix = steps[i+1].prefix
|
nextPrefix = steps[i+1].prefix
|
||||||
}
|
}
|
||||||
|
|
||||||
us, nOffset := tryUnquoteString(s)
|
us, nOffset := tryUnquoteString(s, steps[i].fieldOpt)
|
||||||
if nOffset >= 0 {
|
if nOffset >= 0 {
|
||||||
// Matched quoted string
|
// Matched quoted string
|
||||||
matches[i] = us
|
matches[i] = us
|
||||||
|
@ -125,31 +123,45 @@ func (ptn *pattern) apply(s string) {
|
||||||
matches[i] = s
|
matches[i] = s
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
n := strings.Index(s, nextPrefix)
|
n, prefixLen := prefixIndex(s, nextPrefix)
|
||||||
if n < 0 {
|
if n < 0 {
|
||||||
// Mismatch
|
// Mismatch
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
matches[i] = s[:n]
|
matches[i] = s[:n]
|
||||||
s = s[n+len(nextPrefix):]
|
s = s[n+prefixLen:]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func tryUnquoteString(s string) (string, int) {
|
func prefixIndex(s, prefix string) (int, int) {
|
||||||
|
if len(prefix) == 0 {
|
||||||
|
return 0, 0
|
||||||
|
}
|
||||||
|
n := strings.Index(s, prefix)
|
||||||
|
if n < 0 {
|
||||||
|
return -1, 0
|
||||||
|
}
|
||||||
|
return n, len(prefix)
|
||||||
|
}
|
||||||
|
|
||||||
|
func tryUnquoteString(s, opt string) (string, int) {
|
||||||
|
if opt == "plain" {
|
||||||
|
return "", -1
|
||||||
|
}
|
||||||
if len(s) == 0 {
|
if len(s) == 0 {
|
||||||
return s, -1
|
return "", -1
|
||||||
}
|
}
|
||||||
if s[0] != '"' && s[0] != '`' {
|
if s[0] != '"' && s[0] != '`' {
|
||||||
return s, -1
|
return "", -1
|
||||||
}
|
}
|
||||||
qp, err := strconv.QuotedPrefix(s)
|
qp, err := strconv.QuotedPrefix(s)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return s, -1
|
return "", -1
|
||||||
}
|
}
|
||||||
us, err := strconv.Unquote(qp)
|
us, err := strconv.Unquote(qp)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return s, -1
|
return "", -1
|
||||||
}
|
}
|
||||||
return us, len(qp)
|
return us, len(qp)
|
||||||
}
|
}
|
||||||
|
@ -160,7 +172,7 @@ func parsePatternSteps(s string) ([]patternStep, error) {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unescape prefixes
|
// unescape prefixes
|
||||||
for i := range steps {
|
for i := range steps {
|
||||||
step := &steps[i]
|
step := &steps[i]
|
||||||
step.prefix = html.UnescapeString(step.prefix)
|
step.prefix = html.UnescapeString(step.prefix)
|
||||||
|
@ -171,9 +183,10 @@ func parsePatternSteps(s string) ([]patternStep, error) {
|
||||||
step := &steps[i]
|
step := &steps[i]
|
||||||
field := step.field
|
field := step.field
|
||||||
if n := strings.IndexByte(field, ':'); n >= 0 {
|
if n := strings.IndexByte(field, ':'); n >= 0 {
|
||||||
step.opt = field[:n]
|
step.fieldOpt = strings.TrimSpace(field[:n])
|
||||||
step.field = field[n+1:]
|
field = field[n+1:]
|
||||||
}
|
}
|
||||||
|
step.field = strings.TrimSpace(field)
|
||||||
}
|
}
|
||||||
|
|
||||||
return steps, nil
|
return steps, nil
|
||||||
|
|
|
@ -63,6 +63,9 @@ func TestPatternApply(t *testing.T) {
|
||||||
f(`foo=<bar> `, "foo=`bar baz,abc` def", []string{"bar baz,abc"})
|
f(`foo=<bar> `, "foo=`bar baz,abc` def", []string{"bar baz,abc"})
|
||||||
f(`<foo>`, `"foo,\"bar"`, []string{`foo,"bar`})
|
f(`<foo>`, `"foo,\"bar"`, []string{`foo,"bar`})
|
||||||
f(`<foo>,"bar`, `"foo,\"bar"`, []string{`foo,"bar`})
|
f(`<foo>,"bar`, `"foo,\"bar"`, []string{`foo,"bar`})
|
||||||
|
|
||||||
|
// disable automatic unquoting of quoted field
|
||||||
|
f(`[<plain:foo>]`, `["foo","bar"]`, []string{`"foo","bar"`})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestParsePatternFailure(t *testing.T) {
|
func TestParsePatternFailure(t *testing.T) {
|
||||||
|
@ -205,15 +208,15 @@ func TestParsePatternStepsSuccess(t *testing.T) {
|
||||||
prefix: ">",
|
prefix: ">",
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
f("<q:foo>bar<abc:baz:c:y>f<:foo:bar:baz>", []patternStep{
|
f("< q : foo >bar<plain : baz:c:y>f<:foo:bar:baz>", []patternStep{
|
||||||
{
|
{
|
||||||
field: "foo",
|
field: "foo",
|
||||||
opt: "q",
|
fieldOpt: "q",
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
prefix: "bar",
|
prefix: "bar",
|
||||||
field: "baz:c:y",
|
field: "baz:c:y",
|
||||||
opt: "abc",
|
fieldOpt: "plain",
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
prefix: "f",
|
prefix: "f",
|
||||||
|
|
|
@ -99,6 +99,30 @@ func TestPipeExtract(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
// single row, disable unquoting
|
||||||
|
f(`extract 'foo=[< plain : bar >]' from x`, [][]Field{
|
||||||
|
{
|
||||||
|
{"x", `a foo=["bc","de"]`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"x", `a foo=["bc","de"]`},
|
||||||
|
{"bar", `"bc","de"`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
// single row, default unquoting
|
||||||
|
f(`extract 'foo=[< bar >]' from x`, [][]Field{
|
||||||
|
{
|
||||||
|
{"x", `a foo=["bc","de"]`},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"x", `a foo=["bc","de"]`},
|
||||||
|
{"bar", `bc`},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
// single row, overwirte existing column
|
// single row, overwirte existing column
|
||||||
f(`extract "foo=<bar> baz=<xx>" from x`, [][]Field{
|
f(`extract "foo=<bar> baz=<xx>" from x`, [][]Field{
|
||||||
{
|
{
|
||||||
|
|
|
@ -10,7 +10,8 @@ import (
|
||||||
//
|
//
|
||||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#field-names-pipe
|
// See https://docs.victoriametrics.com/victorialogs/logsql/#field-names-pipe
|
||||||
type pipeFieldNames struct {
|
type pipeFieldNames struct {
|
||||||
// resultName is the name of the column to write results to.
|
// resultName is an optional name of the column to write results to.
|
||||||
|
// By default results are written into 'name' column.
|
||||||
resultName string
|
resultName string
|
||||||
|
|
||||||
// isFirstPipe is set to true if '| field_names' pipe is the first in the query.
|
// isFirstPipe is set to true if '| field_names' pipe is the first in the query.
|
||||||
|
@ -20,7 +21,11 @@ type pipeFieldNames struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFieldNames) String() string {
|
func (pf *pipeFieldNames) String() string {
|
||||||
return "field_names as " + quoteTokenIfNeeded(pf.resultName)
|
s := "field_names"
|
||||||
|
if pf.resultName != "name" {
|
||||||
|
s += " as " + quoteTokenIfNeeded(pf.resultName)
|
||||||
|
}
|
||||||
|
return s
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pf *pipeFieldNames) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
func (pf *pipeFieldNames) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||||
|
@ -34,13 +39,6 @@ func (pf *pipeFieldNames) updateNeededFields(neededFields, unneededFields fields
|
||||||
|
|
||||||
func (pf *pipeFieldNames) newPipeProcessor(workersCount int, stopCh <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
func (pf *pipeFieldNames) newPipeProcessor(workersCount int, stopCh <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||||
shards := make([]pipeFieldNamesProcessorShard, workersCount)
|
shards := make([]pipeFieldNamesProcessorShard, workersCount)
|
||||||
for i := range shards {
|
|
||||||
shards[i] = pipeFieldNamesProcessorShard{
|
|
||||||
pipeFieldNamesProcessorShardNopad: pipeFieldNamesProcessorShardNopad{
|
|
||||||
m: make(map[string]struct{}),
|
|
||||||
},
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pfp := &pipeFieldNamesProcessor{
|
pfp := &pipeFieldNamesProcessor{
|
||||||
pf: pf,
|
pf: pf,
|
||||||
|
@ -68,8 +66,15 @@ type pipeFieldNamesProcessorShard struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
type pipeFieldNamesProcessorShardNopad struct {
|
type pipeFieldNamesProcessorShardNopad struct {
|
||||||
// m holds unique field names.
|
// m holds hits per each field name
|
||||||
m map[string]struct{}
|
m map[string]*uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
func (shard *pipeFieldNamesProcessorShard) getM() map[string]*uint64 {
|
||||||
|
if shard.m == nil {
|
||||||
|
shard.m = make(map[string]*uint64)
|
||||||
|
}
|
||||||
|
return shard.m
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pfp *pipeFieldNamesProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pfp *pipeFieldNamesProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -78,12 +83,21 @@ func (pfp *pipeFieldNamesProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
}
|
}
|
||||||
|
|
||||||
shard := &pfp.shards[workerID]
|
shard := &pfp.shards[workerID]
|
||||||
|
m := shard.getM()
|
||||||
|
|
||||||
cs := br.getColumns()
|
cs := br.getColumns()
|
||||||
for _, c := range cs {
|
for _, c := range cs {
|
||||||
if _, ok := shard.m[c.name]; !ok {
|
pHits, ok := m[c.name]
|
||||||
|
if !ok {
|
||||||
nameCopy := strings.Clone(c.name)
|
nameCopy := strings.Clone(c.name)
|
||||||
shard.m[nameCopy] = struct{}{}
|
hits := uint64(0)
|
||||||
|
pHits = &hits
|
||||||
|
m[nameCopy] = pHits
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Assume that the column is set for all the rows in the block.
|
||||||
|
// This is much faster than reading all the column values and counting non-empty rows.
|
||||||
|
*pHits += uint64(len(br.timestamps))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -94,15 +108,25 @@ func (pfp *pipeFieldNamesProcessor) flush() error {
|
||||||
|
|
||||||
// merge state across shards
|
// merge state across shards
|
||||||
shards := pfp.shards
|
shards := pfp.shards
|
||||||
m := shards[0].m
|
m := shards[0].getM()
|
||||||
shards = shards[1:]
|
shards = shards[1:]
|
||||||
for i := range shards {
|
for i := range shards {
|
||||||
for k := range shards[i].m {
|
for name, pHitsSrc := range shards[i].getM() {
|
||||||
m[k] = struct{}{}
|
pHits, ok := m[name]
|
||||||
|
if !ok {
|
||||||
|
m[name] = pHitsSrc
|
||||||
|
} else {
|
||||||
|
*pHits += *pHitsSrc
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if pfp.pf.isFirstPipe {
|
if pfp.pf.isFirstPipe {
|
||||||
m["_time"] = struct{}{}
|
pHits := m["_stream"]
|
||||||
|
if pHits == nil {
|
||||||
|
hits := uint64(0)
|
||||||
|
pHits = &hits
|
||||||
|
}
|
||||||
|
m["_time"] = pHits
|
||||||
}
|
}
|
||||||
|
|
||||||
// write result
|
// write result
|
||||||
|
@ -110,8 +134,11 @@ func (pfp *pipeFieldNamesProcessor) flush() error {
|
||||||
pfp: pfp,
|
pfp: pfp,
|
||||||
}
|
}
|
||||||
wctx.rcs[0].name = pfp.pf.resultName
|
wctx.rcs[0].name = pfp.pf.resultName
|
||||||
for k := range m {
|
wctx.rcs[1].name = "hits"
|
||||||
wctx.writeRow(k)
|
|
||||||
|
for name, pHits := range m {
|
||||||
|
hits := string(marshalUint64String(nil, *pHits))
|
||||||
|
wctx.writeRow(name, hits)
|
||||||
}
|
}
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
|
||||||
|
@ -120,7 +147,7 @@ func (pfp *pipeFieldNamesProcessor) flush() error {
|
||||||
|
|
||||||
type pipeFieldNamesWriteContext struct {
|
type pipeFieldNamesWriteContext struct {
|
||||||
pfp *pipeFieldNamesProcessor
|
pfp *pipeFieldNamesProcessor
|
||||||
rcs [1]resultColumn
|
rcs [2]resultColumn
|
||||||
br blockResult
|
br blockResult
|
||||||
|
|
||||||
// rowsCount is the number of rows in the current block
|
// rowsCount is the number of rows in the current block
|
||||||
|
@ -130,9 +157,10 @@ type pipeFieldNamesWriteContext struct {
|
||||||
valuesLen int
|
valuesLen int
|
||||||
}
|
}
|
||||||
|
|
||||||
func (wctx *pipeFieldNamesWriteContext) writeRow(v string) {
|
func (wctx *pipeFieldNamesWriteContext) writeRow(name, hits string) {
|
||||||
wctx.rcs[0].addValue(v)
|
wctx.rcs[0].addValue(name)
|
||||||
wctx.valuesLen += len(v)
|
wctx.rcs[1].addValue(hits)
|
||||||
|
wctx.valuesLen += len(name) + len(hits)
|
||||||
wctx.rowsCount++
|
wctx.rowsCount++
|
||||||
if wctx.valuesLen >= 1_000_000 {
|
if wctx.valuesLen >= 1_000_000 {
|
||||||
wctx.flush()
|
wctx.flush()
|
||||||
|
@ -145,11 +173,12 @@ func (wctx *pipeFieldNamesWriteContext) flush() {
|
||||||
wctx.valuesLen = 0
|
wctx.valuesLen = 0
|
||||||
|
|
||||||
// Flush rcs to ppBase
|
// Flush rcs to ppBase
|
||||||
br.setResultColumns(wctx.rcs[:1], wctx.rowsCount)
|
br.setResultColumns(wctx.rcs[:], wctx.rowsCount)
|
||||||
wctx.rowsCount = 0
|
wctx.rowsCount = 0
|
||||||
wctx.pfp.ppBase.writeBlock(0, br)
|
wctx.pfp.ppBase.writeBlock(0, br)
|
||||||
br.reset()
|
br.reset()
|
||||||
wctx.rcs[0].resetValues()
|
wctx.rcs[0].resetValues()
|
||||||
|
wctx.rcs[1].resetValues()
|
||||||
}
|
}
|
||||||
|
|
||||||
func parsePipeFieldNames(lex *lexer) (*pipeFieldNames, error) {
|
func parsePipeFieldNames(lex *lexer) (*pipeFieldNames, error) {
|
||||||
|
@ -158,13 +187,21 @@ func parsePipeFieldNames(lex *lexer) (*pipeFieldNames, error) {
|
||||||
}
|
}
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
|
|
||||||
|
resultName := "name"
|
||||||
if lex.isKeyword("as") {
|
if lex.isKeyword("as") {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
}
|
name, err := parseFieldName(lex)
|
||||||
resultName, err := parseFieldName(lex)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, fmt.Errorf("cannot parse result name for 'field_names': %w", err)
|
return nil, fmt.Errorf("cannot parse result name for 'field_names': %w", err)
|
||||||
}
|
}
|
||||||
|
resultName = name
|
||||||
|
} else if !lex.isKeyword("", "|") {
|
||||||
|
name, err := parseFieldName(lex)
|
||||||
|
if err != nil {
|
||||||
|
return nil, fmt.Errorf("cannot parse result name for 'field_names': %w", err)
|
||||||
|
}
|
||||||
|
resultName = name
|
||||||
|
}
|
||||||
|
|
||||||
pf := &pipeFieldNames{
|
pf := &pipeFieldNames{
|
||||||
resultName: resultName,
|
resultName: resultName,
|
||||||
|
|
|
@ -10,6 +10,7 @@ func TestParsePipeFieldNamesSuccess(t *testing.T) {
|
||||||
expectParsePipeSuccess(t, pipeStr)
|
expectParsePipeSuccess(t, pipeStr)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
f(`field_names`)
|
||||||
f(`field_names as x`)
|
f(`field_names as x`)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -19,7 +20,6 @@ func TestParsePipeFieldNamesFailure(t *testing.T) {
|
||||||
expectParsePipeFailure(t, pipeStr)
|
expectParsePipeFailure(t, pipeStr)
|
||||||
}
|
}
|
||||||
|
|
||||||
f(`field_names`)
|
|
||||||
f(`field_names(foo)`)
|
f(`field_names(foo)`)
|
||||||
f(`field_names a b`)
|
f(`field_names a b`)
|
||||||
f(`field_names as`)
|
f(`field_names as`)
|
||||||
|
@ -32,32 +32,47 @@ func TestPipeFieldNames(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// single row, result column doesn't clash with original columns
|
// single row, result column doesn't clash with original columns
|
||||||
f("field_names as x", [][]Field{
|
f("field_names", [][]Field{
|
||||||
{
|
{
|
||||||
{"_msg", `{"foo":"bar"}`},
|
{"_msg", `{"foo":"bar"}`},
|
||||||
{"a", `test`},
|
{"a", `test`},
|
||||||
},
|
},
|
||||||
}, [][]Field{
|
}, [][]Field{
|
||||||
{
|
{
|
||||||
{"x", "_msg"},
|
{"name", "_msg"},
|
||||||
|
{"hits", "1"},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"x", "a"},
|
{"name", "a"},
|
||||||
|
{"hits", "1"},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
// single row, result column do clashes with original columns
|
// single row, result column do clashes with original columns
|
||||||
f("field_names as _msg", [][]Field{
|
f("field_names as x", [][]Field{
|
||||||
{
|
{
|
||||||
{"_msg", `{"foo":"bar"}`},
|
|
||||||
{"a", `test`},
|
{"a", `test`},
|
||||||
|
{"b", "aaa"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `bar`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `bar`},
|
||||||
|
{"c", `bar`},
|
||||||
},
|
},
|
||||||
}, [][]Field{
|
}, [][]Field{
|
||||||
{
|
{
|
||||||
{"_msg", "_msg"},
|
{"x", "a"},
|
||||||
|
{"hits", "3"},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
{"_msg", "a"},
|
{"x", "b"},
|
||||||
|
{"hits", "1"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"x", "c"},
|
||||||
|
{"hits", "1"},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
|
@ -137,7 +137,7 @@ func (shard *pipeFormatProcessorShard) formatRow(pf *pipeFormat, br *blockResult
|
||||||
if step.field != "" {
|
if step.field != "" {
|
||||||
c := br.getColumnByName(step.field)
|
c := br.getColumnByName(step.field)
|
||||||
v := c.getValueAtRow(br, rowIdx)
|
v := c.getValueAtRow(br, rowIdx)
|
||||||
if step.opt == "q" {
|
if step.fieldOpt == "q" {
|
||||||
b = strconv.AppendQuote(b, v)
|
b = strconv.AppendQuote(b, v)
|
||||||
} else {
|
} else {
|
||||||
b = append(b, v...)
|
b = append(b, v...)
|
||||||
|
|
|
@ -477,14 +477,12 @@ func (wctx *pipeTopkWriteContext) writeNextRow(shard *pipeTopkProcessorShard) bo
|
||||||
wctx.rcs = rcs
|
wctx.rcs = rcs
|
||||||
}
|
}
|
||||||
|
|
||||||
var tmpBuf []byte
|
|
||||||
byColumns := r.byColumns
|
byColumns := r.byColumns
|
||||||
byColumnsIsTime := r.byColumnsIsTime
|
byColumnsIsTime := r.byColumnsIsTime
|
||||||
for i := range byFields {
|
for i := range byFields {
|
||||||
v := byColumns[i]
|
v := byColumns[i]
|
||||||
if byColumnsIsTime[i] {
|
if byColumnsIsTime[i] {
|
||||||
tmpBuf = marshalTimestampRFC3339NanoString(tmpBuf[:0], r.timestamp)
|
v = string(marshalTimestampRFC3339NanoString(nil, r.timestamp))
|
||||||
v = bytesutil.ToUnsafeString(tmpBuf)
|
|
||||||
}
|
}
|
||||||
rcs[i].addValue(v)
|
rcs[i].addValue(v)
|
||||||
wctx.valuesLen += len(v)
|
wctx.valuesLen += len(v)
|
||||||
|
|
|
@ -20,6 +20,9 @@ type pipeUniq struct {
|
||||||
// fields contains field names for returning unique values
|
// fields contains field names for returning unique values
|
||||||
byFields []string
|
byFields []string
|
||||||
|
|
||||||
|
// if hitsFieldName isn't empty, then the number of hits per each unique value is stored in this field.
|
||||||
|
hitsFieldName string
|
||||||
|
|
||||||
limit uint64
|
limit uint64
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -28,6 +31,9 @@ func (pu *pipeUniq) String() string {
|
||||||
if len(pu.byFields) > 0 {
|
if len(pu.byFields) > 0 {
|
||||||
s += " by (" + fieldNamesString(pu.byFields) + ")"
|
s += " by (" + fieldNamesString(pu.byFields) + ")"
|
||||||
}
|
}
|
||||||
|
if pu.hitsFieldName != "" {
|
||||||
|
s += " hits"
|
||||||
|
}
|
||||||
if pu.limit > 0 {
|
if pu.limit > 0 {
|
||||||
s += fmt.Sprintf(" limit %d", pu.limit)
|
s += fmt.Sprintf(" limit %d", pu.limit)
|
||||||
}
|
}
|
||||||
|
@ -53,7 +59,6 @@ func (pu *pipeUniq) newPipeProcessor(workersCount int, stopCh <-chan struct{}, c
|
||||||
shards[i] = pipeUniqProcessorShard{
|
shards[i] = pipeUniqProcessorShard{
|
||||||
pipeUniqProcessorShardNopad: pipeUniqProcessorShardNopad{
|
pipeUniqProcessorShardNopad: pipeUniqProcessorShardNopad{
|
||||||
pu: pu,
|
pu: pu,
|
||||||
m: make(map[string]struct{}),
|
|
||||||
stateSizeBudget: stateSizeBudgetChunk,
|
stateSizeBudget: stateSizeBudgetChunk,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
@ -98,8 +103,8 @@ type pipeUniqProcessorShardNopad struct {
|
||||||
// pu points to the parent pipeUniq.
|
// pu points to the parent pipeUniq.
|
||||||
pu *pipeUniq
|
pu *pipeUniq
|
||||||
|
|
||||||
// m holds unique rows.
|
// m holds per-row hits.
|
||||||
m map[string]struct{}
|
m map[string]*uint64
|
||||||
|
|
||||||
// keyBuf is a temporary buffer for building keys for m.
|
// keyBuf is a temporary buffer for building keys for m.
|
||||||
keyBuf []byte
|
keyBuf []byte
|
||||||
|
@ -120,6 +125,7 @@ func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
needHits := shard.pu.hitsFieldName != ""
|
||||||
byFields := shard.pu.byFields
|
byFields := shard.pu.byFields
|
||||||
if len(byFields) == 0 {
|
if len(byFields) == 0 {
|
||||||
// Take into account all the columns in br.
|
// Take into account all the columns in br.
|
||||||
|
@ -132,7 +138,7 @@ func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(c.name))
|
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(c.name))
|
||||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(v))
|
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(v))
|
||||||
}
|
}
|
||||||
shard.updateState(bytesutil.ToUnsafeString(keyBuf))
|
shard.updateState(bytesutil.ToUnsafeString(keyBuf), 1)
|
||||||
}
|
}
|
||||||
shard.keyBuf = keyBuf
|
shard.keyBuf = keyBuf
|
||||||
return true
|
return true
|
||||||
|
@ -142,20 +148,34 @@ func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||||
c := br.getColumnByName(byFields[0])
|
c := br.getColumnByName(byFields[0])
|
||||||
if c.isConst {
|
if c.isConst {
|
||||||
v := c.valuesEncoded[0]
|
v := c.valuesEncoded[0]
|
||||||
shard.updateState(v)
|
shard.updateState(v, uint64(len(br.timestamps)))
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
if c.valueType == valueTypeDict {
|
if c.valueType == valueTypeDict {
|
||||||
|
if needHits {
|
||||||
|
a := encoding.GetUint64s(len(c.dictValues))
|
||||||
|
hits := a.A
|
||||||
|
valuesEncoded := c.getValuesEncoded(br)
|
||||||
|
for _, v := range valuesEncoded {
|
||||||
|
idx := unmarshalUint8(v)
|
||||||
|
hits[idx]++
|
||||||
|
}
|
||||||
|
for i, v := range c.dictValues {
|
||||||
|
shard.updateState(v, hits[i])
|
||||||
|
}
|
||||||
|
encoding.PutUint64s(a)
|
||||||
|
} else {
|
||||||
for _, v := range c.dictValues {
|
for _, v := range c.dictValues {
|
||||||
shard.updateState(v)
|
shard.updateState(v, 0)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
values := c.getValues(br)
|
values := c.getValues(br)
|
||||||
for i, v := range values {
|
for i, v := range values {
|
||||||
if i == 0 || values[i-1] != values[i] {
|
if needHits || i == 0 || values[i-1] != values[i] {
|
||||||
shard.updateState(v)
|
shard.updateState(v, 1)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return true
|
return true
|
||||||
|
@ -174,7 +194,7 @@ func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||||
for i := range br.timestamps {
|
for i := range br.timestamps {
|
||||||
seenValue := true
|
seenValue := true
|
||||||
for _, values := range columnValues {
|
for _, values := range columnValues {
|
||||||
if i == 0 || values[i-1] != values[i] {
|
if needHits || i == 0 || values[i-1] != values[i] {
|
||||||
seenValue = false
|
seenValue = false
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
@ -187,19 +207,31 @@ func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||||
for _, values := range columnValues {
|
for _, values := range columnValues {
|
||||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(values[i]))
|
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(values[i]))
|
||||||
}
|
}
|
||||||
shard.updateState(bytesutil.ToUnsafeString(keyBuf))
|
shard.updateState(bytesutil.ToUnsafeString(keyBuf), 1)
|
||||||
}
|
}
|
||||||
shard.keyBuf = keyBuf
|
shard.keyBuf = keyBuf
|
||||||
|
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
func (shard *pipeUniqProcessorShard) updateState(v string) {
|
func (shard *pipeUniqProcessorShard) updateState(v string, hits uint64) {
|
||||||
if _, ok := shard.m[v]; !ok {
|
m := shard.getM()
|
||||||
|
pHits, ok := m[v]
|
||||||
|
if !ok {
|
||||||
vCopy := strings.Clone(v)
|
vCopy := strings.Clone(v)
|
||||||
shard.m[vCopy] = struct{}{}
|
hits := uint64(0)
|
||||||
shard.stateSizeBudget -= len(vCopy) + int(unsafe.Sizeof(vCopy))
|
pHits = &hits
|
||||||
|
m[vCopy] = pHits
|
||||||
|
shard.stateSizeBudget -= len(vCopy) + int(unsafe.Sizeof(vCopy)+unsafe.Sizeof(hits)+unsafe.Sizeof(pHits))
|
||||||
}
|
}
|
||||||
|
*pHits += hits
|
||||||
|
}
|
||||||
|
|
||||||
|
func (shard *pipeUniqProcessorShard) getM() map[string]*uint64 {
|
||||||
|
if shard.m == nil {
|
||||||
|
shard.m = make(map[string]*uint64)
|
||||||
|
}
|
||||||
|
return shard.m
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pup *pipeUniqProcessor) writeBlock(workerID uint, br *blockResult) {
|
func (pup *pipeUniqProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||||
|
@ -235,17 +267,26 @@ func (pup *pipeUniqProcessor) flush() error {
|
||||||
|
|
||||||
// merge state across shards
|
// merge state across shards
|
||||||
shards := pup.shards
|
shards := pup.shards
|
||||||
m := shards[0].m
|
m := shards[0].getM()
|
||||||
shards = shards[1:]
|
shards = shards[1:]
|
||||||
for i := range shards {
|
for i := range shards {
|
||||||
if needStop(pup.stopCh) {
|
if needStop(pup.stopCh) {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
for k := range shards[i].m {
|
for k, pHitsSrc := range shards[i].getM() {
|
||||||
m[k] = struct{}{}
|
pHits, ok := m[k]
|
||||||
|
if !ok {
|
||||||
|
m[k] = pHitsSrc
|
||||||
|
} else {
|
||||||
|
*pHits += *pHitsSrc
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// There is little sense in returning partial hits when the limit on the number of unique entries is reached.
|
||||||
|
// It is better from UX experience is to return zero hits instead.
|
||||||
|
resetHits := pup.pu.limit > 0 && uint64(len(m)) >= pup.pu.limit
|
||||||
|
|
||||||
// write result
|
// write result
|
||||||
wctx := &pipeUniqWriteContext{
|
wctx := &pipeUniqWriteContext{
|
||||||
|
@ -254,8 +295,23 @@ func (pup *pipeUniqProcessor) flush() error {
|
||||||
byFields := pup.pu.byFields
|
byFields := pup.pu.byFields
|
||||||
var rowFields []Field
|
var rowFields []Field
|
||||||
|
|
||||||
|
addHitsFieldIfNeeded := func(dst []Field, hits uint64) []Field {
|
||||||
|
if pup.pu.hitsFieldName == "" {
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
if resetHits {
|
||||||
|
hits = 0
|
||||||
|
}
|
||||||
|
hitsStr := string(marshalUint64String(nil, hits))
|
||||||
|
dst = append(dst, Field{
|
||||||
|
Name: pup.pu.hitsFieldName,
|
||||||
|
Value: hitsStr,
|
||||||
|
})
|
||||||
|
return dst
|
||||||
|
}
|
||||||
|
|
||||||
if len(byFields) == 0 {
|
if len(byFields) == 0 {
|
||||||
for k := range m {
|
for k, pHits := range m {
|
||||||
if needStop(pup.stopCh) {
|
if needStop(pup.stopCh) {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -280,11 +336,12 @@ func (pup *pipeUniqProcessor) flush() error {
|
||||||
Value: bytesutil.ToUnsafeString(value),
|
Value: bytesutil.ToUnsafeString(value),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
rowFields = addHitsFieldIfNeeded(rowFields, *pHits)
|
||||||
wctx.writeRow(rowFields)
|
wctx.writeRow(rowFields)
|
||||||
}
|
}
|
||||||
} else if len(byFields) == 1 {
|
} else if len(byFields) == 1 {
|
||||||
fieldName := byFields[0]
|
fieldName := byFields[0]
|
||||||
for k := range m {
|
for k, pHits := range m {
|
||||||
if needStop(pup.stopCh) {
|
if needStop(pup.stopCh) {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -293,10 +350,11 @@ func (pup *pipeUniqProcessor) flush() error {
|
||||||
Name: fieldName,
|
Name: fieldName,
|
||||||
Value: k,
|
Value: k,
|
||||||
})
|
})
|
||||||
|
rowFields = addHitsFieldIfNeeded(rowFields, *pHits)
|
||||||
wctx.writeRow(rowFields)
|
wctx.writeRow(rowFields)
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
for k := range m {
|
for k, pHits := range m {
|
||||||
if needStop(pup.stopCh) {
|
if needStop(pup.stopCh) {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -317,6 +375,7 @@ func (pup *pipeUniqProcessor) flush() error {
|
||||||
})
|
})
|
||||||
fieldIdx++
|
fieldIdx++
|
||||||
}
|
}
|
||||||
|
rowFields = addHitsFieldIfNeeded(rowFields, *pHits)
|
||||||
wctx.writeRow(rowFields)
|
wctx.writeRow(rowFields)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -418,6 +477,16 @@ func parsePipeUniq(lex *lexer) (*pipeUniq, error) {
|
||||||
pu.byFields = bfs
|
pu.byFields = bfs
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if lex.isKeyword("hits") {
|
||||||
|
lex.nextToken()
|
||||||
|
hitsFieldName := "hits"
|
||||||
|
for slices.Contains(pu.byFields, hitsFieldName) {
|
||||||
|
hitsFieldName += "s"
|
||||||
|
}
|
||||||
|
|
||||||
|
pu.hitsFieldName = hitsFieldName
|
||||||
|
}
|
||||||
|
|
||||||
if lex.isKeyword("limit") {
|
if lex.isKeyword("limit") {
|
||||||
lex.nextToken()
|
lex.nextToken()
|
||||||
n, ok := tryParseUint64(lex.token)
|
n, ok := tryParseUint64(lex.token)
|
||||||
|
|
|
@ -11,11 +11,15 @@ func TestParsePipeUniqSuccess(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
f(`uniq`)
|
f(`uniq`)
|
||||||
|
f(`uniq hits`)
|
||||||
f(`uniq limit 10`)
|
f(`uniq limit 10`)
|
||||||
|
f(`uniq 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) limit 10`)
|
f(`uniq by (x, y) limit 10`)
|
||||||
|
f(`uniq by (x, y) hits limit 10`)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestParsePipeUniqFailure(t *testing.T) {
|
func TestParsePipeUniqFailure(t *testing.T) {
|
||||||
|
@ -26,6 +30,7 @@ func TestParsePipeUniqFailure(t *testing.T) {
|
||||||
|
|
||||||
f(`uniq foo`)
|
f(`uniq foo`)
|
||||||
f(`uniq by`)
|
f(`uniq by`)
|
||||||
|
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`)
|
||||||
}
|
}
|
||||||
|
@ -62,6 +67,62 @@ func TestPipeUniq(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
{"hits", "2"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
{"hits", "1"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
|
f("uniq hits limit 2", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
{"hits", "0"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
{"hits", "0"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
f("uniq by (a)", [][]Field{
|
f("uniq by (a)", [][]Field{
|
||||||
{
|
{
|
||||||
{"a", `2`},
|
{"a", `2`},
|
||||||
|
@ -82,6 +143,27 @@ func TestPipeUniq(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq by (a) hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"hits", "3"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
f("uniq by (b)", [][]Field{
|
f("uniq by (b)", [][]Field{
|
||||||
{
|
{
|
||||||
{"a", `2`},
|
{"a", `2`},
|
||||||
|
@ -105,6 +187,31 @@ func TestPipeUniq(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq by (b) hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"b", "3"},
|
||||||
|
{"hits", "2"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"b", "54"},
|
||||||
|
{"hits", "1"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
f("uniq by (c)", [][]Field{
|
f("uniq by (c)", [][]Field{
|
||||||
{
|
{
|
||||||
{"a", `2`},
|
{"a", `2`},
|
||||||
|
@ -128,6 +235,31 @@ func TestPipeUniq(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq by (c) hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"c", ""},
|
||||||
|
{"hits", "2"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"c", "d"},
|
||||||
|
{"hits", "1"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
f("uniq by (d)", [][]Field{
|
f("uniq by (d)", [][]Field{
|
||||||
{
|
{
|
||||||
{"a", `2`},
|
{"a", `2`},
|
||||||
|
@ -148,6 +280,27 @@ func TestPipeUniq(t *testing.T) {
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq by (d) hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"d", ""},
|
||||||
|
{"hits", "3"},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
|
||||||
f("uniq by (a, b)", [][]Field{
|
f("uniq by (a, b)", [][]Field{
|
||||||
{
|
{
|
||||||
{"a", `2`},
|
{"a", `2`},
|
||||||
|
@ -172,6 +325,33 @@ func TestPipeUniq(t *testing.T) {
|
||||||
{"b", "54"},
|
{"b", "54"},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
|
||||||
|
f("uniq by (a, b) hits", [][]Field{
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `3`},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", `2`},
|
||||||
|
{"b", `54`},
|
||||||
|
{"c", "d"},
|
||||||
|
},
|
||||||
|
}, [][]Field{
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "3"},
|
||||||
|
{"hits", "2"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "2"},
|
||||||
|
{"b", "54"},
|
||||||
|
{"hits", "1"},
|
||||||
|
},
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestPipeUniqUpdateNeededFields(t *testing.T) {
|
func TestPipeUniqUpdateNeededFields(t *testing.T) {
|
||||||
|
|
|
@ -145,9 +145,9 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetFieldNames returns field names from q results for the given tenantIDs.
|
// GetFieldNames returns field names from q results for the given tenantIDs.
|
||||||
func (s *Storage) GetFieldNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) {
|
func (s *Storage) GetFieldNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]ValueWithHits, error) {
|
||||||
pipes := append([]pipe{}, q.pipes...)
|
pipes := append([]pipe{}, q.pipes...)
|
||||||
pipeStr := "field_names as names | sort by (names)"
|
pipeStr := "field_names"
|
||||||
lex := newLexer(pipeStr)
|
lex := newLexer(pipeStr)
|
||||||
|
|
||||||
pf, err := parsePipeFieldNames(lex)
|
pf, err := parsePipeFieldNames(lex)
|
||||||
|
@ -156,36 +156,24 @@ func (s *Storage) GetFieldNames(ctx context.Context, tenantIDs []TenantID, q *Qu
|
||||||
}
|
}
|
||||||
pf.isFirstPipe = len(pipes) == 0
|
pf.isFirstPipe = len(pipes) == 0
|
||||||
|
|
||||||
if !lex.isKeyword("|") {
|
|
||||||
logger.Panicf("BUG: unexpected token after 'field_names' pipe at [%s]: %q", pipeStr, lex.token)
|
|
||||||
}
|
|
||||||
lex.nextToken()
|
|
||||||
|
|
||||||
ps, err := parsePipeSort(lex)
|
|
||||||
if err != nil {
|
|
||||||
logger.Panicf("BUG: unexpected error when parsing 'sort' pipe at [%s]: %s", pipeStr, err)
|
|
||||||
}
|
|
||||||
if !lex.isEnd() {
|
if !lex.isEnd() {
|
||||||
logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s)
|
logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s)
|
||||||
}
|
}
|
||||||
|
|
||||||
pipes = append(pipes, pf, ps)
|
pipes = append(pipes, pf)
|
||||||
|
|
||||||
q = &Query{
|
q = &Query{
|
||||||
f: q.f,
|
f: q.f,
|
||||||
pipes: pipes,
|
pipes: pipes,
|
||||||
}
|
}
|
||||||
|
|
||||||
return s.runSingleColumnQuery(ctx, tenantIDs, q)
|
return s.runValuesWithHitsQuery(ctx, tenantIDs, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetFieldValues returns unique values for the given fieldName returned by q for the given tenantIDs.
|
func (s *Storage) getFieldValuesNoHits(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string) ([]string, error) {
|
||||||
//
|
|
||||||
// If limit > 0, then up to limit unique values are returned.
|
|
||||||
func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]string, error) {
|
|
||||||
pipes := append([]pipe{}, q.pipes...)
|
pipes := append([]pipe{}, q.pipes...)
|
||||||
quotedFieldName := quoteTokenIfNeeded(fieldName)
|
quotedFieldName := quoteTokenIfNeeded(fieldName)
|
||||||
pipeStr := fmt.Sprintf("uniq by (%s) limit %d | sort by (%s)", quotedFieldName, limit, quotedFieldName)
|
pipeStr := fmt.Sprintf("uniq by (%s)", quotedFieldName)
|
||||||
lex := newLexer(pipeStr)
|
lex := newLexer(pipeStr)
|
||||||
|
|
||||||
pu, err := parsePipeUniq(lex)
|
pu, err := parsePipeUniq(lex)
|
||||||
|
@ -193,87 +181,17 @@ func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Q
|
||||||
logger.Panicf("BUG: unexpected error when parsing 'uniq' pipe at [%s]: %s", pipeStr, err)
|
logger.Panicf("BUG: unexpected error when parsing 'uniq' pipe at [%s]: %s", pipeStr, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
if !lex.isKeyword("|") {
|
|
||||||
logger.Panicf("BUG: unexpected token after 'uniq' pipe at [%s]: %q", pipeStr, lex.token)
|
|
||||||
}
|
|
||||||
lex.nextToken()
|
|
||||||
|
|
||||||
ps, err := parsePipeSort(lex)
|
|
||||||
if err != nil {
|
|
||||||
logger.Panicf("BUG: unexpected error when parsing 'sort' pipe at [%s]: %s", pipeStr, err)
|
|
||||||
}
|
|
||||||
if !lex.isEnd() {
|
if !lex.isEnd() {
|
||||||
logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s)
|
logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s)
|
||||||
}
|
}
|
||||||
|
|
||||||
pipes = append(pipes, pu, ps)
|
pipes = append(pipes, pu)
|
||||||
|
|
||||||
q = &Query{
|
q = &Query{
|
||||||
f: q.f,
|
f: q.f,
|
||||||
pipes: pipes,
|
pipes: pipes,
|
||||||
}
|
}
|
||||||
|
|
||||||
return s.runSingleColumnQuery(ctx, tenantIDs, q)
|
|
||||||
}
|
|
||||||
|
|
||||||
// GetStreamLabelNames returns stream label names from q results for the given tenantIDs.
|
|
||||||
func (s *Storage) GetStreamLabelNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) {
|
|
||||||
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
var names []string
|
|
||||||
m := make(map[string]struct{})
|
|
||||||
forEachStreamLabel(streams, func(label Field) {
|
|
||||||
if _, ok := m[label.Name]; !ok {
|
|
||||||
nameCopy := strings.Clone(label.Name)
|
|
||||||
names = append(names, nameCopy)
|
|
||||||
m[nameCopy] = struct{}{}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
sortStrings(names)
|
|
||||||
|
|
||||||
return names, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// GetStreamLabelValues returns stream label values for the given labelName from q results for the given tenantIDs.
|
|
||||||
//
|
|
||||||
// If limit > 9, then up to limit unique label values are returned.
|
|
||||||
func (s *Storage) GetStreamLabelValues(ctx context.Context, tenantIDs []TenantID, q *Query, labelName string, limit uint64) ([]string, error) {
|
|
||||||
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
var values []string
|
|
||||||
m := make(map[string]struct{})
|
|
||||||
forEachStreamLabel(streams, func(label Field) {
|
|
||||||
if label.Name != labelName {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
if _, ok := m[label.Value]; !ok {
|
|
||||||
valueCopy := strings.Clone(label.Value)
|
|
||||||
values = append(values, valueCopy)
|
|
||||||
m[valueCopy] = struct{}{}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
if uint64(len(values)) > limit {
|
|
||||||
values = values[:limit]
|
|
||||||
}
|
|
||||||
sortStrings(values)
|
|
||||||
|
|
||||||
return values, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// GetStreams returns streams from q results for the given tenantIDs.
|
|
||||||
//
|
|
||||||
// If limit > 0, then up to limit unique streams are returned.
|
|
||||||
func (s *Storage) GetStreams(ctx context.Context, tenantIDs []TenantID, q *Query, limit uint64) ([]string, error) {
|
|
||||||
return s.GetFieldValues(ctx, tenantIDs, q, "_stream", limit)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (s *Storage) runSingleColumnQuery(ctx context.Context, tenantIDs []TenantID, q *Query) ([]string, error) {
|
|
||||||
var values []string
|
var values []string
|
||||||
var valuesLock sync.Mutex
|
var valuesLock sync.Mutex
|
||||||
writeBlockResult := func(_ uint, br *blockResult) {
|
writeBlockResult := func(_ uint, br *blockResult) {
|
||||||
|
@ -283,13 +201,14 @@ func (s *Storage) runSingleColumnQuery(ctx context.Context, tenantIDs []TenantID
|
||||||
|
|
||||||
cs := br.getColumns()
|
cs := br.getColumns()
|
||||||
if len(cs) != 1 {
|
if len(cs) != 1 {
|
||||||
logger.Panicf("BUG: expecting only a single column; got %d columns", len(cs))
|
logger.Panicf("BUG: expecting one column; got %d columns", len(cs))
|
||||||
}
|
}
|
||||||
|
|
||||||
columnValues := cs[0].getValues(br)
|
columnValues := cs[0].getValues(br)
|
||||||
|
|
||||||
columnValuesCopy := make([]string, len(columnValues))
|
columnValuesCopy := make([]string, len(columnValues))
|
||||||
for i, v := range columnValues {
|
for i := range columnValues {
|
||||||
columnValuesCopy[i] = strings.Clone(v)
|
columnValuesCopy[i] = strings.Clone(columnValues[i])
|
||||||
}
|
}
|
||||||
|
|
||||||
valuesLock.Lock()
|
valuesLock.Lock()
|
||||||
|
@ -297,21 +216,182 @@ func (s *Storage) runSingleColumnQuery(ctx context.Context, tenantIDs []TenantID
|
||||||
valuesLock.Unlock()
|
valuesLock.Unlock()
|
||||||
}
|
}
|
||||||
|
|
||||||
err := s.runQuery(ctx, tenantIDs, q, writeBlockResult)
|
if err := s.runQuery(ctx, tenantIDs, q, writeBlockResult); err != nil {
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
return values, nil
|
return values, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetFieldValues returns unique values with the number of hits for the given fieldName returned by q for the given tenantIDs.
|
||||||
|
//
|
||||||
|
// If limit > 0, then up to limit unique values are returned.
|
||||||
|
func (s *Storage) GetFieldValues(ctx context.Context, tenantIDs []TenantID, q *Query, fieldName string, limit uint64) ([]ValueWithHits, error) {
|
||||||
|
pipes := append([]pipe{}, q.pipes...)
|
||||||
|
quotedFieldName := quoteTokenIfNeeded(fieldName)
|
||||||
|
pipeStr := fmt.Sprintf("uniq by (%s) hits limit %d", quotedFieldName, limit)
|
||||||
|
lex := newLexer(pipeStr)
|
||||||
|
|
||||||
|
pu, err := parsePipeUniq(lex)
|
||||||
|
if err != nil {
|
||||||
|
logger.Panicf("BUG: unexpected error when parsing 'uniq' pipe at [%s]: %s", pipeStr, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if !lex.isEnd() {
|
||||||
|
logger.Panicf("BUG: unexpected tail left after parsing pipes [%s]: %q", pipeStr, lex.s)
|
||||||
|
}
|
||||||
|
|
||||||
|
pipes = append(pipes, pu)
|
||||||
|
|
||||||
|
q = &Query{
|
||||||
|
f: q.f,
|
||||||
|
pipes: pipes,
|
||||||
|
}
|
||||||
|
|
||||||
|
return s.runValuesWithHitsQuery(ctx, tenantIDs, q)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ValueWithHits contains value and hits.
|
||||||
|
type ValueWithHits struct {
|
||||||
|
Value string
|
||||||
|
Hits uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
func toValuesWithHits(m map[string]*uint64) []ValueWithHits {
|
||||||
|
results := make([]ValueWithHits, 0, len(m))
|
||||||
|
for k, pHits := range m {
|
||||||
|
results = append(results, ValueWithHits{
|
||||||
|
Value: k,
|
||||||
|
Hits: *pHits,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
sortValuesWithHits(results)
|
||||||
|
return results
|
||||||
|
}
|
||||||
|
|
||||||
|
func sortValuesWithHits(results []ValueWithHits) {
|
||||||
|
slices.SortFunc(results, func(a, b ValueWithHits) int {
|
||||||
|
if a.Hits == b.Hits {
|
||||||
|
if a.Value == b.Value {
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
if lessString(a.Value, b.Value) {
|
||||||
|
return -1
|
||||||
|
}
|
||||||
|
return 1
|
||||||
|
}
|
||||||
|
// Sort in descending order of hits
|
||||||
|
if a.Hits < b.Hits {
|
||||||
|
return 1
|
||||||
|
}
|
||||||
|
return -1
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetStreamLabelNames returns stream label names from q results for the given tenantIDs.
|
||||||
|
func (s *Storage) GetStreamLabelNames(ctx context.Context, tenantIDs []TenantID, q *Query) ([]ValueWithHits, error) {
|
||||||
|
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
m := make(map[string]*uint64)
|
||||||
|
forEachStreamLabel(streams, func(label Field, hits uint64) {
|
||||||
|
pHits, ok := m[label.Name]
|
||||||
|
if !ok {
|
||||||
|
nameCopy := strings.Clone(label.Name)
|
||||||
|
hitsLocal := uint64(0)
|
||||||
|
pHits = &hitsLocal
|
||||||
|
m[nameCopy] = pHits
|
||||||
|
}
|
||||||
|
*pHits += hits
|
||||||
|
})
|
||||||
|
names := toValuesWithHits(m)
|
||||||
|
return names, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetStreamLabelValues returns stream label values for the given labelName from q results for the given tenantIDs.
|
||||||
|
//
|
||||||
|
// If limit > 9, then up to limit unique label values are returned.
|
||||||
|
func (s *Storage) GetStreamLabelValues(ctx context.Context, tenantIDs []TenantID, q *Query, labelName string, limit uint64) ([]ValueWithHits, error) {
|
||||||
|
streams, err := s.GetStreams(ctx, tenantIDs, q, math.MaxUint64)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
m := make(map[string]*uint64)
|
||||||
|
forEachStreamLabel(streams, func(label Field, hits uint64) {
|
||||||
|
if label.Name != labelName {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
pHits, ok := m[label.Value]
|
||||||
|
if !ok {
|
||||||
|
valueCopy := strings.Clone(label.Value)
|
||||||
|
hitsLocal := uint64(0)
|
||||||
|
pHits = &hitsLocal
|
||||||
|
m[valueCopy] = pHits
|
||||||
|
}
|
||||||
|
*pHits += hits
|
||||||
|
})
|
||||||
|
values := toValuesWithHits(m)
|
||||||
|
if limit > 0 && uint64(len(values)) > limit {
|
||||||
|
values = values[:limit]
|
||||||
|
}
|
||||||
|
return values, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetStreams returns streams from q results for the given tenantIDs.
|
||||||
|
//
|
||||||
|
// If limit > 0, then up to limit unique streams are returned.
|
||||||
|
func (s *Storage) GetStreams(ctx context.Context, tenantIDs []TenantID, q *Query, limit uint64) ([]ValueWithHits, error) {
|
||||||
|
return s.GetFieldValues(ctx, tenantIDs, q, "_stream", limit)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *Storage) runValuesWithHitsQuery(ctx context.Context, tenantIDs []TenantID, q *Query) ([]ValueWithHits, error) {
|
||||||
|
var results []ValueWithHits
|
||||||
|
var resultsLock sync.Mutex
|
||||||
|
writeBlockResult := func(_ uint, br *blockResult) {
|
||||||
|
if len(br.timestamps) == 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
cs := br.getColumns()
|
||||||
|
if len(cs) != 2 {
|
||||||
|
logger.Panicf("BUG: expecting two columns; got %d columns", len(cs))
|
||||||
|
}
|
||||||
|
|
||||||
|
columnValues := cs[0].getValues(br)
|
||||||
|
columnHits := cs[1].getValues(br)
|
||||||
|
|
||||||
|
valuesWithHits := make([]ValueWithHits, len(columnValues))
|
||||||
|
for i := range columnValues {
|
||||||
|
x := &valuesWithHits[i]
|
||||||
|
hits, _ := tryParseUint64(columnHits[i])
|
||||||
|
x.Value = strings.Clone(columnValues[i])
|
||||||
|
x.Hits = hits
|
||||||
|
}
|
||||||
|
|
||||||
|
resultsLock.Lock()
|
||||||
|
results = append(results, valuesWithHits...)
|
||||||
|
resultsLock.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
err := s.runQuery(ctx, tenantIDs, q, writeBlockResult)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
sortValuesWithHits(results)
|
||||||
|
|
||||||
|
return results, nil
|
||||||
|
}
|
||||||
|
|
||||||
func (s *Storage) initFilterInValues(ctx context.Context, tenantIDs []TenantID, q *Query) (*Query, error) {
|
func (s *Storage) initFilterInValues(ctx context.Context, tenantIDs []TenantID, q *Query) (*Query, error) {
|
||||||
if !hasFilterInWithQueryForFilter(q.f) && !hasFilterInWithQueryForPipes(q.pipes) {
|
if !hasFilterInWithQueryForFilter(q.f) && !hasFilterInWithQueryForPipes(q.pipes) {
|
||||||
return q, nil
|
return q, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
getFieldValues := func(q *Query, fieldName string) ([]string, error) {
|
getFieldValues := func(q *Query, fieldName string) ([]string, error) {
|
||||||
return s.GetFieldValues(ctx, tenantIDs, q, fieldName, 0)
|
return s.getFieldValuesNoHits(ctx, tenantIDs, q, fieldName)
|
||||||
}
|
}
|
||||||
cache := make(map[string][]string)
|
cache := make(map[string][]string)
|
||||||
fNew, err := initFilterInValuesForFilter(cache, q.f, getFieldValues)
|
fNew, err := initFilterInValuesForFilter(cache, q.f, getFieldValues)
|
||||||
|
@ -1007,16 +1087,17 @@ func getFilterTimeRange(f filter) (int64, int64) {
|
||||||
return math.MinInt64, math.MaxInt64
|
return math.MinInt64, math.MaxInt64
|
||||||
}
|
}
|
||||||
|
|
||||||
func forEachStreamLabel(streams []string, f func(label Field)) {
|
func forEachStreamLabel(streams []ValueWithHits, f func(label Field, hits uint64)) {
|
||||||
var labels []Field
|
var labels []Field
|
||||||
for _, stream := range streams {
|
for i := range streams {
|
||||||
var err error
|
var err error
|
||||||
labels, err = parseStreamLabels(labels[:0], stream)
|
labels, err = parseStreamLabels(labels[:0], streams[i].Value)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
for i := range labels {
|
hits := streams[i].Hits
|
||||||
f(labels[i])
|
for j := range labels {
|
||||||
|
f(labels[j], hits)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1042,7 +1123,7 @@ func parseStreamLabels(dst []Field, s string) ([]Field, error) {
|
||||||
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 label value in double quotes at [%s]", s)
|
||||||
}
|
}
|
||||||
|
|
|
@ -3,7 +3,6 @@ package logstorage
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"regexp"
|
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
@ -582,7 +581,7 @@ func TestStorageSearch(t *testing.T) {
|
||||||
f,
|
f,
|
||||||
&filterRegexp{
|
&filterRegexp{
|
||||||
fieldName: "_msg",
|
fieldName: "_msg",
|
||||||
re: regexp.MustCompile("message [02] at "),
|
re: mustCompileRegex("message [02] at "),
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
|
@ -435,9 +435,9 @@ func parseRelabelConfig(rc *RelabelConfig) (*parsedRelabelConfig, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func isDefaultRegex(expr string) bool {
|
func isDefaultRegex(expr string) bool {
|
||||||
prefix, suffix := regexutil.Simplify(expr)
|
prefix, suffix := regexutil.SimplifyPromRegex(expr)
|
||||||
if prefix != "" {
|
if prefix != "" {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
return suffix == "(?-s:.*)"
|
return suffix == "(?s:.*)"
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,13 +19,21 @@ import (
|
||||||
//
|
//
|
||||||
// The rest of regexps are also optimized by returning cached match results for the same input strings.
|
// The rest of regexps are also optimized by returning cached match results for the same input strings.
|
||||||
type PromRegex struct {
|
type PromRegex struct {
|
||||||
|
// exprStr is the original expression.
|
||||||
|
exprStr string
|
||||||
|
|
||||||
// prefix contains literal prefix for regex.
|
// prefix contains literal prefix for regex.
|
||||||
// For example, prefix="foo" for regex="foo(a|b)"
|
// For example, prefix="foo" for regex="foo(a|b)"
|
||||||
prefix string
|
prefix string
|
||||||
|
|
||||||
// Suffix contains regex suffix left after removing the prefix.
|
// isOnlyPrefix is set to true if the regex contains only the prefix.
|
||||||
// For example, suffix="a|b" for regex="foo(a|b)"
|
isOnlyPrefix bool
|
||||||
suffix string
|
|
||||||
|
// isSuffixDotStar is set to true if suffix is ".*"
|
||||||
|
isSuffixDotStar bool
|
||||||
|
|
||||||
|
// isSuffixDotPlus is set to true if suffix is ".+"
|
||||||
|
isSuffixDotPlus bool
|
||||||
|
|
||||||
// substrDotStar contains literal string for regex suffix=".*string.*"
|
// substrDotStar contains literal string for regex suffix=".*string.*"
|
||||||
substrDotStar string
|
substrDotStar string
|
||||||
|
@ -46,18 +54,25 @@ func NewPromRegex(expr string) (*PromRegex, error) {
|
||||||
if _, err := regexp.Compile(expr); err != nil {
|
if _, err := regexp.Compile(expr); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
prefix, suffix := Simplify(expr)
|
prefix, suffix := SimplifyPromRegex(expr)
|
||||||
orValues := GetOrValues(suffix)
|
sre := mustParseRegexp(suffix)
|
||||||
substrDotStar := getSubstringLiteral(suffix, ".*")
|
orValues := getOrValues(sre)
|
||||||
substrDotPlus := getSubstringLiteral(suffix, ".+")
|
isOnlyPrefix := len(orValues) == 1 && orValues[0] == ""
|
||||||
|
isSuffixDotStar := isDotOp(sre, syntax.OpStar)
|
||||||
|
isSuffixDotPlus := isDotOp(sre, syntax.OpPlus)
|
||||||
|
substrDotStar := getSubstringLiteral(sre, syntax.OpStar)
|
||||||
|
substrDotPlus := getSubstringLiteral(sre, syntax.OpPlus)
|
||||||
// It is expected that Optimize returns valid regexp in suffix, so use MustCompile here.
|
// It is expected that Optimize returns valid regexp in suffix, so use MustCompile here.
|
||||||
// Anchor suffix to the beginning and the end of the matching string.
|
// Anchor suffix to the beginning and the end of the matching string.
|
||||||
suffixExpr := "^(?:" + suffix + ")$"
|
suffixExpr := "^(?:" + suffix + ")$"
|
||||||
reSuffix := regexp.MustCompile(suffixExpr)
|
reSuffix := regexp.MustCompile(suffixExpr)
|
||||||
reSuffixMatcher := bytesutil.NewFastStringMatcher(reSuffix.MatchString)
|
reSuffixMatcher := bytesutil.NewFastStringMatcher(reSuffix.MatchString)
|
||||||
pr := &PromRegex{
|
pr := &PromRegex{
|
||||||
|
exprStr: expr,
|
||||||
prefix: prefix,
|
prefix: prefix,
|
||||||
suffix: suffix,
|
isOnlyPrefix: isOnlyPrefix,
|
||||||
|
isSuffixDotStar: isSuffixDotStar,
|
||||||
|
isSuffixDotPlus: isSuffixDotPlus,
|
||||||
substrDotStar: substrDotStar,
|
substrDotStar: substrDotStar,
|
||||||
substrDotPlus: substrDotPlus,
|
substrDotPlus: substrDotPlus,
|
||||||
orValues: orValues,
|
orValues: orValues,
|
||||||
|
@ -71,19 +86,25 @@ func NewPromRegex(expr string) (*PromRegex, error) {
|
||||||
// The pr is automatically anchored to the beginning and to the end
|
// The pr is automatically anchored to the beginning and to the end
|
||||||
// of the matching string with '^' and '$'.
|
// of the matching string with '^' and '$'.
|
||||||
func (pr *PromRegex) MatchString(s string) bool {
|
func (pr *PromRegex) MatchString(s string) bool {
|
||||||
|
if pr.isOnlyPrefix {
|
||||||
|
return s == pr.prefix
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(pr.prefix) > 0 {
|
||||||
if !strings.HasPrefix(s, pr.prefix) {
|
if !strings.HasPrefix(s, pr.prefix) {
|
||||||
// Fast path - s has another prefix than pr.
|
// Fast path - s has another prefix than pr.
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
s = s[len(pr.prefix):]
|
s = s[len(pr.prefix):]
|
||||||
if len(pr.orValues) > 0 {
|
}
|
||||||
// Fast path - pr contains only alternate strings such as 'foo|bar|baz'
|
|
||||||
for _, v := range pr.orValues {
|
if pr.isSuffixDotStar {
|
||||||
if s == v {
|
// Fast path - the pr contains "prefix.*"
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
}
|
if pr.isSuffixDotPlus {
|
||||||
return false
|
// Fast path - the pr contains "prefix.+"
|
||||||
|
return len(s) > 0
|
||||||
}
|
}
|
||||||
if pr.substrDotStar != "" {
|
if pr.substrDotStar != "" {
|
||||||
// Fast path - pr contains ".*someText.*"
|
// Fast path - pr contains ".*someText.*"
|
||||||
|
@ -94,45 +115,22 @@ func (pr *PromRegex) MatchString(s string) bool {
|
||||||
n := strings.Index(s, pr.substrDotPlus)
|
n := strings.Index(s, pr.substrDotPlus)
|
||||||
return n > 0 && n+len(pr.substrDotPlus) < len(s)
|
return n > 0 && n+len(pr.substrDotPlus) < len(s)
|
||||||
}
|
}
|
||||||
switch pr.suffix {
|
|
||||||
case ".*":
|
if len(pr.orValues) > 0 {
|
||||||
// Fast path - the pr contains "prefix.*"
|
// Fast path - pr contains only alternate strings such as 'foo|bar|baz'
|
||||||
|
for _, v := range pr.orValues {
|
||||||
|
if s == v {
|
||||||
return true
|
return true
|
||||||
case ".+":
|
|
||||||
// Fast path - the pr contains "prefix.+"
|
|
||||||
return len(s) > 0
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
// Fall back to slow path by matching the original regexp.
|
// Fall back to slow path by matching the original regexp.
|
||||||
return pr.reSuffixMatcher.Match(s)
|
return pr.reSuffixMatcher.Match(s)
|
||||||
}
|
}
|
||||||
|
|
||||||
// getSubstringLiteral returns regex part from expr surrounded by prefixSuffix.
|
// String returns string representation of pr.
|
||||||
//
|
func (pr *PromRegex) String() string {
|
||||||
// For example, if expr=".+foo.+" and prefixSuffix=".+", then the function returns "foo".
|
return pr.exprStr
|
||||||
//
|
|
||||||
// An empty string is returned if expr doesn't contain the given prefixSuffix prefix and suffix
|
|
||||||
// or if the regex part surrounded by prefixSuffix contains alternate regexps.
|
|
||||||
func getSubstringLiteral(expr, prefixSuffix string) string {
|
|
||||||
// Verify that the expr doesn't contain alternate regexps. In this case it is unsafe removing prefix and suffix.
|
|
||||||
sre, err := syntax.Parse(expr, syntax.Perl)
|
|
||||||
if err != nil {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
if sre.Op == syntax.OpAlternate {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
|
|
||||||
if !strings.HasPrefix(expr, prefixSuffix) {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
expr = expr[len(prefixSuffix):]
|
|
||||||
if !strings.HasSuffix(expr, prefixSuffix) {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
expr = expr[:len(expr)-len(prefixSuffix)]
|
|
||||||
prefix, suffix := Simplify(expr)
|
|
||||||
if suffix != "" {
|
|
||||||
return ""
|
|
||||||
}
|
|
||||||
return prefix
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,6 +8,7 @@ import (
|
||||||
func TestPromRegexParseFailure(t *testing.T) {
|
func TestPromRegexParseFailure(t *testing.T) {
|
||||||
f := func(expr string) {
|
f := func(expr string) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
|
|
||||||
pr, err := NewPromRegex(expr)
|
pr, err := NewPromRegex(expr)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatalf("expecting non-nil error for expr=%s", expr)
|
t.Fatalf("expecting non-nil error for expr=%s", expr)
|
||||||
|
@ -23,10 +24,15 @@ func TestPromRegexParseFailure(t *testing.T) {
|
||||||
func TestPromRegex(t *testing.T) {
|
func TestPromRegex(t *testing.T) {
|
||||||
f := func(expr, s string, resultExpected bool) {
|
f := func(expr, s string, resultExpected bool) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
|
|
||||||
pr, err := NewPromRegex(expr)
|
pr, err := NewPromRegex(expr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("unexpected error: %s", err)
|
t.Fatalf("unexpected error: %s", err)
|
||||||
}
|
}
|
||||||
|
exprResult := pr.String()
|
||||||
|
if exprResult != expr {
|
||||||
|
t.Fatalf("unexpected string representation for %q: %q", expr, exprResult)
|
||||||
|
}
|
||||||
result := pr.MatchString(s)
|
result := pr.MatchString(s)
|
||||||
if result != resultExpected {
|
if result != resultExpected {
|
||||||
t.Fatalf("unexpected result when matching %q against %q; got %v; want %v", expr, s, result, resultExpected)
|
t.Fatalf("unexpected result when matching %q against %q; got %v; want %v", expr, s, result, resultExpected)
|
||||||
|
@ -40,6 +46,7 @@ func TestPromRegex(t *testing.T) {
|
||||||
t.Fatalf("unexpected result when matching %q against %q during sanity check; got %v; want %v", exprAnchored, s, result, resultExpected)
|
t.Fatalf("unexpected result when matching %q against %q during sanity check; got %v; want %v", exprAnchored, s, result, resultExpected)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
f("", "", true)
|
f("", "", true)
|
||||||
f("", "foo", false)
|
f("", "foo", false)
|
||||||
f("foo", "", false)
|
f("foo", "", false)
|
||||||
|
@ -118,4 +125,8 @@ func TestPromRegex(t *testing.T) {
|
||||||
f(".*;|;.*", "foo;bar", false)
|
f(".*;|;.*", "foo;bar", false)
|
||||||
f(".*;|;.*", "foo;", true)
|
f(".*;|;.*", "foo;", true)
|
||||||
f(".*;|;.*", ";foo", true)
|
f(".*;|;.*", ";foo", true)
|
||||||
|
|
||||||
|
f(".*foo(bar|baz)", "fooxfoobaz", true)
|
||||||
|
f(".*foo(bar|baz)", "fooxfooban", false)
|
||||||
|
f(".*foo(bar|baz)", "fooxfooban foobar", true)
|
||||||
}
|
}
|
||||||
|
|
209
lib/regexutil/regex.go
Normal file
209
lib/regexutil/regex.go
Normal file
|
@ -0,0 +1,209 @@
|
||||||
|
package regexutil
|
||||||
|
|
||||||
|
import (
|
||||||
|
"regexp"
|
||||||
|
"regexp/syntax"
|
||||||
|
"strings"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Regex implements an optimized string matching for Go regex.
|
||||||
|
//
|
||||||
|
// The following regexs are optimized:
|
||||||
|
//
|
||||||
|
// - plain string such as "foobar"
|
||||||
|
// - alternate strings such as "foo|bar|baz"
|
||||||
|
// - prefix match such as "foo.*" or "foo.+"
|
||||||
|
// - substring match such as ".*foo.*" or ".+bar.+"
|
||||||
|
type Regex struct {
|
||||||
|
// exprStr is the original expression.
|
||||||
|
exprStr string
|
||||||
|
|
||||||
|
// prefix contains literal prefix for regex.
|
||||||
|
// For example, prefix="foo" for regex="foo(a|b)"
|
||||||
|
prefix string
|
||||||
|
|
||||||
|
// isOnlyPrefix is set to true if the regex contains only the prefix.
|
||||||
|
isOnlyPrefix bool
|
||||||
|
|
||||||
|
// isSuffixDotStar is set to true if suffix is ".*"
|
||||||
|
isSuffixDotStar bool
|
||||||
|
|
||||||
|
// isSuffixDotPlus is set to true if suffix is ".+"
|
||||||
|
isSuffixDotPlus bool
|
||||||
|
|
||||||
|
// substrDotStar contains literal string for regex suffix=".*string.*"
|
||||||
|
substrDotStar string
|
||||||
|
|
||||||
|
// substrDotPlus contains literal string for regex suffix=".+string.+"
|
||||||
|
substrDotPlus string
|
||||||
|
|
||||||
|
// orValues contains or values for the suffix regex.
|
||||||
|
// For example, orValues contain ["foo","bar","baz"] for regex suffix="foo|bar|baz"
|
||||||
|
orValues []string
|
||||||
|
|
||||||
|
// suffixRe is the regexp for suffix
|
||||||
|
suffixRe *regexp.Regexp
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewRegex returns Regex for the given expr.
|
||||||
|
func NewRegex(expr string) (*Regex, error) {
|
||||||
|
if _, err := regexp.Compile(expr); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
prefix, suffix := SimplifyRegex(expr)
|
||||||
|
sre := mustParseRegexp(suffix)
|
||||||
|
orValues := getOrValues(sre)
|
||||||
|
isOnlyPrefix := len(orValues) == 1 && orValues[0] == ""
|
||||||
|
isSuffixDotStar := isDotOp(sre, syntax.OpStar)
|
||||||
|
isSuffixDotPlus := isDotOp(sre, syntax.OpPlus)
|
||||||
|
substrDotStar := getSubstringLiteral(sre, syntax.OpStar)
|
||||||
|
substrDotPlus := getSubstringLiteral(sre, syntax.OpPlus)
|
||||||
|
|
||||||
|
suffixAnchored := suffix
|
||||||
|
if len(prefix) > 0 {
|
||||||
|
suffixAnchored = "^(?:" + suffix + ")"
|
||||||
|
}
|
||||||
|
// The suffixAnchored must be properly compiled, since it has been already checked above.
|
||||||
|
// Otherwise it is a bug, which must be fixed.
|
||||||
|
suffixRe := regexp.MustCompile(suffixAnchored)
|
||||||
|
|
||||||
|
r := &Regex{
|
||||||
|
exprStr: expr,
|
||||||
|
prefix: prefix,
|
||||||
|
isOnlyPrefix: isOnlyPrefix,
|
||||||
|
isSuffixDotStar: isSuffixDotStar,
|
||||||
|
isSuffixDotPlus: isSuffixDotPlus,
|
||||||
|
substrDotStar: substrDotStar,
|
||||||
|
substrDotPlus: substrDotPlus,
|
||||||
|
orValues: orValues,
|
||||||
|
suffixRe: suffixRe,
|
||||||
|
}
|
||||||
|
return r, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// MatchString returns true if s matches r.
|
||||||
|
func (r *Regex) MatchString(s string) bool {
|
||||||
|
if r.isOnlyPrefix {
|
||||||
|
return strings.Contains(s, r.prefix)
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(r.prefix) == 0 {
|
||||||
|
return r.matchStringNoPrefix(s)
|
||||||
|
}
|
||||||
|
return r.matchStringWithPrefix(s)
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetLiterals returns literals for r.
|
||||||
|
func (r *Regex) GetLiterals() []string {
|
||||||
|
sre := mustParseRegexp(r.exprStr)
|
||||||
|
for sre.Op == syntax.OpCapture {
|
||||||
|
sre = sre.Sub[0]
|
||||||
|
}
|
||||||
|
|
||||||
|
v, ok := getLiteral(sre)
|
||||||
|
if ok {
|
||||||
|
return []string{v}
|
||||||
|
}
|
||||||
|
|
||||||
|
if sre.Op != syntax.OpConcat {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
var a []string
|
||||||
|
for _, sub := range sre.Sub {
|
||||||
|
v, ok := getLiteral(sub)
|
||||||
|
if ok {
|
||||||
|
a = append(a, v)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return a
|
||||||
|
}
|
||||||
|
|
||||||
|
// String returns string represetnation for r
|
||||||
|
func (r *Regex) String() string {
|
||||||
|
return r.exprStr
|
||||||
|
}
|
||||||
|
|
||||||
|
func (r *Regex) matchStringNoPrefix(s string) bool {
|
||||||
|
if r.isSuffixDotStar {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
if r.isSuffixDotPlus {
|
||||||
|
return len(s) > 0
|
||||||
|
}
|
||||||
|
if r.substrDotStar != "" {
|
||||||
|
// Fast path - r contains ".*someText.*"
|
||||||
|
return strings.Contains(s, r.substrDotStar)
|
||||||
|
}
|
||||||
|
if r.substrDotPlus != "" {
|
||||||
|
// Fast path - r contains ".+someText.+"
|
||||||
|
n := strings.Index(s, r.substrDotPlus)
|
||||||
|
return n > 0 && n+len(r.substrDotPlus) < len(s)
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(r.orValues) == 0 {
|
||||||
|
// Fall back to slow path by matching the suffix regexp.
|
||||||
|
return r.suffixRe.MatchString(s)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Fast path - compare s to r.orValues
|
||||||
|
for _, v := range r.orValues {
|
||||||
|
if strings.Contains(s, v) {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (r *Regex) matchStringWithPrefix(s string) bool {
|
||||||
|
n := strings.Index(s, r.prefix)
|
||||||
|
if n < 0 {
|
||||||
|
// Fast path - s doesn't contain the needed prefix
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
sNext := s[n+1:]
|
||||||
|
s = s[n+len(r.prefix):]
|
||||||
|
|
||||||
|
if r.isSuffixDotStar {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
if r.isSuffixDotPlus {
|
||||||
|
return len(s) > 0
|
||||||
|
}
|
||||||
|
if r.substrDotStar != "" {
|
||||||
|
// Fast path - r contains ".*someText.*"
|
||||||
|
return strings.Contains(s, r.substrDotStar)
|
||||||
|
}
|
||||||
|
if r.substrDotPlus != "" {
|
||||||
|
// Fast path - r contains ".+someText.+"
|
||||||
|
n := strings.Index(s, r.substrDotPlus)
|
||||||
|
return n > 0 && n+len(r.substrDotPlus) < len(s)
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
if len(r.orValues) == 0 {
|
||||||
|
// Fall back to slow path by matching the suffix regexp.
|
||||||
|
if r.suffixRe.MatchString(s) {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Fast path - compare s to r.orValues
|
||||||
|
for _, v := range r.orValues {
|
||||||
|
if strings.HasPrefix(s, v) {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Mismatch. Try again starting from the next char.
|
||||||
|
s = sNext
|
||||||
|
n := strings.Index(s, r.prefix)
|
||||||
|
if n < 0 {
|
||||||
|
// Fast path - s doesn't contain the needed prefix
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
sNext = s[n+1:]
|
||||||
|
s = s[n+len(r.prefix):]
|
||||||
|
}
|
||||||
|
}
|
171
lib/regexutil/regex_test.go
Normal file
171
lib/regexutil/regex_test.go
Normal file
|
@ -0,0 +1,171 @@
|
||||||
|
package regexutil
|
||||||
|
|
||||||
|
import (
|
||||||
|
"reflect"
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestNewRegexFailure(t *testing.T) {
|
||||||
|
f := func(expr string) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
r, err := NewRegex(expr)
|
||||||
|
if err == nil {
|
||||||
|
t.Fatalf("expecting non-nil error when parsing %q; got %q", expr, r)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("[foo")
|
||||||
|
f("(foo")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestRegexMatchString(t *testing.T) {
|
||||||
|
f := func(expr, s string, resultExpected bool) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
r, err := NewRegex(expr)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("cannot parse %q: %s", expr, err)
|
||||||
|
}
|
||||||
|
exprResult := r.String()
|
||||||
|
if exprResult != expr {
|
||||||
|
t.Fatalf("unexpected string representation for %q: %q", expr, exprResult)
|
||||||
|
}
|
||||||
|
result := r.MatchString(s)
|
||||||
|
if result != resultExpected {
|
||||||
|
t.Fatalf("unexpected result when matching %q against regex=%q; got %v; want %v", s, expr, result, resultExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", "", true)
|
||||||
|
f("", "foo", true)
|
||||||
|
f("foo", "", false)
|
||||||
|
f(".*", "", true)
|
||||||
|
f(".*", "foo", true)
|
||||||
|
f(".+", "", false)
|
||||||
|
f(".+", "foo", true)
|
||||||
|
f("foo.*", "bar", false)
|
||||||
|
f("foo.*", "foo", true)
|
||||||
|
f("foo.*", "a foo", true)
|
||||||
|
f("foo.*", "a foo a", true)
|
||||||
|
f("foo.*", "foobar", true)
|
||||||
|
f("foo.*", "a foobar", true)
|
||||||
|
f("foo.+", "bar", false)
|
||||||
|
f("foo.+", "foo", false)
|
||||||
|
f("foo.+", "a foo", false)
|
||||||
|
f("foo.+", "foobar", true)
|
||||||
|
f("foo.+", "a foobar", true)
|
||||||
|
f("foo|bar", "", false)
|
||||||
|
f("foo|bar", "a", false)
|
||||||
|
f("foo|bar", "foo", true)
|
||||||
|
f("foo|bar", "a foo", true)
|
||||||
|
f("foo|bar", "foo a", true)
|
||||||
|
f("foo|bar", "a foo a", true)
|
||||||
|
f("foo|bar", "bar", true)
|
||||||
|
f("foo|bar", "foobar", true)
|
||||||
|
f("foo(bar|baz)", "a", false)
|
||||||
|
f("foo(bar|baz)", "foobar", true)
|
||||||
|
f("foo(bar|baz)", "foobaz", true)
|
||||||
|
f("foo(bar|baz)", "foobaza", true)
|
||||||
|
f("foo(bar|baz)", "a foobaz a", true)
|
||||||
|
f("foo(bar|baz)", "foobal", false)
|
||||||
|
f("^foo|b(ar)$", "foo", true)
|
||||||
|
f("^foo|b(ar)$", "foo a", true)
|
||||||
|
f("^foo|b(ar)$", "a foo", false)
|
||||||
|
f("^foo|b(ar)$", "bar", true)
|
||||||
|
f("^foo|b(ar)$", "a bar", true)
|
||||||
|
f("^foo|b(ar)$", "barz", false)
|
||||||
|
f("^foo|b(ar)$", "ar", false)
|
||||||
|
f(".*foo.*", "foo", true)
|
||||||
|
f(".*foo.*", "afoobar", true)
|
||||||
|
f(".*foo.*", "abc", false)
|
||||||
|
f("foo.*bar.*", "foobar", true)
|
||||||
|
f("foo.*bar.*", "foo_bar_", true)
|
||||||
|
f("foo.*bar.*", "a foo bar baz", true)
|
||||||
|
f("foo.*bar.*", "foobaz", false)
|
||||||
|
f("foo.*bar.*", "baz foo", false)
|
||||||
|
f(".+foo.+", "foo", false)
|
||||||
|
f(".+foo.+", "afoobar", true)
|
||||||
|
f(".+foo.+", "afoo", false)
|
||||||
|
f(".+foo.+", "abc", false)
|
||||||
|
f("foo.+bar.+", "foobar", false)
|
||||||
|
f("foo.+bar.+", "foo_bar_", true)
|
||||||
|
f("foo.+bar.+", "a foo_bar_", true)
|
||||||
|
f("foo.+bar.+", "foobaz", false)
|
||||||
|
f("foo.+bar.+", "abc", false)
|
||||||
|
f(".+foo.*", "foo", false)
|
||||||
|
f(".+foo.*", "afoo", true)
|
||||||
|
f(".+foo.*", "afoobar", true)
|
||||||
|
f(".*(a|b).*", "a", true)
|
||||||
|
f(".*(a|b).*", "ax", true)
|
||||||
|
f(".*(a|b).*", "xa", true)
|
||||||
|
f(".*(a|b).*", "xay", true)
|
||||||
|
f(".*(a|b).*", "xzy", false)
|
||||||
|
f("^(?:true)$", "true", true)
|
||||||
|
f("^(?:true)$", "false", false)
|
||||||
|
|
||||||
|
f(".+;|;.+", ";", false)
|
||||||
|
f(".+;|;.+", "foo", false)
|
||||||
|
f(".+;|;.+", "foo;bar", true)
|
||||||
|
f(".+;|;.+", "foo;", true)
|
||||||
|
f(".+;|;.+", ";foo", true)
|
||||||
|
f(".+foo|bar|baz.+", "foo", false)
|
||||||
|
f(".+foo|bar|baz.+", "afoo", true)
|
||||||
|
f(".+foo|bar|baz.+", "fooa", false)
|
||||||
|
f(".+foo|bar|baz.+", "afooa", true)
|
||||||
|
f(".+foo|bar|baz.+", "bar", true)
|
||||||
|
f(".+foo|bar|baz.+", "abar", true)
|
||||||
|
f(".+foo|bar|baz.+", "abara", true)
|
||||||
|
f(".+foo|bar|baz.+", "bara", true)
|
||||||
|
f(".+foo|bar|baz.+", "baz", false)
|
||||||
|
f(".+foo|bar|baz.+", "baza", true)
|
||||||
|
f(".+foo|bar|baz.+", "abaz", false)
|
||||||
|
f(".+foo|bar|baz.+", "abaza", true)
|
||||||
|
f(".+foo|bar|baz.+", "afoo|bar|baza", true)
|
||||||
|
f(".+(foo|bar|baz).+", "bar", false)
|
||||||
|
f(".+(foo|bar|baz).+", "bara", false)
|
||||||
|
f(".+(foo|bar|baz).+", "abar", false)
|
||||||
|
f(".+(foo|bar|baz).+", "abara", true)
|
||||||
|
f(".+(foo|bar|baz).+", "afooa", true)
|
||||||
|
f(".+(foo|bar|baz).+", "abaza", true)
|
||||||
|
|
||||||
|
f(".*;|;.*", ";", true)
|
||||||
|
f(".*;|;.*", "foo", false)
|
||||||
|
f(".*;|;.*", "foo;bar", true)
|
||||||
|
f(".*;|;.*", "foo;", true)
|
||||||
|
f(".*;|;.*", ";foo", true)
|
||||||
|
|
||||||
|
f("^bar", "foobarbaz", false)
|
||||||
|
f("^foo", "foobarbaz", true)
|
||||||
|
f("bar$", "foobarbaz", false)
|
||||||
|
f("baz$", "foobarbaz", true)
|
||||||
|
f("(bar$|^foo)", "foobarbaz", true)
|
||||||
|
f("(bar$^boo)", "foobarbaz", false)
|
||||||
|
f("foo(bar|baz)", "a fooxfoobaz a", true)
|
||||||
|
f("foo(bar|baz)", "a fooxfooban a", false)
|
||||||
|
f("foo(bar|baz)", "a fooxfooban foobar a", true)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetLiterals(t *testing.T) {
|
||||||
|
f := func(expr string, literalsExpected []string) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
r, err := NewRegex(expr)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("cannot parse %q: %s", expr, err)
|
||||||
|
}
|
||||||
|
literals := r.GetLiterals()
|
||||||
|
if !reflect.DeepEqual(literals, literalsExpected) {
|
||||||
|
t.Fatalf("unexpected literals; got %q; want %q", literals, literalsExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", nil)
|
||||||
|
f("foo bar baz", []string{"foo bar baz"})
|
||||||
|
f("foo.*bar(a|b)baz.+", []string{"foo", "bar", "baz"})
|
||||||
|
f("(foo[ab](?:bar))", []string{"foo", "bar"})
|
||||||
|
f("foo|bar", nil)
|
||||||
|
f("((foo|bar)baz xxx(?:yzabc))", []string{"baz xxxyzabc"})
|
||||||
|
f("((foo|bar)baz xxx(?:yzabc)*)", []string{"baz xxx"})
|
||||||
|
f("((foo|bar)baz? xxx(?:yzabc)*)", []string{"ba", " xxx"})
|
||||||
|
}
|
111
lib/regexutil/regex_timing_test.go
Normal file
111
lib/regexutil/regex_timing_test.go
Normal file
|
@ -0,0 +1,111 @@
|
||||||
|
package regexutil
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"regexp"
|
||||||
|
"testing"
|
||||||
|
)
|
||||||
|
|
||||||
|
func BenchmarkRegexMatchString(b *testing.B) {
|
||||||
|
b.Run("unpotimized-noprefix-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "xbar.*|baz", "axbarz", true)
|
||||||
|
})
|
||||||
|
b.Run("unpotimized-noprefix-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "xbar.*|baz", "zfoobaxz", false)
|
||||||
|
})
|
||||||
|
b.Run("unpotimized-prefix-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo(bar.*|baz)", "afoobarz", true)
|
||||||
|
})
|
||||||
|
b.Run("unpotimized-prefix-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo(bar.*|baz)", "zfoobaxz", false)
|
||||||
|
})
|
||||||
|
b.Run("dot-star-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".*", "foo", true)
|
||||||
|
})
|
||||||
|
b.Run("dot-plus-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".+", "foo", true)
|
||||||
|
})
|
||||||
|
b.Run("dot-plus-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".+", "", false)
|
||||||
|
})
|
||||||
|
b.Run("literal-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo", "afoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("literal-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo", "abaraa", false)
|
||||||
|
})
|
||||||
|
b.Run("prefix-dot-star-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo.*", "afoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("prefix-dot-star-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo.*", "axoobar", false)
|
||||||
|
})
|
||||||
|
b.Run("prefix-dot-plus-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo.+", "afoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("prefix-dot-plus-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo.+", "axoobar", false)
|
||||||
|
})
|
||||||
|
b.Run("or-values-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo|bar|baz", "abaz", true)
|
||||||
|
})
|
||||||
|
b.Run("or-values-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "foo|bar|baz", "axaz", false)
|
||||||
|
})
|
||||||
|
b.Run("prefix-or-values-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "x(foo|bar|baz)", "axbaz", true)
|
||||||
|
})
|
||||||
|
b.Run("prefix-or-values-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "x(foo|bar|baz)", "aabaz", false)
|
||||||
|
})
|
||||||
|
b.Run("substring-dot-star-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".*foo.*", "afoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("substring-dot-star-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".*foo.*", "abarbaz", false)
|
||||||
|
})
|
||||||
|
b.Run("substring-dot-plus-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".+foo.+", "afoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("substring-dot-plus-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, ".+foo.+", "abarbaz", false)
|
||||||
|
})
|
||||||
|
b.Run("prefix-substring-dot-star-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "a.*foo.*", "bafoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("prefix-substring-dot-star-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "a.*foo.*", "babarbaz", false)
|
||||||
|
})
|
||||||
|
b.Run("prefix-substring-dot-plus-match", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "a.+foo.+", "babfoobar", true)
|
||||||
|
})
|
||||||
|
b.Run("prefix-substring-dot-plus-mismatch", func(b *testing.B) {
|
||||||
|
benchmarkRegexMatchString(b, "a.+foo.+", "babarbaz", false)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
func benchmarkRegexMatchString(b *testing.B, expr, s string, resultExpected bool) {
|
||||||
|
r, err := NewRegex(expr)
|
||||||
|
if err != nil {
|
||||||
|
panic(fmt.Errorf("unexpected error: %w", err))
|
||||||
|
}
|
||||||
|
re := regexp.MustCompile(expr)
|
||||||
|
f := func(b *testing.B, matchString func(s string) bool) {
|
||||||
|
b.SetBytes(1)
|
||||||
|
b.ReportAllocs()
|
||||||
|
b.RunParallel(func(pb *testing.PB) {
|
||||||
|
for pb.Next() {
|
||||||
|
result := matchString(s)
|
||||||
|
if result != resultExpected {
|
||||||
|
panic(fmt.Errorf("unexpected result when matching %s against %s; got %v; want %v", s, expr, result, resultExpected))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
b.Run("Regex", func(b *testing.B) {
|
||||||
|
f(b, r.MatchString)
|
||||||
|
})
|
||||||
|
b.Run("StandardRegex", func(b *testing.B) {
|
||||||
|
f(b, re.MatchString)
|
||||||
|
})
|
||||||
|
}
|
|
@ -18,24 +18,38 @@ func RemoveStartEndAnchors(expr string) string {
|
||||||
return expr
|
return expr
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetOrValues returns "or" values from the given regexp expr.
|
// GetOrValuesRegex returns "or" values from the given regexp expr.
|
||||||
|
//
|
||||||
|
// It returns ["foo", "bar"] for "foo|bar" regexp.
|
||||||
|
// It returns ["foo"] for "foo" regexp.
|
||||||
|
// It returns [""] for "" regexp.
|
||||||
|
// It returns an empty list if it is impossible to extract "or" values from the regexp.
|
||||||
|
func GetOrValuesRegex(expr string) []string {
|
||||||
|
return getOrValuesRegex(expr, true)
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetOrValuesPromRegex returns "or" values from the given Prometheus-like regexp expr.
|
||||||
//
|
//
|
||||||
// It ignores start and end anchors ('^') and ('$') at the start and the end of expr.
|
// It ignores start and end anchors ('^') and ('$') at the start and the end of expr.
|
||||||
// It returns ["foo", "bar"] for "foo|bar" regexp.
|
// It returns ["foo", "bar"] for "foo|bar" regexp.
|
||||||
// It returns ["foo"] for "foo" regexp.
|
// It returns ["foo"] for "foo" regexp.
|
||||||
// It returns [""] for "" regexp.
|
// It returns [""] for "" regexp.
|
||||||
// It returns an empty list if it is impossible to extract "or" values from the regexp.
|
// It returns an empty list if it is impossible to extract "or" values from the regexp.
|
||||||
func GetOrValues(expr string) []string {
|
func GetOrValuesPromRegex(expr string) []string {
|
||||||
expr = RemoveStartEndAnchors(expr)
|
expr = RemoveStartEndAnchors(expr)
|
||||||
prefix, tailExpr := Simplify(expr)
|
return getOrValuesRegex(expr, false)
|
||||||
|
}
|
||||||
|
|
||||||
|
func getOrValuesRegex(expr string, keepAnchors bool) []string {
|
||||||
|
prefix, tailExpr := simplifyRegex(expr, keepAnchors)
|
||||||
if tailExpr == "" {
|
if tailExpr == "" {
|
||||||
return []string{prefix}
|
return []string{prefix}
|
||||||
}
|
}
|
||||||
sre, err := syntax.Parse(tailExpr, syntax.Perl)
|
sre, err := parseRegexp(tailExpr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(fmt.Errorf("BUG: unexpected error when parsing verified tailExpr=%q: %w", tailExpr, err))
|
return nil
|
||||||
}
|
}
|
||||||
orValues := getOrValuesExt(sre)
|
orValues := getOrValues(sre)
|
||||||
|
|
||||||
// Sort orValues for faster index seek later
|
// Sort orValues for faster index seek later
|
||||||
sort.Strings(orValues)
|
sort.Strings(orValues)
|
||||||
|
@ -50,21 +64,22 @@ func GetOrValues(expr string) []string {
|
||||||
return orValues
|
return orValues
|
||||||
}
|
}
|
||||||
|
|
||||||
func getOrValuesExt(sre *syntax.Regexp) []string {
|
func getOrValues(sre *syntax.Regexp) []string {
|
||||||
switch sre.Op {
|
switch sre.Op {
|
||||||
case syntax.OpCapture:
|
case syntax.OpCapture:
|
||||||
return getOrValuesExt(sre.Sub[0])
|
return getOrValues(sre.Sub[0])
|
||||||
case syntax.OpLiteral:
|
case syntax.OpLiteral:
|
||||||
if !isLiteral(sre) {
|
v, ok := getLiteral(sre)
|
||||||
|
if !ok {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
return []string{string(sre.Rune)}
|
return []string{v}
|
||||||
case syntax.OpEmptyMatch:
|
case syntax.OpEmptyMatch:
|
||||||
return []string{""}
|
return []string{""}
|
||||||
case syntax.OpAlternate:
|
case syntax.OpAlternate:
|
||||||
a := make([]string, 0, len(sre.Sub))
|
a := make([]string, 0, len(sre.Sub))
|
||||||
for _, reSub := range sre.Sub {
|
for _, reSub := range sre.Sub {
|
||||||
ca := getOrValuesExt(reSub)
|
ca := getOrValues(reSub)
|
||||||
if len(ca) == 0 {
|
if len(ca) == 0 {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -94,7 +109,7 @@ func getOrValuesExt(sre *syntax.Regexp) []string {
|
||||||
if len(sre.Sub) < 1 {
|
if len(sre.Sub) < 1 {
|
||||||
return []string{""}
|
return []string{""}
|
||||||
}
|
}
|
||||||
prefixes := getOrValuesExt(sre.Sub[0])
|
prefixes := getOrValues(sre.Sub[0])
|
||||||
if len(prefixes) == 0 {
|
if len(prefixes) == 0 {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -102,7 +117,7 @@ func getOrValuesExt(sre *syntax.Regexp) []string {
|
||||||
return prefixes
|
return prefixes
|
||||||
}
|
}
|
||||||
sre.Sub = sre.Sub[1:]
|
sre.Sub = sre.Sub[1:]
|
||||||
suffixes := getOrValuesExt(sre)
|
suffixes := getOrValues(sre)
|
||||||
if len(suffixes) == 0 {
|
if len(suffixes) == 0 {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -123,46 +138,87 @@ func getOrValuesExt(sre *syntax.Regexp) []string {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func isLiteral(sre *syntax.Regexp) bool {
|
func getLiteral(sre *syntax.Regexp) (string, bool) {
|
||||||
if sre.Op == syntax.OpCapture {
|
if sre.Op == syntax.OpCapture {
|
||||||
return isLiteral(sre.Sub[0])
|
return getLiteral(sre.Sub[0])
|
||||||
}
|
}
|
||||||
return sre.Op == syntax.OpLiteral && sre.Flags&syntax.FoldCase == 0
|
if sre.Op == syntax.OpLiteral && sre.Flags&syntax.FoldCase == 0 {
|
||||||
|
return string(sre.Rune), true
|
||||||
|
}
|
||||||
|
return "", false
|
||||||
}
|
}
|
||||||
|
|
||||||
const maxOrValues = 100
|
const maxOrValues = 100
|
||||||
|
|
||||||
// Simplify simplifies the given expr.
|
// SimplifyRegex simplifies the given regexp expr.
|
||||||
|
//
|
||||||
|
// It returns plaintext pefix and the remaining regular expression
|
||||||
|
// without capturing parens.
|
||||||
|
func SimplifyRegex(expr string) (string, string) {
|
||||||
|
prefix, suffix := simplifyRegex(expr, true)
|
||||||
|
sre := mustParseRegexp(suffix)
|
||||||
|
|
||||||
|
if isDotOp(sre, syntax.OpStar) {
|
||||||
|
return prefix, ""
|
||||||
|
}
|
||||||
|
if sre.Op == syntax.OpConcat {
|
||||||
|
subs := sre.Sub
|
||||||
|
if prefix == "" {
|
||||||
|
// Drop .* at the start
|
||||||
|
for len(subs) > 0 && isDotOp(subs[0], syntax.OpStar) {
|
||||||
|
subs = subs[1:]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Drop .* at the end.
|
||||||
|
for len(subs) > 0 && isDotOp(subs[len(subs)-1], syntax.OpStar) {
|
||||||
|
subs = subs[:len(subs)-1]
|
||||||
|
}
|
||||||
|
|
||||||
|
sre.Sub = subs
|
||||||
|
if len(subs) == 0 {
|
||||||
|
return prefix, ""
|
||||||
|
}
|
||||||
|
suffix = sre.String()
|
||||||
|
}
|
||||||
|
return prefix, suffix
|
||||||
|
}
|
||||||
|
|
||||||
|
// SimplifyPromRegex simplifies the given Prometheus-like expr.
|
||||||
//
|
//
|
||||||
// It returns plaintext prefix and the remaining regular expression
|
// It returns plaintext prefix and the remaining regular expression
|
||||||
// with dropped '^' and '$' anchors at the beginning and the end
|
// with dropped '^' and '$' anchors at the beginning and at the end
|
||||||
// of the regular expression.
|
// of the regular expression.
|
||||||
//
|
//
|
||||||
// The function removes capturing parens from the expr,
|
// The function removes capturing parens from the expr,
|
||||||
// so it cannot be used when capturing parens are necessary.
|
// so it cannot be used when capturing parens are necessary.
|
||||||
func Simplify(expr string) (string, string) {
|
func SimplifyPromRegex(expr string) (string, string) {
|
||||||
sre, err := syntax.Parse(expr, syntax.Perl)
|
return simplifyRegex(expr, false)
|
||||||
|
}
|
||||||
|
|
||||||
|
func simplifyRegex(expr string, keepAnchors bool) (string, string) {
|
||||||
|
sre, err := parseRegexp(expr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// Cannot parse the regexp. Return it all as prefix.
|
// Cannot parse the regexp. Return it all as prefix.
|
||||||
return expr, ""
|
return expr, ""
|
||||||
}
|
}
|
||||||
sre = simplifyRegexp(sre, false)
|
sre = simplifyRegexp(sre, keepAnchors, keepAnchors)
|
||||||
if sre == emptyRegexp {
|
if sre == emptyRegexp {
|
||||||
return "", ""
|
return "", ""
|
||||||
}
|
}
|
||||||
if isLiteral(sre) {
|
v, ok := getLiteral(sre)
|
||||||
return string(sre.Rune), ""
|
if ok {
|
||||||
|
return v, ""
|
||||||
}
|
}
|
||||||
var prefix string
|
var prefix string
|
||||||
if sre.Op == syntax.OpConcat {
|
if sre.Op == syntax.OpConcat {
|
||||||
sub0 := sre.Sub[0]
|
prefix, ok = getLiteral(sre.Sub[0])
|
||||||
if isLiteral(sub0) {
|
if ok {
|
||||||
prefix = string(sub0.Rune)
|
|
||||||
sre.Sub = sre.Sub[1:]
|
sre.Sub = sre.Sub[1:]
|
||||||
if len(sre.Sub) == 0 {
|
if len(sre.Sub) == 0 {
|
||||||
return prefix, ""
|
return prefix, ""
|
||||||
}
|
}
|
||||||
sre = simplifyRegexp(sre, true)
|
sre = simplifyRegexp(sre, true, keepAnchors)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if _, err := syntax.Compile(sre); err != nil {
|
if _, err := syntax.Compile(sre); err != nil {
|
||||||
|
@ -171,44 +227,42 @@ func Simplify(expr string) (string, string) {
|
||||||
}
|
}
|
||||||
s := sre.String()
|
s := sre.String()
|
||||||
s = strings.ReplaceAll(s, "(?:)", "")
|
s = strings.ReplaceAll(s, "(?:)", "")
|
||||||
s = strings.ReplaceAll(s, "(?-s:.)", ".")
|
s = strings.ReplaceAll(s, "(?s:.)", ".")
|
||||||
s = strings.ReplaceAll(s, "(?-m:$)", "$")
|
s = strings.ReplaceAll(s, "(?m:$)", "$")
|
||||||
return prefix, s
|
return prefix, s
|
||||||
}
|
}
|
||||||
|
|
||||||
func simplifyRegexp(sre *syntax.Regexp, hasPrefix bool) *syntax.Regexp {
|
func simplifyRegexp(sre *syntax.Regexp, keepBeginOp, keepEndOp bool) *syntax.Regexp {
|
||||||
s := sre.String()
|
s := sre.String()
|
||||||
for {
|
for {
|
||||||
sre = simplifyRegexpExt(sre, hasPrefix, false)
|
sre = simplifyRegexpExt(sre, keepBeginOp, keepEndOp)
|
||||||
sre = sre.Simplify()
|
sre = sre.Simplify()
|
||||||
if sre.Op == syntax.OpBeginText || sre.Op == syntax.OpEndText {
|
if !keepBeginOp && sre.Op == syntax.OpBeginText {
|
||||||
|
sre = emptyRegexp
|
||||||
|
} else if !keepEndOp && sre.Op == syntax.OpEndText {
|
||||||
sre = emptyRegexp
|
sre = emptyRegexp
|
||||||
}
|
}
|
||||||
sNew := sre.String()
|
sNew := sre.String()
|
||||||
if sNew == s {
|
if sNew == s {
|
||||||
return sre
|
return sre
|
||||||
}
|
}
|
||||||
var err error
|
sre = mustParseRegexp(sNew)
|
||||||
sre, err = syntax.Parse(sNew, syntax.Perl)
|
|
||||||
if err != nil {
|
|
||||||
panic(fmt.Errorf("BUG: cannot parse simplified regexp %q: %w", sNew, err))
|
|
||||||
}
|
|
||||||
s = sNew
|
s = sNew
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func simplifyRegexpExt(sre *syntax.Regexp, hasPrefix, hasSuffix bool) *syntax.Regexp {
|
func simplifyRegexpExt(sre *syntax.Regexp, keepBeginOp, keepEndOp bool) *syntax.Regexp {
|
||||||
switch sre.Op {
|
switch sre.Op {
|
||||||
case syntax.OpCapture:
|
case syntax.OpCapture:
|
||||||
// Substitute all the capture regexps with non-capture regexps.
|
// Substitute all the capture regexps with non-capture regexps.
|
||||||
sre.Op = syntax.OpAlternate
|
sre.Op = syntax.OpAlternate
|
||||||
sre.Sub[0] = simplifyRegexpExt(sre.Sub[0], hasPrefix, hasSuffix)
|
sre.Sub[0] = simplifyRegexpExt(sre.Sub[0], keepBeginOp, keepEndOp)
|
||||||
if sre.Sub[0] == emptyRegexp {
|
if sre.Sub[0] == emptyRegexp {
|
||||||
return emptyRegexp
|
return emptyRegexp
|
||||||
}
|
}
|
||||||
return sre
|
return sre
|
||||||
case syntax.OpStar, syntax.OpPlus, syntax.OpQuest, syntax.OpRepeat:
|
case syntax.OpStar, syntax.OpPlus, syntax.OpQuest, syntax.OpRepeat:
|
||||||
sre.Sub[0] = simplifyRegexpExt(sre.Sub[0], hasPrefix, hasSuffix)
|
sre.Sub[0] = simplifyRegexpExt(sre.Sub[0], keepBeginOp, keepEndOp)
|
||||||
if sre.Sub[0] == emptyRegexp {
|
if sre.Sub[0] == emptyRegexp {
|
||||||
return emptyRegexp
|
return emptyRegexp
|
||||||
}
|
}
|
||||||
|
@ -216,13 +270,13 @@ func simplifyRegexpExt(sre *syntax.Regexp, hasPrefix, hasSuffix bool) *syntax.Re
|
||||||
case syntax.OpAlternate:
|
case syntax.OpAlternate:
|
||||||
// Do not remove empty captures from OpAlternate, since this may break regexp.
|
// Do not remove empty captures from OpAlternate, since this may break regexp.
|
||||||
for i, sub := range sre.Sub {
|
for i, sub := range sre.Sub {
|
||||||
sre.Sub[i] = simplifyRegexpExt(sub, hasPrefix, hasSuffix)
|
sre.Sub[i] = simplifyRegexpExt(sub, keepBeginOp, keepEndOp)
|
||||||
}
|
}
|
||||||
return sre
|
return sre
|
||||||
case syntax.OpConcat:
|
case syntax.OpConcat:
|
||||||
subs := sre.Sub[:0]
|
subs := sre.Sub[:0]
|
||||||
for i, sub := range sre.Sub {
|
for i, sub := range sre.Sub {
|
||||||
sub = simplifyRegexpExt(sub, hasPrefix || len(subs) > 0, hasSuffix || i+1 < len(sre.Sub))
|
sub = simplifyRegexpExt(sub, keepBeginOp || len(subs) > 0, keepEndOp || i+1 < len(sre.Sub))
|
||||||
if sub != emptyRegexp {
|
if sub != emptyRegexp {
|
||||||
subs = append(subs, sub)
|
subs = append(subs, sub)
|
||||||
}
|
}
|
||||||
|
@ -230,12 +284,12 @@ func simplifyRegexpExt(sre *syntax.Regexp, hasPrefix, hasSuffix bool) *syntax.Re
|
||||||
sre.Sub = subs
|
sre.Sub = subs
|
||||||
// Remove anchros from the beginning and the end of regexp, since they
|
// Remove anchros from the beginning and the end of regexp, since they
|
||||||
// will be added later.
|
// will be added later.
|
||||||
if !hasPrefix {
|
if !keepBeginOp {
|
||||||
for len(sre.Sub) > 0 && sre.Sub[0].Op == syntax.OpBeginText {
|
for len(sre.Sub) > 0 && sre.Sub[0].Op == syntax.OpBeginText {
|
||||||
sre.Sub = sre.Sub[1:]
|
sre.Sub = sre.Sub[1:]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !hasSuffix {
|
if !keepEndOp {
|
||||||
for len(sre.Sub) > 0 && sre.Sub[len(sre.Sub)-1].Op == syntax.OpEndText {
|
for len(sre.Sub) > 0 && sre.Sub[len(sre.Sub)-1].Op == syntax.OpEndText {
|
||||||
sre.Sub = sre.Sub[:len(sre.Sub)-1]
|
sre.Sub = sre.Sub[:len(sre.Sub)-1]
|
||||||
}
|
}
|
||||||
|
@ -254,6 +308,44 @@ func simplifyRegexpExt(sre *syntax.Regexp, hasPrefix, hasSuffix bool) *syntax.Re
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// getSubstringLiteral returns regex part from sre surrounded by .+ or .* depending on the prefixSuffixOp.
|
||||||
|
//
|
||||||
|
// For example, if sre=".+foo.+" and prefixSuffix=syntax.OpPlus, then the function returns "foo".
|
||||||
|
//
|
||||||
|
// An empty string is returned if sre doesn't contain the given prefixSuffixOp prefix and suffix.
|
||||||
|
func getSubstringLiteral(sre *syntax.Regexp, prefixSuffixOp syntax.Op) string {
|
||||||
|
if sre.Op != syntax.OpConcat || len(sre.Sub) != 3 {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
if !isDotOp(sre.Sub[0], prefixSuffixOp) || !isDotOp(sre.Sub[2], prefixSuffixOp) {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
v, ok := getLiteral(sre.Sub[1])
|
||||||
|
if !ok {
|
||||||
|
return ""
|
||||||
|
}
|
||||||
|
return v
|
||||||
|
}
|
||||||
|
|
||||||
|
func isDotOp(sre *syntax.Regexp, op syntax.Op) bool {
|
||||||
|
if sre.Op != op {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return sre.Sub[0].Op == syntax.OpAnyChar
|
||||||
|
}
|
||||||
|
|
||||||
var emptyRegexp = &syntax.Regexp{
|
var emptyRegexp = &syntax.Regexp{
|
||||||
Op: syntax.OpEmptyMatch,
|
Op: syntax.OpEmptyMatch,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func parseRegexp(expr string) (*syntax.Regexp, error) {
|
||||||
|
return syntax.Parse(expr, syntax.Perl|syntax.DotNL)
|
||||||
|
}
|
||||||
|
|
||||||
|
func mustParseRegexp(expr string) *syntax.Regexp {
|
||||||
|
sre, err := parseRegexp(expr)
|
||||||
|
if err != nil {
|
||||||
|
panic(fmt.Errorf("BUG: cannot parse already verified regexp %q: %w", expr, err))
|
||||||
|
}
|
||||||
|
return sre
|
||||||
|
}
|
||||||
|
|
|
@ -5,10 +5,51 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestGetOrValues(t *testing.T) {
|
func TestGetOrValuesRegex(t *testing.T) {
|
||||||
f := func(s string, valuesExpected []string) {
|
f := func(s string, valuesExpected []string) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
values := GetOrValues(s)
|
values := GetOrValuesRegex(s)
|
||||||
|
if !reflect.DeepEqual(values, valuesExpected) {
|
||||||
|
t.Fatalf("unexpected values for s=%q; got %q; want %q", s, values, valuesExpected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", []string{""})
|
||||||
|
f("foo", []string{"foo"})
|
||||||
|
f("^foo$", nil)
|
||||||
|
f("|foo", []string{"", "foo"})
|
||||||
|
f("|foo|", []string{"", "", "foo"})
|
||||||
|
f("foo.+", nil)
|
||||||
|
f("foo.*", nil)
|
||||||
|
f(".*", nil)
|
||||||
|
f("foo|.*", nil)
|
||||||
|
f("(fo((o)))|(bar)", []string{"bar", "foo"})
|
||||||
|
f("foobar", []string{"foobar"})
|
||||||
|
f("z|x|c", []string{"c", "x", "z"})
|
||||||
|
f("foo|bar", []string{"bar", "foo"})
|
||||||
|
f("(foo|bar)", []string{"bar", "foo"})
|
||||||
|
f("(foo|bar)baz", []string{"barbaz", "foobaz"})
|
||||||
|
f("[a-z][a-z]", nil)
|
||||||
|
f("[a-d]", []string{"a", "b", "c", "d"})
|
||||||
|
f("x[a-d]we", []string{"xawe", "xbwe", "xcwe", "xdwe"})
|
||||||
|
f("foo(bar|baz)", []string{"foobar", "foobaz"})
|
||||||
|
f("foo(ba[rz]|(xx|o))", []string{"foobar", "foobaz", "fooo", "fooxx"})
|
||||||
|
f("foo(?:bar|baz)x(qwe|rt)", []string{"foobarxqwe", "foobarxrt", "foobazxqwe", "foobazxrt"})
|
||||||
|
f("foo(bar||baz)", []string{"foo", "foobar", "foobaz"})
|
||||||
|
f("(a|b|c)(d|e|f|0|1|2)(g|h|k|x|y|z)", nil)
|
||||||
|
f("(?i)foo", nil)
|
||||||
|
f("(?i)(foo|bar)", nil)
|
||||||
|
f("^foo|bar$", nil)
|
||||||
|
f("^(foo|bar)$", nil)
|
||||||
|
f("^a(foo|b(?:a|r))$", nil)
|
||||||
|
f("^a(foo$|b(?:a$|r))$", nil)
|
||||||
|
f("^a(^foo|bar$)z$", nil)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestGetOrValuesPromRegex(t *testing.T) {
|
||||||
|
f := func(s string, valuesExpected []string) {
|
||||||
|
t.Helper()
|
||||||
|
values := GetOrValuesPromRegex(s)
|
||||||
if !reflect.DeepEqual(values, valuesExpected) {
|
if !reflect.DeepEqual(values, valuesExpected) {
|
||||||
t.Fatalf("unexpected values for s=%q; got %q; want %q", s, values, valuesExpected)
|
t.Fatalf("unexpected values for s=%q; got %q; want %q", s, values, valuesExpected)
|
||||||
}
|
}
|
||||||
|
@ -46,10 +87,87 @@ func TestGetOrValues(t *testing.T) {
|
||||||
f("^a(^foo|bar$)z$", nil)
|
f("^a(^foo|bar$)z$", nil)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestSimplify(t *testing.T) {
|
func TestSimplifyRegex(t *testing.T) {
|
||||||
f := func(s, expectedPrefix, expectedSuffix string) {
|
f := func(s, expectedPrefix, expectedSuffix string) {
|
||||||
t.Helper()
|
t.Helper()
|
||||||
prefix, suffix := Simplify(s)
|
prefix, suffix := SimplifyRegex(s)
|
||||||
|
if prefix != expectedPrefix {
|
||||||
|
t.Fatalf("unexpected prefix for s=%q; got %q; want %q", s, prefix, expectedPrefix)
|
||||||
|
}
|
||||||
|
if suffix != expectedSuffix {
|
||||||
|
t.Fatalf("unexpected suffix for s=%q; got %q; want %q", s, suffix, expectedSuffix)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
f("", "", "")
|
||||||
|
f(".*", "", "")
|
||||||
|
f(".*(.*).*", "", "")
|
||||||
|
f("foo.*", "foo", "")
|
||||||
|
f(".*foo.*", "", "foo")
|
||||||
|
f("^", "", "\\A")
|
||||||
|
f("$", "", "(?-m:$)")
|
||||||
|
f("^()$", "", "(?-m:\\A$)")
|
||||||
|
f("^(?:)$", "", "(?-m:\\A$)")
|
||||||
|
f("^foo|^bar$|baz", "", "(?-m:\\Afoo|\\Abar$|baz)")
|
||||||
|
f("^(foo$|^bar)$", "", "(?-m:\\A(?:foo$|\\Abar)$)")
|
||||||
|
f("^a(foo$|bar)$", "", "(?-m:\\Aa(?:foo$|bar)$)")
|
||||||
|
f("^a(^foo|bar$)z$", "", "(?-m:\\Aa(?:\\Afoo|bar$)z$)")
|
||||||
|
f("foobar", "foobar", "")
|
||||||
|
f("foo$|^foobar", "", "(?-m:foo$|\\Afoobar)")
|
||||||
|
f("^(foo$|^foobar)$", "", "(?-m:\\A(?:foo$|\\Afoobar)$)")
|
||||||
|
f("foobar|foobaz", "fooba", "[rz]")
|
||||||
|
f("(fo|(zar|bazz)|x)", "", "fo|zar|bazz|x")
|
||||||
|
f("(тестЧЧ|тест)", "тест", "ЧЧ|")
|
||||||
|
f("foo(bar|baz|bana)", "fooba", "[rz]|na")
|
||||||
|
f("^foobar|foobaz", "", "\\Afoobar|foobaz")
|
||||||
|
f("^foobar|^foobaz$", "", "(?-m:\\Afoobar|\\Afoobaz$)")
|
||||||
|
f("foobar|foobaz", "fooba", "[rz]")
|
||||||
|
f("(?:^foobar|^foobaz)aa.*", "", "(?:\\Afoobar|\\Afoobaz)aa")
|
||||||
|
f("foo[bar]+", "foo", "[abr]+")
|
||||||
|
f("foo[a-z]+", "foo", "[a-z]+")
|
||||||
|
f("foo[bar]*", "foo", "[abr]*")
|
||||||
|
f("foo[a-z]*", "foo", "[a-z]*")
|
||||||
|
f("foo[x]+", "foo", "x+")
|
||||||
|
f("foo[^x]+", "foo", "[^x]+")
|
||||||
|
f("foo[x]*", "foo", "x*")
|
||||||
|
f("foo[^x]*", "foo", "[^x]*")
|
||||||
|
f("foo[x]*bar", "foo", "x*bar")
|
||||||
|
f("fo\\Bo[x]*bar?", "fo", "\\Box*bar?")
|
||||||
|
f("foo.+bar", "foo", "(?s:.+bar)")
|
||||||
|
f("a(b|c.*).+", "a", "(?s:(?:b|c.*).+)")
|
||||||
|
f("ab|ac", "a", "[bc]")
|
||||||
|
f("(?i)xyz", "", "(?i:XYZ)")
|
||||||
|
f("(?i)foo|bar", "", "(?i:FOO|BAR)")
|
||||||
|
f("(?i)up.+x", "", "(?is:UP.+X)")
|
||||||
|
f("(?smi)xy.*z$", "", "(?ims:XY.*Z$)")
|
||||||
|
|
||||||
|
// test invalid regexps
|
||||||
|
f("a(", "a(", "")
|
||||||
|
f("a[", "a[", "")
|
||||||
|
f("a[]", "a[]", "")
|
||||||
|
f("a{", "a{", "")
|
||||||
|
f("a{}", "a{}", "")
|
||||||
|
f("invalid(regexp", "invalid(regexp", "")
|
||||||
|
|
||||||
|
// The transformed regexp mustn't match aba
|
||||||
|
f("a?(^ba|c)", "", "a?(?:\\Aba|c)")
|
||||||
|
|
||||||
|
// The transformed regexp mustn't match barx
|
||||||
|
f("(foo|bar$)x*", "", "(?-m:(?:foo|bar$)x*)")
|
||||||
|
|
||||||
|
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5297
|
||||||
|
f(".+;|;.+", "", "(?s:.+;|;.+)")
|
||||||
|
f("^(.+);|;(.+)$", "", "(?s-m:\\A.+;|;.+$)")
|
||||||
|
f("^(.+);$|^;(.+)$", "", "(?s-m:\\A.+;$|\\A;.+$)")
|
||||||
|
f(".*;|;.*", "", "(?s:.*;|;.*)")
|
||||||
|
f("^(.*);|;(.*)$", "", "(?s-m:\\A.*;|;.*$)")
|
||||||
|
f("^(.*);$|^;(.*)$", "", "(?s-m:\\A.*;$|\\A;.*$)")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestSimplifyPromRegex(t *testing.T) {
|
||||||
|
f := func(s, expectedPrefix, expectedSuffix string) {
|
||||||
|
t.Helper()
|
||||||
|
prefix, suffix := SimplifyPromRegex(s)
|
||||||
if prefix != expectedPrefix {
|
if prefix != expectedPrefix {
|
||||||
t.Fatalf("unexpected prefix for s=%q; got %q; want %q", s, prefix, expectedPrefix)
|
t.Fatalf("unexpected prefix for s=%q; got %q; want %q", s, prefix, expectedPrefix)
|
||||||
}
|
}
|
||||||
|
@ -77,7 +195,7 @@ func TestSimplify(t *testing.T) {
|
||||||
f("^foobar|foobaz", "fooba", "[rz]")
|
f("^foobar|foobaz", "fooba", "[rz]")
|
||||||
f("^foobar|^foobaz$", "fooba", "[rz]")
|
f("^foobar|^foobaz$", "fooba", "[rz]")
|
||||||
f("foobar|foobaz", "fooba", "[rz]")
|
f("foobar|foobaz", "fooba", "[rz]")
|
||||||
f("(?:^foobar|^foobaz)aa.*", "fooba", "(?-s:[rz]aa.*)")
|
f("(?:^foobar|^foobaz)aa.*", "fooba", "(?s:[rz]aa.*)")
|
||||||
f("foo[bar]+", "foo", "[abr]+")
|
f("foo[bar]+", "foo", "[abr]+")
|
||||||
f("foo[a-z]+", "foo", "[a-z]+")
|
f("foo[a-z]+", "foo", "[a-z]+")
|
||||||
f("foo[bar]*", "foo", "[abr]*")
|
f("foo[bar]*", "foo", "[abr]*")
|
||||||
|
@ -88,12 +206,12 @@ func TestSimplify(t *testing.T) {
|
||||||
f("foo[^x]*", "foo", "[^x]*")
|
f("foo[^x]*", "foo", "[^x]*")
|
||||||
f("foo[x]*bar", "foo", "x*bar")
|
f("foo[x]*bar", "foo", "x*bar")
|
||||||
f("fo\\Bo[x]*bar?", "fo", "\\Box*bar?")
|
f("fo\\Bo[x]*bar?", "fo", "\\Box*bar?")
|
||||||
f("foo.+bar", "foo", "(?-s:.+bar)")
|
f("foo.+bar", "foo", "(?s:.+bar)")
|
||||||
f("a(b|c.*).+", "a", "(?-s:(?:b|c.*).+)")
|
f("a(b|c.*).+", "a", "(?s:(?:b|c.*).+)")
|
||||||
f("ab|ac", "a", "[bc]")
|
f("ab|ac", "a", "[bc]")
|
||||||
f("(?i)xyz", "", "(?i:XYZ)")
|
f("(?i)xyz", "", "(?i:XYZ)")
|
||||||
f("(?i)foo|bar", "", "(?i:FOO|BAR)")
|
f("(?i)foo|bar", "", "(?i:FOO|BAR)")
|
||||||
f("(?i)up.+x", "", "(?i-s:UP.+X)")
|
f("(?i)up.+x", "", "(?is:UP.+X)")
|
||||||
f("(?smi)xy.*z$", "", "(?ims:XY.*Z$)")
|
f("(?smi)xy.*z$", "", "(?ims:XY.*Z$)")
|
||||||
|
|
||||||
// test invalid regexps
|
// test invalid regexps
|
||||||
|
@ -111,12 +229,12 @@ func TestSimplify(t *testing.T) {
|
||||||
f("(foo|bar$)x*", "", "(?-m:(?:foo|bar$)x*)")
|
f("(foo|bar$)x*", "", "(?-m:(?:foo|bar$)x*)")
|
||||||
|
|
||||||
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5297
|
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5297
|
||||||
f(".+;|;.+", "", "(?-s:.+;|;.+)")
|
f(".+;|;.+", "", "(?s:.+;|;.+)")
|
||||||
f("^(.+);|;(.+)$", "", "(?-s:.+;|;.+)")
|
f("^(.+);|;(.+)$", "", "(?s:.+;|;.+)")
|
||||||
f("^(.+);$|^;(.+)$", "", "(?-s:.+;|;.+)")
|
f("^(.+);$|^;(.+)$", "", "(?s:.+;|;.+)")
|
||||||
f(".*;|;.*", "", "(?-s:.*;|;.*)")
|
f(".*;|;.*", "", "(?s:.*;|;.*)")
|
||||||
f("^(.*);|;(.*)$", "", "(?-s:.*;|;.*)")
|
f("^(.*);|;(.*)$", "", "(?s:.*;|;.*)")
|
||||||
f("^(.*);$|^;(.*)$", "", "(?-s:.*;|;.*)")
|
f("^(.*);$|^;(.*)$", "", "(?s:.*;|;.*)")
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRemoveStartEndAnchors(t *testing.T) {
|
func TestRemoveStartEndAnchors(t *testing.T) {
|
||||||
|
|
|
@ -548,7 +548,7 @@ func getRegexpFromCache(expr string) (*regexpCacheValue, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
sExpr := expr
|
sExpr := expr
|
||||||
orValues := regexutil.GetOrValues(sExpr)
|
orValues := regexutil.GetOrValuesPromRegex(sExpr)
|
||||||
var reMatch func(b []byte) bool
|
var reMatch func(b []byte) bool
|
||||||
var reCost uint64
|
var reCost uint64
|
||||||
var literalSuffix string
|
var literalSuffix string
|
||||||
|
@ -881,7 +881,7 @@ func simplifyRegexp(expr string) (string, string) {
|
||||||
// Make a copy of expr before using it,
|
// Make a copy of expr before using it,
|
||||||
// since it may be constructed via bytesutil.ToUnsafeString()
|
// since it may be constructed via bytesutil.ToUnsafeString()
|
||||||
expr = string(append([]byte{}, expr...))
|
expr = string(append([]byte{}, expr...))
|
||||||
prefix, suffix := regexutil.Simplify(expr)
|
prefix, suffix := regexutil.SimplifyPromRegex(expr)
|
||||||
|
|
||||||
// Put the prefix and the suffix to the cache.
|
// Put the prefix and the suffix to the cache.
|
||||||
ps := &prefixSuffix{
|
ps := &prefixSuffix{
|
||||||
|
|
|
@ -1183,7 +1183,7 @@ func TestSimplifyRegexp(t *testing.T) {
|
||||||
f("ab|ad", "a", "[bd]")
|
f("ab|ad", "a", "[bd]")
|
||||||
f("(?i)xyz", "", "(?i:XYZ)")
|
f("(?i)xyz", "", "(?i:XYZ)")
|
||||||
f("(?i)foo|bar", "", "(?i:FOO|BAR)")
|
f("(?i)foo|bar", "", "(?i:FOO|BAR)")
|
||||||
f("(?i)up.+x", "", "(?i-s:UP.+X)")
|
f("(?i)up.+x", "", "(?is:UP.+X)")
|
||||||
f("(?smi)xy.*z$", "", "(?ims:XY.*Z$)")
|
f("(?smi)xy.*z$", "", "(?ims:XY.*Z$)")
|
||||||
|
|
||||||
// test invalid regexps
|
// test invalid regexps
|
||||||
|
|
Loading…
Reference in a new issue