mirror of
https://github.com/VictoriaMetrics/VictoriaMetrics.git
synced 2024-11-21 14:44:00 +00:00
lib/logstorage: initial implementation of pipes in LogsQL
See https://docs.victoriametrics.com/victorialogs/logsql/#pipes
This commit is contained in:
parent
e66465cb03
commit
9dbd0f9085
119 changed files with 24178 additions and 14059 deletions
7
app/vlogsgenerator/Makefile
Normal file
7
app/vlogsgenerator/Makefile
Normal file
|
@ -0,0 +1,7 @@
|
|||
# All these commands must run from repository root.
|
||||
|
||||
vlogsgenerator:
|
||||
APP_NAME=vlogsgenerator $(MAKE) app-local
|
||||
|
||||
vlogsgenerator-race:
|
||||
APP_NAME=vlogsgenerator RACE=-race $(MAKE) app-local
|
156
app/vlogsgenerator/README.md
Normal file
156
app/vlogsgenerator/README.md
Normal file
|
@ -0,0 +1,156 @@
|
|||
# vlogsgenerator
|
||||
|
||||
Logs generator for [VictoriaLogs](https://docs.victoriametrics.com/victorialogs/).
|
||||
|
||||
## How to build vlogsgenerator?
|
||||
|
||||
Run `make vlogsgenerator` from the repository root. This builds `bin/vlogsgenerator` binary.
|
||||
|
||||
## How run vlogsgenerator?
|
||||
|
||||
`vlogsgenerator` generates logs in [JSON line format](https://jsonlines.org/) suitable for the ingestion
|
||||
via [`/insert/jsonline` endpoint at VictoriaLogs](https://docs.victoriametrics.com/victorialogs/data-ingestion/#json-stream-api).
|
||||
|
||||
By default it writes the generated logs into `stdout`. For example, the following command writes generated logs to `stdout`:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator
|
||||
```
|
||||
|
||||
It is possible to redirect the generated logs to file. For example, the following command writes the generated logs to `logs.json` file:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator > logs.json
|
||||
```
|
||||
|
||||
The generated logs at `logs.json` file can be inspected with the following command:
|
||||
|
||||
```
|
||||
head logs.json | jq .
|
||||
```
|
||||
|
||||
Below is an example output:
|
||||
|
||||
```json
|
||||
{
|
||||
"_time": "2024-05-08T14:34:00.854Z",
|
||||
"_msg": "message for the stream 8 and worker 0; ip=185.69.136.129; uuid=b4fe8f1a-c93c-dea3-ba11-5b9f0509291e; u64=8996587920687045253",
|
||||
"host": "host_8",
|
||||
"worker_id": "0",
|
||||
"run_id": "f9b3deee-e6b6-7f56-5deb-1586e4e81725",
|
||||
"const_0": "some value 0 8",
|
||||
"const_1": "some value 1 8",
|
||||
"const_2": "some value 2 8",
|
||||
"var_0": "some value 0 12752539384823438260",
|
||||
"dict_0": "warn",
|
||||
"dict_1": "info",
|
||||
"u8_0": "6",
|
||||
"u16_0": "35202",
|
||||
"u32_0": "1964973739",
|
||||
"u64_0": "4810489083243239145",
|
||||
"float_0": "1.868",
|
||||
"ip_0": "250.34.75.125",
|
||||
"timestamp_0": "1799-03-16T01:34:18.311Z"
|
||||
}
|
||||
{
|
||||
"_time": "2024-05-08T14:34:00.854Z",
|
||||
"_msg": "message for the stream 9 and worker 0; ip=164.244.254.194; uuid=7e8373b1-ce0d-1ce7-8e96-4bcab8955598; u64=13949903463741076522",
|
||||
"host": "host_9",
|
||||
"worker_id": "0",
|
||||
"run_id": "f9b3deee-e6b6-7f56-5deb-1586e4e81725",
|
||||
"const_0": "some value 0 9",
|
||||
"const_1": "some value 1 9",
|
||||
"const_2": "some value 2 9",
|
||||
"var_0": "some value 0 5371555382075206134",
|
||||
"dict_0": "INFO",
|
||||
"dict_1": "FATAL",
|
||||
"u8_0": "219",
|
||||
"u16_0": "31459",
|
||||
"u32_0": "3918836777",
|
||||
"u64_0": "6593354256620219850",
|
||||
"float_0": "1.085",
|
||||
"ip_0": "253.151.88.158",
|
||||
"timestamp_0": "2042-10-05T16:42:57.082Z"
|
||||
}
|
||||
```
|
||||
|
||||
The `run_id` field uniquely identifies every `vlogsgenerator` invocation.
|
||||
|
||||
### How to write logs to VictoriaLogs?
|
||||
|
||||
The generated logs can be written directly to VictoriaLogs by passing the address of [`/insert/jsonline` endpoint](https://docs.victoriametrics.com/victorialogs/data-ingestion/#json-stream-api)
|
||||
to `-addr` command-line flag. For example, the following command writes the generated logs to VictoriaLogs running at `localhost`:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator -addr=http://localhost:9428/insert/jsonline
|
||||
```
|
||||
|
||||
### Configuration
|
||||
|
||||
`vlogsgenerator` accepts various command-line flags, which can be used for configuring the number and the shape of the generated logs.
|
||||
These flags can be inspected by running `vlogsgenerator -help`. Below are the most interesting flags:
|
||||
|
||||
* `-start` - starting timestamp for generating logs. Logs are evenly generated on the [`-start` ... `-end`] interval.
|
||||
* `-end` - ending timestamp for generating logs. Logs are evenly generated on the [`-start` ... `-end`] interval.
|
||||
* `-activeStreams` - the number of active [log streams](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) to generate.
|
||||
* `-logsPerStream` - the number of log entries to generate per each log stream. Log entries are evenly distributed on the [`-start` ... `-end`] interval.
|
||||
|
||||
The total number of generated logs can be calculated as `-activeStreams` * `-logsPerStream`.
|
||||
|
||||
For example, the following command generates `1_000_000` log entries on the time range `[2024-01-01 - 2024-02-01]` across `100`
|
||||
[log streams](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields), where every logs stream contains `10_000` log entries,
|
||||
and writes them to `http://localhost:9428/insert/jsonline`:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator \
|
||||
-start=2024-01-01 -end=2024-02-01 \
|
||||
-activeStreams=100 \
|
||||
-logsPerStream=10_000 \
|
||||
-addr=http://localhost:9428/insert/jsonline
|
||||
```
|
||||
|
||||
### Churn rate
|
||||
|
||||
It is possible to generate churn rate for active [log streams](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields)
|
||||
by specifying `-totalStreams` command-line flag bigger than `-activeStreams`. For example, the following command generates
|
||||
logs for `1000` total streams, while the number of active streams equals to `100`. This means that at every time there are logs for `100` streams,
|
||||
but these streams change over the given [`-start` ... `-end`] time range, so the total number of streams on the given time range becomes `1000`:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator \
|
||||
-start=2024-01-01 -end=2024-02-01 \
|
||||
-activeStreams=100 \
|
||||
-totalStreams=1_000 \
|
||||
-logsPerStream=10_000 \
|
||||
-addr=http://localhost:9428/insert/jsonline
|
||||
```
|
||||
|
||||
In this case the total number of generated logs equals to `-totalStreams` * `-logsPerStream` = `10_000_000`.
|
||||
|
||||
### Benchmark tuning
|
||||
|
||||
By default `vlogsgenerator` generates and writes logs by a single worker. This may limit the maximum data ingestion rate during benchmarks.
|
||||
The number of workers can be changed via `-workers` command-line flag. For example, the following command generates and writes logs with `16` workers:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator \
|
||||
-start=2024-01-01 -end=2024-02-01 \
|
||||
-activeStreams=100 \
|
||||
-logsPerStream=10_000 \
|
||||
-addr=http://localhost:9428/insert/jsonline \
|
||||
-workers=16
|
||||
```
|
||||
|
||||
### Output statistics
|
||||
|
||||
Every 10 seconds `vlogsgenerator` writes statistics about the generated logs into `stderr`. The frequency of the generated statistics can be adjusted via `-statInterval` command-line flag.
|
||||
For example, the following command writes statistics every 2 seconds:
|
||||
|
||||
```
|
||||
bin/vlogsgenerator \
|
||||
-start=2024-01-01 -end=2024-02-01 \
|
||||
-activeStreams=100 \
|
||||
-logsPerStream=10_000 \
|
||||
-addr=http://localhost:9428/insert/jsonline \
|
||||
-statInterval=2s
|
||||
```
|
339
app/vlogsgenerator/main.go
Normal file
339
app/vlogsgenerator/main.go
Normal file
|
@ -0,0 +1,339 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"math/rand"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"os"
|
||||
"strconv"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/buildinfo"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/envflag"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
|
||||
)
|
||||
|
||||
var (
|
||||
addr = flag.String("addr", "stdout", "HTTP address to push the generated logs to; if it is set to stdout, then logs are generated to stdout")
|
||||
workers = flag.Int("workers", 1, "The number of workers to use to push logs to -addr")
|
||||
|
||||
start = newTimeFlag("start", "-1d", "Generated logs start from this time; see https://docs.victoriametrics.com/#timestamp-formats")
|
||||
end = newTimeFlag("end", "0s", "Generated logs end at this time; see https://docs.victoriametrics.com/#timestamp-formats")
|
||||
activeStreams = flag.Int("activeStreams", 100, "The number of active log streams to generate; see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields")
|
||||
totalStreams = flag.Int("totalStreams", 0, "The number of total log streams; if -totalStreams > -activeStreams, then some active streams are substituted with new streams "+
|
||||
"during data generation")
|
||||
logsPerStream = flag.Int64("logsPerStream", 1_000, "The number of log entries to generate per each log stream. Log entries are evenly distributed between -start and -end")
|
||||
constFieldsPerLog = flag.Int("constFieldsPerLog", 3, "The number of fields with constaint values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
varFieldsPerLog = flag.Int("varFieldsPerLog", 1, "The number of fields with variable values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
dictFieldsPerLog = flag.Int("dictFieldsPerLog", 2, "The number of fields with up to 8 different values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
u8FieldsPerLog = flag.Int("u8FieldsPerLog", 1, "The number of fields with uint8 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
u16FieldsPerLog = flag.Int("u16FieldsPerLog", 1, "The number of fields with uint16 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
u32FieldsPerLog = flag.Int("u32FieldsPerLog", 1, "The number of fields with uint32 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
u64FieldsPerLog = flag.Int("u64FieldsPerLog", 1, "The number of fields with uint64 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
floatFieldsPerLog = flag.Int("floatFieldsPerLog", 1, "The number of fields with float64 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
ipFieldsPerLog = flag.Int("ipFieldsPerLog", 1, "The number of fields with IPv4 values to generate per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
timestampFieldsPerLog = flag.Int("timestampFieldsPerLog", 1, "The number of fields with ISO8601 timestamps per each log entry; "+
|
||||
"see https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model")
|
||||
|
||||
statInterval = flag.Duration("statInterval", 10*time.Second, "The interval between publishing the stats")
|
||||
)
|
||||
|
||||
func main() {
|
||||
// Write flags and help message to stdout, since it is easier to grep or pipe.
|
||||
flag.CommandLine.SetOutput(os.Stdout)
|
||||
envflag.Parse()
|
||||
buildinfo.Init()
|
||||
logger.Init()
|
||||
|
||||
var remoteWriteURL *url.URL
|
||||
if *addr != "stdout" {
|
||||
urlParsed, err := url.Parse(*addr)
|
||||
if err != nil {
|
||||
logger.Fatalf("cannot parse -addr=%q: %s", *addr, err)
|
||||
}
|
||||
qs, err := url.ParseQuery(urlParsed.RawQuery)
|
||||
if err != nil {
|
||||
logger.Fatalf("cannot parse query string in -addr=%q: %w", *addr, err)
|
||||
}
|
||||
qs.Set("_stream_fields", "host,worker_id")
|
||||
urlParsed.RawQuery = qs.Encode()
|
||||
remoteWriteURL = urlParsed
|
||||
}
|
||||
|
||||
if start.nsec >= end.nsec {
|
||||
logger.Fatalf("-start=%s must be smaller than -end=%s", start, end)
|
||||
}
|
||||
if *activeStreams <= 0 {
|
||||
logger.Fatalf("-activeStreams must be bigger than 0; got %d", *activeStreams)
|
||||
}
|
||||
if *logsPerStream <= 0 {
|
||||
logger.Fatalf("-logsPerStream must be bigger than 0; got %d", *logsPerStream)
|
||||
}
|
||||
if *totalStreams < *activeStreams {
|
||||
*totalStreams = *activeStreams
|
||||
}
|
||||
|
||||
cfg := &workerConfig{
|
||||
url: remoteWriteURL,
|
||||
activeStreams: *activeStreams,
|
||||
totalStreams: *totalStreams,
|
||||
}
|
||||
|
||||
// divide total and active streams among workers
|
||||
if *workers <= 0 {
|
||||
logger.Fatalf("-workers must be bigger than 0; got %d", *workers)
|
||||
}
|
||||
if *workers > *activeStreams {
|
||||
logger.Fatalf("-workers=%d cannot exceed -activeStreams=%d", *workers, *activeStreams)
|
||||
}
|
||||
cfg.activeStreams /= *workers
|
||||
cfg.totalStreams /= *workers
|
||||
|
||||
logger.Infof("start -workers=%d workers for ingesting -logsPerStream=%d log entries per each -totalStreams=%d (-activeStreams=%d) on a time range -start=%s, -end=%s to -addr=%s",
|
||||
*workers, *logsPerStream, *totalStreams, *activeStreams, toRFC3339(start.nsec), toRFC3339(end.nsec), *addr)
|
||||
|
||||
startTime := time.Now()
|
||||
var wg sync.WaitGroup
|
||||
for i := 0; i < *workers; i++ {
|
||||
wg.Add(1)
|
||||
go func(workerID int) {
|
||||
defer wg.Done()
|
||||
generateAndPushLogs(cfg, workerID)
|
||||
}(i)
|
||||
}
|
||||
|
||||
go func() {
|
||||
prevEntries := uint64(0)
|
||||
prevBytes := uint64(0)
|
||||
ticker := time.NewTicker(*statInterval)
|
||||
for range ticker.C {
|
||||
currEntries := logEntriesCount.Load()
|
||||
deltaEntries := currEntries - prevEntries
|
||||
rateEntries := float64(deltaEntries) / statInterval.Seconds()
|
||||
|
||||
currBytes := bytesGenerated.Load()
|
||||
deltaBytes := currBytes - prevBytes
|
||||
rateBytes := float64(deltaBytes) / statInterval.Seconds()
|
||||
logger.Infof("generated %dK log entries (%dK total) at %.0fK entries/sec, %dMB (%dMB total) at %.0fMB/sec",
|
||||
deltaEntries/1e3, currEntries/1e3, rateEntries/1e3, deltaBytes/1e6, currBytes/1e6, rateBytes/1e6)
|
||||
|
||||
prevEntries = currEntries
|
||||
prevBytes = currBytes
|
||||
}
|
||||
}()
|
||||
|
||||
wg.Wait()
|
||||
|
||||
dSecs := time.Since(startTime).Seconds()
|
||||
currEntries := logEntriesCount.Load()
|
||||
currBytes := bytesGenerated.Load()
|
||||
rateEntries := float64(currEntries) / dSecs
|
||||
rateBytes := float64(currBytes) / dSecs
|
||||
logger.Infof("ingested %dK log entries (%dMB) in %.3f seconds; avg ingestion rate: %.0fK entries/sec, %.0fMB/sec", currEntries/1e3, currBytes/1e6, dSecs, rateEntries/1e3, rateBytes/1e6)
|
||||
}
|
||||
|
||||
var logEntriesCount atomic.Uint64
|
||||
|
||||
var bytesGenerated atomic.Uint64
|
||||
|
||||
type workerConfig struct {
|
||||
url *url.URL
|
||||
activeStreams int
|
||||
totalStreams int
|
||||
}
|
||||
|
||||
type statWriter struct {
|
||||
w io.Writer
|
||||
}
|
||||
|
||||
func (sw *statWriter) Write(p []byte) (int, error) {
|
||||
bytesGenerated.Add(uint64(len(p)))
|
||||
return sw.w.Write(p)
|
||||
}
|
||||
|
||||
func generateAndPushLogs(cfg *workerConfig, workerID int) {
|
||||
pr, pw := io.Pipe()
|
||||
sw := &statWriter{
|
||||
w: pw,
|
||||
}
|
||||
bw := bufio.NewWriter(sw)
|
||||
doneCh := make(chan struct{})
|
||||
go func() {
|
||||
generateLogs(bw, workerID, cfg.activeStreams, cfg.totalStreams)
|
||||
_ = bw.Flush()
|
||||
_ = pw.Close()
|
||||
close(doneCh)
|
||||
}()
|
||||
|
||||
if cfg.url == nil {
|
||||
_, err := io.Copy(os.Stdout, pr)
|
||||
if err != nil {
|
||||
logger.Fatalf("unexpected error when writing logs to stdout: %s", err)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
req, err := http.NewRequest("POST", cfg.url.String(), pr)
|
||||
if err != nil {
|
||||
logger.Fatalf("cannot create request to %q: %s", cfg.url, err)
|
||||
}
|
||||
|
||||
resp, err := http.DefaultClient.Do(req)
|
||||
if err != nil {
|
||||
logger.Fatalf("cannot perform request to %q: %s", cfg.url, err)
|
||||
}
|
||||
if resp.StatusCode/100 != 2 {
|
||||
logger.Fatalf("unexpected status code got from %q: %d; want 2xx", cfg.url, err)
|
||||
}
|
||||
|
||||
// Wait until all the generateLogs goroutine is finished.
|
||||
<-doneCh
|
||||
}
|
||||
|
||||
func generateLogs(bw *bufio.Writer, workerID, activeStreams, totalStreams int) {
|
||||
streamLifetime := int64(float64(end.nsec-start.nsec) * (float64(activeStreams) / float64(totalStreams)))
|
||||
streamStep := int64(float64(end.nsec-start.nsec) / float64(totalStreams-activeStreams+1))
|
||||
step := streamLifetime / (*logsPerStream - 1)
|
||||
|
||||
currNsec := start.nsec
|
||||
for currNsec < end.nsec {
|
||||
firstStreamID := int((currNsec - start.nsec) / streamStep)
|
||||
generateLogsAtTimestamp(bw, workerID, currNsec, firstStreamID, activeStreams)
|
||||
currNsec += step
|
||||
}
|
||||
}
|
||||
|
||||
var runID = toUUID(rand.Uint64(), rand.Uint64())
|
||||
|
||||
func generateLogsAtTimestamp(bw *bufio.Writer, workerID int, ts int64, firstStreamID, activeStreams int) {
|
||||
streamID := firstStreamID
|
||||
timeStr := toRFC3339(ts)
|
||||
for i := 0; i < activeStreams; i++ {
|
||||
ip := toIPv4(rand.Uint32())
|
||||
uuid := toUUID(rand.Uint64(), rand.Uint64())
|
||||
fmt.Fprintf(bw, `{"_time":%q,"_msg":"message for the stream %d and worker %d; ip=%s; uuid=%s; u64=%d","host":"host_%d","worker_id":"%d"`,
|
||||
timeStr, streamID, workerID, ip, uuid, rand.Uint64(), streamID, workerID)
|
||||
fmt.Fprintf(bw, `,"run_id":"%s"`, runID)
|
||||
for j := 0; j < *constFieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"const_%d":"some value %d %d"`, j, j, streamID)
|
||||
}
|
||||
for j := 0; j < *varFieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"var_%d":"some value %d %d"`, j, j, rand.Uint64())
|
||||
}
|
||||
for j := 0; j < *dictFieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"dict_%d":"%s"`, j, dictValues[rand.Intn(len(dictValues))])
|
||||
}
|
||||
for j := 0; j < *u8FieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"u8_%d":"%d"`, j, uint8(rand.Uint32()))
|
||||
}
|
||||
for j := 0; j < *u16FieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"u16_%d":"%d"`, j, uint16(rand.Uint32()))
|
||||
}
|
||||
for j := 0; j < *u32FieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"u32_%d":"%d"`, j, rand.Uint32())
|
||||
}
|
||||
for j := 0; j < *u64FieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"u64_%d":"%d"`, j, rand.Uint64())
|
||||
}
|
||||
for j := 0; j < *floatFieldsPerLog; j++ {
|
||||
fmt.Fprintf(bw, `,"float_%d":"%v"`, j, math.Round(10_000*rand.Float64())/1000)
|
||||
}
|
||||
for j := 0; j < *ipFieldsPerLog; j++ {
|
||||
ip := toIPv4(rand.Uint32())
|
||||
fmt.Fprintf(bw, `,"ip_%d":"%s"`, j, ip)
|
||||
}
|
||||
for j := 0; j < *timestampFieldsPerLog; j++ {
|
||||
timestamp := toISO8601(int64(rand.Uint64()))
|
||||
fmt.Fprintf(bw, `,"timestamp_%d":"%s"`, j, timestamp)
|
||||
}
|
||||
fmt.Fprintf(bw, "}\n")
|
||||
|
||||
logEntriesCount.Add(1)
|
||||
streamID++
|
||||
}
|
||||
}
|
||||
|
||||
var dictValues = []string{
|
||||
"debug",
|
||||
"info",
|
||||
"warn",
|
||||
"error",
|
||||
"fatal",
|
||||
"ERROR",
|
||||
"FATAL",
|
||||
"INFO",
|
||||
}
|
||||
|
||||
func newTimeFlag(name, defaultValue, description string) *timeFlag {
|
||||
var tf timeFlag
|
||||
if err := tf.Set(defaultValue); err != nil {
|
||||
logger.Panicf("invalid defaultValue=%q for flag %q: %w", defaultValue, name, err)
|
||||
}
|
||||
flag.Var(&tf, name, description)
|
||||
return &tf
|
||||
}
|
||||
|
||||
type timeFlag struct {
|
||||
s string
|
||||
nsec int64
|
||||
}
|
||||
|
||||
func (tf *timeFlag) Set(s string) error {
|
||||
msec, err := promutils.ParseTimeMsec(s)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse time from %q: %w", s, err)
|
||||
}
|
||||
tf.s = s
|
||||
tf.nsec = msec * 1e6
|
||||
return nil
|
||||
}
|
||||
|
||||
func (tf *timeFlag) String() string {
|
||||
return tf.s
|
||||
}
|
||||
|
||||
func toRFC3339(nsec int64) string {
|
||||
return time.Unix(0, nsec).UTC().Format(time.RFC3339Nano)
|
||||
}
|
||||
|
||||
func toISO8601(nsec int64) string {
|
||||
return time.Unix(0, nsec).UTC().Format("2006-01-02T15:04:05.000Z")
|
||||
}
|
||||
|
||||
func toIPv4(n uint32) string {
|
||||
dst := make([]byte, 0, len("255.255.255.255"))
|
||||
dst = marshalUint64(dst, uint64(n>>24))
|
||||
dst = append(dst, '.')
|
||||
dst = marshalUint64(dst, uint64((n>>16)&0xff))
|
||||
dst = append(dst, '.')
|
||||
dst = marshalUint64(dst, uint64((n>>8)&0xff))
|
||||
dst = append(dst, '.')
|
||||
dst = marshalUint64(dst, uint64(n&0xff))
|
||||
return string(dst)
|
||||
}
|
||||
|
||||
func toUUID(a, b uint64) string {
|
||||
return fmt.Sprintf("%08x-%04x-%04x-%04x-%012x", a&(1<<32-1), (a>>32)&(1<<16-1), (a >> 48), b&(1<<16-1), b>>16)
|
||||
}
|
||||
|
||||
// marshalUint64 appends string representation of n to dst and returns the result.
|
||||
func marshalUint64(dst []byte, n uint64) []byte {
|
||||
return strconv.AppendUint(dst, n, 10)
|
||||
}
|
|
@ -1,6 +1,7 @@
|
|||
package logsql
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/app/vlstorage"
|
||||
|
@ -17,8 +18,8 @@ var (
|
|||
"too big value for this flag may result in high memory usage since the sorting is performed in memory")
|
||||
)
|
||||
|
||||
// ProcessQueryRequest handles /select/logsql/query request
|
||||
func ProcessQueryRequest(w http.ResponseWriter, r *http.Request, stopCh <-chan struct{}, cancel func()) {
|
||||
// ProcessQueryRequest handles /select/logsql/query request.
|
||||
func ProcessQueryRequest(ctx context.Context, w http.ResponseWriter, r *http.Request) {
|
||||
// Extract tenantID
|
||||
tenantID, err := logstorage.GetTenantIDFromRequest(r)
|
||||
if err != nil {
|
||||
|
@ -42,15 +43,18 @@ func ProcessQueryRequest(w http.ResponseWriter, r *http.Request, stopCh <-chan s
|
|||
sw := getSortWriter()
|
||||
sw.Init(w, maxSortBufferSize.IntN(), limit)
|
||||
tenantIDs := []logstorage.TenantID{tenantID}
|
||||
vlstorage.RunQuery(tenantIDs, q, stopCh, func(columns []logstorage.BlockColumn) {
|
||||
|
||||
ctxWithCancel, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
|
||||
writeBlock := func(_ uint, timestamps []int64, columns []logstorage.BlockColumn) {
|
||||
if len(columns) == 0 {
|
||||
return
|
||||
}
|
||||
rowsCount := len(columns[0].Values)
|
||||
|
||||
bb := blockResultPool.Get()
|
||||
for rowIdx := 0; rowIdx < rowsCount; rowIdx++ {
|
||||
WriteJSONRow(bb, columns, rowIdx)
|
||||
for i := range timestamps {
|
||||
WriteJSONRow(bb, columns, i)
|
||||
}
|
||||
|
||||
if !sw.TryWrite(bb.B) {
|
||||
|
@ -58,9 +62,17 @@ func ProcessQueryRequest(w http.ResponseWriter, r *http.Request, stopCh <-chan s
|
|||
}
|
||||
|
||||
blockResultPool.Put(bb)
|
||||
})
|
||||
}
|
||||
|
||||
err = vlstorage.RunQuery(ctxWithCancel, tenantIDs, q, writeBlock)
|
||||
|
||||
sw.FinalFlush()
|
||||
putSortWriter(sw)
|
||||
|
||||
if err != nil {
|
||||
httpserver.Errorf(w, r, "cannot execute query [%s]: %s", qStr, err)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
var blockResultPool bytesutil.ByteBufferPool
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package vlselect
|
||||
|
||||
import (
|
||||
"context"
|
||||
"embed"
|
||||
"flag"
|
||||
"fmt"
|
||||
|
@ -141,15 +140,11 @@ func RequestHandler(w http.ResponseWriter, r *http.Request) bool {
|
|||
}
|
||||
}
|
||||
|
||||
ctxWithCancel, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
stopCh = ctxWithCancel.Done()
|
||||
|
||||
switch {
|
||||
case path == "/logsql/query":
|
||||
logsqlQueryRequests.Inc()
|
||||
httpserver.EnableCORS(w, r)
|
||||
logsql.ProcessQueryRequest(w, r, stopCh, cancel)
|
||||
logsql.ProcessQueryRequest(ctx, w, r)
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
|
|
|
@ -1,10 +1,11 @@
|
|||
package vlstorage
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/metrics"
|
||||
|
@ -61,10 +62,16 @@ func Init() {
|
|||
|
||||
var ss logstorage.StorageStats
|
||||
strg.UpdateStats(&ss)
|
||||
logger.Infof("successfully opened storage in %.3f seconds; partsCount: %d; blocksCount: %d; rowsCount: %d; sizeBytes: %d",
|
||||
time.Since(startTime).Seconds(), ss.FileParts, ss.FileBlocks, ss.FileRowsCount, ss.CompressedFileSize)
|
||||
storageMetrics = initStorageMetrics(strg)
|
||||
logger.Infof("successfully opened storage in %.3f seconds; smallParts: %d; bigParts: %d; smallPartBlocks: %d; bigPartBlocks: %d; smallPartRows: %d; bigPartRows: %d; "+
|
||||
"smallPartSize: %d bytes; bigPartSize: %d bytes",
|
||||
time.Since(startTime).Seconds(), ss.SmallParts, ss.BigParts, ss.SmallPartBlocks, ss.BigPartBlocks, ss.SmallPartRowsCount, ss.BigPartRowsCount,
|
||||
ss.CompressedSmallPartSize, ss.CompressedBigPartSize)
|
||||
|
||||
// register storage metrics
|
||||
storageMetrics = metrics.NewSet()
|
||||
storageMetrics.RegisterMetricsWriter(func(w io.Writer) {
|
||||
writeStorageMetrics(w, strg)
|
||||
})
|
||||
metrics.RegisterSet(storageMetrics)
|
||||
}
|
||||
|
||||
|
@ -99,117 +106,61 @@ func MustAddRows(lr *logstorage.LogRows) {
|
|||
strg.MustAddRows(lr)
|
||||
}
|
||||
|
||||
// RunQuery runs the given q and calls processBlock for the returned data blocks
|
||||
func RunQuery(tenantIDs []logstorage.TenantID, q *logstorage.Query, stopCh <-chan struct{}, processBlock func(columns []logstorage.BlockColumn)) {
|
||||
strg.RunQuery(tenantIDs, q, stopCh, processBlock)
|
||||
// RunQuery runs the given q and calls writeBlock for the returned data blocks
|
||||
func RunQuery(ctx context.Context, tenantIDs []logstorage.TenantID, q *logstorage.Query, writeBlock func(workerID uint, timestamps []int64, columns []logstorage.BlockColumn)) error {
|
||||
return strg.RunQuery(ctx, tenantIDs, q, writeBlock)
|
||||
}
|
||||
|
||||
func initStorageMetrics(strg *logstorage.Storage) *metrics.Set {
|
||||
ssCache := &logstorage.StorageStats{}
|
||||
var ssCacheLock sync.Mutex
|
||||
var lastUpdateTime time.Time
|
||||
func writeStorageMetrics(w io.Writer, strg *logstorage.Storage) {
|
||||
var ss logstorage.StorageStats
|
||||
strg.UpdateStats(&ss)
|
||||
|
||||
m := func() *logstorage.StorageStats {
|
||||
ssCacheLock.Lock()
|
||||
defer ssCacheLock.Unlock()
|
||||
if time.Since(lastUpdateTime) < time.Second {
|
||||
return ssCache
|
||||
}
|
||||
var ss logstorage.StorageStats
|
||||
strg.UpdateStats(&ss)
|
||||
ssCache = &ss
|
||||
lastUpdateTime = time.Now()
|
||||
return ssCache
|
||||
metrics.WriteGaugeUint64(w, fmt.Sprintf(`vl_free_disk_space_bytes{path=%q}`, *storageDataPath), fs.MustGetFreeSpace(*storageDataPath))
|
||||
|
||||
isReadOnly := uint64(0)
|
||||
if ss.IsReadOnly {
|
||||
isReadOnly = 1
|
||||
}
|
||||
metrics.WriteGaugeUint64(w, fmt.Sprintf(`vl_storage_is_read_only{path=%q}`, *storageDataPath), isReadOnly)
|
||||
|
||||
ms := metrics.NewSet()
|
||||
metrics.WriteGaugeUint64(w, `vl_active_merges{type="storage/inmemory"}`, ss.InmemoryActiveMerges)
|
||||
metrics.WriteGaugeUint64(w, `vl_active_merges{type="storage/small"}`, ss.SmallPartActiveMerges)
|
||||
metrics.WriteGaugeUint64(w, `vl_active_merges{type="storage/big"}`, ss.BigPartActiveMerges)
|
||||
|
||||
ms.NewGauge(fmt.Sprintf(`vl_free_disk_space_bytes{path=%q}`, *storageDataPath), func() float64 {
|
||||
return float64(fs.MustGetFreeSpace(*storageDataPath))
|
||||
})
|
||||
ms.NewGauge(fmt.Sprintf(`vl_storage_is_read_only{path=%q}`, *storageDataPath), func() float64 {
|
||||
if m().IsReadOnly {
|
||||
return 1
|
||||
}
|
||||
return 0
|
||||
})
|
||||
metrics.WriteCounterUint64(w, `vl_merges_total{type="storage/inmemory"}`, ss.InmemoryMergesTotal)
|
||||
metrics.WriteCounterUint64(w, `vl_merges_total{type="storage/small"}`, ss.SmallPartMergesTotal)
|
||||
metrics.WriteCounterUint64(w, `vl_merges_total{type="storage/big"}`, ss.BigPartMergesTotal)
|
||||
|
||||
ms.NewGauge(`vl_active_merges{type="inmemory"}`, func() float64 {
|
||||
return float64(m().InmemoryActiveMerges)
|
||||
})
|
||||
ms.NewGauge(`vl_merges_total{type="inmemory"}`, func() float64 {
|
||||
return float64(m().InmemoryMergesTotal)
|
||||
})
|
||||
ms.NewGauge(`vl_active_merges{type="file"}`, func() float64 {
|
||||
return float64(m().FileActiveMerges)
|
||||
})
|
||||
ms.NewGauge(`vl_merges_total{type="file"}`, func() float64 {
|
||||
return float64(m().FileMergesTotal)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_rows{type="storage/inmemory"}`, ss.InmemoryRowsCount)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_rows{type="storage/small"}`, ss.SmallPartRowsCount)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_rows{type="storage/big"}`, ss.BigPartRowsCount)
|
||||
|
||||
ms.NewGauge(`vl_storage_rows{type="inmemory"}`, func() float64 {
|
||||
return float64(m().InmemoryRowsCount)
|
||||
})
|
||||
ms.NewGauge(`vl_storage_rows{type="file"}`, func() float64 {
|
||||
return float64(m().FileRowsCount)
|
||||
})
|
||||
ms.NewGauge(`vl_storage_parts{type="inmemory"}`, func() float64 {
|
||||
return float64(m().InmemoryParts)
|
||||
})
|
||||
ms.NewGauge(`vl_storage_parts{type="file"}`, func() float64 {
|
||||
return float64(m().FileParts)
|
||||
})
|
||||
ms.NewGauge(`vl_storage_blocks{type="inmemory"}`, func() float64 {
|
||||
return float64(m().InmemoryBlocks)
|
||||
})
|
||||
ms.NewGauge(`vl_storage_blocks{type="file"}`, func() float64 {
|
||||
return float64(m().FileBlocks)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_parts{type="storage/inmemory"}`, ss.InmemoryParts)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_parts{type="storage/small"}`, ss.SmallParts)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_parts{type="storage/big"}`, ss.BigParts)
|
||||
|
||||
ms.NewGauge(`vl_partitions`, func() float64 {
|
||||
return float64(m().PartitionsCount)
|
||||
})
|
||||
ms.NewGauge(`vl_streams_created_total`, func() float64 {
|
||||
return float64(m().StreamsCreatedTotal)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_blocks{type="storage/inmemory"}`, ss.InmemoryBlocks)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_blocks{type="storage/small"}`, ss.SmallPartBlocks)
|
||||
metrics.WriteGaugeUint64(w, `vl_storage_blocks{type="storage/big"}`, ss.BigPartBlocks)
|
||||
|
||||
ms.NewGauge(`vl_indexdb_rows`, func() float64 {
|
||||
return float64(m().IndexdbItemsCount)
|
||||
})
|
||||
ms.NewGauge(`vl_indexdb_parts`, func() float64 {
|
||||
return float64(m().IndexdbPartsCount)
|
||||
})
|
||||
ms.NewGauge(`vl_indexdb_blocks`, func() float64 {
|
||||
return float64(m().IndexdbBlocksCount)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_partitions`, ss.PartitionsCount)
|
||||
metrics.WriteCounterUint64(w, `vl_streams_created_total`, ss.StreamsCreatedTotal)
|
||||
|
||||
ms.NewGauge(`vl_data_size_bytes{type="indexdb"}`, func() float64 {
|
||||
return float64(m().IndexdbSizeBytes)
|
||||
})
|
||||
ms.NewGauge(`vl_data_size_bytes{type="storage"}`, func() float64 {
|
||||
dm := m()
|
||||
return float64(dm.CompressedInmemorySize + dm.CompressedFileSize)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_indexdb_rows`, ss.IndexdbItemsCount)
|
||||
metrics.WriteGaugeUint64(w, `vl_indexdb_parts`, ss.IndexdbPartsCount)
|
||||
metrics.WriteGaugeUint64(w, `vl_indexdb_blocks`, ss.IndexdbBlocksCount)
|
||||
|
||||
ms.NewGauge(`vl_compressed_data_size_bytes{type="inmemory"}`, func() float64 {
|
||||
return float64(m().CompressedInmemorySize)
|
||||
})
|
||||
ms.NewGauge(`vl_compressed_data_size_bytes{type="file"}`, func() float64 {
|
||||
return float64(m().CompressedFileSize)
|
||||
})
|
||||
ms.NewGauge(`vl_uncompressed_data_size_bytes{type="inmemory"}`, func() float64 {
|
||||
return float64(m().UncompressedInmemorySize)
|
||||
})
|
||||
ms.NewGauge(`vl_uncompressed_data_size_bytes{type="file"}`, func() float64 {
|
||||
return float64(m().UncompressedFileSize)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_data_size_bytes{type="indexdb"}`, ss.IndexdbSizeBytes)
|
||||
metrics.WriteGaugeUint64(w, `vl_data_size_bytes{type="storage"}`, ss.CompressedInmemorySize+ss.CompressedSmallPartSize+ss.CompressedBigPartSize)
|
||||
|
||||
ms.NewGauge(`vl_rows_dropped_total{reason="too_big_timestamp"}`, func() float64 {
|
||||
return float64(m().RowsDroppedTooBigTimestamp)
|
||||
})
|
||||
ms.NewGauge(`vl_rows_dropped_total{reason="too_small_timestamp"}`, func() float64 {
|
||||
return float64(m().RowsDroppedTooSmallTimestamp)
|
||||
})
|
||||
metrics.WriteGaugeUint64(w, `vl_compressed_data_size_bytes{type="storage/inmemory"}`, ss.CompressedInmemorySize)
|
||||
metrics.WriteGaugeUint64(w, `vl_compressed_data_size_bytes{type="storage/small"}`, ss.CompressedSmallPartSize)
|
||||
metrics.WriteGaugeUint64(w, `vl_compressed_data_size_bytes{type="storage/big"}`, ss.CompressedBigPartSize)
|
||||
|
||||
return ms
|
||||
metrics.WriteGaugeUint64(w, `vl_uncompressed_data_size_bytes{type="storage/inmemory"}`, ss.UncompressedInmemorySize)
|
||||
metrics.WriteGaugeUint64(w, `vl_uncompressed_data_size_bytes{type="storage/small"}`, ss.UncompressedSmallPartSize)
|
||||
metrics.WriteGaugeUint64(w, `vl_uncompressed_data_size_bytes{type="storage/big"}`, ss.UncompressedBigPartSize)
|
||||
|
||||
metrics.WriteCounterUint64(w, `vl_rows_dropped_total{reason="too_big_timestamp"}`, ss.RowsDroppedTooBigTimestamp)
|
||||
metrics.WriteCounterUint64(w, `vl_rows_dropped_total{reason="too_small_timestamp"}`, ss.RowsDroppedTooSmallTimestamp)
|
||||
}
|
||||
|
|
|
@ -19,6 +19,23 @@ according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/QuickSta
|
|||
|
||||
## tip
|
||||
|
||||
* FEATURE: return all the log fields by default in query results. Previously only [`_stream`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields), [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) and [`_msg`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) fields were returned by default.
|
||||
* FEATURE: add support for returning only the requested log [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#fields-pipe).
|
||||
* FEATURE: add support for calculating various stats over [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). Grouping by arbitrary set of [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) is supported. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#stats-pipe) for details.
|
||||
* FEATURE: add support for sorting the returned results. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe).
|
||||
* FEATURE: add support for returning unique results. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#uniq-pipe).
|
||||
* FEATURE: add support for limiting the number of returned results. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#limiters).
|
||||
* FEATURE: add support for copying and renaming the selected log fields. See [these](https://docs.victoriametrics.com/victorialogs/logsql/#copy-pipe) and [these](https://docs.victoriametrics.com/victorialogs/logsql/#rename-pipe) docs.
|
||||
* FEATURE: allow using `_` inside numbers. For example, `score:range[1_000, 5_000_000]` for [`range` filter](https://docs.victoriametrics.com/victorialogs/logsql/#range-filter).
|
||||
* FEATURE: allow numbers in hexadecimal and binary form. For example, `response_size:range[0xff, 0b10001101101]` for [`range` filter](https://docs.victoriametrics.com/victorialogs/logsql/#range-filter).
|
||||
* FEATURE: allow using duration and byte size suffixes in numeric values inside LogsQL queries. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#numeric-values).
|
||||
* FEATURE: improve data ingestion performance by up to 50%.
|
||||
* FEATURE: optimize performance for [LogsQL query](https://docs.victoriametrics.com/victorialogs/logsql/), which contains multiple filters for [words](https://docs.victoriametrics.com/victorialogs/logsql/#word-filter) or [phrases](https://docs.victoriametrics.com/victorialogs/logsql/#phrase-filter) delimited with [`AND` operator](https://docs.victoriametrics.com/victorialogs/logsql/#logical-filter). For example, `foo AND bar` query must find [log messages](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) with `foo` and `bar` words at faster speed.
|
||||
|
||||
* BUGFIX: prevent from possible corruption of short [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) during data ingestion.
|
||||
* BUGFIX: prevent from additional CPU usage for up to a few seconds after canceling the query.
|
||||
* BUGFIX: prevent from returning log entries with emtpy `_stream` field in the form `"_stream":""` in [search query results](https://docs.victoriametrics.com/victorialogs/querying/). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6042).
|
||||
|
||||
## [v0.5.2](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v0.5.2-victorialogs)
|
||||
|
||||
Released at 2024-04-11
|
||||
|
|
|
@ -19,7 +19,7 @@ It provides the following features:
|
|||
See [word filter](#word-filter), [phrase filter](#phrase-filter) and [prefix filter](#prefix-filter).
|
||||
- Ability to combine filters into arbitrary complex [logical filters](#logical-filter).
|
||||
- Ability to extract structured fields from unstructured logs at query time. See [these docs](#transformations).
|
||||
- Ability to calculate various stats over the selected log entries. See [these docs](#stats).
|
||||
- Ability to calculate various stats over the selected log entries. See [these docs](#stats-pipe).
|
||||
|
||||
## LogsQL tutorial
|
||||
|
||||
|
@ -83,20 +83,8 @@ So LogsQL allows omitting `AND` words. For example, the following query is equiv
|
|||
error _time:5m
|
||||
```
|
||||
|
||||
The query returns the following [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) by default:
|
||||
|
||||
- [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field)
|
||||
- [`_stream` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields)
|
||||
- [`_time` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field)
|
||||
|
||||
Logs may contain arbitrary number of other fields. If you need obtaining some of these fields in query results,
|
||||
then just refer them in the query with `field_name:*` [filter](#any-value-filter). See [these docs](#querying-specific-fields) for more details.
|
||||
|
||||
For example, the following query returns `host.hostname` field additionally to `_msg`, `_stream` and `_time` fields:
|
||||
|
||||
```logsql
|
||||
error _time:5m host.hostname:*
|
||||
```
|
||||
The query returns all the [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) by default.
|
||||
See [how to query specific fields](#querying-specific-fields).
|
||||
|
||||
Suppose the query above selects too many rows because some buggy app pushes invalid error logs to VictoriaLogs. Suppose the app adds `buggy_app` [word](#word) to every log line.
|
||||
Then the following query removes all the logs from the buggy app, allowing us paying attention to the real errors:
|
||||
|
@ -189,17 +177,22 @@ These words are taken into account by full-text search filters such as
|
|||
|
||||
#### Query syntax
|
||||
|
||||
LogsQL query consists of the following parts delimited by `|`:
|
||||
LogsQL query must contain [filters](#filters) for selecting the matching logs. At least a single filter is required.
|
||||
For example, the following query selects all the logs for the last 5 minutes by using [`_time` filter](#time-filter):
|
||||
|
||||
- [Filters](#filters), which select log entries for further processing. This part is required in LogsQL. Other parts are optional.
|
||||
- Optional [stream context](#stream-context), which allows selecting surrounding log lines for the matching log lines.
|
||||
- Optional [transformations](#transformations) for the selected log fields.
|
||||
For example, an additional fields can be extracted or constructed from existing fields.
|
||||
- Optional [post-filters](#post-filters) for post-filtering of the selected results. For example, post-filtering can filter
|
||||
results based on the fields constructed by [transformations](#transformations).
|
||||
- Optional [stats](#stats) transformations, which can calculate various stats across selected results.
|
||||
- Optional [sorting](#sorting), which can sort the results by the sepcified fields.
|
||||
- Optional [limiters](#limiters), which can apply various limits on the selected results.
|
||||
```logsql
|
||||
_time:5m
|
||||
```
|
||||
|
||||
Additionally to filters, LogQL query may contain arbitrary mix of optional actions for processing the selected logs. These actions are delimited by `|` and are known as `pipes`.
|
||||
For example, the following query uses [`stats` pipe](#stats-pipe) for returning the number of [log messages](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field)
|
||||
with the `error` [word](#word) for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m error | stats count() errors
|
||||
```
|
||||
|
||||
See [the list of supported pipes in LogsQL](#pipes).
|
||||
|
||||
## Filters
|
||||
|
||||
|
@ -839,6 +832,12 @@ parentheses with square brackets. For example:
|
|||
- `range(1, 10]` includes `10` in the matching range
|
||||
- `range[1, 10]` includes `1` and `10` in the matching range
|
||||
|
||||
The range boundaries can be expressed in the following forms:
|
||||
|
||||
- Hexadecimal form. For example, `range(0xff, 0xABCD)`.
|
||||
- Binary form. Form example, `range(0b100110, 0b11111101)`
|
||||
- Integer form with `_` delimiters for better readability. For example, `range(1_000, 2_345_678)`.
|
||||
|
||||
Note that the `range()` filter doesn't match [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||
with non-numeric values alongside numeric values. For example, `range(1, 10)` doesn't match `the request took 4.2 seconds`
|
||||
[log message](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field), since the `4.2` number is surrounded by other text.
|
||||
|
@ -950,6 +949,19 @@ This query doesn't match the following log messages:
|
|||
- `foo`, since it is too short
|
||||
- `foo bar baz abc`, sinc it is too long
|
||||
|
||||
It is possible to use `inf` as the upper bound. For example, the following query matches [log messages](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field)
|
||||
with the length bigger or equal to 5 chars:
|
||||
|
||||
```logsql
|
||||
len_range(5, inf)
|
||||
```
|
||||
|
||||
The range boundaries can be expressed in the following forms:
|
||||
|
||||
- Hexadecimal form. For example, `len_range(0xff, 0xABCD)`.
|
||||
- Binary form. Form example, `len_range(0b100110, 0b11111101)`
|
||||
- Integer form with `_` delimiters for better readability. For example, `len_range(1_000, 2_345_678)`.
|
||||
|
||||
By default the `len_range()` is applied to the [`_msg` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field).
|
||||
Put the [field name](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) in front of the `len_range()` in order to apply
|
||||
the filter to the needed field. For example, the following query matches log entries with the `foo` field length in the range `[10, 20]` chars:
|
||||
|
@ -1018,6 +1030,552 @@ Performance tips:
|
|||
|
||||
- See [other performance tips](#performance-tips).
|
||||
|
||||
## Pipes
|
||||
|
||||
Additionally to [filters](#filters), LogsQL query may contain arbitrary mix of '|'-delimited actions known as `pipes`.
|
||||
For example, the following query uses [`stats`](#stats-pipe), [`sort`](#sort-pipe) and [`limit`](#limit-pipe) pipes
|
||||
for returning top 10 [log streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields)
|
||||
with the biggest number of logs during the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats by (_stream) count() per_stream_logs | sort by (per_stream_logs desc) | limit 10
|
||||
```
|
||||
|
||||
LogsQL supports the following pipes:
|
||||
|
||||
- [`copy`](#copy-pipe) copies [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- [`delete`](#delete-pipe) deletes [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- [`fields`](#fields-pipe) selects the given set of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- [`limit`](#limit-pipe) limits the number selected logs.
|
||||
- [`offset`](#offset-pipe) skips the given number of selected logs.
|
||||
- [`rename`](#rename-pipe) renames [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- [`sort`](#sort-pipe) sorts logs by the given [fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- [`stats`](#stats-pipe) calculates various stats over the selected logs.
|
||||
- [`uniq`](#uniq-pipe) returns unique log entires.
|
||||
|
||||
### copy pipe
|
||||
|
||||
If some [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) must be copied, then `| copy src1 as dst1, ..., srcN as dstN` [pipe](#pipes) can be used.
|
||||
For example, the following query copies `host` field to `server` for logs over the last 5 minutes, so the output contains both `host` and `server` fields:
|
||||
|
||||
```logsq
|
||||
_time:5m | copy host as server
|
||||
```
|
||||
|
||||
Multiple fields can be copied with a single `| copy ...` pipe. For example, the following query copies
|
||||
[`_time` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) to `timestamp`, while [`_msg` field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field)
|
||||
is copied to `message`:
|
||||
|
||||
```logsql
|
||||
_time:5m | copy _time as timestmap, _msg as message
|
||||
```
|
||||
|
||||
The `as` keyword is optional.
|
||||
|
||||
`cp` keyword can be used instead of `copy` for convenience. For example, `_time:5m | cp foo bar` is equivalent to `_time:5m | copy foo as bar`.
|
||||
|
||||
See also:
|
||||
|
||||
- [`rename` pipe](#rename-pipe)
|
||||
- [`fields` pipe](#fields-pipe)
|
||||
- [`delete` pipe](#delete-pipe)
|
||||
|
||||
### delete pipe
|
||||
|
||||
If some [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) must be deleted, then `| delete field1, ..., fieldN` [pipe](#pipes) can be used.
|
||||
For example, the following query deletes `host` and `app` fields from the logs over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | delete host, app
|
||||
```
|
||||
|
||||
`del` and `rm` keywords can be used instead of `delete` for convenience. For example, `_time:5m | del host` is equivalent to `_time:5m | rm host` and `_time:5m | delete host`.
|
||||
|
||||
See also:
|
||||
|
||||
- [`rename` pipe](#rename-pipe)
|
||||
- [`fields` pipe](#fields-pipe)
|
||||
|
||||
### fields pipe
|
||||
|
||||
By default all the [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) are returned in the response.
|
||||
It is possible to select the given set of log fields with `| fields field1, ..., fieldN` [pipe](#pipes). For example, the following query selects only `host`
|
||||
and [`_msg`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#message-field) fields from logs for the last 5 minutes:
|
||||
|
||||
```logsq
|
||||
_time:5m | fields host, _msg
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`copy` pipe](#copy-pipe)
|
||||
- [`rename` pipe](#rename-pipe)
|
||||
- [`delete` pipe](#delete-pipe)
|
||||
|
||||
### limit pipe
|
||||
|
||||
If only a subset of selected logs must be processed, then `| limit N` [pipe](#pipes) can be used, where `N` can contain any [supported integer numeric value](#numeric-values).
|
||||
For example, the following query returns up to 100 logs over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | limit 100
|
||||
```
|
||||
|
||||
`head` keyword can be used instead of `limit` for convenience. For example, `_time:5m | head 100` is equivalent to `_time:5m | limit 100`.
|
||||
|
||||
By default rows are selected in arbitrary order because of performance reasons, so the query above can return different sets of logs every time it is executed.
|
||||
[`sort` pipe](#sort-pipe) can be used for making sure the logs are in the same order before applying `limit ...` to them.
|
||||
|
||||
See also:
|
||||
|
||||
- [`offset` pipe](#offset-pipe)
|
||||
|
||||
### offset pipe
|
||||
|
||||
If some selected logs must be skipped after [`sort`](#sort-pipe), then `| offset N` [pipe](#pipes) can be used, where `N` can contain any [supported integer numeric value](#numeric-values).
|
||||
For example, the following query skips the first 100 logs over the last 5 minutes after soring them by [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field):
|
||||
|
||||
```logsql
|
||||
_time:5m | sort by (_time) | offset 100
|
||||
```
|
||||
|
||||
`skip` keyword can be used instead of `offset` keyword for convenience. For example, `_time:5m | skip 10` is equivalent to `_time:5m | offset 10`.
|
||||
|
||||
Note that skipping rows without sorting has little sense, since they can be returned in arbitrary order because of performance reasons.
|
||||
Rows can be sorted with [`sort` pipe](#sort-pipe).
|
||||
|
||||
See also:
|
||||
|
||||
- [`limit` pipe](#limit-pipe)
|
||||
|
||||
### rename pipe
|
||||
|
||||
If some [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) must be renamed, then `| rename src1 as dst1, ..., srcN as dstN` [pipe](#pipes) can be used.
|
||||
For example, the following query renames `host` field to `server` for logs over the last 5 minutes, so the output contains `server` field instead of `host` field:
|
||||
|
||||
```logsql
|
||||
_time:5m | rename host as server
|
||||
```
|
||||
|
||||
Multiple fields can be renamed with a single `| rename ...` pipe. For example, the following query renames `host` to `instance` and `app` to `job`:
|
||||
|
||||
```logsql
|
||||
_time:5m | rename host as instance, app as job
|
||||
```
|
||||
|
||||
The `as` keyword is optional.
|
||||
|
||||
`mv` keyword can be used instead of `rename` keyword for convenience. For example, `_time:5m | mv foo bar` is equivalent to `_time:5m | rename foo as bar`.
|
||||
|
||||
See also:
|
||||
|
||||
- [`copy` pipe](#copy-pipe)
|
||||
- [`fields` pipe](#fields-pipe)
|
||||
- [`delete` pipe](#delete-pipe)
|
||||
|
||||
### sort pipe
|
||||
|
||||
By default logs are selected in arbitrary order because of performance reasons. If logs must be sorted, then `| sort by (field1, ..., fieldN)` [pipe](#pipes) must be used.
|
||||
For example, the following query returns logs for the last 5 minutes sorted by [`_stream`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields)
|
||||
and then by [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field):
|
||||
|
||||
```logsql
|
||||
_time:5m | sort by (_stream, _time)
|
||||
```
|
||||
|
||||
Add `desc` after the given log field in order to sort in reverse order of this field. For example, the following query sorts log fields in reverse order of `request_duration_seconds` field:
|
||||
|
||||
```logsql
|
||||
_time:5m | sort by (request_duration_seconds desc)
|
||||
```
|
||||
|
||||
The reverse order can be applied globally via `desc` keyword after `by(...)` clause:
|
||||
|
||||
```logsql
|
||||
_time:5m | sort by (foo, bar) desc
|
||||
```
|
||||
|
||||
Note that sorting of big number of logs can be slow and can consume a lot of additional memory.
|
||||
It is recommended limiting the number of logs before sorting with the following approaches:
|
||||
|
||||
- Reducing the selected time range with [time filter](#time-filter).
|
||||
- Using more specific [filters](#filters), so they select less logs.
|
||||
|
||||
See also:
|
||||
|
||||
- [`stats` pipe](#stats-pipe)
|
||||
- [`limit` pipe](#limit-pipe)
|
||||
- [`offset` pipe](#offset-pipe)
|
||||
|
||||
### uniq pipe
|
||||
|
||||
`| uniq ...` pipe allows returning only unique results over the selected logs. For example, the following LogsQL query
|
||||
returns uniq values for `ip` [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | uniq by (ip)
|
||||
```
|
||||
|
||||
It is possible to specify multiple fields inside `by(...)` clause. In this case all the unique sets for the given fields
|
||||
are returned. For example, the following query returns all the unique `(host, path)` pairs for the logs over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | uniq by (host, path)
|
||||
```
|
||||
|
||||
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.
|
||||
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:
|
||||
|
||||
```logsql
|
||||
_time:5m | uniq by (host, path) limit 100
|
||||
```
|
||||
|
||||
### stats pipe
|
||||
|
||||
`| stats ...` pipe allows calculating various stats over the selected logs. For example, the following LogsQL query
|
||||
uses [`count` stats function](#count-stats) for calculating the number of logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count() logs_total
|
||||
```
|
||||
|
||||
`| stats ...` pipe has the following basic format:
|
||||
|
||||
```logsql
|
||||
... | stats
|
||||
stats_func1(...) as result_name1,
|
||||
...
|
||||
stats_funcN(...) as result_nameN
|
||||
```
|
||||
|
||||
Where `stats_func*` is any of the supported [stats function](#stats-pipe-functions), while `result_name*` is the name of the log field
|
||||
to store the result of the corresponding stats function. The `as` keyword is optional.
|
||||
|
||||
For example, the following query calculates the following stats for logs over the last 5 minutes:
|
||||
|
||||
- the number of logs with the help of [`count` stats function](#count-stats);
|
||||
- the number of unique [log streams](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) with the help of [`count_uniq` stats function](#count_uniq-stats):
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count() logs_total, count_uniq(_stream) streams_total
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`sort` pipe](#sort-pipe)
|
||||
|
||||
|
||||
#### Stats by fields
|
||||
|
||||
The following LogsQL syntax can be used for calculating independent stats per group of log fields:
|
||||
|
||||
```logsql
|
||||
... | stats by (field1, ..., fieldM)
|
||||
stats_func1(...) as result_name1,
|
||||
...
|
||||
stats_funcN(...) as result_nameN
|
||||
```
|
||||
|
||||
This calculates `stats_func*` per each `(field1, ..., fieldM)` group of [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
|
||||
For example, the following query calculates the number of logs and unique ip addresses over the last 5 minutes,
|
||||
grouped by `(host, path)` fields:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats by (host, path) count() logs_total, count_uniq(ip) ips_total
|
||||
```
|
||||
|
||||
#### Stats by time buckets
|
||||
|
||||
The following syntax can be used for calculating stats grouped by time buckets:
|
||||
|
||||
```logsql
|
||||
... | stats by (_time:step)
|
||||
stats_func1(...) as result_name1,
|
||||
...
|
||||
stats_funcN(...) as result_nameN
|
||||
```
|
||||
|
||||
This calculates `stats_func*` per each `step` of [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) field.
|
||||
The `step` can have any [duration value](#duration-values). For example, the following LogsQL query returns per-minute number of logs and unique ip addresses
|
||||
over the last 5 minutes:
|
||||
|
||||
```
|
||||
_time:5m | stats by (_time:1m) count() logs_total, count_uniq(ip) ips_total
|
||||
```
|
||||
|
||||
Additionally, the following `step` values are supported:
|
||||
|
||||
- `nanosecond` - equals to `1ns` [duration](#duration-values).
|
||||
- `microsecond` - equals to `1µs` [duration](#duration-values).
|
||||
- `millisecond` - equals to `1ms` [duration](#duration-values).
|
||||
- `second` - equals to `1s` [duration](#duration-values).
|
||||
- `minute` - equals to `1m` [duration](#duration-values).
|
||||
- `hour` - equalst to `1h` [duration](#duration-values).
|
||||
- `day` - equals to `1d` [duration](#duration-values).
|
||||
- `week` - equals to `1w` [duration](#duration-values).
|
||||
- `month` - equals to one month. It properly takes into account the number of days per each month.
|
||||
- `year` - equals to one year. It properly takes into account the number of days per each year.
|
||||
|
||||
#### Stats by time buckets with timezone offset
|
||||
|
||||
VictoriaLogs stores [`_time`](https://docs.victoriametrics.com/victorialogs/keyconcepts/#time-field) values as [Unix time](https://en.wikipedia.org/wiki/Unix_time)
|
||||
in nanoseconds. This time corresponds to [UTC](https://en.wikipedia.org/wiki/Coordinated_Universal_Time) time zone. Sometimes it is needed calculating stats
|
||||
grouped by days or weeks at non-UTC timezone. This is possible with the following syntax:
|
||||
|
||||
```logsql
|
||||
... | stats by (_time:step offset timezone_offset) ...
|
||||
```
|
||||
|
||||
For example, the following query calculates per-day number of logs over the last week, in `UTC+02:00` [time zone](https://en.wikipedia.org/wiki/Time_zone):
|
||||
|
||||
```logsql
|
||||
_time:1w | stats by (_time:1d offset 2h) count() logs_total
|
||||
```
|
||||
|
||||
#### Stats by field buckets
|
||||
|
||||
Every log field inside `| stats by (...)` can be bucketed in the same way at `_time` field in [this example](#stats-by-time-buckets).
|
||||
Any [numeric value](#numeric-values) can be used as `step` value for the bucket. For example, the following query calculates
|
||||
the number of requests for the last hour, bucketed by 10KB of `request_size_bytes` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model):
|
||||
|
||||
```logsql
|
||||
_time:1h | stats by (request_size_bytes:10KB) count() requests
|
||||
```
|
||||
|
||||
#### Stats by IPv4 buckets
|
||||
|
||||
Stats can be bucketed by [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) containing [IPv4 addresses](https://en.wikipedia.org/wiki/IP_address)
|
||||
via the `ip_field_name:/network_mask` syntax inside `by(...)` clause. For example, the following query returns the number of log entries per `/24` subnetwork
|
||||
extracted from the `ip` [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) during the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats by (ip:/24) count() requests_per_subnet
|
||||
```
|
||||
|
||||
## stats pipe functions
|
||||
|
||||
LogsQL supports the following functions for [`stats` pipe](#stats-pipe):
|
||||
|
||||
- [`avg`](#avg-stats) calculates the average value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`count`](#count-stats) calculates the number of log entries.
|
||||
- [`count_empty`](#count_empty-stats) calculates the number logs with empty [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`count_uniq`](#count_uniq-stats) calculates the number of unique non-empty values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`max`](#max-stats) calcualtes the maximum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`min`](#min-stats) calculates the minumum value over the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`sum`](#sum-stats) calculates the sum for the given numeric [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`uniq_values`](#uniq_values-stats) returns unique non-empty values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
- [`values`](#values-stats) returns all the values for the given [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
|
||||
### avg stats
|
||||
|
||||
`avg(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the average value across
|
||||
all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
Non-numeric values are ignored.
|
||||
|
||||
For example, the following query returns the average value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats avg(duration) avg_duration
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`min`](#min-stats)
|
||||
- [`max`](#max-stats)
|
||||
- [`sum`](#sum-stats)
|
||||
- [`count`](#count-stats)
|
||||
|
||||
### count stats
|
||||
|
||||
`count()` calculates the number of selected logs.
|
||||
|
||||
For example, the following query returns the number of logs over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count() logs
|
||||
```
|
||||
|
||||
It is possible calculating the number of logs with non-empty values for some [log field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
with the `count(fieldName)` syntax. For example, the following query returns the number of logs with non-empty `username` field over the last 5 minutes:
|
||||
|
||||
```logsq
|
||||
_time:5m | stats count(username) logs_with_username
|
||||
```
|
||||
|
||||
If multiple fields are enumerated inside `count()`, then it counts the number of logs with at least a single non-empty field mentioned inside `count()`.
|
||||
For example, the following query returns the number of logs with non-empty `username` or `password` [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count(username, password) logs_with_username_or_password
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`count_uniq`](#count_uniq-stats)
|
||||
- [`count_empty`](#count_empty-stats)
|
||||
- [`sum`](#sum-stats)
|
||||
- [`avg`](#avg-stats)
|
||||
|
||||
### count_empty stats
|
||||
|
||||
`count_empty(field1, ..., fieldN)` calculates the number of logs with empty `(field1, ..., fieldN)` tuples.
|
||||
|
||||
For example, the following query calculates the number of logs with empty `username` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
during the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count_empty(username) logs_with_missing_username
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`count`](#count-stats)
|
||||
- [`count_uniq`](#count_uniq-stats)
|
||||
|
||||
### count_uniq stats
|
||||
|
||||
`count_uniq(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the number of unique non-empty `(field1, ..., fieldN)` tuples.
|
||||
|
||||
For example, the following query returns the number of unique non-empty values for `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count_uniq(ip) ips
|
||||
```
|
||||
|
||||
The following query returns the number of unique `(host, path)` pairs for the corresponding [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count_uniq(host, path) unique_host_path_pairs
|
||||
```
|
||||
|
||||
Every unique value is stored in memory during query execution. Big number of unique values may require a lot of memory.
|
||||
Sometimes it is needed to know whether the number of unique values reaches some limit. In this case add `limit N` just after `count_uniq(...)`
|
||||
for limiting the number of counted unique values up to `N`, while limiting the maximum memory usage. For example, the following query counts
|
||||
up to `1_000_000` unique values for the `ip` field:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats count_uniq(ip) limit 1_000_000 as ips_1_000_000
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`uniq_values`](#uniq_values-stats)
|
||||
- [`count`](#count-stats)
|
||||
|
||||
### max stats
|
||||
|
||||
`max(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the maximum value across
|
||||
all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
Non-numeric values are ignored.
|
||||
|
||||
For example, the following query returns the maximum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats max(duration) max_duration
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`min`](#min-stats)
|
||||
- [`avg`](#avg-stats)
|
||||
- [`sum`](#sum-stats)
|
||||
- [`count`](#count-stats)
|
||||
|
||||
### min stats
|
||||
|
||||
`min(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the minimum value across
|
||||
all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
Non-numeric values are ignored.
|
||||
|
||||
For example, the following query returns the minimum value for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats min(duration) min_duration
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`max`](#max-stats)
|
||||
- [`avg`](#avg-stats)
|
||||
- [`sum`](#sum-stats)
|
||||
- [`count`](#count-stats)
|
||||
|
||||
### sum stats
|
||||
|
||||
`sum(field1, ..., fieldN)` [stats pipe](#stats-pipe) calculates the sum of numeric values across
|
||||
all the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
|
||||
For example, the following query returns the sum of numeric values for the `duration` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats sum(duration) sum_duration
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`count`](#count-stats)
|
||||
- [`avg`](#avg-stats)
|
||||
- [`max`](#max-stats)
|
||||
- [`min`](#min-stats)
|
||||
|
||||
### uniq_values stats
|
||||
|
||||
`uniq_values(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns the unique non-empty values across
|
||||
the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
The returned values are sorted and encoded in JSON array.
|
||||
|
||||
For example, the following query returns unique non-empty values for the `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats uniq_values(ip) unique_ips
|
||||
```
|
||||
|
||||
Every unique value is stored in memory during query execution. Big number of unique values may require a lot of memory. Sometimes it is enough to return
|
||||
only a subset of unique values. In this case add `limit N` after `uniq_values(...)` in order to limit the number of returned unique values to `N`,
|
||||
while limiting the maximum memory usage.
|
||||
For example, the following query returns up to `100` unique values for the `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over the logs for the last 5 minutes. Note that arbitrary subset of unique `ip` values is returned every time:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats uniq_values(ip) limit 100 as unique_ips_100
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`values`](#values-stats)
|
||||
- [`count_uniq`](#count_uniq-stats)
|
||||
- [`count`](#count-stats)
|
||||
|
||||
### values stats
|
||||
|
||||
`values(field1, ..., fieldN)` [stats pipe](#stats-pipe) returns all the values (including empty values)
|
||||
for the mentioned [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
The returned values are encoded in JSON array.
|
||||
|
||||
For example, the following query returns all the values for the `ip` [field](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
|
||||
over logs for the last 5 minutes:
|
||||
|
||||
```logsql
|
||||
_time:5m | stats values(ip) ips
|
||||
```
|
||||
|
||||
See also:
|
||||
|
||||
- [`uniq_values`](#uniq_values-stats)
|
||||
- [`count`](#count-stats)
|
||||
- [`count_empty`](#count_empty-stats)
|
||||
|
||||
## Stream context
|
||||
|
||||
LogsQL will support the ability to select the given number of surrounding log lines for the selected log lines
|
||||
|
@ -1039,11 +1597,9 @@ LogsQL will support the following transformations for the [selected](#filters) l
|
|||
- Creating a new field from existing [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||
according to the provided format.
|
||||
- Creating a new field according to math calculations over existing [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- Copying of the existing [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
- Parsing duration strings into floating-point seconds for further [stats calculations](#stats).
|
||||
- Parsing duration strings into floating-point seconds for further [stats calculations](#stats-pipe).
|
||||
- Creating a boolean field with the result of arbitrary [post-filters](#post-filters) applied to the current fields.
|
||||
Boolean fields may be useful for [conditional stats calculation](#stats).
|
||||
- Creating an integer field with the length of the given field value. This can be useful for [stats calculations](#stats).
|
||||
- Creating an integer field with the length of the given field value. This can be useful for [stats calculations](#stats-pipe).
|
||||
|
||||
See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) for details.
|
||||
|
||||
|
@ -1062,69 +1618,80 @@ See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) fo
|
|||
|
||||
## Stats
|
||||
|
||||
It is possible to perform stats calculations on the [selected log entries](#filters) at client side with `sort`, `uniq`, etc. Unix commands
|
||||
according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/querying/#command-line).
|
||||
Stats over the selected logs can be calculated via [`stats` pipe](#stats-pipe).
|
||||
|
||||
LogsQL will support calculating the following stats based on the [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||
LogsQL will support calculating the following additional stats based on the [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||
and fields created by [transformations](#transformations):
|
||||
|
||||
- The number of selected logs.
|
||||
- The number of non-empty values for the given field.
|
||||
- The number of unique values for the given field.
|
||||
- The min, max, avg, and sum for the given field.
|
||||
- The median and [percentile](https://en.wikipedia.org/wiki/Percentile) for the given field.
|
||||
|
||||
It will be possible specifying an optional condition [filter](#post-filters) when calculating the stats.
|
||||
For example, `sumIf(response_size, is_admin:true)` calculates the total response size for admins only.
|
||||
For example, `sum(response_size) if (is_admin:true)` calculates the total response size for admins only.
|
||||
|
||||
It will be possible to group stats by the specified [fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model)
|
||||
and by the specified time buckets.
|
||||
|
||||
See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) for details.
|
||||
It is possible to perform stats calculations on the [selected log entries](#filters) at client side with `sort`, `uniq`, etc. Unix commands
|
||||
according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/querying/#command-line).
|
||||
|
||||
## Sorting
|
||||
|
||||
By default VictoriaLogs sorts the returned results by [`_time` field](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field)
|
||||
if their total size doesn't exceed `-select.maxSortBufferSize` command-line value (by default it is set to one megabytes).
|
||||
Otherwise sorting is skipped because of performance and efficiency concerns described [here](https://docs.victoriametrics.com/VictoriaLogs/querying/).
|
||||
if their total size doesn't exceed `-select.maxSortBufferSize` command-line value (by default it is set to 1MB).
|
||||
Otherwise sorting is skipped because of performance reasons.
|
||||
|
||||
It is possible to sort the [selected log entries](#filters) at client side with `sort` Unix command
|
||||
according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/querying/#command-line).
|
||||
|
||||
LogsQL will support results' sorting by the given set of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model).
|
||||
|
||||
See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) for details.
|
||||
Use [`sort` pipe](#sort-pipe) for sorting the results.
|
||||
|
||||
## Limiters
|
||||
|
||||
It is possible to limit the returned results with `head`, `tail`, `less`, etc. Unix commands
|
||||
according to [these docs](https://docs.victoriametrics.com/VictoriaLogs/querying/#command-line).
|
||||
LogsQL provides the following [pipes](#pipes) for limiting the number of returned log entries:
|
||||
|
||||
LogsQL will support the ability to limit the number of returned results alongside the ability to page the returned results.
|
||||
Additionally, LogsQL will provide the ability to select fields, which must be returned in the response.
|
||||
|
||||
See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) for details.
|
||||
- [`fields`](#fields-pipe) and [`delete`](#delete-pipe) pipes allow limiting the set of [log fields](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model) to return.
|
||||
- [`limit` pipe](#limit-pipe) allows limiting the number of log entries to return.
|
||||
|
||||
## Querying specific fields
|
||||
|
||||
By default VictoriaLogs query response contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields.
|
||||
Specific log fields can be queried via [`fields` pipe](#fields-pipe).
|
||||
|
||||
If you want selecting other fields from the ingested [structured logs](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#data-model),
|
||||
then they must be mentioned in query filters. For example, if you want selecting `log.level` field, and this field isn't mentioned in the query yet, then add
|
||||
`log.level:*` [filter](#any-value-filter) filter to the end of the query.
|
||||
The `field_name:*` filter doesn't return log entries with empty or missing `field_name`. If you want returning log entries
|
||||
with and without the given field, then `(field_name:* OR field_name:"")` filter can be used.
|
||||
See the following docs for details:
|
||||
## Numeric values
|
||||
|
||||
- [Any value filter](#any-value-filter)
|
||||
- [Empty value filter](#empty-value-filter)
|
||||
- [Logical filter](#logical-filter)
|
||||
LogsQL accepts numeric values in the following formats:
|
||||
|
||||
In the future LogsQL will support `| fields field1, field2, ... fieldN` syntax for selecting the listed fields.
|
||||
It will also support the ability to select all the fields for the matching log entries with `| fields *` syntax.
|
||||
See the [Roadmap](https://docs.victoriametrics.com/VictoriaLogs/Roadmap.html) for details.
|
||||
- regular integers like `12345` or `-12345`
|
||||
- regular floating point numbers like `0.123` or `-12.34`
|
||||
- [short numeric format](#short-numeric-values)
|
||||
- [duration format](#duration-values)
|
||||
|
||||
### Short numeric values
|
||||
|
||||
LogsQL accepts integer and floating point values with the following suffixes:
|
||||
|
||||
- `K` and `KB` - the value is multiplied by `10^3`
|
||||
- `M` and `MB` - the value is multiplied by `10^6`
|
||||
- `G` and `GB` - the value is multiplied by `10^9`
|
||||
- `T` and `TB` - the value is multiplied by `10^12`
|
||||
- `Ki` and `KiB` - the value is multiplied by `2^10`
|
||||
- `Mi` and `MiB` - the value is multiplied by `2^20`
|
||||
- `Gi` and `GiB` - the value is multiplied by `2^30`
|
||||
- `Ti` and `TiB` - the value is multiplied by `2^40`
|
||||
|
||||
All the numbers may contain `_` delimiters, which may improve readability of the query. For example, `1_234_567` is equivalent to `1234567`,
|
||||
while `1.234_567` is equivalent to `1.234567`.
|
||||
|
||||
## Duration values
|
||||
|
||||
LogsQL accepts duration values with the following suffixes at places where the duration is allowed:
|
||||
|
||||
- `ns` - nanoseconds. For example, `123ns`.
|
||||
- `µs` - microseconds. For example, `1.23µs`.
|
||||
- `ms` - milliseconds. For example, `1.23456ms`
|
||||
- `s` - seconds. For example, `1.234s`
|
||||
- `m` - minutes. For example, `1.5m`
|
||||
- `h` - hours. For example, `1.5h`
|
||||
- `d` - days. For example, `1.5d`
|
||||
- `w` - weeks. For example, `1w`
|
||||
- `y` - years as 365 days. For example, `1.5y`
|
||||
|
||||
Multiple durations can be combined. For example, `1h33m55s`.
|
||||
|
||||
Internally duration values are converted into nanoseconds.
|
||||
|
||||
## Performance tips
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ See [these docs](https://docs.victoriametrics.com/VictoriaLogs/) for details.
|
|||
The following functionality is planned in the future versions of VictoriaLogs:
|
||||
|
||||
- Support for [data ingestion](https://docs.victoriametrics.com/VictoriaLogs/data-ingestion/) from popular log collectors and formats:
|
||||
- OpenTelemetry for logs
|
||||
- Fluentd
|
||||
- Syslog
|
||||
- Journald (systemd)
|
||||
|
@ -37,9 +38,6 @@ The following functionality is planned in the future versions of VictoriaLogs:
|
|||
- [Stream context](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#stream-context).
|
||||
- [Transformation functions](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#transformations).
|
||||
- [Post-filtering](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#post-filters).
|
||||
- [Stats calculations](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#stats).
|
||||
- [Sorting](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#sorting).
|
||||
- [Limiters](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#limiters).
|
||||
- The ability to use subqueries inside [in()](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#multi-exact-filter) function.
|
||||
- Live tailing for [LogsQL filters](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#filters) aka `tail -f`.
|
||||
- Web UI with the following abilities:
|
||||
|
|
|
@ -79,10 +79,8 @@ The command should return the following response:
|
|||
{"_msg":"cannot open file","_stream":"{}","_time":"2023-06-21T04:24:24Z","host.name":"host123"}
|
||||
```
|
||||
|
||||
The response by default contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields plus the explicitly mentioned fields.
|
||||
See [these docs](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields) for details.
|
||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
See [how to query specific fields](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields).
|
||||
|
||||
The duration of requests to `/insert/elasticsearch/_bulk` can be monitored with `vl_http_request_duration_seconds{path="/insert/elasticsearch/_bulk"}` metric.
|
||||
|
||||
|
@ -133,10 +131,8 @@ The command should return the following response:
|
|||
{"_msg":"oh no!","_stream":"{stream=\"stream1\"}","_time":"2023-06-20T15:32:10.567Z","log.level":"error"}
|
||||
```
|
||||
|
||||
The response by default contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields plus the explicitly mentioned fields.
|
||||
See [these docs](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields) for details.
|
||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
See [how to query specific fields](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields).
|
||||
|
||||
The duration of requests to `/insert/jsonline` can be monitored with `vl_http_request_duration_seconds{path="/insert/jsonline"}` metric.
|
||||
|
||||
|
@ -174,10 +170,8 @@ The command should return the following response:
|
|||
{"_msg":"foo fizzbuzz bar","_stream":"{instance=\"host123\",job=\"app42\"}","_time":"2023-07-20T23:01:19.288676497Z"}
|
||||
```
|
||||
|
||||
The response by default contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields plus the explicitly mentioned fields.
|
||||
See [these docs](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields) for details.
|
||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
See [how to query specific fields](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields).
|
||||
|
||||
The duration of requests to `/insert/loki/api/v1/push` can be monitored with `vl_http_request_duration_seconds{path="/insert/loki/api/v1/push"}` metric.
|
||||
|
||||
|
@ -232,10 +226,8 @@ using [any value filter](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.ht
|
|||
while `head` cancels query execution after reading the first 10 log lines. See [these docs](https://docs.victoriametrics.com/VictoriaLogs/querying/#command-line)
|
||||
for more details on how `head` integrates with VictoriaLogs.
|
||||
|
||||
The response by default contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields plus the explicitly mentioned fields.
|
||||
See [these docs](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields) for details.
|
||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
See [how to query specific fields](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields).
|
||||
|
||||
VictoriaLogs provides the following command-line flags, which can help debugging data ingestion issues:
|
||||
|
||||
|
|
|
@ -31,10 +31,8 @@ For example, the following query returns all the log entries with the `error` wo
|
|||
curl http://localhost:9428/select/logsql/query -d 'query=error'
|
||||
```
|
||||
|
||||
The response by default contains [`_msg`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#message-field),
|
||||
[`_stream`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#stream-fields) and
|
||||
[`_time`](https://docs.victoriametrics.com/VictoriaLogs/keyConcepts.html#time-field) fields plus the explicitly mentioned fields.
|
||||
See [these docs](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields) for details.
|
||||
The response by default contains all the [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
|
||||
See [how to query specific fields](https://docs.victoriametrics.com/VictoriaLogs/LogsQL.html#querying-specific-fields).
|
||||
|
||||
The `query` argument can be passed either in the request url itself (aka HTTP GET request) or via request body
|
||||
with the `x-www-form-urlencoded` encoding (aka HTTP POST request). The HTTP POST is useful for sending long queries
|
||||
|
|
36
lib/encoding/float.go
Normal file
36
lib/encoding/float.go
Normal file
|
@ -0,0 +1,36 @@
|
|||
package encoding
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// GetFloat64s returns a slice of float64 values with the given size.
|
||||
//
|
||||
// When the returned slice is no longer needed, it is advised calling PutFloat64s() on it,
|
||||
// so it could be re-used.
|
||||
func GetFloat64s(size int) *Float64s {
|
||||
v := float64sPool.Get()
|
||||
if v == nil {
|
||||
v = &Float64s{}
|
||||
}
|
||||
a := v.(*Float64s)
|
||||
a.A = slicesutil.SetLength(a.A, size)
|
||||
return a
|
||||
}
|
||||
|
||||
// PutFloat64s returns a to the pool, so it can be re-used via GetFloat64s.
|
||||
//
|
||||
// The a cannot be used after returning to the pull.
|
||||
func PutFloat64s(a *Float64s) {
|
||||
a.A = a.A[:0]
|
||||
float64sPool.Put(a)
|
||||
}
|
||||
|
||||
var float64sPool sync.Pool
|
||||
|
||||
// Float64s holds an array of float64 values.
|
||||
type Float64s struct {
|
||||
A []float64
|
||||
}
|
|
@ -4,6 +4,7 @@ import (
|
|||
"fmt"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// marshalInt64NearestDelta2 encodes src using `nearest delta2` encoding
|
||||
|
@ -69,15 +70,21 @@ func unmarshalInt64NearestDelta2(dst []int64, src []byte, firstValue int64, item
|
|||
return nil, fmt.Errorf("unexpected tail left after unmarshaling %d items from %d bytes; tail size=%d; src=%X; tail=%X", itemsCount, len(src), len(tail), src, tail)
|
||||
}
|
||||
|
||||
dstLen := len(dst)
|
||||
dst = slicesutil.SetLength(dst, dstLen+itemsCount)
|
||||
as := dst[dstLen:]
|
||||
|
||||
v := firstValue
|
||||
d1 := is.A[0]
|
||||
dst = append(dst, v)
|
||||
as[0] = v
|
||||
v += d1
|
||||
dst = append(dst, v)
|
||||
for _, d2 := range is.A[1:] {
|
||||
as[1] = v
|
||||
as = as[2:]
|
||||
for i, d2 := range is.A[1:] {
|
||||
d1 += d2
|
||||
v += d1
|
||||
dst = append(dst, v)
|
||||
as[i] = v
|
||||
}
|
||||
|
||||
return dst, nil
|
||||
}
|
||||
|
|
|
@ -1,9 +1,26 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
)
|
||||
|
||||
func getArena() *arena {
|
||||
v := arenaPool.Get()
|
||||
if v == nil {
|
||||
return &arena{}
|
||||
}
|
||||
return v.(*arena)
|
||||
}
|
||||
|
||||
func putArena(a *arena) {
|
||||
a.reset()
|
||||
arenaPool.Put(a)
|
||||
}
|
||||
|
||||
var arenaPool sync.Pool
|
||||
|
||||
type arena struct {
|
||||
b []byte
|
||||
}
|
||||
|
@ -12,7 +29,15 @@ func (a *arena) reset() {
|
|||
a.b = a.b[:0]
|
||||
}
|
||||
|
||||
func (a *arena) sizeBytes() int {
|
||||
return len(a.b)
|
||||
}
|
||||
|
||||
func (a *arena) copyBytes(b []byte) []byte {
|
||||
if len(b) == 0 {
|
||||
return b
|
||||
}
|
||||
|
||||
ab := a.b
|
||||
abLen := len(ab)
|
||||
ab = append(ab, b...)
|
||||
|
@ -21,7 +46,21 @@ func (a *arena) copyBytes(b []byte) []byte {
|
|||
return result
|
||||
}
|
||||
|
||||
func (a *arena) copyBytesToString(b []byte) string {
|
||||
bCopy := a.copyBytes(b)
|
||||
return bytesutil.ToUnsafeString(bCopy)
|
||||
}
|
||||
|
||||
func (a *arena) copyString(s string) string {
|
||||
b := bytesutil.ToUnsafeBytes(s)
|
||||
return a.copyBytesToString(b)
|
||||
}
|
||||
|
||||
func (a *arena) newBytes(size int) []byte {
|
||||
if size <= 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
ab := a.b
|
||||
abLen := len(ab)
|
||||
ab = bytesutil.ResizeWithCopyMayOverallocate(ab, abLen+size)
|
||||
|
|
80
lib/logstorage/arena_test.go
Normal file
80
lib/logstorage/arena_test.go
Normal file
|
@ -0,0 +1,80 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
)
|
||||
|
||||
func TestArena(t *testing.T) {
|
||||
values := []string{"foo", "bar", "", "adsfjkljsdfdsf", "dsfsopq", "io234"}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
a := getArena()
|
||||
if n := a.sizeBytes(); n != 0 {
|
||||
t.Fatalf("unexpected non-zero size of empty arena: %d", n)
|
||||
}
|
||||
|
||||
// add values to arena
|
||||
valuesCopy := make([]string, len(values))
|
||||
valuesLen := 0
|
||||
for j, v := range values {
|
||||
vCopy := a.copyString(v)
|
||||
if vCopy != v {
|
||||
t.Fatalf("unexpected value; got %q; want %q", vCopy, v)
|
||||
}
|
||||
valuesCopy[j] = vCopy
|
||||
valuesLen += len(v)
|
||||
}
|
||||
|
||||
// verify that the values returned from arena match the original values
|
||||
for j, v := range values {
|
||||
vCopy := valuesCopy[j]
|
||||
if vCopy != v {
|
||||
t.Fatalf("unexpected value; got %q; want %q", vCopy, v)
|
||||
}
|
||||
}
|
||||
|
||||
if n := a.sizeBytes(); n != valuesLen {
|
||||
t.Fatalf("unexpected arena size; got %d; want %d", n, valuesLen)
|
||||
}
|
||||
|
||||
// Try allocating slices with different lengths
|
||||
bs := make([]string, 100)
|
||||
for j := range bs {
|
||||
b := a.newBytes(j)
|
||||
if len(b) != j {
|
||||
t.Fatalf("unexpected len(b); got %d; want %d", len(b), j)
|
||||
}
|
||||
valuesLen += j
|
||||
if n := a.sizeBytes(); n != valuesLen {
|
||||
t.Fatalf("unexpected arena size; got %d; want %d", n, valuesLen)
|
||||
}
|
||||
for k := range b {
|
||||
b[k] = byte(k)
|
||||
}
|
||||
bs[j] = bytesutil.ToUnsafeString(b)
|
||||
}
|
||||
|
||||
// verify that the allocated slices didn't change
|
||||
for j, v := range bs {
|
||||
b := make([]byte, j)
|
||||
for k := 0; k < j; k++ {
|
||||
b[k] = byte(k)
|
||||
}
|
||||
if v != string(b) {
|
||||
t.Fatalf("unexpected value at index %d; got %X; want %X", j, v, b)
|
||||
}
|
||||
}
|
||||
|
||||
// verify that the values returned from arena match the original values
|
||||
for j, v := range values {
|
||||
vCopy := valuesCopy[j]
|
||||
if vCopy != v {
|
||||
t.Fatalf("unexpected value; got %q; want %q", vCopy, v)
|
||||
}
|
||||
}
|
||||
|
||||
putArena(a)
|
||||
}
|
||||
}
|
151
lib/logstorage/bitmap.go
Normal file
151
lib/logstorage/bitmap.go
Normal file
|
@ -0,0 +1,151 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"math/bits"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
func getBitmap(bitsLen int) *bitmap {
|
||||
v := bitmapPool.Get()
|
||||
if v == nil {
|
||||
v = &bitmap{}
|
||||
}
|
||||
bm := v.(*bitmap)
|
||||
bm.init(bitsLen)
|
||||
return bm
|
||||
}
|
||||
|
||||
func putBitmap(bm *bitmap) {
|
||||
bm.reset()
|
||||
bitmapPool.Put(bm)
|
||||
}
|
||||
|
||||
var bitmapPool sync.Pool
|
||||
|
||||
type bitmap struct {
|
||||
a []uint64
|
||||
bitsLen int
|
||||
}
|
||||
|
||||
func (bm *bitmap) reset() {
|
||||
bm.resetBits()
|
||||
bm.a = bm.a[:0]
|
||||
|
||||
bm.bitsLen = 0
|
||||
}
|
||||
|
||||
func (bm *bitmap) copyFrom(src *bitmap) {
|
||||
bm.reset()
|
||||
|
||||
bm.a = append(bm.a[:0], src.a...)
|
||||
bm.bitsLen = src.bitsLen
|
||||
}
|
||||
|
||||
func (bm *bitmap) init(bitsLen int) {
|
||||
a := bm.a
|
||||
wordsLen := (bitsLen + 63) / 64
|
||||
a = slicesutil.SetLength(a, wordsLen)
|
||||
bm.a = a
|
||||
bm.bitsLen = bitsLen
|
||||
}
|
||||
|
||||
func (bm *bitmap) resetBits() {
|
||||
a := bm.a
|
||||
for i := range a {
|
||||
a[i] = 0
|
||||
}
|
||||
}
|
||||
|
||||
func (bm *bitmap) setBits() {
|
||||
a := bm.a
|
||||
for i := range a {
|
||||
a[i] = ^uint64(0)
|
||||
}
|
||||
tailBits := bm.bitsLen % 64
|
||||
if tailBits > 0 && len(a) > 0 {
|
||||
// Zero bits outside bitsLen at the last word
|
||||
a[len(a)-1] &= (uint64(1) << tailBits) - 1
|
||||
}
|
||||
}
|
||||
|
||||
func (bm *bitmap) isZero() bool {
|
||||
for _, word := range bm.a {
|
||||
if word != 0 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (bm *bitmap) areAllBitsSet() bool {
|
||||
a := bm.a
|
||||
for i, word := range a {
|
||||
if word != (1<<64)-1 {
|
||||
if i+1 < len(a) {
|
||||
return false
|
||||
}
|
||||
tailBits := bm.bitsLen % 64
|
||||
if tailBits == 0 || word != (uint64(1)<<tailBits)-1 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (bm *bitmap) andNot(x *bitmap) {
|
||||
if bm.bitsLen != x.bitsLen {
|
||||
logger.Panicf("BUG: cannot merge bitmaps with distinct lengths; %d vs %d", bm.bitsLen, x.bitsLen)
|
||||
}
|
||||
a := bm.a
|
||||
b := x.a
|
||||
for i := range a {
|
||||
a[i] &= ^b[i]
|
||||
}
|
||||
}
|
||||
|
||||
func (bm *bitmap) or(x *bitmap) {
|
||||
if bm.bitsLen != x.bitsLen {
|
||||
logger.Panicf("BUG: cannot merge bitmaps with distinct lengths; %d vs %d", bm.bitsLen, x.bitsLen)
|
||||
}
|
||||
a := bm.a
|
||||
b := x.a
|
||||
for i := range a {
|
||||
a[i] |= b[i]
|
||||
}
|
||||
}
|
||||
|
||||
// forEachSetBit calls f for each set bit and clears that bit if f returns false
|
||||
func (bm *bitmap) forEachSetBit(f func(idx int) bool) {
|
||||
a := bm.a
|
||||
bitsLen := bm.bitsLen
|
||||
for i, word := range a {
|
||||
if word == 0 {
|
||||
continue
|
||||
}
|
||||
for j := 0; j < 64; j++ {
|
||||
mask := uint64(1) << j
|
||||
if (word & mask) == 0 {
|
||||
continue
|
||||
}
|
||||
idx := i*64 + j
|
||||
if idx >= bitsLen {
|
||||
break
|
||||
}
|
||||
if !f(idx) {
|
||||
a[i] &= ^mask
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (bm *bitmap) onesCount() int {
|
||||
n := 0
|
||||
for _, word := range bm.a {
|
||||
n += bits.OnesCount64(word)
|
||||
}
|
||||
return n
|
||||
}
|
106
lib/logstorage/bitmap_test.go
Normal file
106
lib/logstorage/bitmap_test.go
Normal file
|
@ -0,0 +1,106 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestBitmap(t *testing.T) {
|
||||
for i := 0; i < 100; i++ {
|
||||
bm := getBitmap(i)
|
||||
if bm.bitsLen != i {
|
||||
t.Fatalf("unexpected bits length: %d; want %d", bm.bitsLen, i)
|
||||
}
|
||||
|
||||
if !bm.isZero() {
|
||||
t.Fatalf("all the bits must be zero for bitmap with %d bits", i)
|
||||
}
|
||||
if i == 0 && !bm.areAllBitsSet() {
|
||||
t.Fatalf("areAllBitsSet() must return true for bitmap with 0 bits")
|
||||
}
|
||||
if i > 0 && bm.areAllBitsSet() {
|
||||
t.Fatalf("areAllBitsSet() must return false on new bitmap with %d bits; %#v", i, bm)
|
||||
}
|
||||
if n := bm.onesCount(); n != 0 {
|
||||
t.Fatalf("unexpected number of set bits; got %d; want %d", n, 0)
|
||||
}
|
||||
|
||||
bm.setBits()
|
||||
|
||||
if n := bm.onesCount(); n != i {
|
||||
t.Fatalf("unexpected number of set bits; got %d; want %d", n, i)
|
||||
}
|
||||
|
||||
// Make sure that all the bits are set.
|
||||
nextIdx := 0
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
if idx >= i {
|
||||
t.Fatalf("index must be smaller than %d", i)
|
||||
}
|
||||
if idx != nextIdx {
|
||||
t.Fatalf("unexpected idx; got %d; want %d", idx, nextIdx)
|
||||
}
|
||||
nextIdx++
|
||||
return true
|
||||
})
|
||||
|
||||
if !bm.areAllBitsSet() {
|
||||
t.Fatalf("all bits must be set for bitmap with %d bits", i)
|
||||
}
|
||||
|
||||
// Clear a part of bits
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
return idx%2 != 0
|
||||
})
|
||||
|
||||
if i <= 1 && !bm.isZero() {
|
||||
t.Fatalf("bm.isZero() must return true for bitmap with %d bits", i)
|
||||
}
|
||||
if i > 1 && bm.isZero() {
|
||||
t.Fatalf("bm.isZero() must return false, since some bits are set for bitmap with %d bits", i)
|
||||
}
|
||||
if i == 0 && !bm.areAllBitsSet() {
|
||||
t.Fatalf("areAllBitsSet() must return true for bitmap with 0 bits")
|
||||
}
|
||||
if i > 0 && bm.areAllBitsSet() {
|
||||
t.Fatalf("some bits mustn't be set for bitmap with %d bits", i)
|
||||
}
|
||||
|
||||
nextIdx = 1
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
if idx != nextIdx {
|
||||
t.Fatalf("unexpected idx; got %d; want %d", idx, nextIdx)
|
||||
}
|
||||
nextIdx += 2
|
||||
return true
|
||||
})
|
||||
|
||||
// Clear all the bits
|
||||
bm.forEachSetBit(func(_ int) bool {
|
||||
return false
|
||||
})
|
||||
|
||||
if !bm.isZero() {
|
||||
t.Fatalf("all the bits must be reset for bitmap with %d bits", i)
|
||||
}
|
||||
if i == 0 && !bm.areAllBitsSet() {
|
||||
t.Fatalf("allAllBitsSet() must return true for bitmap with 0 bits")
|
||||
}
|
||||
if i > 0 && bm.areAllBitsSet() {
|
||||
t.Fatalf("areAllBitsSet() must return false for bitmap with %d bits", i)
|
||||
}
|
||||
if n := bm.onesCount(); n != 0 {
|
||||
t.Fatalf("unexpected number of set bits; got %d; want %d", n, 0)
|
||||
}
|
||||
|
||||
bitsCount := 0
|
||||
bm.forEachSetBit(func(_ int) bool {
|
||||
bitsCount++
|
||||
return true
|
||||
})
|
||||
if bitsCount != 0 {
|
||||
t.Fatalf("unexpected non-zero number of set bits remained: %d", bitsCount)
|
||||
}
|
||||
|
||||
putBitmap(bm)
|
||||
}
|
||||
}
|
|
@ -8,6 +8,7 @@ import (
|
|||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// block represents a block of log entries.
|
||||
|
@ -120,11 +121,8 @@ type column struct {
|
|||
func (c *column) reset() {
|
||||
c.name = ""
|
||||
|
||||
values := c.values
|
||||
for i := range values {
|
||||
values[i] = ""
|
||||
}
|
||||
c.values = values[:0]
|
||||
clear(c.values)
|
||||
c.values = c.values[:0]
|
||||
}
|
||||
|
||||
func (c *column) canStoreInConstColumn() bool {
|
||||
|
@ -145,17 +143,14 @@ func (c *column) canStoreInConstColumn() bool {
|
|||
}
|
||||
|
||||
func (c *column) resizeValues(valuesLen int) []string {
|
||||
values := c.values
|
||||
if n := valuesLen - cap(values); n > 0 {
|
||||
values = append(values[:cap(values)], make([]string, n)...)
|
||||
}
|
||||
values = values[:valuesLen]
|
||||
c.values = values
|
||||
return values
|
||||
c.values = slicesutil.SetLength(c.values, valuesLen)
|
||||
return c.values
|
||||
}
|
||||
|
||||
// mustWriteTo writes c to sw and updates ch accordingly.
|
||||
func (c *column) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
||||
//
|
||||
// ch is valid until a.reset() is called.
|
||||
func (c *column) mustWriteTo(a *arena, ch *columnHeader, sw *streamWriters) {
|
||||
ch.reset()
|
||||
|
||||
valuesWriter := &sw.fieldValuesWriter
|
||||
|
@ -165,7 +160,7 @@ func (c *column) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
|||
bloomFilterWriter = &sw.messageBloomFilterWriter
|
||||
}
|
||||
|
||||
ch.name = c.name
|
||||
ch.name = a.copyString(c.name)
|
||||
|
||||
// encode values
|
||||
ve := getValuesEncoder()
|
||||
|
@ -191,7 +186,7 @@ func (c *column) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
|||
bb.B = bloomFilterMarshal(bb.B[:0], tokensBuf.A)
|
||||
putTokensBuf(tokensBuf)
|
||||
} else {
|
||||
// there is no need in ecoding bloom filter for dictiory type,
|
||||
// there is no need in ecoding bloom filter for dictionary type,
|
||||
// since it isn't used during querying - all the dictionary values are available in ch.valuesDict
|
||||
bb.B = bb.B[:0]
|
||||
}
|
||||
|
@ -226,6 +221,8 @@ func (b *block) assertValid() {
|
|||
// MustInitFromRows initializes b from the given timestamps and rows.
|
||||
//
|
||||
// It is expected that timestamps are sorted.
|
||||
//
|
||||
// b is valid until rows are changed.
|
||||
func (b *block) MustInitFromRows(timestamps []int64, rows [][]Field) {
|
||||
b.reset()
|
||||
|
||||
|
@ -235,6 +232,9 @@ func (b *block) MustInitFromRows(timestamps []int64, rows [][]Field) {
|
|||
b.sortColumnsByName()
|
||||
}
|
||||
|
||||
// mustInitiFromRows initializes b from rows.
|
||||
//
|
||||
// b is valid until rows are changed.
|
||||
func (b *block) mustInitFromRows(rows [][]Field) {
|
||||
rowsLen := len(rows)
|
||||
if rowsLen == 0 {
|
||||
|
@ -366,13 +366,8 @@ func (b *block) extendColumns() *column {
|
|||
}
|
||||
|
||||
func (b *block) resizeColumns(columnsLen int) []column {
|
||||
cs := b.columns[:0]
|
||||
if n := columnsLen - cap(cs); n > 0 {
|
||||
cs = append(cs[:cap(cs)], make([]column, n)...)
|
||||
}
|
||||
cs = cs[:columnsLen]
|
||||
b.columns = cs
|
||||
return cs
|
||||
b.columns = slicesutil.SetLength(b.columns, columnsLen)
|
||||
return b.columns
|
||||
}
|
||||
|
||||
func (b *block) sortColumnsByName() {
|
||||
|
@ -424,23 +419,23 @@ func (b *block) InitFromBlockData(bd *blockData, sbu *stringsBlockUnmarshaler, v
|
|||
for i := range cds {
|
||||
cd := &cds[i]
|
||||
c := &cs[i]
|
||||
c.name = cd.name
|
||||
c.name = sbu.copyString(cd.name)
|
||||
c.values, err = sbu.unmarshal(c.values[:0], cd.valuesData, uint64(rowsCount))
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot unmarshal column %d: %w", i, err)
|
||||
}
|
||||
if err = vd.decodeInplace(c.values, cd.valueType, &cd.valuesDict); err != nil {
|
||||
if err = vd.decodeInplace(c.values, cd.valueType, cd.valuesDict.values); err != nil {
|
||||
return fmt.Errorf("cannot decode column values: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
// unmarshal constColumns
|
||||
b.constColumns = append(b.constColumns[:0], bd.constColumns...)
|
||||
b.constColumns = sbu.appendFields(b.constColumns[:0], bd.constColumns)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// mustWriteTo writes b with the given sid to sw and updates bh accordingly
|
||||
// mustWriteTo writes b with the given sid to sw and updates bh accordingly.
|
||||
func (b *block) mustWriteTo(sid *streamID, bh *blockHeader, sw *streamWriters) {
|
||||
// Do not store the version used for encoding directly in the block data, since:
|
||||
// - all the blocks in the same part use the same encoding
|
||||
|
@ -458,16 +453,22 @@ func (b *block) mustWriteTo(sid *streamID, bh *blockHeader, sw *streamWriters) {
|
|||
|
||||
// Marshal columns
|
||||
cs := b.columns
|
||||
|
||||
a := getArena()
|
||||
csh := getColumnsHeader()
|
||||
|
||||
chs := csh.resizeColumnHeaders(len(cs))
|
||||
for i := range cs {
|
||||
cs[i].mustWriteTo(&chs[i], sw)
|
||||
cs[i].mustWriteTo(a, &chs[i], sw)
|
||||
}
|
||||
csh.constColumns = append(csh.constColumns[:0], b.constColumns...)
|
||||
csh.constColumns = appendFields(a, csh.constColumns[:0], b.constColumns)
|
||||
|
||||
bb := longTermBufPool.Get()
|
||||
bb.B = csh.marshal(bb.B)
|
||||
|
||||
putColumnsHeader(csh)
|
||||
putArena(a)
|
||||
|
||||
bh.columnsHeaderOffset = sw.columnsHeaderWriter.bytesWritten
|
||||
bh.columnsHeaderSize = uint64(len(bb.B))
|
||||
if bh.columnsHeaderSize > maxColumnsHeaderSize {
|
||||
|
@ -489,13 +490,7 @@ func (b *block) appendRowsTo(dst *rows) {
|
|||
for i := range b.timestamps {
|
||||
fieldsLen := len(fieldsBuf)
|
||||
// copy const columns
|
||||
for j := range ccs {
|
||||
cc := &ccs[j]
|
||||
fieldsBuf = append(fieldsBuf, Field{
|
||||
Name: cc.Name,
|
||||
Value: cc.Value,
|
||||
})
|
||||
}
|
||||
fieldsBuf = append(fieldsBuf, ccs...)
|
||||
// copy other columns
|
||||
for j := range cs {
|
||||
c := &cs[j]
|
||||
|
@ -520,7 +515,7 @@ func areSameFieldsInRows(rows [][]Field) bool {
|
|||
fields := rows[0]
|
||||
|
||||
// Verify that all the field names are unique
|
||||
m := make(map[string]struct{}, len(fields))
|
||||
m := getFieldsSet()
|
||||
for i := range fields {
|
||||
f := &fields[i]
|
||||
if _, ok := m[f.Name]; ok {
|
||||
|
@ -529,6 +524,7 @@ func areSameFieldsInRows(rows [][]Field) bool {
|
|||
}
|
||||
m[f.Name] = struct{}{}
|
||||
}
|
||||
putFieldsSet(m)
|
||||
|
||||
// Verify that all the fields are the same across rows
|
||||
rows = rows[1:]
|
||||
|
@ -546,6 +542,21 @@ func areSameFieldsInRows(rows [][]Field) bool {
|
|||
return true
|
||||
}
|
||||
|
||||
func getFieldsSet() map[string]struct{} {
|
||||
v := fieldsSetPool.Get()
|
||||
if v == nil {
|
||||
return make(map[string]struct{})
|
||||
}
|
||||
return v.(map[string]struct{})
|
||||
}
|
||||
|
||||
func putFieldsSet(m map[string]struct{}) {
|
||||
clear(m)
|
||||
fieldsSetPool.Put(m)
|
||||
}
|
||||
|
||||
var fieldsSetPool sync.Pool
|
||||
|
||||
var columnIdxsPool sync.Pool
|
||||
|
||||
func getColumnIdxs() map[string]int {
|
||||
|
@ -557,9 +568,7 @@ func getColumnIdxs() map[string]int {
|
|||
}
|
||||
|
||||
func putColumnIdxs(m map[string]int) {
|
||||
for k := range m {
|
||||
delete(m, k)
|
||||
}
|
||||
clear(m)
|
||||
columnIdxsPool.Put(m)
|
||||
}
|
||||
|
||||
|
|
|
@ -4,6 +4,7 @@ import (
|
|||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// blockData contains packed data for a single block.
|
||||
|
@ -29,11 +30,6 @@ type blockData struct {
|
|||
|
||||
// constColumns contains data for const columns across the block
|
||||
constColumns []Field
|
||||
|
||||
// a is used for storing byte slices for timestamps and columns.
|
||||
//
|
||||
// It reduces fragmentation for them.
|
||||
a arena
|
||||
}
|
||||
|
||||
// reset resets bd for subsequent re-use
|
||||
|
@ -54,42 +50,37 @@ func (bd *blockData) reset() {
|
|||
ccs[i].Reset()
|
||||
}
|
||||
bd.constColumns = ccs[:0]
|
||||
|
||||
bd.a.reset()
|
||||
}
|
||||
|
||||
func (bd *blockData) resizeColumnsData(columnsDataLen int) []columnData {
|
||||
cds := bd.columnsData
|
||||
if n := columnsDataLen - cap(cds); n > 0 {
|
||||
cds = append(cds[:cap(cds)], make([]columnData, n)...)
|
||||
}
|
||||
cds = cds[:columnsDataLen]
|
||||
bd.columnsData = cds
|
||||
return cds
|
||||
bd.columnsData = slicesutil.SetLength(bd.columnsData, columnsDataLen)
|
||||
return bd.columnsData
|
||||
}
|
||||
|
||||
// copyFrom copies src to bd.
|
||||
func (bd *blockData) copyFrom(src *blockData) {
|
||||
//
|
||||
// bd is valid until a.reset() is called.
|
||||
func (bd *blockData) copyFrom(a *arena, src *blockData) {
|
||||
bd.reset()
|
||||
|
||||
bd.streamID = src.streamID
|
||||
bd.uncompressedSizeBytes = src.uncompressedSizeBytes
|
||||
bd.rowsCount = src.rowsCount
|
||||
bd.timestampsData.copyFrom(&src.timestampsData, &bd.a)
|
||||
bd.timestampsData.copyFrom(a, &src.timestampsData)
|
||||
|
||||
cdsSrc := src.columnsData
|
||||
cds := bd.resizeColumnsData(len(cdsSrc))
|
||||
for i := range cds {
|
||||
cds[i].copyFrom(&cdsSrc[i], &bd.a)
|
||||
cds[i].copyFrom(a, &cdsSrc[i])
|
||||
}
|
||||
bd.columnsData = cds
|
||||
|
||||
bd.constColumns = append(bd.constColumns[:0], src.constColumns...)
|
||||
bd.constColumns = appendFields(a, bd.constColumns[:0], src.constColumns)
|
||||
}
|
||||
|
||||
// unmarshalRows appends unmarshaled from bd log entries to dst.
|
||||
//
|
||||
// The returned log entries are valid until sbu and vd are valid.
|
||||
// The unmarshaled log entries are valid until sbu and vd are reset.
|
||||
func (bd *blockData) unmarshalRows(dst *rows, sbu *stringsBlockUnmarshaler, vd *valuesDecoder) error {
|
||||
b := getBlock()
|
||||
defer putBlock(b)
|
||||
|
@ -101,7 +92,7 @@ func (bd *blockData) unmarshalRows(dst *rows, sbu *stringsBlockUnmarshaler, vd *
|
|||
return nil
|
||||
}
|
||||
|
||||
// mustWriteTo writes bd with the given sid to sw and updates bh accordingly
|
||||
// mustWriteTo writes bd to sw and updates bh accordingly
|
||||
func (bd *blockData) mustWriteTo(bh *blockHeader, sw *streamWriters) {
|
||||
// Do not store the version used for encoding directly in the block data, since:
|
||||
// - all the blocks in the same part use the same encoding
|
||||
|
@ -118,16 +109,22 @@ func (bd *blockData) mustWriteTo(bh *blockHeader, sw *streamWriters) {
|
|||
|
||||
// Marshal columns
|
||||
cds := bd.columnsData
|
||||
|
||||
a := getArena()
|
||||
csh := getColumnsHeader()
|
||||
|
||||
chs := csh.resizeColumnHeaders(len(cds))
|
||||
for i := range cds {
|
||||
cds[i].mustWriteTo(&chs[i], sw)
|
||||
cds[i].mustWriteTo(a, &chs[i], sw)
|
||||
}
|
||||
csh.constColumns = append(csh.constColumns[:0], bd.constColumns...)
|
||||
csh.constColumns = appendFields(a, csh.constColumns[:0], bd.constColumns)
|
||||
|
||||
bb := longTermBufPool.Get()
|
||||
bb.B = csh.marshal(bb.B)
|
||||
|
||||
putColumnsHeader(csh)
|
||||
putArena(a)
|
||||
|
||||
bh.columnsHeaderOffset = sw.columnsHeaderWriter.bytesWritten
|
||||
bh.columnsHeaderSize = uint64(len(bb.B))
|
||||
if bh.columnsHeaderSize > maxColumnsHeaderSize {
|
||||
|
@ -138,7 +135,9 @@ func (bd *blockData) mustWriteTo(bh *blockHeader, sw *streamWriters) {
|
|||
}
|
||||
|
||||
// mustReadFrom reads block data associated with bh from sr to bd.
|
||||
func (bd *blockData) mustReadFrom(bh *blockHeader, sr *streamReaders) {
|
||||
//
|
||||
// The bd is valid until a.reset() is called.
|
||||
func (bd *blockData) mustReadFrom(a *arena, bh *blockHeader, sr *streamReaders) {
|
||||
bd.reset()
|
||||
|
||||
bd.streamID = bh.streamID
|
||||
|
@ -146,7 +145,7 @@ func (bd *blockData) mustReadFrom(bh *blockHeader, sr *streamReaders) {
|
|||
bd.rowsCount = bh.rowsCount
|
||||
|
||||
// Read timestamps
|
||||
bd.timestampsData.mustReadFrom(&bh.timestampsHeader, sr, &bd.a)
|
||||
bd.timestampsData.mustReadFrom(a, &bh.timestampsHeader, sr)
|
||||
|
||||
// Read columns
|
||||
if bh.columnsHeaderOffset != sr.columnsHeaderReader.bytesRead {
|
||||
|
@ -161,18 +160,20 @@ func (bd *blockData) mustReadFrom(bh *blockHeader, sr *streamReaders) {
|
|||
bb.B = bytesutil.ResizeNoCopyMayOverallocate(bb.B, int(columnsHeaderSize))
|
||||
sr.columnsHeaderReader.MustReadFull(bb.B)
|
||||
|
||||
cshA := getArena()
|
||||
csh := getColumnsHeader()
|
||||
if err := csh.unmarshal(bb.B); err != nil {
|
||||
if err := csh.unmarshal(cshA, bb.B); err != nil {
|
||||
logger.Panicf("FATAL: %s: cannot unmarshal columnsHeader: %s", sr.columnsHeaderReader.Path(), err)
|
||||
}
|
||||
longTermBufPool.Put(bb)
|
||||
chs := csh.columnHeaders
|
||||
cds := bd.resizeColumnsData(len(chs))
|
||||
for i := range chs {
|
||||
cds[i].mustReadFrom(&chs[i], sr, &bd.a)
|
||||
cds[i].mustReadFrom(a, &chs[i], sr)
|
||||
}
|
||||
bd.constColumns = append(bd.constColumns[:0], csh.constColumns...)
|
||||
bd.constColumns = appendFields(a, bd.constColumns[:0], csh.constColumns)
|
||||
putColumnsHeader(csh)
|
||||
putArena(cshA)
|
||||
}
|
||||
|
||||
// timestampsData contains the encoded timestamps data.
|
||||
|
@ -199,7 +200,9 @@ func (td *timestampsData) reset() {
|
|||
}
|
||||
|
||||
// copyFrom copies src to td.
|
||||
func (td *timestampsData) copyFrom(src *timestampsData, a *arena) {
|
||||
//
|
||||
// td is valid until a.reset() is called.
|
||||
func (td *timestampsData) copyFrom(a *arena, src *timestampsData) {
|
||||
td.reset()
|
||||
|
||||
td.data = a.copyBytes(src.data)
|
||||
|
@ -224,7 +227,9 @@ func (td *timestampsData) mustWriteTo(th *timestampsHeader, sw *streamWriters) {
|
|||
}
|
||||
|
||||
// mustReadFrom reads timestamps data associated with th from sr to td.
|
||||
func (td *timestampsData) mustReadFrom(th *timestampsHeader, sr *streamReaders, a *arena) {
|
||||
//
|
||||
// td is valid until a.reset() is called.
|
||||
func (td *timestampsData) mustReadFrom(a *arena, th *timestampsHeader, sr *streamReaders) {
|
||||
td.reset()
|
||||
|
||||
td.marshalType = th.marshalType
|
||||
|
@ -287,22 +292,26 @@ func (cd *columnData) reset() {
|
|||
}
|
||||
|
||||
// copyFrom copies src to cd.
|
||||
func (cd *columnData) copyFrom(src *columnData, a *arena) {
|
||||
//
|
||||
// cd is valid until a.reset() is called.
|
||||
func (cd *columnData) copyFrom(a *arena, src *columnData) {
|
||||
cd.reset()
|
||||
|
||||
cd.name = src.name
|
||||
cd.name = a.copyString(src.name)
|
||||
cd.valueType = src.valueType
|
||||
|
||||
cd.minValue = src.minValue
|
||||
cd.maxValue = src.maxValue
|
||||
cd.valuesDict.copyFrom(&src.valuesDict)
|
||||
cd.valuesDict.copyFrom(a, &src.valuesDict)
|
||||
|
||||
cd.valuesData = a.copyBytes(src.valuesData)
|
||||
cd.bloomFilterData = a.copyBytes(src.bloomFilterData)
|
||||
}
|
||||
|
||||
// mustWriteTo writes cd to sw and updates ch accordingly.
|
||||
func (cd *columnData) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
||||
//
|
||||
// ch is valid until a.reset() is called.
|
||||
func (cd *columnData) mustWriteTo(a *arena, ch *columnHeader, sw *streamWriters) {
|
||||
ch.reset()
|
||||
|
||||
valuesWriter := &sw.fieldValuesWriter
|
||||
|
@ -312,12 +321,12 @@ func (cd *columnData) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
|||
bloomFilterWriter = &sw.messageBloomFilterWriter
|
||||
}
|
||||
|
||||
ch.name = cd.name
|
||||
ch.name = a.copyString(cd.name)
|
||||
ch.valueType = cd.valueType
|
||||
|
||||
ch.minValue = cd.minValue
|
||||
ch.maxValue = cd.maxValue
|
||||
ch.valuesDict.copyFrom(&cd.valuesDict)
|
||||
ch.valuesDict.copyFrom(a, &cd.valuesDict)
|
||||
|
||||
// marshal values
|
||||
ch.valuesSize = uint64(len(cd.valuesData))
|
||||
|
@ -337,7 +346,9 @@ func (cd *columnData) mustWriteTo(ch *columnHeader, sw *streamWriters) {
|
|||
}
|
||||
|
||||
// mustReadFrom reads columns data associated with ch from sr to cd.
|
||||
func (cd *columnData) mustReadFrom(ch *columnHeader, sr *streamReaders, a *arena) {
|
||||
//
|
||||
// cd is valid until a.reset() is called.
|
||||
func (cd *columnData) mustReadFrom(a *arena, ch *columnHeader, sr *streamReaders) {
|
||||
cd.reset()
|
||||
|
||||
valuesReader := &sr.fieldValuesReader
|
||||
|
@ -347,12 +358,12 @@ func (cd *columnData) mustReadFrom(ch *columnHeader, sr *streamReaders, a *arena
|
|||
bloomFilterReader = &sr.messageBloomFilterReader
|
||||
}
|
||||
|
||||
cd.name = ch.name
|
||||
cd.name = a.copyString(ch.name)
|
||||
cd.valueType = ch.valueType
|
||||
|
||||
cd.minValue = ch.minValue
|
||||
cd.maxValue = ch.maxValue
|
||||
cd.valuesDict.copyFrom(&ch.valuesDict)
|
||||
cd.valuesDict.copyFrom(a, &ch.valuesDict)
|
||||
|
||||
// read values
|
||||
if ch.valuesOffset != valuesReader.bytesRead {
|
||||
|
|
|
@ -51,20 +51,23 @@ func TestBlockDataReset(t *testing.T) {
|
|||
func TestBlockDataCopyFrom(t *testing.T) {
|
||||
f := func(bd *blockData) {
|
||||
t.Helper()
|
||||
|
||||
a := getArena()
|
||||
defer putArena(a)
|
||||
|
||||
var bd2 blockData
|
||||
bd2.copyFrom(bd)
|
||||
bd2.a.b = nil
|
||||
bd2.copyFrom(a, bd)
|
||||
if !reflect.DeepEqual(bd, &bd2) {
|
||||
t.Fatalf("unexpected blockData copy\ngot\n%v\nwant\n%v", &bd2, bd)
|
||||
}
|
||||
|
||||
// Try copying it again to the same destination
|
||||
bd2.copyFrom(bd)
|
||||
bd2.a.b = nil
|
||||
bd2.copyFrom(a, bd)
|
||||
if !reflect.DeepEqual(bd, &bd2) {
|
||||
t.Fatalf("unexpected blockData copy to the same destination\ngot\n%v\nwant\n%v", &bd2, bd)
|
||||
}
|
||||
}
|
||||
|
||||
f(&blockData{})
|
||||
|
||||
bd := &blockData{
|
||||
|
|
|
@ -8,6 +8,7 @@ import (
|
|||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// blockHeader contains information about a single block.
|
||||
|
@ -263,23 +264,13 @@ func (csh *columnsHeader) getColumnHeader(name string) *columnHeader {
|
|||
}
|
||||
|
||||
func (csh *columnsHeader) resizeConstColumns(columnsLen int) []Field {
|
||||
ccs := csh.constColumns
|
||||
if n := columnsLen - cap(ccs); n > 0 {
|
||||
ccs = append(ccs[:cap(ccs)], make([]Field, n)...)
|
||||
}
|
||||
ccs = ccs[:columnsLen]
|
||||
csh.constColumns = ccs
|
||||
return ccs
|
||||
csh.constColumns = slicesutil.SetLength(csh.constColumns, columnsLen)
|
||||
return csh.constColumns
|
||||
}
|
||||
|
||||
func (csh *columnsHeader) resizeColumnHeaders(columnHeadersLen int) []columnHeader {
|
||||
chs := csh.columnHeaders
|
||||
if n := columnHeadersLen - cap(chs); n > 0 {
|
||||
chs = append(chs[:cap(chs)], make([]columnHeader, n)...)
|
||||
}
|
||||
chs = chs[:columnHeadersLen]
|
||||
csh.columnHeaders = chs
|
||||
return chs
|
||||
csh.columnHeaders = slicesutil.SetLength(csh.columnHeaders, columnHeadersLen)
|
||||
return csh.columnHeaders
|
||||
}
|
||||
|
||||
func (csh *columnsHeader) marshal(dst []byte) []byte {
|
||||
|
@ -298,7 +289,10 @@ func (csh *columnsHeader) marshal(dst []byte) []byte {
|
|||
return dst
|
||||
}
|
||||
|
||||
func (csh *columnsHeader) unmarshal(src []byte) error {
|
||||
// unmarshal unmarshals csh from src.
|
||||
//
|
||||
// csh is valid until a.reset() is called.
|
||||
func (csh *columnsHeader) unmarshal(a *arena, src []byte) error {
|
||||
csh.reset()
|
||||
|
||||
// unmarshal columnHeaders
|
||||
|
@ -312,7 +306,7 @@ func (csh *columnsHeader) unmarshal(src []byte) error {
|
|||
src = tail
|
||||
chs := csh.resizeColumnHeaders(int(n))
|
||||
for i := range chs {
|
||||
tail, err = chs[i].unmarshal(src)
|
||||
tail, err = chs[i].unmarshal(a, src)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot unmarshal columnHeader %d out of %d columnHeaders: %w", i, len(chs), err)
|
||||
}
|
||||
|
@ -331,7 +325,7 @@ func (csh *columnsHeader) unmarshal(src []byte) error {
|
|||
src = tail
|
||||
ccs := csh.resizeConstColumns(int(n))
|
||||
for i := range ccs {
|
||||
tail, err = ccs[i].unmarshal(src)
|
||||
tail, err = ccs[i].unmarshal(a, src)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot unmarshal constColumn %d out of %d columns: %w", i, len(ccs), err)
|
||||
}
|
||||
|
@ -357,7 +351,7 @@ func (csh *columnsHeader) unmarshal(src []byte) error {
|
|||
//
|
||||
// Tokens in bloom filter depend on valueType:
|
||||
//
|
||||
// - valueTypeString stores lowercased tokens seen in all the values
|
||||
// - valueTypeString stores tokens seen in all the values
|
||||
// - valueTypeDict doesn't store anything in the bloom filter, since all the encoded values
|
||||
// are available directly in the valuesDict field
|
||||
// - valueTypeUint8, valueTypeUint16, valueTypeUint32 and valueTypeUint64 stores encoded uint values
|
||||
|
@ -423,12 +417,16 @@ func (ch *columnHeader) marshal(dst []byte) []byte {
|
|||
minValue := math.Float64frombits(ch.minValue)
|
||||
maxValue := math.Float64frombits(ch.maxValue)
|
||||
if minValue > maxValue {
|
||||
logger.Panicf("BUG: minValue=%g must be smaller than maxValue=%g", minValue, maxValue)
|
||||
logger.Panicf("BUG: minValue=%g must be smaller than maxValue=%g for valueTypeFloat64", minValue, maxValue)
|
||||
}
|
||||
} else {
|
||||
if ch.minValue > ch.maxValue {
|
||||
logger.Panicf("BUG: minValue=%d must be smaller than maxValue=%d", ch.minValue, ch.maxValue)
|
||||
} else if ch.valueType == valueTypeTimestampISO8601 {
|
||||
minValue := int64(ch.minValue)
|
||||
maxValue := int64(ch.maxValue)
|
||||
if minValue > maxValue {
|
||||
logger.Panicf("BUG: minValue=%g must be smaller than maxValue=%g for valueTypeTimestampISO8601", minValue, maxValue)
|
||||
}
|
||||
} else if ch.minValue > ch.maxValue {
|
||||
logger.Panicf("BUG: minValue=%d must be smaller than maxValue=%d for valueType=%d", ch.minValue, ch.maxValue, ch.valueType)
|
||||
}
|
||||
|
||||
// Encode common fields - ch.name and ch.valueType
|
||||
|
@ -498,7 +496,9 @@ func (ch *columnHeader) marshalBloomFilters(dst []byte) []byte {
|
|||
}
|
||||
|
||||
// unmarshal unmarshals ch from src and returns the tail left after unmarshaling.
|
||||
func (ch *columnHeader) unmarshal(src []byte) ([]byte, error) {
|
||||
//
|
||||
// ch is valid until a.reset() is called.
|
||||
func (ch *columnHeader) unmarshal(a *arena, src []byte) ([]byte, error) {
|
||||
ch.reset()
|
||||
|
||||
srcOrig := src
|
||||
|
@ -508,8 +508,7 @@ func (ch *columnHeader) unmarshal(src []byte) ([]byte, error) {
|
|||
if err != nil {
|
||||
return srcOrig, fmt.Errorf("cannot unmarshal column name: %w", err)
|
||||
}
|
||||
// Do not use bytesutil.InternBytes(data) here, since it works slower than the string(data) in prod
|
||||
ch.name = string(data)
|
||||
ch.name = a.copyBytesToString(data)
|
||||
src = tail
|
||||
|
||||
// Unmarshal value type
|
||||
|
@ -528,7 +527,7 @@ func (ch *columnHeader) unmarshal(src []byte) ([]byte, error) {
|
|||
}
|
||||
src = tail
|
||||
case valueTypeDict:
|
||||
tail, err = ch.valuesDict.unmarshal(src)
|
||||
tail, err = ch.valuesDict.unmarshal(a, src)
|
||||
if err != nil {
|
||||
return srcOrig, fmt.Errorf("cannot unmarshal dict at valueTypeDict for column %q: %w", ch.name, err)
|
||||
}
|
||||
|
@ -711,10 +710,10 @@ type timestampsHeader struct {
|
|||
// blockSize is the size of the timestamps block inside timestampsFilename file
|
||||
blockSize uint64
|
||||
|
||||
// minTimestamp is the mimumum timestamp seen in the block
|
||||
// minTimestamp is the mimumum timestamp seen in the block in nanoseconds
|
||||
minTimestamp int64
|
||||
|
||||
// maxTimestamp is the maximum timestamp seen in the block
|
||||
// maxTimestamp is the maximum timestamp seen in the block in nanoseconds
|
||||
maxTimestamp int64
|
||||
|
||||
// marshalType is the type used for encoding the timestamps block
|
||||
|
|
|
@ -55,12 +55,16 @@ func TestBlockHeaderMarshalUnmarshal(t *testing.T) {
|
|||
func TestColumnsHeaderMarshalUnmarshal(t *testing.T) {
|
||||
f := func(csh *columnsHeader, marshaledLen int) {
|
||||
t.Helper()
|
||||
|
||||
a := getArena()
|
||||
defer putArena(a)
|
||||
|
||||
data := csh.marshal(nil)
|
||||
if len(data) != marshaledLen {
|
||||
t.Fatalf("unexpected lengths of the marshaled columnsHeader; got %d; want %d", len(data), marshaledLen)
|
||||
}
|
||||
csh2 := &columnsHeader{}
|
||||
err := csh2.unmarshal(data)
|
||||
err := csh2.unmarshal(a, data)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in unmarshal: %s", err)
|
||||
}
|
||||
|
@ -68,6 +72,7 @@ func TestColumnsHeaderMarshalUnmarshal(t *testing.T) {
|
|||
t.Fatalf("unexpected blockHeader unmarshaled\ngot\n%v\nwant\n%v", csh2, csh)
|
||||
}
|
||||
}
|
||||
|
||||
f(&columnsHeader{}, 2)
|
||||
f(&columnsHeader{
|
||||
columnHeaders: []columnHeader{
|
||||
|
@ -149,13 +154,18 @@ func TestBlockHeaderUnmarshalFailure(t *testing.T) {
|
|||
func TestColumnsHeaderUnmarshalFailure(t *testing.T) {
|
||||
f := func(data []byte) {
|
||||
t.Helper()
|
||||
|
||||
a := getArena()
|
||||
defer putArena(a)
|
||||
|
||||
csh := getColumnsHeader()
|
||||
defer putColumnsHeader(csh)
|
||||
err := csh.unmarshal(data)
|
||||
err := csh.unmarshal(a, data)
|
||||
if err == nil {
|
||||
t.Fatalf("expecting non-nil error")
|
||||
}
|
||||
}
|
||||
|
||||
f(nil)
|
||||
f([]byte("foo"))
|
||||
|
||||
|
@ -315,12 +325,16 @@ func TestMarshalUnmarshalBlockHeaders(t *testing.T) {
|
|||
func TestColumnHeaderMarshalUnmarshal(t *testing.T) {
|
||||
f := func(ch *columnHeader, marshaledLen int) {
|
||||
t.Helper()
|
||||
|
||||
a := getArena()
|
||||
defer putArena(a)
|
||||
|
||||
data := ch.marshal(nil)
|
||||
if len(data) != marshaledLen {
|
||||
t.Fatalf("unexpected marshaled length of columnHeader; got %d; want %d", len(data), marshaledLen)
|
||||
}
|
||||
var ch2 columnHeader
|
||||
tail, err := ch2.unmarshal(data)
|
||||
tail, err := ch2.unmarshal(a, data)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in umarshal(%v): %s", ch, err)
|
||||
}
|
||||
|
@ -331,6 +345,7 @@ func TestColumnHeaderMarshalUnmarshal(t *testing.T) {
|
|||
t.Fatalf("unexpected columnHeader after unmarshal;\ngot\n%v\nwant\n%v", &ch2, ch)
|
||||
}
|
||||
}
|
||||
|
||||
f(&columnHeader{
|
||||
name: "foo",
|
||||
valueType: valueTypeUint8,
|
||||
|
@ -349,9 +364,13 @@ func TestColumnHeaderMarshalUnmarshal(t *testing.T) {
|
|||
func TestColumnHeaderUnmarshalFailure(t *testing.T) {
|
||||
f := func(data []byte) {
|
||||
t.Helper()
|
||||
|
||||
a := getArena()
|
||||
defer putArena(a)
|
||||
|
||||
dataOrig := append([]byte{}, data...)
|
||||
var ch columnHeader
|
||||
tail, err := ch.unmarshal(data)
|
||||
tail, err := ch.unmarshal(a, data)
|
||||
if err == nil {
|
||||
t.Fatalf("expecting non-nil error")
|
||||
}
|
||||
|
@ -359,6 +378,7 @@ func TestColumnHeaderUnmarshalFailure(t *testing.T) {
|
|||
t.Fatalf("unexpected tail left; got %q; want %q", tail, dataOrig)
|
||||
}
|
||||
}
|
||||
|
||||
f(nil)
|
||||
f([]byte("foo"))
|
||||
|
||||
|
|
1874
lib/logstorage/block_result.go
Normal file
1874
lib/logstorage/block_result.go
Normal file
File diff suppressed because it is too large
Load diff
|
@ -1,9 +1,7 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
|
@ -14,7 +12,7 @@ type blockSearchWork struct {
|
|||
// p is the part where the block belongs to.
|
||||
p *part
|
||||
|
||||
// so contains search options for the block search
|
||||
// so contains search options for the block search.
|
||||
so *searchOptions
|
||||
|
||||
// bh is the header of the block to search.
|
||||
|
@ -65,6 +63,9 @@ type blockSearch struct {
|
|||
|
||||
// csh is the columnsHeader associated with the given block
|
||||
csh columnsHeader
|
||||
|
||||
// a is used for storing unmarshaled data in csh
|
||||
a arena
|
||||
}
|
||||
|
||||
func (bs *blockSearch) reset() {
|
||||
|
@ -90,6 +91,7 @@ func (bs *blockSearch) reset() {
|
|||
|
||||
bs.sbu.reset()
|
||||
bs.csh.reset()
|
||||
bs.a.reset()
|
||||
}
|
||||
|
||||
func (bs *blockSearch) partPath() string {
|
||||
|
@ -101,44 +103,30 @@ func (bs *blockSearch) search(bsw *blockSearchWork) {
|
|||
|
||||
bs.bsw = bsw
|
||||
|
||||
bs.csh.initFromBlockHeader(bsw.p, &bsw.bh)
|
||||
bs.csh.initFromBlockHeader(&bs.a, bsw.p, &bsw.bh)
|
||||
|
||||
// search rows matching the given filter
|
||||
bm := getFilterBitmap(int(bsw.bh.rowsCount))
|
||||
bm := getBitmap(int(bsw.bh.rowsCount))
|
||||
defer putBitmap(bm)
|
||||
|
||||
bm.setBits()
|
||||
bs.bsw.so.filter.apply(bs, bm)
|
||||
|
||||
bs.br.mustInit(bs, bm)
|
||||
if bm.isZero() {
|
||||
putFilterBitmap(bm)
|
||||
// The filter doesn't match any logs in the current block.
|
||||
return
|
||||
}
|
||||
|
||||
// fetch the requested columns to bs.br.
|
||||
for _, columnName := range bs.bsw.so.resultColumnNames {
|
||||
switch columnName {
|
||||
case "_stream":
|
||||
bs.br.addStreamColumn(bs)
|
||||
case "_time":
|
||||
bs.br.addTimeColumn()
|
||||
default:
|
||||
v := bs.csh.getConstColumnValue(columnName)
|
||||
if v != "" {
|
||||
bs.br.addConstColumn(v)
|
||||
continue
|
||||
}
|
||||
ch := bs.csh.getColumnHeader(columnName)
|
||||
if ch == nil {
|
||||
bs.br.addConstColumn("")
|
||||
} else {
|
||||
bs.br.addColumn(bs, ch, bm)
|
||||
}
|
||||
}
|
||||
if bs.bsw.so.needAllColumns {
|
||||
bs.br.fetchAllColumns(bs, bm)
|
||||
} else {
|
||||
bs.br.fetchRequestedColumns(bs, bm)
|
||||
}
|
||||
putFilterBitmap(bm)
|
||||
}
|
||||
|
||||
func (csh *columnsHeader) initFromBlockHeader(p *part, bh *blockHeader) {
|
||||
func (csh *columnsHeader) initFromBlockHeader(a *arena, p *part, bh *blockHeader) {
|
||||
bb := longTermBufPool.Get()
|
||||
columnsHeaderSize := bh.columnsHeaderSize
|
||||
if columnsHeaderSize > maxColumnsHeaderSize {
|
||||
|
@ -147,7 +135,7 @@ func (csh *columnsHeader) initFromBlockHeader(p *part, bh *blockHeader) {
|
|||
bb.B = bytesutil.ResizeNoCopyMayOverallocate(bb.B, int(columnsHeaderSize))
|
||||
p.columnsHeaderFile.MustReadAt(bb.B, int64(bh.columnsHeaderOffset))
|
||||
|
||||
if err := csh.unmarshal(bb.B); err != nil {
|
||||
if err := csh.unmarshal(a, bb.B); err != nil {
|
||||
logger.Panicf("FATAL: %s: cannot unmarshal columns header: %s", p.path, err)
|
||||
}
|
||||
longTermBufPool.Put(bb)
|
||||
|
@ -286,360 +274,3 @@ func (ih *indexBlockHeader) mustReadBlockHeaders(dst []blockHeader, p *part) []b
|
|||
|
||||
return dst
|
||||
}
|
||||
|
||||
type blockResult struct {
|
||||
buf []byte
|
||||
valuesBuf []string
|
||||
|
||||
// streamID is streamID for the given blockResult
|
||||
streamID streamID
|
||||
|
||||
// cs contain values for result columns
|
||||
cs []blockResultColumn
|
||||
|
||||
// timestamps contain timestamps for the selected log entries
|
||||
timestamps []int64
|
||||
}
|
||||
|
||||
func (br *blockResult) reset() {
|
||||
br.buf = br.buf[:0]
|
||||
|
||||
vb := br.valuesBuf
|
||||
for i := range vb {
|
||||
vb[i] = ""
|
||||
}
|
||||
br.valuesBuf = vb[:0]
|
||||
|
||||
br.streamID.reset()
|
||||
|
||||
cs := br.cs
|
||||
for i := range cs {
|
||||
cs[i].reset()
|
||||
}
|
||||
br.cs = cs[:0]
|
||||
|
||||
br.timestamps = br.timestamps[:0]
|
||||
}
|
||||
|
||||
func (br *blockResult) RowsCount() int {
|
||||
return len(br.timestamps)
|
||||
}
|
||||
|
||||
func (br *blockResult) mustInit(bs *blockSearch, bm *filterBitmap) {
|
||||
br.reset()
|
||||
|
||||
br.streamID = bs.bsw.bh.streamID
|
||||
|
||||
if !bm.isZero() {
|
||||
// Initialize timestamps, since they are used for determining the number of rows in br.RowsCount()
|
||||
srcTimestamps := bs.getTimestamps()
|
||||
dstTimestamps := br.timestamps[:0]
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
ts := srcTimestamps[idx]
|
||||
dstTimestamps = append(dstTimestamps, ts)
|
||||
return true
|
||||
})
|
||||
br.timestamps = dstTimestamps
|
||||
}
|
||||
}
|
||||
|
||||
func (br *blockResult) addColumn(bs *blockSearch, ch *columnHeader, bm *filterBitmap) {
|
||||
buf := br.buf
|
||||
valuesBuf := br.valuesBuf
|
||||
valuesBufLen := len(valuesBuf)
|
||||
var dictValues []string
|
||||
|
||||
appendValue := func(v string) {
|
||||
bufLen := len(buf)
|
||||
buf = append(buf, v...)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
valuesBuf = append(valuesBuf, s)
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeDict:
|
||||
dictValues = ch.valuesDict.values
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 1 {
|
||||
logger.Panicf("FATAL: %s: unexpected dict value size for column %q; got %d bytes; want 1 byte", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
dictIdx := v[0]
|
||||
if int(dictIdx) >= len(dictValues) {
|
||||
logger.Panicf("FATAL: %s: too big dict index for column %q: %d; should be smaller than %d", bs.partPath(), ch.name, dictIdx, len(dictValues))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeUint8:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 1 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for uint8 column %q; got %d bytes; want 1 byte", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeUint16:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 2 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for uint16 column %q; got %d bytes; want 2 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeUint32:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for uint32 column %q; got %d bytes; want 4 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeUint64:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for uint64 column %q; got %d bytes; want 8 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeFloat64:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for float64 column %q; got %d bytes; want 8 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeIPv4:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for ipv4 column %q; got %d bytes; want 4 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
case valueTypeTimestampISO8601:
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected size for timestmap column %q; got %d bytes; want 8 bytes", bs.partPath(), ch.name, len(v))
|
||||
}
|
||||
appendValue(v)
|
||||
return true
|
||||
})
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d for column %q", bs.partPath(), ch.valueType, ch.name)
|
||||
}
|
||||
|
||||
encodedValues := valuesBuf[valuesBufLen:]
|
||||
|
||||
valuesBufLen = len(valuesBuf)
|
||||
for _, v := range dictValues {
|
||||
appendValue(v)
|
||||
}
|
||||
dictValues = valuesBuf[valuesBufLen:]
|
||||
|
||||
br.cs = append(br.cs, blockResultColumn{
|
||||
valueType: ch.valueType,
|
||||
dictValues: dictValues,
|
||||
encodedValues: encodedValues,
|
||||
})
|
||||
br.buf = buf
|
||||
br.valuesBuf = valuesBuf
|
||||
}
|
||||
|
||||
func (br *blockResult) addTimeColumn() {
|
||||
br.cs = append(br.cs, blockResultColumn{
|
||||
isTime: true,
|
||||
})
|
||||
}
|
||||
|
||||
func (br *blockResult) addStreamColumn(bs *blockSearch) {
|
||||
bb := bbPool.Get()
|
||||
bb.B = bs.bsw.p.pt.appendStreamTagsByStreamID(bb.B[:0], &br.streamID)
|
||||
if len(bb.B) > 0 {
|
||||
st := GetStreamTags()
|
||||
mustUnmarshalStreamTags(st, bb.B)
|
||||
bb.B = st.marshalString(bb.B[:0])
|
||||
PutStreamTags(st)
|
||||
}
|
||||
s := bytesutil.ToUnsafeString(bb.B)
|
||||
br.addConstColumn(s)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func (br *blockResult) addConstColumn(value string) {
|
||||
buf := br.buf
|
||||
bufLen := len(buf)
|
||||
buf = append(buf, value...)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
br.buf = buf
|
||||
|
||||
valuesBuf := br.valuesBuf
|
||||
valuesBufLen := len(valuesBuf)
|
||||
valuesBuf = append(valuesBuf, s)
|
||||
br.valuesBuf = valuesBuf
|
||||
|
||||
br.cs = append(br.cs, blockResultColumn{
|
||||
isConst: true,
|
||||
valueType: valueTypeUnknown,
|
||||
encodedValues: valuesBuf[valuesBufLen:],
|
||||
})
|
||||
}
|
||||
|
||||
// getColumnValues returns values for the column with the given idx.
|
||||
//
|
||||
// The returned values are valid until br.reset() is called.
|
||||
func (br *blockResult) getColumnValues(idx int) []string {
|
||||
c := &br.cs[idx]
|
||||
if c.values != nil {
|
||||
return c.values
|
||||
}
|
||||
|
||||
buf := br.buf
|
||||
valuesBuf := br.valuesBuf
|
||||
valuesBufLen := len(valuesBuf)
|
||||
|
||||
if c.isConst {
|
||||
v := c.encodedValues[0]
|
||||
for range br.timestamps {
|
||||
valuesBuf = append(valuesBuf, v)
|
||||
}
|
||||
c.values = valuesBuf[valuesBufLen:]
|
||||
br.valuesBuf = valuesBuf
|
||||
return c.values
|
||||
}
|
||||
if c.isTime {
|
||||
for _, timestamp := range br.timestamps {
|
||||
t := time.Unix(0, timestamp).UTC()
|
||||
bufLen := len(buf)
|
||||
buf = t.AppendFormat(buf, time.RFC3339Nano)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
valuesBuf = append(valuesBuf, s)
|
||||
}
|
||||
c.values = valuesBuf[valuesBufLen:]
|
||||
br.buf = buf
|
||||
br.valuesBuf = valuesBuf
|
||||
return c.values
|
||||
}
|
||||
|
||||
appendValue := func(v string) {
|
||||
bufLen := len(buf)
|
||||
buf = append(buf, v...)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
valuesBuf = append(valuesBuf, s)
|
||||
}
|
||||
|
||||
switch c.valueType {
|
||||
case valueTypeString:
|
||||
c.values = c.encodedValues
|
||||
return c.values
|
||||
case valueTypeDict:
|
||||
dictValues := c.dictValues
|
||||
for _, v := range c.encodedValues {
|
||||
dictIdx := v[0]
|
||||
appendValue(dictValues[dictIdx])
|
||||
}
|
||||
case valueTypeUint8:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
n := uint64(v[0])
|
||||
bb.B = strconv.AppendUint(bb.B[:0], n, 10)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeUint16:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint16(b))
|
||||
bb.B = strconv.AppendUint(bb.B[:0], n, 10)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeUint32:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint32(b))
|
||||
bb.B = strconv.AppendUint(bb.B[:0], n, 10)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeUint64:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := encoding.UnmarshalUint64(b)
|
||||
bb.B = strconv.AppendUint(bb.B[:0], n, 10)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeFloat64:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
bb.B = toFloat64String(bb.B[:0], v)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeIPv4:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
bb.B = toIPv4String(bb.B[:0], v)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
case valueTypeTimestampISO8601:
|
||||
bb := bbPool.Get()
|
||||
for _, v := range c.encodedValues {
|
||||
bb.B = toTimestampISO8601String(bb.B[:0], v)
|
||||
appendValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
default:
|
||||
logger.Panicf("BUG: unknown valueType=%d", c.valueType)
|
||||
}
|
||||
|
||||
c.values = valuesBuf[valuesBufLen:]
|
||||
br.buf = buf
|
||||
br.valuesBuf = valuesBuf
|
||||
|
||||
return c.values
|
||||
}
|
||||
|
||||
type blockResultColumn struct {
|
||||
// isConst is set to true if the column is const.
|
||||
//
|
||||
// The column value is stored in encodedValues[0]
|
||||
isConst bool
|
||||
|
||||
// isTime is set to true if the column contains _time values.
|
||||
//
|
||||
// The column values are stored in blockResult.timestamps
|
||||
isTime bool
|
||||
|
||||
// valueType is the type of non-cost value
|
||||
valueType valueType
|
||||
|
||||
// dictValues contain dictionary values for valueTypeDict column
|
||||
dictValues []string
|
||||
|
||||
// encodedValues contain encoded values for non-const column
|
||||
encodedValues []string
|
||||
|
||||
// values contain decoded values after getColumnValues() call for the given column
|
||||
values []string
|
||||
}
|
||||
|
||||
func (c *blockResultColumn) reset() {
|
||||
c.isConst = false
|
||||
c.isTime = false
|
||||
c.valueType = valueTypeUnknown
|
||||
c.dictValues = nil
|
||||
c.encodedValues = nil
|
||||
c.values = nil
|
||||
}
|
||||
|
|
|
@ -59,6 +59,9 @@ type blockStreamMerger struct {
|
|||
// bd is unpacked into rows when needed.
|
||||
bd blockData
|
||||
|
||||
// a holds bd data.
|
||||
a arena
|
||||
|
||||
// rows is pending log entries.
|
||||
rows rows
|
||||
|
||||
|
@ -99,6 +102,7 @@ func (bsm *blockStreamMerger) resetRows() {
|
|||
bsm.vd = nil
|
||||
}
|
||||
bsm.bd.reset()
|
||||
bsm.a.reset()
|
||||
|
||||
bsm.rows.reset()
|
||||
bsm.rowsTmp.reset()
|
||||
|
@ -138,7 +142,8 @@ func (bsm *blockStreamMerger) mustWriteBlock(bd *blockData, bsw *blockStreamWrit
|
|||
bsw.MustWriteBlockData(bd)
|
||||
} else {
|
||||
// Slow path - copy the bd to the curr bd.
|
||||
bsm.bd.copyFrom(bd)
|
||||
bsm.a.reset()
|
||||
bsm.bd.copyFrom(&bsm.a, bd)
|
||||
bsm.uniqueFields = uniqueFields
|
||||
}
|
||||
case bsm.uniqueFields+uniqueFields >= maxColumnsPerBlock:
|
||||
|
@ -150,7 +155,8 @@ func (bsm *blockStreamMerger) mustWriteBlock(bd *blockData, bsw *blockStreamWrit
|
|||
if uniqueFields >= maxColumnsPerBlock {
|
||||
bsw.MustWriteBlockData(bd)
|
||||
} else {
|
||||
bsm.bd.copyFrom(bd)
|
||||
bsm.a.reset()
|
||||
bsm.bd.copyFrom(&bsm.a, bd)
|
||||
bsm.uniqueFields = uniqueFields
|
||||
}
|
||||
case bd.uncompressedSizeBytes >= maxUncompressedBlockSize:
|
||||
|
@ -218,6 +224,7 @@ func (bsm *blockStreamMerger) mustMergeRows(bd *blockData) {
|
|||
// Unmarshal log entries from bsm.bd
|
||||
bsm.mustUnmarshalRows(&bsm.bd)
|
||||
bsm.bd.reset()
|
||||
bsm.a.reset()
|
||||
}
|
||||
|
||||
// Unmarshal log entries from bd
|
||||
|
|
|
@ -112,6 +112,9 @@ type blockStreamReader struct {
|
|||
// blockData contains the data for the last read block
|
||||
blockData blockData
|
||||
|
||||
// a contains data for blockData
|
||||
a arena
|
||||
|
||||
// ph is the header for the part
|
||||
ph partHeader
|
||||
|
||||
|
@ -149,6 +152,7 @@ type blockStreamReader struct {
|
|||
// reset resets bsr, so it can be re-used
|
||||
func (bsr *blockStreamReader) reset() {
|
||||
bsr.blockData.reset()
|
||||
bsr.a.reset()
|
||||
bsr.ph.reset()
|
||||
bsr.streamReaders.reset()
|
||||
|
||||
|
@ -247,6 +251,8 @@ func (bsr *blockStreamReader) MustInitFromFilePart(path string) {
|
|||
// NextBlock reads the next block from bsr and puts it into bsr.blockData.
|
||||
//
|
||||
// false is returned if there are no other blocks.
|
||||
//
|
||||
// bsr.blockData is valid until the next call to NextBlock().
|
||||
func (bsr *blockStreamReader) NextBlock() bool {
|
||||
for bsr.nextBlockIdx >= len(bsr.blockHeaders) {
|
||||
if !bsr.nextIndexBlock() {
|
||||
|
@ -275,7 +281,8 @@ func (bsr *blockStreamReader) NextBlock() bool {
|
|||
}
|
||||
|
||||
// Read bsr.blockData
|
||||
bsr.blockData.mustReadFrom(bh, &bsr.streamReaders)
|
||||
bsr.a.reset()
|
||||
bsr.blockData.mustReadFrom(&bsr.a, bh, &bsr.streamReaders)
|
||||
|
||||
bsr.globalUncompressedSizeBytes += bh.uncompressedSizeBytes
|
||||
bsr.globalRowsCount += bh.rowsCount
|
||||
|
|
|
@ -169,11 +169,11 @@ func TestBlockMustInitFromRowsFullBlock(t *testing.T) {
|
|||
b := getBlock()
|
||||
defer putBlock(b)
|
||||
b.MustInitFromRows(timestamps, rows)
|
||||
b.assertValid()
|
||||
if n := b.Len(); n != len(rows) {
|
||||
t.Fatalf("unexpected total log entries; got %d; want %d", n, len(rows))
|
||||
}
|
||||
if b.uncompressedSizeBytes() < maxUncompressedBlockSize {
|
||||
t.Fatalf("expecting full block")
|
||||
if n := b.uncompressedSizeBytes(); n < maxUncompressedBlockSize {
|
||||
t.Fatalf("expecting full block with %d bytes; got %d bytes", maxUncompressedBlockSize, n)
|
||||
}
|
||||
b.assertValid()
|
||||
}
|
||||
|
|
|
@ -5,9 +5,11 @@ import (
|
|||
"sync"
|
||||
"unsafe"
|
||||
|
||||
"github.com/cespare/xxhash/v2"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/cespare/xxhash/v2"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/slicesutil"
|
||||
)
|
||||
|
||||
// bloomFilterHashesCount is the number of different hashes to use for bloom filter.
|
||||
|
@ -53,11 +55,7 @@ func (bf *bloomFilter) unmarshal(src []byte) error {
|
|||
}
|
||||
bf.reset()
|
||||
wordsCount := len(src) / 8
|
||||
bits := bf.bits
|
||||
if n := wordsCount - cap(bits); n > 0 {
|
||||
bits = append(bits[:cap(bits)], make([]uint64, n)...)
|
||||
}
|
||||
bits = bits[:wordsCount]
|
||||
bits := slicesutil.SetLength(bf.bits, wordsCount)
|
||||
for i := range bits {
|
||||
bits[i] = encoding.UnmarshalUint64(src)
|
||||
src = src[8:]
|
||||
|
@ -70,11 +68,7 @@ func (bf *bloomFilter) unmarshal(src []byte) error {
|
|||
func (bf *bloomFilter) mustInit(tokens []string) {
|
||||
bitsCount := len(tokens) * bloomFilterBitsPerItem
|
||||
wordsCount := (bitsCount + 63) / 64
|
||||
bits := bf.bits
|
||||
if n := wordsCount - cap(bits); n > 0 {
|
||||
bits = append(bits[:cap(bits)], make([]uint64, n)...)
|
||||
}
|
||||
bits = bits[:wordsCount]
|
||||
bits := slicesutil.SetLength(bf.bits, wordsCount)
|
||||
bloomFilterAdd(bits, tokens)
|
||||
bf.bits = bits
|
||||
}
|
||||
|
|
|
@ -14,7 +14,7 @@ const maxUncompressedBlockSize = 2 * 1024 * 1024
|
|||
const maxRowsPerBlock = 8 * 1024 * 1024
|
||||
|
||||
// maxColumnsPerBlock is the maximum number of columns per block.
|
||||
const maxColumnsPerBlock = 10000
|
||||
const maxColumnsPerBlock = 1_000
|
||||
|
||||
// MaxFieldNameSize is the maximum size in bytes for field name.
|
||||
//
|
||||
|
|
|
@ -17,6 +17,22 @@ import (
|
|||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
|
||||
)
|
||||
|
||||
// The maximum size of big part.
|
||||
//
|
||||
// This number limits the maximum time required for building big part.
|
||||
// This time shouldn't exceed a few days.
|
||||
const maxBigPartSize = 1e12
|
||||
|
||||
// The maximum number of inmemory parts in the partition.
|
||||
//
|
||||
// The actual number of inmemory parts may exceed this value if in-memory mergers
|
||||
// cannot keep up with the rate of creating new in-memory parts.
|
||||
const maxInmemoryPartsPerPartition = 20
|
||||
|
||||
// The interval for guaranteed flush of recently ingested data from memory to on-disk parts,
|
||||
// so they survive process crash.
|
||||
var dataFlushInterval = 5 * time.Second
|
||||
|
||||
// Default number of parts to merge at once.
|
||||
//
|
||||
// This number has been obtained empirically - it gives the lowest possible overhead.
|
||||
|
@ -31,11 +47,6 @@ const defaultPartsToMerge = 15
|
|||
// The 1.7 is good enough for production workloads.
|
||||
const minMergeMultiplier = 1.7
|
||||
|
||||
// The maximum number of inmemory parts in the partition.
|
||||
//
|
||||
// If the number of inmemory parts reaches this value, then assisted merge runs during data ingestion.
|
||||
const maxInmemoryPartsPerPartition = 20
|
||||
|
||||
// datadb represents a database with log data
|
||||
type datadb struct {
|
||||
// mergeIdx is used for generating unique directory names for parts
|
||||
|
@ -43,8 +54,12 @@ type datadb struct {
|
|||
|
||||
inmemoryMergesTotal atomic.Uint64
|
||||
inmemoryActiveMerges atomic.Int64
|
||||
fileMergesTotal atomic.Uint64
|
||||
fileActiveMerges atomic.Int64
|
||||
|
||||
smallPartMergesTotal atomic.Uint64
|
||||
smallPartActiveMerges atomic.Int64
|
||||
|
||||
bigPartMergesTotal atomic.Uint64
|
||||
bigPartActiveMerges atomic.Int64
|
||||
|
||||
// pt is the partition the datadb belongs to
|
||||
pt *partition
|
||||
|
@ -58,8 +73,11 @@ type datadb struct {
|
|||
// inmemoryParts contains a list of inmemory parts
|
||||
inmemoryParts []*partWrapper
|
||||
|
||||
// fileParts contains a list of file-based parts
|
||||
fileParts []*partWrapper
|
||||
// smallParts contains a list of file-based small parts
|
||||
smallParts []*partWrapper
|
||||
|
||||
// bigParts contains a list of file-based big parts
|
||||
bigParts []*partWrapper
|
||||
|
||||
// partsLock protects parts from concurrent access
|
||||
partsLock sync.Mutex
|
||||
|
@ -75,16 +93,6 @@ type datadb struct {
|
|||
// It must be closed under partsLock in order to prevent from calling wg.Add()
|
||||
// after stopCh is closed.
|
||||
stopCh chan struct{}
|
||||
|
||||
// oldInmemoryPartsFlushersCount is the number of currently running flushers for old in-memory parts
|
||||
//
|
||||
// This variable must be accessed under partsLock.
|
||||
oldInmemoryPartsFlushersCount int
|
||||
|
||||
// mergeWorkersCount is the number of currently running merge workers
|
||||
//
|
||||
// This variable must be accessed under partsLock.
|
||||
mergeWorkersCount int
|
||||
}
|
||||
|
||||
// partWrapper is a wrapper for opened part.
|
||||
|
@ -140,7 +148,7 @@ func (pw *partWrapper) decRef() {
|
|||
|
||||
func mustCreateDatadb(path string) {
|
||||
fs.MustMkdirFailIfExist(path)
|
||||
mustWritePartNames(path, []string{})
|
||||
mustWritePartNames(path, nil, nil)
|
||||
}
|
||||
|
||||
// mustOpenDatadb opens datadb at the given path with the given flushInterval for in-memory data.
|
||||
|
@ -151,8 +159,9 @@ func mustOpenDatadb(pt *partition, path string, flushInterval time.Duration) *da
|
|||
partNames := mustReadPartNames(path)
|
||||
mustRemoveUnusedDirs(path, partNames)
|
||||
|
||||
pws := make([]*partWrapper, len(partNames))
|
||||
for i, partName := range partNames {
|
||||
var smallParts []*partWrapper
|
||||
var bigParts []*partWrapper
|
||||
for _, partName := range partNames {
|
||||
// Make sure the partName exists on disk.
|
||||
// If it is missing, then manual action from the user is needed,
|
||||
// since this is unexpected state, which cannot occur under normal operation,
|
||||
|
@ -166,181 +175,277 @@ func mustOpenDatadb(pt *partition, path string, flushInterval time.Duration) *da
|
|||
}
|
||||
|
||||
p := mustOpenFilePart(pt, partPath)
|
||||
pws[i] = newPartWrapper(p, nil, time.Time{})
|
||||
pw := newPartWrapper(p, nil, time.Time{})
|
||||
if p.ph.CompressedSizeBytes > getMaxInmemoryPartSize() {
|
||||
bigParts = append(bigParts, pw)
|
||||
} else {
|
||||
smallParts = append(smallParts, pw)
|
||||
}
|
||||
}
|
||||
|
||||
ddb := &datadb{
|
||||
pt: pt,
|
||||
flushInterval: flushInterval,
|
||||
path: path,
|
||||
fileParts: pws,
|
||||
smallParts: smallParts,
|
||||
bigParts: bigParts,
|
||||
stopCh: make(chan struct{}),
|
||||
}
|
||||
ddb.mergeIdx.Store(uint64(time.Now().UnixNano()))
|
||||
|
||||
// Start merge workers in the hope they'll merge the remaining parts
|
||||
ddb.partsLock.Lock()
|
||||
n := getMergeWorkersCount()
|
||||
for i := 0; i < n; i++ {
|
||||
ddb.startMergeWorkerLocked()
|
||||
}
|
||||
ddb.partsLock.Unlock()
|
||||
ddb.startBackgroundWorkers()
|
||||
|
||||
return ddb
|
||||
}
|
||||
|
||||
// startOldInmemoryPartsFlusherLocked starts flusher for old in-memory parts to disk.
|
||||
//
|
||||
// This function must be called under partsLock.
|
||||
func (ddb *datadb) startOldInmemoryPartsFlusherLocked() {
|
||||
func (ddb *datadb) startBackgroundWorkers() {
|
||||
// Start file parts mergers, so they could start merging unmerged parts if needed.
|
||||
// There is no need in starting in-memory parts mergers, since there are no in-memory parts yet.
|
||||
ddb.startSmallPartsMergers()
|
||||
ddb.startBigPartsMergers()
|
||||
|
||||
ddb.startInmemoryPartsFlusher()
|
||||
}
|
||||
|
||||
var (
|
||||
inmemoryPartsConcurrencyCh = make(chan struct{}, cgroup.AvailableCPUs())
|
||||
smallPartsConcurrencyCh = make(chan struct{}, cgroup.AvailableCPUs())
|
||||
bigPartsConcurrencyCh = make(chan struct{}, cgroup.AvailableCPUs())
|
||||
)
|
||||
|
||||
func (ddb *datadb) startSmallPartsMergers() {
|
||||
ddb.partsLock.Lock()
|
||||
for i := 0; i < cap(smallPartsConcurrencyCh); i++ {
|
||||
ddb.startSmallPartsMergerLocked()
|
||||
}
|
||||
ddb.partsLock.Unlock()
|
||||
}
|
||||
|
||||
func (ddb *datadb) startBigPartsMergers() {
|
||||
ddb.partsLock.Lock()
|
||||
for i := 0; i < cap(bigPartsConcurrencyCh); i++ {
|
||||
ddb.startBigPartsMergerLocked()
|
||||
}
|
||||
ddb.partsLock.Unlock()
|
||||
}
|
||||
|
||||
func (ddb *datadb) startInmemoryPartsMergerLocked() {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
maxWorkers := getMergeWorkersCount()
|
||||
if ddb.oldInmemoryPartsFlushersCount >= maxWorkers {
|
||||
return
|
||||
}
|
||||
ddb.oldInmemoryPartsFlushersCount++
|
||||
ddb.wg.Add(1)
|
||||
go func() {
|
||||
ddb.flushOldInmemoryParts()
|
||||
ddb.inmemoryPartsMerger()
|
||||
ddb.wg.Done()
|
||||
}()
|
||||
}
|
||||
|
||||
func (ddb *datadb) flushOldInmemoryParts() {
|
||||
ticker := time.NewTicker(time.Second)
|
||||
func (ddb *datadb) startSmallPartsMergerLocked() {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
ddb.wg.Add(1)
|
||||
go func() {
|
||||
ddb.smallPartsMerger()
|
||||
ddb.wg.Done()
|
||||
}()
|
||||
}
|
||||
|
||||
func (ddb *datadb) startBigPartsMergerLocked() {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
ddb.wg.Add(1)
|
||||
go func() {
|
||||
ddb.bigPartsMerger()
|
||||
ddb.wg.Done()
|
||||
}()
|
||||
}
|
||||
|
||||
func (ddb *datadb) startInmemoryPartsFlusher() {
|
||||
ddb.wg.Add(1)
|
||||
go func() {
|
||||
ddb.inmemoryPartsFlusher()
|
||||
ddb.wg.Done()
|
||||
}()
|
||||
}
|
||||
|
||||
func (ddb *datadb) inmemoryPartsFlusher() {
|
||||
// Do not add jitter to d in order to guarantee the flush interval
|
||||
ticker := time.NewTicker(dataFlushInterval)
|
||||
defer ticker.Stop()
|
||||
var parts, partsToMerge []*partWrapper
|
||||
|
||||
for !needStop(ddb.stopCh) {
|
||||
ddb.partsLock.Lock()
|
||||
parts = appendNotInMergePartsLocked(parts[:0], ddb.inmemoryParts)
|
||||
currentTime := time.Now()
|
||||
partsToFlush := parts[:0]
|
||||
for _, pw := range parts {
|
||||
if pw.flushDeadline.Before(currentTime) {
|
||||
partsToFlush = append(partsToFlush, pw)
|
||||
}
|
||||
}
|
||||
// Do not take into account available disk space when flushing in-memory parts to disk,
|
||||
// since otherwise the outdated in-memory parts may remain in-memory, which, in turn,
|
||||
// may result in increased memory usage plus possible loss of historical data.
|
||||
// It is better to crash on out of disk error in this case.
|
||||
partsToMerge = appendPartsToMerge(partsToMerge[:0], partsToFlush, math.MaxUint64)
|
||||
if len(partsToMerge) == 0 {
|
||||
partsToMerge = append(partsToMerge[:0], partsToFlush...)
|
||||
}
|
||||
setInMergeLocked(partsToMerge)
|
||||
needStop := false
|
||||
if len(ddb.inmemoryParts) == 0 {
|
||||
// There are no in-memory parts, so stop the flusher.
|
||||
needStop = true
|
||||
ddb.oldInmemoryPartsFlushersCount--
|
||||
}
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
if needStop {
|
||||
return
|
||||
}
|
||||
|
||||
ddb.mustMergeParts(partsToMerge, true)
|
||||
if len(partsToMerge) < len(partsToFlush) {
|
||||
// Continue merging remaining old in-memory parts from partsToFlush list.
|
||||
continue
|
||||
}
|
||||
|
||||
// There are no old in-memory parts to flush. Sleep for a while until these parts appear.
|
||||
for {
|
||||
select {
|
||||
case <-ddb.stopCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
ddb.mustFlushInmemoryPartsToFiles(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// startMergeWorkerLocked starts a merge worker.
|
||||
func (ddb *datadb) mustFlushInmemoryPartsToFiles(isFinal bool) {
|
||||
currentTime := time.Now()
|
||||
var pws []*partWrapper
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
for _, pw := range ddb.inmemoryParts {
|
||||
if !pw.isInMerge && (isFinal || pw.flushDeadline.Before(currentTime)) {
|
||||
pw.isInMerge = true
|
||||
pws = append(pws, pw)
|
||||
}
|
||||
}
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
ddb.mustMergePartsToFiles(pws)
|
||||
}
|
||||
|
||||
func (ddb *datadb) mustMergePartsToFiles(pws []*partWrapper) {
|
||||
wg := getWaitGroup()
|
||||
for len(pws) > 0 {
|
||||
pwsToMerge, pwsRemaining := getPartsForOptimalMerge(pws)
|
||||
wg.Add(1)
|
||||
inmemoryPartsConcurrencyCh <- struct{}{}
|
||||
go func(pwsChunk []*partWrapper) {
|
||||
defer func() {
|
||||
<-inmemoryPartsConcurrencyCh
|
||||
wg.Done()
|
||||
}()
|
||||
|
||||
ddb.mustMergeParts(pwsChunk, true)
|
||||
}(pwsToMerge)
|
||||
pws = pwsRemaining
|
||||
}
|
||||
wg.Wait()
|
||||
putWaitGroup(wg)
|
||||
}
|
||||
|
||||
// getPartsForOptimalMerge returns parts from pws for optimal merge, plus the remaining parts.
|
||||
//
|
||||
// This function must be called under locked partsLock.
|
||||
func (ddb *datadb) startMergeWorkerLocked() {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
// the pws items are replaced by nil after the call. This is needed for helping Go GC to reclaim the referenced items.
|
||||
func getPartsForOptimalMerge(pws []*partWrapper) ([]*partWrapper, []*partWrapper) {
|
||||
pwsToMerge := appendPartsToMerge(nil, pws, math.MaxUint64)
|
||||
if len(pwsToMerge) == 0 {
|
||||
return pws, nil
|
||||
}
|
||||
maxWorkers := getMergeWorkersCount()
|
||||
if ddb.mergeWorkersCount >= maxWorkers {
|
||||
return
|
||||
|
||||
m := partsToMap(pwsToMerge)
|
||||
pwsRemaining := make([]*partWrapper, 0, len(pws)-len(pwsToMerge))
|
||||
for _, pw := range pws {
|
||||
if _, ok := m[pw]; !ok {
|
||||
pwsRemaining = append(pwsRemaining, pw)
|
||||
}
|
||||
}
|
||||
ddb.mergeWorkersCount++
|
||||
ddb.wg.Add(1)
|
||||
go func() {
|
||||
globalMergeLimitCh <- struct{}{}
|
||||
ddb.mustMergeExistingParts()
|
||||
<-globalMergeLimitCh
|
||||
ddb.wg.Done()
|
||||
}()
|
||||
|
||||
// Clear references to pws items, so they could be reclaimed faster by Go GC.
|
||||
for i := range pws {
|
||||
pws[i] = nil
|
||||
}
|
||||
|
||||
return pwsToMerge, pwsRemaining
|
||||
}
|
||||
|
||||
// globalMergeLimitCh limits the number of concurrent merges across all the partitions
|
||||
var globalMergeLimitCh = make(chan struct{}, getMergeWorkersCount())
|
||||
|
||||
func getMergeWorkersCount() int {
|
||||
n := cgroup.AvailableCPUs()
|
||||
if n < 4 {
|
||||
// Use bigger number of workers on systems with small number of CPU cores,
|
||||
// since a single worker may become busy for long time when merging big parts.
|
||||
// Then the remaining workers may continue performing merges
|
||||
// for newly added small parts.
|
||||
return 4
|
||||
func getWaitGroup() *sync.WaitGroup {
|
||||
v := wgPool.Get()
|
||||
if v == nil {
|
||||
return &sync.WaitGroup{}
|
||||
}
|
||||
return n
|
||||
return v.(*sync.WaitGroup)
|
||||
}
|
||||
|
||||
func (ddb *datadb) mustMergeExistingParts() {
|
||||
for !needStop(ddb.stopCh) {
|
||||
maxOutBytes := availableDiskSpace(ddb.path)
|
||||
func putWaitGroup(wg *sync.WaitGroup) {
|
||||
wgPool.Put(wg)
|
||||
}
|
||||
|
||||
var wgPool sync.Pool
|
||||
|
||||
func (ddb *datadb) inmemoryPartsMerger() {
|
||||
for {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
maxOutBytes := ddb.getMaxBigPartSize()
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
parts := make([]*partWrapper, 0, len(ddb.inmemoryParts)+len(ddb.fileParts))
|
||||
parts = appendNotInMergePartsLocked(parts, ddb.inmemoryParts)
|
||||
parts = appendNotInMergePartsLocked(parts, ddb.fileParts)
|
||||
pws := appendPartsToMerge(nil, parts, maxOutBytes)
|
||||
setInMergeLocked(pws)
|
||||
if len(pws) == 0 {
|
||||
ddb.mergeWorkersCount--
|
||||
}
|
||||
pws := getPartsToMergeLocked(ddb.inmemoryParts, maxOutBytes)
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
if len(pws) == 0 {
|
||||
// Nothing to merge at the moment.
|
||||
// Nothing to merge
|
||||
return
|
||||
}
|
||||
|
||||
inmemoryPartsConcurrencyCh <- struct{}{}
|
||||
ddb.mustMergeParts(pws, false)
|
||||
<-inmemoryPartsConcurrencyCh
|
||||
}
|
||||
}
|
||||
|
||||
// appendNotInMergePartsLocked appends src parts with isInMerge=false to dst and returns the result.
|
||||
func (ddb *datadb) smallPartsMerger() {
|
||||
for {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
maxOutBytes := ddb.getMaxBigPartSize()
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
pws := getPartsToMergeLocked(ddb.smallParts, maxOutBytes)
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
if len(pws) == 0 {
|
||||
// Nothing to merge
|
||||
return
|
||||
}
|
||||
|
||||
smallPartsConcurrencyCh <- struct{}{}
|
||||
ddb.mustMergeParts(pws, false)
|
||||
<-smallPartsConcurrencyCh
|
||||
}
|
||||
}
|
||||
|
||||
func (ddb *datadb) bigPartsMerger() {
|
||||
for {
|
||||
if needStop(ddb.stopCh) {
|
||||
return
|
||||
}
|
||||
maxOutBytes := ddb.getMaxBigPartSize()
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
pws := getPartsToMergeLocked(ddb.bigParts, maxOutBytes)
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
if len(pws) == 0 {
|
||||
// Nothing to merge
|
||||
return
|
||||
}
|
||||
|
||||
bigPartsConcurrencyCh <- struct{}{}
|
||||
ddb.mustMergeParts(pws, false)
|
||||
<-bigPartsConcurrencyCh
|
||||
}
|
||||
}
|
||||
|
||||
// getPartsToMergeLocked returns optimal parts to merge from pws.
|
||||
//
|
||||
// This function must be called under partsLock.
|
||||
func appendNotInMergePartsLocked(dst, src []*partWrapper) []*partWrapper {
|
||||
for _, pw := range src {
|
||||
// The summary size of the returned parts must be smaller than maxOutBytes.
|
||||
func getPartsToMergeLocked(pws []*partWrapper, maxOutBytes uint64) []*partWrapper {
|
||||
pwsRemaining := make([]*partWrapper, 0, len(pws))
|
||||
for _, pw := range pws {
|
||||
if !pw.isInMerge {
|
||||
dst = append(dst, pw)
|
||||
pwsRemaining = append(pwsRemaining, pw)
|
||||
}
|
||||
}
|
||||
return dst
|
||||
}
|
||||
|
||||
// setInMergeLocked sets isInMerge flag for pws.
|
||||
//
|
||||
// This function must be called under partsLock.
|
||||
func setInMergeLocked(pws []*partWrapper) {
|
||||
for _, pw := range pws {
|
||||
pwsToMerge := appendPartsToMerge(nil, pwsRemaining, maxOutBytes)
|
||||
|
||||
for _, pw := range pwsToMerge {
|
||||
if pw.isInMerge {
|
||||
logger.Panicf("BUG: partWrapper.isInMerge unexpectedly set to true")
|
||||
logger.Panicf("BUG: partWrapper.isInMerge cannot be set")
|
||||
}
|
||||
pw.isInMerge = true
|
||||
}
|
||||
|
||||
return pwsToMerge
|
||||
}
|
||||
|
||||
func assertIsInMerge(pws []*partWrapper) {
|
||||
|
@ -370,7 +475,7 @@ func (ddb *datadb) mustMergeParts(pws []*partWrapper, isFinal bool) {
|
|||
startTime := time.Now()
|
||||
|
||||
dstPartType := ddb.getDstPartType(pws, isFinal)
|
||||
if dstPartType == partFile {
|
||||
if dstPartType != partInmemory {
|
||||
// Make sure there is enough disk space for performing the merge
|
||||
partsSize := getCompressedSize(pws)
|
||||
needReleaseDiskSpace := tryReserveDiskSpace(ddb.path, partsSize)
|
||||
|
@ -387,14 +492,21 @@ func (ddb *datadb) mustMergeParts(pws []*partWrapper, isFinal bool) {
|
|||
}
|
||||
}
|
||||
|
||||
if dstPartType == partInmemory {
|
||||
switch dstPartType {
|
||||
case partInmemory:
|
||||
ddb.inmemoryMergesTotal.Add(1)
|
||||
ddb.inmemoryActiveMerges.Add(1)
|
||||
defer ddb.inmemoryActiveMerges.Add(-1)
|
||||
} else {
|
||||
ddb.fileMergesTotal.Add(1)
|
||||
ddb.fileActiveMerges.Add(1)
|
||||
defer ddb.fileActiveMerges.Add(-1)
|
||||
case partSmall:
|
||||
ddb.smallPartMergesTotal.Add(1)
|
||||
ddb.smallPartActiveMerges.Add(1)
|
||||
defer ddb.smallPartActiveMerges.Add(-1)
|
||||
case partBig:
|
||||
ddb.bigPartMergesTotal.Add(1)
|
||||
ddb.bigPartActiveMerges.Add(1)
|
||||
defer ddb.bigPartActiveMerges.Add(-1)
|
||||
default:
|
||||
logger.Panicf("BUG: unknown partType=%d", dstPartType)
|
||||
}
|
||||
|
||||
// Initialize destination paths.
|
||||
|
@ -428,7 +540,7 @@ func (ddb *datadb) mustMergeParts(pws []*partWrapper, isFinal bool) {
|
|||
mpNew = getInmemoryPart()
|
||||
bsw.MustInitForInmemoryPart(mpNew)
|
||||
} else {
|
||||
nocache := !shouldUsePageCacheForPartSize(srcSize)
|
||||
nocache := dstPartType == partBig
|
||||
bsw.MustInitForFilePart(dstPartPath, nocache)
|
||||
}
|
||||
|
||||
|
@ -455,7 +567,7 @@ func (ddb *datadb) mustMergeParts(pws []*partWrapper, isFinal bool) {
|
|||
}
|
||||
if needStop(stopCh) {
|
||||
// Remove incomplete destination part
|
||||
if dstPartType == partFile {
|
||||
if dstPartType != partInmemory {
|
||||
fs.MustRemoveAll(dstPartPath)
|
||||
}
|
||||
return
|
||||
|
@ -477,7 +589,7 @@ func (ddb *datadb) mustMergeParts(pws []*partWrapper, isFinal bool) {
|
|||
ddb.swapSrcWithDstParts(pws, pwNew, dstPartType)
|
||||
|
||||
d := time.Since(startTime)
|
||||
if d <= 30*time.Second {
|
||||
if d <= time.Minute {
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -496,21 +608,22 @@ type partType int
|
|||
|
||||
var (
|
||||
partInmemory = partType(0)
|
||||
partFile = partType(1)
|
||||
partSmall = partType(1)
|
||||
partBig = partType(2)
|
||||
)
|
||||
|
||||
func (ddb *datadb) getDstPartType(pws []*partWrapper, isFinal bool) partType {
|
||||
if isFinal {
|
||||
return partFile
|
||||
}
|
||||
dstPartSize := getCompressedSize(pws)
|
||||
if dstPartSize > getMaxInmemoryPartSize() {
|
||||
return partFile
|
||||
if dstPartSize > ddb.getMaxSmallPartSize() {
|
||||
return partBig
|
||||
}
|
||||
if isFinal || dstPartSize > getMaxInmemoryPartSize() {
|
||||
return partSmall
|
||||
}
|
||||
if !areAllInmemoryParts(pws) {
|
||||
// If at least a single source part is located in file,
|
||||
// then the destination part must be in file for durability reasons.
|
||||
return partFile
|
||||
return partSmall
|
||||
}
|
||||
return partInmemory
|
||||
}
|
||||
|
@ -551,54 +664,19 @@ func (ddb *datadb) mustAddRows(lr *LogRows) {
|
|||
return
|
||||
}
|
||||
|
||||
inmemoryPartsConcurrencyCh <- struct{}{}
|
||||
mp := getInmemoryPart()
|
||||
mp.mustInitFromRows(lr)
|
||||
p := mustOpenInmemoryPart(ddb.pt, mp)
|
||||
<-inmemoryPartsConcurrencyCh
|
||||
|
||||
flushDeadline := time.Now().Add(ddb.flushInterval)
|
||||
pw := newPartWrapper(p, mp, flushDeadline)
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
ddb.inmemoryParts = append(ddb.inmemoryParts, pw)
|
||||
ddb.startOldInmemoryPartsFlusherLocked()
|
||||
if len(ddb.inmemoryParts) > defaultPartsToMerge {
|
||||
ddb.startMergeWorkerLocked()
|
||||
}
|
||||
needAssistedMerge := ddb.needAssistedMergeForInmemoryPartsLocked()
|
||||
ddb.startInmemoryPartsMergerLocked()
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
if needAssistedMerge {
|
||||
ddb.assistedMergeForInmemoryParts()
|
||||
}
|
||||
}
|
||||
|
||||
func (ddb *datadb) needAssistedMergeForInmemoryPartsLocked() bool {
|
||||
if len(ddb.inmemoryParts) < maxInmemoryPartsPerPartition {
|
||||
return false
|
||||
}
|
||||
n := 0
|
||||
for _, pw := range ddb.inmemoryParts {
|
||||
if !pw.isInMerge {
|
||||
n++
|
||||
}
|
||||
}
|
||||
return n >= defaultPartsToMerge
|
||||
}
|
||||
|
||||
func (ddb *datadb) assistedMergeForInmemoryParts() {
|
||||
ddb.partsLock.Lock()
|
||||
parts := make([]*partWrapper, 0, len(ddb.inmemoryParts))
|
||||
parts = appendNotInMergePartsLocked(parts, ddb.inmemoryParts)
|
||||
// Do not take into account available disk space when merging in-memory parts,
|
||||
// since otherwise the outdated in-memory parts may remain in-memory, which, in turn,
|
||||
// may result in increased memory usage plus possible loss of historical data.
|
||||
// It is better to crash on out of disk error in this case.
|
||||
pws := make([]*partWrapper, 0, len(parts))
|
||||
pws = appendPartsToMerge(pws[:0], parts, math.MaxUint64)
|
||||
setInMergeLocked(pws)
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
ddb.mustMergeParts(pws, false)
|
||||
}
|
||||
|
||||
// DatadbStats contains various stats for datadb.
|
||||
|
@ -609,41 +687,62 @@ type DatadbStats struct {
|
|||
// InmemoryActiveMerges is the number of currently active inmemory merges performed by the given datadb.
|
||||
InmemoryActiveMerges uint64
|
||||
|
||||
// FileMergesTotal is the number of file merges performed in the given datadb.
|
||||
FileMergesTotal uint64
|
||||
// SmallPartMergesTotal is the number of small file merges performed in the given datadb.
|
||||
SmallPartMergesTotal uint64
|
||||
|
||||
// FileActiveMerges is the number of currently active file merges performed by the given datadb.
|
||||
FileActiveMerges uint64
|
||||
// SmallPartActiveMerges is the number of currently active small file merges performed by the given datadb.
|
||||
SmallPartActiveMerges uint64
|
||||
|
||||
// BigPartMergesTotal is the number of big file merges performed in the given datadb.
|
||||
BigPartMergesTotal uint64
|
||||
|
||||
// BigPartActiveMerges is the number of currently active big file merges performed by the given datadb.
|
||||
BigPartActiveMerges uint64
|
||||
|
||||
// InmemoryRowsCount is the number of rows, which weren't flushed to disk yet.
|
||||
InmemoryRowsCount uint64
|
||||
|
||||
// FileRowsCount is the number of rows stored on disk.
|
||||
FileRowsCount uint64
|
||||
// SmallPartRowsCount is the number of rows stored on disk in small parts.
|
||||
SmallPartRowsCount uint64
|
||||
|
||||
// BigPartRowsCount is the number of rows stored on disk in big parts.
|
||||
BigPartRowsCount uint64
|
||||
|
||||
// InmemoryParts is the number of in-memory parts, which weren't flushed to disk yet.
|
||||
InmemoryParts uint64
|
||||
|
||||
// FileParts is the number of file-based parts stored on disk.
|
||||
FileParts uint64
|
||||
// SmallParts is the number of file-based small parts stored on disk.
|
||||
SmallParts uint64
|
||||
|
||||
// BigParts is the number of file-based big parts stored on disk.
|
||||
BigParts uint64
|
||||
|
||||
// InmemoryBlocks is the number of in-memory blocks, which weren't flushed to disk yet.
|
||||
InmemoryBlocks uint64
|
||||
|
||||
// FileBlocks is the number of file-based blocks stored on disk.
|
||||
FileBlocks uint64
|
||||
// SmallPartBlocks is the number of file-based small blocks stored on disk.
|
||||
SmallPartBlocks uint64
|
||||
|
||||
// BigPartBlocks is the number of file-based big blocks stored on disk.
|
||||
BigPartBlocks uint64
|
||||
|
||||
// CompressedInmemorySize is the size of compressed data stored in memory.
|
||||
CompressedInmemorySize uint64
|
||||
|
||||
// CompressedFileSize is the size of compressed data stored on disk.
|
||||
CompressedFileSize uint64
|
||||
// CompressedSmallPartSize is the size of compressed small parts data stored on disk.
|
||||
CompressedSmallPartSize uint64
|
||||
|
||||
// CompressedBigPartSize is the size of compressed big data stored on disk.
|
||||
CompressedBigPartSize uint64
|
||||
|
||||
// UncompressedInmemorySize is the size of uncompressed data stored in memory.
|
||||
UncompressedInmemorySize uint64
|
||||
|
||||
// UncompressedFileSize is the size of uncompressed data stored on disk.
|
||||
UncompressedFileSize uint64
|
||||
// UncompressedSmallPartSize is the size of uncompressed small data stored on disk.
|
||||
UncompressedSmallPartSize uint64
|
||||
|
||||
// UncompressedBigPartSize is the size of uncompressed big data stored on disk.
|
||||
UncompressedBigPartSize uint64
|
||||
}
|
||||
|
||||
func (s *DatadbStats) reset() {
|
||||
|
@ -652,32 +751,39 @@ func (s *DatadbStats) reset() {
|
|||
|
||||
// RowsCount returns the number of rows stored in datadb.
|
||||
func (s *DatadbStats) RowsCount() uint64 {
|
||||
return s.InmemoryRowsCount + s.FileRowsCount
|
||||
return s.InmemoryRowsCount + s.SmallPartRowsCount + s.BigPartRowsCount
|
||||
}
|
||||
|
||||
// updateStats updates s with ddb stats
|
||||
// updateStats updates s with ddb stats.
|
||||
func (ddb *datadb) updateStats(s *DatadbStats) {
|
||||
s.InmemoryMergesTotal += ddb.inmemoryMergesTotal.Load()
|
||||
s.InmemoryActiveMerges += uint64(ddb.inmemoryActiveMerges.Load())
|
||||
s.FileMergesTotal += ddb.fileMergesTotal.Load()
|
||||
s.FileActiveMerges += uint64(ddb.fileActiveMerges.Load())
|
||||
s.SmallPartMergesTotal += ddb.smallPartMergesTotal.Load()
|
||||
s.SmallPartActiveMerges += uint64(ddb.smallPartActiveMerges.Load())
|
||||
s.BigPartMergesTotal += ddb.bigPartMergesTotal.Load()
|
||||
s.BigPartActiveMerges += uint64(ddb.bigPartActiveMerges.Load())
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
|
||||
s.InmemoryRowsCount += getRowsCount(ddb.inmemoryParts)
|
||||
s.FileRowsCount += getRowsCount(ddb.fileParts)
|
||||
s.SmallPartRowsCount += getRowsCount(ddb.smallParts)
|
||||
s.BigPartRowsCount += getRowsCount(ddb.bigParts)
|
||||
|
||||
s.InmemoryParts += uint64(len(ddb.inmemoryParts))
|
||||
s.FileParts += uint64(len(ddb.fileParts))
|
||||
s.SmallParts += uint64(len(ddb.smallParts))
|
||||
s.BigParts += uint64(len(ddb.bigParts))
|
||||
|
||||
s.InmemoryBlocks += getBlocksCount(ddb.inmemoryParts)
|
||||
s.FileBlocks += getBlocksCount(ddb.fileParts)
|
||||
s.SmallPartBlocks += getBlocksCount(ddb.smallParts)
|
||||
s.BigPartBlocks += getBlocksCount(ddb.bigParts)
|
||||
|
||||
s.CompressedInmemorySize += getCompressedSize(ddb.inmemoryParts)
|
||||
s.CompressedFileSize += getCompressedSize(ddb.fileParts)
|
||||
s.CompressedSmallPartSize += getCompressedSize(ddb.smallParts)
|
||||
s.CompressedBigPartSize += getCompressedSize(ddb.bigParts)
|
||||
|
||||
s.UncompressedInmemorySize += getUncompressedSize(ddb.inmemoryParts)
|
||||
s.UncompressedFileSize += getUncompressedSize(ddb.fileParts)
|
||||
s.UncompressedSmallPartSize += getUncompressedSize(ddb.smallParts)
|
||||
s.UncompressedBigPartSize += getUncompressedSize(ddb.bigParts)
|
||||
|
||||
ddb.partsLock.Unlock()
|
||||
}
|
||||
|
@ -687,29 +793,56 @@ func (ddb *datadb) debugFlush() {
|
|||
// Nothing to do, since all the ingested data is available for search via ddb.inmemoryParts.
|
||||
}
|
||||
|
||||
func (ddb *datadb) mustFlushInmemoryPartsToDisk() {
|
||||
func (ddb *datadb) swapSrcWithDstParts(pws []*partWrapper, pwNew *partWrapper, dstPartType partType) {
|
||||
// Atomically unregister old parts and add new part to pt.
|
||||
partsToRemove := partsToMap(pws)
|
||||
|
||||
removedInmemoryParts := 0
|
||||
removedSmallParts := 0
|
||||
removedBigParts := 0
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
pws := append([]*partWrapper{}, ddb.inmemoryParts...)
|
||||
setInMergeLocked(pws)
|
||||
|
||||
ddb.inmemoryParts, removedInmemoryParts = removeParts(ddb.inmemoryParts, partsToRemove)
|
||||
ddb.smallParts, removedSmallParts = removeParts(ddb.smallParts, partsToRemove)
|
||||
ddb.bigParts, removedBigParts = removeParts(ddb.bigParts, partsToRemove)
|
||||
|
||||
if pwNew != nil {
|
||||
switch dstPartType {
|
||||
case partInmemory:
|
||||
ddb.inmemoryParts = append(ddb.inmemoryParts, pwNew)
|
||||
ddb.startInmemoryPartsMergerLocked()
|
||||
case partSmall:
|
||||
ddb.smallParts = append(ddb.smallParts, pwNew)
|
||||
ddb.startSmallPartsMergerLocked()
|
||||
case partBig:
|
||||
ddb.bigParts = append(ddb.bigParts, pwNew)
|
||||
ddb.startBigPartsMergerLocked()
|
||||
default:
|
||||
logger.Panicf("BUG: unknown partType=%d", dstPartType)
|
||||
}
|
||||
}
|
||||
|
||||
// Atomically store the updated list of file-based parts on disk.
|
||||
// This must be performed under partsLock in order to prevent from races
|
||||
// when multiple concurrently running goroutines update the list.
|
||||
if removedSmallParts > 0 || removedBigParts > 0 || pwNew != nil && dstPartType != partInmemory {
|
||||
smallPartNames := getPartNames(ddb.smallParts)
|
||||
bigPartNames := getPartNames(ddb.bigParts)
|
||||
mustWritePartNames(ddb.path, smallPartNames, bigPartNames)
|
||||
}
|
||||
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
var pwsChunk []*partWrapper
|
||||
for len(pws) > 0 {
|
||||
// Do not take into account available disk space when performing the final flush of in-memory parts to disk,
|
||||
// since otherwise these parts will be lost.
|
||||
// It is better to crash on out of disk error in this case.
|
||||
pwsChunk = appendPartsToMerge(pwsChunk[:0], pws, math.MaxUint64)
|
||||
if len(pwsChunk) == 0 {
|
||||
pwsChunk = append(pwsChunk[:0], pws...)
|
||||
}
|
||||
partsToRemove := partsToMap(pwsChunk)
|
||||
removedParts := 0
|
||||
pws, removedParts = removeParts(pws, partsToRemove)
|
||||
if removedParts != len(pwsChunk) {
|
||||
logger.Panicf("BUG: unexpected number of parts removed; got %d; want %d", removedParts, len(pwsChunk))
|
||||
}
|
||||
removedParts := removedInmemoryParts + removedSmallParts + removedBigParts
|
||||
if removedParts != len(partsToRemove) {
|
||||
logger.Panicf("BUG: unexpected number of parts removed; got %d, want %d", removedParts, len(partsToRemove))
|
||||
}
|
||||
|
||||
ddb.mustMergeParts(pwsChunk, true)
|
||||
// Mark old parts as must be deleted and decrement reference count, so they are eventually closed and deleted.
|
||||
for _, pw := range pws {
|
||||
pw.mustDrop.Store(true)
|
||||
pw.decRef()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -724,54 +857,6 @@ func partsToMap(pws []*partWrapper) map[*partWrapper]struct{} {
|
|||
return m
|
||||
}
|
||||
|
||||
func (ddb *datadb) swapSrcWithDstParts(pws []*partWrapper, pwNew *partWrapper, dstPartType partType) {
|
||||
// Atomically unregister old parts and add new part to pt.
|
||||
partsToRemove := partsToMap(pws)
|
||||
removedInmemoryParts := 0
|
||||
removedFileParts := 0
|
||||
|
||||
ddb.partsLock.Lock()
|
||||
|
||||
ddb.inmemoryParts, removedInmemoryParts = removeParts(ddb.inmemoryParts, partsToRemove)
|
||||
ddb.fileParts, removedFileParts = removeParts(ddb.fileParts, partsToRemove)
|
||||
if pwNew != nil {
|
||||
switch dstPartType {
|
||||
case partInmemory:
|
||||
ddb.inmemoryParts = append(ddb.inmemoryParts, pwNew)
|
||||
ddb.startOldInmemoryPartsFlusherLocked()
|
||||
case partFile:
|
||||
ddb.fileParts = append(ddb.fileParts, pwNew)
|
||||
default:
|
||||
logger.Panicf("BUG: unknown partType=%d", dstPartType)
|
||||
}
|
||||
if len(ddb.inmemoryParts)+len(ddb.fileParts) > defaultPartsToMerge {
|
||||
ddb.startMergeWorkerLocked()
|
||||
}
|
||||
}
|
||||
|
||||
// Atomically store the updated list of file-based parts on disk.
|
||||
// This must be performed under partsLock in order to prevent from races
|
||||
// when multiple concurrently running goroutines update the list.
|
||||
if removedFileParts > 0 || pwNew != nil && dstPartType == partFile {
|
||||
partNames := getPartNames(ddb.fileParts)
|
||||
mustWritePartNames(ddb.path, partNames)
|
||||
}
|
||||
|
||||
ddb.partsLock.Unlock()
|
||||
|
||||
removedParts := removedInmemoryParts + removedFileParts
|
||||
if removedParts != len(partsToRemove) {
|
||||
logger.Panicf("BUG: unexpected number of parts removed; got %d, want %d", removedParts, len(partsToRemove))
|
||||
}
|
||||
|
||||
// Mark old parts as must be deleted and decrement reference count,
|
||||
// so they are eventually closed and deleted.
|
||||
for _, pw := range pws {
|
||||
pw.mustDrop.Store(true)
|
||||
pw.decRef()
|
||||
}
|
||||
}
|
||||
|
||||
func removeParts(pws []*partWrapper, partsToRemove map[*partWrapper]struct{}) ([]*partWrapper, int) {
|
||||
dst := pws[:0]
|
||||
for _, pw := range pws {
|
||||
|
@ -853,6 +938,34 @@ func (ddb *datadb) releasePartsToMerge(pws []*partWrapper) {
|
|||
ddb.partsLock.Unlock()
|
||||
}
|
||||
|
||||
func (ddb *datadb) getMaxBigPartSize() uint64 {
|
||||
return getMaxOutBytes(ddb.path)
|
||||
}
|
||||
|
||||
func (ddb *datadb) getMaxSmallPartSize() uint64 {
|
||||
// Small parts are cached in the OS page cache,
|
||||
// so limit their size by the remaining free RAM.
|
||||
mem := memory.Remaining()
|
||||
n := uint64(mem) / defaultPartsToMerge
|
||||
if n < 10e6 {
|
||||
n = 10e6
|
||||
}
|
||||
// Make sure the output part fits available disk space for small parts.
|
||||
sizeLimit := getMaxOutBytes(ddb.path)
|
||||
if n > sizeLimit {
|
||||
n = sizeLimit
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func getMaxOutBytes(path string) uint64 {
|
||||
n := availableDiskSpace(path)
|
||||
if n > maxBigPartSize {
|
||||
n = maxBigPartSize
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func availableDiskSpace(path string) uint64 {
|
||||
available := fs.MustGetFreeSpace(path)
|
||||
reserved := reservedDiskSpace.Load()
|
||||
|
@ -865,7 +978,7 @@ func availableDiskSpace(path string) uint64 {
|
|||
func tryReserveDiskSpace(path string, n uint64) bool {
|
||||
available := fs.MustGetFreeSpace(path)
|
||||
reserved := reserveDiskSpace(n)
|
||||
if available > reserved {
|
||||
if available >= reserved {
|
||||
return true
|
||||
}
|
||||
releaseDiskSpace(n)
|
||||
|
@ -908,20 +1021,29 @@ func mustCloseDatadb(ddb *datadb) {
|
|||
ddb.wg.Wait()
|
||||
|
||||
// flush in-memory data to disk
|
||||
ddb.mustFlushInmemoryPartsToDisk()
|
||||
ddb.mustFlushInmemoryPartsToFiles(true)
|
||||
if len(ddb.inmemoryParts) > 0 {
|
||||
logger.Panicf("BUG: the number of in-memory parts must be zero after flushing them to disk; got %d", len(ddb.inmemoryParts))
|
||||
}
|
||||
ddb.inmemoryParts = nil
|
||||
|
||||
// close file parts
|
||||
for _, pw := range ddb.fileParts {
|
||||
// close small parts
|
||||
for _, pw := range ddb.smallParts {
|
||||
pw.decRef()
|
||||
if n := pw.refCount.Load(); n != 0 {
|
||||
logger.Panicf("BUG: ther are %d references to filePart", n)
|
||||
logger.Panicf("BUG: there are %d references to smallPart", n)
|
||||
}
|
||||
}
|
||||
ddb.fileParts = nil
|
||||
ddb.smallParts = nil
|
||||
|
||||
// close big parts
|
||||
for _, pw := range ddb.bigParts {
|
||||
pw.decRef()
|
||||
if n := pw.refCount.Load(); n != 0 {
|
||||
logger.Panicf("BUG: there are %d references to bigPart", n)
|
||||
}
|
||||
}
|
||||
ddb.bigParts = nil
|
||||
|
||||
ddb.path = ""
|
||||
ddb.pt = nil
|
||||
|
@ -941,7 +1063,9 @@ func getPartNames(pws []*partWrapper) []string {
|
|||
return partNames
|
||||
}
|
||||
|
||||
func mustWritePartNames(path string, partNames []string) {
|
||||
func mustWritePartNames(path string, smallPartNames, bigPartNames []string) {
|
||||
partNames := append([]string{}, smallPartNames...)
|
||||
partNames = append(partNames, bigPartNames...)
|
||||
data, err := json.Marshal(partNames)
|
||||
if err != nil {
|
||||
logger.Panicf("BUG: cannot marshal partNames to JSON: %s", err)
|
||||
|
@ -1102,8 +1226,3 @@ func getBlocksCount(pws []*partWrapper) uint64 {
|
|||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func shouldUsePageCacheForPartSize(size uint64) bool {
|
||||
mem := memory.Remaining() / defaultPartsToMerge
|
||||
return size <= uint64(mem)
|
||||
}
|
||||
|
|
|
@ -47,6 +47,22 @@ func (sbu *stringsBlockUnmarshaler) reset() {
|
|||
sbu.data = sbu.data[:0]
|
||||
}
|
||||
|
||||
func (sbu *stringsBlockUnmarshaler) copyString(s string) string {
|
||||
dataLen := len(sbu.data)
|
||||
sbu.data = append(sbu.data, s...)
|
||||
return bytesutil.ToUnsafeString(sbu.data[dataLen:])
|
||||
}
|
||||
|
||||
func (sbu *stringsBlockUnmarshaler) appendFields(dst, src []Field) []Field {
|
||||
for _, f := range src {
|
||||
dst = append(dst, Field{
|
||||
Name: sbu.copyString(f.Name),
|
||||
Value: sbu.copyString(f.Value),
|
||||
})
|
||||
}
|
||||
return dst
|
||||
}
|
||||
|
||||
// unmarshal unmarshals itemsCount strings from src, appends them to dst and returns the result.
|
||||
//
|
||||
// The returned strings are valid until sbu.reset() call.
|
||||
|
|
80
lib/logstorage/fields_set.go
Normal file
80
lib/logstorage/fields_set.go
Normal file
|
@ -0,0 +1,80 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"strings"
|
||||
)
|
||||
|
||||
type fieldsSet map[string]struct{}
|
||||
|
||||
func newFieldsSet() fieldsSet {
|
||||
return fieldsSet(map[string]struct{}{})
|
||||
}
|
||||
|
||||
func (fs fieldsSet) reset() {
|
||||
clear(fs)
|
||||
}
|
||||
|
||||
func (fs fieldsSet) String() string {
|
||||
a := fs.getAll()
|
||||
return "[" + strings.Join(a, ",") + "]"
|
||||
}
|
||||
|
||||
func (fs fieldsSet) clone() fieldsSet {
|
||||
fsNew := newFieldsSet()
|
||||
for _, f := range fs.getAll() {
|
||||
fsNew.add(f)
|
||||
}
|
||||
return fsNew
|
||||
}
|
||||
|
||||
func (fs fieldsSet) getAll() []string {
|
||||
a := make([]string, 0, len(fs))
|
||||
for f := range fs {
|
||||
a = append(a, f)
|
||||
}
|
||||
sort.Strings(a)
|
||||
return a
|
||||
}
|
||||
|
||||
func (fs fieldsSet) contains(field string) bool {
|
||||
_, ok := fs[field]
|
||||
if !ok {
|
||||
_, ok = fs["*"]
|
||||
}
|
||||
return ok
|
||||
}
|
||||
|
||||
func (fs fieldsSet) removeAll(fields []string) {
|
||||
for _, f := range fields {
|
||||
fs.remove(f)
|
||||
}
|
||||
}
|
||||
|
||||
func (fs fieldsSet) remove(field string) {
|
||||
if field == "*" {
|
||||
fs.reset()
|
||||
return
|
||||
}
|
||||
if !fs.contains("*") {
|
||||
delete(fs, field)
|
||||
}
|
||||
}
|
||||
|
||||
func (fs fieldsSet) addAll(fields []string) {
|
||||
for _, f := range fields {
|
||||
fs.add(f)
|
||||
}
|
||||
}
|
||||
|
||||
func (fs fieldsSet) add(field string) {
|
||||
if fs.contains("*") {
|
||||
return
|
||||
}
|
||||
if field == "*" {
|
||||
fs.reset()
|
||||
fs["*"] = struct{}{}
|
||||
return
|
||||
}
|
||||
fs[field] = struct{}{}
|
||||
}
|
94
lib/logstorage/fields_set_test.go
Normal file
94
lib/logstorage/fields_set_test.go
Normal file
|
@ -0,0 +1,94 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFieldsSet(t *testing.T) {
|
||||
fs := newFieldsSet()
|
||||
|
||||
// verify add, remove and contains
|
||||
if fs.contains("*") {
|
||||
t.Fatalf("fs mustn't contain *")
|
||||
}
|
||||
if fs.contains("foo") {
|
||||
t.Fatalf("fs musn't contain foo")
|
||||
}
|
||||
fs.add("foo")
|
||||
fs.add("bar")
|
||||
s := fs.String()
|
||||
if s != "[bar,foo]" {
|
||||
t.Fatalf("unexpected String() result; got %s; want %s", s, "[bar,foo]")
|
||||
}
|
||||
if !fs.contains("foo") {
|
||||
t.Fatalf("fs must contain foo")
|
||||
}
|
||||
if !fs.contains("bar") {
|
||||
t.Fatalf("fs must contain bar")
|
||||
}
|
||||
if fs.contains("baz") {
|
||||
t.Fatalf("fs musn't contain baz")
|
||||
}
|
||||
if fs.contains("*") {
|
||||
t.Fatalf("fs mustn't contain *")
|
||||
}
|
||||
fs.remove("foo")
|
||||
if fs.contains("foo") {
|
||||
t.Fatalf("fs mustn't contain foo")
|
||||
}
|
||||
fs.remove("bar")
|
||||
if fs.contains("bar") {
|
||||
t.Fatalf("fs mustn't contain bar")
|
||||
}
|
||||
|
||||
// verify *
|
||||
fs.add("*")
|
||||
if !fs.contains("*") {
|
||||
t.Fatalf("fs must contain *")
|
||||
}
|
||||
if !fs.contains("foo") || !fs.contains("bar") || !fs.contains("baz") {
|
||||
t.Fatalf("fs must contain anything")
|
||||
}
|
||||
fs.remove("foo")
|
||||
if !fs.contains("foo") {
|
||||
t.Fatalf("fs must contain anything")
|
||||
}
|
||||
fs.remove("*")
|
||||
if fs.contains("foo") || fs.contains("bar") || fs.contains("baz") {
|
||||
t.Fatalf("fs must be empty")
|
||||
}
|
||||
|
||||
// verify addAll, getAll, removeAll
|
||||
fs.addAll([]string{"foo", "bar"})
|
||||
if !fs.contains("foo") || !fs.contains("bar") {
|
||||
t.Fatalf("fs must contain foo and bar")
|
||||
}
|
||||
a := fs.getAll()
|
||||
if !reflect.DeepEqual(a, []string{"bar", "foo"}) {
|
||||
t.Fatalf("unexpected result from getAll(); got %q; want %q", a, []string{"bar", "foo"})
|
||||
}
|
||||
fs.removeAll([]string{"bar", "baz"})
|
||||
if fs.contains("bar") || fs.contains("baz") {
|
||||
t.Fatalf("fs mustn't contain bar and baz")
|
||||
}
|
||||
if !fs.contains("foo") {
|
||||
t.Fatalf("fs must contain foo")
|
||||
}
|
||||
|
||||
// verify clone
|
||||
fs.addAll([]string{"foo", "bar", "baz"})
|
||||
fsStr := fs.String()
|
||||
fsCopy := fs.clone()
|
||||
fsCopyStr := fsCopy.String()
|
||||
if fsStr != fsCopyStr {
|
||||
t.Fatalf("unexpected clone result; got %s; want %s", fsCopyStr, fsStr)
|
||||
}
|
||||
fsCopy.remove("foo")
|
||||
if fsCopy.contains("foo") {
|
||||
t.Fatalf("fsCopy mustn't contain foo")
|
||||
}
|
||||
if !fs.contains("foo") {
|
||||
t.Fatalf("fs must contain foo")
|
||||
}
|
||||
}
|
10
lib/logstorage/filter.go
Normal file
10
lib/logstorage/filter.go
Normal file
|
@ -0,0 +1,10 @@
|
|||
package logstorage
|
||||
|
||||
// filter must implement filtering for log entries.
|
||||
type filter interface {
|
||||
// String returns string representation of the filter
|
||||
String() string
|
||||
|
||||
// apply must update bm according to the filter applied to the given bs block
|
||||
apply(bs *blockSearch, bm *bitmap)
|
||||
}
|
91
lib/logstorage/filter_and.go
Normal file
91
lib/logstorage/filter_and.go
Normal file
|
@ -0,0 +1,91 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// filterAnd contains filters joined by AND opertor.
|
||||
//
|
||||
// It is expressed as `f1 AND f2 ... AND fN` in LogsQL.
|
||||
type filterAnd struct {
|
||||
filters []filter
|
||||
|
||||
msgTokensOnce sync.Once
|
||||
msgTokens []string
|
||||
}
|
||||
|
||||
func (fa *filterAnd) String() string {
|
||||
filters := fa.filters
|
||||
a := make([]string, len(filters))
|
||||
for i, f := range filters {
|
||||
s := f.String()
|
||||
switch f.(type) {
|
||||
case *filterOr:
|
||||
s = "(" + s + ")"
|
||||
}
|
||||
a[i] = s
|
||||
}
|
||||
return strings.Join(a, " ")
|
||||
}
|
||||
|
||||
func (fa *filterAnd) apply(bs *blockSearch, bm *bitmap) {
|
||||
if tokens := fa.getMsgTokens(); len(tokens) > 0 {
|
||||
// Verify whether fa tokens for the _msg field match bloom filter.
|
||||
ch := bs.csh.getColumnHeader("_msg")
|
||||
if ch == nil {
|
||||
// Fast path - there is no _msg field in the block.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
// Fast path - fa tokens for the _msg field do not match bloom filter.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path - verify every filter separately.
|
||||
for _, f := range fa.filters {
|
||||
f.apply(bs, bm)
|
||||
if bm.isZero() {
|
||||
// Shortcut - there is no need in applying the remaining filters,
|
||||
// since the result will be zero anyway.
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (fa *filterAnd) getMsgTokens() []string {
|
||||
fa.msgTokensOnce.Do(fa.initMsgTokens)
|
||||
return fa.msgTokens
|
||||
}
|
||||
|
||||
func (fa *filterAnd) initMsgTokens() {
|
||||
var a []string
|
||||
for _, f := range fa.filters {
|
||||
switch t := f.(type) {
|
||||
case *filterPhrase:
|
||||
if isMsgFieldName(t.fieldName) {
|
||||
a = append(a, t.getTokens()...)
|
||||
}
|
||||
case *filterSequence:
|
||||
if isMsgFieldName(t.fieldName) {
|
||||
a = append(a, t.getTokens()...)
|
||||
}
|
||||
case *filterExact:
|
||||
if isMsgFieldName(t.fieldName) {
|
||||
a = append(a, t.getTokens()...)
|
||||
}
|
||||
case *filterExactPrefix:
|
||||
if isMsgFieldName(t.fieldName) {
|
||||
a = append(a, t.getTokens()...)
|
||||
}
|
||||
case *filterPrefix:
|
||||
if isMsgFieldName(t.fieldName) {
|
||||
a = append(a, t.getTokens()...)
|
||||
}
|
||||
}
|
||||
}
|
||||
fa.msgTokens = a
|
||||
}
|
115
lib/logstorage/filter_and_test.go
Normal file
115
lib/logstorage/filter_and_test.go
Normal file
|
@ -0,0 +1,115 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterAnd(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"",
|
||||
"a foobar abcdef",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// non-empty intersection
|
||||
fa := &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", []int{2, 6})
|
||||
|
||||
// reverse non-empty intersection
|
||||
fa = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", []int{2, 6})
|
||||
|
||||
// the first filter mismatch
|
||||
fa = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bc",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", nil)
|
||||
|
||||
// the last filter mismatch
|
||||
fa = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "foo",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", nil)
|
||||
|
||||
// empty intersection
|
||||
fa = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foo",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", nil)
|
||||
|
||||
// reverse empty intersection
|
||||
fa = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foo",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fa, "foo", nil)
|
||||
}
|
150
lib/logstorage/filter_any_case_phrase.go
Normal file
150
lib/logstorage/filter_any_case_phrase.go
Normal file
|
@ -0,0 +1,150 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/stringsutil"
|
||||
)
|
||||
|
||||
// filterAnyCasePhrase filters field entries by case-insensitive phrase match.
|
||||
//
|
||||
// An example LogsQL query: `fieldName:i(word)` or `fieldName:i("word1 ... wordN")`
|
||||
type filterAnyCasePhrase struct {
|
||||
fieldName string
|
||||
phrase string
|
||||
|
||||
phraseLowercaseOnce sync.Once
|
||||
phraseLowercase string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) String() string {
|
||||
return fmt.Sprintf("%si(%s)", quoteFieldNameIfNeeded(fp.fieldName), quoteTokenIfNeeded(fp.phrase))
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) getTokens() []string {
|
||||
fp.tokensOnce.Do(fp.initTokens)
|
||||
return fp.tokens
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) initTokens() {
|
||||
fp.tokens = tokenizeStrings(nil, []string{fp.phrase})
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) getPhraseLowercase() string {
|
||||
fp.phraseLowercaseOnce.Do(fp.initPhraseLowercase)
|
||||
return fp.phraseLowercase
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) initPhraseLowercase() {
|
||||
fp.phraseLowercase = strings.ToLower(fp.phrase)
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePhrase) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fp.fieldName
|
||||
phraseLowercase := fp.getPhraseLowercase()
|
||||
|
||||
// Verify whether fp matches const column
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchAnyCasePhrase(v, phraseLowercase) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether fp matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
// It matches anything only for empty phrase.
|
||||
if len(phraseLowercase) > 0 {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fp.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByAnyCasePhrase(bs, ch, bm, phraseLowercase)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByAnyCasePhrase(bs, ch, bm, phraseLowercase)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByExactValue(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByExactValue(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByExactValue(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByExactValue(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByPhrase(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByPhrase(bs, ch, bm, phraseLowercase, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
phraseUppercase := strings.ToUpper(fp.phrase)
|
||||
matchTimestampISO8601ByPhrase(bs, ch, bm, phraseUppercase, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchValuesDictByAnyCasePhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phraseLowercase string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchAnyCasePhrase(v, phraseLowercase) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByAnyCasePhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phraseLowercase string) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchAnyCasePhrase(v, phraseLowercase)
|
||||
})
|
||||
}
|
||||
|
||||
func matchAnyCasePhrase(s, phraseLowercase string) bool {
|
||||
if len(phraseLowercase) == 0 {
|
||||
// Special case - empty phrase matches only empty string.
|
||||
return len(s) == 0
|
||||
}
|
||||
if len(phraseLowercase) > len(s) {
|
||||
return false
|
||||
}
|
||||
|
||||
if isASCIILowercase(s) {
|
||||
// Fast path - s is in lowercase
|
||||
return matchPhrase(s, phraseLowercase)
|
||||
}
|
||||
|
||||
// Slow path - convert s to lowercase before matching
|
||||
bb := bbPool.Get()
|
||||
bb.B = stringsutil.AppendLowercase(bb.B, s)
|
||||
sLowercase := bytesutil.ToUnsafeString(bb.B)
|
||||
ok := matchPhrase(sLowercase, phraseLowercase)
|
||||
bbPool.Put(bb)
|
||||
|
||||
return ok
|
||||
}
|
||||
|
||||
func isASCIILowercase(s string) bool {
|
||||
for i := 0; i < len(s); i++ {
|
||||
c := s[i]
|
||||
if c >= utf8.RuneSelf || (c >= 'A' && c <= 'Z') {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
888
lib/logstorage/filter_any_case_phrase_test.go
Normal file
888
lib/logstorage/filter_any_case_phrase_test.go
Normal file
|
@ -0,0 +1,888 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchAnyCasePhrase(t *testing.T) {
|
||||
f := func(s, phraseLowercase string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchAnyCasePhrase(s, phraseLowercase)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
// empty phrase matches only empty string
|
||||
f("", "", true)
|
||||
f("foo", "", false)
|
||||
f("тест", "", false)
|
||||
|
||||
// empty string doesn't match non-empty phrase
|
||||
f("", "foo", false)
|
||||
f("", "тест", false)
|
||||
|
||||
// full match
|
||||
f("foo", "foo", true)
|
||||
f("FOo", "foo", true)
|
||||
f("Test ТЕСт 123", "test тест 123", true)
|
||||
|
||||
// phrase match
|
||||
f("a foo", "foo", true)
|
||||
f("foo тест bar", "тест", true)
|
||||
f("foo ТЕСТ bar", "тест bar", true)
|
||||
|
||||
// mismatch
|
||||
f("foo", "fo", false)
|
||||
f("тест", "foo", false)
|
||||
f("Тест", "ест", false)
|
||||
}
|
||||
|
||||
func TestFilterAnyCasePhrase(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"aBc DEf",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other column",
|
||||
values: []string{
|
||||
"aSDfdsF",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "Abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "other column",
|
||||
phrase: "ASdfdsf",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "other column",
|
||||
phrase: "sd",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "other-column",
|
||||
values: []string{
|
||||
"X",
|
||||
"x",
|
||||
"x",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"aBC def",
|
||||
"abc DEf",
|
||||
"Abc deF",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: " def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "other-column",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: " 2 ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "other-column",
|
||||
phrase: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"fooBar",
|
||||
"ABc",
|
||||
"afdf foobar BAz",
|
||||
"fddf fOObARbaz",
|
||||
"AfooBarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "FoobAr",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 3, 6})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "baZ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"A Foobar",
|
||||
"aA aBC a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a aFOObarbaz",
|
||||
"a foobar",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "A",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "НгкШ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{8})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "!,",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "aa a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "@",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 5})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3, 4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65535",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "123456",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234.5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "-65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "-1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "+1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5678",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "2.3",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 2, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "27.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02t15:04:05.005z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{4})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "002Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{1})
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-03-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "06",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it has different string representation of the timestamp
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02T16:04:05.005+01:00",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it contains too many digits for millisecond part
|
||||
pf = &filterAnyCasePhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02T15:04:05.00500Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
})
|
||||
}
|
141
lib/logstorage/filter_any_case_prefix.go
Normal file
141
lib/logstorage/filter_any_case_prefix.go
Normal file
|
@ -0,0 +1,141 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/stringsutil"
|
||||
)
|
||||
|
||||
// filterAnyCasePrefix matches the given prefix in lower, upper and mixed case.
|
||||
//
|
||||
// Example LogsQL: `fieldName:i(prefix*)` or `fieldName:i("some prefix"*)`
|
||||
//
|
||||
// A special case `fieldName:i(*)` equals to `fieldName:*` and matches non-emtpy value for the given `fieldName` field.
|
||||
type filterAnyCasePrefix struct {
|
||||
fieldName string
|
||||
prefix string
|
||||
|
||||
prefixLowercaseOnce sync.Once
|
||||
prefixLowercase string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) String() string {
|
||||
if fp.prefix == "" {
|
||||
return quoteFieldNameIfNeeded(fp.fieldName) + "i(*)"
|
||||
}
|
||||
return fmt.Sprintf("%si(%s*)", quoteFieldNameIfNeeded(fp.fieldName), quoteTokenIfNeeded(fp.prefix))
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) getTokens() []string {
|
||||
fp.tokensOnce.Do(fp.initTokens)
|
||||
return fp.tokens
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) initTokens() {
|
||||
fp.tokens = getTokensSkipLast(fp.prefix)
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) getPrefixLowercase() string {
|
||||
fp.prefixLowercaseOnce.Do(fp.initPrefixLowercase)
|
||||
return fp.prefixLowercase
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) initPrefixLowercase() {
|
||||
fp.prefixLowercase = strings.ToLower(fp.prefix)
|
||||
}
|
||||
|
||||
func (fp *filterAnyCasePrefix) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fp.fieldName
|
||||
prefixLowercase := fp.getPrefixLowercase()
|
||||
|
||||
// Verify whether fp matches const column
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchAnyCasePrefix(v, prefixLowercase) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether fp matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fp.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByAnyCasePrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByAnyCasePrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByPrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByPrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByPrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByPrefix(bs, ch, bm, prefixLowercase)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByPrefix(bs, ch, bm, prefixLowercase, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByPrefix(bs, ch, bm, prefixLowercase, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
prefixUppercase := strings.ToUpper(fp.prefix)
|
||||
matchTimestampISO8601ByPrefix(bs, ch, bm, prefixUppercase, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchValuesDictByAnyCasePrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefixLowercase string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchAnyCasePrefix(v, prefixLowercase) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByAnyCasePrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefixLowercase string) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchAnyCasePrefix(v, prefixLowercase)
|
||||
})
|
||||
}
|
||||
|
||||
func matchAnyCasePrefix(s, prefixLowercase string) bool {
|
||||
if len(prefixLowercase) == 0 {
|
||||
// Special case - empty prefix matches any non-empty string.
|
||||
return len(s) > 0
|
||||
}
|
||||
if len(prefixLowercase) > len(s) {
|
||||
return false
|
||||
}
|
||||
|
||||
if isASCIILowercase(s) {
|
||||
// Fast path - s is in lowercase
|
||||
return matchPrefix(s, prefixLowercase)
|
||||
}
|
||||
|
||||
// Slow path - convert s to lowercase before matching
|
||||
bb := bbPool.Get()
|
||||
bb.B = stringsutil.AppendLowercase(bb.B, s)
|
||||
sLowercase := bytesutil.ToUnsafeString(bb.B)
|
||||
ok := matchPrefix(sLowercase, prefixLowercase)
|
||||
bbPool.Put(bb)
|
||||
|
||||
return ok
|
||||
}
|
930
lib/logstorage/filter_any_case_prefix_test.go
Normal file
930
lib/logstorage/filter_any_case_prefix_test.go
Normal file
|
@ -0,0 +1,930 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchAnyCasePrefix(t *testing.T) {
|
||||
f := func(s, prefixLowercase string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchAnyCasePrefix(s, prefixLowercase)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
// empty prefix matches non-empty strings
|
||||
f("", "", false)
|
||||
f("foo", "", true)
|
||||
f("тест", "", true)
|
||||
|
||||
// empty string doesn't match non-empty prefix
|
||||
f("", "foo", false)
|
||||
f("", "тест", false)
|
||||
|
||||
// full match
|
||||
f("foo", "foo", true)
|
||||
f("FOo", "foo", true)
|
||||
f("Test ТЕСт 123", "test тест 123", true)
|
||||
|
||||
// prefix match
|
||||
f("foo", "f", true)
|
||||
f("foo тест bar", "те", true)
|
||||
f("foo ТЕСТ bar", "те", true)
|
||||
|
||||
// mismatch
|
||||
f("foo", "o", false)
|
||||
f("тест", "foo", false)
|
||||
f("Тест", "ест", false)
|
||||
}
|
||||
|
||||
func TestFilterAnyCasePrefix(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"aBc DEf",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other column",
|
||||
values: []string{
|
||||
"aSDfdsf",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ABC",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "other column",
|
||||
prefix: "asdfdSF",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "other column",
|
||||
prefix: "sd",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "other-column",
|
||||
values: []string{
|
||||
"x",
|
||||
"X",
|
||||
"X",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"ABC DEF",
|
||||
"AbC Def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "Abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "AB",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc de",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: " de",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "other-column",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: " 2 ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "other-column",
|
||||
prefix: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"fOObar",
|
||||
"Abc",
|
||||
"aFDf FooBar baz",
|
||||
"fddf FOObarBAZ",
|
||||
"AFoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "FooBar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 3, 4, 6})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 2, 3, 4, 5, 6})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ba",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a fOoBar",
|
||||
"aA aBC A",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a fooBaR",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "нГк",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{8})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "!,",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{9})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa ax",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "qwe rty abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "@",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 5})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3, 4})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65535",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123456",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"0.0002",
|
||||
"-320001",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234.5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "56789",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "-6553",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7344.8943",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "-1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "+1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "23",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "678",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "2.3.",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 2, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "8",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "27.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02t15:04:05.005z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{4})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "002",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{1})
|
||||
|
||||
// mimatch
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-03-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "06",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it has different string representation of the timestamp
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T16:04:05.005+01:00",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it contains too many digits for millisecond part
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T15:04:05.00500Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterAnyCasePrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
})
|
||||
}
|
199
lib/logstorage/filter_exact.go
Normal file
199
lib/logstorage/filter_exact.go
Normal file
|
@ -0,0 +1,199 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterExact matches the exact value.
|
||||
//
|
||||
// Example LogsQL: `fieldName:exact("foo bar")`
|
||||
type filterExact struct {
|
||||
fieldName string
|
||||
value string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fe *filterExact) String() string {
|
||||
return fmt.Sprintf("%sexact(%s)", quoteFieldNameIfNeeded(fe.fieldName), quoteTokenIfNeeded(fe.value))
|
||||
}
|
||||
|
||||
func (fe *filterExact) getTokens() []string {
|
||||
fe.tokensOnce.Do(fe.initTokens)
|
||||
return fe.tokens
|
||||
}
|
||||
|
||||
func (fe *filterExact) initTokens() {
|
||||
fe.tokens = tokenizeStrings(nil, []string{fe.value})
|
||||
}
|
||||
|
||||
func (fe *filterExact) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fe.fieldName
|
||||
value := fe.value
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if value != v {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
// It matches anything only for empty value.
|
||||
if value != "" {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fe.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByExactValue(bs, ch, bm, value)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByExactValue(bs, ch, bm, value, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByExactValue(bs, ch, bm, value, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, value string, tokens []string) {
|
||||
n, ok := tryParseTimestampISO8601(value)
|
||||
if !ok || n < int64(ch.minValue) || n > int64(ch.maxValue) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint64(bb.B, uint64(n))
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, value string, tokens []string) {
|
||||
n, ok := tryParseIPv4(value)
|
||||
if !ok || uint64(n) < ch.minValue || uint64(n) > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint32(bb.B, n)
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, value string, tokens []string) {
|
||||
f, ok := tryParseFloat64(value)
|
||||
if !ok || f < math.Float64frombits(ch.minValue) || f > math.Float64frombits(ch.maxValue) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
n := math.Float64bits(f)
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint64(bb.B, n)
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, value string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if v == value {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, value string, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return v == value
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
n, ok := tryParseUint64(phrase)
|
||||
if !ok || n < ch.minValue || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = append(bb.B, byte(n))
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
n, ok := tryParseUint64(phrase)
|
||||
if !ok || n < ch.minValue || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint16(bb.B, uint16(n))
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
n, ok := tryParseUint64(phrase)
|
||||
if !ok || n < ch.minValue || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint32(bb.B, uint32(n))
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByExactValue(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
n, ok := tryParseUint64(phrase)
|
||||
if !ok || n < ch.minValue || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
bb.B = encoding.MarshalUint64(bb.B, n)
|
||||
matchBinaryValue(bs, ch, bm, bb.B, tokens)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchBinaryValue(bs *blockSearch, ch *columnHeader, bm *bitmap, binValue []byte, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return v == string(binValue)
|
||||
})
|
||||
}
|
229
lib/logstorage/filter_exact_prefix.go
Normal file
229
lib/logstorage/filter_exact_prefix.go
Normal file
|
@ -0,0 +1,229 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterExactPrefix matches the exact prefix.
|
||||
//
|
||||
// Example LogsQL: `fieldName:exact("foo bar"*)
|
||||
type filterExactPrefix struct {
|
||||
fieldName string
|
||||
prefix string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fep *filterExactPrefix) String() string {
|
||||
return fmt.Sprintf("%sexact(%s*)", quoteFieldNameIfNeeded(fep.fieldName), quoteTokenIfNeeded(fep.prefix))
|
||||
}
|
||||
|
||||
func (fep *filterExactPrefix) getTokens() []string {
|
||||
fep.tokensOnce.Do(fep.initTokens)
|
||||
return fep.tokens
|
||||
}
|
||||
|
||||
func (fep *filterExactPrefix) initTokens() {
|
||||
fep.tokens = getTokensSkipLast(fep.prefix)
|
||||
}
|
||||
|
||||
func (fep *filterExactPrefix) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fep.fieldName
|
||||
prefix := fep.prefix
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchExactPrefix(v, prefix) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
if !matchExactPrefix("", prefix) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fep.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByExactPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByExactPrefix(bs, ch, bm, prefix, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
return
|
||||
}
|
||||
if prefix < "0" || prefix > "9" || !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
return
|
||||
}
|
||||
if prefix < "0" || prefix > "9" || len(tokens) > 3 || !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
// An empty prefix matches all the values
|
||||
return
|
||||
}
|
||||
if len(tokens) > 2 || !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchExactPrefix(v, prefix) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchExactPrefix(v, prefix)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchMinMaxExactPrefix(ch, bm, prefix, tokens) {
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint8String(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchMinMaxExactPrefix(ch, bm, prefix, tokens) {
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint16String(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchMinMaxExactPrefix(ch, bm, prefix, tokens) {
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint32String(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByExactPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchMinMaxExactPrefix(ch, bm, prefix, tokens) {
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint64String(bs, bb, v)
|
||||
return matchExactPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchMinMaxExactPrefix(ch *columnHeader, bm *bitmap, prefix string, tokens []string) bool {
|
||||
if prefix == "" {
|
||||
// An empty prefix matches all the values
|
||||
return false
|
||||
}
|
||||
if len(tokens) > 0 {
|
||||
// Non-empty tokens means that the prefix contains at least two tokens.
|
||||
// Multiple tokens cannot match any uint value.
|
||||
bm.resetBits()
|
||||
return false
|
||||
}
|
||||
n, ok := tryParseUint64(prefix)
|
||||
if !ok || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func matchExactPrefix(s, prefix string) bool {
|
||||
return strings.HasPrefix(s, prefix)
|
||||
}
|
564
lib/logstorage/filter_exact_prefix_test.go
Normal file
564
lib/logstorage/filter_exact_prefix_test.go
Normal file
|
@ -0,0 +1,564 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterExactPrefix(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc d",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "xabc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"foobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{1, 5, 6})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "baz",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"aa fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar baz",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{2, 4})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "fobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "aa",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 5})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "999",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"467",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 5})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "999",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"65536",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 5})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "99999",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"123456789012",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 5})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234567890123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 4})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234.567",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{4})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "-65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{3})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "6511",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.2",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.0.",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:06.004Z",
|
||||
"2006-01-02T15:04:06.005Z",
|
||||
"2006-01-02T15:04:07.006Z",
|
||||
"2006-01-02T15:04:10.007Z",
|
||||
"2006-01-02T15:04:12.008Z",
|
||||
"2006-01-02T15:04:15.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fep := &filterExactPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T15:04:05",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "_msg", []int{0, 1, 2})
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "_msg", nil)
|
||||
|
||||
fep = &filterExactPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fep, "_msg", nil)
|
||||
})
|
||||
}
|
582
lib/logstorage/filter_exact_test.go
Normal file
582
lib/logstorage/filter_exact_test.go
Normal file
|
@ -0,0 +1,582 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterExact(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing column",
|
||||
value: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing column",
|
||||
value: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"afoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{1, 6})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "baz",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing column",
|
||||
value: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar baz",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "aa abc a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{2})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "aa a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "fooaaazz a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{1, 5})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"256",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{1, 5})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"65536",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{1, 5})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"12345678901",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{1, 5})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "1234.5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{4})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "-65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{3})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "127.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "foo",
|
||||
value: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fe := &filterExact{
|
||||
fieldName: "_msg",
|
||||
value: "2006-01-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "_msg", []int{4})
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "non-existing-column",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
fe = &filterExact{
|
||||
fieldName: "_msg",
|
||||
value: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "_msg", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "_msg",
|
||||
value: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "_msg", nil)
|
||||
|
||||
fe = &filterExact{
|
||||
fieldName: "_msg",
|
||||
value: "2006-03-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fe, "_msg", nil)
|
||||
})
|
||||
}
|
354
lib/logstorage/filter_in.go
Normal file
354
lib/logstorage/filter_in.go
Normal file
|
@ -0,0 +1,354 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterIn matches any exact value from the values map.
|
||||
//
|
||||
// Example LogsQL: `fieldName:in("foo", "bar baz")`
|
||||
type filterIn struct {
|
||||
fieldName string
|
||||
values []string
|
||||
|
||||
tokenSetsOnce sync.Once
|
||||
tokenSets [][]string
|
||||
|
||||
stringValuesOnce sync.Once
|
||||
stringValues map[string]struct{}
|
||||
|
||||
uint8ValuesOnce sync.Once
|
||||
uint8Values map[string]struct{}
|
||||
|
||||
uint16ValuesOnce sync.Once
|
||||
uint16Values map[string]struct{}
|
||||
|
||||
uint32ValuesOnce sync.Once
|
||||
uint32Values map[string]struct{}
|
||||
|
||||
uint64ValuesOnce sync.Once
|
||||
uint64Values map[string]struct{}
|
||||
|
||||
float64ValuesOnce sync.Once
|
||||
float64Values map[string]struct{}
|
||||
|
||||
ipv4ValuesOnce sync.Once
|
||||
ipv4Values map[string]struct{}
|
||||
|
||||
timestampISO8601ValuesOnce sync.Once
|
||||
timestampISO8601Values map[string]struct{}
|
||||
}
|
||||
|
||||
func (fi *filterIn) String() string {
|
||||
values := fi.values
|
||||
a := make([]string, len(values))
|
||||
for i, value := range values {
|
||||
a[i] = quoteTokenIfNeeded(value)
|
||||
}
|
||||
return fmt.Sprintf("%sin(%s)", quoteFieldNameIfNeeded(fi.fieldName), strings.Join(a, ","))
|
||||
}
|
||||
|
||||
func (fi *filterIn) getTokenSets() [][]string {
|
||||
fi.tokenSetsOnce.Do(fi.initTokenSets)
|
||||
return fi.tokenSets
|
||||
}
|
||||
|
||||
// It is faster to match every row in the block instead of checking too big number of tokenSets against bloom filter.
|
||||
const maxTokenSetsToInit = 1000
|
||||
|
||||
func (fi *filterIn) initTokenSets() {
|
||||
values := fi.values
|
||||
tokenSetsLen := len(values)
|
||||
if tokenSetsLen > maxTokenSetsToInit {
|
||||
tokenSetsLen = maxTokenSetsToInit
|
||||
}
|
||||
tokenSets := make([][]string, 0, tokenSetsLen+1)
|
||||
for _, v := range values {
|
||||
tokens := tokenizeStrings(nil, []string{v})
|
||||
tokenSets = append(tokenSets, tokens)
|
||||
if len(tokens) > maxTokenSetsToInit {
|
||||
break
|
||||
}
|
||||
}
|
||||
fi.tokenSets = tokenSets
|
||||
}
|
||||
|
||||
func (fi *filterIn) getStringValues() map[string]struct{} {
|
||||
fi.stringValuesOnce.Do(fi.initStringValues)
|
||||
return fi.stringValues
|
||||
}
|
||||
|
||||
func (fi *filterIn) initStringValues() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
for _, v := range values {
|
||||
m[v] = struct{}{}
|
||||
}
|
||||
fi.stringValues = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getUint8Values() map[string]struct{} {
|
||||
fi.uint8ValuesOnce.Do(fi.initUint8Values)
|
||||
return fi.uint8Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initUint8Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*1)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseUint64(v)
|
||||
if !ok || n >= (1<<8) {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = append(buf, byte(n))
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.uint8Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getUint16Values() map[string]struct{} {
|
||||
fi.uint16ValuesOnce.Do(fi.initUint16Values)
|
||||
return fi.uint16Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initUint16Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*2)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseUint64(v)
|
||||
if !ok || n >= (1<<16) {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint16(buf, uint16(n))
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.uint16Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getUint32Values() map[string]struct{} {
|
||||
fi.uint32ValuesOnce.Do(fi.initUint32Values)
|
||||
return fi.uint32Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initUint32Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*4)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseUint64(v)
|
||||
if !ok || n >= (1<<32) {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint32(buf, uint32(n))
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.uint32Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getUint64Values() map[string]struct{} {
|
||||
fi.uint64ValuesOnce.Do(fi.initUint64Values)
|
||||
return fi.uint64Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initUint64Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*8)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseUint64(v)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint64(buf, n)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.uint64Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getFloat64Values() map[string]struct{} {
|
||||
fi.float64ValuesOnce.Do(fi.initFloat64Values)
|
||||
return fi.float64Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initFloat64Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*8)
|
||||
for _, v := range values {
|
||||
f, ok := tryParseFloat64(v)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
n := math.Float64bits(f)
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint64(buf, n)
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.float64Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getIPv4Values() map[string]struct{} {
|
||||
fi.ipv4ValuesOnce.Do(fi.initIPv4Values)
|
||||
return fi.ipv4Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initIPv4Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*4)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseIPv4(v)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint32(buf, uint32(n))
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.ipv4Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) getTimestampISO8601Values() map[string]struct{} {
|
||||
fi.timestampISO8601ValuesOnce.Do(fi.initTimestampISO8601Values)
|
||||
return fi.timestampISO8601Values
|
||||
}
|
||||
|
||||
func (fi *filterIn) initTimestampISO8601Values() {
|
||||
values := fi.values
|
||||
m := make(map[string]struct{}, len(values))
|
||||
buf := make([]byte, 0, len(values)*8)
|
||||
for _, v := range values {
|
||||
n, ok := tryParseTimestampISO8601(v)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
bufLen := len(buf)
|
||||
buf = encoding.MarshalUint64(buf, uint64(n))
|
||||
s := bytesutil.ToUnsafeString(buf[bufLen:])
|
||||
m[s] = struct{}{}
|
||||
}
|
||||
fi.timestampISO8601Values = m
|
||||
}
|
||||
|
||||
func (fi *filterIn) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fi.fieldName
|
||||
|
||||
if len(fi.values) == 0 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
stringValues := fi.getStringValues()
|
||||
if _, ok := stringValues[v]; !ok {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
// It matches anything only for empty phrase.
|
||||
stringValues := fi.getStringValues()
|
||||
if _, ok := stringValues[""]; !ok {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokenSets := fi.getTokenSets()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
stringValues := fi.getStringValues()
|
||||
matchAnyValue(bs, ch, bm, stringValues, tokenSets)
|
||||
case valueTypeDict:
|
||||
stringValues := fi.getStringValues()
|
||||
matchValuesDictByAnyValue(bs, ch, bm, stringValues)
|
||||
case valueTypeUint8:
|
||||
binValues := fi.getUint8Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeUint16:
|
||||
binValues := fi.getUint16Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeUint32:
|
||||
binValues := fi.getUint32Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeUint64:
|
||||
binValues := fi.getUint64Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeFloat64:
|
||||
binValues := fi.getFloat64Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeIPv4:
|
||||
binValues := fi.getIPv4Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
case valueTypeTimestampISO8601:
|
||||
binValues := fi.getTimestampISO8601Values()
|
||||
matchAnyValue(bs, ch, bm, binValues, tokenSets)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchAnyValue(bs *blockSearch, ch *columnHeader, bm *bitmap, values map[string]struct{}, tokenSets [][]string) {
|
||||
if !matchBloomFilterAnyTokenSet(bs, ch, tokenSets) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
_, ok := values[v]
|
||||
return ok
|
||||
})
|
||||
}
|
||||
|
||||
func matchBloomFilterAnyTokenSet(bs *blockSearch, ch *columnHeader, tokenSets [][]string) bool {
|
||||
if len(tokenSets) == 0 {
|
||||
return false
|
||||
}
|
||||
if len(tokenSets) > maxTokenSetsToInit || uint64(len(tokenSets)) > 10*bs.bsw.bh.rowsCount {
|
||||
// It is faster to match every row in the block against all the values
|
||||
// instead of using bloom filter for too big number of tokenSets.
|
||||
return true
|
||||
}
|
||||
bf := bs.getBloomFilterForColumn(ch)
|
||||
for _, tokens := range tokenSets {
|
||||
if bf.containsAll(tokens) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func matchValuesDictByAnyValue(bs *blockSearch, ch *columnHeader, bm *bitmap, values map[string]struct{}) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if _, ok := values[v]; ok {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
690
lib/logstorage/filter_in_test.go
Normal file
690
lib/logstorage/filter_in_test.go
Normal file
|
@ -0,0 +1,690 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterIn(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other column",
|
||||
values: []string{
|
||||
"asdfdsf",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"abc def", "abc", "foobar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "other column",
|
||||
values: []string{"asdfdsf", ""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{"", "foo"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"abc", "def"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"", "abc"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "other column",
|
||||
values: []string{"sd"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing column",
|
||||
values: []string{"abc", "def"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"aaaa", "abc def", "foobar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{"", "abc"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"abc def ", "foobar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing column",
|
||||
values: []string{"x"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"afoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"foobar", "aaaa", "abc", "baz"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 6})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bbbb", "", "aaaa"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar", "aaaa"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing column",
|
||||
values: []string{"foobar", "aaaa"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"a foobar", "aa abc a"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 6})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"aa a"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12", "32"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 5})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{3, 4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"33"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"256",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12", "32"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 5})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{3, 4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"33"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"123456"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"65536",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12", "32"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 5})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{3, 4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"33"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12345678901"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"12345678901",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12", "32"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{1, 2, 5})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{3, 4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"33"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"1234", "1", "foobar", "123211"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 5})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"1234.5678901"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"-65536"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{3})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"65536"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"123"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"12345678901234567890"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"127.0.0.1", "24.54.1.2", "127.0.4.2"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{2, 4, 5, 6, 7})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"5"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "foo",
|
||||
values: []string{"255.255.255.255"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fi := &filterIn{
|
||||
fieldName: "_msg",
|
||||
values: []string{"2006-01-02T15:04:05.005Z", "foobar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", []int{4})
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "non-existing-column",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
fi = &filterIn{
|
||||
fieldName: "_msg",
|
||||
values: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "_msg",
|
||||
values: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "_msg",
|
||||
values: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", nil)
|
||||
|
||||
fi = &filterIn{
|
||||
fieldName: "_msg",
|
||||
values: []string{"2006-04-02T15:04:05.005Z"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fi, "_msg", nil)
|
||||
})
|
||||
}
|
115
lib/logstorage/filter_ipv4_range.go
Normal file
115
lib/logstorage/filter_ipv4_range.go
Normal file
|
@ -0,0 +1,115 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterIPv4Range matches the given ipv4 range [minValue..maxValue].
|
||||
//
|
||||
// Example LogsQL: `fieldName:ipv4_range(127.0.0.1, 127.0.0.255)`
|
||||
type filterIPv4Range struct {
|
||||
fieldName string
|
||||
minValue uint32
|
||||
maxValue uint32
|
||||
}
|
||||
|
||||
func (fr *filterIPv4Range) String() string {
|
||||
minValue := string(encoding.MarshalUint32(nil, fr.minValue))
|
||||
maxValue := string(encoding.MarshalUint32(nil, fr.maxValue))
|
||||
return fmt.Sprintf("%sipv4_range(%s, %s)", quoteFieldNameIfNeeded(fr.fieldName), toIPv4String(nil, minValue), toIPv4String(nil, maxValue))
|
||||
}
|
||||
|
||||
func (fr *filterIPv4Range) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fr.fieldName
|
||||
minValue := fr.minValue
|
||||
maxValue := fr.maxValue
|
||||
|
||||
if minValue > maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchIPv4Range(v, minValue, maxValue) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByIPv4Range(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByIPv4Range(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint8:
|
||||
bm.resetBits()
|
||||
case valueTypeUint16:
|
||||
bm.resetBits()
|
||||
case valueTypeUint32:
|
||||
bm.resetBits()
|
||||
case valueTypeUint64:
|
||||
bm.resetBits()
|
||||
case valueTypeFloat64:
|
||||
bm.resetBits()
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeTimestampISO8601:
|
||||
bm.resetBits()
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchValuesDictByIPv4Range(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue uint32) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchIPv4Range(v, minValue, maxValue) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByIPv4Range(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue uint32) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchIPv4Range(v, minValue, maxValue)
|
||||
})
|
||||
}
|
||||
|
||||
func matchIPv4Range(s string, minValue, maxValue uint32) bool {
|
||||
n, ok := tryParseIPv4(s)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return n >= minValue && n <= maxValue
|
||||
}
|
||||
|
||||
func matchIPv4ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue uint32) {
|
||||
if ch.minValue > uint64(maxValue) || ch.maxValue < uint64(minValue) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of IPv4: got %d; want 4", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := encoding.UnmarshalUint32(b)
|
||||
return n >= minValue && n <= maxValue
|
||||
})
|
||||
}
|
402
lib/logstorage/filter_ipv4_range_test.go
Normal file
402
lib/logstorage/filter_ipv4_range_test.go
Normal file
|
@ -0,0 +1,402 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchIPv4Range(t *testing.T) {
|
||||
f := func(s string, minValue, maxValue uint32, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchIPv4Range(s, minValue, maxValue)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
// Invalid IP
|
||||
f("", 0, 1000, false)
|
||||
f("123", 0, 1000, false)
|
||||
|
||||
// range mismatch
|
||||
f("0.0.0.1", 2, 100, false)
|
||||
f("127.0.0.1", 0x6f000000, 0x7f000000, false)
|
||||
|
||||
// range match
|
||||
f("0.0.0.1", 1, 1, true)
|
||||
f("0.0.0.1", 0, 100, true)
|
||||
f("127.0.0.1", 0x7f000000, 0x7f000001, true)
|
||||
}
|
||||
|
||||
func TestFilterIPv4Range(t *testing.T) {
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0x80000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x7f000001,
|
||||
maxValue: 0x7f000001,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0x7f000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "non-existing-column",
|
||||
minValue: 0,
|
||||
maxValue: 20000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x80000000,
|
||||
maxValue: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"127.0.0.1",
|
||||
"Abc",
|
||||
"127.255.255.255",
|
||||
"10.4",
|
||||
"foo 127.0.0.1",
|
||||
"127.0.0.1 bar",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x7f000000,
|
||||
maxValue: 0x80000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 3, 7})
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0x7f000001,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 1000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x7f000002,
|
||||
maxValue: 0x7f7f0000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x80000000,
|
||||
maxValue: 0x7f000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a 10",
|
||||
"127.0.0.1",
|
||||
"20",
|
||||
"15.5",
|
||||
"-5",
|
||||
"a fooBaR",
|
||||
"a 127.0.0.1 dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x7f000000,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 10000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0xffffffff,
|
||||
maxValue: 0x7f000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65535",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65536",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"123456.78901",
|
||||
"-0.2",
|
||||
"2",
|
||||
"-334",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 0x08000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 11})
|
||||
|
||||
// mismatch
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x80000000,
|
||||
maxValue: 0x90000000,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0xff000000,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterIPv4Range{
|
||||
fieldName: "foo",
|
||||
minValue: 0x08000000,
|
||||
maxValue: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// mismatch
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: "_msg",
|
||||
minValue: 0,
|
||||
maxValue: 0xffffffff,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
})
|
||||
}
|
202
lib/logstorage/filter_len_range.go
Normal file
202
lib/logstorage/filter_len_range.go
Normal file
|
@ -0,0 +1,202 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterLenRange matches field values with the length in the given range [minLen, maxLen].
|
||||
//
|
||||
// Example LogsQL: `fieldName:len_range(10, 20)`
|
||||
type filterLenRange struct {
|
||||
fieldName string
|
||||
minLen uint64
|
||||
maxLen uint64
|
||||
|
||||
stringRepr string
|
||||
}
|
||||
|
||||
func (fr *filterLenRange) String() string {
|
||||
return quoteFieldNameIfNeeded(fr.fieldName) + "len_range" + fr.stringRepr
|
||||
}
|
||||
|
||||
func (fr *filterLenRange) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fr.fieldName
|
||||
minLen := fr.minLen
|
||||
maxLen := fr.maxLen
|
||||
|
||||
if minLen > maxLen {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchLenRange(v, minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
if !matchLenRange("", minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByLenRange(bs, ch, bm, minLen, maxLen)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByLenRange(bm, minLen, maxLen)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByLenRange(bm *bitmap, minLen, maxLen uint64) {
|
||||
if minLen > uint64(len(iso8601Timestamp)) || maxLen < uint64(len(iso8601Timestamp)) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func matchIPv4ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if minLen > uint64(len("255.255.255.255")) || maxLen < uint64(len("0.0.0.0")) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if minLen > 24 || maxLen == 0 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchLenRange(v, minLen, maxLen) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchLenRange(v, minLen, maxLen)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if !matchMinMaxValueLen(ch, minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint8String(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if !matchMinMaxValueLen(ch, minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint16String(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if !matchMinMaxValueLen(ch, minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint32String(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByLenRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minLen, maxLen uint64) {
|
||||
if !matchMinMaxValueLen(ch, minLen, maxLen) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint64String(bs, bb, v)
|
||||
return matchLenRange(s, minLen, maxLen)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchLenRange(s string, minLen, maxLen uint64) bool {
|
||||
sLen := uint64(utf8.RuneCountInString(s))
|
||||
return sLen >= minLen && sLen <= maxLen
|
||||
}
|
||||
|
||||
func matchMinMaxValueLen(ch *columnHeader, minLen, maxLen uint64) bool {
|
||||
bb := bbPool.Get()
|
||||
defer bbPool.Put(bb)
|
||||
|
||||
bb.B = marshalUint64(bb.B[:0], ch.minValue)
|
||||
s := bytesutil.ToUnsafeString(bb.B)
|
||||
if maxLen < uint64(len(s)) {
|
||||
return false
|
||||
}
|
||||
bb.B = marshalUint64(bb.B[:0], ch.maxValue)
|
||||
s = bytesutil.ToUnsafeString(bb.B)
|
||||
return minLen <= uint64(len(s))
|
||||
}
|
438
lib/logstorage/filter_len_range_test.go
Normal file
438
lib/logstorage/filter_len_range_test.go
Normal file
|
@ -0,0 +1,438 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchLenRange(t *testing.T) {
|
||||
f := func(s string, minLen, maxLen uint64, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchLenRange(s, minLen, maxLen)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f("", 0, 0, true)
|
||||
f("", 0, 1, true)
|
||||
f("", 1, 1, false)
|
||||
|
||||
f("abc", 0, 2, false)
|
||||
f("abc", 0, 3, true)
|
||||
f("abc", 0, 4, true)
|
||||
f("abc", 3, 4, true)
|
||||
f("abc", 4, 4, false)
|
||||
f("abc", 4, 2, false)
|
||||
|
||||
f("ФЫВА", 3, 3, false)
|
||||
f("ФЫВА", 4, 4, true)
|
||||
f("ФЫВА", 5, 5, false)
|
||||
f("ФЫВА", 0, 10, true)
|
||||
}
|
||||
|
||||
func TestFilterLenRange(t *testing.T) {
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"10",
|
||||
"10",
|
||||
"10",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "non-existing-column",
|
||||
minLen: 0,
|
||||
maxLen: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 3,
|
||||
maxLen: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "non-existing-column",
|
||||
minLen: 10,
|
||||
maxLen: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"10",
|
||||
"Abc",
|
||||
"20",
|
||||
"10.5",
|
||||
"10 AFoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 2, 3})
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 0,
|
||||
maxLen: 1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 20,
|
||||
maxLen: 30,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a 10",
|
||||
"10",
|
||||
"20",
|
||||
"15.5",
|
||||
"-5",
|
||||
"a fooBaR",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 5})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 100,
|
||||
maxLen: 200,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 2,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 6})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 0,
|
||||
maxLen: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 10,
|
||||
maxLen: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"256",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 2,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 6})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 0,
|
||||
maxLen: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 10,
|
||||
maxLen: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"65536",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 2,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 6})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 0,
|
||||
maxLen: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 10,
|
||||
maxLen: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123456789012",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 2,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 6})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 0,
|
||||
maxLen: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 20,
|
||||
maxLen: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"123456.78901",
|
||||
"-0.2",
|
||||
"2",
|
||||
"-334",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 2,
|
||||
maxLen: 2,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 100,
|
||||
maxLen: 200,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 3,
|
||||
maxLen: 7,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 11})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "foo",
|
||||
minLen: 20,
|
||||
maxLen: 30,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterLenRange{
|
||||
fieldName: "_msg",
|
||||
minLen: 10,
|
||||
maxLen: 30,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterLenRange{
|
||||
fieldName: "_msg",
|
||||
minLen: 10,
|
||||
maxLen: 11,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
})
|
||||
}
|
13
lib/logstorage/filter_noop.go
Normal file
13
lib/logstorage/filter_noop.go
Normal file
|
@ -0,0 +1,13 @@
|
|||
package logstorage
|
||||
|
||||
// filterNoop does nothing
|
||||
type filterNoop struct {
|
||||
}
|
||||
|
||||
func (fn *filterNoop) String() string {
|
||||
return ""
|
||||
}
|
||||
|
||||
func (fn *filterNoop) apply(_ *blockSearch, _ *bitmap) {
|
||||
// nothing to do
|
||||
}
|
27
lib/logstorage/filter_not.go
Normal file
27
lib/logstorage/filter_not.go
Normal file
|
@ -0,0 +1,27 @@
|
|||
package logstorage
|
||||
|
||||
// filterNot negates the filter.
|
||||
//
|
||||
// It is expressed as `NOT f` or `!f` in LogsQL.
|
||||
type filterNot struct {
|
||||
f filter
|
||||
}
|
||||
|
||||
func (fn *filterNot) String() string {
|
||||
s := fn.f.String()
|
||||
switch fn.f.(type) {
|
||||
case *filterAnd, *filterOr:
|
||||
s = "(" + s + ")"
|
||||
}
|
||||
return "!" + s
|
||||
}
|
||||
|
||||
func (fn *filterNot) apply(bs *blockSearch, bm *bitmap) {
|
||||
// Minimize the number of rows to check by the filter by applying it
|
||||
// only to the rows, which match the bm, e.g. they may change the bm result.
|
||||
bmTmp := getBitmap(bm.bitsLen)
|
||||
bmTmp.copyFrom(bm)
|
||||
fn.f.apply(bs, bmTmp)
|
||||
bm.andNot(bmTmp)
|
||||
putBitmap(bmTmp)
|
||||
}
|
75
lib/logstorage/filter_not_test.go
Normal file
75
lib/logstorage/filter_not_test.go
Normal file
|
@ -0,0 +1,75 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterNot(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"",
|
||||
"a foobar",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fn := &filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", []int{0, 1, 2, 3, 4, 6, 7, 8, 9})
|
||||
|
||||
fn = &filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", []int{5})
|
||||
|
||||
fn = &filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "non-existing-field",
|
||||
phrase: "foobar",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fn = &filterNot{
|
||||
f: &filterPrefix{
|
||||
fieldName: "non-existing-field",
|
||||
prefix: "",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fn = &filterNot{
|
||||
f: &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", []int{5})
|
||||
|
||||
// mismatch
|
||||
fn = &filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "non-existing-field",
|
||||
phrase: "",
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fn, "foo", nil)
|
||||
}
|
45
lib/logstorage/filter_or.go
Normal file
45
lib/logstorage/filter_or.go
Normal file
|
@ -0,0 +1,45 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"strings"
|
||||
)
|
||||
|
||||
// filterOr contains filters joined by OR operator.
|
||||
//
|
||||
// It is epxressed as `f1 OR f2 ... OR fN` in LogsQL.
|
||||
type filterOr struct {
|
||||
filters []filter
|
||||
}
|
||||
|
||||
func (fo *filterOr) String() string {
|
||||
filters := fo.filters
|
||||
a := make([]string, len(filters))
|
||||
for i, f := range filters {
|
||||
s := f.String()
|
||||
a[i] = s
|
||||
}
|
||||
return strings.Join(a, " or ")
|
||||
}
|
||||
|
||||
func (fo *filterOr) apply(bs *blockSearch, bm *bitmap) {
|
||||
bmResult := getBitmap(bm.bitsLen)
|
||||
bmTmp := getBitmap(bm.bitsLen)
|
||||
for _, f := range fo.filters {
|
||||
// Minimize the number of rows to check by the filter by checking only
|
||||
// the rows, which may change the output bm:
|
||||
// - bm matches them, e.g. the caller wants to get them
|
||||
// - bmResult doesn't match them, e.g. all the previous OR filters didn't match them
|
||||
bmTmp.copyFrom(bm)
|
||||
bmTmp.andNot(bmResult)
|
||||
if bmTmp.isZero() {
|
||||
// Shortcut - there is no need in applying the remaining filters,
|
||||
// since the result already matches all the values from the block.
|
||||
break
|
||||
}
|
||||
f.apply(bs, bmTmp)
|
||||
bmResult.or(bmTmp)
|
||||
}
|
||||
putBitmap(bmTmp)
|
||||
bm.copyFrom(bmResult)
|
||||
putBitmap(bmResult)
|
||||
}
|
130
lib/logstorage/filter_or_test.go
Normal file
130
lib/logstorage/filter_or_test.go
Normal file
|
@ -0,0 +1,130 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterOr(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a",
|
||||
"a foobar abcdef",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// non-empty union
|
||||
fo := &filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "23",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{2, 6, 9})
|
||||
|
||||
// reverse non-empty union
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "23",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{2, 6, 9})
|
||||
|
||||
// first empty result, second non-empty result
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "xabc",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "23",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{9})
|
||||
|
||||
// first non-empty result, second empty result
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "23",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "xabc",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{9})
|
||||
|
||||
// first match all
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "23",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// second match all
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "23",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// both empty results
|
||||
fo = &filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "x23",
|
||||
},
|
||||
&filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "xabc",
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fo, "foo", nil)
|
||||
}
|
319
lib/logstorage/filter_phrase.go
Normal file
319
lib/logstorage/filter_phrase.go
Normal file
|
@ -0,0 +1,319 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"strings"
|
||||
"sync"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterPhrase filters field entries by phrase match (aka full text search).
|
||||
//
|
||||
// A phrase consists of any number of words with delimiters between them.
|
||||
//
|
||||
// An empty phrase matches only an empty string.
|
||||
// A single-word phrase is the simplest LogsQL query: `fieldName:word`
|
||||
//
|
||||
// Multi-word phrase is expressed as `fieldName:"word1 ... wordN"` in LogsQL.
|
||||
//
|
||||
// A special case `fieldName:""` matches any value without `fieldName` field.
|
||||
type filterPhrase struct {
|
||||
fieldName string
|
||||
phrase string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fp *filterPhrase) String() string {
|
||||
return quoteFieldNameIfNeeded(fp.fieldName) + quoteTokenIfNeeded(fp.phrase)
|
||||
}
|
||||
|
||||
func (fp *filterPhrase) getTokens() []string {
|
||||
fp.tokensOnce.Do(fp.initTokens)
|
||||
return fp.tokens
|
||||
}
|
||||
|
||||
func (fp *filterPhrase) initTokens() {
|
||||
fp.tokens = tokenizeStrings(nil, []string{fp.phrase})
|
||||
}
|
||||
|
||||
func (fp *filterPhrase) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fp.fieldName
|
||||
phrase := fp.phrase
|
||||
|
||||
// Verify whether fp matches const column
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchPhrase(v, phrase) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether fp matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
// It matches anything only for empty phrase.
|
||||
if len(phrase) > 0 {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fp.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByPhrase(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByPhrase(bs, ch, bm, phrase)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByPhrase(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByPhrase(bs, ch, bm, phrase, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByPhrase(bs, ch, bm, phrase, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByPhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
_, ok := tryParseTimestampISO8601(phrase)
|
||||
if ok {
|
||||
// Fast path - the phrase contains complete timestamp, so we can use exact search
|
||||
matchTimestampISO8601ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
return
|
||||
}
|
||||
|
||||
// Slow path - the phrase contains incomplete timestamp. Search over string representation of the timestamp.
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return matchPhrase(s, phrase)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByPhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
_, ok := tryParseIPv4(phrase)
|
||||
if ok {
|
||||
// Fast path - phrase contains the full IP address, so we can use exact matching
|
||||
matchIPv4ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
return
|
||||
}
|
||||
|
||||
// Slow path - the phrase may contain a part of IP address. For example, `1.23` should match `1.23.4.5` and `4.1.23.54`.
|
||||
// We cannot compare binary represetnation of ip address and need converting
|
||||
// the ip to string before searching for prefix there.
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchPhrase(s, phrase)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByPhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
// The phrase may contain a part of the floating-point number.
|
||||
// For example, `foo:"123"` must match `123`, `123.456` and `-0.123`.
|
||||
// This means we cannot search in binary representation of floating-point numbers.
|
||||
// Instead, we need searching for the whole phrase in string representation
|
||||
// of floating-point numbers :(
|
||||
_, ok := tryParseFloat64(phrase)
|
||||
if !ok && phrase != "." && phrase != "+" && phrase != "-" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
if n := strings.IndexByte(phrase, '.'); n > 0 && n < len(phrase)-1 {
|
||||
// Fast path - the phrase contains the exact floating-point number, so we can use exact search
|
||||
matchFloat64ByExactValue(bs, ch, bm, phrase, tokens)
|
||||
return
|
||||
}
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchPhrase(s, phrase)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByPhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchPhrase(v, phrase) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByPhrase(bs *blockSearch, ch *columnHeader, bm *bitmap, phrase string, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchPhrase(v, phrase)
|
||||
})
|
||||
}
|
||||
|
||||
func matchPhrase(s, phrase string) bool {
|
||||
if len(phrase) == 0 {
|
||||
// Special case - empty phrase matches only empty string.
|
||||
return len(s) == 0
|
||||
}
|
||||
n := getPhrasePos(s, phrase)
|
||||
return n >= 0
|
||||
}
|
||||
|
||||
func getPhrasePos(s, phrase string) int {
|
||||
if len(phrase) == 0 {
|
||||
return 0
|
||||
}
|
||||
if len(phrase) > len(s) {
|
||||
return -1
|
||||
}
|
||||
|
||||
r := rune(phrase[0])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeRuneInString(phrase)
|
||||
}
|
||||
startsWithToken := isTokenRune(r)
|
||||
|
||||
r = rune(phrase[len(phrase)-1])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeLastRuneInString(phrase)
|
||||
}
|
||||
endsWithToken := isTokenRune(r)
|
||||
|
||||
pos := 0
|
||||
for {
|
||||
n := strings.Index(s[pos:], phrase)
|
||||
if n < 0 {
|
||||
return -1
|
||||
}
|
||||
pos += n
|
||||
// Make sure that the found phrase contains non-token chars at the beginning and at the end
|
||||
if startsWithToken && pos > 0 {
|
||||
r := rune(s[pos-1])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeLastRuneInString(s[:pos])
|
||||
}
|
||||
if r == utf8.RuneError || isTokenRune(r) {
|
||||
pos++
|
||||
continue
|
||||
}
|
||||
}
|
||||
if endsWithToken && pos+len(phrase) < len(s) {
|
||||
r := rune(s[pos+len(phrase)])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeRuneInString(s[pos+len(phrase):])
|
||||
}
|
||||
if r == utf8.RuneError || isTokenRune(r) {
|
||||
pos++
|
||||
continue
|
||||
}
|
||||
}
|
||||
return pos
|
||||
}
|
||||
}
|
||||
|
||||
func matchEncodedValuesDict(bs *blockSearch, ch *columnHeader, bm *bitmap, encodedValues []byte) {
|
||||
if len(encodedValues) == 0 {
|
||||
// Fast path - the phrase is missing in the valuesDict
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// Slow path - iterate over values
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 1 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for dict value: got %d; want 1", bs.partPath(), len(v))
|
||||
}
|
||||
n := bytes.IndexByte(encodedValues, v[0])
|
||||
return n >= 0
|
||||
})
|
||||
}
|
||||
|
||||
func visitValues(bs *blockSearch, ch *columnHeader, bm *bitmap, f func(value string) bool) {
|
||||
if bm.isZero() {
|
||||
// Fast path - nothing to visit
|
||||
return
|
||||
}
|
||||
values := bs.getValuesForColumn(ch)
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
return f(values[idx])
|
||||
})
|
||||
}
|
||||
|
||||
func matchBloomFilterAllTokens(bs *blockSearch, ch *columnHeader, tokens []string) bool {
|
||||
if len(tokens) == 0 {
|
||||
return true
|
||||
}
|
||||
bf := bs.getBloomFilterForColumn(ch)
|
||||
return bf.containsAll(tokens)
|
||||
}
|
||||
|
||||
func quoteFieldNameIfNeeded(s string) string {
|
||||
if isMsgFieldName(s) {
|
||||
return ""
|
||||
}
|
||||
return quoteTokenIfNeeded(s) + ":"
|
||||
}
|
||||
|
||||
func isMsgFieldName(fieldName string) bool {
|
||||
return fieldName == "" || fieldName == "_msg"
|
||||
}
|
||||
|
||||
func toFloat64StringExt(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of floating-point number: got %d; want 8", bs.partPath(), len(v))
|
||||
}
|
||||
bb.B = toFloat64String(bb.B[:0], v)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
||||
|
||||
func toIPv4StringExt(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of IPv4: got %d; want 4", bs.partPath(), len(v))
|
||||
}
|
||||
bb.B = toIPv4String(bb.B[:0], v)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
||||
|
||||
func toTimestampISO8601StringExt(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of ISO8601 timestamp: got %d; want 8", bs.partPath(), len(v))
|
||||
}
|
||||
bb.B = toTimestampISO8601String(bb.B[:0], v)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
893
lib/logstorage/filter_phrase_test.go
Normal file
893
lib/logstorage/filter_phrase_test.go
Normal file
|
@ -0,0 +1,893 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchPhrase(t *testing.T) {
|
||||
f := func(s, phrase string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchPhrase(s, phrase)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f("", "", true)
|
||||
f("foo", "", false)
|
||||
f("", "foo", false)
|
||||
f("foo", "foo", true)
|
||||
f("foo bar", "foo", true)
|
||||
f("foo bar", "bar", true)
|
||||
f("a foo bar", "foo", true)
|
||||
f("a foo bar", "fo", false)
|
||||
f("a foo bar", "oo", false)
|
||||
f("foobar", "foo", false)
|
||||
f("foobar", "bar", false)
|
||||
f("foobar", "oob", false)
|
||||
f("afoobar foo", "foo", true)
|
||||
f("раз два (три!)", "три", true)
|
||||
f("", "foo bar", false)
|
||||
f("foo bar", "foo bar", true)
|
||||
f("(foo bar)", "foo bar", true)
|
||||
f("afoo bar", "foo bar", false)
|
||||
f("afoo bar", "afoo ba", false)
|
||||
f("foo bar! baz", "foo bar!", true)
|
||||
f("a.foo bar! baz", ".foo bar! ", true)
|
||||
f("foo bar! baz", "foo bar! b", false)
|
||||
f("255.255.255.255", "5", false)
|
||||
f("255.255.255.255", "55", false)
|
||||
f("255.255.255.255", "255", true)
|
||||
f("255.255.255.255", "5.255", false)
|
||||
f("255.255.255.255", "255.25", false)
|
||||
f("255.255.255.255", "255.255", true)
|
||||
}
|
||||
|
||||
func TestFilterPhrase(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other column",
|
||||
values: []string{
|
||||
"asdfdsf",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "other column",
|
||||
phrase: "asdfdsf",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "other column",
|
||||
phrase: "sd",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "other-column",
|
||||
values: []string{
|
||||
"x",
|
||||
"x",
|
||||
"x",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: " def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "other-column",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: " 2 ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abc def ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "other-column",
|
||||
phrase: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"afoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 3, 6})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "baz",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing column",
|
||||
phrase: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "НГКШ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{8})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "!,",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "aa a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "@",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 5})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3, 4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65535",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "123456",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "1234.5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "-65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{3})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "-1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "+1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5678",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "2.3",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{1, 2, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "127.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "27.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
pf := &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{4})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "002Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{1})
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "non-existing-column",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-03-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "06",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it has different string representation of the timestamp
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02T16:04:05.005+01:00",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it contains too many digits for millisecond part
|
||||
pf = &filterPhrase{
|
||||
fieldName: "_msg",
|
||||
phrase: "2006-01-02T15:04:05.00500Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, pf, "_msg", nil)
|
||||
})
|
||||
}
|
357
lib/logstorage/filter_prefix.go
Normal file
357
lib/logstorage/filter_prefix.go
Normal file
|
@ -0,0 +1,357 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterPrefix matches the given prefix.
|
||||
//
|
||||
// Example LogsQL: `fieldName:prefix*` or `fieldName:"some prefix"*`
|
||||
//
|
||||
// A special case `fieldName:*` matches non-empty value for the given `fieldName` field
|
||||
type filterPrefix struct {
|
||||
fieldName string
|
||||
prefix string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
}
|
||||
|
||||
func (fp *filterPrefix) String() string {
|
||||
if fp.prefix == "" {
|
||||
return quoteFieldNameIfNeeded(fp.fieldName) + "*"
|
||||
}
|
||||
return fmt.Sprintf("%s%s*", quoteFieldNameIfNeeded(fp.fieldName), quoteTokenIfNeeded(fp.prefix))
|
||||
}
|
||||
|
||||
func (fp *filterPrefix) getTokens() []string {
|
||||
fp.tokensOnce.Do(fp.initTokens)
|
||||
return fp.tokens
|
||||
}
|
||||
|
||||
func (fp *filterPrefix) initTokens() {
|
||||
fp.tokens = getTokensSkipLast(fp.prefix)
|
||||
}
|
||||
|
||||
func (fp *filterPrefix) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fp.fieldName
|
||||
prefix := fp.prefix
|
||||
|
||||
// Verify whether fp matches const column
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchPrefix(v, prefix) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether fp matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fp.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByPrefix(bs, ch, bm, prefix)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByPrefix(bs, ch, bm, prefix, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByPrefix(bs, ch, bm, prefix, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the timestamp values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// There is no sense in trying to parse prefix, since it may contain incomplete timestamp.
|
||||
// We cannot compar binary representation of timestamp and need converting
|
||||
// the timestamp to string before searching for the prefix there.
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the ipv4 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// There is no sense in trying to parse prefix, since it may contain incomplete ip.
|
||||
// We cannot compare binary representation of ip address and need converting
|
||||
// the ip to string before searching for the prefix there.
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the float64 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// The prefix may contain a part of the floating-point number.
|
||||
// For example, `foo:12*` must match `12`, `123.456` and `-0.123`.
|
||||
// This means we cannot search in binary representation of floating-point numbers.
|
||||
// Instead, we need searching for the whole prefix in string representation
|
||||
// of floating-point numbers :(
|
||||
_, ok := tryParseFloat64(prefix)
|
||||
if !ok && prefix != "." && prefix != "+" && prefix != "-" && !strings.HasPrefix(prefix, "e") && !strings.HasPrefix(prefix, "E") {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchPrefix(v, prefix) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchPrefix(v, prefix)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the uint8 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// The prefix may contain a part of the number.
|
||||
// For example, `foo:12*` must match `12` and `123`.
|
||||
// This means we cannot search in binary representation of numbers.
|
||||
// Instead, we need searching for the whole prefix in string representation of numbers :(
|
||||
n, ok := tryParseUint64(prefix)
|
||||
if !ok || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// There is no need in matching against bloom filters, since tokens is empty.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint8String(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the uint16 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// The prefix may contain a part of the number.
|
||||
// For example, `foo:12*` must match `12` and `123`.
|
||||
// This means we cannot search in binary representation of numbers.
|
||||
// Instead, we need searching for the whole prefix in string representation of numbers :(
|
||||
n, ok := tryParseUint64(prefix)
|
||||
if !ok || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// There is no need in matching against bloom filters, since tokens is empty.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint16String(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the uint32 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// The prefix may contain a part of the number.
|
||||
// For example, `foo:12*` must match `12` and `123`.
|
||||
// This means we cannot search in binary representation of numbers.
|
||||
// Instead, we need searching for the whole prefix in string representation of numbers :(
|
||||
n, ok := tryParseUint64(prefix)
|
||||
if !ok || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// There is no need in matching against bloom filters, since tokens is empty.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint32String(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByPrefix(bs *blockSearch, ch *columnHeader, bm *bitmap, prefix string) {
|
||||
if prefix == "" {
|
||||
// Fast path - all the uint64 values match an empty prefix aka `*`
|
||||
return
|
||||
}
|
||||
// The prefix may contain a part of the number.
|
||||
// For example, `foo:12*` must match `12` and `123`.
|
||||
// This means we cannot search in binary representation of numbers.
|
||||
// Instead, we need searching for the whole prefix in string representation of numbers :(
|
||||
n, ok := tryParseUint64(prefix)
|
||||
if !ok || n > ch.maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// There is no need in matching against bloom filters, since tokens is empty.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint64String(bs, bb, v)
|
||||
return matchPrefix(s, prefix)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchPrefix(s, prefix string) bool {
|
||||
if len(prefix) == 0 {
|
||||
// Special case - empty prefix matches any string.
|
||||
return len(s) > 0
|
||||
}
|
||||
if len(prefix) > len(s) {
|
||||
return false
|
||||
}
|
||||
|
||||
r := rune(prefix[0])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeRuneInString(prefix)
|
||||
}
|
||||
startsWithToken := isTokenRune(r)
|
||||
offset := 0
|
||||
for {
|
||||
n := strings.Index(s[offset:], prefix)
|
||||
if n < 0 {
|
||||
return false
|
||||
}
|
||||
offset += n
|
||||
// Make sure that the found phrase contains non-token chars at the beginning
|
||||
if startsWithToken && offset > 0 {
|
||||
r := rune(s[offset-1])
|
||||
if r >= utf8.RuneSelf {
|
||||
r, _ = utf8.DecodeLastRuneInString(s[:offset])
|
||||
}
|
||||
if r == utf8.RuneError || isTokenRune(r) {
|
||||
offset++
|
||||
continue
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
func getTokensSkipLast(s string) []string {
|
||||
for {
|
||||
r, runeSize := utf8.DecodeLastRuneInString(s)
|
||||
if !isTokenRune(r) {
|
||||
break
|
||||
}
|
||||
s = s[:len(s)-runeSize]
|
||||
}
|
||||
return tokenizeStrings(nil, []string{s})
|
||||
}
|
||||
|
||||
func toUint8String(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 1 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint8 number: got %d; want 1", bs.partPath(), len(v))
|
||||
}
|
||||
n := uint64(v[0])
|
||||
bb.B = marshalUint64(bb.B[:0], n)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
||||
|
||||
func toUint16String(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 2 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint16 number: got %d; want 2", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint16(b))
|
||||
bb.B = marshalUint64(bb.B[:0], n)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
||||
|
||||
func toUint32String(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint32 number: got %d; want 4", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint32(b))
|
||||
bb.B = marshalUint64(bb.B[:0], n)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
||||
|
||||
func toUint64String(bs *blockSearch, bb *bytesutil.ByteBuffer, v string) string {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint64 number: got %d; want 8", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := encoding.UnmarshalUint64(b)
|
||||
bb.B = marshalUint64(bb.B[:0], n)
|
||||
return bytesutil.ToUnsafeString(bb.B)
|
||||
}
|
929
lib/logstorage/filter_prefix_test.go
Normal file
929
lib/logstorage/filter_prefix_test.go
Normal file
|
@ -0,0 +1,929 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchPrefix(t *testing.T) {
|
||||
f := func(s, prefix string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchPrefix(s, prefix)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f("", "", false)
|
||||
f("foo", "", true)
|
||||
f("", "foo", false)
|
||||
f("foo", "foo", true)
|
||||
f("foo bar", "foo", true)
|
||||
f("foo bar", "bar", true)
|
||||
f("a foo bar", "foo", true)
|
||||
f("a foo bar", "fo", true)
|
||||
f("a foo bar", "oo", false)
|
||||
f("foobar", "foo", true)
|
||||
f("foobar", "bar", false)
|
||||
f("foobar", "oob", false)
|
||||
f("afoobar foo", "foo", true)
|
||||
f("раз два (три!)", "три", true)
|
||||
f("", "foo bar", false)
|
||||
f("foo bar", "foo bar", true)
|
||||
f("(foo bar)", "foo bar", true)
|
||||
f("afoo bar", "foo bar", false)
|
||||
f("afoo bar", "afoo ba", true)
|
||||
f("foo bar! baz", "foo bar!", true)
|
||||
f("a.foo bar! baz", ".foo bar! ", true)
|
||||
f("foo bar! baz", "foo bar! b", true)
|
||||
f("255.255.255.255", "5", false)
|
||||
f("255.255.255.255", "55", false)
|
||||
f("255.255.255.255", "255", true)
|
||||
f("255.255.255.255", "5.255", false)
|
||||
f("255.255.255.255", "255.25", true)
|
||||
f("255.255.255.255", "255.255", true)
|
||||
}
|
||||
|
||||
func TestFilterPrefix(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other column",
|
||||
values: []string{
|
||||
"asdfdsf",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "other column",
|
||||
prefix: "asdfdsf",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "other column",
|
||||
prefix: "sd",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "other-column",
|
||||
values: []string{
|
||||
"x",
|
||||
"x",
|
||||
"x",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
"1 2 3",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ab",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc de",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: " de",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "other-column",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: " 2 ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "abc def ",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "other-column",
|
||||
prefix: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "x",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "foo",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"afoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 3, 4, 6})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 2, 3, 4, 5, 6})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "ba",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "foobar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "НГК",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{8})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa a",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "!,",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{9})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "aa ax",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "qwe rty abc",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "@",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 5})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3, 4})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65535",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123456",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"1234",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"65536",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "123",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "1234.5678901",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "56789",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "-6553",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "65536",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{3})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "7344.8943",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "-1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "+1234",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "23",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "678",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12345678901234567890",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "12",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "2.3.",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{0})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", []int{1, 2, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "8",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "127.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "27.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "foo",
|
||||
prefix: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fp := &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{4})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "002",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", []int{1})
|
||||
|
||||
// mimatch
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "bar",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-03-02T15:04:05.005Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "06",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it has different string representation of the timestamp
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T16:04:05.005+01:00",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
// This filter shouldn't match row=4, since it contains too many digits for millisecond part
|
||||
fp = &filterPrefix{
|
||||
fieldName: "_msg",
|
||||
prefix: "2006-01-02T15:04:05.00500Z",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
|
||||
fp = &filterPrefix{
|
||||
fieldName: "non-existing-column",
|
||||
prefix: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fp, "_msg", nil)
|
||||
})
|
||||
}
|
203
lib/logstorage/filter_range.go
Normal file
203
lib/logstorage/filter_range.go
Normal file
|
@ -0,0 +1,203 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"math"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterRange matches the given range [minValue..maxValue].
|
||||
//
|
||||
// Example LogsQL: `fieldName:range(minValue, maxValue]`
|
||||
type filterRange struct {
|
||||
fieldName string
|
||||
minValue float64
|
||||
maxValue float64
|
||||
|
||||
stringRepr string
|
||||
}
|
||||
|
||||
func (fr *filterRange) String() string {
|
||||
return quoteFieldNameIfNeeded(fr.fieldName) + "range" + fr.stringRepr
|
||||
}
|
||||
|
||||
func (fr *filterRange) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fr.fieldName
|
||||
minValue := fr.minValue
|
||||
maxValue := fr.maxValue
|
||||
|
||||
if minValue > maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchRange(v, minValue, maxValue) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeIPv4:
|
||||
bm.resetBits()
|
||||
case valueTypeTimestampISO8601:
|
||||
bm.resetBits()
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchFloat64ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
if minValue > math.Float64frombits(ch.maxValue) || maxValue < math.Float64frombits(ch.minValue) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of floating-point number: got %d; want 8", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := encoding.UnmarshalUint64(b)
|
||||
f := math.Float64frombits(n)
|
||||
return f >= minValue && f <= maxValue
|
||||
})
|
||||
}
|
||||
|
||||
func matchValuesDictByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchRange(v, minValue, maxValue) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchRange(v, minValue, maxValue)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
minValueUint, maxValueUint := toUint64Range(minValue, maxValue)
|
||||
if maxValue < 0 || minValueUint > ch.maxValue || maxValueUint < ch.minValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 1 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint8 number: got %d; want 1", bs.partPath(), len(v))
|
||||
}
|
||||
n := uint64(v[0])
|
||||
return n >= minValueUint && n <= maxValueUint
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
minValueUint, maxValueUint := toUint64Range(minValue, maxValue)
|
||||
if maxValue < 0 || minValueUint > ch.maxValue || maxValueUint < ch.minValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 2 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint16 number: got %d; want 2", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint16(b))
|
||||
return n >= minValueUint && n <= maxValueUint
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
minValueUint, maxValueUint := toUint64Range(minValue, maxValue)
|
||||
if maxValue < 0 || minValueUint > ch.maxValue || maxValueUint < ch.minValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 4 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint8 number: got %d; want 4", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := uint64(encoding.UnmarshalUint32(b))
|
||||
return n >= minValueUint && n <= maxValueUint
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue float64) {
|
||||
minValueUint, maxValueUint := toUint64Range(minValue, maxValue)
|
||||
if maxValue < 0 || minValueUint > ch.maxValue || maxValueUint < ch.minValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
if len(v) != 8 {
|
||||
logger.Panicf("FATAL: %s: unexpected length for binary representation of uint8 number: got %d; want 8", bs.partPath(), len(v))
|
||||
}
|
||||
b := bytesutil.ToUnsafeBytes(v)
|
||||
n := encoding.UnmarshalUint64(b)
|
||||
return n >= minValueUint && n <= maxValueUint
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchRange(s string, minValue, maxValue float64) bool {
|
||||
f, ok := tryParseFloat64(s)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return f >= minValue && f <= maxValue
|
||||
}
|
||||
|
||||
func toUint64Range(minValue, maxValue float64) (uint64, uint64) {
|
||||
minValue = math.Ceil(minValue)
|
||||
maxValue = math.Floor(maxValue)
|
||||
return toUint64Clamp(minValue), toUint64Clamp(maxValue)
|
||||
}
|
||||
|
||||
func toUint64Clamp(f float64) uint64 {
|
||||
if f < 0 {
|
||||
return 0
|
||||
}
|
||||
if f > math.MaxUint64 {
|
||||
return math.MaxUint64
|
||||
}
|
||||
return uint64(f)
|
||||
}
|
608
lib/logstorage/filter_range_test.go
Normal file
608
lib/logstorage/filter_range_test.go
Normal file
|
@ -0,0 +1,608 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterRange(t *testing.T) {
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"10",
|
||||
"10",
|
||||
"10",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10,
|
||||
maxValue: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -10,
|
||||
maxValue: 9.99,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 20,
|
||||
maxValue: -10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10.1,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "non-existing-column",
|
||||
minValue: 10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 11,
|
||||
maxValue: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"10",
|
||||
"Abc",
|
||||
"20",
|
||||
"10.5",
|
||||
"10 AFoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 3, 4})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 3, 4})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10.1,
|
||||
maxValue: 19.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{4})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -11,
|
||||
maxValue: 0,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 11,
|
||||
maxValue: 19,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 20.1,
|
||||
maxValue: 100,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 20,
|
||||
maxValue: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a 10",
|
||||
"10",
|
||||
"20",
|
||||
"15.5",
|
||||
"-5",
|
||||
"a fooBaR",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -100,
|
||||
maxValue: 100,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 4, 5})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 10,
|
||||
maxValue: 20,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 4})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -5,
|
||||
maxValue: -5,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{5})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -10,
|
||||
maxValue: -5.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 20.1,
|
||||
maxValue: 100,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 20,
|
||||
maxValue: 10,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{6, 7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: -0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 0.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 2.9,
|
||||
maxValue: 0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65535",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{6, 7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: -0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 0.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 2.9,
|
||||
maxValue: 0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65536",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0,
|
||||
maxValue: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{6, 7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: -0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 0.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 2.9,
|
||||
maxValue: 0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -inf,
|
||||
maxValue: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{6, 7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 1000,
|
||||
maxValue: inf,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{5})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: -0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 0.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 2.9,
|
||||
maxValue: 0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"123456.78901",
|
||||
"-0.2",
|
||||
"2",
|
||||
"-334",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -inf,
|
||||
maxValue: 3,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 7, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 2.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{7})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: 1.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{3, 4, 6, 8})
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 1000,
|
||||
maxValue: inf,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{5})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -1e18,
|
||||
maxValue: -334.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 0.1,
|
||||
maxValue: 0.9,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: 2.9,
|
||||
maxValue: 0.1,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// range filter always mismatches ipv4
|
||||
fr := &filterRange{
|
||||
fieldName: "foo",
|
||||
minValue: -100,
|
||||
maxValue: 100,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// range filter always mismatches timestmap
|
||||
fr := &filterRange{
|
||||
fieldName: "_msg",
|
||||
minValue: -100,
|
||||
maxValue: 100,
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
})
|
||||
}
|
147
lib/logstorage/filter_regexp.go
Normal file
147
lib/logstorage/filter_regexp.go
Normal file
|
@ -0,0 +1,147 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"regexp"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterRegexp matches the given regexp
|
||||
//
|
||||
// Example LogsQL: `fieldName:re("regexp")`
|
||||
type filterRegexp struct {
|
||||
fieldName string
|
||||
re *regexp.Regexp
|
||||
}
|
||||
|
||||
func (fr *filterRegexp) String() string {
|
||||
return fmt.Sprintf("%sre(%q)", quoteFieldNameIfNeeded(fr.fieldName), fr.re.String())
|
||||
}
|
||||
|
||||
func (fr *filterRegexp) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fr.fieldName
|
||||
re := fr.re
|
||||
|
||||
// Verify whether filter matches const column
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !re.MatchString(v) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
if !re.MatchString("") {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByRegexp(bs, ch, bm, re)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByRegexp(bs, ch, bm, re)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByRegexp(bs, ch, bm, re)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByRegexp(bs, ch, bm, re)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if re.MatchString(v) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return re.MatchString(v)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint8String(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint16String(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint32String(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByRegexp(bs *blockSearch, ch *columnHeader, bm *bitmap, re *regexp.Regexp) {
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint64String(bs, bb, v)
|
||||
return re.MatchString(s)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
369
lib/logstorage/filter_regexp_test.go
Normal file
369
lib/logstorage/filter_regexp_test.go
Normal file
|
@ -0,0 +1,369 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"regexp"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestFilterRegexp(t *testing.T) {
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("0.0"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile(`^127\.0\.0\.1$`),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterRegexp{
|
||||
fieldName: "non-existing-column",
|
||||
re: regexp.MustCompile("foo.+bar|"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo.+bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterRegexp{
|
||||
fieldName: "non-existing-column",
|
||||
re: regexp.MustCompile("foo.+bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"127.0.0.1",
|
||||
"Abc",
|
||||
"127.255.255.255",
|
||||
"10.4",
|
||||
"foo 127.0.0.1",
|
||||
"127.0.0.1 bar",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar|^$"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 5, 6})
|
||||
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("27.0"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 5, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("bar.+foo"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a 10",
|
||||
"127.0.0.1",
|
||||
"20",
|
||||
"15.5",
|
||||
"-5",
|
||||
"a fooBaR",
|
||||
"a 127.0.0.1 dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("(?i)foo|йцу"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 6, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("qwe.+rty|^$"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("[32][23]?"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65535",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("[32][23]?"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65536",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("[32][23]?"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("[32][23]?"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"123456.78901",
|
||||
"-0.2",
|
||||
"2",
|
||||
"-334",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("[32][23]?"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("127.0.[40].(1|2)"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 4, 5, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "foo",
|
||||
re: regexp.MustCompile("foo|bar|834"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterRegexp{
|
||||
fieldName: "_msg",
|
||||
re: regexp.MustCompile("2006-[0-9]{2}-.+?(2|5)Z"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", []int{1, 4})
|
||||
|
||||
// mismatch
|
||||
fr = &filterRegexp{
|
||||
fieldName: "_msg",
|
||||
re: regexp.MustCompile("^01|04$"),
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
})
|
||||
}
|
234
lib/logstorage/filter_sequence.go
Normal file
234
lib/logstorage/filter_sequence.go
Normal file
|
@ -0,0 +1,234 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterSequence matches an ordered sequence of phrases
|
||||
//
|
||||
// Example LogsQL: `fieldName:seq(foo, "bar baz")`
|
||||
type filterSequence struct {
|
||||
fieldName string
|
||||
phrases []string
|
||||
|
||||
tokensOnce sync.Once
|
||||
tokens []string
|
||||
|
||||
nonEmptyPhrasesOnce sync.Once
|
||||
nonEmptyPhrases []string
|
||||
}
|
||||
|
||||
func (fs *filterSequence) String() string {
|
||||
phrases := fs.phrases
|
||||
a := make([]string, len(phrases))
|
||||
for i, phrase := range phrases {
|
||||
a[i] = quoteTokenIfNeeded(phrase)
|
||||
}
|
||||
return fmt.Sprintf("%sseq(%s)", quoteFieldNameIfNeeded(fs.fieldName), strings.Join(a, ","))
|
||||
}
|
||||
|
||||
func (fs *filterSequence) getTokens() []string {
|
||||
fs.tokensOnce.Do(fs.initTokens)
|
||||
return fs.tokens
|
||||
}
|
||||
|
||||
func (fs *filterSequence) initTokens() {
|
||||
phrases := fs.getNonEmptyPhrases()
|
||||
tokens := tokenizeStrings(nil, phrases)
|
||||
fs.tokens = tokens
|
||||
}
|
||||
|
||||
func (fs *filterSequence) getNonEmptyPhrases() []string {
|
||||
fs.nonEmptyPhrasesOnce.Do(fs.initNonEmptyPhrases)
|
||||
return fs.nonEmptyPhrases
|
||||
}
|
||||
|
||||
func (fs *filterSequence) initNonEmptyPhrases() {
|
||||
phrases := fs.phrases
|
||||
result := make([]string, 0, len(phrases))
|
||||
for _, phrase := range phrases {
|
||||
if phrase != "" {
|
||||
result = append(result, phrase)
|
||||
}
|
||||
}
|
||||
fs.nonEmptyPhrases = result
|
||||
}
|
||||
|
||||
func (fs *filterSequence) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fs.fieldName
|
||||
phrases := fs.getNonEmptyPhrases()
|
||||
|
||||
if len(phrases) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchSequence(v, phrases) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
// Fast path - there are no matching columns.
|
||||
// It matches anything only for empty phrase.
|
||||
if !matchSequence("", phrases) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
tokens := fs.getTokens()
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringBySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeDict:
|
||||
matchValuesDictBySequence(bs, ch, bm, phrases)
|
||||
case valueTypeUint8:
|
||||
matchUint8BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeUint16:
|
||||
matchUint16BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeUint32:
|
||||
matchUint32BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeUint64:
|
||||
matchUint64BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4BySequence(bs, ch, bm, phrases, tokens)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601BySequence(bs, ch, bm, phrases, tokens)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) == 1 {
|
||||
matchTimestampISO8601ByPhrase(bs, ch, bm, phrases[0], tokens)
|
||||
return
|
||||
}
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
// Slow path - phrases contain incomplete timestamp. Search over string representation of the timestamp.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return matchSequence(s, phrases)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) == 1 {
|
||||
matchIPv4ByPhrase(bs, ch, bm, phrases[0], tokens)
|
||||
return
|
||||
}
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
// Slow path - phrases contain parts of IP address. For example, `1.23` should match `1.23.4.5` and `4.1.23.54`.
|
||||
// We cannot compare binary represetnation of ip address and need converting
|
||||
// the ip to string before searching for prefix there.
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchSequence(s, phrases)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
// The phrase may contain a part of the floating-point number.
|
||||
// For example, `foo:"123"` must match `123`, `123.456` and `-0.123`.
|
||||
// This means we cannot search in binary representation of floating-point numbers.
|
||||
// Instead, we need searching for the whole phrase in string representation
|
||||
// of floating-point numbers :(
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchSequence(s, phrases)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictBySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases []string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchSequence(v, phrases) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringBySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases []string, tokens []string) {
|
||||
if !matchBloomFilterAllTokens(bs, ch, tokens) {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchSequence(v, phrases)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) > 1 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
matchUint8ByExactValue(bs, ch, bm, phrases[0], tokens)
|
||||
}
|
||||
|
||||
func matchUint16BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) > 1 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
matchUint16ByExactValue(bs, ch, bm, phrases[0], tokens)
|
||||
}
|
||||
|
||||
func matchUint32BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) > 1 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
matchUint32ByExactValue(bs, ch, bm, phrases[0], tokens)
|
||||
}
|
||||
|
||||
func matchUint64BySequence(bs *blockSearch, ch *columnHeader, bm *bitmap, phrases, tokens []string) {
|
||||
if len(phrases) > 1 {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
matchUint64ByExactValue(bs, ch, bm, phrases[0], tokens)
|
||||
}
|
||||
|
||||
func matchSequence(s string, phrases []string) bool {
|
||||
for _, phrase := range phrases {
|
||||
n := getPhrasePos(s, phrase)
|
||||
if n < 0 {
|
||||
return false
|
||||
}
|
||||
s = s[n+len(phrase):]
|
||||
}
|
||||
return true
|
||||
}
|
821
lib/logstorage/filter_sequence_test.go
Normal file
821
lib/logstorage/filter_sequence_test.go
Normal file
|
@ -0,0 +1,821 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchSequence(t *testing.T) {
|
||||
f := func(s string, phrases []string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchSequence(s, phrases)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f("", []string{""}, true)
|
||||
f("foo", []string{""}, true)
|
||||
f("", []string{"foo"}, false)
|
||||
f("foo", []string{"foo"}, true)
|
||||
f("foo bar", []string{"foo"}, true)
|
||||
f("foo bar", []string{"bar"}, true)
|
||||
f("foo bar", []string{"foo bar"}, true)
|
||||
f("foo bar", []string{"foo", "bar"}, true)
|
||||
f("foo bar", []string{"foo", " bar"}, true)
|
||||
f("foo bar", []string{"foo ", "bar"}, true)
|
||||
f("foo bar", []string{"foo ", " bar"}, false)
|
||||
f("foo bar", []string{"bar", "foo"}, false)
|
||||
}
|
||||
|
||||
func TestFilterSequence(t *testing.T) {
|
||||
t.Run("single-row", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"def"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc def"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc ", "", "def", ""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"ab"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc", "abc"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc", "def", "foo"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"abc def",
|
||||
"abc def",
|
||||
"abc def",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc", " def"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc ", ""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{"", ""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"abc def ", "foobar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing column",
|
||||
phrases: []string{"x", "yz"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"baz foobar",
|
||||
"abc",
|
||||
"afdf foobar baz",
|
||||
"fddf foobarbaz",
|
||||
"afoobarbaz",
|
||||
"foobar",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"foobar", "baz"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{3})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"baz", "aaaa"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing column",
|
||||
phrases: []string{"foobar", "aaaa"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a bb foo",
|
||||
"bb a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a afoobarbaz",
|
||||
"a foobar bb",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"a", "bb"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 6})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"НГКШ", " "},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"!,", "(!1)"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{9})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"aa a", "bcdasqq"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"@", "!!!!"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"12"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{1, 5})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", "bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234", "567"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"256",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"12"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{1, 5})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", "bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234", "567"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"65536",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"12"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{1, 5})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", "bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234", "567"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"12345678901",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"12"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{1, 5})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", "bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234", "567"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1234",
|
||||
"0",
|
||||
"3454",
|
||||
"-65536",
|
||||
"1234.5678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"-", "65536"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{3})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"1234.", "5678901"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{4})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", "5678901"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{4})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"", ""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"65536", "-"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"5678901", "1234"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"12345678901234567890"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"1.0.127.6",
|
||||
"55.55.55.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"127.0.0.1"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"127", "1"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"127.0.0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{2, 4, 5, 7})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"2.3", ".4"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11})
|
||||
|
||||
// mismatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"5"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"127.", "1", "1", "345"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"27.0"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "foo",
|
||||
phrases: []string{"255.255.255.255"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2006-01-02T15:04:05.001Z",
|
||||
"2006-01-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2006-01-02T15:04:05.005Z",
|
||||
"2006-01-02T15:04:05.006Z",
|
||||
"2006-01-02T15:04:05.007Z",
|
||||
"2006-01-02T15:04:05.008Z",
|
||||
"2006-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fs := &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"2006-01-02T15:04:05.005Z"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{4})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"2006-01", "04:05."},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"2006", "002Z"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{1})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "non-existing-column",
|
||||
phrases: []string{""},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", []int{0, 1, 2, 3, 4, 5, 6, 7, 8})
|
||||
|
||||
// mimatch
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"bar"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"002Z", "2006"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"2006-04-02T15:04:05.005Z", "2023"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", nil)
|
||||
|
||||
fs = &filterSequence{
|
||||
fieldName: "_msg",
|
||||
phrases: []string{"06"},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fs, "_msg", nil)
|
||||
})
|
||||
}
|
53
lib/logstorage/filter_stream.go
Normal file
53
lib/logstorage/filter_stream.go
Normal file
|
@ -0,0 +1,53 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"sync"
|
||||
)
|
||||
|
||||
// filterStream is the filter for `_stream:{...}`
|
||||
type filterStream struct {
|
||||
// f is the filter to apply
|
||||
f *StreamFilter
|
||||
|
||||
// tenantIDs is the list of tenantIDs to search for streamIDs.
|
||||
tenantIDs []TenantID
|
||||
|
||||
// idb is the indexdb to search for streamIDs.
|
||||
idb *indexdb
|
||||
|
||||
streamIDsOnce sync.Once
|
||||
streamIDs map[streamID]struct{}
|
||||
}
|
||||
|
||||
func (fs *filterStream) String() string {
|
||||
s := fs.f.String()
|
||||
if s == "{}" {
|
||||
return ""
|
||||
}
|
||||
return "_stream:" + s
|
||||
}
|
||||
|
||||
func (fs *filterStream) getStreamIDs() map[streamID]struct{} {
|
||||
fs.streamIDsOnce.Do(fs.initStreamIDs)
|
||||
return fs.streamIDs
|
||||
}
|
||||
|
||||
func (fs *filterStream) initStreamIDs() {
|
||||
streamIDs := fs.idb.searchStreamIDs(fs.tenantIDs, fs.f)
|
||||
m := make(map[streamID]struct{}, len(streamIDs))
|
||||
for i := range streamIDs {
|
||||
m[streamIDs[i]] = struct{}{}
|
||||
}
|
||||
fs.streamIDs = m
|
||||
}
|
||||
|
||||
func (fs *filterStream) apply(bs *blockSearch, bm *bitmap) {
|
||||
if fs.f.isEmpty() {
|
||||
return
|
||||
}
|
||||
streamIDs := fs.getStreamIDs()
|
||||
if _, ok := streamIDs[bs.bsw.bh.streamID]; !ok {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
}
|
189
lib/logstorage/filter_string_range.go
Normal file
189
lib/logstorage/filter_string_range.go
Normal file
|
@ -0,0 +1,189 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// filterStringRange matches tie given string range [minValue..maxValue)
|
||||
//
|
||||
// Note that the minValue is included in the range, while the maxValue isn't included in the range.
|
||||
// This simplifies querying distincts log sets with string_range(A, B), string_range(B, C), etc.
|
||||
//
|
||||
// Example LogsQL: `fieldName:string_range(minValue, maxValue)`
|
||||
type filterStringRange struct {
|
||||
fieldName string
|
||||
minValue string
|
||||
maxValue string
|
||||
}
|
||||
|
||||
func (fr *filterStringRange) String() string {
|
||||
return fmt.Sprintf("%sstring_range(%s, %s)", quoteFieldNameIfNeeded(fr.fieldName), quoteTokenIfNeeded(fr.minValue), quoteTokenIfNeeded(fr.maxValue))
|
||||
}
|
||||
|
||||
func (fr *filterStringRange) apply(bs *blockSearch, bm *bitmap) {
|
||||
fieldName := fr.fieldName
|
||||
minValue := fr.minValue
|
||||
maxValue := fr.maxValue
|
||||
|
||||
if minValue > maxValue {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
v := bs.csh.getConstColumnValue(fieldName)
|
||||
if v != "" {
|
||||
if !matchStringRange(v, minValue, maxValue) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Verify whether filter matches other columns
|
||||
ch := bs.csh.getColumnHeader(fieldName)
|
||||
if ch == nil {
|
||||
if !matchStringRange("", minValue, maxValue) {
|
||||
bm.resetBits()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
switch ch.valueType {
|
||||
case valueTypeString:
|
||||
matchStringByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeDict:
|
||||
matchValuesDictByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint8:
|
||||
matchUint8ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint16:
|
||||
matchUint16ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint32:
|
||||
matchUint32ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeUint64:
|
||||
matchUint64ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeFloat64:
|
||||
matchFloat64ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeIPv4:
|
||||
matchIPv4ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
case valueTypeTimestampISO8601:
|
||||
matchTimestampISO8601ByStringRange(bs, ch, bm, minValue, maxValue)
|
||||
default:
|
||||
logger.Panicf("FATAL: %s: unknown valueType=%d", bs.partPath(), ch.valueType)
|
||||
}
|
||||
}
|
||||
|
||||
func matchTimestampISO8601ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toTimestampISO8601StringExt(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchIPv4ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toIPv4StringExt(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchFloat64ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "+" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toFloat64StringExt(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchValuesDictByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
bb := bbPool.Get()
|
||||
for i, v := range ch.valuesDict.values {
|
||||
if matchStringRange(v, minValue, maxValue) {
|
||||
bb.B = append(bb.B, byte(i))
|
||||
}
|
||||
}
|
||||
matchEncodedValuesDict(bs, ch, bm, bb.B)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
return matchStringRange(v, minValue, maxValue)
|
||||
})
|
||||
}
|
||||
|
||||
func matchUint8ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint8String(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint16ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint16String(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint32ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint32String(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchUint64ByStringRange(bs *blockSearch, ch *columnHeader, bm *bitmap, minValue, maxValue string) {
|
||||
if minValue > "9" || maxValue < "0" {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
bb := bbPool.Get()
|
||||
visitValues(bs, ch, bm, func(v string) bool {
|
||||
s := toUint64String(bs, bb, v)
|
||||
return matchStringRange(s, minValue, maxValue)
|
||||
})
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
func matchStringRange(s, minValue, maxValue string) bool {
|
||||
return s >= minValue && s < maxValue
|
||||
}
|
548
lib/logstorage/filter_string_range_test.go
Normal file
548
lib/logstorage/filter_string_range_test.go
Normal file
|
@ -0,0 +1,548 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMatchStringRange(t *testing.T) {
|
||||
f := func(s, minValue, maxValue string, resultExpected bool) {
|
||||
t.Helper()
|
||||
result := matchStringRange(s, minValue, maxValue)
|
||||
if result != resultExpected {
|
||||
t.Fatalf("unexpected result; got %v; want %v", result, resultExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f("foo", "a", "b", false)
|
||||
f("foo", "a", "foa", false)
|
||||
f("foo", "a", "foz", true)
|
||||
f("foo", "foo", "foo", false)
|
||||
f("foo", "foo", "fooa", true)
|
||||
f("foo", "fooa", "foo", false)
|
||||
}
|
||||
|
||||
func TestFilterStringRange(t *testing.T) {
|
||||
t.Run("const-column", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.1",
|
||||
maxValue: "255.",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.1",
|
||||
maxValue: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{0, 1, 2})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "",
|
||||
maxValue: "127.0.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "non-existing-column",
|
||||
minValue: "1",
|
||||
maxValue: "2",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.2",
|
||||
maxValue: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("dict", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"",
|
||||
"127.0.0.1",
|
||||
"Abc",
|
||||
"127.255.255.255",
|
||||
"10.4",
|
||||
"foo 127.0.0.1",
|
||||
"127.0.0.1 bar",
|
||||
"127.0.0.1",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.0",
|
||||
maxValue: "128.0.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 3, 6, 7})
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127",
|
||||
maxValue: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{1, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "0",
|
||||
maxValue: "10",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.2",
|
||||
maxValue: "127.127.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "128.0.0.0",
|
||||
maxValue: "127.0.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("strings", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"A FOO",
|
||||
"a 10",
|
||||
"127.0.0.1",
|
||||
"20",
|
||||
"15.5",
|
||||
"-5",
|
||||
"a fooBaR",
|
||||
"a 127.0.0.1 dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0.1",
|
||||
maxValue: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 3, 4})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "0",
|
||||
maxValue: "10",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "255.255.255.255",
|
||||
maxValue: "127.0.0.1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint8", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "33",
|
||||
maxValue: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{9, 10})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "100",
|
||||
maxValue: "101",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "5",
|
||||
maxValue: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint16", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65535",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "33",
|
||||
maxValue: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{9, 10})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "100",
|
||||
maxValue: "101",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "5",
|
||||
maxValue: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint32", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"65536",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "33",
|
||||
maxValue: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{9, 10})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "100",
|
||||
maxValue: "101",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "5",
|
||||
maxValue: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("uint64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"12345678901",
|
||||
"1",
|
||||
"2",
|
||||
"3",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "33",
|
||||
maxValue: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{9, 10})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "100",
|
||||
maxValue: "101",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "5",
|
||||
maxValue: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("float64", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"123",
|
||||
"12",
|
||||
"32",
|
||||
"0",
|
||||
"0",
|
||||
"123456.78901",
|
||||
"-0.2",
|
||||
"2",
|
||||
"-334",
|
||||
"4",
|
||||
"5",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "33",
|
||||
maxValue: "5",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{9, 10})
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "-0",
|
||||
maxValue: "-1",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{6})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "100",
|
||||
maxValue: "101",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "5",
|
||||
maxValue: "33",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("ipv4", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"1.2.3.4",
|
||||
"0.0.0.0",
|
||||
"127.0.0.1",
|
||||
"254.255.255.255",
|
||||
"127.0.0.1",
|
||||
"127.0.0.1",
|
||||
"127.0.4.2",
|
||||
"127.0.0.1",
|
||||
"12.0.127.6",
|
||||
"55.55.12.55",
|
||||
"66.66.66.66",
|
||||
"7.7.7.7",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "127.0.0",
|
||||
maxValue: "128.0.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", []int{2, 4, 5, 6, 7})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "128.0.0.0",
|
||||
maxValue: "129.0.0.0",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "255.0.0.0",
|
||||
maxValue: "255.255.255.255",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "foo",
|
||||
minValue: "128.0.0.0",
|
||||
maxValue: "",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "foo", nil)
|
||||
})
|
||||
|
||||
t.Run("timestamp-iso8601", func(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "_msg",
|
||||
values: []string{
|
||||
"2005-01-02T15:04:05.001Z",
|
||||
"2006-02-02T15:04:05.002Z",
|
||||
"2006-01-02T15:04:05.003Z",
|
||||
"2006-01-02T15:04:05.004Z",
|
||||
"2026-01-02T15:04:05.005Z",
|
||||
"2026-01-02T15:04:05.006Z",
|
||||
"2026-01-02T15:04:05.007Z",
|
||||
"2026-01-02T15:04:05.008Z",
|
||||
"2026-01-02T15:04:05.009Z",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// match
|
||||
fr := &filterStringRange{
|
||||
fieldName: "_msg",
|
||||
minValue: "2006-01-02",
|
||||
maxValue: "2006-01-03",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", []int{2, 3})
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "_msg",
|
||||
minValue: "",
|
||||
maxValue: "2006",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", []int{0})
|
||||
|
||||
// mismatch
|
||||
fr = &filterStringRange{
|
||||
fieldName: "_msg",
|
||||
minValue: "3",
|
||||
maxValue: "4",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "_msg",
|
||||
minValue: "a",
|
||||
maxValue: "b",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
|
||||
fr = &filterStringRange{
|
||||
fieldName: "_msg",
|
||||
minValue: "2006-01-03",
|
||||
maxValue: "2006-01-02",
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, fr, "_msg", nil)
|
||||
})
|
||||
}
|
253
lib/logstorage/filter_test.go
Normal file
253
lib/logstorage/filter_test.go
Normal file
|
@ -0,0 +1,253 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
)
|
||||
|
||||
func TestComplexFilters(t *testing.T) {
|
||||
columns := []column{
|
||||
{
|
||||
name: "foo",
|
||||
values: []string{
|
||||
"a foo",
|
||||
"a foobar",
|
||||
"aa abc a",
|
||||
"ca afdf a,foobar baz",
|
||||
"a fddf foobarbaz",
|
||||
"a",
|
||||
"a foobar abcdef",
|
||||
"a kjlkjf dfff",
|
||||
"a ТЕСТЙЦУК НГКШ ",
|
||||
"a !!,23.(!1)",
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
// (foobar AND NOT baz AND (abcdef OR xyz))
|
||||
f := &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foobar",
|
||||
},
|
||||
&filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "baz",
|
||||
},
|
||||
},
|
||||
&filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abcdef",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "xyz",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, f, "foo", []int{6})
|
||||
|
||||
// (foobaz AND NOT baz AND (abcdef OR xyz))
|
||||
f = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foobaz",
|
||||
},
|
||||
&filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "baz",
|
||||
},
|
||||
},
|
||||
&filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abcdef",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "xyz",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, f, "foo", nil)
|
||||
|
||||
// (foobaz AND NOT baz AND (abcdef OR xyz OR a))
|
||||
f = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foobar",
|
||||
},
|
||||
&filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "baz",
|
||||
},
|
||||
},
|
||||
&filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abcdef",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "xyz",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, f, "foo", []int{1, 6})
|
||||
|
||||
// (foobaz AND NOT qwert AND (abcdef OR xyz OR a))
|
||||
f = &filterAnd{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "foobar",
|
||||
},
|
||||
&filterNot{
|
||||
f: &filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "qwert",
|
||||
},
|
||||
},
|
||||
&filterOr{
|
||||
filters: []filter{
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "abcdef",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "xyz",
|
||||
},
|
||||
&filterPhrase{
|
||||
fieldName: "foo",
|
||||
phrase: "a",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
testFilterMatchForColumns(t, columns, f, "foo", []int{1, 3, 6})
|
||||
}
|
||||
|
||||
func testFilterMatchForColumns(t *testing.T, columns []column, f filter, neededColumnName string, expectedRowIdxs []int) {
|
||||
t.Helper()
|
||||
|
||||
// Create the test storage
|
||||
const storagePath = "testFilterMatchForColumns"
|
||||
cfg := &StorageConfig{}
|
||||
s := MustOpenStorage(storagePath, cfg)
|
||||
|
||||
// Generate rows
|
||||
tenantID := TenantID{
|
||||
AccountID: 123,
|
||||
ProjectID: 456,
|
||||
}
|
||||
generateRowsFromColumns(s, tenantID, columns)
|
||||
|
||||
var values []string
|
||||
for _, c := range columns {
|
||||
if c.name == neededColumnName {
|
||||
values = c.values
|
||||
break
|
||||
}
|
||||
}
|
||||
expectedResults := make([]string, len(expectedRowIdxs))
|
||||
expectedTimestamps := make([]int64, len(expectedRowIdxs))
|
||||
for i, idx := range expectedRowIdxs {
|
||||
expectedResults[i] = values[idx]
|
||||
expectedTimestamps[i] = int64(idx) * 1e9
|
||||
}
|
||||
|
||||
testFilterMatchForStorage(t, s, tenantID, f, neededColumnName, expectedResults, expectedTimestamps)
|
||||
|
||||
// Close and delete the test storage
|
||||
s.MustClose()
|
||||
fs.MustRemoveAll(storagePath)
|
||||
}
|
||||
|
||||
func testFilterMatchForStorage(t *testing.T, s *Storage, tenantID TenantID, f filter, neededColumnName string, expectedResults []string, expectedTimestamps []int64) {
|
||||
t.Helper()
|
||||
|
||||
so := &genericSearchOptions{
|
||||
tenantIDs: []TenantID{tenantID},
|
||||
filter: f,
|
||||
neededColumnNames: []string{neededColumnName},
|
||||
}
|
||||
workersCount := 3
|
||||
s.search(workersCount, so, nil, func(_ uint, br *blockResult) {
|
||||
// Verify tenantID
|
||||
if !br.streamID.tenantID.equal(&tenantID) {
|
||||
t.Fatalf("unexpected tenantID in blockResult; got %s; want %s", &br.streamID.tenantID, &tenantID)
|
||||
}
|
||||
|
||||
// Verify columns
|
||||
cs := br.getColumns()
|
||||
if len(cs) != 1 {
|
||||
t.Fatalf("unexpected number of columns in blockResult; got %d; want 1", len(cs))
|
||||
}
|
||||
results := cs[0].getValues(br)
|
||||
if !reflect.DeepEqual(results, expectedResults) {
|
||||
t.Fatalf("unexpected results matched;\ngot\n%q\nwant\n%q", results, expectedResults)
|
||||
}
|
||||
|
||||
// Verify timestamps
|
||||
if br.timestamps == nil {
|
||||
br.timestamps = []int64{}
|
||||
}
|
||||
if !reflect.DeepEqual(br.timestamps, expectedTimestamps) {
|
||||
t.Fatalf("unexpected timestamps;\ngot\n%d\nwant\n%d", br.timestamps, expectedTimestamps)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func generateRowsFromColumns(s *Storage, tenantID TenantID, columns []column) {
|
||||
streamTags := []string{
|
||||
"job",
|
||||
"instance",
|
||||
}
|
||||
lr := GetLogRows(streamTags, nil)
|
||||
var fields []Field
|
||||
for i := range columns[0].values {
|
||||
// Add stream tags
|
||||
fields = append(fields[:0], Field{
|
||||
Name: "job",
|
||||
Value: "foobar",
|
||||
}, Field{
|
||||
Name: "instance",
|
||||
Value: "host1:234",
|
||||
})
|
||||
// Add other columns
|
||||
for j := range columns {
|
||||
fields = append(fields, Field{
|
||||
Name: columns[j].name,
|
||||
Value: columns[j].values[i],
|
||||
})
|
||||
}
|
||||
timestamp := int64(i) * 1e9
|
||||
lr.MustAdd(tenantID, timestamp, fields)
|
||||
}
|
||||
s.MustAddRows(lr)
|
||||
PutLogRows(lr)
|
||||
}
|
44
lib/logstorage/filter_time.go
Normal file
44
lib/logstorage/filter_time.go
Normal file
|
@ -0,0 +1,44 @@
|
|||
package logstorage
|
||||
|
||||
// filterTime filters by time.
|
||||
//
|
||||
// It is expressed as `_time:(start, end]` in LogsQL.
|
||||
type filterTime struct {
|
||||
// mintimestamp is the minimum timestamp in nanoseconds to find
|
||||
minTimestamp int64
|
||||
|
||||
// maxTimestamp is the maximum timestamp in nanoseconds to find
|
||||
maxTimestamp int64
|
||||
|
||||
// stringRepr is string representation of the filter
|
||||
stringRepr string
|
||||
}
|
||||
|
||||
func (ft *filterTime) String() string {
|
||||
return "_time:" + ft.stringRepr
|
||||
}
|
||||
|
||||
func (ft *filterTime) apply(bs *blockSearch, bm *bitmap) {
|
||||
minTimestamp := ft.minTimestamp
|
||||
maxTimestamp := ft.maxTimestamp
|
||||
|
||||
if minTimestamp > maxTimestamp {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
|
||||
th := bs.bsw.bh.timestampsHeader
|
||||
if minTimestamp > th.maxTimestamp || maxTimestamp < th.minTimestamp {
|
||||
bm.resetBits()
|
||||
return
|
||||
}
|
||||
if minTimestamp <= th.minTimestamp && maxTimestamp >= th.maxTimestamp {
|
||||
return
|
||||
}
|
||||
|
||||
timestamps := bs.getTimestamps()
|
||||
bm.forEachSetBit(func(idx int) bool {
|
||||
ts := timestamps[idx]
|
||||
return ts >= minTimestamp && ts <= maxTimestamp
|
||||
})
|
||||
}
|
132
lib/logstorage/filter_time_test.go
Normal file
132
lib/logstorage/filter_time_test.go
Normal file
|
@ -0,0 +1,132 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
|
||||
)
|
||||
|
||||
func TestFilterTime(t *testing.T) {
|
||||
timestamps := []int64{
|
||||
1,
|
||||
9,
|
||||
123,
|
||||
456,
|
||||
789,
|
||||
}
|
||||
|
||||
// match
|
||||
ft := &filterTime{
|
||||
minTimestamp: -10,
|
||||
maxTimestamp: 1,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{0})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: -10,
|
||||
maxTimestamp: 10,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{0, 1})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 1,
|
||||
maxTimestamp: 1,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{0})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 2,
|
||||
maxTimestamp: 456,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{1, 2, 3})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 2,
|
||||
maxTimestamp: 457,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{1, 2, 3})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 120,
|
||||
maxTimestamp: 788,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{2, 3})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 120,
|
||||
maxTimestamp: 789,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{2, 3, 4})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 120,
|
||||
maxTimestamp: 10000,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{2, 3, 4})
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 789,
|
||||
maxTimestamp: 1000,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, []int{4})
|
||||
|
||||
// mismatch
|
||||
ft = &filterTime{
|
||||
minTimestamp: -1000,
|
||||
maxTimestamp: 0,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, nil)
|
||||
|
||||
ft = &filterTime{
|
||||
minTimestamp: 790,
|
||||
maxTimestamp: 1000,
|
||||
}
|
||||
testFilterMatchForTimestamps(t, timestamps, ft, nil)
|
||||
}
|
||||
|
||||
func testFilterMatchForTimestamps(t *testing.T, timestamps []int64, f filter, expectedRowIdxs []int) {
|
||||
t.Helper()
|
||||
|
||||
// Create the test storage
|
||||
const storagePath = "testFilterMatchForTimestamps"
|
||||
cfg := &StorageConfig{}
|
||||
s := MustOpenStorage(storagePath, cfg)
|
||||
|
||||
// Generate rows
|
||||
getValue := func(rowIdx int) string {
|
||||
return fmt.Sprintf("some value for row %d", rowIdx)
|
||||
}
|
||||
tenantID := TenantID{
|
||||
AccountID: 123,
|
||||
ProjectID: 456,
|
||||
}
|
||||
generateRowsFromTimestamps(s, tenantID, timestamps, getValue)
|
||||
|
||||
expectedResults := make([]string, len(expectedRowIdxs))
|
||||
expectedTimestamps := make([]int64, len(expectedRowIdxs))
|
||||
for i, idx := range expectedRowIdxs {
|
||||
expectedResults[i] = getValue(idx)
|
||||
expectedTimestamps[i] = timestamps[idx]
|
||||
}
|
||||
|
||||
testFilterMatchForStorage(t, s, tenantID, f, "_msg", expectedResults, expectedTimestamps)
|
||||
|
||||
// Close and delete the test storage
|
||||
s.MustClose()
|
||||
fs.MustRemoveAll(storagePath)
|
||||
}
|
||||
|
||||
func generateRowsFromTimestamps(s *Storage, tenantID TenantID, timestamps []int64, getValue func(rowIdx int) string) {
|
||||
lr := GetLogRows(nil, nil)
|
||||
var fields []Field
|
||||
for i, timestamp := range timestamps {
|
||||
fields = append(fields[:0], Field{
|
||||
Name: "_msg",
|
||||
Value: getValue(i),
|
||||
})
|
||||
lr.MustAdd(tenantID, timestamp, fields)
|
||||
}
|
||||
s.MustAddRows(lr)
|
||||
PutLogRows(lr)
|
||||
}
|
128
lib/logstorage/filter_timing_test.go
Normal file
128
lib/logstorage/filter_timing_test.go
Normal file
|
@ -0,0 +1,128 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func BenchmarkMatchAnyCasePrefix(b *testing.B) {
|
||||
b.Run("match-ascii-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "err", []string{"error here", "another error here", "foo bar baz error"}, true)
|
||||
})
|
||||
b.Run("match-ascii-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "err", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, true)
|
||||
})
|
||||
b.Run("match-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "err", []string{"error здесь", "another error здесь", "тест bar baz error"}, true)
|
||||
})
|
||||
b.Run("match-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "err", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, true)
|
||||
})
|
||||
|
||||
b.Run("mismatch-partial-ascii-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "rror", []string{"error here", "another error here", "foo bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-ascii-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "rror", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "rror", []string{"error здесь", "another error здесь", "тест bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "rror", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, false)
|
||||
})
|
||||
|
||||
b.Run("mismatch-full-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "warning", []string{"error here", "another error here", "foo bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "warning", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "warning", []string{"error здесь", "another error здесь", "тест bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePrefix(b, "warning", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, false)
|
||||
})
|
||||
}
|
||||
|
||||
func benchmarkMatchAnyCasePrefix(b *testing.B, phraseLowercase string, a []string, resultExpected bool) {
|
||||
n := 0
|
||||
for _, s := range a {
|
||||
n += len(s)
|
||||
}
|
||||
|
||||
b.ReportAllocs()
|
||||
b.SetBytes(int64(n))
|
||||
b.RunParallel(func(pb *testing.PB) {
|
||||
for pb.Next() {
|
||||
for _, s := range a {
|
||||
result := matchAnyCasePrefix(s, phraseLowercase)
|
||||
if result != resultExpected {
|
||||
panic(fmt.Errorf("unexpected result for matchAnyCasePrefix(%q, %q); got %v; want %v", s, phraseLowercase, result, resultExpected))
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func BenchmarkMatchAnyCasePhrase(b *testing.B) {
|
||||
b.Run("match-ascii-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "error", []string{"error here", "another error here", "foo bar baz error"}, true)
|
||||
})
|
||||
b.Run("match-ascii-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "error", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, true)
|
||||
})
|
||||
b.Run("match-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "error", []string{"error здесь", "another error здесь", "тест bar baz error"}, true)
|
||||
})
|
||||
b.Run("match-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "error", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, true)
|
||||
})
|
||||
|
||||
b.Run("mismatch-partial-ascii-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "rror", []string{"error here", "another error here", "foo bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-ascii-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "rror", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "rror", []string{"error здесь", "another error здесь", "тест bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-partial-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "rror", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, false)
|
||||
})
|
||||
|
||||
b.Run("mismatch-full-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "warning", []string{"error here", "another error here", "foo bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "warning", []string{"Error here", "another eRROr here", "foo BAR Baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-unicode-lowercase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "warning", []string{"error здесь", "another error здесь", "тест bar baz error"}, false)
|
||||
})
|
||||
b.Run("mismatch-full-unicode-mixcase", func(b *testing.B) {
|
||||
benchmarkMatchAnyCasePhrase(b, "warning", []string{"error Здесь", "another Error здесь", "тEст bar baz ErRor"}, false)
|
||||
})
|
||||
}
|
||||
|
||||
func benchmarkMatchAnyCasePhrase(b *testing.B, phraseLowercase string, a []string, resultExpected bool) {
|
||||
n := 0
|
||||
for _, s := range a {
|
||||
n += len(s)
|
||||
}
|
||||
|
||||
b.ReportAllocs()
|
||||
b.SetBytes(int64(n))
|
||||
b.RunParallel(func(pb *testing.PB) {
|
||||
for pb.Next() {
|
||||
for _, s := range a {
|
||||
result := matchAnyCasePhrase(s, phraseLowercase)
|
||||
if result != resultExpected {
|
||||
panic(fmt.Errorf("unexpected result for matchAnyCasePhrase(%q, %q); got %v; want %v", s, phraseLowercase, result, resultExpected))
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
File diff suppressed because it is too large
Load diff
File diff suppressed because it is too large
Load diff
|
@ -52,9 +52,9 @@ type indexdb struct {
|
|||
// streamsCreatedTotal is the number of log streams created since the indexdb intialization.
|
||||
streamsCreatedTotal atomic.Uint64
|
||||
|
||||
// the generation of the streamFilterCache.
|
||||
// the generation of the filterStreamCache.
|
||||
// It is updated each time new item is added to tb.
|
||||
streamFilterCacheGeneration atomic.Uint32
|
||||
filterStreamCacheGeneration atomic.Uint32
|
||||
|
||||
// path is the path to indexdb
|
||||
path string
|
||||
|
@ -483,11 +483,11 @@ func (idb *indexdb) mustRegisterStream(streamID *streamID, streamTagsCanonical [
|
|||
func (idb *indexdb) invalidateStreamFilterCache() {
|
||||
// This function must be fast, since it is called each
|
||||
// time new indexdb entry is added.
|
||||
idb.streamFilterCacheGeneration.Add(1)
|
||||
idb.filterStreamCacheGeneration.Add(1)
|
||||
}
|
||||
|
||||
func (idb *indexdb) marshalStreamFilterCacheKey(dst []byte, tenantIDs []TenantID, sf *StreamFilter) []byte {
|
||||
dst = encoding.MarshalUint32(dst, idb.streamFilterCacheGeneration.Load())
|
||||
dst = encoding.MarshalUint32(dst, idb.filterStreamCacheGeneration.Load())
|
||||
dst = encoding.MarshalBytes(dst, bytesutil.ToUnsafeBytes(idb.partitionName))
|
||||
dst = encoding.MarshalVarUint64(dst, uint64(len(tenantIDs)))
|
||||
for i := range tenantIDs {
|
||||
|
@ -500,7 +500,7 @@ func (idb *indexdb) marshalStreamFilterCacheKey(dst []byte, tenantIDs []TenantID
|
|||
func (idb *indexdb) loadStreamIDsFromCache(tenantIDs []TenantID, sf *StreamFilter) ([]streamID, bool) {
|
||||
bb := bbPool.Get()
|
||||
bb.B = idb.marshalStreamFilterCacheKey(bb.B[:0], tenantIDs, sf)
|
||||
data := idb.s.streamFilterCache.GetBig(nil, bb.B)
|
||||
data := idb.s.filterStreamCache.GetBig(nil, bb.B)
|
||||
bbPool.Put(bb)
|
||||
if len(data) == 0 {
|
||||
// Cache miss
|
||||
|
@ -537,7 +537,7 @@ func (idb *indexdb) storeStreamIDsToCache(tenantIDs []TenantID, sf *StreamFilter
|
|||
// Store marshaled streamIDs to cache.
|
||||
bb := bbPool.Get()
|
||||
bb.B = idb.marshalStreamFilterCacheKey(bb.B[:0], tenantIDs, sf)
|
||||
idb.s.streamFilterCache.SetBig(bb.B, b)
|
||||
idb.s.filterStreamCache.SetBig(bb.B, b)
|
||||
bbPool.Put(bb)
|
||||
}
|
||||
|
||||
|
|
|
@ -48,9 +48,9 @@ func TestStorageSearchStreamIDs(t *testing.T) {
|
|||
}
|
||||
idb.debugFlush()
|
||||
|
||||
f := func(streamFilter string, expectedStreamIDs []streamID) {
|
||||
f := func(filterStream string, expectedStreamIDs []streamID) {
|
||||
t.Helper()
|
||||
sf := mustNewStreamFilter(streamFilter)
|
||||
sf := mustNewStreamFilter(filterStream)
|
||||
if expectedStreamIDs == nil {
|
||||
expectedStreamIDs = []streamID{}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ type LogRows struct {
|
|||
// timestamps holds stimestamps for rows added to LogRows
|
||||
timestamps []int64
|
||||
|
||||
// rows holds fields for rows atted to LogRows.
|
||||
// rows holds fields for rows added to LogRows.
|
||||
rows [][]Field
|
||||
|
||||
// sf is a helper for sorting fields in every added row
|
||||
|
|
|
@ -4,7 +4,6 @@ import (
|
|||
"fmt"
|
||||
"math"
|
||||
"regexp"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
@ -40,12 +39,17 @@ type lexer struct {
|
|||
currentTimestamp int64
|
||||
}
|
||||
|
||||
// newLexer returns new lexer for the given s.
|
||||
//
|
||||
// The lex.token points to the first token in s.
|
||||
func newLexer(s string) *lexer {
|
||||
return &lexer{
|
||||
lex := &lexer{
|
||||
s: s,
|
||||
sOrig: s,
|
||||
currentTimestamp: time.Now().UnixNano(),
|
||||
}
|
||||
lex.nextToken()
|
||||
return lex
|
||||
}
|
||||
|
||||
func (lex *lexer) isEnd() bool {
|
||||
|
@ -187,35 +191,32 @@ func (lex *lexer) nextToken() {
|
|||
// Query represents LogsQL query.
|
||||
type Query struct {
|
||||
f filter
|
||||
|
||||
pipes []pipe
|
||||
}
|
||||
|
||||
// String returns string representation for q.
|
||||
func (q *Query) String() string {
|
||||
return q.f.String()
|
||||
s := q.f.String()
|
||||
|
||||
for _, p := range q.pipes {
|
||||
s += " | " + p.String()
|
||||
}
|
||||
|
||||
return s
|
||||
}
|
||||
|
||||
func (q *Query) getResultColumnNames() []string {
|
||||
m := make(map[string]struct{})
|
||||
q.f.updateReferencedColumnNames(m)
|
||||
func (q *Query) getNeededColumns() ([]string, []string) {
|
||||
neededFields := newFieldsSet()
|
||||
neededFields.add("*")
|
||||
unneededFields := newFieldsSet()
|
||||
|
||||
// Substitute an empty column name with _msg column
|
||||
if _, ok := m[""]; ok {
|
||||
delete(m, "")
|
||||
m["_msg"] = struct{}{}
|
||||
pipes := q.pipes
|
||||
for i := len(pipes) - 1; i >= 0; i-- {
|
||||
pipes[i].updateNeededFields(neededFields, unneededFields)
|
||||
}
|
||||
|
||||
// unconditionally select _time, _stream and _msg columns
|
||||
// TODO: add the ability to filter out these columns
|
||||
m["_time"] = struct{}{}
|
||||
m["_stream"] = struct{}{}
|
||||
m["_msg"] = struct{}{}
|
||||
|
||||
columnNames := make([]string, 0, len(m))
|
||||
for k := range m {
|
||||
columnNames = append(columnNames, k)
|
||||
}
|
||||
sort.Strings(columnNames)
|
||||
return columnNames
|
||||
return neededFields.getAll(), unneededFields.getAll()
|
||||
}
|
||||
|
||||
// ParseQuery parses s.
|
||||
|
@ -224,33 +225,40 @@ func ParseQuery(s string) (*Query, error) {
|
|||
|
||||
f, err := parseFilter(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse filter expression: %w; context: %s", err, lex.context())
|
||||
return nil, fmt.Errorf("%w; context: [%s]", err, lex.context())
|
||||
}
|
||||
if !lex.isEnd() {
|
||||
return nil, fmt.Errorf("unexpected tail: %q", lex.s)
|
||||
}
|
||||
|
||||
q := &Query{
|
||||
f: f,
|
||||
}
|
||||
|
||||
pipes, err := parsePipes(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("%w; context: [%s]", err, lex.context())
|
||||
}
|
||||
q.pipes = pipes
|
||||
|
||||
if !lex.isEnd() {
|
||||
return nil, fmt.Errorf("unexpected unparsed tail; context: [%s]; tail: [%s]", lex.context(), lex.s)
|
||||
}
|
||||
|
||||
return q, nil
|
||||
}
|
||||
|
||||
func parseFilter(lex *lexer) (filter, error) {
|
||||
if !lex.mustNextToken() || lex.isKeyword("|") {
|
||||
if lex.isKeyword("|", "") {
|
||||
return nil, fmt.Errorf("missing query")
|
||||
}
|
||||
af, err := parseOrFilter(lex, "")
|
||||
fo, err := parseFilterOr(lex, "")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return af, nil
|
||||
return fo, nil
|
||||
}
|
||||
|
||||
func parseOrFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterOr(lex *lexer, fieldName string) (filter, error) {
|
||||
var filters []filter
|
||||
for {
|
||||
f, err := parseAndFilter(lex, fieldName)
|
||||
f, err := parseFilterAnd(lex, fieldName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -260,10 +268,10 @@ func parseOrFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if len(filters) == 1 {
|
||||
return filters[0], nil
|
||||
}
|
||||
of := &orFilter{
|
||||
fo := &filterOr{
|
||||
filters: filters,
|
||||
}
|
||||
return of, nil
|
||||
return fo, nil
|
||||
case lex.isKeyword("or"):
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing filter after 'or'")
|
||||
|
@ -272,7 +280,7 @@ func parseOrFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
}
|
||||
}
|
||||
|
||||
func parseAndFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterAnd(lex *lexer, fieldName string) (filter, error) {
|
||||
var filters []filter
|
||||
for {
|
||||
f, err := parseGenericFilter(lex, fieldName)
|
||||
|
@ -285,10 +293,10 @@ func parseAndFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if len(filters) == 1 {
|
||||
return filters[0], nil
|
||||
}
|
||||
af := &andFilter{
|
||||
fa := &filterAnd{
|
||||
filters: filters,
|
||||
}
|
||||
return af, nil
|
||||
return fa, nil
|
||||
case lex.isKeyword("and"):
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing filter after 'and'")
|
||||
|
@ -307,7 +315,7 @@ func parseGenericFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
return parseGenericFilter(lex, fieldName)
|
||||
case lex.isKeyword("*"):
|
||||
lex.nextToken()
|
||||
f := &prefixFilter{
|
||||
f := &filterPrefix{
|
||||
fieldName: fieldName,
|
||||
prefix: "",
|
||||
}
|
||||
|
@ -318,49 +326,49 @@ func parseGenericFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
}
|
||||
return parseParensFilter(lex, fieldName)
|
||||
case lex.isKeyword("not", "!"):
|
||||
return parseNotFilter(lex, fieldName)
|
||||
return parseFilterNot(lex, fieldName)
|
||||
case lex.isKeyword("exact"):
|
||||
return parseExactFilter(lex, fieldName)
|
||||
return parseFilterExact(lex, fieldName)
|
||||
case lex.isKeyword("i"):
|
||||
return parseAnyCaseFilter(lex, fieldName)
|
||||
case lex.isKeyword("in"):
|
||||
return parseInFilter(lex, fieldName)
|
||||
return parseFilterIn(lex, fieldName)
|
||||
case lex.isKeyword("ipv4_range"):
|
||||
return parseIPv4RangeFilter(lex, fieldName)
|
||||
return parseFilterIPv4Range(lex, fieldName)
|
||||
case lex.isKeyword("len_range"):
|
||||
return parseLenRangeFilter(lex, fieldName)
|
||||
return parseFilterLenRange(lex, fieldName)
|
||||
case lex.isKeyword("range"):
|
||||
return parseRangeFilter(lex, fieldName)
|
||||
return parseFilterRange(lex, fieldName)
|
||||
case lex.isKeyword("re"):
|
||||
return parseRegexpFilter(lex, fieldName)
|
||||
return parseFilterRegexp(lex, fieldName)
|
||||
case lex.isKeyword("seq"):
|
||||
return parseSequenceFilter(lex, fieldName)
|
||||
return parseFilterSequence(lex, fieldName)
|
||||
case lex.isKeyword("string_range"):
|
||||
return parseStringRangeFilter(lex, fieldName)
|
||||
return parseFilterStringRange(lex, fieldName)
|
||||
case lex.isKeyword(`"`, "'", "`"):
|
||||
return nil, fmt.Errorf("improperly quoted string")
|
||||
case lex.isKeyword(",", ")", "[", "]"):
|
||||
return nil, fmt.Errorf("unexpected token %q", lex.token)
|
||||
}
|
||||
phrase := getCompoundPhrase(lex, fieldName)
|
||||
phrase := getCompoundPhrase(lex, fieldName != "")
|
||||
return parseFilterForPhrase(lex, phrase, fieldName)
|
||||
}
|
||||
|
||||
func getCompoundPhrase(lex *lexer, fieldName string) string {
|
||||
func getCompoundPhrase(lex *lexer, allowColon bool) string {
|
||||
phrase := lex.token
|
||||
rawPhrase := lex.rawToken
|
||||
lex.nextToken()
|
||||
suffix := getCompoundSuffix(lex, fieldName)
|
||||
suffix := getCompoundSuffix(lex, allowColon)
|
||||
if suffix == "" {
|
||||
return phrase
|
||||
}
|
||||
return rawPhrase + suffix
|
||||
}
|
||||
|
||||
func getCompoundSuffix(lex *lexer, fieldName string) string {
|
||||
func getCompoundSuffix(lex *lexer, allowColon bool) string {
|
||||
s := ""
|
||||
stopTokens := []string{"*", ",", "(", ")", "[", "]", "|", ""}
|
||||
if fieldName == "" {
|
||||
if !allowColon {
|
||||
stopTokens = append(stopTokens, ":")
|
||||
}
|
||||
for !lex.isSkippedSpace && !lex.isKeyword(stopTokens...) {
|
||||
|
@ -409,14 +417,14 @@ func parseFilterForPhrase(lex *lexer, phrase, fieldName string) (filter, error)
|
|||
if lex.isKeyword("*") && !lex.isSkippedSpace {
|
||||
// The phrase is a search prefix in the form `foo*`.
|
||||
lex.nextToken()
|
||||
f := &prefixFilter{
|
||||
f := &filterPrefix{
|
||||
fieldName: fieldName,
|
||||
prefix: phrase,
|
||||
}
|
||||
return f, nil
|
||||
}
|
||||
// The phrase is a search phrase.
|
||||
f := &phraseFilter{
|
||||
f := &filterPhrase{
|
||||
fieldName: fieldName,
|
||||
phrase: phrase,
|
||||
}
|
||||
|
@ -430,9 +438,9 @@ func parseFilterForPhrase(lex *lexer, phrase, fieldName string) (filter, error)
|
|||
}
|
||||
switch fieldName {
|
||||
case "_time":
|
||||
return parseTimeFilterWithOffset(lex)
|
||||
return parseFilterTimeWithOffset(lex)
|
||||
case "_stream":
|
||||
return parseStreamFilter(lex)
|
||||
return parseFilterStream(lex)
|
||||
default:
|
||||
return parseGenericFilter(lex, fieldName)
|
||||
}
|
||||
|
@ -442,7 +450,7 @@ func parseParensFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing filter after '('")
|
||||
}
|
||||
f, err := parseOrFilter(lex, fieldName)
|
||||
f, err := parseFilterOr(lex, fieldName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -453,7 +461,7 @@ func parseParensFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
return f, nil
|
||||
}
|
||||
|
||||
func parseNotFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterNot(lex *lexer, fieldName string) (filter, error) {
|
||||
notKeyword := lex.token
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing filters after '%s'", notKeyword)
|
||||
|
@ -462,26 +470,26 @@ func parseNotFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
nf, ok := f.(*notFilter)
|
||||
fn, ok := f.(*filterNot)
|
||||
if ok {
|
||||
return nf.f, nil
|
||||
return fn.f, nil
|
||||
}
|
||||
nf = ¬Filter{
|
||||
fn = &filterNot{
|
||||
f: f,
|
||||
}
|
||||
return nf, nil
|
||||
return fn, nil
|
||||
}
|
||||
|
||||
func parseAnyCaseFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
return parseFuncArgMaybePrefix(lex, "i", fieldName, func(phrase string, isPrefixFilter bool) (filter, error) {
|
||||
if isPrefixFilter {
|
||||
f := &anyCasePrefixFilter{
|
||||
return parseFuncArgMaybePrefix(lex, "i", fieldName, func(phrase string, isFilterPrefix bool) (filter, error) {
|
||||
if isFilterPrefix {
|
||||
f := &filterAnyCasePrefix{
|
||||
fieldName: fieldName,
|
||||
prefix: phrase,
|
||||
}
|
||||
return f, nil
|
||||
}
|
||||
f := &anyCasePhraseFilter{
|
||||
f := &filterAnyCasePhrase{
|
||||
fieldName: fieldName,
|
||||
phrase: phrase,
|
||||
}
|
||||
|
@ -493,16 +501,16 @@ func parseFuncArgMaybePrefix(lex *lexer, funcName, fieldName string, callback fu
|
|||
phrase := lex.token
|
||||
lex.nextToken()
|
||||
if !lex.isKeyword("(") {
|
||||
phrase += getCompoundSuffix(lex, fieldName)
|
||||
phrase += getCompoundSuffix(lex, fieldName != "")
|
||||
return parseFilterForPhrase(lex, phrase, fieldName)
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing arg for %s()", funcName)
|
||||
}
|
||||
phrase = getCompoundFuncArg(lex)
|
||||
isPrefixFilter := false
|
||||
isFilterPrefix := false
|
||||
if lex.isKeyword("*") && !lex.isSkippedSpace {
|
||||
isPrefixFilter = true
|
||||
isFilterPrefix = true
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing ')' after %s()", funcName)
|
||||
}
|
||||
|
@ -511,48 +519,54 @@ func parseFuncArgMaybePrefix(lex *lexer, funcName, fieldName string, callback fu
|
|||
return nil, fmt.Errorf("unexpected token %q instead of ')' in %s()", lex.token, funcName)
|
||||
}
|
||||
lex.nextToken()
|
||||
return callback(phrase, isPrefixFilter)
|
||||
return callback(phrase, isFilterPrefix)
|
||||
}
|
||||
|
||||
func parseLenRangeFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterLenRange(lex *lexer, fieldName string) (filter, error) {
|
||||
funcName := lex.token
|
||||
return parseFuncArgs(lex, fieldName, func(args []string) (filter, error) {
|
||||
if len(args) != 2 {
|
||||
return nil, fmt.Errorf("unexpected number of args for %s(); got %d; want 2", funcName, len(args))
|
||||
}
|
||||
minLen, err := strconv.ParseUint(args[0], 10, 64)
|
||||
|
||||
minLen, err := parseUint(args[0])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse minLen at %s(): %w", funcName, err)
|
||||
}
|
||||
maxLen, err := strconv.ParseUint(args[1], 10, 64)
|
||||
|
||||
maxLen, err := parseUint(args[1])
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse maxLen at %s(): %w", funcName, err)
|
||||
}
|
||||
rf := &lenRangeFilter{
|
||||
|
||||
stringRepr := "(" + args[0] + ", " + args[1] + ")"
|
||||
fr := &filterLenRange{
|
||||
fieldName: fieldName,
|
||||
minLen: minLen,
|
||||
maxLen: maxLen,
|
||||
|
||||
stringRepr: stringRepr,
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
})
|
||||
}
|
||||
|
||||
func parseStringRangeFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterStringRange(lex *lexer, fieldName string) (filter, error) {
|
||||
funcName := lex.token
|
||||
return parseFuncArgs(lex, fieldName, func(args []string) (filter, error) {
|
||||
if len(args) != 2 {
|
||||
return nil, fmt.Errorf("unexpected number of args for %s(); got %d; want 2", funcName, len(args))
|
||||
}
|
||||
rf := &stringRangeFilter{
|
||||
fr := &filterStringRange{
|
||||
fieldName: fieldName,
|
||||
minValue: args[0],
|
||||
maxValue: args[1],
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
})
|
||||
}
|
||||
|
||||
func parseIPv4RangeFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterIPv4Range(lex *lexer, fieldName string) (filter, error) {
|
||||
funcName := lex.token
|
||||
return parseFuncArgs(lex, fieldName, func(args []string) (filter, error) {
|
||||
if len(args) == 1 {
|
||||
|
@ -560,12 +574,12 @@ func parseIPv4RangeFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse IPv4 address or IPv4 CIDR %q at %s()", args[0], funcName)
|
||||
}
|
||||
rf := &ipv4RangeFilter{
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: fieldName,
|
||||
minValue: minValue,
|
||||
maxValue: maxValue,
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
}
|
||||
if len(args) != 2 {
|
||||
return nil, fmt.Errorf("unexpected number of args for %s(); got %d; want 2", funcName, len(args))
|
||||
|
@ -578,12 +592,12 @@ func parseIPv4RangeFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse upper bound ip %q in %s()", funcName, args[1])
|
||||
}
|
||||
rf := &ipv4RangeFilter{
|
||||
fr := &filterIPv4Range{
|
||||
fieldName: fieldName,
|
||||
minValue: minValue,
|
||||
maxValue: maxValue,
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -607,9 +621,9 @@ func tryParseIPv4CIDR(s string) (uint32, uint32, bool) {
|
|||
return minValue, maxValue, true
|
||||
}
|
||||
|
||||
func parseInFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterIn(lex *lexer, fieldName string) (filter, error) {
|
||||
return parseFuncArgs(lex, fieldName, func(args []string) (filter, error) {
|
||||
f := &inFilter{
|
||||
f := &filterIn{
|
||||
fieldName: fieldName,
|
||||
values: args,
|
||||
}
|
||||
|
@ -617,26 +631,26 @@ func parseInFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
})
|
||||
}
|
||||
|
||||
func parseSequenceFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterSequence(lex *lexer, fieldName string) (filter, error) {
|
||||
return parseFuncArgs(lex, fieldName, func(args []string) (filter, error) {
|
||||
sf := &sequenceFilter{
|
||||
fs := &filterSequence{
|
||||
fieldName: fieldName,
|
||||
phrases: args,
|
||||
}
|
||||
return sf, nil
|
||||
return fs, nil
|
||||
})
|
||||
}
|
||||
|
||||
func parseExactFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
return parseFuncArgMaybePrefix(lex, "exact", fieldName, func(phrase string, isPrefixFilter bool) (filter, error) {
|
||||
if isPrefixFilter {
|
||||
f := &exactPrefixFilter{
|
||||
func parseFilterExact(lex *lexer, fieldName string) (filter, error) {
|
||||
return parseFuncArgMaybePrefix(lex, "exact", fieldName, func(phrase string, isFilterPrefix bool) (filter, error) {
|
||||
if isFilterPrefix {
|
||||
f := &filterExactPrefix{
|
||||
fieldName: fieldName,
|
||||
prefix: phrase,
|
||||
}
|
||||
return f, nil
|
||||
}
|
||||
f := &exactFilter{
|
||||
f := &filterExact{
|
||||
fieldName: fieldName,
|
||||
value: phrase,
|
||||
}
|
||||
|
@ -644,22 +658,22 @@ func parseExactFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
})
|
||||
}
|
||||
|
||||
func parseRegexpFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterRegexp(lex *lexer, fieldName string) (filter, error) {
|
||||
funcName := lex.token
|
||||
return parseFuncArg(lex, fieldName, func(arg string) (filter, error) {
|
||||
re, err := regexp.Compile(arg)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("invalid regexp %q for %s(): %w", arg, funcName, err)
|
||||
}
|
||||
rf := ®expFilter{
|
||||
fr := &filterRegexp{
|
||||
fieldName: fieldName,
|
||||
re: re,
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
})
|
||||
}
|
||||
|
||||
func parseRangeFilter(lex *lexer, fieldName string) (filter, error) {
|
||||
func parseFilterRange(lex *lexer, fieldName string) (filter, error) {
|
||||
funcName := lex.token
|
||||
lex.nextToken()
|
||||
|
||||
|
@ -671,7 +685,7 @@ func parseRangeFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
case lex.isKeyword("["):
|
||||
includeMinValue = true
|
||||
default:
|
||||
phrase := funcName + getCompoundSuffix(lex, fieldName)
|
||||
phrase := funcName + getCompoundSuffix(lex, fieldName != "")
|
||||
return parseFilterForPhrase(lex, phrase, fieldName)
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
|
@ -711,33 +725,40 @@ func parseRangeFilter(lex *lexer, fieldName string) (filter, error) {
|
|||
stringRepr += "["
|
||||
} else {
|
||||
stringRepr += "("
|
||||
minValue = math.Nextafter(minValue, math.Inf(1))
|
||||
minValue = math.Nextafter(minValue, inf)
|
||||
}
|
||||
stringRepr += minValueStr + "," + maxValueStr
|
||||
stringRepr += minValueStr + ", " + maxValueStr
|
||||
if includeMaxValue {
|
||||
stringRepr += "]"
|
||||
} else {
|
||||
stringRepr += ")"
|
||||
maxValue = math.Nextafter(maxValue, math.Inf(-1))
|
||||
maxValue = math.Nextafter(maxValue, -inf)
|
||||
}
|
||||
|
||||
rf := &rangeFilter{
|
||||
fr := &filterRange{
|
||||
fieldName: fieldName,
|
||||
minValue: minValue,
|
||||
maxValue: maxValue,
|
||||
|
||||
stringRepr: stringRepr,
|
||||
}
|
||||
return rf, nil
|
||||
return fr, nil
|
||||
}
|
||||
|
||||
func parseFloat64(lex *lexer) (float64, string, error) {
|
||||
s := getCompoundToken(lex)
|
||||
f, err := strconv.ParseFloat(s, 64)
|
||||
if err != nil {
|
||||
return 0, "", fmt.Errorf("cannot parse %q as float64: %w", lex.token, err)
|
||||
if err == nil {
|
||||
return f, s, nil
|
||||
}
|
||||
return f, s, nil
|
||||
|
||||
// Try parsing s as integer.
|
||||
// This handles 0x..., 0b... and 0... prefixes, alongside '_' delimiters.
|
||||
n, err := parseInt(s)
|
||||
if err == nil {
|
||||
return float64(n), s, nil
|
||||
}
|
||||
return 0, "", fmt.Errorf("cannot parse %q as float64: %w", lex.token, err)
|
||||
}
|
||||
|
||||
func parseFuncArg(lex *lexer, fieldName string, callback func(args string) (filter, error)) (filter, error) {
|
||||
|
@ -754,7 +775,7 @@ func parseFuncArgs(lex *lexer, fieldName string, callback func(args []string) (f
|
|||
funcName := lex.token
|
||||
lex.nextToken()
|
||||
if !lex.isKeyword("(") {
|
||||
phrase := funcName + getCompoundSuffix(lex, fieldName)
|
||||
phrase := funcName + getCompoundSuffix(lex, fieldName != "")
|
||||
return parseFilterForPhrase(lex, phrase, fieldName)
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
|
@ -801,30 +822,30 @@ func startsWithYear(s string) bool {
|
|||
return c == '-' || c == '+' || c == 'Z' || c == 'z'
|
||||
}
|
||||
|
||||
func parseTimeFilterWithOffset(lex *lexer) (*timeFilter, error) {
|
||||
tf, err := parseTimeFilter(lex)
|
||||
func parseFilterTimeWithOffset(lex *lexer) (*filterTime, error) {
|
||||
ft, err := parseFilterTime(lex)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if !lex.isKeyword("offset") {
|
||||
return tf, nil
|
||||
return ft, nil
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing offset for _time filter %s", tf)
|
||||
return nil, fmt.Errorf("missing offset for _time filter %s", ft)
|
||||
}
|
||||
s := getCompoundToken(lex)
|
||||
d, err := promutils.ParseDuration(s)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse offset for _time filter %s: %w", tf, err)
|
||||
d, ok := tryParseDuration(s)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse offset %q for _time filter %s: %w", s, ft, err)
|
||||
}
|
||||
offset := int64(d)
|
||||
tf.minTimestamp -= offset
|
||||
tf.maxTimestamp -= offset
|
||||
tf.stringRepr += " offset " + s
|
||||
return tf, nil
|
||||
ft.minTimestamp -= offset
|
||||
ft.maxTimestamp -= offset
|
||||
ft.stringRepr += " offset " + s
|
||||
return ft, nil
|
||||
}
|
||||
|
||||
func parseTimeFilter(lex *lexer) (*timeFilter, error) {
|
||||
func parseFilterTime(lex *lexer) (*filterTime, error) {
|
||||
startTimeInclude := false
|
||||
switch {
|
||||
case lex.isKeyword("["):
|
||||
|
@ -842,29 +863,29 @@ func parseTimeFilter(lex *lexer) (*timeFilter, error) {
|
|||
}
|
||||
startTime := int64(t * 1e9)
|
||||
endTime := getMatchingEndTime(startTime, s)
|
||||
tf := &timeFilter{
|
||||
ft := &filterTime{
|
||||
minTimestamp: startTime,
|
||||
maxTimestamp: endTime,
|
||||
|
||||
stringRepr: s,
|
||||
}
|
||||
return tf, nil
|
||||
return ft, nil
|
||||
}
|
||||
// Parse _time:duration, which transforms to '_time:(now-duration, now]'
|
||||
d, err := promutils.ParseDuration(s)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse duration in _time filter: %w", err)
|
||||
d, ok := tryParseDuration(s)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse duration %q in _time filter", s)
|
||||
}
|
||||
if d < 0 {
|
||||
d = -d
|
||||
}
|
||||
tf := &timeFilter{
|
||||
ft := &filterTime{
|
||||
minTimestamp: lex.currentTimestamp - int64(d),
|
||||
maxTimestamp: lex.currentTimestamp,
|
||||
|
||||
stringRepr: s,
|
||||
}
|
||||
return tf, nil
|
||||
return ft, nil
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing start time in _time filter")
|
||||
|
@ -915,13 +936,13 @@ func parseTimeFilter(lex *lexer) (*timeFilter, error) {
|
|||
endTime--
|
||||
}
|
||||
|
||||
tf := &timeFilter{
|
||||
ft := &filterTime{
|
||||
minTimestamp: startTime,
|
||||
maxTimestamp: endTime,
|
||||
|
||||
stringRepr: stringRepr,
|
||||
}
|
||||
return tf, nil
|
||||
return ft, nil
|
||||
}
|
||||
|
||||
func getMatchingEndTime(startTime int64, stringRepr string) int64 {
|
||||
|
@ -972,7 +993,7 @@ func stripTimezoneSuffix(s string) string {
|
|||
return s[:len(s)-len(tz)]
|
||||
}
|
||||
|
||||
func parseStreamFilter(lex *lexer) (*streamFilter, error) {
|
||||
func parseFilterStream(lex *lexer) (*filterStream, error) {
|
||||
if !lex.isKeyword("{") {
|
||||
return nil, fmt.Errorf("unexpected token %q instead of '{' in _stream filter", lex.token)
|
||||
}
|
||||
|
@ -989,12 +1010,12 @@ func parseStreamFilter(lex *lexer) (*streamFilter, error) {
|
|||
switch {
|
||||
case lex.isKeyword("}"):
|
||||
lex.nextToken()
|
||||
sf := &streamFilter{
|
||||
fs := &filterStream{
|
||||
f: &StreamFilter{
|
||||
orFilters: filters,
|
||||
},
|
||||
}
|
||||
return sf, nil
|
||||
return fs, nil
|
||||
case lex.isKeyword("or"):
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("incomplete _stream filter after 'or'")
|
||||
|
@ -1010,14 +1031,11 @@ func parseStreamFilter(lex *lexer) (*streamFilter, error) {
|
|||
|
||||
func newStreamFilter(s string) (*StreamFilter, error) {
|
||||
lex := newLexer(s)
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing '{' in _stream filter")
|
||||
}
|
||||
sf, err := parseStreamFilter(lex)
|
||||
fs, err := parseFilterStream(lex)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return sf.f, nil
|
||||
return fs.f, nil
|
||||
}
|
||||
|
||||
func parseAndStreamFilter(lex *lexer) (*andStreamFilter, error) {
|
||||
|
@ -1168,3 +1186,47 @@ var reservedKeywords = func() map[string]struct{} {
|
|||
}
|
||||
return m
|
||||
}()
|
||||
|
||||
func parseUint(s string) (uint64, error) {
|
||||
if strings.EqualFold(s, "inf") || strings.EqualFold(s, "+inf") {
|
||||
return math.MaxUint64, nil
|
||||
}
|
||||
|
||||
n, err := strconv.ParseUint(s, 0, 64)
|
||||
if err == nil {
|
||||
return n, nil
|
||||
}
|
||||
nn, ok := tryParseBytes(s)
|
||||
if !ok {
|
||||
nn, ok = tryParseDuration(s)
|
||||
if !ok {
|
||||
return 0, fmt.Errorf("cannot parse %q as unsigned integer: %w", s, err)
|
||||
}
|
||||
if nn < 0 {
|
||||
return 0, fmt.Errorf("cannot parse negative value %q as unsigned integer", s)
|
||||
}
|
||||
}
|
||||
return uint64(nn), nil
|
||||
}
|
||||
|
||||
func parseInt(s string) (int64, error) {
|
||||
switch {
|
||||
case strings.EqualFold(s, "inf"), strings.EqualFold(s, "+inf"):
|
||||
return math.MaxInt64, nil
|
||||
case strings.EqualFold(s, "-inf"):
|
||||
return math.MinInt64, nil
|
||||
}
|
||||
|
||||
n, err := strconv.ParseInt(s, 0, 64)
|
||||
if err == nil {
|
||||
return n, nil
|
||||
}
|
||||
nn, ok := tryParseBytes(s)
|
||||
if !ok {
|
||||
nn, ok = tryParseDuration(s)
|
||||
if !ok {
|
||||
return 0, fmt.Errorf("cannot parse %q as integer: %w", s, err)
|
||||
}
|
||||
}
|
||||
return nn, nil
|
||||
}
|
||||
|
|
|
@ -3,6 +3,7 @@ package logstorage
|
|||
import (
|
||||
"math"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
@ -12,12 +13,11 @@ func TestLexer(t *testing.T) {
|
|||
t.Helper()
|
||||
lex := newLexer(s)
|
||||
for _, tokenExpected := range tokensExpected {
|
||||
lex.nextToken()
|
||||
if lex.token != tokenExpected {
|
||||
t.Fatalf("unexpected token; got %q; want %q", lex.token, tokenExpected)
|
||||
}
|
||||
lex.nextToken()
|
||||
}
|
||||
lex.nextToken()
|
||||
if lex.token != "" {
|
||||
t.Fatalf("unexpected tail token: %q", lex.token)
|
||||
}
|
||||
|
@ -86,14 +86,14 @@ func TestParseTimeDuration(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
tf, ok := q.f.(*timeFilter)
|
||||
ft, ok := q.f.(*filterTime)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter; got %T; want *timeFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter; got %T; want *filterTime; filter: %s", q.f, q.f)
|
||||
}
|
||||
if tf.stringRepr != s {
|
||||
t.Fatalf("unexpected string represenation for timeFilter; got %q; want %q", tf.stringRepr, s)
|
||||
if ft.stringRepr != s {
|
||||
t.Fatalf("unexpected string represenation for filterTime; got %q; want %q", ft.stringRepr, s)
|
||||
}
|
||||
duration := time.Duration(tf.maxTimestamp - tf.minTimestamp)
|
||||
duration := time.Duration(ft.maxTimestamp - ft.minTimestamp)
|
||||
if duration != durationExpected {
|
||||
t.Fatalf("unexpected duration; got %s; want %s", duration, durationExpected)
|
||||
}
|
||||
|
@ -114,18 +114,18 @@ func TestParseTimeRange(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
tf, ok := q.f.(*timeFilter)
|
||||
ft, ok := q.f.(*filterTime)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter; got %T; want *timeFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter; got %T; want *filterTime; filter: %s", q.f, q.f)
|
||||
}
|
||||
if tf.stringRepr != s {
|
||||
t.Fatalf("unexpected string represenation for timeFilter; got %q; want %q", tf.stringRepr, s)
|
||||
if ft.stringRepr != s {
|
||||
t.Fatalf("unexpected string represenation for filterTime; got %q; want %q", ft.stringRepr, s)
|
||||
}
|
||||
if tf.minTimestamp != minTimestampExpected {
|
||||
t.Fatalf("unexpected minTimestamp; got %s; want %s", timestampToString(tf.minTimestamp), timestampToString(minTimestampExpected))
|
||||
if ft.minTimestamp != minTimestampExpected {
|
||||
t.Fatalf("unexpected minTimestamp; got %s; want %s", timestampToString(ft.minTimestamp), timestampToString(minTimestampExpected))
|
||||
}
|
||||
if tf.maxTimestamp != maxTimestampExpected {
|
||||
t.Fatalf("unexpected maxTimestamp; got %s; want %s", timestampToString(tf.maxTimestamp), timestampToString(maxTimestampExpected))
|
||||
if ft.maxTimestamp != maxTimestampExpected {
|
||||
t.Fatalf("unexpected maxTimestamp; got %s; want %s", timestampToString(ft.maxTimestamp), timestampToString(maxTimestampExpected))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -274,22 +274,22 @@ func TestParseTimeRange(t *testing.T) {
|
|||
f(`[2023-03-01+02:20,2023-04-06T23] offset 30m5s`, minTimestamp, maxTimestamp)
|
||||
}
|
||||
|
||||
func TestParseSequenceFilter(t *testing.T) {
|
||||
func TestParseFilterSequence(t *testing.T) {
|
||||
f := func(s, fieldNameExpected string, phrasesExpected []string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
sf, ok := q.f.(*sequenceFilter)
|
||||
fs, ok := q.f.(*filterSequence)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *sequenceFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterSequence; filter: %s", q.f, q.f)
|
||||
}
|
||||
if sf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", sf.fieldName, fieldNameExpected)
|
||||
if fs.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fs.fieldName, fieldNameExpected)
|
||||
}
|
||||
if !reflect.DeepEqual(sf.phrases, phrasesExpected) {
|
||||
t.Fatalf("unexpected phrases\ngot\n%q\nwant\n%q", sf.phrases, phrasesExpected)
|
||||
if !reflect.DeepEqual(fs.phrases, phrasesExpected) {
|
||||
t.Fatalf("unexpected phrases\ngot\n%q\nwant\n%q", fs.phrases, phrasesExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -299,16 +299,16 @@ func TestParseSequenceFilter(t *testing.T) {
|
|||
f(`seq(foo,bar-baz.aa"bb","c,)d")`, ``, []string{"foo", `bar-baz.aa"bb"`, "c,)d"})
|
||||
}
|
||||
|
||||
func TestParseInFilter(t *testing.T) {
|
||||
func TestParseFilterIn(t *testing.T) {
|
||||
f := func(s, fieldNameExpected string, valuesExpected []string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
f, ok := q.f.(*inFilter)
|
||||
f, ok := q.f.(*filterIn)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *inFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterIn; filter: %s", q.f, q.f)
|
||||
}
|
||||
if f.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", f.fieldName, fieldNameExpected)
|
||||
|
@ -325,25 +325,25 @@ func TestParseInFilter(t *testing.T) {
|
|||
f(`foo-bar:in(foo,bar-baz.aa"bb","c,)d")`, `foo-bar`, []string{"foo", `bar-baz.aa"bb"`, "c,)d"})
|
||||
}
|
||||
|
||||
func TestParseIPv4RangeFilter(t *testing.T) {
|
||||
func TestParseFilterIPv4Range(t *testing.T) {
|
||||
f := func(s, fieldNameExpected string, minValueExpected, maxValueExpected uint32) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
rf, ok := q.f.(*ipv4RangeFilter)
|
||||
fr, ok := q.f.(*filterIPv4Range)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *ipv4RangeFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterIPv4Range; filter: %s", q.f, q.f)
|
||||
}
|
||||
if rf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", rf.fieldName, fieldNameExpected)
|
||||
if fr.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fr.fieldName, fieldNameExpected)
|
||||
}
|
||||
if rf.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %08x; want %08x", rf.minValue, minValueExpected)
|
||||
if fr.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %08x; want %08x", fr.minValue, minValueExpected)
|
||||
}
|
||||
if rf.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %08x; want %08x", rf.maxValue, maxValueExpected)
|
||||
if fr.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %08x; want %08x", fr.maxValue, maxValueExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -356,25 +356,25 @@ func TestParseIPv4RangeFilter(t *testing.T) {
|
|||
f(`ipv4_range(1.2.3.34/0)`, ``, 0, 0xffffffff)
|
||||
}
|
||||
|
||||
func TestParseStringRangeFilter(t *testing.T) {
|
||||
func TestParseFilterStringRange(t *testing.T) {
|
||||
f := func(s, fieldNameExpected, minValueExpected, maxValueExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
rf, ok := q.f.(*stringRangeFilter)
|
||||
fr, ok := q.f.(*filterStringRange)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *stringRangeFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterStringRange; filter: %s", q.f, q.f)
|
||||
}
|
||||
if rf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", rf.fieldName, fieldNameExpected)
|
||||
if fr.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fr.fieldName, fieldNameExpected)
|
||||
}
|
||||
if rf.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %q; want %q", rf.minValue, minValueExpected)
|
||||
if fr.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %q; want %q", fr.minValue, minValueExpected)
|
||||
}
|
||||
if rf.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %q; want %q", rf.maxValue, maxValueExpected)
|
||||
if fr.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %q; want %q", fr.maxValue, maxValueExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -382,18 +382,18 @@ func TestParseStringRangeFilter(t *testing.T) {
|
|||
f(`abc:string_range("foo,bar", "baz) !")`, `abc`, `foo,bar`, `baz) !`)
|
||||
}
|
||||
|
||||
func TestParseRegexpFilter(t *testing.T) {
|
||||
func TestParseFilterRegexp(t *testing.T) {
|
||||
f := func(s, reExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery("re(" + s + ")")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
rf, ok := q.f.(*regexpFilter)
|
||||
fr, ok := q.f.(*filterRegexp)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *regexpFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterRegexp; filter: %s", q.f, q.f)
|
||||
}
|
||||
if reString := rf.re.String(); reString != reExpected {
|
||||
if reString := fr.re.String(); reString != reExpected {
|
||||
t.Fatalf("unexpected regexp; got %q; want %q", reString, reExpected)
|
||||
}
|
||||
}
|
||||
|
@ -404,22 +404,22 @@ func TestParseRegexpFilter(t *testing.T) {
|
|||
f(`"foo(bar|baz),x[y]"`, `foo(bar|baz),x[y]`)
|
||||
}
|
||||
|
||||
func TestParseAnyCasePhraseFilter(t *testing.T) {
|
||||
func TestParseAnyCaseFilterPhrase(t *testing.T) {
|
||||
f := func(s, fieldNameExpected, phraseExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
pf, ok := q.f.(*anyCasePhraseFilter)
|
||||
fp, ok := q.f.(*filterAnyCasePhrase)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *anyCasePhraseFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterAnyCasePhrase; filter: %s", q.f, q.f)
|
||||
}
|
||||
if pf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", pf.fieldName, fieldNameExpected)
|
||||
if fp.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fp.fieldName, fieldNameExpected)
|
||||
}
|
||||
if pf.phrase != phraseExpected {
|
||||
t.Fatalf("unexpected phrase; got %q; want %q", pf.phrase, phraseExpected)
|
||||
if fp.phrase != phraseExpected {
|
||||
t.Fatalf("unexpected phrase; got %q; want %q", fp.phrase, phraseExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -429,22 +429,22 @@ func TestParseAnyCasePhraseFilter(t *testing.T) {
|
|||
f(`"abc-de.fg":i("foo-bar+baz")`, `abc-de.fg`, `foo-bar+baz`)
|
||||
}
|
||||
|
||||
func TestParseAnyCasePrefixFilter(t *testing.T) {
|
||||
func TestParseAnyCaseFilterPrefix(t *testing.T) {
|
||||
f := func(s, fieldNameExpected, prefixExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
pf, ok := q.f.(*anyCasePrefixFilter)
|
||||
fp, ok := q.f.(*filterAnyCasePrefix)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *anyCasePrefixFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterAnyCasePrefix; filter: %s", q.f, q.f)
|
||||
}
|
||||
if pf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", pf.fieldName, fieldNameExpected)
|
||||
if fp.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fp.fieldName, fieldNameExpected)
|
||||
}
|
||||
if pf.prefix != prefixExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", pf.prefix, prefixExpected)
|
||||
if fp.prefix != prefixExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", fp.prefix, prefixExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -456,22 +456,22 @@ func TestParseAnyCasePrefixFilter(t *testing.T) {
|
|||
f(`"abc-de.fg":i("foo-bar*baz *"*)`, `abc-de.fg`, `foo-bar*baz *`)
|
||||
}
|
||||
|
||||
func TestParsePhraseFilter(t *testing.T) {
|
||||
func TestParseFilterPhrase(t *testing.T) {
|
||||
f := func(s, fieldNameExpected, phraseExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
pf, ok := q.f.(*phraseFilter)
|
||||
fp, ok := q.f.(*filterPhrase)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *phraseFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterPhrase; filter: %s", q.f, q.f)
|
||||
}
|
||||
if pf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", pf.fieldName, fieldNameExpected)
|
||||
if fp.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fp.fieldName, fieldNameExpected)
|
||||
}
|
||||
if pf.phrase != phraseExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", pf.phrase, phraseExpected)
|
||||
if fp.phrase != phraseExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", fp.phrase, phraseExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -483,22 +483,22 @@ func TestParsePhraseFilter(t *testing.T) {
|
|||
f(`"foo:bar*,( baz"`, ``, `foo:bar*,( baz`)
|
||||
}
|
||||
|
||||
func TestParsePrefixFilter(t *testing.T) {
|
||||
func TestParseFilterPrefix(t *testing.T) {
|
||||
f := func(s, fieldNameExpected, prefixExpected string) {
|
||||
t.Helper()
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
pf, ok := q.f.(*prefixFilter)
|
||||
fp, ok := q.f.(*filterPrefix)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *prefixFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterPrefix; filter: %s", q.f, q.f)
|
||||
}
|
||||
if pf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", pf.fieldName, fieldNameExpected)
|
||||
if fp.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fp.fieldName, fieldNameExpected)
|
||||
}
|
||||
if pf.prefix != prefixExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", pf.prefix, prefixExpected)
|
||||
if fp.prefix != prefixExpected {
|
||||
t.Fatalf("unexpected prefix; got %q; want %q", fp.prefix, prefixExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -517,18 +517,18 @@ func TestParseRangeFilter(t *testing.T) {
|
|||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
rf, ok := q.f.(*rangeFilter)
|
||||
fr, ok := q.f.(*filterRange)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected filter type; got %T; want *ipv4RangeFilter; filter: %s", q.f, q.f)
|
||||
t.Fatalf("unexpected filter type; got %T; want *filterIPv4Range; filter: %s", q.f, q.f)
|
||||
}
|
||||
if rf.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", rf.fieldName, fieldNameExpected)
|
||||
if fr.fieldName != fieldNameExpected {
|
||||
t.Fatalf("unexpected fieldName; got %q; want %q", fr.fieldName, fieldNameExpected)
|
||||
}
|
||||
if rf.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %v; want %v", rf.minValue, minValueExpected)
|
||||
if fr.minValue != minValueExpected {
|
||||
t.Fatalf("unexpected minValue; got %v; want %v", fr.minValue, minValueExpected)
|
||||
}
|
||||
if rf.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %v; want %v", rf.maxValue, maxValueExpected)
|
||||
if fr.maxValue != maxValueExpected {
|
||||
t.Fatalf("unexpected maxValue; got %v; want %v", fr.maxValue, maxValueExpected)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -537,9 +537,15 @@ func TestParseRangeFilter(t *testing.T) {
|
|||
f(`range:range["-1.234e5", "-2e-5"]`, `range`, -1.234e5, -2e-5)
|
||||
|
||||
f(`_msg:range[1, 2]`, `_msg`, 1, 2)
|
||||
f(`:range(1, 2)`, ``, math.Nextafter(1, math.Inf(1)), math.Nextafter(2, math.Inf(-1)))
|
||||
f(`range[1, 2)`, ``, 1, math.Nextafter(2, math.Inf(-1)))
|
||||
f(`range("1", 2]`, ``, math.Nextafter(1, math.Inf(1)), 2)
|
||||
f(`:range(1, 2)`, ``, math.Nextafter(1, inf), math.Nextafter(2, -inf))
|
||||
f(`range[1, 2)`, ``, 1, math.Nextafter(2, -inf))
|
||||
f(`range("1", 2]`, ``, math.Nextafter(1, inf), 2)
|
||||
|
||||
f(`response_size:range[1KB, 10MiB]`, `response_size`, 1_000, 10*(1<<20))
|
||||
f(`response_size:range[1G, 10Ti]`, `response_size`, 1_000_000_000, 10*(1<<40))
|
||||
f(`response_size:range[10, inf]`, `response_size`, 10, inf)
|
||||
|
||||
f(`duration:range[100ns, 1y2w2.5m3s5ms]`, `duration`, 100, 1*nsecsPerYear+2*nsecsPerWeek+2.5*nsecsPerMinute+3*nsecsPerSecond+5*nsecsPerMillisecond)
|
||||
}
|
||||
|
||||
func TestParseQuerySuccess(t *testing.T) {
|
||||
|
@ -743,16 +749,25 @@ func TestParseQuerySuccess(t *testing.T) {
|
|||
f(`ipv4_range(1.2.3.4,)`, `ipv4_range(1.2.3.4, 1.2.3.4)`)
|
||||
|
||||
// len_range filter
|
||||
f(`len_range(10, 20)`, `len_range(10,20)`)
|
||||
f(`foo:len_range("10", 20, )`, `foo:len_range(10,20)`)
|
||||
f(`len_range(10, 20)`, `len_range(10, 20)`)
|
||||
f(`foo:len_range("10", 20, )`, `foo:len_range(10, 20)`)
|
||||
f(`len_RANGe(10, inf)`, `len_range(10, inf)`)
|
||||
f(`len_range(10, +InF)`, `len_range(10, +InF)`)
|
||||
f(`len_range(10, 1_000_000)`, `len_range(10, 1_000_000)`)
|
||||
f(`len_range(0x10,0b100101)`, `len_range(0x10, 0b100101)`)
|
||||
f(`len_range(1.5KB, 22MB100KB)`, `len_range(1.5KB, 22MB100KB)`)
|
||||
|
||||
// range filter
|
||||
f(`range(1.234, 5656.43454)`, `range(1.234,5656.43454)`)
|
||||
f(`foo:range(-2343.344, 2343.4343)`, `foo:range(-2343.344,2343.4343)`)
|
||||
f(`range(-1.234e-5 , 2.34E+3)`, `range(-1.234e-5,2.34E+3)`)
|
||||
f(`range[123, 456)`, `range[123,456)`)
|
||||
f(`range(123, 445]`, `range(123,445]`)
|
||||
f(`range("1.234e-4", -23)`, `range(1.234e-4,-23)`)
|
||||
f(`range(1.234, 5656.43454)`, `range(1.234, 5656.43454)`)
|
||||
f(`foo:range(-2343.344, 2343.4343)`, `foo:range(-2343.344, 2343.4343)`)
|
||||
f(`range(-1.234e-5 , 2.34E+3)`, `range(-1.234e-5, 2.34E+3)`)
|
||||
f(`range[123, 456)`, `range[123, 456)`)
|
||||
f(`range(123, 445]`, `range(123, 445]`)
|
||||
f(`range("1.234e-4", -23)`, `range(1.234e-4, -23)`)
|
||||
f(`range(1_000, 0o7532)`, `range(1_000, 0o7532)`)
|
||||
f(`range(0x1ff, inf)`, `range(0x1ff, inf)`)
|
||||
f(`range(-INF,+inF)`, `range(-INF, +inF)`)
|
||||
f(`range(1.5K, 22.5GiB)`, `range(1.5K, 22.5GiB)`)
|
||||
|
||||
// re filter
|
||||
f("re('foo|ba(r.+)')", `re("foo|ba(r.+)")`)
|
||||
|
@ -803,6 +818,154 @@ func TestParseQuerySuccess(t *testing.T) {
|
|||
and (_stream:{job="a"} or _stream:{instance!="b"})
|
||||
and (err* or ip:(ipv4_range(1.2.3.0, 1.2.3.255) and not 1.2.3.4))`,
|
||||
`(_time:(2023-04-20,now] or _time:[-10m,-1m)) (_stream:{job="a"} or _stream:{instance!="b"}) (err* or ip:ipv4_range(1.2.3.0, 1.2.3.255) !ip:1.2.3.4)`)
|
||||
|
||||
// fields pipe
|
||||
f(`foo|fields *`, `foo | fields *`)
|
||||
f(`foo | fields bar`, `foo | fields bar`)
|
||||
f(`foo|FIELDS bar,Baz , "a,b|c"`, `foo | fields bar, Baz, "a,b|c"`)
|
||||
f(`foo | Fields x.y, "abc:z/a", _b$c`, `foo | fields x.y, "abc:z/a", "_b$c"`)
|
||||
f(`foo | fields "", a`, `foo | fields _msg, a`)
|
||||
|
||||
// multiple fields pipes
|
||||
f(`foo | fields bar | fields baz, abc`, `foo | fields bar | fields baz, abc`)
|
||||
|
||||
// copy and cp pipe
|
||||
f(`* | copy foo as bar`, `* | copy foo as bar`)
|
||||
f(`* | cp foo bar`, `* | copy foo as bar`)
|
||||
f(`* | COPY foo as bar, x y | Copy a as b`, `* | copy foo as bar, x as y | copy a as b`)
|
||||
|
||||
// rename and mv pipe
|
||||
f(`* | rename foo as bar`, `* | rename foo as bar`)
|
||||
f(`* | mv foo bar`, `* | rename foo as bar`)
|
||||
f(`* | RENAME foo AS bar, x y | Rename a as b`, `* | rename foo as bar, x as y | rename a as b`)
|
||||
|
||||
// delete, del and rm pipe
|
||||
f(`* | delete foo`, `* | delete foo`)
|
||||
f(`* | del foo`, `* | delete foo`)
|
||||
f(`* | rm foo`, `* | delete foo`)
|
||||
f(`* | DELETE foo, bar`, `* | delete foo, bar`)
|
||||
|
||||
// limit and head pipe
|
||||
f(`foo | limit 10`, `foo | limit 10`)
|
||||
f(`foo | head 10`, `foo | limit 10`)
|
||||
f(`foo | HEAD 1_123_432`, `foo | limit 1123432`)
|
||||
f(`foo | head 10K`, `foo | limit 10000`)
|
||||
|
||||
// multiple limit pipes
|
||||
f(`foo | limit 100 | limit 10 | limit 234`, `foo | limit 100 | limit 10 | limit 234`)
|
||||
|
||||
// offset and skip pipe
|
||||
f(`foo | skip 10`, `foo | offset 10`)
|
||||
f(`foo | offset 10`, `foo | offset 10`)
|
||||
f(`foo | skip 12_345M`, `foo | offset 12345000000`)
|
||||
|
||||
// multiple offset pipes
|
||||
f(`foo | offset 10 | offset 100`, `foo | offset 10 | offset 100`)
|
||||
|
||||
// stats pipe count
|
||||
f(`* | STATS bY (foo, b.a/r, "b az",) count(*) XYz`, `* | stats by (foo, "b.a/r", "b az") count(*) as XYz`)
|
||||
f(`* | stats by() COUNT(x, 'a).b,c|d',) as qwert`, `* | stats count(x, "a).b,c|d") as qwert`)
|
||||
f(`* | stats count() x`, `* | stats count(*) as x`)
|
||||
f(`* | stats count(*) x`, `* | stats count(*) as x`)
|
||||
f(`* | stats count(foo,*,bar) x`, `* | stats count(*) as x`)
|
||||
f(`* | stats count('') foo`, `* | stats count(_msg) as foo`)
|
||||
f(`* | stats count(foo) ''`, `* | stats count(foo) as _msg`)
|
||||
|
||||
// stats pipe count_empty
|
||||
f(`* | stats count_empty() x`, `* | stats count_empty(*) as x`)
|
||||
f(`* | stats by (x, y) count_empty(a,b,c) x`, `* | stats by (x, y) count_empty(a, b, c) as x`)
|
||||
|
||||
// stats pipe sum
|
||||
f(`* | stats Sum(foo) bar`, `* | stats sum(foo) as bar`)
|
||||
f(`* | stats BY(x, y, ) SUM(foo,bar,) bar`, `* | stats by (x, y) sum(foo, bar) as bar`)
|
||||
f(`* | stats sum() x`, `* | stats sum(*) as x`)
|
||||
f(`* | stats sum(*) x`, `* | stats sum(*) as x`)
|
||||
f(`* | stats sum(foo,*,bar) x`, `* | stats sum(*) as x`)
|
||||
|
||||
// stats pipe max
|
||||
f(`* | stats Max(foo) bar`, `* | stats max(foo) as bar`)
|
||||
f(`* | stats BY(x, y, ) MAX(foo,bar,) bar`, `* | stats by (x, y) max(foo, bar) as bar`)
|
||||
f(`* | stats max() x`, `* | stats max(*) as x`)
|
||||
f(`* | stats max(*) x`, `* | stats max(*) as x`)
|
||||
f(`* | stats max(foo,*,bar) x`, `* | stats max(*) as x`)
|
||||
|
||||
// stats pipe min
|
||||
f(`* | stats Min(foo) bar`, `* | stats min(foo) as bar`)
|
||||
f(`* | stats BY(x, y, ) MIN(foo,bar,) bar`, `* | stats by (x, y) min(foo, bar) as bar`)
|
||||
f(`* | stats min() x`, `* | stats min(*) as x`)
|
||||
f(`* | stats min(*) x`, `* | stats min(*) as x`)
|
||||
f(`* | stats min(foo,*,bar) x`, `* | stats min(*) as x`)
|
||||
|
||||
// stats pipe avg
|
||||
f(`* | stats Avg(foo) bar`, `* | stats avg(foo) as bar`)
|
||||
f(`* | stats BY(x, y, ) AVG(foo,bar,) bar`, `* | stats by (x, y) avg(foo, bar) as bar`)
|
||||
f(`* | stats avg() x`, `* | stats avg(*) as x`)
|
||||
f(`* | stats avg(*) x`, `* | stats avg(*) as x`)
|
||||
f(`* | stats avg(foo,*,bar) x`, `* | stats avg(*) as x`)
|
||||
|
||||
// stats pipe count_uniq
|
||||
f(`* | stats count_uniq(foo) bar`, `* | stats count_uniq(foo) as bar`)
|
||||
f(`* | stats by(x, y) count_uniq(foo,bar) LiMit 10 As baz`, `* | stats by (x, y) count_uniq(foo, bar) limit 10 as baz`)
|
||||
f(`* | stats by(x) count_uniq(*) z`, `* | stats by (x) count_uniq(*) as z`)
|
||||
f(`* | stats by(x) count_uniq() z`, `* | stats by (x) count_uniq(*) as z`)
|
||||
f(`* | stats by(x) count_uniq(a,*,b) z`, `* | stats by (x) count_uniq(*) as z`)
|
||||
|
||||
// stats pipe uniq_values
|
||||
f(`* | stats uniq_values(foo) bar`, `* | stats uniq_values(foo) as bar`)
|
||||
f(`* | stats uniq_values(foo) limit 10 bar`, `* | stats uniq_values(foo) limit 10 as bar`)
|
||||
f(`* | stats by(x, y) uniq_values(foo, bar) as baz`, `* | stats by (x, y) uniq_values(foo, bar) as baz`)
|
||||
f(`* | stats by(x) uniq_values(*) y`, `* | stats by (x) uniq_values(*) as y`)
|
||||
f(`* | stats by(x) uniq_values() limit 1_000 AS y`, `* | stats by (x) uniq_values(*) limit 1000 as y`)
|
||||
f(`* | stats by(x) uniq_values(a,*,b) y`, `* | stats by (x) uniq_values(*) as y`)
|
||||
|
||||
// stats pipe values
|
||||
f(`* | stats values(foo) bar`, `* | stats values(foo) as bar`)
|
||||
f(`* | stats values(foo) limit 10 bar`, `* | stats values(foo) limit 10 as bar`)
|
||||
f(`* | stats by(x, y) values(foo, bar) as baz`, `* | stats by (x, y) values(foo, bar) as baz`)
|
||||
f(`* | stats by(x) values(*) y`, `* | stats by (x) values(*) as y`)
|
||||
f(`* | stats by(x) values() limit 1_000 AS y`, `* | stats by (x) values(*) limit 1000 as y`)
|
||||
f(`* | stats by(x) values(a,*,b) y`, `* | stats by (x) values(*) as y`)
|
||||
|
||||
// stats pipe multiple funcs
|
||||
f(`* | stats count() "foo.bar:baz", count_uniq(a) bar`, `* | stats count(*) as "foo.bar:baz", count_uniq(a) as bar`)
|
||||
f(`* | stats by (x, y) count(*) foo, count_uniq(a,b) bar`, `* | stats by (x, y) count(*) as foo, count_uniq(a, b) as bar`)
|
||||
|
||||
// stats pipe with grouping buckets
|
||||
f(`* | stats by(_time:1d, response_size:1_000KiB, request_duration:5s, foo) count() as foo`, `* | stats by (_time:1d, response_size:1_000KiB, request_duration:5s, foo) count(*) as foo`)
|
||||
f(`*|stats by(client_ip:/24, server_ip:/16) count() foo`, `* | stats by (client_ip:/24, server_ip:/16) count(*) as foo`)
|
||||
f(`* | stats by(_time:1d offset 2h) count() as foo`, `* | stats by (_time:1d offset 2h) count(*) as foo`)
|
||||
f(`* | stats by(_time:1d offset -2.5h5m) count() as foo`, `* | stats by (_time:1d offset -2.5h5m) count(*) as foo`)
|
||||
f(`* | stats by (_time:nanosecond) count() foo`, `* | stats by (_time:nanosecond) count(*) as foo`)
|
||||
f(`* | stats by (_time:microsecond) count() foo`, `* | stats by (_time:microsecond) count(*) as foo`)
|
||||
f(`* | stats by (_time:millisecond) count() foo`, `* | stats by (_time:millisecond) count(*) as foo`)
|
||||
f(`* | stats by (_time:second) count() foo`, `* | stats by (_time:second) count(*) as foo`)
|
||||
f(`* | stats by (_time:minute) count() foo`, `* | stats by (_time:minute) count(*) as foo`)
|
||||
f(`* | stats by (_time:hour) count() foo`, `* | stats by (_time:hour) count(*) as foo`)
|
||||
f(`* | stats by (_time:day) count() foo`, `* | stats by (_time:day) count(*) as foo`)
|
||||
f(`* | stats by (_time:week) count() foo`, `* | stats by (_time:week) count(*) as foo`)
|
||||
f(`* | stats by (_time:month) count() foo`, `* | stats by (_time:month) count(*) as foo`)
|
||||
f(`* | stats by (_time:year offset 6.5h) count() foo`, `* | stats by (_time:year offset 6.5h) count(*) as foo`)
|
||||
|
||||
// sort pipe
|
||||
f(`* | sort`, `* | sort`)
|
||||
f(`* | sort desc`, `* | sort desc`)
|
||||
f(`* | sort by()`, `* | sort`)
|
||||
f(`* | sort bY (foo)`, `* | sort by (foo)`)
|
||||
f(`* | sORt bY (_time, _stream DEsc, host)`, `* | sort by (_time, _stream desc, host)`)
|
||||
f(`* | sort bY (foo desc, bar,) desc`, `* | sort by (foo desc, bar) desc`)
|
||||
|
||||
// uniq pipe
|
||||
f(`* | uniq`, `* | uniq`)
|
||||
f(`* | uniq by()`, `* | uniq`)
|
||||
f(`* | uniq by(*)`, `* | uniq`)
|
||||
f(`* | uniq by(foo,*,bar)`, `* | uniq`)
|
||||
f(`* | uniq by(f1,f2)`, `* | uniq by (f1, f2)`)
|
||||
f(`* | uniq by(f1,f2) limit 10`, `* | uniq by (f1, f2) limit 10`)
|
||||
f(`* | uniq limit 10`, `* | uniq limit 10`)
|
||||
|
||||
// multiple different pipes
|
||||
f(`* | fields foo, bar | limit 100 | stats by(foo,bar) count(baz) as qwert`, `* | fields foo, bar | limit 100 | stats by (foo, bar) count(baz) as qwert`)
|
||||
f(`* | skip 100 | head 20 | skip 10`, `* | offset 100 | limit 20 | offset 10`)
|
||||
}
|
||||
|
||||
func TestParseQueryFailure(t *testing.T) {
|
||||
|
@ -998,4 +1161,280 @@ func TestParseQueryFailure(t *testing.T) {
|
|||
f(`string_range(foo, bar`)
|
||||
f(`string_range(foo)`)
|
||||
f(`string_range(foo, bar, baz)`)
|
||||
|
||||
// missing filter
|
||||
f(`| fields *`)
|
||||
|
||||
// missing pipe keyword
|
||||
f(`foo |`)
|
||||
|
||||
// unknown pipe keyword
|
||||
f(`foo | bar`)
|
||||
f(`foo | fields bar | baz`)
|
||||
|
||||
// missing field in fields pipe
|
||||
f(`foo | fields`)
|
||||
f(`foo | fields ,`)
|
||||
f(`foo | fields bar,`)
|
||||
f(`foo | fields bar,,`)
|
||||
|
||||
// invalid copy and cp pipe
|
||||
f(`foo | copy`)
|
||||
f(`foo | cp`)
|
||||
f(`foo | copy foo`)
|
||||
f(`foo | copy foo,`)
|
||||
f(`foo | copy foo,,`)
|
||||
|
||||
// invalid rename and mv pipe
|
||||
f(`foo | rename`)
|
||||
f(`foo | mv`)
|
||||
f(`foo | rename foo`)
|
||||
f(`foo | rename foo,`)
|
||||
f(`foo | rename foo,,`)
|
||||
|
||||
// invalid delete pipe
|
||||
f(`foo | delete`)
|
||||
f(`foo | del`)
|
||||
f(`foo | rm`)
|
||||
f(`foo | delete foo,`)
|
||||
f(`foo | delete foo,,`)
|
||||
|
||||
// missing limit and head pipe value
|
||||
f(`foo | limit`)
|
||||
f(`foo | head`)
|
||||
|
||||
// invalid limit pipe value
|
||||
f(`foo | limit bar`)
|
||||
f(`foo | limit -123`)
|
||||
|
||||
// missing offset and skip pipe value
|
||||
f(`foo | offset`)
|
||||
f(`foo | skip`)
|
||||
|
||||
// invalid offset pipe value
|
||||
f(`foo | offset bar`)
|
||||
f(`foo | offset -10`)
|
||||
|
||||
// missing stats
|
||||
f(`foo | stats`)
|
||||
|
||||
// invalid stats
|
||||
f(`foo | stats bar`)
|
||||
|
||||
// invalid stats count
|
||||
f(`foo | stats count`)
|
||||
f(`foo | stats count(`)
|
||||
f(`foo | stats count bar`)
|
||||
f(`foo | stats count(bar`)
|
||||
f(`foo | stats count(bar)`)
|
||||
f(`foo | stats count() as`)
|
||||
f(`foo | stats count() as |`)
|
||||
|
||||
// invalid stats count_empty
|
||||
f(`foo | stats count_empty`)
|
||||
f(`foo | stats count_empty() as`)
|
||||
f(`foo | stats count_empty() as |`)
|
||||
|
||||
// invalid stats sum
|
||||
f(`foo | stats sum`)
|
||||
f(`foo | stats sum()`)
|
||||
|
||||
// invalid stats max
|
||||
f(`foo | stats max`)
|
||||
f(`foo | stats max()`)
|
||||
|
||||
// invalid stats min
|
||||
f(`foo | stats min`)
|
||||
f(`foo | stats min()`)
|
||||
|
||||
// invalid stats avg
|
||||
f(`foo | stats avg`)
|
||||
f(`foo | stats avg()`)
|
||||
|
||||
// invalid stats count_uniq
|
||||
f(`foo | stats count_uniq`)
|
||||
f(`foo | stats count_uniq()`)
|
||||
f(`foo | stats count_uniq() limit`)
|
||||
f(`foo | stats count_uniq() limit foo`)
|
||||
f(`foo | stats count_uniq() limit 0.5`)
|
||||
f(`foo | stats count_uniq() limit -1`)
|
||||
|
||||
// invalid stats uniq_values
|
||||
f(`foo | stats uniq_values`)
|
||||
f(`foo | stats uniq_values()`)
|
||||
f(`foo | stats uniq_values() limit`)
|
||||
f(`foo | stats uniq_values(a) limit foo`)
|
||||
f(`foo | stats uniq_values(a) limit 0.5`)
|
||||
f(`foo | stats uniq_values(a) limit -1`)
|
||||
|
||||
// invalid stats values
|
||||
f(`foo | stats values`)
|
||||
f(`foo | stats values()`)
|
||||
f(`foo | stats values() limit`)
|
||||
f(`foo | stats values(a) limit foo`)
|
||||
f(`foo | stats values(a) limit 0.5`)
|
||||
f(`foo | stats values(a) limit -1`)
|
||||
|
||||
// invalid stats grouping fields
|
||||
f(`foo | stats by(foo:bar) count() baz`)
|
||||
f(`foo | stats by(foo:/bar) count() baz`)
|
||||
f(`foo | stats by(foo:-1h) count() baz`)
|
||||
f(`foo | stats by (foo:1h offset) count() baz`)
|
||||
f(`foo | stats by (foo:1h offset bar) count() baz`)
|
||||
|
||||
// invalid stats by clause
|
||||
f(`foo | stats by`)
|
||||
f(`foo | stats by bar`)
|
||||
f(`foo | stats by(`)
|
||||
f(`foo | stats by(bar`)
|
||||
f(`foo | stats by(bar,`)
|
||||
f(`foo | stats by(bar)`)
|
||||
|
||||
// invalid sort pipe
|
||||
f(`foo | sort bar`)
|
||||
f(`foo | sort by`)
|
||||
f(`foo | sort by(`)
|
||||
f(`foo | sort by(baz`)
|
||||
f(`foo | sort by(baz,`)
|
||||
f(`foo | sort by(bar) foo`)
|
||||
|
||||
// invalid uniq pipe
|
||||
f(`foo | uniq bar`)
|
||||
f(`foo | uniq limit`)
|
||||
f(`foo | uniq by(`)
|
||||
f(`foo | uniq by(a`)
|
||||
f(`foo | uniq by(a,`)
|
||||
f(`foo | uniq by(a) bar`)
|
||||
f(`foo | uniq by(a) limit -10`)
|
||||
f(`foo | uniq by(a) limit foo`)
|
||||
}
|
||||
|
||||
func TestQueryGetNeededColumns(t *testing.T) {
|
||||
f := func(s, neededColumnsExpected, unneededColumnsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
q, err := ParseQuery(s)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse query %s: %s", s, err)
|
||||
}
|
||||
|
||||
needed, unneeded := q.getNeededColumns()
|
||||
neededColumns := strings.Join(needed, ",")
|
||||
unneededColumns := strings.Join(unneeded, ",")
|
||||
|
||||
if neededColumns != neededColumnsExpected {
|
||||
t.Fatalf("unexpected neededColumns; got %q; want %q", neededColumns, neededColumnsExpected)
|
||||
}
|
||||
if unneededColumns != unneededColumnsExpected {
|
||||
t.Fatalf("unexpected unneededColumns; got %q; want %q", unneededColumns, unneededColumnsExpected)
|
||||
}
|
||||
}
|
||||
|
||||
f(`*`, `*`, ``)
|
||||
f(`foo bar`, `*`, ``)
|
||||
f(`foo:bar _time:5m baz`, `*`, ``)
|
||||
|
||||
f(`* | fields *`, `*`, ``)
|
||||
f(`* | fields * | offset 10`, `*`, ``)
|
||||
f(`* | fields * | offset 10 | limit 20`, `*`, ``)
|
||||
f(`* | fields foo`, `foo`, ``)
|
||||
f(`* | fields foo, bar`, `bar,foo`, ``)
|
||||
f(`* | fields foo, bar | fields baz, bar`, `bar`, ``)
|
||||
f(`* | fields foo, bar | fields baz, a`, ``, ``)
|
||||
f(`* | fields f1, f2 | rm f3, f4`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | rm f2, f3`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | rm f1, f2, f3`, ``, ``)
|
||||
f(`* | fields f1, f2 | cp f1 f2, f3 f4`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | cp f1 f3, f4 f5`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | cp f2 f3, f4 f5`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | cp f2 f3, f4 f1`, `f2`, ``)
|
||||
f(`* | fields f1, f2 | mv f1 f2, f3 f4`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | mv f1 f3, f4 f5`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | mv f2 f3, f4 f5`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | mv f2 f3, f4 f1`, `f2`, ``)
|
||||
f(`* | fields f1, f2 | stats count() r1`, ``, ``)
|
||||
f(`* | fields f1, f2 | stats count_uniq() r1`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | stats count(f1) r1`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | stats count(f1,f2,f3) r1`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | stats by(b1) count() r1`, ``, ``)
|
||||
f(`* | fields f1, f2 | stats by(b1,f1) count() r1`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | stats by(b1,f1) count(f1) r1`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | stats by(b1,f1) count(f1,f2,f3) r1`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f3)`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f1,f3)`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f3) | stats count() r1`, ``, ``)
|
||||
f(`* | fields f1, f2 | sort by(f1) | stats count() r1`, `f1`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f1) | stats count(f2,f3) r1`, `f1,f2`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f3) | fields f2`, `f2`, ``)
|
||||
f(`* | fields f1, f2 | sort by(f1,f3) | fields f2`, `f1,f2`, ``)
|
||||
|
||||
f(`* | cp foo bar`, `*`, `bar`)
|
||||
f(`* | cp foo bar, baz a`, `*`, `a,bar`)
|
||||
f(`* | cp foo bar, baz a | fields foo,a,b`, `b,baz,foo`, ``)
|
||||
f(`* | cp foo bar, baz a | fields bar,a,b`, `b,baz,foo`, ``)
|
||||
f(`* | cp foo bar, baz a | fields baz,a,b`, `b,baz`, ``)
|
||||
f(`* | cp foo bar | fields bar,a`, `a,foo`, ``)
|
||||
f(`* | cp foo bar | fields baz,a`, `a,baz`, ``)
|
||||
f(`* | cp foo bar | fields foo,a`, `a,foo`, ``)
|
||||
f(`* | cp f1 f2 | rm f1`, `*`, `f2`)
|
||||
f(`* | cp f1 f2 | rm f2`, `*`, `f2`)
|
||||
f(`* | cp f1 f2 | rm f3`, `*`, `f2,f3`)
|
||||
|
||||
f(`* | mv foo bar`, `*`, `bar`)
|
||||
f(`* | mv foo bar, baz a`, `*`, `a,bar`)
|
||||
f(`* | mv foo bar, baz a | fields foo,a,b`, `b,baz`, ``)
|
||||
f(`* | mv foo bar, baz a | fields bar,a,b`, `b,baz,foo`, ``)
|
||||
f(`* | mv foo bar, baz a | fields baz,a,b`, `b,baz`, ``)
|
||||
f(`* | mv foo bar, baz a | fields baz,foo,b`, `b`, ``)
|
||||
f(`* | mv foo bar | fields bar,a`, `a,foo`, ``)
|
||||
f(`* | mv foo bar | fields baz,a`, `a,baz`, ``)
|
||||
f(`* | mv foo bar | fields foo,a`, `a`, ``)
|
||||
f(`* | mv f1 f2 | rm f1`, `*`, `f2`)
|
||||
f(`* | mv f1 f2 | rm f2,f3`, `*`, `f1,f2,f3`)
|
||||
f(`* | mv f1 f2 | rm f3`, `*`, `f2,f3`)
|
||||
|
||||
f(`* | sort by (f1)`, `*`, ``)
|
||||
f(`* | sort by (f1) | fields f2`, `f1,f2`, ``)
|
||||
f(`* | sort by (f1) | fields *`, `*`, ``)
|
||||
f(`* | sort by (f1) | sort by (f2,f3 desc) desc`, `*`, ``)
|
||||
f(`* | sort by (f1) | sort by (f2,f3 desc) desc | fields f4`, `f1,f2,f3,f4`, ``)
|
||||
f(`* | sort by (f1) | sort by (f2,f3 desc) desc | fields f4 | rm f1,f2,f5`, `f1,f2,f3,f4`, ``)
|
||||
|
||||
f(`* | stats by(f1) count(f2) r1, count(f3,f4) r2`, `f1,f2,f3,f4`, ``)
|
||||
f(`* | stats by(f1) count(f2) r1, count(f3,f4) r2 | fields f5,f6`, ``, ``)
|
||||
f(`* | stats by(f1) count(f2) r1, count(f3,f4) r2 | fields f1,f5`, `f1`, ``)
|
||||
f(`* | stats by(f1) count(f2) r1, count(f3,f4) r2 | fields r1`, `f1,f2`, ``)
|
||||
f(`* | stats by(f1) count(f2) r1, count(f3,f4) r2 | fields r2,r3`, `f1,f3,f4`, ``)
|
||||
f(`_time:5m | stats by(_time:day) count() r1 | stats values(_time) r2`, `_time`, ``)
|
||||
f(`* | stats count(f1) r1 | stats count() r1`, ``, ``)
|
||||
f(`* | stats count(f1) r1 | stats count() r2`, ``, ``)
|
||||
f(`* | stats count(f1) r1 | stats count(r1) r2`, `f1`, ``)
|
||||
f(`* | stats count(f1) r1 | stats count(f1) r2`, ``, ``)
|
||||
f(`* | stats count(f1) r1 | stats count(f1,r1) r1`, `f1`, ``)
|
||||
f(`* | stats count(f1,f2) r1 | stats count(f2) r1, count(r1) r2`, `f1,f2`, ``)
|
||||
f(`* | stats count(f1,f2) r1 | stats count(f2) r1, count(r1) r2 | fields r1`, ``, ``)
|
||||
f(`* | stats count(f1,f2) r1 | stats count(f2) r1, count(r1) r2 | fields r2`, `f1,f2`, ``)
|
||||
f(`* | stats by(f3,f4) count(f1,f2) r1 | stats count(f2) r1, count(r1) r2 | fields r2`, `f1,f2,f3,f4`, ``)
|
||||
f(`* | stats by(f3,f4) count(f1,f2) r1 | stats count(f3) r1, count(r1) r2 | fields r1`, `f3,f4`, ``)
|
||||
|
||||
f(`* | uniq`, `*`, ``)
|
||||
f(`* | uniq by (f1,f2)`, `f1,f2`, ``)
|
||||
f(`* | uniq by (f1,f2) | fields f1,f3`, `f1,f2`, ``)
|
||||
f(`* | uniq by (f1,f2) | rm f1,f3`, `f1,f2`, ``)
|
||||
f(`* | uniq by (f1,f2) | fields f3`, `f1,f2`, ``)
|
||||
|
||||
f(`* | rm f1, f2`, `*`, `f1,f2`)
|
||||
f(`* | rm f1, f2 | mv f2 f3`, `*`, `f1,f2,f3`)
|
||||
f(`* | rm f1, f2 | cp f2 f3`, `*`, `f1,f2,f3`)
|
||||
f(`* | rm f1, f2 | mv f2 f3 | sort by(f4)`, `*`, `f1,f2,f3`)
|
||||
f(`* | rm f1, f2 | mv f2 f3 | sort by(f1)`, `*`, `f1,f2,f3`)
|
||||
f(`* | rm f1, f2 | fields f3`, `f3`, ``)
|
||||
f(`* | rm f1, f2 | fields f1,f3`, `f3`, ``)
|
||||
f(`* | rm f1, f2 | stats count() f1`, ``, ``)
|
||||
f(`* | rm f1, f2 | stats count(f3) r1`, `f3`, ``)
|
||||
f(`* | rm f1, f2 | stats count(f1) r1`, ``, ``)
|
||||
f(`* | rm f1, f2 | stats count(f1,f3) r1`, `f3`, ``)
|
||||
f(`* | rm f1, f2 | stats by(f1) count(f2) r1`, ``, ``)
|
||||
f(`* | rm f1, f2 | stats by(f3) count(f2) r1`, `f3`, ``)
|
||||
f(`* | rm f1, f2 | stats by(f3) count(f4) r1`, `f3,f4`, ``)
|
||||
}
|
||||
|
|
|
@ -26,14 +26,20 @@ func TestPartitionLifecycle(t *testing.T) {
|
|||
if ddbStats.InmemoryParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of in-memory parts in empty partition: %d", ddbStats.InmemoryParts)
|
||||
}
|
||||
if ddbStats.FileParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of file parts in empty partition: %d", ddbStats.FileParts)
|
||||
if ddbStats.SmallParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of small file parts in empty partition: %d", ddbStats.SmallParts)
|
||||
}
|
||||
if ddbStats.BigParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of big file parts in empty partition: %d", ddbStats.BigParts)
|
||||
}
|
||||
if ddbStats.CompressedInmemorySize != 0 {
|
||||
t.Fatalf("unexpected non-zero size of inmemory parts for empty partition")
|
||||
}
|
||||
if ddbStats.CompressedFileSize != 0 {
|
||||
t.Fatalf("unexpected non-zero size of file parts for empty partition")
|
||||
if ddbStats.CompressedSmallPartSize != 0 {
|
||||
t.Fatalf("unexpected non-zero size of small file parts for empty partition")
|
||||
}
|
||||
if ddbStats.CompressedBigPartSize != 0 {
|
||||
t.Fatalf("unexpected non-zero size of big file parts for empty partition")
|
||||
}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
mustClosePartition(pt)
|
||||
|
@ -87,8 +93,8 @@ func TestPartitionMustAddRowsSerial(t *testing.T) {
|
|||
if ddbStats.InmemoryParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of in-memory parts after re-opening the partition: %d", ddbStats.InmemoryParts)
|
||||
}
|
||||
if ddbStats.FileParts == 0 {
|
||||
t.Fatalf("the number of file parts must be greater than 0 after re-opening the partition")
|
||||
if ddbStats.SmallParts+ddbStats.BigParts == 0 {
|
||||
t.Fatalf("the number of small parts must be greater than 0 after re-opening the partition")
|
||||
}
|
||||
|
||||
// Try adding entries for multiple streams at a time
|
||||
|
@ -115,7 +121,7 @@ func TestPartitionMustAddRowsSerial(t *testing.T) {
|
|||
if ddbStats.InmemoryParts != 0 {
|
||||
t.Fatalf("unexpected non-zero number of in-memory parts after re-opening the partition: %d", ddbStats.InmemoryParts)
|
||||
}
|
||||
if ddbStats.FileParts == 0 {
|
||||
if ddbStats.SmallParts+ddbStats.BigParts == 0 {
|
||||
t.Fatalf("the number of file parts must be greater than 0 after re-opening the partition")
|
||||
}
|
||||
|
||||
|
@ -172,16 +178,16 @@ func TestPartitionMustAddRowsConcurrent(t *testing.T) {
|
|||
// When the storage is no longer needed, closeTestStorage() must be called.
|
||||
func newTestStorage() *Storage {
|
||||
streamIDCache := workingsetcache.New(1024 * 1024)
|
||||
streamFilterCache := workingsetcache.New(1024 * 1024)
|
||||
filterStreamCache := workingsetcache.New(1024 * 1024)
|
||||
return &Storage{
|
||||
flushInterval: time.Second,
|
||||
streamIDCache: streamIDCache,
|
||||
streamFilterCache: streamFilterCache,
|
||||
filterStreamCache: filterStreamCache,
|
||||
}
|
||||
}
|
||||
|
||||
// closeTestStorage closes storage created via newTestStorage().
|
||||
func closeTestStorage(s *Storage) {
|
||||
s.streamIDCache.Stop()
|
||||
s.streamFilterCache.Stop()
|
||||
s.filterStreamCache.Stop()
|
||||
}
|
||||
|
|
133
lib/logstorage/pipe.go
Normal file
133
lib/logstorage/pipe.go
Normal file
|
@ -0,0 +1,133 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
type pipe interface {
|
||||
// String returns string representation of the pipe.
|
||||
String() string
|
||||
|
||||
// updateNeededFields must update neededFields and unneededFields with fields it needs and not needs at the input.
|
||||
updateNeededFields(neededFields, unneededFields fieldsSet)
|
||||
|
||||
// newPipeProcessor must return new pipeProcessor for the given ppBase.
|
||||
//
|
||||
// workersCount is the number of goroutine workers, which will call writeBlock() method.
|
||||
//
|
||||
// If stopCh is closed, the returned pipeProcessor must stop performing CPU-intensive tasks which take more than a few milliseconds.
|
||||
// It is OK to continue processing pipeProcessor calls if they take less than a few milliseconds.
|
||||
//
|
||||
// The returned pipeProcessor may call cancel() at any time in order to notify worker goroutines to stop sending new data to pipeProcessor.
|
||||
newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor
|
||||
}
|
||||
|
||||
// pipeProcessor must process a single pipe.
|
||||
type pipeProcessor interface {
|
||||
// writeBlock must write the given block of data to the given pipeProcessor.
|
||||
//
|
||||
// writeBlock is called concurrently from worker goroutines.
|
||||
// The workerID is the id of the worker goroutine, which calls the writeBlock.
|
||||
// It is in the range 0 ... workersCount-1 .
|
||||
//
|
||||
// It is OK to modify br contents inside writeBlock. The caller mustn't rely on br contents after writeBlock call.
|
||||
// It is forbidden to hold references to br after returning from writeBlock, since the caller may re-use it.
|
||||
//
|
||||
// If any error occurs at writeBlock, then cancel() must be called by pipeProcessor in order to notify worker goroutines
|
||||
// to stop sending new data. The occurred error must be returned from flush().
|
||||
//
|
||||
// cancel() may be called also when the pipeProcessor decides to stop accepting new data, even if there is no any error.
|
||||
writeBlock(workerID uint, br *blockResult)
|
||||
|
||||
// flush must flush all the data accumulated in the pipeProcessor to the base pipeProcessor.
|
||||
//
|
||||
// flush is called after all the worker goroutines are stopped.
|
||||
//
|
||||
// It is guaranteed that flush() is called for every pipeProcessor returned from pipe.newPipeProcessor().
|
||||
flush() error
|
||||
}
|
||||
|
||||
type defaultPipeProcessor func(workerID uint, br *blockResult)
|
||||
|
||||
func newDefaultPipeProcessor(writeBlock func(workerID uint, br *blockResult)) pipeProcessor {
|
||||
return defaultPipeProcessor(writeBlock)
|
||||
}
|
||||
|
||||
func (dpp defaultPipeProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
dpp(workerID, br)
|
||||
}
|
||||
|
||||
func (dpp defaultPipeProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipes(lex *lexer) ([]pipe, error) {
|
||||
var pipes []pipe
|
||||
for !lex.isKeyword(")", "") {
|
||||
if !lex.isKeyword("|") {
|
||||
return nil, fmt.Errorf("expecting '|'")
|
||||
}
|
||||
if !lex.mustNextToken() {
|
||||
return nil, fmt.Errorf("missing token after '|'")
|
||||
}
|
||||
switch {
|
||||
case lex.isKeyword("stats"):
|
||||
ps, err := parsePipeStats(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'stats' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, ps)
|
||||
case lex.isKeyword("sort"):
|
||||
ps, err := parsePipeSort(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'sort' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, ps)
|
||||
case lex.isKeyword("uniq"):
|
||||
pu, err := parsePipeUniq(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'uniq' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pu)
|
||||
case lex.isKeyword("limit", "head"):
|
||||
pl, err := parsePipeLimit(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'limit' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pl)
|
||||
case lex.isKeyword("offset", "skip"):
|
||||
ps, err := parsePipeOffset(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'offset' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, ps)
|
||||
case lex.isKeyword("fields"):
|
||||
pf, err := parsePipeFields(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'fields' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pf)
|
||||
case lex.isKeyword("copy", "cp"):
|
||||
pc, err := parsePipeCopy(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'copy' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pc)
|
||||
case lex.isKeyword("rename", "mv"):
|
||||
pr, err := parsePipeRename(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'rename' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pr)
|
||||
case lex.isKeyword("delete", "del", "rm"):
|
||||
pd, err := parsePipeDelete(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'delete' pipe: %w", err)
|
||||
}
|
||||
pipes = append(pipes, pd)
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected pipe %q", lex.token)
|
||||
}
|
||||
}
|
||||
return pipes, nil
|
||||
}
|
122
lib/logstorage/pipe_copy.go
Normal file
122
lib/logstorage/pipe_copy.go
Normal file
|
@ -0,0 +1,122 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// pipeCopy implements '| copy ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#copy-pipe
|
||||
type pipeCopy struct {
|
||||
// srcFields contains a list of source fields to copy
|
||||
srcFields []string
|
||||
|
||||
// dstFields contains a list of destination fields
|
||||
dstFields []string
|
||||
}
|
||||
|
||||
func (pc *pipeCopy) String() string {
|
||||
if len(pc.srcFields) == 0 {
|
||||
logger.Panicf("BUG: pipeCopy must contain at least a single srcField")
|
||||
}
|
||||
|
||||
a := make([]string, len(pc.srcFields))
|
||||
for i, srcField := range pc.srcFields {
|
||||
dstField := pc.dstFields[i]
|
||||
a[i] = quoteTokenIfNeeded(srcField) + " as " + quoteTokenIfNeeded(dstField)
|
||||
}
|
||||
return "copy " + strings.Join(a, ", ")
|
||||
}
|
||||
|
||||
func (pc *pipeCopy) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
neededSrcFields := make([]bool, len(pc.srcFields))
|
||||
for i, dstField := range pc.dstFields {
|
||||
if neededFields.contains(dstField) && !unneededFields.contains(dstField) {
|
||||
neededSrcFields[i] = true
|
||||
}
|
||||
}
|
||||
if neededFields.contains("*") {
|
||||
// update only unneeded fields
|
||||
unneededFields.addAll(pc.dstFields)
|
||||
for i, srcField := range pc.srcFields {
|
||||
if neededSrcFields[i] {
|
||||
unneededFields.remove(srcField)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// update only needed fields and reset unneeded fields
|
||||
neededFields.removeAll(pc.dstFields)
|
||||
for i, srcField := range pc.srcFields {
|
||||
if neededSrcFields[i] {
|
||||
neededFields.add(srcField)
|
||||
}
|
||||
}
|
||||
unneededFields.reset()
|
||||
}
|
||||
}
|
||||
|
||||
func (pc *pipeCopy) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||
return &pipeCopyProcessor{
|
||||
pc: pc,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeCopyProcessor struct {
|
||||
pc *pipeCopy
|
||||
ppBase pipeProcessor
|
||||
}
|
||||
|
||||
func (pcp *pipeCopyProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
br.copyColumns(pcp.pc.srcFields, pcp.pc.dstFields)
|
||||
pcp.ppBase.writeBlock(workerID, br)
|
||||
}
|
||||
|
||||
func (pcp *pipeCopyProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeCopy(lex *lexer) (*pipeCopy, error) {
|
||||
if !lex.isKeyword("copy", "cp") {
|
||||
return nil, fmt.Errorf("expecting 'copy' or 'cp'; got %q", lex.token)
|
||||
}
|
||||
|
||||
var srcFields []string
|
||||
var dstFields []string
|
||||
for {
|
||||
lex.nextToken()
|
||||
srcField, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse src field name: %w", err)
|
||||
}
|
||||
if lex.isKeyword("as") {
|
||||
lex.nextToken()
|
||||
}
|
||||
dstField, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse dst field name: %w", err)
|
||||
}
|
||||
|
||||
srcFields = append(srcFields, srcField)
|
||||
dstFields = append(dstFields, dstField)
|
||||
|
||||
switch {
|
||||
case lex.isKeyword("|", ")", ""):
|
||||
pc := &pipeCopy{
|
||||
srcFields: srcFields,
|
||||
dstFields: dstFields,
|
||||
}
|
||||
return pc, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',', '|' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
81
lib/logstorage/pipe_copy_test.go
Normal file
81
lib/logstorage/pipe_copy_test.go
Normal file
|
@ -0,0 +1,81 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeCopyUpdateNeededFields(t *testing.T) {
|
||||
f := func(s string, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeCopy(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("copy s1 d1, s2 d2", "*", "", "*", "d1,d2")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src and dst
|
||||
f("copy s1 d1 ,s2 d2", "*", "f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
f("copy s1 d1 ,s2 d2", "*", "s1,f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with dst
|
||||
f("copy s1 d1, s2 d2", "*", "d2,f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src and dst
|
||||
f("copy s1 d1, s2 d2", "*", "s1,d1,f1,f2", "*", "d1,d2,f1,f2,s1")
|
||||
f("copy s1 d1, s2 d2", "*", "s1,d2,f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// needed fields do not intersect with src and dst
|
||||
f("copy s1 d1, s2 d2", "f1,f2", "", "f1,f2", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("copy s1 d1, s2 d2", "s1,f1,f2", "", "s1,f1,f2", "")
|
||||
|
||||
// needed fields intersect with dst
|
||||
f("copy s1 d1, s2 d2", "d1,f1,f2", "", "f1,f2,s1", "")
|
||||
|
||||
// needed fields intersect with src and dst
|
||||
f("copy s1 d1, s2 d2", "s1,d1,f1,f2", "", "s1,f1,f2", "")
|
||||
f("copy s1 d1, s2 d2", "s1,d2,f1,f2", "", "s1,s2,f1,f2", "")
|
||||
f("copy s1 d1, s2 d2", "s2,d1,f1,f2", "", "s1,s2,f1,f2", "")
|
||||
}
|
||||
|
||||
func assertNeededFields(t *testing.T, nfs, unfs fieldsSet, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfsStr := nfs.String()
|
||||
unfsStr := unfs.String()
|
||||
|
||||
nfsExpected := newTestFieldsSet(neededFieldsExpected)
|
||||
unfsExpected := newTestFieldsSet(unneededFieldsExpected)
|
||||
nfsExpectedStr := nfsExpected.String()
|
||||
unfsExpectedStr := unfsExpected.String()
|
||||
|
||||
if nfsStr != nfsExpectedStr {
|
||||
t.Fatalf("unexpected needed fields; got %s; want %s", nfsStr, nfsExpectedStr)
|
||||
}
|
||||
if unfsStr != unfsExpectedStr {
|
||||
t.Fatalf("unexpected unneeded fields; got %s; want %s", unfsStr, unfsExpectedStr)
|
||||
}
|
||||
}
|
||||
|
||||
func newTestFieldsSet(fields string) fieldsSet {
|
||||
fs := newFieldsSet()
|
||||
if fields != "" {
|
||||
fs.addAll(strings.Split(fields, ","))
|
||||
}
|
||||
return fs
|
||||
}
|
86
lib/logstorage/pipe_delete.go
Normal file
86
lib/logstorage/pipe_delete.go
Normal file
|
@ -0,0 +1,86 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// pipeDelete implements '| delete ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#delete-pipe
|
||||
type pipeDelete struct {
|
||||
// fields contains a list of fields to delete
|
||||
fields []string
|
||||
}
|
||||
|
||||
func (pd *pipeDelete) String() string {
|
||||
if len(pd.fields) == 0 {
|
||||
logger.Panicf("BUG: pipeDelete must contain at least a single field")
|
||||
}
|
||||
|
||||
return "delete " + fieldNamesString(pd.fields)
|
||||
}
|
||||
|
||||
func (pd *pipeDelete) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
if neededFields.contains("*") {
|
||||
// update only unneeded fields
|
||||
unneededFields.addAll(pd.fields)
|
||||
} else {
|
||||
// update only needed fields
|
||||
neededFields.removeAll(pd.fields)
|
||||
}
|
||||
}
|
||||
|
||||
func (pd *pipeDelete) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||
return &pipeDeleteProcessor{
|
||||
pd: pd,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeDeleteProcessor struct {
|
||||
pd *pipeDelete
|
||||
ppBase pipeProcessor
|
||||
}
|
||||
|
||||
func (pdp *pipeDeleteProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
br.deleteColumns(pdp.pd.fields)
|
||||
pdp.ppBase.writeBlock(workerID, br)
|
||||
}
|
||||
|
||||
func (pdp *pipeDeleteProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeDelete(lex *lexer) (*pipeDelete, error) {
|
||||
if !lex.isKeyword("delete", "del", "rm") {
|
||||
return nil, fmt.Errorf("expecting 'delete', 'del' or 'rm'; got %q", lex.token)
|
||||
}
|
||||
|
||||
var fields []string
|
||||
for {
|
||||
lex.nextToken()
|
||||
field, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse field name: %w", err)
|
||||
}
|
||||
|
||||
fields = append(fields, field)
|
||||
|
||||
switch {
|
||||
case lex.isKeyword("|", ")", ""):
|
||||
pd := &pipeDelete{
|
||||
fields: fields,
|
||||
}
|
||||
return pd, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',', '|' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
38
lib/logstorage/pipe_delete_test.go
Normal file
38
lib/logstorage/pipe_delete_test.go
Normal file
|
@ -0,0 +1,38 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeDeleteUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeDelete(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("del s1,s2", "*", "", "*", "s1,s2")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src
|
||||
f("del s1,s2", "*", "f1,f2", "*", "s1,s2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
f("del s1,s2", "*", "s1,f1,f2", "*", "s1,s2,f1,f2")
|
||||
|
||||
// needed fields do not intersect with src
|
||||
f("del s1,s2", "f1,f2", "", "f1,f2", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("del s1,s2", "s1,f1,f2", "", "f1,f2", "")
|
||||
}
|
107
lib/logstorage/pipe_fields.go
Normal file
107
lib/logstorage/pipe_fields.go
Normal file
|
@ -0,0 +1,107 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"slices"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// pipeFields implements '| fields ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#fields-pipe
|
||||
type pipeFields struct {
|
||||
// fields contains list of fields to fetch
|
||||
fields []string
|
||||
|
||||
// whether fields contains star
|
||||
containsStar bool
|
||||
}
|
||||
|
||||
func (pf *pipeFields) String() string {
|
||||
if len(pf.fields) == 0 {
|
||||
logger.Panicf("BUG: pipeFields must contain at least a single field")
|
||||
}
|
||||
return "fields " + fieldNamesString(pf.fields)
|
||||
}
|
||||
|
||||
func (pf *pipeFields) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
if pf.containsStar {
|
||||
return
|
||||
}
|
||||
if neededFields.contains("*") {
|
||||
// subtract unneeded fields from pf.fields
|
||||
neededFields.reset()
|
||||
neededFields.addAll(pf.fields)
|
||||
for _, f := range unneededFields.getAll() {
|
||||
neededFields.remove(f)
|
||||
}
|
||||
} else {
|
||||
// intersect needed fields with pf.fields
|
||||
neededFieldsOrig := neededFields.clone()
|
||||
neededFields.reset()
|
||||
for _, f := range pf.fields {
|
||||
if neededFieldsOrig.contains(f) {
|
||||
neededFields.add(f)
|
||||
}
|
||||
}
|
||||
}
|
||||
unneededFields.reset()
|
||||
}
|
||||
|
||||
func (pf *pipeFields) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||
return &pipeFieldsProcessor{
|
||||
pf: pf,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeFieldsProcessor struct {
|
||||
pf *pipeFields
|
||||
ppBase pipeProcessor
|
||||
}
|
||||
|
||||
func (pfp *pipeFieldsProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
if !pfp.pf.containsStar {
|
||||
br.setColumns(pfp.pf.fields)
|
||||
}
|
||||
pfp.ppBase.writeBlock(workerID, br)
|
||||
}
|
||||
|
||||
func (pfp *pipeFieldsProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeFields(lex *lexer) (*pipeFields, error) {
|
||||
if !lex.isKeyword("fields") {
|
||||
return nil, fmt.Errorf("expecting 'fields'; got %q", lex.token)
|
||||
}
|
||||
|
||||
var fields []string
|
||||
for {
|
||||
lex.nextToken()
|
||||
field, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse field name: %w", err)
|
||||
}
|
||||
fields = append(fields, field)
|
||||
switch {
|
||||
case lex.isKeyword("|", ")", ""):
|
||||
if slices.Contains(fields, "*") {
|
||||
fields = []string{"*"}
|
||||
}
|
||||
pf := &pipeFields{
|
||||
fields: fields,
|
||||
containsStar: slices.Contains(fields, "*"),
|
||||
}
|
||||
return pf, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',', '|' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
42
lib/logstorage/pipe_fields_test.go
Normal file
42
lib/logstorage/pipe_fields_test.go
Normal file
|
@ -0,0 +1,42 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeFieldsUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeFields(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("fields s1, s2", "*", "", "s1,s2", "")
|
||||
f("fields *", "*", "", "*", "")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src
|
||||
f("fields s1, s2", "*", "f1,f2", "s1,s2", "")
|
||||
f("fields *", "*", "f1,f2", "*", "f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
f("fields s1, s2", "*", "s1,f1,f2", "s2", "")
|
||||
f("fields *", "*", "s1,f1,f2", "*", "s1,f1,f2")
|
||||
|
||||
// needed fields do not intersect with src
|
||||
f("fields s1, s2", "f1,f2", "", "", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("fields s1, s2", "s1,f1,f2", "", "s1", "")
|
||||
f("fields *", "s1,f1,f2", "", "s1,f1,f2", "")
|
||||
}
|
89
lib/logstorage/pipe_limit.go
Normal file
89
lib/logstorage/pipe_limit.go
Normal file
|
@ -0,0 +1,89 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync/atomic"
|
||||
)
|
||||
|
||||
// pipeLimit implements '| limit ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#limit-pipe
|
||||
type pipeLimit struct {
|
||||
n uint64
|
||||
}
|
||||
|
||||
func (pl *pipeLimit) String() string {
|
||||
return fmt.Sprintf("limit %d", pl.n)
|
||||
}
|
||||
|
||||
func (pl *pipeLimit) updateNeededFields(_, _ fieldsSet) {
|
||||
}
|
||||
|
||||
func (pl *pipeLimit) newPipeProcessor(_ int, _ <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
||||
if pl.n == 0 {
|
||||
// Special case - notify the caller to stop writing data to the returned pipeLimitProcessor
|
||||
cancel()
|
||||
}
|
||||
return &pipeLimitProcessor{
|
||||
pl: pl,
|
||||
cancel: cancel,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeLimitProcessor struct {
|
||||
pl *pipeLimit
|
||||
cancel func()
|
||||
ppBase pipeProcessor
|
||||
|
||||
rowsProcessed atomic.Uint64
|
||||
}
|
||||
|
||||
func (plp *pipeLimitProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
rowsProcessed := plp.rowsProcessed.Add(uint64(len(br.timestamps)))
|
||||
if rowsProcessed <= plp.pl.n {
|
||||
// Fast path - write all the rows to ppBase.
|
||||
plp.ppBase.writeBlock(workerID, br)
|
||||
return
|
||||
}
|
||||
|
||||
// Slow path - overflow. Write the remaining rows if needed.
|
||||
rowsProcessed -= uint64(len(br.timestamps))
|
||||
if rowsProcessed >= plp.pl.n {
|
||||
// Nothing to write. There is no need in cancel() call, since it has been called by another goroutine.
|
||||
return
|
||||
}
|
||||
|
||||
// Write remaining rows.
|
||||
keepRows := plp.pl.n - rowsProcessed
|
||||
br.truncateRows(int(keepRows))
|
||||
plp.ppBase.writeBlock(workerID, br)
|
||||
|
||||
// Notify the caller that it should stop passing more data to writeBlock().
|
||||
plp.cancel()
|
||||
}
|
||||
|
||||
func (plp *pipeLimitProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeLimit(lex *lexer) (*pipeLimit, error) {
|
||||
if !lex.isKeyword("limit", "head") {
|
||||
return nil, fmt.Errorf("expecting 'limit' or 'head'; got %q", lex.token)
|
||||
}
|
||||
|
||||
lex.nextToken()
|
||||
n, err := parseUint(lex.token)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse rows limit from %q: %w", lex.token, err)
|
||||
}
|
||||
lex.nextToken()
|
||||
pl := &pipeLimit{
|
||||
n: n,
|
||||
}
|
||||
return pl, nil
|
||||
}
|
76
lib/logstorage/pipe_offset.go
Normal file
76
lib/logstorage/pipe_offset.go
Normal file
|
@ -0,0 +1,76 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync/atomic"
|
||||
)
|
||||
|
||||
// pipeOffset implements '| offset ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#offset-pipe
|
||||
type pipeOffset struct {
|
||||
n uint64
|
||||
}
|
||||
|
||||
func (po *pipeOffset) String() string {
|
||||
return fmt.Sprintf("offset %d", po.n)
|
||||
}
|
||||
|
||||
func (po *pipeOffset) updateNeededFields(_, _ fieldsSet) {
|
||||
}
|
||||
|
||||
func (po *pipeOffset) newPipeProcessor(workersCount int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||
return &pipeOffsetProcessor{
|
||||
po: po,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeOffsetProcessor struct {
|
||||
po *pipeOffset
|
||||
ppBase pipeProcessor
|
||||
|
||||
rowsProcessed atomic.Uint64
|
||||
}
|
||||
|
||||
func (pop *pipeOffsetProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
rowsProcessed := pop.rowsProcessed.Add(uint64(len(br.timestamps)))
|
||||
if rowsProcessed <= pop.po.n {
|
||||
return
|
||||
}
|
||||
|
||||
rowsProcessed -= uint64(len(br.timestamps))
|
||||
if rowsProcessed >= pop.po.n {
|
||||
pop.ppBase.writeBlock(workerID, br)
|
||||
return
|
||||
}
|
||||
|
||||
rowsSkip := pop.po.n - rowsProcessed
|
||||
br.skipRows(int(rowsSkip))
|
||||
pop.ppBase.writeBlock(workerID, br)
|
||||
}
|
||||
|
||||
func (pop *pipeOffsetProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeOffset(lex *lexer) (*pipeOffset, error) {
|
||||
if !lex.isKeyword("offset", "skip") {
|
||||
return nil, fmt.Errorf("expecting 'offset' or 'skip'; got %q", lex.token)
|
||||
}
|
||||
|
||||
lex.nextToken()
|
||||
n, err := parseUint(lex.token)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse the number of rows to skip from %q: %w", lex.token, err)
|
||||
}
|
||||
lex.nextToken()
|
||||
po := &pipeOffset{
|
||||
n: n,
|
||||
}
|
||||
return po, nil
|
||||
}
|
126
lib/logstorage/pipe_rename.go
Normal file
126
lib/logstorage/pipe_rename.go
Normal file
|
@ -0,0 +1,126 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
// pipeRename implements '| rename ...' pipe.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#rename-pipe
|
||||
type pipeRename struct {
|
||||
// srcFields contains a list of source fields to rename
|
||||
srcFields []string
|
||||
|
||||
// dstFields contains a list of destination fields
|
||||
dstFields []string
|
||||
}
|
||||
|
||||
func (pr *pipeRename) String() string {
|
||||
if len(pr.srcFields) == 0 {
|
||||
logger.Panicf("BUG: pipeRename must contain at least a single srcField")
|
||||
}
|
||||
|
||||
a := make([]string, len(pr.srcFields))
|
||||
for i, srcField := range pr.srcFields {
|
||||
dstField := pr.dstFields[i]
|
||||
a[i] = quoteTokenIfNeeded(srcField) + " as " + quoteTokenIfNeeded(dstField)
|
||||
}
|
||||
return "rename " + strings.Join(a, ", ")
|
||||
}
|
||||
|
||||
func (pr *pipeRename) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
neededSrcFields := make([]bool, len(pr.srcFields))
|
||||
for i, dstField := range pr.dstFields {
|
||||
if neededFields.contains(dstField) && !unneededFields.contains(dstField) {
|
||||
neededSrcFields[i] = true
|
||||
}
|
||||
}
|
||||
if neededFields.contains("*") {
|
||||
// update only unneeded fields
|
||||
unneededFields.addAll(pr.dstFields)
|
||||
for i, srcField := range pr.srcFields {
|
||||
if neededSrcFields[i] {
|
||||
unneededFields.remove(srcField)
|
||||
} else {
|
||||
unneededFields.add(srcField)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// update only needed fields and reset unneeded fields
|
||||
neededFields.removeAll(pr.dstFields)
|
||||
for i, srcField := range pr.srcFields {
|
||||
if neededSrcFields[i] {
|
||||
neededFields.add(srcField)
|
||||
} else {
|
||||
neededFields.remove(srcField)
|
||||
}
|
||||
}
|
||||
unneededFields.reset()
|
||||
}
|
||||
}
|
||||
|
||||
func (pr *pipeRename) newPipeProcessor(_ int, _ <-chan struct{}, _ func(), ppBase pipeProcessor) pipeProcessor {
|
||||
return &pipeRenameProcessor{
|
||||
pr: pr,
|
||||
ppBase: ppBase,
|
||||
}
|
||||
}
|
||||
|
||||
type pipeRenameProcessor struct {
|
||||
pr *pipeRename
|
||||
ppBase pipeProcessor
|
||||
}
|
||||
|
||||
func (prp *pipeRenameProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
br.renameColumns(prp.pr.srcFields, prp.pr.dstFields)
|
||||
prp.ppBase.writeBlock(workerID, br)
|
||||
}
|
||||
|
||||
func (prp *pipeRenameProcessor) flush() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeRename(lex *lexer) (*pipeRename, error) {
|
||||
if !lex.isKeyword("rename", "mv") {
|
||||
return nil, fmt.Errorf("expecting 'rename' or 'mv'; got %q", lex.token)
|
||||
}
|
||||
|
||||
var srcFields []string
|
||||
var dstFields []string
|
||||
for {
|
||||
lex.nextToken()
|
||||
srcField, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse src field name: %w", err)
|
||||
}
|
||||
if lex.isKeyword("as") {
|
||||
lex.nextToken()
|
||||
}
|
||||
dstField, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse dst field name: %w", err)
|
||||
}
|
||||
|
||||
srcFields = append(srcFields, srcField)
|
||||
dstFields = append(dstFields, dstField)
|
||||
|
||||
switch {
|
||||
case lex.isKeyword("|", ")", ""):
|
||||
pr := &pipeRename{
|
||||
srcFields: srcFields,
|
||||
dstFields: dstFields,
|
||||
}
|
||||
return pr, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',', '|' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
54
lib/logstorage/pipe_rename_test.go
Normal file
54
lib/logstorage/pipe_rename_test.go
Normal file
|
@ -0,0 +1,54 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeRenameUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeRename(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("rename s1 d1, s2 d2", "*", "", "*", "d1,d2")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src and dst
|
||||
f("rename s1 d1, s2 d2", "*", "f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
// mv s1 d1, s2 d2 | rm s1, f1, f2 (d1, d2, f1, f2)
|
||||
f("rename s1 d1, s2 d2", "*", "s1,f1,f2", "*", "d1,d2,f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with dst
|
||||
f("rename s1 d1, s2 d2", "*", "d2,f1,f2", "*", "d1,d2,f1,f2,s2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src and dst
|
||||
f("rename s1 d1, s2 d2", "*", "s1,d1,f1,f2", "*", "d1,d2,f1,f2,s1")
|
||||
f("rename s1 d1, s2 d2", "*", "s1,d2,f1,f2", "*", "d1,d2,f1,f2,s2")
|
||||
|
||||
// needed fields do not intersect with src and dst
|
||||
f("rename s1 d1, s2 d2", "f1,f2", "", "f1,f2", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("rename s1 d1, s2 d2", "s1,f1,f2", "", "f1,f2", "")
|
||||
|
||||
// needed fields intersect with dst
|
||||
f("rename s1 d1, s2 d2", "d1,f1,f2", "", "f1,f2,s1", "")
|
||||
|
||||
// needed fields intersect with src and dst
|
||||
f("rename s1 d1, s2 d2", "s1,d1,f1,f2", "", "s1,f1,f2", "")
|
||||
f("rename s1 d1, s2 d2", "s1,d2,f1,f2", "", "s2,f1,f2", "")
|
||||
f("rename s1 d1, s2 d2", "s2,d1,f1,f2", "", "s1,f1,f2", "")
|
||||
}
|
754
lib/logstorage/pipe_sort.go
Normal file
754
lib/logstorage/pipe_sort.go
Normal file
|
@ -0,0 +1,754 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
"fmt"
|
||||
"math"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"unsafe"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
|
||||
)
|
||||
|
||||
// pipeSort processes '| sort ...' queries.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#sort-pipe
|
||||
type pipeSort struct {
|
||||
// byFields contains field names for sorting from 'by(...)' clause.
|
||||
byFields []*bySortField
|
||||
|
||||
// whether to apply descending order
|
||||
isDesc bool
|
||||
}
|
||||
|
||||
func (ps *pipeSort) String() string {
|
||||
s := "sort"
|
||||
if len(ps.byFields) > 0 {
|
||||
a := make([]string, len(ps.byFields))
|
||||
for i, bf := range ps.byFields {
|
||||
a[i] = bf.String()
|
||||
}
|
||||
s += " by (" + strings.Join(a, ", ") + ")"
|
||||
}
|
||||
if ps.isDesc {
|
||||
s += " desc"
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (ps *pipeSort) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
if len(ps.byFields) == 0 {
|
||||
neededFields.add("*")
|
||||
unneededFields.reset()
|
||||
} else {
|
||||
for _, bf := range ps.byFields {
|
||||
neededFields.add(bf.name)
|
||||
unneededFields.remove(bf.name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ps *pipeSort) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
||||
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
||||
|
||||
shards := make([]pipeSortProcessorShard, workersCount)
|
||||
for i := range shards {
|
||||
shard := &shards[i]
|
||||
shard.ps = ps
|
||||
shard.stateSizeBudget = stateSizeBudgetChunk
|
||||
maxStateSize -= stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
psp := &pipeSortProcessor{
|
||||
ps: ps,
|
||||
stopCh: stopCh,
|
||||
cancel: cancel,
|
||||
ppBase: ppBase,
|
||||
|
||||
shards: shards,
|
||||
|
||||
maxStateSize: maxStateSize,
|
||||
}
|
||||
psp.stateSizeBudget.Store(maxStateSize)
|
||||
|
||||
return psp
|
||||
}
|
||||
|
||||
type pipeSortProcessor struct {
|
||||
ps *pipeSort
|
||||
stopCh <-chan struct{}
|
||||
cancel func()
|
||||
ppBase pipeProcessor
|
||||
|
||||
shards []pipeSortProcessorShard
|
||||
|
||||
maxStateSize int64
|
||||
stateSizeBudget atomic.Int64
|
||||
}
|
||||
|
||||
type pipeSortProcessorShard struct {
|
||||
pipeSortProcessorShardNopad
|
||||
|
||||
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||
_ [128 - unsafe.Sizeof(pipeSortProcessorShardNopad{})%128]byte
|
||||
}
|
||||
|
||||
type pipeSortProcessorShardNopad struct {
|
||||
// ps points to the parent pipeSort.
|
||||
ps *pipeSort
|
||||
|
||||
// blocks holds all the blocks with logs written to the shard.
|
||||
blocks []sortBlock
|
||||
|
||||
// rowRefs holds references to all the rows stored in blocks.
|
||||
//
|
||||
// Sorting sorts rowRefs, while blocks remain unchanged. This should speed up sorting.
|
||||
rowRefs []sortRowRef
|
||||
|
||||
// rowRefNext points to the next index at rowRefs during merge shards phase
|
||||
rowRefNext int
|
||||
|
||||
// stateSizeBudget is the remaining budget for the whole state size for the shard.
|
||||
// The per-shard budget is provided in chunks from the parent pipeSortProcessor.
|
||||
stateSizeBudget int
|
||||
}
|
||||
|
||||
// sortBlock represents a block of logs for sorting.
|
||||
type sortBlock struct {
|
||||
// br is a result block to sort
|
||||
br *blockResult
|
||||
|
||||
// byColumns refers block data for 'by(...)' columns
|
||||
byColumns []sortBlockByColumn
|
||||
|
||||
// otherColumns refers block data for other than 'by(...)' columns
|
||||
otherColumns []*blockResultColumn
|
||||
}
|
||||
|
||||
// sortBlockByColumn represents data for a single column from 'sort by(...)' clause.
|
||||
type sortBlockByColumn struct {
|
||||
// c contains column data
|
||||
c *blockResultColumn
|
||||
|
||||
// i64Values contains int64 numbers parsed from values
|
||||
i64Values []int64
|
||||
|
||||
// f64Values contains float64 numbers parsed from values
|
||||
f64Values []float64
|
||||
}
|
||||
|
||||
// sortRowRef is the reference to a single log entry written to `sort` pipe.
|
||||
type sortRowRef struct {
|
||||
// blockIdx is the index of the block at pipeSortProcessorShard.blocks.
|
||||
blockIdx int
|
||||
|
||||
// rowIdx is the index of the log entry inside the block referenced by blockIdx.
|
||||
rowIdx int
|
||||
}
|
||||
|
||||
func (c *sortBlockByColumn) getI64ValueAtRow(rowIdx int) int64 {
|
||||
if c.c.isConst {
|
||||
return c.i64Values[0]
|
||||
}
|
||||
return c.i64Values[rowIdx]
|
||||
}
|
||||
|
||||
func (c *sortBlockByColumn) getF64ValueAtRow(rowIdx int) float64 {
|
||||
if c.c.isConst {
|
||||
return c.f64Values[0]
|
||||
}
|
||||
return c.f64Values[rowIdx]
|
||||
}
|
||||
|
||||
// writeBlock writes br to shard.
|
||||
func (shard *pipeSortProcessorShard) writeBlock(br *blockResult) {
|
||||
// clone br, so it could be owned by shard
|
||||
br = br.clone()
|
||||
cs := br.getColumns()
|
||||
|
||||
byFields := shard.ps.byFields
|
||||
if len(byFields) == 0 {
|
||||
// Sort by all the columns
|
||||
|
||||
// Generate byColumns
|
||||
var rc resultColumn
|
||||
bb := bbPool.Get()
|
||||
for i := range br.timestamps {
|
||||
// JSON-encode all the columns per each row into a single string
|
||||
// and sort rows by the resulting string.
|
||||
bb.B = bb.B[:0]
|
||||
for _, c := range cs {
|
||||
v := c.getValueAtRow(br, i)
|
||||
bb.B = marshalJSONKeyValue(bb.B, c.name, v)
|
||||
bb.B = append(bb.B, ',')
|
||||
}
|
||||
rc.addValue(bytesutil.ToUnsafeString(bb.B))
|
||||
}
|
||||
bbPool.Put(bb)
|
||||
|
||||
i64Values := make([]int64, len(br.timestamps))
|
||||
f64Values := make([]float64, len(br.timestamps))
|
||||
for i := range f64Values {
|
||||
f64Values[i] = nan
|
||||
}
|
||||
byColumns := []sortBlockByColumn{
|
||||
{
|
||||
c: &blockResultColumn{
|
||||
valueType: valueTypeString,
|
||||
encodedValues: rc.values,
|
||||
},
|
||||
i64Values: i64Values,
|
||||
f64Values: f64Values,
|
||||
},
|
||||
}
|
||||
shard.stateSizeBudget -= len(rc.buf) + int(unsafe.Sizeof(byColumns[0])+unsafe.Sizeof(*byColumns[0].c))
|
||||
|
||||
// Append br to shard.blocks.
|
||||
shard.blocks = append(shard.blocks, sortBlock{
|
||||
br: br,
|
||||
byColumns: byColumns,
|
||||
otherColumns: cs,
|
||||
})
|
||||
} else {
|
||||
// Collect values for columns from byFields.
|
||||
byColumns := make([]sortBlockByColumn, len(byFields))
|
||||
for i, bf := range byFields {
|
||||
c := br.getColumnByName(bf.name)
|
||||
bc := &byColumns[i]
|
||||
bc.c = c
|
||||
|
||||
if c.isTime {
|
||||
// Do not initialize bc.i64Values and bc.f64Values, since they aren't used.
|
||||
// This saves some memory.
|
||||
continue
|
||||
}
|
||||
if c.isConst {
|
||||
bc.i64Values = shard.createInt64Values(c.encodedValues)
|
||||
bc.f64Values = shard.createFloat64Values(c.encodedValues)
|
||||
continue
|
||||
}
|
||||
|
||||
// pre-populate values in order to track better br memory usage
|
||||
values := c.getValues(br)
|
||||
bc.i64Values = shard.createInt64Values(values)
|
||||
bc.f64Values = shard.createFloat64Values(values)
|
||||
}
|
||||
shard.stateSizeBudget -= len(byColumns) * int(unsafe.Sizeof(byColumns[0]))
|
||||
|
||||
// Collect values for other columns.
|
||||
otherColumns := make([]*blockResultColumn, 0, len(cs))
|
||||
for _, c := range cs {
|
||||
isByField := false
|
||||
for _, bf := range byFields {
|
||||
if bf.name == c.name {
|
||||
isByField = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !isByField {
|
||||
otherColumns = append(otherColumns, c)
|
||||
}
|
||||
}
|
||||
shard.stateSizeBudget -= len(otherColumns) * int(unsafe.Sizeof(otherColumns[0]))
|
||||
|
||||
// Append br to shard.blocks.
|
||||
shard.blocks = append(shard.blocks, sortBlock{
|
||||
br: br,
|
||||
byColumns: byColumns,
|
||||
otherColumns: otherColumns,
|
||||
})
|
||||
}
|
||||
|
||||
shard.stateSizeBudget -= br.sizeBytes()
|
||||
shard.stateSizeBudget -= int(unsafe.Sizeof(shard.blocks[0]))
|
||||
|
||||
// Add row references to rowRefs.
|
||||
blockIdx := len(shard.blocks) - 1
|
||||
rowRefs := shard.rowRefs
|
||||
rowRefsLen := len(rowRefs)
|
||||
for i := range br.timestamps {
|
||||
rowRefs = append(rowRefs, sortRowRef{
|
||||
blockIdx: blockIdx,
|
||||
rowIdx: i,
|
||||
})
|
||||
}
|
||||
shard.rowRefs = rowRefs
|
||||
shard.stateSizeBudget -= (len(rowRefs) - rowRefsLen) * int(unsafe.Sizeof(rowRefs[0]))
|
||||
}
|
||||
|
||||
func (shard *pipeSortProcessorShard) createInt64Values(values []string) []int64 {
|
||||
a := make([]int64, len(values))
|
||||
for i, v := range values {
|
||||
i64, ok := tryParseInt64(v)
|
||||
if ok {
|
||||
a[i] = i64
|
||||
continue
|
||||
}
|
||||
u32, _ := tryParseIPv4(v)
|
||||
a[i] = int64(u32)
|
||||
// Do not try parsing timestamp and duration, since they may be negative.
|
||||
// This breaks sorting.
|
||||
}
|
||||
|
||||
shard.stateSizeBudget -= len(a) * int(unsafe.Sizeof(a[0]))
|
||||
|
||||
return a
|
||||
}
|
||||
|
||||
func (shard *pipeSortProcessorShard) createFloat64Values(values []string) []float64 {
|
||||
a := make([]float64, len(values))
|
||||
for i, v := range values {
|
||||
f, ok := tryParseFloat64(v)
|
||||
if !ok {
|
||||
f = nan
|
||||
}
|
||||
a[i] = f
|
||||
}
|
||||
|
||||
shard.stateSizeBudget -= len(a) * int(unsafe.Sizeof(a[0]))
|
||||
|
||||
return a
|
||||
}
|
||||
|
||||
func (psp *pipeSortProcessorShard) Len() int {
|
||||
return len(psp.rowRefs)
|
||||
}
|
||||
|
||||
func (psp *pipeSortProcessorShard) Swap(i, j int) {
|
||||
rowRefs := psp.rowRefs
|
||||
rowRefs[i], rowRefs[j] = rowRefs[j], rowRefs[i]
|
||||
}
|
||||
|
||||
func (psp *pipeSortProcessorShard) Less(i, j int) bool {
|
||||
return sortBlockLess(psp, i, psp, j)
|
||||
}
|
||||
|
||||
func (psp *pipeSortProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
shard := &psp.shards[workerID]
|
||||
|
||||
for shard.stateSizeBudget < 0 {
|
||||
// steal some budget for the state size from the global budget.
|
||||
remaining := psp.stateSizeBudget.Add(-stateSizeBudgetChunk)
|
||||
if remaining < 0 {
|
||||
// The state size is too big. Stop processing data in order to avoid OOM crash.
|
||||
if remaining+stateSizeBudgetChunk >= 0 {
|
||||
// Notify worker goroutines to stop calling writeBlock() in order to save CPU time.
|
||||
psp.cancel()
|
||||
}
|
||||
return
|
||||
}
|
||||
shard.stateSizeBudget += stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
shard.writeBlock(br)
|
||||
}
|
||||
|
||||
func (psp *pipeSortProcessor) flush() error {
|
||||
if n := psp.stateSizeBudget.Load(); n <= 0 {
|
||||
return fmt.Errorf("cannot calculate [%s], since it requires more than %dMB of memory", psp.ps.String(), psp.maxStateSize/(1<<20))
|
||||
}
|
||||
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
// Sort every shard in parallel
|
||||
var wg sync.WaitGroup
|
||||
shards := psp.shards
|
||||
for i := range shards {
|
||||
wg.Add(1)
|
||||
go func(shard *pipeSortProcessorShard) {
|
||||
// TODO: interrupt long sorting when psp.stopCh is closed.
|
||||
sort.Sort(shard)
|
||||
wg.Done()
|
||||
}(&shards[i])
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
// Merge sorted results across shards
|
||||
sh := pipeSortProcessorShardsHeap(make([]*pipeSortProcessorShard, 0, len(shards)))
|
||||
for i := range shards {
|
||||
shard := &shards[i]
|
||||
if shard.Len() > 0 {
|
||||
sh = append(sh, shard)
|
||||
}
|
||||
}
|
||||
if len(sh) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
heap.Init(&sh)
|
||||
|
||||
wctx := &pipeSortWriteContext{
|
||||
psp: psp,
|
||||
}
|
||||
var shardNext *pipeSortProcessorShard
|
||||
|
||||
for len(sh) > 1 {
|
||||
shard := sh[0]
|
||||
wctx.writeRow(shard, shard.rowRefNext)
|
||||
shard.rowRefNext++
|
||||
|
||||
if shard.rowRefNext >= len(shard.rowRefs) {
|
||||
_ = heap.Pop(&sh)
|
||||
shardNext = nil
|
||||
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
continue
|
||||
}
|
||||
|
||||
if shardNext == nil {
|
||||
shardNext = sh[1]
|
||||
if len(sh) > 2 && sortBlockLess(sh[2], sh[2].rowRefNext, shardNext, shardNext.rowRefNext) {
|
||||
shardNext = sh[2]
|
||||
}
|
||||
}
|
||||
|
||||
if sortBlockLess(shardNext, shardNext.rowRefNext, shard, shard.rowRefNext) {
|
||||
heap.Fix(&sh, 0)
|
||||
shardNext = nil
|
||||
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
if len(sh) == 1 {
|
||||
shard := sh[0]
|
||||
for shard.rowRefNext < len(shard.rowRefs) {
|
||||
wctx.writeRow(shard, shard.rowRefNext)
|
||||
shard.rowRefNext++
|
||||
}
|
||||
}
|
||||
wctx.flush()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type pipeSortWriteContext struct {
|
||||
psp *pipeSortProcessor
|
||||
rcs []resultColumn
|
||||
br blockResult
|
||||
|
||||
valuesLen int
|
||||
}
|
||||
|
||||
func (wctx *pipeSortWriteContext) writeRow(shard *pipeSortProcessorShard, rowIdx int) {
|
||||
rr := shard.rowRefs[rowIdx]
|
||||
b := &shard.blocks[rr.blockIdx]
|
||||
|
||||
byFields := shard.ps.byFields
|
||||
rcs := wctx.rcs
|
||||
|
||||
areEqualColumns := len(rcs) == len(byFields)+len(b.otherColumns)
|
||||
if areEqualColumns {
|
||||
for i, c := range b.otherColumns {
|
||||
if rcs[len(byFields)+i].name != c.name {
|
||||
areEqualColumns = false
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !areEqualColumns {
|
||||
// send the current block to bbBase and construct a block with new set of columns
|
||||
wctx.flush()
|
||||
|
||||
rcs = wctx.rcs[:0]
|
||||
for _, bf := range byFields {
|
||||
rcs = append(rcs, resultColumn{
|
||||
name: bf.name,
|
||||
})
|
||||
}
|
||||
for _, c := range b.otherColumns {
|
||||
rcs = append(rcs, resultColumn{
|
||||
name: c.name,
|
||||
})
|
||||
}
|
||||
wctx.rcs = rcs
|
||||
}
|
||||
|
||||
br := b.br
|
||||
byColumns := b.byColumns
|
||||
for i := range byFields {
|
||||
v := byColumns[i].c.getValueAtRow(br, rr.rowIdx)
|
||||
rcs[i].addValue(v)
|
||||
wctx.valuesLen += len(v)
|
||||
}
|
||||
|
||||
for i, c := range b.otherColumns {
|
||||
v := c.getValueAtRow(br, rr.rowIdx)
|
||||
rcs[len(byFields)+i].addValue(v)
|
||||
wctx.valuesLen += len(v)
|
||||
}
|
||||
|
||||
if wctx.valuesLen >= 1_000_000 {
|
||||
wctx.flush()
|
||||
}
|
||||
}
|
||||
|
||||
func (wctx *pipeSortWriteContext) flush() {
|
||||
rcs := wctx.rcs
|
||||
br := &wctx.br
|
||||
|
||||
wctx.valuesLen = 0
|
||||
|
||||
if len(rcs) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// Flush rcs to ppBase
|
||||
br.setResultColumns(rcs)
|
||||
wctx.psp.ppBase.writeBlock(0, br)
|
||||
br.reset()
|
||||
for i := range rcs {
|
||||
rcs[i].resetKeepName()
|
||||
}
|
||||
}
|
||||
|
||||
type pipeSortProcessorShardsHeap []*pipeSortProcessorShard
|
||||
|
||||
func (sh *pipeSortProcessorShardsHeap) Len() int {
|
||||
return len(*sh)
|
||||
}
|
||||
|
||||
func (sh *pipeSortProcessorShardsHeap) Swap(i, j int) {
|
||||
a := *sh
|
||||
a[i], a[j] = a[j], a[i]
|
||||
}
|
||||
|
||||
func (sh *pipeSortProcessorShardsHeap) Less(i, j int) bool {
|
||||
a := *sh
|
||||
shardA := a[i]
|
||||
shardB := a[j]
|
||||
return sortBlockLess(shardA, shardA.rowRefNext, shardB, shardB.rowRefNext)
|
||||
}
|
||||
|
||||
func (sh *pipeSortProcessorShardsHeap) Push(x any) {
|
||||
shard := x.(*pipeSortProcessorShard)
|
||||
*sh = append(*sh, shard)
|
||||
}
|
||||
|
||||
func (sh *pipeSortProcessorShardsHeap) Pop() any {
|
||||
a := *sh
|
||||
x := a[len(a)-1]
|
||||
a[len(a)-1] = nil
|
||||
*sh = a[:len(a)-1]
|
||||
return x
|
||||
}
|
||||
|
||||
func sortBlockLess(shardA *pipeSortProcessorShard, rowIdxA int, shardB *pipeSortProcessorShard, rowIdxB int) bool {
|
||||
byFields := shardA.ps.byFields
|
||||
|
||||
rrA := shardA.rowRefs[rowIdxA]
|
||||
rrB := shardB.rowRefs[rowIdxB]
|
||||
bA := &shardA.blocks[rrA.blockIdx]
|
||||
bB := &shardB.blocks[rrB.blockIdx]
|
||||
for idx := range bA.byColumns {
|
||||
cA := &bA.byColumns[idx]
|
||||
cB := &bB.byColumns[idx]
|
||||
isDesc := len(byFields) > 0 && byFields[idx].isDesc
|
||||
if shardA.ps.isDesc {
|
||||
isDesc = !isDesc
|
||||
}
|
||||
|
||||
if cA.c.isConst && cB.c.isConst {
|
||||
// Fast path - compare const values
|
||||
ccA := cA.c.encodedValues[0]
|
||||
ccB := cB.c.encodedValues[0]
|
||||
if ccA == ccB {
|
||||
continue
|
||||
}
|
||||
return cA.c.encodedValues[0] < cB.c.encodedValues[0]
|
||||
}
|
||||
|
||||
if cA.c.isTime && cB.c.isTime {
|
||||
// Fast path - sort by _time
|
||||
tA := bA.br.timestamps[rrA.rowIdx]
|
||||
tB := bB.br.timestamps[rrB.rowIdx]
|
||||
if tA == tB {
|
||||
continue
|
||||
}
|
||||
if isDesc {
|
||||
return tB < tA
|
||||
}
|
||||
return tA < tB
|
||||
}
|
||||
if cA.c.isTime {
|
||||
// treat timestamps as smaller than other values
|
||||
return true
|
||||
}
|
||||
if cB.c.isTime {
|
||||
// treat timestamps as smaller than other values
|
||||
return false
|
||||
}
|
||||
|
||||
// Try sorting by int64 values at first
|
||||
uA := cA.getI64ValueAtRow(rrA.rowIdx)
|
||||
uB := cB.getI64ValueAtRow(rrB.rowIdx)
|
||||
if uA != 0 && uB != 0 {
|
||||
if uA == uB {
|
||||
continue
|
||||
}
|
||||
if isDesc {
|
||||
return uB < uA
|
||||
}
|
||||
return uA < uB
|
||||
}
|
||||
|
||||
// Try sorting by float64 then
|
||||
fA := cA.getF64ValueAtRow(rrA.rowIdx)
|
||||
fB := cB.getF64ValueAtRow(rrB.rowIdx)
|
||||
if !math.IsNaN(fA) && !math.IsNaN(fB) {
|
||||
if fA == fB {
|
||||
continue
|
||||
}
|
||||
if isDesc {
|
||||
return fB < fA
|
||||
}
|
||||
return fA < fB
|
||||
}
|
||||
|
||||
// Fall back to string sorting
|
||||
sA := cA.c.getValueAtRow(bA.br, rrA.rowIdx)
|
||||
sB := cB.c.getValueAtRow(bB.br, rrB.rowIdx)
|
||||
if sA == sB {
|
||||
continue
|
||||
}
|
||||
if isDesc {
|
||||
return sB < sA
|
||||
}
|
||||
return sA < sB
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func parsePipeSort(lex *lexer) (*pipeSort, error) {
|
||||
if !lex.isKeyword("sort") {
|
||||
return nil, fmt.Errorf("expecting 'sort'; got %q", lex.token)
|
||||
}
|
||||
lex.nextToken()
|
||||
|
||||
var ps pipeSort
|
||||
if lex.isKeyword("by") {
|
||||
lex.nextToken()
|
||||
bfs, err := parseBySortFields(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'by' clause: %w", err)
|
||||
}
|
||||
ps.byFields = bfs
|
||||
}
|
||||
|
||||
if lex.isKeyword("desc") {
|
||||
lex.nextToken()
|
||||
ps.isDesc = true
|
||||
}
|
||||
|
||||
return &ps, nil
|
||||
}
|
||||
|
||||
// bySortField represents 'by (...)' part of the pipeSort.
|
||||
type bySortField struct {
|
||||
// the name of the field to sort
|
||||
name string
|
||||
|
||||
// whether the sorting for the given field in descending order
|
||||
isDesc bool
|
||||
}
|
||||
|
||||
func (bf *bySortField) String() string {
|
||||
s := quoteTokenIfNeeded(bf.name)
|
||||
if bf.isDesc {
|
||||
s += " desc"
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func parseBySortFields(lex *lexer) ([]*bySortField, error) {
|
||||
if !lex.isKeyword("(") {
|
||||
return nil, fmt.Errorf("missing `(`")
|
||||
}
|
||||
var bfs []*bySortField
|
||||
for {
|
||||
lex.nextToken()
|
||||
if lex.isKeyword(")") {
|
||||
lex.nextToken()
|
||||
return bfs, nil
|
||||
}
|
||||
fieldName, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse field name: %w", err)
|
||||
}
|
||||
bf := &bySortField{
|
||||
name: fieldName,
|
||||
}
|
||||
if lex.isKeyword("desc") {
|
||||
lex.nextToken()
|
||||
bf.isDesc = true
|
||||
}
|
||||
bfs = append(bfs, bf)
|
||||
switch {
|
||||
case lex.isKeyword(")"):
|
||||
lex.nextToken()
|
||||
return bfs, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func marshalJSONKeyValue(dst []byte, k, v string) []byte {
|
||||
dst = strconv.AppendQuote(dst, k)
|
||||
dst = append(dst, ':')
|
||||
dst = strconv.AppendQuote(dst, v)
|
||||
return dst
|
||||
}
|
||||
|
||||
func tryParseInt64(s string) (int64, bool) {
|
||||
if len(s) == 0 {
|
||||
return 0, false
|
||||
}
|
||||
|
||||
isMinus := s[0] == '-'
|
||||
if isMinus {
|
||||
s = s[1:]
|
||||
}
|
||||
u64, ok := tryParseUint64(s)
|
||||
if !ok {
|
||||
return 0, false
|
||||
}
|
||||
if !isMinus {
|
||||
if u64 > math.MaxInt64 {
|
||||
return 0, false
|
||||
}
|
||||
return int64(u64), true
|
||||
}
|
||||
if u64 > -math.MinInt64 {
|
||||
return 0, false
|
||||
}
|
||||
return -int64(u64), true
|
||||
}
|
38
lib/logstorage/pipe_sort_test.go
Normal file
38
lib/logstorage/pipe_sort_test.go
Normal file
|
@ -0,0 +1,38 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeSortUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeSort(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("sort by(s1,s2)", "*", "", "*", "")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src
|
||||
f("sort by(s1,s2)", "*", "f1,f2", "*", "f1,f2")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
f("sort by(s1,s2)", "*", "s1,f1,f2", "*", "f1,f2")
|
||||
|
||||
// needed fields do not intersect with src
|
||||
f("sort by(s1,s2)", "f1,f2", "", "s1,s2,f1,f2", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("sort by(s1,s2)", "s1,f1,f2", "", "s1,s2,f1,f2", "")
|
||||
}
|
815
lib/logstorage/pipe_stats.go
Normal file
815
lib/logstorage/pipe_stats.go
Normal file
|
@ -0,0 +1,815 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"slices"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"unsafe"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
|
||||
)
|
||||
|
||||
// pipeStats processes '| stats ...' queries.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#stats-pipe
|
||||
type pipeStats struct {
|
||||
// byFields contains field names with optional buckets from 'by(...)' clause.
|
||||
byFields []*byStatsField
|
||||
|
||||
// resultNames contains names of output results generated by funcs.
|
||||
resultNames []string
|
||||
|
||||
// funcs contains stats functions to execute.
|
||||
funcs []statsFunc
|
||||
}
|
||||
|
||||
type statsFunc interface {
|
||||
// String returns string representation of statsFunc
|
||||
String() string
|
||||
|
||||
// neededFields returns the needed fields for calculating the given stats
|
||||
neededFields() []string
|
||||
|
||||
// newStatsProcessor must create new statsProcessor for calculating stats for the given statsFunc.
|
||||
//
|
||||
// It also must return the size in bytes of the returned statsProcessor.
|
||||
newStatsProcessor() (statsProcessor, int)
|
||||
}
|
||||
|
||||
// statsProcessor must process stats for some statsFunc.
|
||||
//
|
||||
// All the statsProcessor methods are called from a single goroutine at a time,
|
||||
// so there is no need in the internal synchronization.
|
||||
type statsProcessor interface {
|
||||
// updateStatsForAllRows must update statsProcessor stats for all the rows in br.
|
||||
//
|
||||
// It must return the change of internal state size in bytes for the statsProcessor.
|
||||
updateStatsForAllRows(br *blockResult) int
|
||||
|
||||
// updateStatsForRow must update statsProcessor stats for the row at rowIndex in br.
|
||||
//
|
||||
// It must return the change of internal state size in bytes for the statsProcessor.
|
||||
updateStatsForRow(br *blockResult, rowIndex int) int
|
||||
|
||||
// mergeState must merge sfp state into statsProcessor state.
|
||||
mergeState(sfp statsProcessor)
|
||||
|
||||
// finalizeStats must return the collected stats result from statsProcessor.
|
||||
finalizeStats() string
|
||||
}
|
||||
|
||||
func (ps *pipeStats) String() string {
|
||||
s := "stats "
|
||||
if len(ps.byFields) > 0 {
|
||||
a := make([]string, len(ps.byFields))
|
||||
for i := range ps.byFields {
|
||||
a[i] = ps.byFields[i].String()
|
||||
}
|
||||
s += "by (" + strings.Join(a, ", ") + ") "
|
||||
}
|
||||
|
||||
if len(ps.funcs) == 0 {
|
||||
logger.Panicf("BUG: pipeStats must contain at least a single statsFunc")
|
||||
}
|
||||
a := make([]string, len(ps.funcs))
|
||||
for i, f := range ps.funcs {
|
||||
a[i] = f.String() + " as " + quoteTokenIfNeeded(ps.resultNames[i])
|
||||
}
|
||||
s += strings.Join(a, ", ")
|
||||
return s
|
||||
}
|
||||
|
||||
func (ps *pipeStats) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
neededFieldsOrig := neededFields.clone()
|
||||
neededFields.reset()
|
||||
|
||||
byFields := make([]string, len(ps.byFields))
|
||||
for i, bf := range ps.byFields {
|
||||
byFields[i] = bf.name
|
||||
}
|
||||
|
||||
for _, f := range byFields {
|
||||
if neededFieldsOrig.contains(f) && !unneededFields.contains(f) {
|
||||
neededFields.addAll(byFields)
|
||||
}
|
||||
}
|
||||
|
||||
for i, resultName := range ps.resultNames {
|
||||
if neededFieldsOrig.contains(resultName) && !unneededFields.contains(resultName) {
|
||||
funcFields := ps.funcs[i].neededFields()
|
||||
neededFields.addAll(byFields)
|
||||
neededFields.addAll(funcFields)
|
||||
}
|
||||
}
|
||||
|
||||
unneededFields.reset()
|
||||
}
|
||||
|
||||
const stateSizeBudgetChunk = 1 << 20
|
||||
|
||||
func (ps *pipeStats) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
||||
maxStateSize := int64(float64(memory.Allowed()) * 0.3)
|
||||
|
||||
shards := make([]pipeStatsProcessorShard, workersCount)
|
||||
for i := range shards {
|
||||
shard := &shards[i]
|
||||
shard.ps = ps
|
||||
shard.m = make(map[string]*pipeStatsGroup)
|
||||
shard.stateSizeBudget = stateSizeBudgetChunk
|
||||
maxStateSize -= stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
psp := &pipeStatsProcessor{
|
||||
ps: ps,
|
||||
stopCh: stopCh,
|
||||
cancel: cancel,
|
||||
ppBase: ppBase,
|
||||
|
||||
shards: shards,
|
||||
|
||||
maxStateSize: maxStateSize,
|
||||
}
|
||||
psp.stateSizeBudget.Store(maxStateSize)
|
||||
|
||||
return psp
|
||||
}
|
||||
|
||||
type pipeStatsProcessor struct {
|
||||
ps *pipeStats
|
||||
stopCh <-chan struct{}
|
||||
cancel func()
|
||||
ppBase pipeProcessor
|
||||
|
||||
shards []pipeStatsProcessorShard
|
||||
|
||||
maxStateSize int64
|
||||
stateSizeBudget atomic.Int64
|
||||
}
|
||||
|
||||
type pipeStatsProcessorShard struct {
|
||||
pipeStatsProcessorShardNopad
|
||||
|
||||
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||
_ [128 - unsafe.Sizeof(pipeStatsProcessorShardNopad{})%128]byte
|
||||
}
|
||||
|
||||
type pipeStatsProcessorShardNopad struct {
|
||||
ps *pipeStats
|
||||
m map[string]*pipeStatsGroup
|
||||
|
||||
columnValues [][]string
|
||||
keyBuf []byte
|
||||
|
||||
stateSizeBudget int
|
||||
}
|
||||
|
||||
func (shard *pipeStatsProcessorShard) writeBlock(br *blockResult) {
|
||||
byFields := shard.ps.byFields
|
||||
|
||||
if len(byFields) == 0 {
|
||||
// Fast path - pass all the rows to a single group with empty key.
|
||||
psg := shard.getPipeStatsGroup(nil)
|
||||
shard.stateSizeBudget -= psg.updateStatsForAllRows(br)
|
||||
return
|
||||
}
|
||||
if len(byFields) == 1 {
|
||||
// Special case for grouping by a single column.
|
||||
bf := byFields[0]
|
||||
c := br.getColumnByName(bf.name)
|
||||
if c.isConst {
|
||||
// Fast path for column with constant value.
|
||||
v := br.getBucketedValue(c.encodedValues[0], bf)
|
||||
shard.keyBuf = encoding.MarshalBytes(shard.keyBuf[:0], bytesutil.ToUnsafeBytes(v))
|
||||
psg := shard.getPipeStatsGroup(shard.keyBuf)
|
||||
shard.stateSizeBudget -= psg.updateStatsForAllRows(br)
|
||||
return
|
||||
}
|
||||
|
||||
values := c.getBucketedValues(br, bf)
|
||||
if areConstValues(values) {
|
||||
// Fast path for column with constant values.
|
||||
shard.keyBuf = encoding.MarshalBytes(shard.keyBuf[:0], bytesutil.ToUnsafeBytes(values[0]))
|
||||
psg := shard.getPipeStatsGroup(shard.keyBuf)
|
||||
shard.stateSizeBudget -= psg.updateStatsForAllRows(br)
|
||||
return
|
||||
}
|
||||
|
||||
// Slower generic path for a column with different values.
|
||||
var psg *pipeStatsGroup
|
||||
keyBuf := shard.keyBuf[:0]
|
||||
for i := range br.timestamps {
|
||||
if i <= 0 || values[i-1] != values[i] {
|
||||
keyBuf = encoding.MarshalBytes(keyBuf[:0], bytesutil.ToUnsafeBytes(values[i]))
|
||||
psg = shard.getPipeStatsGroup(keyBuf)
|
||||
}
|
||||
shard.stateSizeBudget -= psg.updateStatsForRow(br, i)
|
||||
}
|
||||
shard.keyBuf = keyBuf
|
||||
return
|
||||
}
|
||||
|
||||
// Obtain columns for byFields
|
||||
columnValues := shard.columnValues[:0]
|
||||
for _, bf := range byFields {
|
||||
c := br.getColumnByName(bf.name)
|
||||
values := c.getBucketedValues(br, bf)
|
||||
columnValues = append(columnValues, values)
|
||||
}
|
||||
shard.columnValues = columnValues
|
||||
|
||||
// Verify whether all the 'by (...)' columns are constant.
|
||||
areAllConstColumns := true
|
||||
for _, values := range columnValues {
|
||||
if !areConstValues(values) {
|
||||
areAllConstColumns = false
|
||||
break
|
||||
}
|
||||
}
|
||||
if areAllConstColumns {
|
||||
// Fast path for constant 'by (...)' columns.
|
||||
keyBuf := shard.keyBuf[:0]
|
||||
for _, values := range columnValues {
|
||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(values[0]))
|
||||
}
|
||||
psg := shard.getPipeStatsGroup(keyBuf)
|
||||
shard.stateSizeBudget -= psg.updateStatsForAllRows(br)
|
||||
shard.keyBuf = keyBuf
|
||||
return
|
||||
}
|
||||
|
||||
// The slowest path - group by multiple columns with different values across rows.
|
||||
var psg *pipeStatsGroup
|
||||
keyBuf := shard.keyBuf[:0]
|
||||
for i := range br.timestamps {
|
||||
// Verify whether the key for 'by (...)' fields equals the previous key
|
||||
sameValue := i > 0
|
||||
for _, values := range columnValues {
|
||||
if i <= 0 || values[i-1] != values[i] {
|
||||
sameValue = false
|
||||
break
|
||||
}
|
||||
}
|
||||
if !sameValue {
|
||||
// Construct new key for the 'by (...)' fields
|
||||
keyBuf = keyBuf[:0]
|
||||
for _, values := range columnValues {
|
||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(values[i]))
|
||||
}
|
||||
psg = shard.getPipeStatsGroup(keyBuf)
|
||||
}
|
||||
shard.stateSizeBudget -= psg.updateStatsForRow(br, i)
|
||||
}
|
||||
shard.keyBuf = keyBuf
|
||||
}
|
||||
|
||||
func (shard *pipeStatsProcessorShard) getPipeStatsGroup(key []byte) *pipeStatsGroup {
|
||||
psg := shard.m[string(key)]
|
||||
if psg != nil {
|
||||
return psg
|
||||
}
|
||||
|
||||
sfps := make([]statsProcessor, len(shard.ps.funcs))
|
||||
for i, f := range shard.ps.funcs {
|
||||
sfp, stateSize := f.newStatsProcessor()
|
||||
sfps[i] = sfp
|
||||
shard.stateSizeBudget -= stateSize
|
||||
}
|
||||
psg = &pipeStatsGroup{
|
||||
sfps: sfps,
|
||||
}
|
||||
shard.m[string(key)] = psg
|
||||
shard.stateSizeBudget -= len(key) + int(unsafe.Sizeof("")+unsafe.Sizeof(psg)+unsafe.Sizeof(sfps[0])*uintptr(len(sfps)))
|
||||
|
||||
return psg
|
||||
}
|
||||
|
||||
type pipeStatsGroup struct {
|
||||
sfps []statsProcessor
|
||||
}
|
||||
|
||||
func (psg *pipeStatsGroup) updateStatsForAllRows(br *blockResult) int {
|
||||
n := 0
|
||||
for _, sfp := range psg.sfps {
|
||||
n += sfp.updateStatsForAllRows(br)
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func (psg *pipeStatsGroup) updateStatsForRow(br *blockResult, rowIdx int) int {
|
||||
n := 0
|
||||
for _, sfp := range psg.sfps {
|
||||
n += sfp.updateStatsForRow(br, rowIdx)
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func (psp *pipeStatsProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
shard := &psp.shards[workerID]
|
||||
|
||||
for shard.stateSizeBudget < 0 {
|
||||
// steal some budget for the state size from the global budget.
|
||||
remaining := psp.stateSizeBudget.Add(-stateSizeBudgetChunk)
|
||||
if remaining < 0 {
|
||||
// The state size is too big. Stop processing data in order to avoid OOM crash.
|
||||
if remaining+stateSizeBudgetChunk >= 0 {
|
||||
// Notify worker goroutines to stop calling writeBlock() in order to save CPU time.
|
||||
psp.cancel()
|
||||
}
|
||||
return
|
||||
}
|
||||
shard.stateSizeBudget += stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
shard.writeBlock(br)
|
||||
}
|
||||
|
||||
func (psp *pipeStatsProcessor) flush() error {
|
||||
if n := psp.stateSizeBudget.Load(); n <= 0 {
|
||||
return fmt.Errorf("cannot calculate [%s], since it requires more than %dMB of memory", psp.ps.String(), psp.maxStateSize/(1<<20))
|
||||
}
|
||||
|
||||
// Merge states across shards
|
||||
shards := psp.shards
|
||||
m := shards[0].m
|
||||
shards = shards[1:]
|
||||
for i := range shards {
|
||||
shard := &shards[i]
|
||||
for key, psg := range shard.m {
|
||||
// shard.m may be quite big, so this loop can take a lot of time and CPU.
|
||||
// Stop processing data as soon as stopCh is closed without wasting additional CPU time.
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
spgBase := m[key]
|
||||
if spgBase == nil {
|
||||
m[key] = psg
|
||||
} else {
|
||||
for i, sfp := range spgBase.sfps {
|
||||
sfp.mergeState(psg.sfps[i])
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Write per-group states to ppBase
|
||||
byFields := psp.ps.byFields
|
||||
if len(byFields) == 0 && len(m) == 0 {
|
||||
// Special case - zero matching rows.
|
||||
_ = shards[0].getPipeStatsGroup(nil)
|
||||
m = shards[0].m
|
||||
}
|
||||
|
||||
rcs := make([]resultColumn, 0, len(byFields)+len(psp.ps.resultNames))
|
||||
for _, bf := range byFields {
|
||||
rcs = append(rcs, resultColumn{
|
||||
name: bf.name,
|
||||
})
|
||||
}
|
||||
for _, resultName := range psp.ps.resultNames {
|
||||
rcs = append(rcs, resultColumn{
|
||||
name: resultName,
|
||||
})
|
||||
}
|
||||
var br blockResult
|
||||
|
||||
var values []string
|
||||
valuesLen := 0
|
||||
for key, psg := range m {
|
||||
// m may be quite big, so this loop can take a lot of time and CPU.
|
||||
// Stop processing data as soon as stopCh is closed without wasting additional CPU time.
|
||||
select {
|
||||
case <-psp.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
// Unmarshal values for byFields from key.
|
||||
values = values[:0]
|
||||
keyBuf := bytesutil.ToUnsafeBytes(key)
|
||||
for len(keyBuf) > 0 {
|
||||
tail, v, err := encoding.UnmarshalBytes(keyBuf)
|
||||
if err != nil {
|
||||
logger.Panicf("BUG: cannot unmarshal value from keyBuf=%q: %w", keyBuf, err)
|
||||
}
|
||||
values = append(values, bytesutil.ToUnsafeString(v))
|
||||
keyBuf = tail
|
||||
}
|
||||
if len(values) != len(byFields) {
|
||||
logger.Panicf("BUG: unexpected number of values decoded from keyBuf; got %d; want %d", len(values), len(byFields))
|
||||
}
|
||||
|
||||
// calculate values for stats functions
|
||||
for _, sfp := range psg.sfps {
|
||||
value := sfp.finalizeStats()
|
||||
values = append(values, value)
|
||||
}
|
||||
|
||||
if len(values) != len(rcs) {
|
||||
logger.Panicf("BUG: len(values)=%d must be equal to len(rcs)=%d", len(values), len(rcs))
|
||||
}
|
||||
for i, v := range values {
|
||||
rcs[i].addValue(v)
|
||||
valuesLen += len(v)
|
||||
}
|
||||
if valuesLen >= 1_000_000 {
|
||||
br.setResultColumns(rcs)
|
||||
psp.ppBase.writeBlock(0, &br)
|
||||
br.reset()
|
||||
for i := range rcs {
|
||||
rcs[i].resetKeepName()
|
||||
}
|
||||
valuesLen = 0
|
||||
}
|
||||
}
|
||||
|
||||
br.setResultColumns(rcs)
|
||||
psp.ppBase.writeBlock(0, &br)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func parsePipeStats(lex *lexer) (*pipeStats, error) {
|
||||
if !lex.isKeyword("stats") {
|
||||
return nil, fmt.Errorf("expecting 'stats'; got %q", lex.token)
|
||||
}
|
||||
|
||||
lex.nextToken()
|
||||
|
||||
var ps pipeStats
|
||||
if lex.isKeyword("by") {
|
||||
lex.nextToken()
|
||||
bfs, err := parseByStatsFields(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'by' clause: %w", err)
|
||||
}
|
||||
ps.byFields = bfs
|
||||
}
|
||||
|
||||
var resultNames []string
|
||||
var funcs []statsFunc
|
||||
for {
|
||||
sf, resultName, err := parseStatsFunc(lex)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
resultNames = append(resultNames, resultName)
|
||||
funcs = append(funcs, sf)
|
||||
if lex.isKeyword("|", ")", "") {
|
||||
ps.resultNames = resultNames
|
||||
ps.funcs = funcs
|
||||
return &ps, nil
|
||||
}
|
||||
if !lex.isKeyword(",") {
|
||||
return nil, fmt.Errorf("unexpected token %q; want ',', '|' or ')'", lex.token)
|
||||
}
|
||||
lex.nextToken()
|
||||
}
|
||||
}
|
||||
|
||||
func parseStatsFunc(lex *lexer) (statsFunc, string, error) {
|
||||
var sf statsFunc
|
||||
switch {
|
||||
case lex.isKeyword("count"):
|
||||
scs, err := parseStatsCount(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'count' func: %w", err)
|
||||
}
|
||||
sf = scs
|
||||
case lex.isKeyword("count_empty"):
|
||||
scs, err := parseStatsCountEmpty(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'count_empty' func: %w", err)
|
||||
}
|
||||
sf = scs
|
||||
case lex.isKeyword("count_uniq"):
|
||||
sus, err := parseStatsCountUniq(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'count_uniq' func: %w", err)
|
||||
}
|
||||
sf = sus
|
||||
case lex.isKeyword("sum"):
|
||||
sss, err := parseStatsSum(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'sum' func: %w", err)
|
||||
}
|
||||
sf = sss
|
||||
case lex.isKeyword("max"):
|
||||
sms, err := parseStatsMax(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'max' func: %w", err)
|
||||
}
|
||||
sf = sms
|
||||
case lex.isKeyword("min"):
|
||||
sms, err := parseStatsMin(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'min' func: %w", err)
|
||||
}
|
||||
sf = sms
|
||||
case lex.isKeyword("avg"):
|
||||
sas, err := parseStatsAvg(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'avg' func: %w", err)
|
||||
}
|
||||
sf = sas
|
||||
case lex.isKeyword("uniq_values"):
|
||||
sus, err := parseStatsUniqValues(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'uniq_values' func: %w", err)
|
||||
}
|
||||
sf = sus
|
||||
case lex.isKeyword("values"):
|
||||
svs, err := parseStatsValues(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse 'values' func: %w", err)
|
||||
}
|
||||
sf = svs
|
||||
default:
|
||||
return nil, "", fmt.Errorf("unknown stats func %q", lex.token)
|
||||
}
|
||||
|
||||
resultName, err := parseResultName(lex)
|
||||
if err != nil {
|
||||
return nil, "", fmt.Errorf("cannot parse result name for %s: %w", sf, err)
|
||||
}
|
||||
return sf, resultName, nil
|
||||
}
|
||||
|
||||
func parseResultName(lex *lexer) (string, error) {
|
||||
if lex.isKeyword("as") {
|
||||
lex.nextToken()
|
||||
}
|
||||
resultName, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return resultName, nil
|
||||
}
|
||||
|
||||
var zeroByStatsField = &byStatsField{}
|
||||
|
||||
// byStatsField represents 'by (...)' part of the pipeStats.
|
||||
//
|
||||
// It can have either 'name' representation or 'name:bucket' or 'name:buket offset off' representation,
|
||||
// where `bucket` and `off` can contain duration, size or numeric value for creating different buckets
|
||||
// for 'value/bucket'.
|
||||
type byStatsField struct {
|
||||
name string
|
||||
|
||||
// bucketSizeStr is string representation of the bucket size
|
||||
bucketSizeStr string
|
||||
|
||||
// bucketSize is the bucket for grouping the given field values with value/bucketSize calculations
|
||||
bucketSize float64
|
||||
|
||||
// bucketOffsetStr is string representation of the offset for bucketSize
|
||||
bucketOffsetStr string
|
||||
|
||||
// bucketOffset is the offset for bucketSize
|
||||
bucketOffset float64
|
||||
}
|
||||
|
||||
func (bf *byStatsField) String() string {
|
||||
s := quoteTokenIfNeeded(bf.name)
|
||||
if bf.bucketSizeStr != "" {
|
||||
s += ":" + bf.bucketSizeStr
|
||||
if bf.bucketOffsetStr != "" {
|
||||
s += " offset " + bf.bucketOffsetStr
|
||||
}
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (bf *byStatsField) hasBucketConfig() bool {
|
||||
return len(bf.bucketSizeStr) > 0 || len(bf.bucketOffsetStr) > 0
|
||||
}
|
||||
|
||||
func parseByStatsFields(lex *lexer) ([]*byStatsField, error) {
|
||||
if !lex.isKeyword("(") {
|
||||
return nil, fmt.Errorf("missing `(`")
|
||||
}
|
||||
var bfs []*byStatsField
|
||||
for {
|
||||
lex.nextToken()
|
||||
if lex.isKeyword(")") {
|
||||
lex.nextToken()
|
||||
return bfs, nil
|
||||
}
|
||||
fieldName, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse field name: %w", err)
|
||||
}
|
||||
bf := &byStatsField{
|
||||
name: fieldName,
|
||||
}
|
||||
if lex.isKeyword(":") {
|
||||
// Parse bucket size
|
||||
lex.nextToken()
|
||||
bucketSizeStr := lex.token
|
||||
lex.nextToken()
|
||||
if bucketSizeStr == "/" {
|
||||
bucketSizeStr += lex.token
|
||||
lex.nextToken()
|
||||
}
|
||||
if bucketSizeStr != "year" && bucketSizeStr != "month" {
|
||||
bucketSize, ok := tryParseBucketSize(bucketSizeStr)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse bucket size for field %q: %q", fieldName, bucketSizeStr)
|
||||
}
|
||||
bf.bucketSize = bucketSize
|
||||
}
|
||||
bf.bucketSizeStr = bucketSizeStr
|
||||
|
||||
// Parse bucket offset
|
||||
if lex.isKeyword("offset") {
|
||||
lex.nextToken()
|
||||
bucketOffsetStr := lex.token
|
||||
lex.nextToken()
|
||||
if bucketOffsetStr == "-" {
|
||||
bucketOffsetStr += lex.token
|
||||
lex.nextToken()
|
||||
}
|
||||
bucketOffset, ok := tryParseBucketOffset(bucketOffsetStr)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse bucket offset for field %q: %q", fieldName, bucketOffsetStr)
|
||||
}
|
||||
bf.bucketOffsetStr = bucketOffsetStr
|
||||
bf.bucketOffset = bucketOffset
|
||||
}
|
||||
}
|
||||
bfs = append(bfs, bf)
|
||||
switch {
|
||||
case lex.isKeyword(")"):
|
||||
lex.nextToken()
|
||||
return bfs, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// tryParseBucketOffset tries parsing bucket offset, which can have the following formats:
|
||||
//
|
||||
// - integer number: 12345
|
||||
// - floating-point number: 1.2345
|
||||
// - duration: 1.5s - it is converted to nanoseconds
|
||||
// - bytes: 1.5KiB
|
||||
func tryParseBucketOffset(s string) (float64, bool) {
|
||||
// Try parsing s as floating point number
|
||||
if f, ok := tryParseFloat64(s); ok {
|
||||
return f, true
|
||||
}
|
||||
|
||||
// Try parsing s as duration (1s, 5m, etc.)
|
||||
if nsecs, ok := tryParseDuration(s); ok {
|
||||
return float64(nsecs), true
|
||||
}
|
||||
|
||||
// Try parsing s as bytes (KiB, MB, etc.)
|
||||
if n, ok := tryParseBytes(s); ok {
|
||||
return float64(n), true
|
||||
}
|
||||
|
||||
return 0, false
|
||||
}
|
||||
|
||||
// tryParseBucketSize tries parsing bucket size, which can have the following formats:
|
||||
//
|
||||
// - integer number: 12345
|
||||
// - floating-point number: 1.2345
|
||||
// - duration: 1.5s - it is converted to nanoseconds
|
||||
// - bytes: 1.5KiB
|
||||
// - ipv4 mask: /24
|
||||
func tryParseBucketSize(s string) (float64, bool) {
|
||||
switch s {
|
||||
case "nanosecond":
|
||||
return 1, true
|
||||
case "microsecond":
|
||||
return nsecsPerMicrosecond, true
|
||||
case "millisecond":
|
||||
return nsecsPerMillisecond, true
|
||||
case "second":
|
||||
return nsecsPerSecond, true
|
||||
case "minute":
|
||||
return nsecsPerMinute, true
|
||||
case "hour":
|
||||
return nsecsPerHour, true
|
||||
case "day":
|
||||
return nsecsPerDay, true
|
||||
case "week":
|
||||
return nsecsPerWeek, true
|
||||
}
|
||||
|
||||
// Try parsing s as floating point number
|
||||
if f, ok := tryParseFloat64(s); ok {
|
||||
return f, true
|
||||
}
|
||||
|
||||
// Try parsing s as duration (1s, 5m, etc.)
|
||||
if nsecs, ok := tryParseDuration(s); ok {
|
||||
return float64(nsecs), true
|
||||
}
|
||||
|
||||
// Try parsing s as bytes (KiB, MB, etc.)
|
||||
if n, ok := tryParseBytes(s); ok {
|
||||
return float64(n), true
|
||||
}
|
||||
|
||||
if n, ok := tryParseIPv4Mask(s); ok {
|
||||
return float64(n), true
|
||||
}
|
||||
|
||||
return 0, false
|
||||
}
|
||||
|
||||
// parseFieldNamesForStatsFunc parses field names for statsFunc.
|
||||
//
|
||||
// It returns ["*"] if the fields names list is empty or if it contains "*" field.
|
||||
func parseFieldNamesForStatsFunc(lex *lexer, funcName string) ([]string, error) {
|
||||
if !lex.isKeyword(funcName) {
|
||||
return nil, fmt.Errorf("unexpected func; got %q; want %q", lex.token, funcName)
|
||||
}
|
||||
lex.nextToken()
|
||||
fields, err := parseFieldNamesInParens(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse %q args: %w", funcName, err)
|
||||
}
|
||||
if len(fields) == 0 || slices.Contains(fields, "*") {
|
||||
fields = []string{"*"}
|
||||
}
|
||||
return fields, nil
|
||||
}
|
||||
|
||||
func parseFieldNamesInParens(lex *lexer) ([]string, error) {
|
||||
if !lex.isKeyword("(") {
|
||||
return nil, fmt.Errorf("missing `(`")
|
||||
}
|
||||
var fields []string
|
||||
for {
|
||||
lex.nextToken()
|
||||
if lex.isKeyword(")") {
|
||||
lex.nextToken()
|
||||
return fields, nil
|
||||
}
|
||||
if lex.isKeyword(",") {
|
||||
return nil, fmt.Errorf("unexpected `,`")
|
||||
}
|
||||
field, err := parseFieldName(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse field name: %w", err)
|
||||
}
|
||||
fields = append(fields, field)
|
||||
switch {
|
||||
case lex.isKeyword(")"):
|
||||
lex.nextToken()
|
||||
return fields, nil
|
||||
case lex.isKeyword(","):
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected token: %q; expecting ',' or ')'", lex.token)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func parseFieldName(lex *lexer) (string, error) {
|
||||
if lex.isKeyword(",", "(", ")", "[", "]", "|", ":", "") {
|
||||
return "", fmt.Errorf("unexpected token: %q", lex.token)
|
||||
}
|
||||
fieldName := getCompoundPhrase(lex, false)
|
||||
fieldName = getCanonicalColumnName(fieldName)
|
||||
return fieldName, nil
|
||||
}
|
||||
|
||||
func fieldNamesString(fields []string) string {
|
||||
a := make([]string, len(fields))
|
||||
for i, f := range fields {
|
||||
if f != "*" {
|
||||
f = quoteTokenIfNeeded(f)
|
||||
}
|
||||
a[i] = f
|
||||
}
|
||||
return strings.Join(a, ", ")
|
||||
}
|
||||
|
||||
func areConstValues(values []string) bool {
|
||||
if len(values) == 0 {
|
||||
return false
|
||||
}
|
||||
v := values[0]
|
||||
for i := 1; i < len(values); i++ {
|
||||
if v != values[i] {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
65
lib/logstorage/pipe_stats_test.go
Normal file
65
lib/logstorage/pipe_stats_test.go
Normal file
|
@ -0,0 +1,65 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeStatsUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeStats(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error when parsing %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("stats count() r1", "*", "", "", "")
|
||||
f("stats count(*) r1", "*", "", "", "")
|
||||
f("stats count(f1,f2) r1", "*", "", "f1,f2", "")
|
||||
f("stats count(f1,f2) r1, sum(f3,f4) r2", "*", "", "f1,f2,f3,f4", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "*", "", "b1,b2,f1,f2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1, count(f1,f3) r2", "*", "", "b1,b2,f1,f2,f3", "")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with stats fields
|
||||
f("stats count() r1", "*", "f1,f2", "", "")
|
||||
f("stats count(*) r1", "*", "f1,f2", "", "")
|
||||
f("stats count(f1,f2) r1", "*", "f3,f4", "f1,f2", "")
|
||||
f("stats count(f1,f2) r1, sum(f3,f4) r2", "*", "f5,f6", "f1,f2,f3,f4", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "*", "f3,f4", "b1,b2,f1,f2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1, count(f1,f3) r2", "*", "f4,f5", "b1,b2,f1,f2,f3", "")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with stats fields
|
||||
f("stats count() r1", "*", "r1,r2", "", "")
|
||||
f("stats count(*) r1", "*", "r1,r2", "", "")
|
||||
f("stats count(f1,f2) r1", "*", "r1,r2", "", "")
|
||||
f("stats count(f1,f2) r1, sum(f3,f4) r2", "*", "r1,r3", "f3,f4", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "*", "r1,r2", "b1,b2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "*", "r1,r2,b1", "b1,b2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "*", "r1,r2,b1,b2", "", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1, count(f1,f3) r2", "*", "r1,r3", "b1,b2,f1,f3", "")
|
||||
|
||||
// needed fields do not intersect with stats fields
|
||||
f("stats count() r1", "r2", "", "", "")
|
||||
f("stats count(*) r1", "r2", "", "", "")
|
||||
f("stats count(f1,f2) r1", "r2", "", "", "")
|
||||
f("stats count(f1,f2) r1, sum(f3,f4) r2", "r3", "", "", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "r2", "", "", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1, count(f1,f3) r2", "r3", "", "", "")
|
||||
|
||||
// needed fields intersect with stats fields
|
||||
f("stats count() r1", "r1,r2", "", "", "")
|
||||
f("stats count(*) r1", "r1,r2", "", "", "")
|
||||
f("stats count(f1,f2) r1", "r1,r2", "", "f1,f2", "")
|
||||
f("stats count(f1,f2) r1, sum(f3,f4) r2", "r1,r3", "", "f1,f2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1", "r1,r2", "", "b1,b2,f1,f2", "")
|
||||
f("stats by (b1,b2) count(f1,f2) r1, count(f1,f3) r2", "r1,r3", "", "b1,b2,f1,f2", "")
|
||||
}
|
392
lib/logstorage/pipe_uniq.go
Normal file
392
lib/logstorage/pipe_uniq.go
Normal file
|
@ -0,0 +1,392 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"slices"
|
||||
"sync/atomic"
|
||||
"unsafe"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
|
||||
)
|
||||
|
||||
// pipeUniq processes '| uniq ...' queries.
|
||||
//
|
||||
// See https://docs.victoriametrics.com/victorialogs/logsql/#uniq-pipe
|
||||
type pipeUniq struct {
|
||||
// fields contains field names for returning unique values
|
||||
byFields []string
|
||||
|
||||
limit uint64
|
||||
}
|
||||
|
||||
func (pu *pipeUniq) String() string {
|
||||
s := "uniq"
|
||||
if len(pu.byFields) > 0 {
|
||||
s += " by (" + fieldNamesString(pu.byFields) + ")"
|
||||
}
|
||||
if pu.limit > 0 {
|
||||
s += fmt.Sprintf(" limit %d", pu.limit)
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func (pu *pipeUniq) updateNeededFields(neededFields, unneededFields fieldsSet) {
|
||||
neededFields.reset()
|
||||
unneededFields.reset()
|
||||
|
||||
if len(pu.byFields) == 0 {
|
||||
neededFields.add("*")
|
||||
} else {
|
||||
neededFields.addAll(pu.byFields)
|
||||
}
|
||||
}
|
||||
|
||||
func (pu *pipeUniq) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppBase pipeProcessor) pipeProcessor {
|
||||
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
|
||||
|
||||
shards := make([]pipeUniqProcessorShard, workersCount)
|
||||
for i := range shards {
|
||||
shard := &shards[i]
|
||||
shard.pu = pu
|
||||
shard.m = make(map[string]struct{})
|
||||
shard.stateSizeBudget = stateSizeBudgetChunk
|
||||
maxStateSize -= stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
pup := &pipeUniqProcessor{
|
||||
pu: pu,
|
||||
stopCh: stopCh,
|
||||
cancel: cancel,
|
||||
ppBase: ppBase,
|
||||
|
||||
shards: shards,
|
||||
|
||||
maxStateSize: maxStateSize,
|
||||
}
|
||||
pup.stateSizeBudget.Store(maxStateSize)
|
||||
|
||||
return pup
|
||||
}
|
||||
|
||||
type pipeUniqProcessor struct {
|
||||
pu *pipeUniq
|
||||
stopCh <-chan struct{}
|
||||
cancel func()
|
||||
ppBase pipeProcessor
|
||||
|
||||
shards []pipeUniqProcessorShard
|
||||
|
||||
maxStateSize int64
|
||||
stateSizeBudget atomic.Int64
|
||||
}
|
||||
|
||||
type pipeUniqProcessorShard struct {
|
||||
pipeUniqProcessorShardNopad
|
||||
|
||||
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
|
||||
_ [128 - unsafe.Sizeof(pipeUniqProcessorShardNopad{})%128]byte
|
||||
}
|
||||
|
||||
type pipeUniqProcessorShardNopad struct {
|
||||
// pu points to the parent pipeUniq.
|
||||
pu *pipeUniq
|
||||
|
||||
// m holds unique rows.
|
||||
m map[string]struct{}
|
||||
|
||||
// keyBuf is a temporary buffer for building keys for m.
|
||||
keyBuf []byte
|
||||
|
||||
// columnValues is a temporary buffer for the processed column values.
|
||||
columnValues [][]string
|
||||
|
||||
// stateSizeBudget is the remaining budget for the whole state size for the shard.
|
||||
// The per-shard budget is provided in chunks from the parent pipeUniqProcessor.
|
||||
stateSizeBudget int
|
||||
}
|
||||
|
||||
// writeBlock writes br to shard.
|
||||
//
|
||||
// It returns false if the block cannot be written because of the exceeded limit.
|
||||
func (shard *pipeUniqProcessorShard) writeBlock(br *blockResult) bool {
|
||||
if limit := shard.pu.limit; limit > 0 && uint64(len(shard.m)) >= limit {
|
||||
return false
|
||||
}
|
||||
|
||||
m := shard.m
|
||||
byFields := shard.pu.byFields
|
||||
if len(byFields) == 0 {
|
||||
// Take into account all the columns in br.
|
||||
keyBuf := shard.keyBuf
|
||||
cs := br.getColumns()
|
||||
for i := range br.timestamps {
|
||||
keyBuf = keyBuf[:0]
|
||||
for _, c := range cs {
|
||||
v := c.getValueAtRow(br, i)
|
||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(c.name))
|
||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(v))
|
||||
}
|
||||
if _, ok := m[string(keyBuf)]; !ok {
|
||||
m[string(keyBuf)] = struct{}{}
|
||||
shard.stateSizeBudget -= len(keyBuf) + int(unsafe.Sizeof(""))
|
||||
}
|
||||
}
|
||||
shard.keyBuf = keyBuf
|
||||
return true
|
||||
}
|
||||
|
||||
// Take into account only the selected columns.
|
||||
columnValues := shard.columnValues[:0]
|
||||
for _, f := range byFields {
|
||||
c := br.getColumnByName(f)
|
||||
columnValues = append(columnValues, c.getValues(br))
|
||||
}
|
||||
shard.columnValues = columnValues
|
||||
|
||||
keyBuf := shard.keyBuf
|
||||
for i := range br.timestamps {
|
||||
seenValue := true
|
||||
for _, values := range columnValues {
|
||||
if i == 0 || values[i-1] != values[i] {
|
||||
seenValue = false
|
||||
break
|
||||
}
|
||||
}
|
||||
if seenValue {
|
||||
continue
|
||||
}
|
||||
|
||||
keyBuf = keyBuf[:0]
|
||||
for _, values := range columnValues {
|
||||
keyBuf = encoding.MarshalBytes(keyBuf, bytesutil.ToUnsafeBytes(values[i]))
|
||||
}
|
||||
if _, ok := m[string(keyBuf)]; !ok {
|
||||
m[string(keyBuf)] = struct{}{}
|
||||
shard.stateSizeBudget -= len(keyBuf) + int(unsafe.Sizeof(""))
|
||||
}
|
||||
}
|
||||
shard.keyBuf = keyBuf
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (pup *pipeUniqProcessor) writeBlock(workerID uint, br *blockResult) {
|
||||
if len(br.timestamps) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
shard := &pup.shards[workerID]
|
||||
|
||||
for shard.stateSizeBudget < 0 {
|
||||
// steal some budget for the state size from the global budget.
|
||||
remaining := pup.stateSizeBudget.Add(-stateSizeBudgetChunk)
|
||||
if remaining < 0 {
|
||||
// The state size is too big. Stop processing data in order to avoid OOM crash.
|
||||
if remaining+stateSizeBudgetChunk >= 0 {
|
||||
// Notify worker goroutines to stop calling writeBlock() in order to save CPU time.
|
||||
pup.cancel()
|
||||
}
|
||||
return
|
||||
}
|
||||
shard.stateSizeBudget += stateSizeBudgetChunk
|
||||
}
|
||||
|
||||
if !shard.writeBlock(br) {
|
||||
pup.cancel()
|
||||
}
|
||||
}
|
||||
|
||||
func (pup *pipeUniqProcessor) flush() error {
|
||||
if n := pup.stateSizeBudget.Load(); n <= 0 {
|
||||
return fmt.Errorf("cannot calculate [%s], since it requires more than %dMB of memory", pup.pu.String(), pup.maxStateSize/(1<<20))
|
||||
}
|
||||
|
||||
// merge state across shards
|
||||
shards := pup.shards
|
||||
m := shards[0].m
|
||||
shards = shards[1:]
|
||||
for i := range shards {
|
||||
select {
|
||||
case <-pup.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
for k := range shards[i].m {
|
||||
m[k] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
// write result
|
||||
wctx := &pipeUniqWriteContext{
|
||||
pup: pup,
|
||||
}
|
||||
byFields := pup.pu.byFields
|
||||
var rowFields []Field
|
||||
|
||||
if len(byFields) == 0 {
|
||||
for k := range m {
|
||||
select {
|
||||
case <-pup.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
rowFields = rowFields[:0]
|
||||
keyBuf := bytesutil.ToUnsafeBytes(k)
|
||||
for len(keyBuf) > 0 {
|
||||
tail, name, err := encoding.UnmarshalBytes(keyBuf)
|
||||
if err != nil {
|
||||
logger.Panicf("BUG: cannot unmarshal field name: %s", err)
|
||||
}
|
||||
keyBuf = tail
|
||||
|
||||
tail, value, err := encoding.UnmarshalBytes(keyBuf)
|
||||
if err != nil {
|
||||
logger.Panicf("BUG: cannot unmarshal field value: %s", err)
|
||||
}
|
||||
keyBuf = tail
|
||||
|
||||
rowFields = append(rowFields, Field{
|
||||
Name: bytesutil.ToUnsafeString(name),
|
||||
Value: bytesutil.ToUnsafeString(value),
|
||||
})
|
||||
}
|
||||
wctx.writeRow(rowFields)
|
||||
}
|
||||
} else {
|
||||
for k := range m {
|
||||
select {
|
||||
case <-pup.stopCh:
|
||||
return nil
|
||||
default:
|
||||
}
|
||||
|
||||
rowFields = rowFields[:0]
|
||||
keyBuf := bytesutil.ToUnsafeBytes(k)
|
||||
fieldIdx := 0
|
||||
for len(keyBuf) > 0 {
|
||||
tail, value, err := encoding.UnmarshalBytes(keyBuf)
|
||||
if err != nil {
|
||||
logger.Panicf("BUG: cannot unmarshal field value: %s", err)
|
||||
}
|
||||
keyBuf = tail
|
||||
|
||||
rowFields = append(rowFields, Field{
|
||||
Name: byFields[fieldIdx],
|
||||
Value: bytesutil.ToUnsafeString(value),
|
||||
})
|
||||
fieldIdx++
|
||||
}
|
||||
wctx.writeRow(rowFields)
|
||||
}
|
||||
}
|
||||
|
||||
wctx.flush()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type pipeUniqWriteContext struct {
|
||||
pup *pipeUniqProcessor
|
||||
rcs []resultColumn
|
||||
br blockResult
|
||||
|
||||
rowsWritten uint64
|
||||
|
||||
valuesLen int
|
||||
}
|
||||
|
||||
func (wctx *pipeUniqWriteContext) writeRow(rowFields []Field) {
|
||||
if limit := wctx.pup.pu.limit; limit > 0 && wctx.rowsWritten >= limit {
|
||||
return
|
||||
}
|
||||
wctx.rowsWritten++
|
||||
|
||||
rcs := wctx.rcs
|
||||
|
||||
areEqualColumns := len(rcs) == len(rowFields)
|
||||
if areEqualColumns {
|
||||
for i, f := range rowFields {
|
||||
if rcs[i].name != f.Name {
|
||||
areEqualColumns = false
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !areEqualColumns {
|
||||
// send the current block to bbBase and construct a block with new set of columns
|
||||
wctx.flush()
|
||||
|
||||
rcs = wctx.rcs[:0]
|
||||
for _, f := range rowFields {
|
||||
rcs = append(rcs, resultColumn{
|
||||
name: f.Name,
|
||||
})
|
||||
}
|
||||
wctx.rcs = rcs
|
||||
}
|
||||
|
||||
for i, f := range rowFields {
|
||||
v := f.Value
|
||||
rcs[i].addValue(v)
|
||||
wctx.valuesLen += len(v)
|
||||
}
|
||||
if wctx.valuesLen >= 1_000_000 {
|
||||
wctx.flush()
|
||||
}
|
||||
}
|
||||
|
||||
func (wctx *pipeUniqWriteContext) flush() {
|
||||
rcs := wctx.rcs
|
||||
br := &wctx.br
|
||||
|
||||
wctx.valuesLen = 0
|
||||
|
||||
if len(rcs) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// Flush rcs to ppBase
|
||||
br.setResultColumns(rcs)
|
||||
wctx.pup.ppBase.writeBlock(0, br)
|
||||
br.reset()
|
||||
for i := range rcs {
|
||||
rcs[i].resetKeepName()
|
||||
}
|
||||
}
|
||||
|
||||
func parsePipeUniq(lex *lexer) (*pipeUniq, error) {
|
||||
if !lex.isKeyword("uniq") {
|
||||
return nil, fmt.Errorf("expecting 'uniq'; got %q", lex.token)
|
||||
}
|
||||
lex.nextToken()
|
||||
|
||||
var pu pipeUniq
|
||||
if lex.isKeyword("by") {
|
||||
lex.nextToken()
|
||||
bfs, err := parseFieldNamesInParens(lex)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot parse 'by' clause: %w", err)
|
||||
}
|
||||
if slices.Contains(bfs, "*") {
|
||||
bfs = nil
|
||||
}
|
||||
pu.byFields = bfs
|
||||
}
|
||||
|
||||
if lex.isKeyword("limit") {
|
||||
lex.nextToken()
|
||||
n, ok := tryParseUint64(lex.token)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("cannot parse 'limit %s'", lex.token)
|
||||
}
|
||||
lex.nextToken()
|
||||
pu.limit = n
|
||||
}
|
||||
|
||||
return &pu, nil
|
||||
}
|
45
lib/logstorage/pipe_uniq_test.go
Normal file
45
lib/logstorage/pipe_uniq_test.go
Normal file
|
@ -0,0 +1,45 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestPipeUniqUpdateNeededFields(t *testing.T) {
|
||||
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
|
||||
t.Helper()
|
||||
|
||||
nfs := newTestFieldsSet(neededFields)
|
||||
unfs := newTestFieldsSet(unneededFields)
|
||||
|
||||
lex := newLexer(s)
|
||||
p, err := parsePipeUniq(lex)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot parse %s: %s", s, err)
|
||||
}
|
||||
p.updateNeededFields(nfs, unfs)
|
||||
|
||||
assertNeededFields(t, nfs, unfs, neededFieldsExpected, unneededFieldsExpected)
|
||||
}
|
||||
|
||||
// all the needed fields
|
||||
f("uniq", "*", "", "*", "")
|
||||
f("uniq by()", "*", "", "*", "")
|
||||
f("uniq by(*)", "*", "", "*", "")
|
||||
f("uniq by(f1,f2)", "*", "", "f1,f2", "")
|
||||
|
||||
// all the needed fields, unneeded fields do not intersect with src
|
||||
f("uniq by(s1, s2)", "*", "f1,f2", "s1,s2", "")
|
||||
f("uniq", "*", "f1,f2", "*", "")
|
||||
|
||||
// all the needed fields, unneeded fields intersect with src
|
||||
f("uniq by(s1, s2)", "*", "s1,f1,f2", "s1,s2", "")
|
||||
f("uniq by(*)", "*", "s1,f1,f2", "*", "")
|
||||
f("uniq by(s1, s2)", "*", "s1,s2,f1", "s1,s2", "")
|
||||
|
||||
// needed fields do not intersect with src
|
||||
f("uniq by (s1, s2)", "f1,f2", "", "s1,s2", "")
|
||||
|
||||
// needed fields intersect with src
|
||||
f("uniq by (s1, s2)", "s1,f1,f2", "", "s1,s2", "")
|
||||
f("uniq by (*)", "s1,f1,f2", "", "*", "")
|
||||
}
|
|
@ -24,10 +24,7 @@ func (f *Field) Reset() {
|
|||
|
||||
// String returns string representation of f.
|
||||
func (f *Field) String() string {
|
||||
name := f.Name
|
||||
if name == "" {
|
||||
name = "_msg"
|
||||
}
|
||||
name := getCanonicalColumnName(f.Name)
|
||||
return fmt.Sprintf("%q:%q", name, f.Value)
|
||||
}
|
||||
|
||||
|
@ -37,7 +34,7 @@ func (f *Field) marshal(dst []byte) []byte {
|
|||
return dst
|
||||
}
|
||||
|
||||
func (f *Field) unmarshal(src []byte) ([]byte, error) {
|
||||
func (f *Field) unmarshal(a *arena, src []byte) ([]byte, error) {
|
||||
srcOrig := src
|
||||
|
||||
// Unmarshal field name
|
||||
|
@ -45,8 +42,7 @@ func (f *Field) unmarshal(src []byte) ([]byte, error) {
|
|||
if err != nil {
|
||||
return srcOrig, fmt.Errorf("cannot unmarshal field name: %w", err)
|
||||
}
|
||||
// Do not use bytesutil.InternBytes(b) here, since it works slower than the string(b) in prod
|
||||
f.Name = string(b)
|
||||
f.Name = a.copyBytesToString(b)
|
||||
src = tail
|
||||
|
||||
// Unmarshal field value
|
||||
|
@ -54,13 +50,22 @@ func (f *Field) unmarshal(src []byte) ([]byte, error) {
|
|||
if err != nil {
|
||||
return srcOrig, fmt.Errorf("cannot unmarshal field value: %w", err)
|
||||
}
|
||||
// Do not use bytesutil.InternBytes(b) here, since it works slower than the string(b) in prod
|
||||
f.Value = string(b)
|
||||
f.Value = a.copyBytesToString(b)
|
||||
src = tail
|
||||
|
||||
return src, nil
|
||||
}
|
||||
|
||||
func appendFields(a *arena, dst, src []Field) []Field {
|
||||
for _, f := range src {
|
||||
dst = append(dst, Field{
|
||||
Name: a.copyString(f.Name),
|
||||
Value: a.copyString(f.Value),
|
||||
})
|
||||
}
|
||||
return dst
|
||||
}
|
||||
|
||||
// rows is an aux structure used during rows merge
|
||||
type rows struct {
|
||||
fieldsBuf []Field
|
||||
|
@ -121,3 +126,10 @@ func (rs *rows) mergeRows(timestampsA, timestampsB []int64, fieldsA, fieldsB [][
|
|||
rs.appendRows(timestampsA, fieldsA)
|
||||
}
|
||||
}
|
||||
|
||||
func getCanonicalColumnName(columnName string) string {
|
||||
if columnName == "" {
|
||||
return "_msg"
|
||||
}
|
||||
return columnName
|
||||
}
|
||||
|
|
102
lib/logstorage/stats_avg.go
Normal file
102
lib/logstorage/stats_avg.go
Normal file
|
@ -0,0 +1,102 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"math"
|
||||
"slices"
|
||||
"strconv"
|
||||
"unsafe"
|
||||
)
|
||||
|
||||
type statsAvg struct {
|
||||
fields []string
|
||||
containsStar bool
|
||||
}
|
||||
|
||||
func (sa *statsAvg) String() string {
|
||||
return "avg(" + fieldNamesString(sa.fields) + ")"
|
||||
}
|
||||
|
||||
func (sa *statsAvg) neededFields() []string {
|
||||
return sa.fields
|
||||
}
|
||||
|
||||
func (sa *statsAvg) newStatsProcessor() (statsProcessor, int) {
|
||||
sap := &statsAvgProcessor{
|
||||
sa: sa,
|
||||
}
|
||||
return sap, int(unsafe.Sizeof(*sap))
|
||||
}
|
||||
|
||||
type statsAvgProcessor struct {
|
||||
sa *statsAvg
|
||||
|
||||
sum float64
|
||||
count uint64
|
||||
}
|
||||
|
||||
func (sap *statsAvgProcessor) updateStatsForAllRows(br *blockResult) int {
|
||||
if sap.sa.containsStar {
|
||||
// Scan all the columns
|
||||
for _, c := range br.getColumns() {
|
||||
f, count := c.sumValues(br)
|
||||
sap.sum += f
|
||||
sap.count += uint64(count)
|
||||
}
|
||||
} else {
|
||||
// Scan the requested columns
|
||||
for _, field := range sap.sa.fields {
|
||||
c := br.getColumnByName(field)
|
||||
f, count := c.sumValues(br)
|
||||
sap.sum += f
|
||||
sap.count += uint64(count)
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (sap *statsAvgProcessor) updateStatsForRow(br *blockResult, rowIdx int) int {
|
||||
if sap.sa.containsStar {
|
||||
// Scan all the fields for the given row
|
||||
for _, c := range br.getColumns() {
|
||||
f := c.getFloatValueAtRow(rowIdx)
|
||||
if !math.IsNaN(f) {
|
||||
sap.sum += f
|
||||
sap.count++
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Scan only the given fields for the given row
|
||||
for _, field := range sap.sa.fields {
|
||||
c := br.getColumnByName(field)
|
||||
f := c.getFloatValueAtRow(rowIdx)
|
||||
if !math.IsNaN(f) {
|
||||
sap.sum += f
|
||||
sap.count++
|
||||
}
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (sap *statsAvgProcessor) mergeState(sfp statsProcessor) {
|
||||
src := sfp.(*statsAvgProcessor)
|
||||
sap.sum += src.sum
|
||||
sap.count += src.count
|
||||
}
|
||||
|
||||
func (sap *statsAvgProcessor) finalizeStats() string {
|
||||
avg := sap.sum / float64(sap.count)
|
||||
return strconv.FormatFloat(avg, 'f', -1, 64)
|
||||
}
|
||||
|
||||
func parseStatsAvg(lex *lexer) (*statsAvg, error) {
|
||||
fields, err := parseFieldNamesForStatsFunc(lex, "avg")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sa := &statsAvg{
|
||||
fields: fields,
|
||||
containsStar: slices.Contains(fields, "*"),
|
||||
}
|
||||
return sa, nil
|
||||
}
|
207
lib/logstorage/stats_count.go
Normal file
207
lib/logstorage/stats_count.go
Normal file
|
@ -0,0 +1,207 @@
|
|||
package logstorage
|
||||
|
||||
import (
|
||||
"slices"
|
||||
"strconv"
|
||||
"unsafe"
|
||||
|
||||
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
|
||||
)
|
||||
|
||||
type statsCount struct {
|
||||
fields []string
|
||||
containsStar bool
|
||||
}
|
||||
|
||||
func (sc *statsCount) String() string {
|
||||
return "count(" + fieldNamesString(sc.fields) + ")"
|
||||
}
|
||||
|
||||
func (sc *statsCount) neededFields() []string {
|
||||
if sc.containsStar {
|
||||
// There is no need in fetching any columns for count(*) - the number of matching rows can be calculated as len(blockResult.timestamps)
|
||||
return nil
|
||||
}
|
||||
return sc.fields
|
||||
}
|
||||
|
||||
func (sc *statsCount) newStatsProcessor() (statsProcessor, int) {
|
||||
scp := &statsCountProcessor{
|
||||
sc: sc,
|
||||
}
|
||||
return scp, int(unsafe.Sizeof(*scp))
|
||||
}
|
||||
|
||||
type statsCountProcessor struct {
|
||||
sc *statsCount
|
||||
|
||||
rowsCount uint64
|
||||
}
|
||||
|
||||
func (scp *statsCountProcessor) updateStatsForAllRows(br *blockResult) int {
|
||||
fields := scp.sc.fields
|
||||
if scp.sc.containsStar {
|
||||
// Fast path - unconditionally count all the columns.
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
if len(fields) == 1 {
|
||||
// Fast path for count(single_column)
|
||||
c := br.getColumnByName(fields[0])
|
||||
if c.isConst {
|
||||
if c.encodedValues[0] != "" {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
}
|
||||
return 0
|
||||
}
|
||||
if c.isTime {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
switch c.valueType {
|
||||
case valueTypeString:
|
||||
for _, v := range c.encodedValues {
|
||||
if v != "" {
|
||||
scp.rowsCount++
|
||||
}
|
||||
}
|
||||
return 0
|
||||
case valueTypeDict:
|
||||
zeroDictIdx := slices.Index(c.dictValues, "")
|
||||
if zeroDictIdx < 0 {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
for _, v := range c.encodedValues {
|
||||
if int(v[0]) != zeroDictIdx {
|
||||
scp.rowsCount++
|
||||
}
|
||||
}
|
||||
return 0
|
||||
case valueTypeUint8, valueTypeUint16, valueTypeUint32, valueTypeUint64, valueTypeFloat64, valueTypeIPv4, valueTypeTimestampISO8601:
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
default:
|
||||
logger.Panicf("BUG: unknown valueType=%d", c.valueType)
|
||||
return 0
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path - count rows containing at least a single non-empty value for the fields enumerated inside count().
|
||||
bm := getBitmap(len(br.timestamps))
|
||||
defer putBitmap(bm)
|
||||
|
||||
bm.setBits()
|
||||
for _, f := range fields {
|
||||
c := br.getColumnByName(f)
|
||||
if c.isConst {
|
||||
if c.encodedValues[0] != "" {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
continue
|
||||
}
|
||||
if c.isTime {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
switch c.valueType {
|
||||
case valueTypeString:
|
||||
bm.forEachSetBit(func(i int) bool {
|
||||
return c.encodedValues[i] == ""
|
||||
})
|
||||
case valueTypeDict:
|
||||
if !slices.Contains(c.dictValues, "") {
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
}
|
||||
bm.forEachSetBit(func(i int) bool {
|
||||
dictIdx := c.encodedValues[i][0]
|
||||
return c.dictValues[dictIdx] == ""
|
||||
})
|
||||
case valueTypeUint8, valueTypeUint16, valueTypeUint32, valueTypeUint64, valueTypeFloat64, valueTypeIPv4, valueTypeTimestampISO8601:
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
return 0
|
||||
default:
|
||||
logger.Panicf("BUG: unknown valueType=%d", c.valueType)
|
||||
return 0
|
||||
}
|
||||
}
|
||||
|
||||
scp.rowsCount += uint64(len(br.timestamps))
|
||||
scp.rowsCount -= uint64(bm.onesCount())
|
||||
return 0
|
||||
}
|
||||
|
||||
func (scp *statsCountProcessor) updateStatsForRow(br *blockResult, rowIdx int) int {
|
||||
fields := scp.sc.fields
|
||||
if scp.sc.containsStar {
|
||||
// Fast path - unconditionally count the given column
|
||||
scp.rowsCount++
|
||||
return 0
|
||||
}
|
||||
if len(fields) == 1 {
|
||||
// Fast path for count(single_column)
|
||||
c := br.getColumnByName(fields[0])
|
||||
if c.isConst {
|
||||
if c.encodedValues[0] != "" {
|
||||
scp.rowsCount++
|
||||
}
|
||||
return 0
|
||||
}
|
||||
if c.isTime {
|
||||
scp.rowsCount++
|
||||
return 0
|
||||
}
|
||||
switch c.valueType {
|
||||
case valueTypeString:
|
||||
if v := c.encodedValues[rowIdx]; v != "" {
|
||||
scp.rowsCount++
|
||||
}
|
||||
return 0
|
||||
case valueTypeDict:
|
||||
dictIdx := c.encodedValues[rowIdx][0]
|
||||
if v := c.dictValues[dictIdx]; v != "" {
|
||||
scp.rowsCount++
|
||||
}
|
||||
return 0
|
||||
case valueTypeUint8, valueTypeUint16, valueTypeUint32, valueTypeUint64, valueTypeFloat64, valueTypeIPv4, valueTypeTimestampISO8601:
|
||||
scp.rowsCount++
|
||||
return 0
|
||||
default:
|
||||
logger.Panicf("BUG: unknown valueType=%d", c.valueType)
|
||||
return 0
|
||||
}
|
||||
}
|
||||
|
||||
// Slow path - count the row at rowIdx if at least a single field enumerated inside count() is non-empty
|
||||
for _, f := range fields {
|
||||
c := br.getColumnByName(f)
|
||||
if v := c.getValueAtRow(br, rowIdx); v != "" {
|
||||
scp.rowsCount++
|
||||
return 0
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (scp *statsCountProcessor) mergeState(sfp statsProcessor) {
|
||||
src := sfp.(*statsCountProcessor)
|
||||
scp.rowsCount += src.rowsCount
|
||||
}
|
||||
|
||||
func (scp *statsCountProcessor) finalizeStats() string {
|
||||
return strconv.FormatUint(scp.rowsCount, 10)
|
||||
}
|
||||
|
||||
func parseStatsCount(lex *lexer) (*statsCount, error) {
|
||||
fields, err := parseFieldNamesForStatsFunc(lex, "count")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sc := &statsCount{
|
||||
fields: fields,
|
||||
containsStar: slices.Contains(fields, "*"),
|
||||
}
|
||||
return sc, nil
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue