lib/logstorage: add stream_context pipe, which allows selecting surrounding logs for the matching logs

This commit is contained in:
Aliaksandr Valialkin 2024-06-28 19:14:29 +02:00
parent d7185f1b77
commit d4ca651547
No known key found for this signature in database
GPG key ID: 52C003EE2BCDB9EB
14 changed files with 827 additions and 106 deletions

View file

@ -450,7 +450,7 @@ type logRow struct {
} }
func sortLogRows(rows []logRow) { func sortLogRows(rows []logRow) {
sort.Slice(rows, func(i, j int) bool { sort.SliceStable(rows, func(i, j int) bool {
return rows[i].timestamp < rows[j].timestamp return rows[i].timestamp < rows[j].timestamp
}) })
} }

View file

@ -19,6 +19,7 @@ according to [these docs](https://docs.victoriametrics.com/victorialogs/quicksta
## tip ## tip
* FEATURE: add ability to select surrounding logs in front and after the selected logs via [`stream_context` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#stream_context-pipe). This functionality may be useful for investigating stacktraces panics or some correlated log messages.
* FEATURE: add ability to return top `N` `"fields"` groups from [`/select/logsql/hits` HTTP endpoint](https://docs.victoriametrics.com/victorialogs/querying/#querying-hits-stats), by specifying `fields_limit=N` query arg. This query arg is going to be used in [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6545). * FEATURE: add ability to return top `N` `"fields"` groups from [`/select/logsql/hits` HTTP endpoint](https://docs.victoriametrics.com/victorialogs/querying/#querying-hits-stats), by specifying `fields_limit=N` query arg. This query arg is going to be used in [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6545).
* BUGFIX: fix `runtime error: index out of range [0] with length 0` panic when empty lines are ingested via [Syslog format](https://docs.victoriametrics.com/victorialogs/data-ingestion/syslog/) by Cisco controllers. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6548). * BUGFIX: fix `runtime error: index out of range [0] with length 0` panic when empty lines are ingested via [Syslog format](https://docs.victoriametrics.com/victorialogs/data-ingestion/syslog/) by Cisco controllers. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6548).

View file

@ -1305,6 +1305,8 @@ LogsQL supports the following pipes:
- [`replace_regexp`](#replace_regexp-pipe) updates [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with regular expressions. - [`replace_regexp`](#replace_regexp-pipe) updates [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) with regular expressions.
- [`sort`](#sort-pipe) sorts logs by the given [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). - [`sort`](#sort-pipe) sorts logs by the given [fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
- [`stats`](#stats-pipe) calculates various stats over the selected logs. - [`stats`](#stats-pipe) calculates various stats over the selected logs.
- [`stream_context`](#stream_context-pipe) allows selecting surrounding logs in front and after the matching logs
per each [log stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields).
- [`top`](#top-pipe) returns top `N` field sets with the maximum number of matching logs. - [`top`](#top-pipe) returns top `N` field sets with the maximum number of matching logs.
- [`uniq`](#uniq-pipe) returns unique log entires. - [`uniq`](#uniq-pipe) returns unique log entires.
- [`unpack_json`](#unpack_json-pipe) unpacks JSON messages from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model). - [`unpack_json`](#unpack_json-pipe) unpacks JSON messages from [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model).
@ -2300,6 +2302,30 @@ _time:5m | stats
count() total count() total
``` ```
### stream_context pipe
`| stream_context ...` [pipe](#pipes) allows selecting surrounding logs for the matching logs in [logs stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields).
For example, the following query returns up to 10 additional logs after every log message with the `panic` [word](#word) across all the logs for the last 5 minutes:
```logsql
_time:5m panic | stream_context after 10
```
The following query returns up to 5 additional logs in front of eevery log message with the `stacktrace` [word](#word) across all the logs for the last 5 minutes:
```logsql
_time:5m stacktrace | stream_context before 5
```
The following query returns up to 2 logs in frount of the log message with the `error` [word](#word) and up to 5 logs after this log message
across all the logs for the last 5 minutes:
```logsql
_time:5m error | stream_context before 2 after 5
```
The `| stream_context` [pipe](#pipes) must go first just after the [filters](#filters).
### top pipe ### top pipe
`| top N by (field1, ..., fieldN)` [pipe](#pipes) returns top `N` sets for `(field1, ..., fieldN)` [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model) `| top N by (field1, ..., fieldN)` [pipe](#pipes) returns top `N` sets for `(field1, ..., fieldN)` [log fields](https://docs.victoriametrics.com/victorialogs/keyconcepts/#data-model)
@ -3042,10 +3068,7 @@ See also:
## Stream context ## Stream context
LogsQL will support the ability to select the given number of surrounding log lines for the selected log lines See [`stream_context` pipe](#stream_context-pipe).
on a [per-stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) basis.
See the [Roadmap](https://docs.victoriametrics.com/victorialogs/roadmap/) for details.
## Transformations ## Transformations

View file

@ -29,6 +29,7 @@ VictoriaLogs provides the following features:
- VictoriaLogs supports multitenancy - see [these docs](#multitenancy). - VictoriaLogs supports multitenancy - see [these docs](#multitenancy).
- VictoriaLogs supports out-of-order logs' ingestion aka backfilling. - VictoriaLogs supports out-of-order logs' ingestion aka backfilling.
- VictoriaLogs supports live tailing for newly ingested logs. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#live-tailing). - VictoriaLogs supports live tailing for newly ingested logs. See [these docs](https://docs.victoriametrics.com/victorialogs/querying/#live-tailing).
- VictoriaLogs supports selecting surrounding logs in front and after the selected logs. See [these docs](https://docs.victoriametrics.com/victorialogs/logsql/#stream_context-pipe).
- VictoriaLogs provides web UI for querying logs - see [these docs](https://docs.victoriametrics.com/victorialogs/querying/#web-ui). - VictoriaLogs provides web UI for querying logs - see [these docs](https://docs.victoriametrics.com/victorialogs/querying/#web-ui).
VictoriaLogs is at the Preview stage now. It is ready for evaluation in production and verifying the claims given above. VictoriaLogs is at the Preview stage now. It is ready for evaluation in production and verifying the claims given above.

View file

@ -34,8 +34,6 @@ The following functionality is planned in the future versions of VictoriaLogs:
- Fluentd - Fluentd
- Journald (systemd) - Journald (systemd)
- Datadog protocol for logs - Datadog protocol for logs
- Add missing functionality to [LogsQL](https://docs.victoriametrics.com/victorialogs/logsql/):
- [Stream context](https://docs.victoriametrics.com/victorialogs/logsql/#stream-context).
- Integration with Grafana ([partially done](https://github.com/VictoriaMetrics/victorialogs-datasource)). - Integration with Grafana ([partially done](https://github.com/VictoriaMetrics/victorialogs-datasource)).
- Ability to make instant snapshots and backups in the way [similar to VictoriaMetrics](https://docs.victoriametrics.com/#how-to-work-with-snapshots). - Ability to make instant snapshots and backups in the way [similar to VictoriaMetrics](https://docs.victoriametrics.com/#how-to-work-with-snapshots).
- Cluster version of VictoriaLogs. - Cluster version of VictoriaLogs.

View file

@ -247,7 +247,7 @@ This can help narrowing down and eliminating high-cardinality fields from [log s
### Other fields ### Other fields
Every ingested log entry may contain arbitrary number of [fields](#data-model) additionally to [`_msg`](#message-field) and [`_time`](#time-field). Every ingested log entry may contain arbitrary number of [fields](#data-model) additionally to [`_msg`](#message-field) and [`_time`](#time-field).
For example, `level`, `ip`, `user_id`, `trace_id`, etc. Such fields can be used for simplifying and optimizing [search queries](#https://docs.victoriametrics.com/victorialogs/logsql/). For example, `level`, `ip`, `user_id`, `trace_id`, etc. Such fields can be used for simplifying and optimizing [search queries](https://docs.victoriametrics.com/victorialogs/logsql/).
It is usually faster to search over a dedicated `trace_id` field instead of searching for the `trace_id` inside long [log message](#message-field). It is usually faster to search over a dedicated `trace_id` field instead of searching for the `trace_id` inside long [log message](#message-field).
E.g. the `trace_id:="XXXX-YYYY-ZZZZ"` query usually works faster than the `_msg:"trace_id=XXXX-YYYY-ZZZZ"` query. E.g. the `trace_id:="XXXX-YYYY-ZZZZ"` query usually works faster than the `_msg:"trace_id=XXXX-YYYY-ZZZZ"` query.

View file

@ -472,3 +472,13 @@ returns logs with the `cannot open file` phrase over the last 5 minutes:
```logsql ```logsql
_time:5m "cannot open file" _time:5m "cannot open file"
``` ```
## How to select all the logs for a particular stacktrace or panic?
Use [`stream_context` pipe](https://docs.victoriametrics.com/victorialogs/logsql/#stream_context-pipe) for selecting surrounding logs for the given log.
For example, the following query selects up to 10 logs in front of every log message containing the `stacktrace` [word](https://docs.victoriametrics.com/victorialogs/logsql/#word),
plus up to 100 logs after the given log message:
```logsql
_time:5m stacktrace | stream_context before 10 after 100
```

View file

@ -196,11 +196,7 @@ func testFilterMatchForColumns(t *testing.T, columns []column, f filter, neededC
func testFilterMatchForStorage(t *testing.T, s *Storage, tenantID TenantID, f filter, neededColumnName string, expectedValues []string, expectedTimestamps []int64) { func testFilterMatchForStorage(t *testing.T, s *Storage, tenantID TenantID, f filter, neededColumnName string, expectedValues []string, expectedTimestamps []int64) {
t.Helper() t.Helper()
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{neededColumnName, "_time"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{neededColumnName, "_time"},
}
type result struct { type result struct {
value string value string

View file

@ -350,7 +350,26 @@ func (q *Query) CanReturnLastNResults() bool {
// GetFilterTimeRange returns filter time range for the given q. // GetFilterTimeRange returns filter time range for the given q.
func (q *Query) GetFilterTimeRange() (int64, int64) { func (q *Query) GetFilterTimeRange() (int64, int64) {
return getFilterTimeRange(q.f) switch t := q.f.(type) {
case *filterAnd:
minTimestamp := int64(math.MinInt64)
maxTimestamp := int64(math.MaxInt64)
for _, filter := range t.filters {
ft, ok := filter.(*filterTime)
if ok {
if ft.minTimestamp > minTimestamp {
minTimestamp = ft.minTimestamp
}
if ft.maxTimestamp < maxTimestamp {
maxTimestamp = ft.maxTimestamp
}
}
}
return minTimestamp, maxTimestamp
case *filterTime:
return t.minTimestamp, t.maxTimestamp
}
return math.MinInt64, math.MaxInt64
} }
// AddTimeFilter adds global filter _time:[start ... end] to q. // AddTimeFilter adds global filter _time:[start ... end] to q.

View file

@ -219,6 +219,12 @@ func parsePipe(lex *lexer) (pipe, error) {
return nil, fmt.Errorf("cannot parse 'stats' pipe: %w", err) return nil, fmt.Errorf("cannot parse 'stats' pipe: %w", err)
} }
return ps, nil return ps, nil
case lex.isKeyword("stream_context"):
pc, err := parsePipeStreamContext(lex)
if err != nil {
return nil, fmt.Errorf("cannot parse 'stream_context' pipe: %w", err)
}
return pc, nil
case lex.isKeyword("top"): case lex.isKeyword("top"):
pt, err := parsePipeTop(lex) pt, err := parsePipeTop(lex)
if err != nil { if err != nil {
@ -298,6 +304,7 @@ var pipeNames = func() map[string]struct{} {
"replace_regexp", "replace_regexp",
"sort", "sort",
"stats", "stats",
"stream_context",
"top", "top",
"uniq", "uniq",
"unpack_json", "unpack_json",

View file

@ -0,0 +1,511 @@
package logstorage
import (
"context"
"fmt"
"math"
"sort"
"strings"
"sync"
"sync/atomic"
"unsafe"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
)
// pipeStreamContext processes '| stream_context ...' queries.
//
// See https://docs.victoriametrics.com/victorialogs/logsql/#stream_context-pipe
type pipeStreamContext struct {
// linesBefore is the number of lines to return before the matching line
linesBefore int
// linesAfter is the number of lines to return after the matching line
linesAfter int
}
func (pc *pipeStreamContext) String() string {
s := "stream_context"
if pc.linesBefore > 0 {
s += fmt.Sprintf(" before %d", pc.linesBefore)
}
if pc.linesAfter > 0 {
s += fmt.Sprintf(" after %d", pc.linesAfter)
}
return s
}
func (pc *pipeStreamContext) canLiveTail() bool {
return false
}
var neededFieldsForStreamContext = []string{
"_time",
"_stream_id",
}
func (pc *pipeStreamContext) updateNeededFields(neededFields, unneededFields fieldsSet) {
neededFields.addFields(neededFieldsForStreamContext)
unneededFields.removeFields(neededFieldsForStreamContext)
}
func (pc *pipeStreamContext) optimize() {
// nothing to do
}
func (pc *pipeStreamContext) hasFilterInWithQuery() bool {
return false
}
func (pc *pipeStreamContext) initFilterInValues(_ map[string][]string, _ getFieldValuesFunc) (pipe, error) {
return pc, nil
}
func (pc *pipeStreamContext) newPipeProcessor(workersCount int, stopCh <-chan struct{}, cancel func(), ppNext pipeProcessor) pipeProcessor {
maxStateSize := int64(float64(memory.Allowed()) * 0.2)
shards := make([]pipeStreamContextProcessorShard, workersCount)
for i := range shards {
shards[i] = pipeStreamContextProcessorShard{
pipeStreamContextProcessorShardNopad: pipeStreamContextProcessorShardNopad{
pc: pc,
stateSizeBudget: stateSizeBudgetChunk,
},
}
maxStateSize -= stateSizeBudgetChunk
}
pcp := &pipeStreamContextProcessor{
pc: pc,
stopCh: stopCh,
cancel: cancel,
ppNext: ppNext,
shards: shards,
maxStateSize: maxStateSize,
}
pcp.stateSizeBudget.Store(maxStateSize)
return pcp
}
type pipeStreamContextProcessor struct {
pc *pipeStreamContext
stopCh <-chan struct{}
cancel func()
ppNext pipeProcessor
shards []pipeStreamContextProcessorShard
getStreamRows func(streamID string, stateSizeBudget int) ([]streamContextRow, error)
maxStateSize int64
stateSizeBudget atomic.Int64
}
func (pcp *pipeStreamContextProcessor) init(ctx context.Context, s *Storage, minTimestamp, maxTimestamp int64) {
pcp.getStreamRows = func(streamID string, stateSizeBudget int) ([]streamContextRow, error) {
return getStreamRows(ctx, s, streamID, minTimestamp, maxTimestamp, stateSizeBudget)
}
}
func getStreamRows(ctx context.Context, s *Storage, streamID string, minTimestamp, maxTimestamp int64, stateSizeBudget int) ([]streamContextRow, error) {
tenantID, ok := getTenantIDFromStreamIDString(streamID)
if !ok {
logger.Panicf("BUG: cannot obtain tenantID from streamID %q", streamID)
}
qStr := "_stream_id:" + streamID
q, err := ParseQuery(qStr)
if err != nil {
logger.Panicf("BUG: cannot parse query [%s]: %s", qStr, err)
}
q.AddTimeFilter(minTimestamp, maxTimestamp)
ctxWithCancel, cancel := context.WithCancel(ctx)
defer cancel()
var mu sync.Mutex
var rows []streamContextRow
stateSize := 0
writeBlock := func(_ uint, br *blockResult) {
mu.Lock()
defer mu.Unlock()
if stateSize > stateSizeBudget {
cancel()
}
cs := br.getColumns()
for i, timestamp := range br.timestamps {
fields := make([]Field, len(cs))
stateSize += int(unsafe.Sizeof(fields[0])) * len(fields)
for j, c := range cs {
v := c.getValueAtRow(br, i)
fields[j] = Field{
Name: strings.Clone(c.name),
Value: strings.Clone(v),
}
stateSize += len(c.name) + len(v)
}
row := streamContextRow{
timestamp: timestamp,
fields: fields,
}
stateSize += int(unsafe.Sizeof(row))
rows = append(rows, row)
}
}
if err := s.runQuery(ctxWithCancel, []TenantID{tenantID}, q, writeBlock); err != nil {
return nil, err
}
if stateSize > stateSizeBudget {
return nil, fmt.Errorf("more than %dMB of memory is needed for query [%s]", stateSizeBudget/(1<<20), q)
}
return rows, nil
}
func getTenantIDFromStreamIDString(s string) (TenantID, bool) {
var sid streamID
if !sid.tryUnmarshalFromString(s) {
return TenantID{}, false
}
return sid.tenantID, true
}
type pipeStreamContextProcessorShard struct {
pipeStreamContextProcessorShardNopad
// The padding prevents false sharing on widespread platforms with 128 mod (cache line size) = 0 .
_ [128 - unsafe.Sizeof(pipeStreamContextProcessorShardNopad{})%128]byte
}
type streamContextRow struct {
timestamp int64
fields []Field
}
type pipeStreamContextProcessorShardNopad struct {
// pc points to the parent pipeStreamContext.
pc *pipeStreamContext
// m holds per-stream matching rows
m map[string][]streamContextRow
// stateSizeBudget is the remaining budget for the whole state size for the shard.
// The per-shard budget is provided in chunks from the parent pipeStreamContextProcessor.
stateSizeBudget int
}
// writeBlock writes br to shard.
func (shard *pipeStreamContextProcessorShard) writeBlock(br *blockResult) {
m := shard.getM()
cs := br.getColumns()
cStreamID := br.getColumnByName("_stream_id")
stateSize := 0
for i, timestamp := range br.timestamps {
fields := make([]Field, len(cs))
stateSize += int(unsafe.Sizeof(fields[0])) * len(fields)
for j, c := range cs {
v := c.getValueAtRow(br, i)
fields[j] = Field{
Name: strings.Clone(c.name),
Value: strings.Clone(v),
}
stateSize += len(c.name) + len(v)
}
row := streamContextRow{
timestamp: timestamp,
fields: fields,
}
stateSize += int(unsafe.Sizeof(row))
streamID := cStreamID.getValueAtRow(br, i)
rows, ok := m[streamID]
if !ok {
stateSize += len(streamID)
}
rows = append(rows, row)
streamID = strings.Clone(streamID)
m[streamID] = rows
}
shard.stateSizeBudget -= stateSize
}
func (shard *pipeStreamContextProcessorShard) getM() map[string][]streamContextRow {
if shard.m == nil {
shard.m = make(map[string][]streamContextRow)
}
return shard.m
}
func (pcp *pipeStreamContextProcessor) writeBlock(workerID uint, br *blockResult) {
if len(br.timestamps) == 0 {
return
}
if pcp.pc.linesBefore <= 0 && pcp.pc.linesAfter <= 0 {
// Fast path - there is no need to fetch stream context.
pcp.ppNext.writeBlock(workerID, br)
return
}
shard := &pcp.shards[workerID]
for shard.stateSizeBudget < 0 {
// steal some budget for the state size from the global budget.
remaining := pcp.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.
pcp.cancel()
}
return
}
shard.stateSizeBudget += stateSizeBudgetChunk
}
shard.writeBlock(br)
}
func (pcp *pipeStreamContextProcessor) flush() error {
if pcp.pc.linesBefore <= 0 && pcp.pc.linesAfter <= 0 {
// Fast path - nothing to do.
return nil
}
n := pcp.stateSizeBudget.Load()
if n <= 0 {
return fmt.Errorf("cannot calculate [%s], since it requires more than %dMB of memory", pcp.pc.String(), pcp.maxStateSize/(1<<20))
}
if n > math.MaxInt {
logger.Panicf("BUG: stateSizeBudget shouldn't exceed math.MaxInt=%v; got %d", math.MaxInt, n)
}
stateSizeBudget := int(n)
// merge state across shards
shards := pcp.shards
m := shards[0].getM()
shards = shards[1:]
for i := range shards {
if needStop(pcp.stopCh) {
return nil
}
for streamID, rowsSrc := range shards[i].getM() {
rows, ok := m[streamID]
if !ok {
m[streamID] = rowsSrc
} else {
m[streamID] = append(rows, rowsSrc...)
}
}
}
// write result
wctx := &pipeStreamContextWriteContext{
pcp: pcp,
}
for streamID, rows := range m {
streamRows, err := pcp.getStreamRows(streamID, stateSizeBudget)
if err != nil {
return fmt.Errorf("cannot read rows for _stream_id=%q: %w", streamID, err)
}
if needStop(pcp.stopCh) {
return nil
}
resultRows, err := getStreamContextRows(streamRows, rows, pcp.pc.linesBefore, pcp.pc.linesAfter)
if err != nil {
return fmt.Errorf("cannot obtain context rows for _stream_id=%q: %w", streamID, err)
}
for _, rowFields := range resultRows {
wctx.writeRow(rowFields)
}
}
wctx.flush()
return nil
}
func getStreamContextRows(streamRows, rows []streamContextRow, linesBefore, linesAfter int) ([][]Field, error) {
sortStreamContextRows(streamRows)
sortStreamContextRows(rows)
var resultRows [][]Field
idxNext := 0
for _, r := range rows {
idx := getStreamContextRowIdx(streamRows, r.timestamp)
if idx < 0 {
// This error may happen when streamRows became out of sync with rows.
// For example, when some streamRows were deleted after obtaining rows.
return nil, fmt.Errorf("missing row for timestamp=%d; len(streamRows)=%d, len(rows)=%d", r.timestamp, len(streamRows), len(rows))
}
idxStart := idx - linesBefore
if idxStart < idxNext {
idxStart = idxNext
}
for idxStart < idx {
resultRows = append(resultRows, streamRows[idxStart].fields)
idxStart++
}
if idx >= idxNext {
resultRows = append(resultRows, streamRows[idx].fields)
idxNext = idx + 1
}
idxEnd := idx + 1 + linesAfter
for idxNext < idxEnd && idxNext < len(streamRows) {
resultRows = append(resultRows, streamRows[idxNext].fields)
idxNext++
}
if idxNext >= len(streamRows) {
break
}
}
return resultRows, nil
}
func getStreamContextRowIdx(rows []streamContextRow, timestamp int64) int {
n := sort.Search(len(rows), func(i int) bool {
return rows[i].timestamp >= timestamp
})
if n == len(rows) {
return -1
}
if rows[n].timestamp != timestamp {
return -1
}
return n
}
func sortStreamContextRows(rows []streamContextRow) {
sort.SliceStable(rows, func(i, j int) bool {
return rows[i].timestamp < rows[j].timestamp
})
}
type pipeStreamContextWriteContext struct {
pcp *pipeStreamContextProcessor
rcs []resultColumn
br blockResult
// rowsCount is the number of rows in the current block
rowsCount int
// valuesLen is the total length of values in the current block
valuesLen int
}
func (wctx *pipeStreamContextWriteContext) writeRow(rowFields []Field) {
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 ppNext and construct a block with new set of columns
wctx.flush()
rcs = wctx.rcs[:0]
for _, f := range rowFields {
rcs = appendResultColumnWithName(rcs, f.Name)
}
wctx.rcs = rcs
}
for i, f := range rowFields {
v := f.Value
rcs[i].addValue(v)
wctx.valuesLen += len(v)
}
wctx.rowsCount++
if wctx.valuesLen >= 1_000_000 {
wctx.flush()
}
}
func (wctx *pipeStreamContextWriteContext) flush() {
rcs := wctx.rcs
br := &wctx.br
wctx.valuesLen = 0
// Flush rcs to ppNext
br.setResultColumns(rcs, wctx.rowsCount)
wctx.rowsCount = 0
wctx.pcp.ppNext.writeBlock(0, br)
br.reset()
for i := range rcs {
rcs[i].resetValues()
}
}
func parsePipeStreamContext(lex *lexer) (*pipeStreamContext, error) {
if !lex.isKeyword("stream_context") {
return nil, fmt.Errorf("expecting 'stream_context'; got %q", lex.token)
}
lex.nextToken()
linesBefore := 0
beforeSet := false
if lex.isKeyword("before") {
lex.nextToken()
f, s, err := parseNumber(lex)
if err != nil {
return nil, fmt.Errorf("cannot parse 'before' value in 'stream_context': %w", err)
}
if f < 0 {
return nil, fmt.Errorf("'before' value cannot be smaller than 0; got %q", s)
}
linesBefore = int(f)
beforeSet = true
}
linesAfter := 0
afterSet := false
if lex.isKeyword("after") {
lex.nextToken()
f, s, err := parseNumber(lex)
if err != nil {
return nil, fmt.Errorf("cannot parse 'after' value in 'stream_context': %w", err)
}
if f < 0 {
return nil, fmt.Errorf("'after' value cannot be smaller than 0; got %q", s)
}
linesAfter = int(f)
afterSet = true
}
if !beforeSet && !afterSet {
return nil, fmt.Errorf("missing 'before N' or 'after N' in 'stream_context'")
}
pc := &pipeStreamContext{
linesBefore: linesBefore,
linesAfter: linesAfter,
}
return pc, nil
}

View file

@ -0,0 +1,116 @@
package logstorage
import (
"testing"
)
func TestParsePipeStreamContextSuccess(t *testing.T) {
f := func(pipeStr string) {
t.Helper()
expectParsePipeSuccess(t, pipeStr)
}
f(`stream_context before 5`)
f(`stream_context after 10`)
f(`stream_context before 10 after 20`)
}
func TestParsePipeStreamContextFailure(t *testing.T) {
f := func(pipeStr string) {
t.Helper()
expectParsePipeFailure(t, pipeStr)
}
f(`stream_context`)
f(`stream_context before`)
f(`stream_context after`)
f(`stream_context before after`)
f(`stream_context after before`)
f(`stream_context before -4`)
f(`stream_context after -4`)
}
func TestPipeStreamContext(t *testing.T) {
f := func(pipeStr string, rows, rowsExpected [][]Field) {
t.Helper()
expectPipeResults(t, pipeStr, rows, rowsExpected)
}
f("stream_context before 0", [][]Field{
{
{"a", `2`},
{"b", `3`},
},
{
{"a", "2"},
{"b", "3"},
},
{
{"a", `2`},
{"b", `54`},
{"c", "d"},
},
}, [][]Field{
{
{"a", `2`},
{"b", `3`},
},
{
{"a", "2"},
{"b", "3"},
},
{
{"a", `2`},
{"b", `54`},
{"c", "d"},
},
})
f("stream_context after 0", [][]Field{
{
{"a", `2`},
{"b", `3`},
},
{
{"a", "2"},
{"b", "3"},
},
{
{"a", `2`},
{"b", `54`},
{"c", "d"},
},
}, [][]Field{
{
{"a", `2`},
{"b", `3`},
},
{
{"a", "2"},
{"b", "3"},
},
{
{"a", `2`},
{"b", `54`},
{"c", "d"},
},
})
}
func TestPipeStreamContextUpdateNeededFields(t *testing.T) {
f := func(s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected string) {
t.Helper()
expectPipeNeededFields(t, s, neededFields, unneededFields, neededFieldsExpected, unneededFieldsExpected)
}
// all the needed fields
f("stream_context before 10", "*", "", "*", "")
// plus unneeded fields
f("stream_context after 4", "*", "f1,f2", "*", "f1,f2")
f("stream_context after 4", "*", "_time,f1,_stream_id", "*", "f1")
// needed fields
f("stream_context before 3", "f1,f2", "", "_stream_id,_time,f1,f2", "")
f("stream_context before 3", "_time,f1,_stream_id", "", "_stream_id,_time,f1", "")
}

View file

@ -22,6 +22,12 @@ type genericSearchOptions struct {
// If it is empty, then the search is performed by tenantIDs // If it is empty, then the search is performed by tenantIDs
streamIDs []streamID streamIDs []streamID
// minTimestamp is the minimum timestamp for the search
minTimestamp int64
// maxTimestamp is the maximum timestamp for the search
maxTimestamp int64
// filter is the filter to use for the search // filter is the filter to use for the search
filter filter filter filter
@ -110,10 +116,14 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
return streamIDs[i].less(&streamIDs[j]) return streamIDs[i].less(&streamIDs[j])
}) })
minTimestamp, maxTimestamp := q.GetFilterTimeRange()
neededColumnNames, unneededColumnNames := q.getNeededColumns() neededColumnNames, unneededColumnNames := q.getNeededColumns()
so := &genericSearchOptions{ so := &genericSearchOptions{
tenantIDs: tenantIDs, tenantIDs: tenantIDs,
streamIDs: streamIDs, streamIDs: streamIDs,
minTimestamp: minTimestamp,
maxTimestamp: maxTimestamp,
filter: q.f, filter: q.f,
neededColumnNames: neededColumnNames, neededColumnNames: neededColumnNames,
unneededColumnNames: unneededColumnNames, unneededColumnNames: unneededColumnNames,
@ -127,10 +137,21 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
stopCh := ctx.Done() stopCh := ctx.Done()
cancels := make([]func(), len(q.pipes)) cancels := make([]func(), len(q.pipes))
pps := make([]pipeProcessor, len(q.pipes)) pps := make([]pipeProcessor, len(q.pipes))
var errPipe error
for i := len(q.pipes) - 1; i >= 0; i-- { for i := len(q.pipes) - 1; i >= 0; i-- {
p := q.pipes[i] p := q.pipes[i]
ctxChild, cancel := context.WithCancel(ctx) ctxChild, cancel := context.WithCancel(ctx)
pp = p.newPipeProcessor(workersCount, stopCh, cancel, pp) pp = p.newPipeProcessor(workersCount, stopCh, cancel, pp)
pcp, ok := pp.(*pipeStreamContextProcessor)
if ok {
pcp.init(ctx, s, minTimestamp, maxTimestamp)
if i > 0 {
errPipe = fmt.Errorf("[%s] pipe must go after [%s] filter; now it goes after the [%s] pipe", p, q.f, q.pipes[i-1])
}
}
stopCh = ctxChild.Done() stopCh = ctxChild.Done()
ctx = ctxChild ctx = ctxChild
@ -138,7 +159,9 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
pps[i] = pp pps[i] = pp
} }
s.search(workersCount, so, stopCh, pp.writeBlock) if errPipe == nil {
s.search(workersCount, so, stopCh, pp.writeBlock)
}
var errFlush error var errFlush error
for i, pp := range pps { for i, pp := range pps {
@ -151,6 +174,11 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
if err := ppMain.flush(); err != nil && errFlush == nil { if err := ppMain.flush(); err != nil && errFlush == nil {
errFlush = err errFlush = err
} }
if errPipe != nil {
return errPipe
}
return errFlush return errFlush
} }
@ -642,19 +670,15 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
}(uint(i)) }(uint(i))
} }
// Obtain time range from so.filter
f := so.filter
minTimestamp, maxTimestamp := getFilterTimeRange(f)
// Select partitions according to the selected time range // Select partitions according to the selected time range
s.partitionsLock.Lock() s.partitionsLock.Lock()
ptws := s.partitions ptws := s.partitions
minDay := minTimestamp / nsecPerDay minDay := so.minTimestamp / nsecPerDay
n := sort.Search(len(ptws), func(i int) bool { n := sort.Search(len(ptws), func(i int) bool {
return ptws[i].day >= minDay return ptws[i].day >= minDay
}) })
ptws = ptws[n:] ptws = ptws[n:]
maxDay := maxTimestamp / nsecPerDay maxDay := so.maxTimestamp / nsecPerDay
n = sort.Search(len(ptws), func(i int) bool { n = sort.Search(len(ptws), func(i int) bool {
return ptws[i].day > maxDay return ptws[i].day > maxDay
}) })
@ -665,8 +689,7 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
s.partitionsLock.Unlock() s.partitionsLock.Unlock()
// Obtain common filterStream from f // Obtain common filterStream from f
var sf *StreamFilter sf, f := getCommonStreamFilter(so.filter)
sf, f = getCommonStreamFilter(f)
// Schedule concurrent search across matching partitions. // Schedule concurrent search across matching partitions.
psfs := make([]partitionSearchFinalizer, len(ptws)) psfs := make([]partitionSearchFinalizer, len(ptws))
@ -675,7 +698,7 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
partitionSearchConcurrencyLimitCh <- struct{}{} partitionSearchConcurrencyLimitCh <- struct{}{}
wgSearchers.Add(1) wgSearchers.Add(1)
go func(idx int, pt *partition) { go func(idx int, pt *partition) {
psfs[idx] = pt.search(minTimestamp, maxTimestamp, sf, f, so, workCh, stopCh) psfs[idx] = pt.search(sf, f, so, workCh, stopCh)
wgSearchers.Done() wgSearchers.Done()
<-partitionSearchConcurrencyLimitCh <-partitionSearchConcurrencyLimitCh
}(i, ptw.pt) }(i, ptw.pt)
@ -704,7 +727,7 @@ var partitionSearchConcurrencyLimitCh = make(chan struct{}, cgroup.AvailableCPUs
type partitionSearchFinalizer func() type partitionSearchFinalizer func()
func (pt *partition) search(minTimestamp, maxTimestamp int64, sf *StreamFilter, f filter, so *genericSearchOptions, workCh chan<- *blockSearchWorkBatch, stopCh <-chan struct{}) partitionSearchFinalizer { func (pt *partition) search(sf *StreamFilter, f filter, so *genericSearchOptions, workCh chan<- *blockSearchWorkBatch, stopCh <-chan struct{}) partitionSearchFinalizer {
if needStop(stopCh) { if needStop(stopCh) {
// Do not spend CPU time on search, since it is already stopped. // Do not spend CPU time on search, since it is already stopped.
return func() {} return func() {}
@ -728,8 +751,8 @@ func (pt *partition) search(minTimestamp, maxTimestamp int64, sf *StreamFilter,
soInternal := &searchOptions{ soInternal := &searchOptions{
tenantIDs: tenantIDs, tenantIDs: tenantIDs,
streamIDs: streamIDs, streamIDs: streamIDs,
minTimestamp: minTimestamp, minTimestamp: so.minTimestamp,
maxTimestamp: maxTimestamp, maxTimestamp: so.maxTimestamp,
filter: f, filter: f,
neededColumnNames: so.neededColumnNames, neededColumnNames: so.neededColumnNames,
unneededColumnNames: so.unneededColumnNames, unneededColumnNames: so.unneededColumnNames,
@ -1095,29 +1118,6 @@ func getCommonStreamFilter(f filter) (*StreamFilter, filter) {
return nil, f return nil, f
} }
func getFilterTimeRange(f filter) (int64, int64) {
switch t := f.(type) {
case *filterAnd:
minTimestamp := int64(math.MinInt64)
maxTimestamp := int64(math.MaxInt64)
for _, filter := range t.filters {
ft, ok := filter.(*filterTime)
if ok {
if ft.minTimestamp > minTimestamp {
minTimestamp = ft.minTimestamp
}
if ft.maxTimestamp < maxTimestamp {
maxTimestamp = ft.maxTimestamp
}
}
}
return minTimestamp, maxTimestamp
case *filterTime:
return t.minTimestamp, t.maxTimestamp
}
return math.MinInt64, math.MaxInt64
}
func forEachStreamField(streams []ValueWithHits, f func(f Field, hits uint64)) { func forEachStreamField(streams []ValueWithHits, f func(f Field, hits uint64)) {
var fields []Field var fields []Field
for i := range streams { for i := range streams {

View file

@ -3,6 +3,7 @@ package logstorage
import ( import (
"context" "context"
"fmt" "fmt"
"math"
"reflect" "reflect"
"sort" "sort"
"strings" "strings"
@ -87,7 +88,7 @@ func TestStorageRunQuery(t *testing.T) {
t.Helper() t.Helper()
err := s.RunQuery(context.Background(), tenantIDs, q, writeBlock) err := s.RunQuery(context.Background(), tenantIDs, q, writeBlock)
if err != nil { if err != nil {
t.Fatalf("unexpected error returned from the query %s: %s", q, err) t.Fatalf("unexpected error returned from the query [%s]: %s", q, err)
} }
} }
@ -656,6 +657,78 @@ func TestStorageRunQuery(t *testing.T) {
}, },
}) })
}) })
t.Run("stream_context-noop-1", func(t *testing.T) {
f(t, `"message 3 at block 1"
| stream_context before 0
| stats count() rows`, [][]Field{
{
{"rows", "33"},
},
})
})
t.Run("stream_context-noop-2", func(t *testing.T) {
f(t, `"message 3 at block 1"
| stream_context before 0 after 0
| stats count() rows`, [][]Field{
{
{"rows", "33"},
},
})
})
t.Run("stream_context-before-1", func(t *testing.T) {
f(t, `"message 3 at block 1"
| stream_context before 1
| stats count() rows`, [][]Field{
{
{"rows", "66"},
},
})
})
t.Run("stream_context-after-1", func(t *testing.T) {
f(t, `"message 3 at block 1"
| stream_context after 1
| stats count() rows`, [][]Field{
{
{"rows", "66"},
},
})
})
t.Run("stream_context-before-after-1", func(t *testing.T) {
f(t, `"message 3 at block 1"
| stream_context before 1 after 1
| stats count() rows`, [][]Field{
{
{"rows", "99"},
},
})
})
t.Run("stream_context-before-1000", func(t *testing.T) {
f(t, `"message 4"
| stream_context before 1000
| stats count() rows`, [][]Field{
{
{"rows", "825"},
},
})
})
t.Run("stream_context-after-1000", func(t *testing.T) {
f(t, `"message 4"
| stream_context after 1000
| stats count() rows`, [][]Field{
{
{"rows", "495"},
},
})
})
t.Run("stream_context-before-after-1000", func(t *testing.T) {
f(t, `"message 4"
| stream_context before 1000 after 1000
| stats count() rows`, [][]Field{
{
{"rows", "1155"},
},
})
})
// Close the storage and delete its data // Close the storage and delete its data
s.MustClose() s.MustClose()
@ -757,11 +830,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil) f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
processBlock := func(_ uint, _ *blockResult) { processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match")) panic(fmt.Errorf("unexpected match"))
} }
@ -775,11 +844,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil) f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
processBlock := func(_ uint, _ *blockResult) { processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match")) panic(fmt.Errorf("unexpected match"))
} }
@ -793,11 +858,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil) f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
processBlock := func(_ uint, _ *blockResult) { processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match")) panic(fmt.Errorf("unexpected match"))
} }
@ -812,11 +873,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil) f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -833,11 +890,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil) f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{ so := newTestGenericSearchOptions(allTenantIDs, f, []string{"_msg"})
tenantIDs: allTenantIDs,
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -854,11 +907,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf) f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{ so := newTestGenericSearchOptions(allTenantIDs, f, []string{"_msg"})
tenantIDs: allTenantIDs,
filter: f,
neededColumnNames: []string{"_msg"},
}
processBlock := func(_ uint, _ *blockResult) { processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match")) panic(fmt.Errorf("unexpected match"))
} }
@ -874,11 +923,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf) f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -900,11 +945,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf) f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -934,11 +975,7 @@ func TestStorageSearch(t *testing.T) {
}, },
}, },
} }
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -959,11 +996,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp + (rowsPerBlock-2)*1e9 minTimestamp := baseTimestamp + (rowsPerBlock-2)*1e9
maxTimestamp := baseTimestamp + (rowsPerBlock-1)*1e9 - 1 maxTimestamp := baseTimestamp + (rowsPerBlock-1)*1e9 - 1
f := getBaseFilter(minTimestamp, maxTimestamp, sf) f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
var rowsCountTotal atomic.Uint32 var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) { processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps))) rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -984,11 +1017,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp + (rowsPerBlock+1)*1e9 minTimestamp := baseTimestamp + (rowsPerBlock+1)*1e9
maxTimestamp := baseTimestamp + (rowsPerBlock+2)*1e9 maxTimestamp := baseTimestamp + (rowsPerBlock+2)*1e9
f := getBaseFilter(minTimestamp, maxTimestamp, sf) f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{ so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
processBlock := func(_ uint, _ *blockResult) { processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match")) panic(fmt.Errorf("unexpected match"))
} }
@ -1020,3 +1049,13 @@ func TestParseStreamFieldsSuccess(t *testing.T) {
f(`{a="b",c="d"}`, `{"a":"b","c":"d"}`) f(`{a="b",c="d"}`, `{"a":"b","c":"d"}`)
f(`{a="a=,b\"c}",b="d"}`, `{"a":"a=,b\"c}","b":"d"}`) f(`{a="a=,b\"c}",b="d"}`, `{"a":"a=,b\"c}","b":"d"}`)
} }
func newTestGenericSearchOptions(tenantIDs []TenantID, f filter, neededColumns []string) *genericSearchOptions {
return &genericSearchOptions{
tenantIDs: tenantIDs,
minTimestamp: math.MinInt64,
maxTimestamp: math.MaxInt64,
filter: f,
neededColumnNames: neededColumns,
}
}