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) {
sort.Slice(rows, func(i, j int) bool {
sort.SliceStable(rows, func(i, j int) bool {
return rows[i].timestamp < rows[j].timestamp
})
}

View file

@ -19,6 +19,7 @@ according to [these docs](https://docs.victoriametrics.com/victorialogs/quicksta
## 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).
* 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.
- [`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.
- [`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.
- [`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).
@ -2300,6 +2302,30 @@ _time:5m | stats
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 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
LogsQL will support the ability to select the given number of surrounding log lines for the selected log lines
on a [per-stream](https://docs.victoriametrics.com/victorialogs/keyconcepts/#stream-fields) basis.
See the [Roadmap](https://docs.victoriametrics.com/victorialogs/roadmap/) for details.
See [`stream_context` pipe](#stream_context-pipe).
## Transformations

View file

@ -29,6 +29,7 @@ VictoriaLogs provides the following features:
- VictoriaLogs supports multitenancy - see [these docs](#multitenancy).
- 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 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 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
- Journald (systemd)
- 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)).
- 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.

View file

@ -247,7 +247,7 @@ This can help narrowing down and eliminating high-cardinality fields from [log s
### Other fields
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).
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
_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) {
t.Helper()
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{neededColumnName, "_time"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{neededColumnName, "_time"})
type result struct {
value string

View file

@ -350,7 +350,26 @@ func (q *Query) CanReturnLastNResults() bool {
// GetFilterTimeRange returns filter time range for the given q.
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.

View file

@ -219,6 +219,12 @@ func parsePipe(lex *lexer) (pipe, error) {
return nil, fmt.Errorf("cannot parse 'stats' pipe: %w", err)
}
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"):
pt, err := parsePipeTop(lex)
if err != nil {
@ -298,6 +304,7 @@ var pipeNames = func() map[string]struct{} {
"replace_regexp",
"sort",
"stats",
"stream_context",
"top",
"uniq",
"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
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 filter
@ -110,10 +116,14 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
return streamIDs[i].less(&streamIDs[j])
})
minTimestamp, maxTimestamp := q.GetFilterTimeRange()
neededColumnNames, unneededColumnNames := q.getNeededColumns()
so := &genericSearchOptions{
tenantIDs: tenantIDs,
streamIDs: streamIDs,
minTimestamp: minTimestamp,
maxTimestamp: maxTimestamp,
filter: q.f,
neededColumnNames: neededColumnNames,
unneededColumnNames: unneededColumnNames,
@ -127,10 +137,21 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
stopCh := ctx.Done()
cancels := make([]func(), len(q.pipes))
pps := make([]pipeProcessor, len(q.pipes))
var errPipe error
for i := len(q.pipes) - 1; i >= 0; i-- {
p := q.pipes[i]
ctxChild, cancel := context.WithCancel(ctx)
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()
ctx = ctxChild
@ -138,7 +159,9 @@ func (s *Storage) runQuery(ctx context.Context, tenantIDs []TenantID, q *Query,
pps[i] = pp
}
if errPipe == nil {
s.search(workersCount, so, stopCh, pp.writeBlock)
}
var errFlush error
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 {
errFlush = err
}
if errPipe != nil {
return errPipe
}
return errFlush
}
@ -642,19 +670,15 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
}(uint(i))
}
// Obtain time range from so.filter
f := so.filter
minTimestamp, maxTimestamp := getFilterTimeRange(f)
// Select partitions according to the selected time range
s.partitionsLock.Lock()
ptws := s.partitions
minDay := minTimestamp / nsecPerDay
minDay := so.minTimestamp / nsecPerDay
n := sort.Search(len(ptws), func(i int) bool {
return ptws[i].day >= minDay
})
ptws = ptws[n:]
maxDay := maxTimestamp / nsecPerDay
maxDay := so.maxTimestamp / nsecPerDay
n = sort.Search(len(ptws), func(i int) bool {
return ptws[i].day > maxDay
})
@ -665,8 +689,7 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
s.partitionsLock.Unlock()
// Obtain common filterStream from f
var sf *StreamFilter
sf, f = getCommonStreamFilter(f)
sf, f := getCommonStreamFilter(so.filter)
// Schedule concurrent search across matching partitions.
psfs := make([]partitionSearchFinalizer, len(ptws))
@ -675,7 +698,7 @@ func (s *Storage) search(workersCount int, so *genericSearchOptions, stopCh <-ch
partitionSearchConcurrencyLimitCh <- struct{}{}
wgSearchers.Add(1)
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()
<-partitionSearchConcurrencyLimitCh
}(i, ptw.pt)
@ -704,7 +727,7 @@ var partitionSearchConcurrencyLimitCh = make(chan struct{}, cgroup.AvailableCPUs
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) {
// Do not spend CPU time on search, since it is already stopped.
return func() {}
@ -728,8 +751,8 @@ func (pt *partition) search(minTimestamp, maxTimestamp int64, sf *StreamFilter,
soInternal := &searchOptions{
tenantIDs: tenantIDs,
streamIDs: streamIDs,
minTimestamp: minTimestamp,
maxTimestamp: maxTimestamp,
minTimestamp: so.minTimestamp,
maxTimestamp: so.maxTimestamp,
filter: f,
neededColumnNames: so.neededColumnNames,
unneededColumnNames: so.unneededColumnNames,
@ -1095,29 +1118,6 @@ func getCommonStreamFilter(f filter) (*StreamFilter, filter) {
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)) {
var fields []Field
for i := range streams {

View file

@ -3,6 +3,7 @@ package logstorage
import (
"context"
"fmt"
"math"
"reflect"
"sort"
"strings"
@ -87,7 +88,7 @@ func TestStorageRunQuery(t *testing.T) {
t.Helper()
err := s.RunQuery(context.Background(), tenantIDs, q, writeBlock)
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
s.MustClose()
@ -757,11 +830,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match"))
}
@ -775,11 +844,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match"))
}
@ -793,11 +858,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match"))
}
@ -812,11 +873,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -833,11 +890,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, nil)
so := &genericSearchOptions{
tenantIDs: allTenantIDs,
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions(allTenantIDs, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -854,11 +907,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{
tenantIDs: allTenantIDs,
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions(allTenantIDs, f, []string{"_msg"})
processBlock := func(_ uint, _ *blockResult) {
panic(fmt.Errorf("unexpected match"))
}
@ -874,11 +923,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -900,11 +945,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp
maxTimestamp := baseTimestamp + rowsPerBlock*1e9 + blocksPerStream
f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -934,11 +975,7 @@ func TestStorageSearch(t *testing.T) {
},
},
}
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -959,11 +996,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp + (rowsPerBlock-2)*1e9
maxTimestamp := baseTimestamp + (rowsPerBlock-1)*1e9 - 1
f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
var rowsCountTotal atomic.Uint32
processBlock := func(_ uint, br *blockResult) {
rowsCountTotal.Add(uint32(len(br.timestamps)))
@ -984,11 +1017,7 @@ func TestStorageSearch(t *testing.T) {
minTimestamp := baseTimestamp + (rowsPerBlock+1)*1e9
maxTimestamp := baseTimestamp + (rowsPerBlock+2)*1e9
f := getBaseFilter(minTimestamp, maxTimestamp, sf)
so := &genericSearchOptions{
tenantIDs: []TenantID{tenantID},
filter: f,
neededColumnNames: []string{"_msg"},
}
so := newTestGenericSearchOptions([]TenantID{tenantID}, f, []string{"_msg"})
processBlock := func(_ uint, _ *blockResult) {
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="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,
}
}