prometheus/model/textparse/benchmark_test.go
George Krajcsovits acd9aa0afb
fix(textparse/protobuf): metric family name corrupted by NHCB parser (#17156)
* fix(textparse): implement NHCB parsing in ProtoBuf parser directly

The NHCB conversion does some validation, but we can only return error
from Parser.Next() not Parser.Histogram(). So the conversion needs to
happen in Next().

There are 2 cases:
1. "always_scrape_classic_histograms" is enabled, in which case we
convert after returning the classic series. This is to be consistent
with the PromParser text parser, which collects NHCB while spitting out
classic series; then returns the NHCB.
2. "always_scrape_classic_histograms" is disabled. In which case we never
return the classic series.

Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>

* refactor(textparse): skip classic series instead of adding NHCB around

Do not return the first classic series from the EntryType state,
switch to EntrySeries. This means we need to start the histogram
field state from -3 , not -2.

In EntrySeries, skip classic series if needed.

Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>

* reuse nhcb converter

Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>

* test(textparse/nhcb): test corrupting metric family name

NHCB parse doesn't always copy the metric name from the underlying
parser. When called via HELP, UNIT, the string is directly referenced
which means that the read-ahead of NHCB can corrupt it.

Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>
2025-09-08 17:26:41 +02:00

342 lines
8.8 KiB
Go

// Copyright 2024 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package textparse
import (
"bytes"
"errors"
"fmt"
"io"
"os"
"path/filepath"
"strings"
"testing"
"github.com/prometheus/common/expfmt"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"github.com/prometheus/prometheus/model/exemplar"
"github.com/prometheus/prometheus/model/labels"
)
// BenchmarkParse... set of benchmarks analyze efficiency of parsing various
// datasets with different parsers. It mimics how scrape/scrape.go#append use parsers
// and allows comparison with expfmt decoders if applicable.
//
// NOTE(bwplotka): Previous iterations of this benchmark had different cases for isolated
// Series, Series+Metrics with and without reuse, Series+CT. Those cases are sometimes
// good to know if you are working on a certain optimization, but it does not
// make sense to persist such cases for everybody (e.g. for CI one day).
// For local iteration, feel free to adjust cases/comment out code etc.
//
// NOTE(bwplotka): Those benchmarks are purposefully categorized per data-sets,
// to avoid temptation to assess "what parser (OM, proto, prom) is the fastest,
// in general" here due to not every parser supporting every data set type.
// Use scrape.BenchmarkScrapeLoopAppend if you want one benchmark comparing parsers fairly.
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkParsePromText' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkParsePromText(b *testing.B) {
data := readTestdataFile(b, "alltypes.237mfs.prom.txt")
for _, parser := range []string{
"promtext",
"omtext", // Compare how omtext parser deals with Prometheus text format.
"expfmt-promtext",
} {
b.Run(fmt.Sprintf("parser=%v", parser), func(b *testing.B) {
if strings.HasPrefix(parser, "expfmt-") {
benchExpFmt(b, data, parser)
} else {
benchParse(b, data, parser)
}
})
}
}
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkParsePromText_NoMeta' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkParsePromText_NoMeta(b *testing.B) {
data := readTestdataFile(b, "alltypes.237mfs.nometa.prom.txt")
for _, parser := range []string{
"promtext",
"expfmt-promtext",
} {
b.Run(fmt.Sprintf("parser=%v", parser), func(b *testing.B) {
if strings.HasPrefix(parser, "expfmt-") {
benchExpFmt(b, data, parser)
} else {
benchParse(b, data, parser)
}
})
}
}
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkParseOMText' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkParseOMText(b *testing.B) {
data := readTestdataFile(b, "alltypes.5mfs.om.txt")
// TODO(bwplotka): Add comparison with expfmt.TypeOpenMetrics once expfmt
// support OM exemplars, see https://github.com/prometheus/common/issues/703.
benchParse(b, data, "omtext")
}
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkParsePromProto' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkParsePromProto(b *testing.B) {
data := createTestProtoBuf(b).Bytes()
// TODO(bwplotka): Add comparison with expfmt.TypeProtoDelim once expfmt
// support GAUGE_HISTOGRAM, see https://github.com/prometheus/common/issues/430.
benchParse(b, data, "promproto")
}
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkParseOpenMetricsNHCB' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkParseOpenMetricsNHCB(b *testing.B) {
data := readTestdataFile(b, "1histogram.om.txt")
for _, parser := range []string{
"omtext", // Measure OM parser baseline for histograms.
"omtext_with_nhcb", // Measure NHCB over OM parser.
} {
b.Run(fmt.Sprintf("parser=%v", parser), func(b *testing.B) {
benchParse(b, data, parser)
})
}
}
func benchParse(b *testing.B, data []byte, parser string) {
type newParser func([]byte, *labels.SymbolTable) Parser
var newParserFn newParser
switch parser {
case "promtext":
newParserFn = func(b []byte, st *labels.SymbolTable) Parser {
return NewPromParser(b, st, false)
}
case "promproto":
newParserFn = func(b []byte, st *labels.SymbolTable) Parser {
return NewProtobufParser(b, true, false, false, st)
}
case "omtext":
newParserFn = func(b []byte, st *labels.SymbolTable) Parser {
return NewOpenMetricsParser(b, st, WithOMParserCTSeriesSkipped())
}
case "omtext_with_nhcb":
newParserFn = func(buf []byte, st *labels.SymbolTable) Parser {
p, err := New(buf, "application/openmetrics-text", "", false, true, false, false, st)
require.NoError(b, err)
return p
}
default:
b.Fatal("unknown parser", parser)
}
var (
res labels.Labels
e exemplar.Exemplar
)
b.SetBytes(int64(len(data)))
b.ReportAllocs()
b.ResetTimer()
st := labels.NewSymbolTable()
for i := 0; i < b.N; i++ {
p := newParserFn(data, st)
Inner:
for {
t, err := p.Next()
switch t {
case EntryInvalid:
if errors.Is(err, io.EOF) {
break Inner
}
b.Fatal(err)
case EntryType:
_, _ = p.Type()
continue
case EntryHelp:
_, _ = p.Help()
continue
case EntryUnit:
_, _ = p.Unit()
continue
case EntryComment:
continue
case EntryHistogram:
_, _, _, _ = p.Histogram()
case EntrySeries:
_, _, _ = p.Series()
default:
b.Fatal("not implemented entry", t)
}
p.Labels(&res)
_ = p.CreatedTimestamp()
for hasExemplar := p.Exemplar(&e); hasExemplar; hasExemplar = p.Exemplar(&e) {
}
}
}
}
func benchExpFmt(b *testing.B, data []byte, expFormatTypeStr string) {
expfmtFormatType := expfmt.TypeUnknown
switch expFormatTypeStr {
case "expfmt-promtext":
expfmtFormatType = expfmt.TypeProtoText
case "expfmt-promproto":
expfmtFormatType = expfmt.TypeProtoDelim
case "expfmt-omtext":
expfmtFormatType = expfmt.TypeOpenMetrics
default:
b.Fatal("unknown expfmt format type", expFormatTypeStr)
}
b.SetBytes(int64(len(data)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
decSamples := make(model.Vector, 0, 50)
sdec := expfmt.SampleDecoder{
Dec: expfmt.NewDecoder(bytes.NewReader(data), expfmt.NewFormat(expfmtFormatType)),
Opts: &expfmt.DecodeOptions{
Timestamp: model.TimeFromUnixNano(0),
},
}
for {
if err := sdec.Decode(&decSamples); err != nil {
if errors.Is(err, io.EOF) {
break
}
b.Fatal(err)
}
decSamples = decSamples[:0]
}
}
}
func readTestdataFile(tb testing.TB, file string) []byte {
tb.Helper()
f, err := os.Open(filepath.Join("testdata", file))
require.NoError(tb, err)
tb.Cleanup(func() {
_ = f.Close()
})
buf, err := io.ReadAll(f)
require.NoError(tb, err)
return buf
}
/*
export bench=v1 && go test ./model/textparse/... \
-run '^$' -bench '^BenchmarkCreatedTimestampPromProto' \
-benchtime 2s -count 6 -cpu 2 -benchmem -timeout 999m \
| tee ${bench}.txt
*/
func BenchmarkCreatedTimestampPromProto(b *testing.B) {
data := createTestProtoBuf(b).Bytes()
st := labels.NewSymbolTable()
p := NewProtobufParser(data, true, false, false, st)
found := false
Inner:
for {
t, err := p.Next()
switch t {
case EntryInvalid:
b.Fatal(err)
case EntryType:
m, _ := p.Type()
if string(m) == "go_memstats_alloc_bytes_total" {
found = true
break Inner
}
// Parser impl requires this (bug?)
case EntryHistogram:
_, _, _, _ = p.Histogram()
case EntrySeries:
_, _, _ = p.Series()
}
}
require.True(b, found)
b.Run("case=no-ct", func(b *testing.B) {
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
if p.CreatedTimestamp() != 0 {
b.Fatal("should be nil")
}
}
})
found = false
Inner2:
for {
t, err := p.Next()
switch t {
case EntryInvalid:
b.Fatal(err)
case EntryType:
m, _ := p.Type()
if string(m) == "test_counter_with_createdtimestamp" {
found = true
break Inner2
}
case EntryHistogram:
_, _, _, _ = p.Histogram()
case EntrySeries:
_, _, _ = p.Series()
}
}
require.True(b, found)
b.Run("case=ct", func(b *testing.B) {
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
if p.CreatedTimestamp() == 0 {
b.Fatal("should be not nil")
}
}
})
}