lib/logstorage: initial implementation of pipes in LogsQL

See https://docs.victoriametrics.com/victorialogs/logsql/#pipes
This commit is contained in:
Aliaksandr Valialkin 2024-05-12 16:33:29 +02:00
parent 9dc9c892b7
commit 147704aab0
No known key found for this signature in database
GPG key ID: 52C003EE2BCDB9EB
119 changed files with 24178 additions and 14059 deletions

View 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

View 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
View 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)
}

View file

@ -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

View file

@ -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

View file

@ -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)
}

View file

@ -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

View file

@ -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

View file

@ -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:

View file

@ -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:

View file

@ -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
View 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
}

View file

@ -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
}

View file

@ -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)

View 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
View 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
}

View 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)
}
}

View file

@ -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)
}

View file

@ -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 {

View file

@ -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{

View file

@ -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

View file

@ -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"))

File diff suppressed because it is too large Load diff

View file

@ -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
}

View file

@ -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

View file

@ -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

View file

@ -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()
}

View file

@ -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
}

View file

@ -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.
//

View file

@ -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)
}

View file

@ -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.

View 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{}{}
}

View 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
View 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)
}

View 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
}

View 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)
}

View 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
}

View 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)
})
}

View 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
}

View 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)
})
}

View 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)
})
}

View 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)
}

View 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)
})
}

View 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
View 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)
}

View 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)
})
}

View 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
})
}

View 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)
})
}

View 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))
}

View 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)
})
}

View 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
}

View 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)
}

View 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)
}

View 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)
}

View 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)
}

View 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)
}

View 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)
})
}

View 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)
}

View 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)
})
}

View 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)
}

View 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)
})
}

View 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)
}

View 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)
})
}

View 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
}

View 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)
})
}

View 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
}
}

View 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
}

View 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)
})
}

View 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)
}

View 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
})
}

View 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)
}

View 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

View file

@ -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)
}

View file

@ -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{}
}

View file

@ -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

View file

@ -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 = &notFilter{
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 := &regexpFilter{
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
}

View file

@ -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`, ``)
}

View file

@ -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
View 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
View 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)
}
}
}

View 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
}

View 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)
}
}
}

View 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", "")
}

View 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)
}
}
}

View 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", "")
}

View 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
}

View 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
}

View 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)
}
}
}

View 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
View 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
}

View 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", "")
}

View 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
}

View 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
View 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
}

View 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", "", "*", "")
}

View file

@ -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
View 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
}

View 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