mirror of
https://github.com/VictoriaMetrics/VictoriaMetrics.git
synced 2024-11-21 14:44:00 +00:00
app/vmselect: optimize /api/v1/labels
and /api/v1/label/.../values
handlers when match[]
query arg is passed to them
This commit is contained in:
parent
4a94cd81ce
commit
61e03f172b
14 changed files with 600 additions and 1521 deletions
|
@ -205,7 +205,8 @@ func MetricsIndexHandler(startTime time.Time, at *auth.Token, w http.ResponseWri
|
|||
deadline := searchutils.GetDeadlineForQuery(r, startTime)
|
||||
jsonp := r.FormValue("jsonp")
|
||||
denyPartialResponse := searchutils.GetDenyPartialResponse(r)
|
||||
metricNames, isPartial, err := netstorage.GetLabelValues(nil, at, denyPartialResponse, "__name__", 0, deadline)
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, 0, 0, nil, 0)
|
||||
metricNames, isPartial, err := netstorage.GetLabelValues(nil, at, denyPartialResponse, "__name__", sq, 0, deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf(`cannot obtain metric names: %w`, err)
|
||||
}
|
||||
|
|
|
@ -366,15 +366,6 @@ func selectHandler(qt *querytracer.Tracer, startTime time.Time, w http.ResponseW
|
|||
return true
|
||||
}
|
||||
return true
|
||||
case "prometheus/api/v1/labels/count":
|
||||
labelsCountRequests.Inc()
|
||||
httpserver.EnableCORS(w, r)
|
||||
if err := prometheus.LabelsCountHandler(startTime, at, w, r); err != nil {
|
||||
labelsCountErrors.Inc()
|
||||
sendPrometheusError(w, r, err)
|
||||
return true
|
||||
}
|
||||
return true
|
||||
case "prometheus/api/v1/status/tsdb":
|
||||
statusTSDBRequests.Inc()
|
||||
httpserver.EnableCORS(w, r)
|
||||
|
@ -608,9 +599,6 @@ var (
|
|||
labelsRequests = metrics.NewCounter(`vm_http_requests_total{path="/select/{}/prometheus/api/v1/labels"}`)
|
||||
labelsErrors = metrics.NewCounter(`vm_http_request_errors_total{path="/select/{}/prometheus/api/v1/labels"}`)
|
||||
|
||||
labelsCountRequests = metrics.NewCounter(`vm_http_requests_total{path="/select/{}/prometheus/api/v1/labels/count"}`)
|
||||
labelsCountErrors = metrics.NewCounter(`vm_http_request_errors_total{path="/select/{}/prometheus/api/v1/labels/count"}`)
|
||||
|
||||
statusTSDBRequests = metrics.NewCounter(`vm_http_requests_total{path="/select/{}/prometheus/api/v1/status/tsdb"}`)
|
||||
statusTSDBErrors = metrics.NewCounter(`vm_http_request_errors_total{path="/select/{}/prometheus/api/v1/status/tsdb"}`)
|
||||
|
||||
|
|
|
@ -683,62 +683,57 @@ func DeleteSeries(qt *querytracer.Tracer, at *auth.Token, sq *storage.SearchQuer
|
|||
return deletedTotal, nil
|
||||
}
|
||||
|
||||
// GetLabelsOnTimeRange returns labels for the given tr until the given deadline.
|
||||
func GetLabelsOnTimeRange(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, tr storage.TimeRange, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get labels on timeRange=%s", &tr)
|
||||
// GetLabelNames returns label names matching the given sq until the given deadline.
|
||||
func GetLabelNames(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, sq *storage.SearchQuery, maxLabelNames int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get labels: %s", sq)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
requestData := sq.Marshal(nil)
|
||||
// Send the query to all the storage nodes in parallel.
|
||||
type nodeResult struct {
|
||||
labels []string
|
||||
err error
|
||||
labelNames []string
|
||||
err error
|
||||
}
|
||||
snr := startStorageNodesRequest(qt, denyPartialResponse, func(qt *querytracer.Tracer, idx int, sn *storageNode) interface{} {
|
||||
sn.labelsOnTimeRangeRequests.Inc()
|
||||
labels, err := sn.getLabelsOnTimeRange(qt, at.AccountID, at.ProjectID, tr, limit, deadline)
|
||||
sn.labelNamesRequests.Inc()
|
||||
labelNames, err := sn.getLabelNames(qt, requestData, maxLabelNames, deadline)
|
||||
if err != nil {
|
||||
sn.labelsOnTimeRangeErrors.Inc()
|
||||
err = fmt.Errorf("cannot get labels on time range from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
sn.labelNamesErrors.Inc()
|
||||
err = fmt.Errorf("cannot get labels from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
}
|
||||
return &nodeResult{
|
||||
labels: labels,
|
||||
err: err,
|
||||
labelNames: labelNames,
|
||||
err: err,
|
||||
}
|
||||
})
|
||||
|
||||
// Collect results
|
||||
var labels []string
|
||||
isPartial, err := snr.collectResults(partialLabelsOnTimeRangeResults, func(result interface{}) error {
|
||||
var labelNames []string
|
||||
isPartial, err := snr.collectResults(partialLabelNamesResults, func(result interface{}) error {
|
||||
nr := result.(*nodeResult)
|
||||
if nr.err != nil {
|
||||
return nr.err
|
||||
}
|
||||
labels = append(labels, nr.labels...)
|
||||
labelNames = append(labelNames, nr.labelNames...)
|
||||
return nil
|
||||
})
|
||||
qt.Printf("get %d non-duplicated labels", len(labels))
|
||||
qt.Printf("get %d non-duplicated labels", len(labelNames))
|
||||
if err != nil {
|
||||
return nil, isPartial, fmt.Errorf("cannot fetch labels on time range from vmstorage nodes: %w", err)
|
||||
return nil, isPartial, fmt.Errorf("cannot fetch labels from vmstorage nodes: %w", err)
|
||||
}
|
||||
|
||||
// Deduplicate labels
|
||||
labels = deduplicateStrings(labels)
|
||||
qt.Printf("get %d unique labels after de-duplication", len(labels))
|
||||
// Substitute "" with "__name__"
|
||||
for i := range labels {
|
||||
if labels[i] == "" {
|
||||
labels[i] = "__name__"
|
||||
}
|
||||
labelNames = deduplicateStrings(labelNames)
|
||||
qt.Printf("get %d unique labels after de-duplication", len(labelNames))
|
||||
if maxLabelNames > 0 && maxLabelNames < len(labelNames) {
|
||||
labelNames = labelNames[:maxLabelNames]
|
||||
}
|
||||
if limit > 0 && limit < len(labels) {
|
||||
labels = labels[:limit]
|
||||
}
|
||||
// Sort labels like Prometheus does
|
||||
sort.Strings(labels)
|
||||
qt.Printf("sort %d labels", len(labels))
|
||||
return labels, isPartial, nil
|
||||
// Sort labelNames like Prometheus does
|
||||
sort.Strings(labelNames)
|
||||
qt.Printf("sort %d labels", len(labelNames))
|
||||
return labelNames, isPartial, nil
|
||||
}
|
||||
|
||||
// GetGraphiteTags returns Graphite tags until the given deadline.
|
||||
|
@ -748,7 +743,8 @@ func GetGraphiteTags(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse
|
|||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
labels, isPartial, err := GetLabels(qt, at, denyPartialResponse, 0, deadline)
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, 0, 0, nil, 0)
|
||||
labels, isPartial, err := GetLabelNames(qt, at, denyPartialResponse, sq, 0, deadline)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
@ -788,159 +784,15 @@ func hasString(a []string, s string) bool {
|
|||
return false
|
||||
}
|
||||
|
||||
// GetLabels returns labels until the given deadline.
|
||||
func GetLabels(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get labels")
|
||||
// GetLabelValues returns label values matching the given labelName and sq until the given deadline.
|
||||
func GetLabelValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, labelName string, sq *storage.SearchQuery,
|
||||
maxLabelValues int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get values for label %s: %s", labelName, sq)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
// Send the query to all the storage nodes in parallel.
|
||||
type nodeResult struct {
|
||||
labels []string
|
||||
err error
|
||||
}
|
||||
snr := startStorageNodesRequest(qt, denyPartialResponse, func(qt *querytracer.Tracer, idx int, sn *storageNode) interface{} {
|
||||
sn.labelsRequests.Inc()
|
||||
labels, err := sn.getLabels(qt, at.AccountID, at.ProjectID, limit, deadline)
|
||||
if err != nil {
|
||||
sn.labelsErrors.Inc()
|
||||
err = fmt.Errorf("cannot get labels from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
}
|
||||
return &nodeResult{
|
||||
labels: labels,
|
||||
err: err,
|
||||
}
|
||||
})
|
||||
|
||||
// Collect results
|
||||
var labels []string
|
||||
isPartial, err := snr.collectResults(partialLabelsResults, func(result interface{}) error {
|
||||
nr := result.(*nodeResult)
|
||||
if nr.err != nil {
|
||||
return nr.err
|
||||
}
|
||||
labels = append(labels, nr.labels...)
|
||||
return nil
|
||||
})
|
||||
qt.Printf("get %d non-duplicated labels from global index", len(labels))
|
||||
if err != nil {
|
||||
return nil, isPartial, fmt.Errorf("cannot fetch labels from vmstorage nodes: %w", err)
|
||||
}
|
||||
|
||||
// Deduplicate labels
|
||||
labels = deduplicateStrings(labels)
|
||||
qt.Printf("get %d unique labels after de-duplication", len(labels))
|
||||
// Substitute "" with "__name__"
|
||||
for i := range labels {
|
||||
if labels[i] == "" {
|
||||
labels[i] = "__name__"
|
||||
}
|
||||
}
|
||||
// Sort labels like Prometheus does
|
||||
if limit > 0 && limit < len(labels) {
|
||||
labels = labels[:limit]
|
||||
}
|
||||
sort.Strings(labels)
|
||||
qt.Printf("sort %d labels", len(labels))
|
||||
return labels, isPartial, nil
|
||||
}
|
||||
|
||||
// GetLabelValuesOnTimeRange returns label values for the given labelName on the given tr
|
||||
// until the given deadline.
|
||||
func GetLabelValuesOnTimeRange(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, labelName string,
|
||||
tr storage.TimeRange, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get values for label %s on a timeRange %s", labelName, &tr)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
if labelName == "__name__" {
|
||||
labelName = ""
|
||||
}
|
||||
|
||||
// Send the query to all the storage nodes in parallel.
|
||||
type nodeResult struct {
|
||||
labelValues []string
|
||||
err error
|
||||
}
|
||||
snr := startStorageNodesRequest(qt, denyPartialResponse, func(qt *querytracer.Tracer, idx int, sn *storageNode) interface{} {
|
||||
sn.labelValuesOnTimeRangeRequests.Inc()
|
||||
labelValues, err := sn.getLabelValuesOnTimeRange(qt, at.AccountID, at.ProjectID, labelName, tr, limit, deadline)
|
||||
if err != nil {
|
||||
sn.labelValuesOnTimeRangeErrors.Inc()
|
||||
err = fmt.Errorf("cannot get label values on time range from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
}
|
||||
return &nodeResult{
|
||||
labelValues: labelValues,
|
||||
err: err,
|
||||
}
|
||||
})
|
||||
|
||||
// Collect results
|
||||
var labelValues []string
|
||||
isPartial, err := snr.collectResults(partialLabelValuesOnTimeRangeResults, func(result interface{}) error {
|
||||
nr := result.(*nodeResult)
|
||||
if nr.err != nil {
|
||||
return nr.err
|
||||
}
|
||||
labelValues = append(labelValues, nr.labelValues...)
|
||||
return nil
|
||||
})
|
||||
qt.Printf("get %d non-duplicated label values", len(labelValues))
|
||||
if err != nil {
|
||||
return nil, isPartial, fmt.Errorf("cannot fetch label values on time range from vmstorage nodes: %w", err)
|
||||
}
|
||||
|
||||
// Deduplicate label values
|
||||
labelValues = deduplicateStrings(labelValues)
|
||||
qt.Printf("get %d unique label values after de-duplication", len(labelValues))
|
||||
// Sort labelValues like Prometheus does
|
||||
if limit > 0 && limit < len(labelValues) {
|
||||
labelValues = labelValues[:limit]
|
||||
}
|
||||
sort.Strings(labelValues)
|
||||
qt.Printf("sort %d label values", len(labelValues))
|
||||
return labelValues, isPartial, nil
|
||||
}
|
||||
|
||||
// GetGraphiteTagValues returns tag values for the given tagName until the given deadline.
|
||||
func GetGraphiteTagValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, tagName, filter string, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get graphite tag values for tagName=%s, filter=%s, limit=%d", tagName, filter, limit)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
if tagName == "name" {
|
||||
tagName = ""
|
||||
}
|
||||
tagValues, isPartial, err := GetLabelValues(qt, at, denyPartialResponse, tagName, 0, deadline)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
if len(filter) > 0 {
|
||||
tagValues, err = applyGraphiteRegexpFilter(filter, tagValues)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
if limit > 0 && limit < len(tagValues) {
|
||||
tagValues = tagValues[:limit]
|
||||
}
|
||||
return tagValues, isPartial, nil
|
||||
}
|
||||
|
||||
// GetLabelValues returns label values for the given labelName
|
||||
// until the given deadline.
|
||||
func GetLabelValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, labelName string, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get values for label %s", labelName)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
if labelName == "__name__" {
|
||||
labelName = ""
|
||||
}
|
||||
requestData := sq.Marshal(nil)
|
||||
|
||||
// Send the query to all the storage nodes in parallel.
|
||||
type nodeResult struct {
|
||||
|
@ -949,7 +801,7 @@ func GetLabelValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse
|
|||
}
|
||||
snr := startStorageNodesRequest(qt, denyPartialResponse, func(qt *querytracer.Tracer, idx int, sn *storageNode) interface{} {
|
||||
sn.labelValuesRequests.Inc()
|
||||
labelValues, err := sn.getLabelValues(qt, at.AccountID, at.ProjectID, labelName, limit, deadline)
|
||||
labelValues, err := sn.getLabelValues(qt, labelName, requestData, maxLabelValues, deadline)
|
||||
if err != nil {
|
||||
sn.labelValuesErrors.Inc()
|
||||
err = fmt.Errorf("cannot get label values from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
|
@ -979,14 +831,41 @@ func GetLabelValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse
|
|||
labelValues = deduplicateStrings(labelValues)
|
||||
qt.Printf("get %d unique label values after de-duplication", len(labelValues))
|
||||
// Sort labelValues like Prometheus does
|
||||
if limit > 0 && limit < len(labelValues) {
|
||||
labelValues = labelValues[:limit]
|
||||
if maxLabelValues > 0 && maxLabelValues < len(labelValues) {
|
||||
labelValues = labelValues[:maxLabelValues]
|
||||
}
|
||||
sort.Strings(labelValues)
|
||||
qt.Printf("sort %d label values", len(labelValues))
|
||||
return labelValues, isPartial, nil
|
||||
}
|
||||
|
||||
// GetGraphiteTagValues returns tag values for the given tagName until the given deadline.
|
||||
func GetGraphiteTagValues(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, tagName, filter string, limit int, deadline searchutils.Deadline) ([]string, bool, error) {
|
||||
qt = qt.NewChild("get graphite tag values for tagName=%s, filter=%s, limit=%d", tagName, filter, limit)
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
if tagName == "name" {
|
||||
tagName = ""
|
||||
}
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, 0, 0, nil, 0)
|
||||
tagValues, isPartial, err := GetLabelValues(qt, at, denyPartialResponse, tagName, sq, 0, deadline)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
if len(filter) > 0 {
|
||||
tagValues, err = applyGraphiteRegexpFilter(filter, tagValues)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
if limit > 0 && limit < len(tagValues) {
|
||||
tagValues = tagValues[:limit]
|
||||
}
|
||||
return tagValues, isPartial, nil
|
||||
}
|
||||
|
||||
// GetTagValueSuffixes returns tag value suffixes for the given tagKey and the given tagValuePrefix.
|
||||
//
|
||||
// It can be used for implementing https://graphite-api.readthedocs.io/en/latest/api.html#metrics-find
|
||||
|
@ -1039,89 +918,6 @@ func GetTagValueSuffixes(qt *querytracer.Tracer, at *auth.Token, denyPartialResp
|
|||
return suffixes, isPartial, nil
|
||||
}
|
||||
|
||||
// GetLabelEntries returns all the label entries for at until the given deadline.
|
||||
func GetLabelEntries(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, deadline searchutils.Deadline) ([]storage.TagEntry, bool, error) {
|
||||
qt = qt.NewChild("get label entries")
|
||||
defer qt.Done()
|
||||
if deadline.Exceeded() {
|
||||
return nil, false, fmt.Errorf("timeout exceeded before starting the query processing: %s", deadline.String())
|
||||
}
|
||||
// Send the query to all the storage nodes in parallel.
|
||||
type nodeResult struct {
|
||||
labelEntries []storage.TagEntry
|
||||
err error
|
||||
}
|
||||
snr := startStorageNodesRequest(qt, denyPartialResponse, func(qt *querytracer.Tracer, idx int, sn *storageNode) interface{} {
|
||||
sn.labelEntriesRequests.Inc()
|
||||
labelEntries, err := sn.getLabelEntries(qt, at.AccountID, at.ProjectID, deadline)
|
||||
if err != nil {
|
||||
sn.labelEntriesErrors.Inc()
|
||||
err = fmt.Errorf("cannot get label entries from vmstorage %s: %w", sn.connPool.Addr(), err)
|
||||
}
|
||||
return &nodeResult{
|
||||
labelEntries: labelEntries,
|
||||
err: err,
|
||||
}
|
||||
})
|
||||
|
||||
// Collect results
|
||||
var labelEntries []storage.TagEntry
|
||||
isPartial, err := snr.collectResults(partialLabelEntriesResults, func(result interface{}) error {
|
||||
nr := result.(*nodeResult)
|
||||
if nr.err != nil {
|
||||
return nr.err
|
||||
}
|
||||
labelEntries = append(labelEntries, nr.labelEntries...)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, isPartial, fmt.Errorf("cannot featch label etnries from vmstorage nodes: %w", err)
|
||||
}
|
||||
qt.Printf("get %d label entries before de-duplication", len(labelEntries))
|
||||
|
||||
// Substitute "" with "__name__"
|
||||
for i := range labelEntries {
|
||||
e := &labelEntries[i]
|
||||
if e.Key == "" {
|
||||
e.Key = "__name__"
|
||||
}
|
||||
}
|
||||
|
||||
// Deduplicate label entries
|
||||
labelEntries = deduplicateLabelEntries(labelEntries)
|
||||
qt.Printf("left %d label entries after de-duplication", len(labelEntries))
|
||||
|
||||
// Sort labelEntries by the number of label values in each entry.
|
||||
sort.Slice(labelEntries, func(i, j int) bool {
|
||||
a, b := labelEntries[i].Values, labelEntries[j].Values
|
||||
if len(a) != len(b) {
|
||||
return len(a) > len(b)
|
||||
}
|
||||
return labelEntries[i].Key > labelEntries[j].Key
|
||||
})
|
||||
qt.Printf("sort %d label entries", len(labelEntries))
|
||||
|
||||
return labelEntries, isPartial, nil
|
||||
}
|
||||
|
||||
func deduplicateLabelEntries(src []storage.TagEntry) []storage.TagEntry {
|
||||
m := make(map[string][]string, len(src))
|
||||
for i := range src {
|
||||
e := &src[i]
|
||||
m[e.Key] = append(m[e.Key], e.Values...)
|
||||
}
|
||||
dst := make([]storage.TagEntry, 0, len(m))
|
||||
for key, values := range m {
|
||||
values := deduplicateStrings(values)
|
||||
sort.Strings(values)
|
||||
dst = append(dst, storage.TagEntry{
|
||||
Key: key,
|
||||
Values: values,
|
||||
})
|
||||
}
|
||||
return dst
|
||||
}
|
||||
|
||||
func deduplicateStrings(a []string) []string {
|
||||
m := make(map[string]bool, len(a))
|
||||
for _, s := range a {
|
||||
|
@ -1679,36 +1475,18 @@ type storageNode struct {
|
|||
// The number of DeleteSeries request errors to storageNode.
|
||||
deleteSeriesErrors *metrics.Counter
|
||||
|
||||
// The number of requests to labels.
|
||||
labelsOnTimeRangeRequests *metrics.Counter
|
||||
// The number of requests to labelNames.
|
||||
labelNamesRequests *metrics.Counter
|
||||
|
||||
// The number of requests to labels.
|
||||
labelsRequests *metrics.Counter
|
||||
|
||||
// The number of errors during requests to labels.
|
||||
labelsOnTimeRangeErrors *metrics.Counter
|
||||
|
||||
// The number of errors during requests to labels.
|
||||
labelsErrors *metrics.Counter
|
||||
|
||||
// The number of requests to labelValuesOnTimeRange.
|
||||
labelValuesOnTimeRangeRequests *metrics.Counter
|
||||
// The number of errors during requests to labelNames.
|
||||
labelNamesErrors *metrics.Counter
|
||||
|
||||
// The number of requests to labelValues.
|
||||
labelValuesRequests *metrics.Counter
|
||||
|
||||
// The number of errors during requests to labelValuesOnTimeRange.
|
||||
labelValuesOnTimeRangeErrors *metrics.Counter
|
||||
|
||||
// The number of errors during requests to labelValues.
|
||||
labelValuesErrors *metrics.Counter
|
||||
|
||||
// The number of requests to labelEntries.
|
||||
labelEntriesRequests *metrics.Counter
|
||||
|
||||
// The number of errors during requests to labelEntries.
|
||||
labelEntriesErrors *metrics.Counter
|
||||
|
||||
// The number of requests to tagValueSuffixes.
|
||||
tagValueSuffixesRequests *metrics.Counter
|
||||
|
||||
|
@ -1778,66 +1556,33 @@ func (sn *storageNode) deleteMetrics(qt *querytracer.Tracer, requestData []byte,
|
|||
return deletedCount, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelsOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, tr storage.TimeRange, limit int, deadline searchutils.Deadline) ([]string, error) {
|
||||
func (sn *storageNode) getLabelNames(qt *querytracer.Tracer, requestData []byte, maxLabelNames int, deadline searchutils.Deadline) ([]string, error) {
|
||||
var labels []string
|
||||
f := func(bc *handshake.BufferedConn) error {
|
||||
ls, err := sn.getLabelsOnTimeRangeOnConn(bc, accountID, projectID, tr, limit)
|
||||
ls, err := sn.getLabelNamesOnConn(bc, requestData, maxLabelNames)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
labels = ls
|
||||
return nil
|
||||
}
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelsOnTimeRange_v3", f, deadline); err != nil {
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelNames_v5", f, deadline); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return labels, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabels(qt *querytracer.Tracer, accountID, projectID uint32, limit int, deadline searchutils.Deadline) ([]string, error) {
|
||||
var labels []string
|
||||
f := func(bc *handshake.BufferedConn) error {
|
||||
ls, err := sn.getLabelsOnConn(bc, accountID, projectID, limit)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
labels = ls
|
||||
return nil
|
||||
}
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labels_v4", f, deadline); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return labels, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelValuesOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, labelName string,
|
||||
tr storage.TimeRange, limit int, deadline searchutils.Deadline) ([]string, error) {
|
||||
func (sn *storageNode) getLabelValues(qt *querytracer.Tracer, labelName string, requestData []byte, maxLabelValues int, deadline searchutils.Deadline) ([]string, error) {
|
||||
var labelValues []string
|
||||
f := func(bc *handshake.BufferedConn) error {
|
||||
lvs, err := sn.getLabelValuesOnTimeRangeOnConn(bc, accountID, projectID, labelName, tr, limit)
|
||||
lvs, err := sn.getLabelValuesOnConn(bc, labelName, requestData, maxLabelValues)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
labelValues = lvs
|
||||
return nil
|
||||
}
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelValuesOnTimeRange_v3", f, deadline); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return labelValues, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelValues(qt *querytracer.Tracer, accountID, projectID uint32, labelName string, limit int, deadline searchutils.Deadline) ([]string, error) {
|
||||
var labelValues []string
|
||||
f := func(bc *handshake.BufferedConn) error {
|
||||
lvs, err := sn.getLabelValuesOnConn(bc, accountID, projectID, labelName, limit)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
labelValues = lvs
|
||||
return nil
|
||||
}
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelValues_v4", f, deadline); err != nil {
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelValues_v5", f, deadline); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return labelValues, nil
|
||||
|
@ -1860,22 +1605,6 @@ func (sn *storageNode) getTagValueSuffixes(qt *querytracer.Tracer, accountID, pr
|
|||
return suffixes, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelEntries(qt *querytracer.Tracer, accountID, projectID uint32, deadline searchutils.Deadline) ([]storage.TagEntry, error) {
|
||||
var tagEntries []storage.TagEntry
|
||||
f := func(bc *handshake.BufferedConn) error {
|
||||
tes, err := sn.getLabelEntriesOnConn(bc, accountID, projectID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
tagEntries = tes
|
||||
return nil
|
||||
}
|
||||
if err := sn.execOnConnWithPossibleRetry(qt, "labelEntries_v4", f, deadline); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return tagEntries, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getTSDBStatusForDate(qt *querytracer.Tracer, accountID, projectID uint32,
|
||||
date uint64, topN, maxMetrics int, deadline searchutils.Deadline) (*storage.TSDBStatus, error) {
|
||||
var status *storage.TSDBStatus
|
||||
|
@ -2142,18 +1871,15 @@ func (sn *storageNode) deleteMetricsOnConn(bc *handshake.BufferedConn, requestDa
|
|||
return int(deletedCount), nil
|
||||
}
|
||||
|
||||
const maxLabelSize = 16 * 1024 * 1024
|
||||
const maxLabelNameSize = 16 * 1024 * 1024
|
||||
|
||||
func (sn *storageNode) getLabelsOnTimeRangeOnConn(bc *handshake.BufferedConn, accountID, projectID uint32, tr storage.TimeRange, limit int) ([]string, error) {
|
||||
func (sn *storageNode) getLabelNamesOnConn(bc *handshake.BufferedConn, requestData []byte, maxLabelNames int) ([]string, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
return nil, err
|
||||
if err := writeBytes(bc, requestData); err != nil {
|
||||
return nil, fmt.Errorf("cannot write requestData: %w", err)
|
||||
}
|
||||
if err := writeTimeRange(bc, tr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := writeLimit(bc, limit); err != nil {
|
||||
return nil, err
|
||||
if err := writeLimit(bc, maxLabelNames); err != nil {
|
||||
return nil, fmt.Errorf("cannot write maxLabelNames=%d: %w", maxLabelNames, err)
|
||||
}
|
||||
if err := bc.Flush(); err != nil {
|
||||
return nil, fmt.Errorf("cannot flush request to conn: %w", err)
|
||||
|
@ -2171,43 +1897,7 @@ func (sn *storageNode) getLabelsOnTimeRangeOnConn(bc *handshake.BufferedConn, ac
|
|||
// Read response
|
||||
var labels []string
|
||||
for {
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read labels: %w", err)
|
||||
}
|
||||
if len(buf) == 0 {
|
||||
// Reached the end of the response
|
||||
return labels, nil
|
||||
}
|
||||
labels = append(labels, string(buf))
|
||||
}
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelsOnConn(bc *handshake.BufferedConn, accountID, projectID uint32, limit int) ([]string, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := writeLimit(bc, limit); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := bc.Flush(); err != nil {
|
||||
return nil, fmt.Errorf("cannot flush request to conn: %w", err)
|
||||
}
|
||||
|
||||
// Read response error.
|
||||
buf, err := readBytes(nil, bc, maxErrorMessageSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read error message: %w", err)
|
||||
}
|
||||
if len(buf) > 0 {
|
||||
return nil, newErrRemote(buf)
|
||||
}
|
||||
|
||||
// Read response
|
||||
var labels []string
|
||||
for {
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelSize)
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelNameSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read labels: %w", err)
|
||||
}
|
||||
|
@ -2221,51 +1911,16 @@ func (sn *storageNode) getLabelsOnConn(bc *handshake.BufferedConn, accountID, pr
|
|||
|
||||
const maxLabelValueSize = 16 * 1024 * 1024
|
||||
|
||||
func (sn *storageNode) getLabelValuesOnTimeRangeOnConn(bc *handshake.BufferedConn, accountID, projectID uint32, labelName string, tr storage.TimeRange, limit int) ([]string, error) {
|
||||
func (sn *storageNode) getLabelValuesOnConn(bc *handshake.BufferedConn, labelName string, requestData []byte, maxLabelValues int) ([]string, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := writeBytes(bc, []byte(labelName)); err != nil {
|
||||
return nil, fmt.Errorf("cannot send labelName=%q to conn: %w", labelName, err)
|
||||
}
|
||||
if err := writeTimeRange(bc, tr); err != nil {
|
||||
return nil, err
|
||||
if err := writeBytes(bc, requestData); err != nil {
|
||||
return nil, fmt.Errorf("cannot write requestData: %w", err)
|
||||
}
|
||||
if err := writeLimit(bc, limit); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := bc.Flush(); err != nil {
|
||||
return nil, fmt.Errorf("cannot flush labelName to conn: %w", err)
|
||||
}
|
||||
|
||||
// Read response error.
|
||||
buf, err := readBytes(nil, bc, maxErrorMessageSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read error message: %w", err)
|
||||
}
|
||||
if len(buf) > 0 {
|
||||
return nil, newErrRemote(buf)
|
||||
}
|
||||
|
||||
// Read response
|
||||
labelValues, _, err := readLabelValues(buf, bc)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return labelValues, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelValuesOnConn(bc *handshake.BufferedConn, accountID, projectID uint32, labelName string, limit int) ([]string, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := writeBytes(bc, []byte(labelName)); err != nil {
|
||||
return nil, fmt.Errorf("cannot send labelName=%q to conn: %w", labelName, err)
|
||||
}
|
||||
if err := writeLimit(bc, limit); err != nil {
|
||||
return nil, err
|
||||
if err := writeLimit(bc, maxLabelValues); err != nil {
|
||||
return nil, fmt.Errorf("cannot write maxLabelValues=%d: %w", maxLabelValues, err)
|
||||
}
|
||||
if err := bc.Flush(); err != nil {
|
||||
return nil, fmt.Errorf("cannot flush labelName to conn: %w", err)
|
||||
|
@ -2352,48 +2007,6 @@ func (sn *storageNode) getTagValueSuffixesOnConn(bc *handshake.BufferedConn, acc
|
|||
return suffixes, nil
|
||||
}
|
||||
|
||||
func (sn *storageNode) getLabelEntriesOnConn(bc *handshake.BufferedConn, accountID, projectID uint32) ([]storage.TagEntry, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := bc.Flush(); err != nil {
|
||||
return nil, fmt.Errorf("cannot flush request to conn: %w", err)
|
||||
}
|
||||
|
||||
// Read response error.
|
||||
buf, err := readBytes(nil, bc, maxErrorMessageSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read error message: %w", err)
|
||||
}
|
||||
if len(buf) > 0 {
|
||||
return nil, newErrRemote(buf)
|
||||
}
|
||||
|
||||
// Read response.
|
||||
var labelEntries []storage.TagEntry
|
||||
for {
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read label: %w", err)
|
||||
}
|
||||
if len(buf) == 0 {
|
||||
// Reached the end of the response
|
||||
return labelEntries, nil
|
||||
}
|
||||
label := string(buf)
|
||||
var values []string
|
||||
values, buf, err = readLabelValues(buf, bc)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read values for label %q: %w", label, err)
|
||||
}
|
||||
labelEntries = append(labelEntries, storage.TagEntry{
|
||||
Key: label,
|
||||
Values: values,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func (sn *storageNode) getTSDBStatusForDateOnConn(bc *handshake.BufferedConn, accountID, projectID uint32, date uint64, topN, maxMetrics int) (*storage.TSDBStatus, error) {
|
||||
// Send the request to sn.
|
||||
if err := sendAccountIDProjectID(bc, accountID, projectID); err != nil {
|
||||
|
@ -2493,7 +2106,7 @@ func readTopHeapEntries(bc *handshake.BufferedConn) ([]storage.TopHeapEntry, err
|
|||
var a []storage.TopHeapEntry
|
||||
var buf []byte
|
||||
for i := uint64(0); i < n; i++ {
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelSize)
|
||||
buf, err = readBytes(buf[:0], bc, maxLabelNameSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot read label name: %w", err)
|
||||
}
|
||||
|
@ -2745,34 +2358,28 @@ func InitStorageNodes(addrs []string) {
|
|||
|
||||
concurrentQueries: metrics.NewCounter(fmt.Sprintf(`vm_concurrent_queries{name="vmselect", addr=%q}`, addr)),
|
||||
|
||||
registerMetricNamesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="registerMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
registerMetricNamesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="registerMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
deleteSeriesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="deleteSeries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
deleteSeriesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="deleteSeries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelsOnTimeRangeRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelsOnTimeRange", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelsRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labels", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelsOnTimeRangeErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelsOnTimeRange", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelsErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labels", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesOnTimeRangeRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelValuesOnTimeRange", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelValues", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesOnTimeRangeErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelValuesOnTimeRange", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelValues", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelEntriesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelEntries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelEntriesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelEntries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tagValueSuffixesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tagValueSuffixes", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tagValueSuffixesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tagValueSuffixes", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tsdbStatus", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tsdbStatus", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusWithFiltersRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tsdbStatusWithFilters", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusWithFiltersErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tsdbStatusWithFilters", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
seriesCountRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="seriesCount", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
seriesCountErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="seriesCount", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchMetricNamesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="searchMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="search", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchMetricNamesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="searchMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="search", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
metricBlocksRead: metrics.NewCounter(fmt.Sprintf(`vm_metric_blocks_read_total{name="vmselect", addr=%q}`, addr)),
|
||||
metricRowsRead: metrics.NewCounter(fmt.Sprintf(`vm_metric_rows_read_total{name="vmselect", addr=%q}`, addr)),
|
||||
registerMetricNamesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="registerMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
registerMetricNamesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="registerMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
deleteSeriesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="deleteSeries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
deleteSeriesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="deleteSeries", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelNamesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelNamesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="labelValues", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
labelValuesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="labelValues", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tagValueSuffixesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tagValueSuffixes", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tagValueSuffixesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tagValueSuffixes", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tsdbStatus", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tsdbStatus", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusWithFiltersRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="tsdbStatusWithFilters", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
tsdbStatusWithFiltersErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="tsdbStatusWithFilters", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
seriesCountRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="seriesCount", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
seriesCountErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="seriesCount", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchMetricNamesRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="searchMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchRequests: metrics.NewCounter(fmt.Sprintf(`vm_requests_total{action="search", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchMetricNamesErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="searchMetricNames", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
searchErrors: metrics.NewCounter(fmt.Sprintf(`vm_request_errors_total{action="search", type="rpcClient", name="vmselect", addr=%q}`, addr)),
|
||||
metricBlocksRead: metrics.NewCounter(fmt.Sprintf(`vm_metric_blocks_read_total{name="vmselect", addr=%q}`, addr)),
|
||||
metricRowsRead: metrics.NewCounter(fmt.Sprintf(`vm_metric_rows_read_total{name="vmselect", addr=%q}`, addr)),
|
||||
}
|
||||
storageNodes = append(storageNodes, sn)
|
||||
}
|
||||
|
@ -2784,16 +2391,13 @@ func Stop() {
|
|||
}
|
||||
|
||||
var (
|
||||
partialLabelsOnTimeRangeResults = metrics.NewCounter(`vm_partial_results_total{type="labels_on_time_range", name="vmselect"}`)
|
||||
partialLabelsResults = metrics.NewCounter(`vm_partial_results_total{type="labels", name="vmselect"}`)
|
||||
partialLabelValuesOnTimeRangeResults = metrics.NewCounter(`vm_partial_results_total{type="label_values_on_time_range", name="vmselect"}`)
|
||||
partialLabelValuesResults = metrics.NewCounter(`vm_partial_results_total{type="label_values", name="vmselect"}`)
|
||||
partialTagValueSuffixesResults = metrics.NewCounter(`vm_partial_results_total{type="tag_value_suffixes", name="vmselect"}`)
|
||||
partialLabelEntriesResults = metrics.NewCounter(`vm_partial_results_total{type="label_entries", name="vmselect"}`)
|
||||
partialTSDBStatusResults = metrics.NewCounter(`vm_partial_results_total{type="tsdb_status", name="vmselect"}`)
|
||||
partialSeriesCountResults = metrics.NewCounter(`vm_partial_results_total{type="series_count", name="vmselect"}`)
|
||||
partialSearchMetricNamesResults = metrics.NewCounter(`vm_partial_results_total{type="search_metric_names", name="vmselect"}`)
|
||||
partialSearchResults = metrics.NewCounter(`vm_partial_results_total{type="search", name="vmselect"}`)
|
||||
partialLabelNamesResults = metrics.NewCounter(`vm_partial_results_total{type="labels_names", name="vmselect"}`)
|
||||
partialLabelValuesResults = metrics.NewCounter(`vm_partial_results_total{type="label_values", name="vmselect"}`)
|
||||
partialTagValueSuffixesResults = metrics.NewCounter(`vm_partial_results_total{type="tag_value_suffixes", name="vmselect"}`)
|
||||
partialTSDBStatusResults = metrics.NewCounter(`vm_partial_results_total{type="tsdb_status", name="vmselect"}`)
|
||||
partialSeriesCountResults = metrics.NewCounter(`vm_partial_results_total{type="series_count", name="vmselect"}`)
|
||||
partialSearchMetricNamesResults = metrics.NewCounter(`vm_partial_results_total{type="search_metric_names", name="vmselect"}`)
|
||||
partialSearchResults = metrics.NewCounter(`vm_partial_results_total{type="search", name="vmselect"}`)
|
||||
)
|
||||
|
||||
func applyGraphiteRegexpFilter(filter string, ss []string) ([]string, error) {
|
||||
|
|
|
@ -1,18 +0,0 @@
|
|||
{% import "github.com/VictoriaMetrics/VictoriaMetrics/lib/storage" %}
|
||||
|
||||
{% stripspace %}
|
||||
LabelsCountResponse generates response for /api/v1/labels/count .
|
||||
{% func LabelsCountResponse(isPartial bool, labelEntries []storage.TagEntry) %}
|
||||
{
|
||||
"status":"success",
|
||||
"isPartial":{% if isPartial %}true{% else %}false{% endif %},
|
||||
"data":{
|
||||
{% for i, e := range labelEntries %}
|
||||
{%q= e.Key %}:{%d= len(e.Values) %}
|
||||
{% if i+1 < len(labelEntries) %},{% endif %}
|
||||
{% endfor %}
|
||||
}
|
||||
}
|
||||
{% endfunc %}
|
||||
|
||||
{% endstripspace %}
|
|
@ -1,86 +0,0 @@
|
|||
// Code generated by qtc from "labels_count_response.qtpl". DO NOT EDIT.
|
||||
// See https://github.com/valyala/quicktemplate for details.
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:1
|
||||
package prometheus
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:1
|
||||
import "github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
|
||||
|
||||
// LabelsCountResponse generates response for /api/v1/labels/count .
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:5
|
||||
import (
|
||||
qtio422016 "io"
|
||||
|
||||
qt422016 "github.com/valyala/quicktemplate"
|
||||
)
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:5
|
||||
var (
|
||||
_ = qtio422016.Copy
|
||||
_ = qt422016.AcquireByteBuffer
|
||||
)
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:5
|
||||
func StreamLabelsCountResponse(qw422016 *qt422016.Writer, isPartial bool, labelEntries []storage.TagEntry) {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:5
|
||||
qw422016.N().S(`{"status":"success","isPartial":`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
if isPartial {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
qw422016.N().S(`true`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
} else {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
qw422016.N().S(`false`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
}
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:8
|
||||
qw422016.N().S(`,"data":{`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:10
|
||||
for i, e := range labelEntries {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:11
|
||||
qw422016.N().Q(e.Key)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:11
|
||||
qw422016.N().S(`:`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:11
|
||||
qw422016.N().D(len(e.Values))
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:12
|
||||
if i+1 < len(labelEntries) {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:12
|
||||
qw422016.N().S(`,`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:12
|
||||
}
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:13
|
||||
}
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:13
|
||||
qw422016.N().S(`}}`)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
}
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
func WriteLabelsCountResponse(qq422016 qtio422016.Writer, isPartial bool, labelEntries []storage.TagEntry) {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
qw422016 := qt422016.AcquireWriter(qq422016)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
StreamLabelsCountResponse(qw422016, isPartial, labelEntries)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
qt422016.ReleaseWriter(qw422016)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
}
|
||||
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
func LabelsCountResponse(isPartial bool, labelEntries []storage.TagEntry) string {
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
qb422016 := qt422016.AcquireByteBuffer()
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
WriteLabelsCountResponse(qb422016, isPartial, labelEntries)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
qs422016 := string(qb422016.B)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
qt422016.ReleaseByteBuffer(qb422016)
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
return qs422016
|
||||
//line app/vmselect/prometheus/labels_count_response.qtpl:16
|
||||
}
|
|
@ -6,7 +6,6 @@ import (
|
|||
"math"
|
||||
"net"
|
||||
"net/http"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
@ -513,40 +512,11 @@ func LabelValuesHandler(qt *querytracer.Tracer, startTime time.Time, at *auth.To
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var labelValues []string
|
||||
var isPartial bool
|
||||
denyPartialResponse := searchutils.GetDenyPartialResponse(r)
|
||||
if len(cp.filterss) == 0 {
|
||||
if cp.IsDefaultTimeRange() {
|
||||
labelValues, isPartial, err = netstorage.GetLabelValues(qt, at, denyPartialResponse, labelName, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf(`cannot obtain label values for %q: %w`, labelName, err)
|
||||
}
|
||||
} else {
|
||||
if cp.start == 0 {
|
||||
cp.start = cp.end - defaultStep
|
||||
}
|
||||
tr := storage.TimeRange{
|
||||
MinTimestamp: cp.start,
|
||||
MaxTimestamp: cp.end,
|
||||
}
|
||||
labelValues, isPartial, err = netstorage.GetLabelValuesOnTimeRange(qt, at, denyPartialResponse, labelName, tr, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf(`cannot obtain label values on time range for %q: %w`, labelName, err)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Extended functionality that allows filtering by label filters and time range
|
||||
// i.e. /api/v1/label/foo/values?match[]=foobar{baz="abc"}&start=...&end=...
|
||||
// is equivalent to `label_values(foobar{baz="abc"}, foo)` call on the selected
|
||||
// time range in Grafana templating.
|
||||
if cp.start == 0 {
|
||||
cp.start = cp.end - defaultStep
|
||||
}
|
||||
labelValues, isPartial, err = labelValuesWithMatches(qt, at, denyPartialResponse, labelName, cp, limit)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain label values for %q on time range [%d...%d]: %w", labelName, cp.start, cp.end, err)
|
||||
}
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, cp.start, cp.end, cp.filterss, *maxUniqueTimeseries)
|
||||
labelValues, isPartial, err := netstorage.GetLabelValues(qt, at, denyPartialResponse, labelName, sq, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain values for label %q: %w", labelName, err)
|
||||
}
|
||||
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
|
@ -559,94 +529,8 @@ func LabelValuesHandler(qt *querytracer.Tracer, startTime time.Time, at *auth.To
|
|||
return nil
|
||||
}
|
||||
|
||||
func labelValuesWithMatches(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, labelName string, cp *commonParams, limit int) ([]string, bool, error) {
|
||||
// Add `labelName!=''` tag filter in order to filter out series without the labelName.
|
||||
// There is no need in adding `__name__!=''` filter, since all the time series should
|
||||
// already have non-empty name.
|
||||
if labelName != "__name__" {
|
||||
key := []byte(labelName)
|
||||
for i, tfs := range cp.filterss {
|
||||
cp.filterss[i] = append(tfs, storage.TagFilter{
|
||||
Key: key,
|
||||
IsNegative: true,
|
||||
})
|
||||
}
|
||||
}
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, cp.start, cp.end, cp.filterss, *maxSeriesLimit)
|
||||
m := make(map[string]struct{})
|
||||
var isPartial bool
|
||||
if cp.end-cp.start > 24*3600*1000 {
|
||||
// It is cheaper to call SearchMetricNames on time ranges exceeding a day.
|
||||
mns, isPartialResponse, err := netstorage.SearchMetricNames(qt, at, denyPartialResponse, sq, cp.deadline)
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch time series for %q: %w", sq, err)
|
||||
}
|
||||
isPartial = isPartialResponse
|
||||
for _, mn := range mns {
|
||||
labelValue := mn.GetTagValue(labelName)
|
||||
if len(labelValue) == 0 {
|
||||
continue
|
||||
}
|
||||
m[string(labelValue)] = struct{}{}
|
||||
}
|
||||
} else {
|
||||
rss, isPartialResponse, err := netstorage.ProcessSearchQuery(qt, at, denyPartialResponse, sq, false, cp.deadline)
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch data for %q: %w", sq, err)
|
||||
}
|
||||
isPartial = isPartialResponse
|
||||
var mLock sync.Mutex
|
||||
err = rss.RunParallel(qt, func(rs *netstorage.Result, workerID uint) error {
|
||||
labelValue := rs.MetricName.GetTagValue(labelName)
|
||||
if len(labelValue) == 0 {
|
||||
return nil
|
||||
}
|
||||
mLock.Lock()
|
||||
m[string(labelValue)] = struct{}{}
|
||||
mLock.Unlock()
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch label values from storage: %w", err)
|
||||
}
|
||||
}
|
||||
labelValues := make([]string, 0, len(m))
|
||||
for labelValue := range m {
|
||||
labelValues = append(labelValues, labelValue)
|
||||
}
|
||||
if limit > 0 && len(labelValues) > limit {
|
||||
labelValues = labelValues[:limit]
|
||||
}
|
||||
sort.Strings(labelValues)
|
||||
qt.Printf("sort %d label values", len(labelValues))
|
||||
return labelValues, isPartial, nil
|
||||
}
|
||||
|
||||
var labelValuesDuration = metrics.NewSummary(`vm_request_duration_seconds{path="/api/v1/label/{}/values"}`)
|
||||
|
||||
// LabelsCountHandler processes /api/v1/labels/count request.
|
||||
func LabelsCountHandler(startTime time.Time, at *auth.Token, w http.ResponseWriter, r *http.Request) error {
|
||||
defer labelsCountDuration.UpdateDuration(startTime)
|
||||
|
||||
deadline := searchutils.GetDeadlineForStatusRequest(r, startTime)
|
||||
denyPartialResponse := searchutils.GetDenyPartialResponse(r)
|
||||
labelEntries, isPartial, err := netstorage.GetLabelEntries(nil, at, denyPartialResponse, deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf(`cannot obtain label entries: %w`, err)
|
||||
}
|
||||
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
bw := bufferedwriter.Get(w)
|
||||
defer bufferedwriter.Put(bw)
|
||||
WriteLabelsCountResponse(bw, isPartial, labelEntries)
|
||||
if err := bw.Flush(); err != nil {
|
||||
return fmt.Errorf("cannot send labels count response to remote client: %w", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var labelsCountDuration = metrics.NewSummary(`vm_request_duration_seconds{path="/api/v1/labels/count"}`)
|
||||
|
||||
const secsPerDay = 3600 * 24
|
||||
|
||||
// TSDBStatusHandler processes /api/v1/status/tsdb request.
|
||||
|
@ -739,38 +623,11 @@ func LabelsHandler(qt *querytracer.Tracer, startTime time.Time, at *auth.Token,
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var labels []string
|
||||
var isPartial bool
|
||||
denyPartialResponse := searchutils.GetDenyPartialResponse(r)
|
||||
if len(cp.filterss) == 0 {
|
||||
if cp.IsDefaultTimeRange() {
|
||||
labels, isPartial, err = netstorage.GetLabels(qt, at, denyPartialResponse, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain labels: %w", err)
|
||||
}
|
||||
} else {
|
||||
if cp.start == 0 {
|
||||
cp.start = cp.end - defaultStep
|
||||
}
|
||||
tr := storage.TimeRange{
|
||||
MinTimestamp: cp.start,
|
||||
MaxTimestamp: cp.end,
|
||||
}
|
||||
labels, isPartial, err = netstorage.GetLabelsOnTimeRange(qt, at, denyPartialResponse, tr, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain labels on time range: %w", err)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Extended functionality that allows filtering by label filters and time range
|
||||
// i.e. /api/v1/labels?match[]=foobar{baz="abc"}&start=...&end=...
|
||||
if cp.start == 0 {
|
||||
cp.start = cp.end - defaultStep
|
||||
}
|
||||
labels, isPartial, err = labelsWithMatches(qt, at, denyPartialResponse, cp, limit)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain labels for timeRange=[%d..%d]: %w", cp.start, cp.end, err)
|
||||
}
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, cp.start, cp.end, cp.filterss, *maxUniqueTimeseries)
|
||||
labels, isPartial, err := netstorage.GetLabelNames(qt, at, denyPartialResponse, sq, limit, cp.deadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot obtain labels: %w", err)
|
||||
}
|
||||
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
|
@ -783,57 +640,6 @@ func LabelsHandler(qt *querytracer.Tracer, startTime time.Time, at *auth.Token,
|
|||
return nil
|
||||
}
|
||||
|
||||
func labelsWithMatches(qt *querytracer.Tracer, at *auth.Token, denyPartialResponse bool, cp *commonParams, limit int) ([]string, bool, error) {
|
||||
sq := storage.NewSearchQuery(at.AccountID, at.ProjectID, cp.start, cp.end, cp.filterss, *maxSeriesLimit)
|
||||
m := make(map[string]struct{})
|
||||
isPartial := false
|
||||
if cp.end-cp.start > 24*3600*1000 {
|
||||
// It is cheaper to call SearchMetricNames on time ranges exceeding a day.
|
||||
mns, isPartialResponse, err := netstorage.SearchMetricNames(qt, at, denyPartialResponse, sq, cp.deadline)
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch time series for %q: %w", sq, err)
|
||||
}
|
||||
isPartial = isPartialResponse
|
||||
for _, mn := range mns {
|
||||
for _, tag := range mn.Tags {
|
||||
m[string(tag.Key)] = struct{}{}
|
||||
}
|
||||
}
|
||||
if len(mns) > 0 {
|
||||
m["__name__"] = struct{}{}
|
||||
}
|
||||
} else {
|
||||
rss, isPartialResponse, err := netstorage.ProcessSearchQuery(qt, at, denyPartialResponse, sq, false, cp.deadline)
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch data for %q: %w", sq, err)
|
||||
}
|
||||
isPartial = isPartialResponse
|
||||
var mLock sync.Mutex
|
||||
err = rss.RunParallel(qt, func(rs *netstorage.Result, workerID uint) error {
|
||||
mLock.Lock()
|
||||
for _, tag := range rs.MetricName.Tags {
|
||||
m[string(tag.Key)] = struct{}{}
|
||||
}
|
||||
m["__name__"] = struct{}{}
|
||||
mLock.Unlock()
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, false, fmt.Errorf("cannot fetch labels from storage: %w", err)
|
||||
}
|
||||
}
|
||||
labels := make([]string, 0, len(m))
|
||||
for label := range m {
|
||||
labels = append(labels, label)
|
||||
}
|
||||
if limit > 0 && limit < len(labels) {
|
||||
labels = labels[:limit]
|
||||
}
|
||||
sort.Strings(labels)
|
||||
qt.Printf("sort %d labels", len(labels))
|
||||
return labels, isPartial, nil
|
||||
}
|
||||
|
||||
var labelsDuration = metrics.NewSummary(`vm_request_duration_seconds{path="/api/v1/labels"}`)
|
||||
|
||||
// SeriesCountHandler processes /api/v1/series/count request.
|
||||
|
|
|
@ -513,7 +513,7 @@ func (s *Server) processVMSelectRequest(ctx *vmselectRequestCtx) error {
|
|||
|
||||
// Process the rpcName call.
|
||||
if err := s.processVMSelectRPC(ctx, rpcName); err != nil {
|
||||
return err
|
||||
return fmt.Errorf("cannot execute %q: %s", rpcName, err)
|
||||
}
|
||||
|
||||
// Finish query trace.
|
||||
|
@ -531,18 +531,12 @@ func (s *Server) processVMSelectRPC(ctx *vmselectRequestCtx, rpcName string) err
|
|||
return s.processVMSelectSearch(ctx)
|
||||
case "searchMetricNames_v3":
|
||||
return s.processVMSelectSearchMetricNames(ctx)
|
||||
case "labelValuesOnTimeRange_v3":
|
||||
return s.processVMSelectLabelValuesOnTimeRange(ctx)
|
||||
case "labelValues_v4":
|
||||
case "labelValues_v5":
|
||||
return s.processVMSelectLabelValues(ctx)
|
||||
case "tagValueSuffixes_v3":
|
||||
return s.processVMSelectTagValueSuffixes(ctx)
|
||||
case "labelEntries_v4":
|
||||
return s.processVMSelectLabelEntries(ctx)
|
||||
case "labelsOnTimeRange_v3":
|
||||
return s.processVMSelectLabelsOnTimeRange(ctx)
|
||||
case "labels_v4":
|
||||
return s.processVMSelectLabels(ctx)
|
||||
case "labelNames_v5":
|
||||
return s.processVMSelectLabelNames(ctx)
|
||||
case "seriesCount_v4":
|
||||
return s.processVMSelectSeriesCount(ctx)
|
||||
case "tsdbStatus_v4":
|
||||
|
@ -554,7 +548,7 @@ func (s *Server) processVMSelectRPC(ctx *vmselectRequestCtx, rpcName string) err
|
|||
case "registerMetricNames_v3":
|
||||
return s.processVMSelectRegisterMetricNames(ctx)
|
||||
default:
|
||||
return fmt.Errorf("unsupported rpcName: %q", ctx.dataBuf)
|
||||
return fmt.Errorf("unsupported rpcName: %q", rpcName)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -641,28 +635,31 @@ func (s *Server) processVMSelectDeleteMetrics(ctx *vmselectRequestCtx) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectLabelsOnTimeRange(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelsOnTimeRangeRequests.Inc()
|
||||
func (s *Server) processVMSelectLabelNames(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelNamesRequests.Inc()
|
||||
|
||||
// Read request
|
||||
accountID, projectID, err := ctx.readAccountIDProjectID()
|
||||
if err != nil {
|
||||
if err := ctx.readSearchQuery(); err != nil {
|
||||
return err
|
||||
}
|
||||
tr, err := ctx.readTimeRange()
|
||||
maxLabelNames, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
return err
|
||||
return fmt.Errorf("cannot read maxLabelNames: %w", err)
|
||||
}
|
||||
limit, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if limit <= 0 || limit > *maxTagKeysPerSearch {
|
||||
limit = *maxTagKeysPerSearch
|
||||
if maxLabelNames <= 0 || maxLabelNames > *maxTagKeysPerSearch {
|
||||
maxLabelNames = *maxTagKeysPerSearch
|
||||
}
|
||||
|
||||
// Search for tag keys
|
||||
labels, err := s.storage.SearchTagKeysOnTimeRange(accountID, projectID, tr, limit, ctx.deadline)
|
||||
// Execute the request
|
||||
tr := storage.TimeRange{
|
||||
MinTimestamp: ctx.sq.MinTimestamp,
|
||||
MaxTimestamp: ctx.sq.MaxTimestamp,
|
||||
}
|
||||
if err := ctx.setupTfss(s.storage, tr); err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
maxMetrics := ctx.getMaxMetrics()
|
||||
labelNames, err := s.storage.SearchLabelNamesWithFiltersOnTimeRange(ctx.qt, ctx.sq.AccountID, ctx.sq.ProjectID, ctx.tfss, tr, maxLabelNames, maxMetrics, ctx.deadline)
|
||||
if err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
|
@ -672,62 +669,12 @@ func (s *Server) processVMSelectLabelsOnTimeRange(ctx *vmselectRequestCtx) error
|
|||
return fmt.Errorf("cannot send empty error message: %w", err)
|
||||
}
|
||||
|
||||
// Send labels to vmselect
|
||||
for _, label := range labels {
|
||||
if len(label) == 0 {
|
||||
// Do this substitution in order to prevent clashing with 'end of response' marker.
|
||||
label = "__name__"
|
||||
}
|
||||
if err := ctx.writeString(label); err != nil {
|
||||
return fmt.Errorf("cannot write label %q: %w", label, err)
|
||||
// Send labelNames to vmselect
|
||||
for _, labelName := range labelNames {
|
||||
if err := ctx.writeString(labelName); err != nil {
|
||||
return fmt.Errorf("cannot write label name %q: %w", labelName, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Send 'end of response' marker
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send 'end of response' marker")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectLabels(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelsRequests.Inc()
|
||||
|
||||
// Read request
|
||||
accountID, projectID, err := ctx.readAccountIDProjectID()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
limit, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if limit <= 0 || limit > *maxTagKeysPerSearch {
|
||||
limit = *maxTagKeysPerSearch
|
||||
}
|
||||
|
||||
// Search for tag keys
|
||||
labels, err := s.storage.SearchTagKeys(accountID, projectID, limit, ctx.deadline)
|
||||
if err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
|
||||
// Send an empty error message to vmselect.
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send empty error message: %w", err)
|
||||
}
|
||||
|
||||
// Send labels to vmselect
|
||||
for _, label := range labels {
|
||||
if len(label) == 0 {
|
||||
// Do this substitution in order to prevent clashing with 'end of response' marker.
|
||||
label = "__name__"
|
||||
}
|
||||
if err := ctx.writeString(label); err != nil {
|
||||
return fmt.Errorf("cannot write label %q: %w", label, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Send 'end of response' marker
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send 'end of response' marker")
|
||||
|
@ -737,66 +684,36 @@ func (s *Server) processVMSelectLabels(ctx *vmselectRequestCtx) error {
|
|||
|
||||
const maxLabelValueSize = 16 * 1024
|
||||
|
||||
func (s *Server) processVMSelectLabelValuesOnTimeRange(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelValuesOnTimeRangeRequests.Inc()
|
||||
|
||||
// Read request
|
||||
accountID, projectID, err := ctx.readAccountIDProjectID()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := ctx.readDataBufBytes(maxLabelValueSize); err != nil {
|
||||
return fmt.Errorf("cannot read labelName: %w", err)
|
||||
}
|
||||
labelName := string(ctx.dataBuf)
|
||||
tr, err := ctx.readTimeRange()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
limit, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if limit <= 0 || limit > *maxTagValuesPerSearch {
|
||||
limit = *maxTagValuesPerSearch
|
||||
}
|
||||
|
||||
// Search for tag values
|
||||
labelValues, err := s.storage.SearchTagValuesOnTimeRange(accountID, projectID, []byte(labelName), tr, limit, ctx.deadline)
|
||||
if err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
|
||||
// Send an empty error message to vmselect.
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send empty error message: %w", err)
|
||||
}
|
||||
|
||||
return writeLabelValues(ctx, labelValues)
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectLabelValues(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelValuesRequests.Inc()
|
||||
|
||||
// Read request
|
||||
accountID, projectID, err := ctx.readAccountIDProjectID()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := ctx.readDataBufBytes(maxLabelValueSize); err != nil {
|
||||
return fmt.Errorf("cannot read labelName: %w", err)
|
||||
}
|
||||
labelName := ctx.dataBuf
|
||||
limit, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
labelName := string(ctx.dataBuf)
|
||||
if err := ctx.readSearchQuery(); err != nil {
|
||||
return err
|
||||
}
|
||||
if limit <= 0 || limit > *maxTagValuesPerSearch {
|
||||
limit = *maxTagValuesPerSearch
|
||||
maxLabelValues, err := ctx.readLimit()
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot read maxLabelValues: %w", err)
|
||||
}
|
||||
if maxLabelValues <= 0 || maxLabelValues > *maxTagValuesPerSearch {
|
||||
maxLabelValues = *maxTagValuesPerSearch
|
||||
}
|
||||
|
||||
// Search for tag values
|
||||
labelValues, err := s.storage.SearchTagValues(accountID, projectID, labelName, limit, ctx.deadline)
|
||||
// Execute the request
|
||||
tr := storage.TimeRange{
|
||||
MinTimestamp: ctx.sq.MinTimestamp,
|
||||
MaxTimestamp: ctx.sq.MaxTimestamp,
|
||||
}
|
||||
if err := ctx.setupTfss(s.storage, tr); err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
maxMetrics := ctx.getMaxMetrics()
|
||||
labelValues, err := s.storage.SearchLabelValuesWithFiltersOnTimeRange(ctx.qt, ctx.sq.AccountID, ctx.sq.ProjectID, labelName, ctx.tfss, tr,
|
||||
maxLabelValues, maxMetrics, ctx.deadline)
|
||||
if err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
|
@ -806,7 +723,21 @@ func (s *Server) processVMSelectLabelValues(ctx *vmselectRequestCtx) error {
|
|||
return fmt.Errorf("cannot send empty error message: %w", err)
|
||||
}
|
||||
|
||||
return writeLabelValues(ctx, labelValues)
|
||||
// Send labelValues to vmselect
|
||||
for _, labelValue := range labelValues {
|
||||
if len(labelValue) == 0 {
|
||||
// Skip empty label values, since they have no sense for prometheus.
|
||||
continue
|
||||
}
|
||||
if err := ctx.writeString(labelValue); err != nil {
|
||||
return fmt.Errorf("cannot write labelValue %q: %w", labelValue, err)
|
||||
}
|
||||
}
|
||||
// Send 'end of label values' marker
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send 'end of response' marker")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectTagValueSuffixes(ctx *vmselectRequestCtx) error {
|
||||
|
@ -865,66 +796,6 @@ func (s *Server) processVMSelectTagValueSuffixes(ctx *vmselectRequestCtx) error
|
|||
return nil
|
||||
}
|
||||
|
||||
func writeLabelValues(ctx *vmselectRequestCtx, labelValues []string) error {
|
||||
for _, labelValue := range labelValues {
|
||||
if len(labelValue) == 0 {
|
||||
// Skip empty label values, since they have no sense for prometheus.
|
||||
continue
|
||||
}
|
||||
if err := ctx.writeString(labelValue); err != nil {
|
||||
return fmt.Errorf("cannot write labelValue %q: %w", labelValue, err)
|
||||
}
|
||||
}
|
||||
// Send 'end of label values' marker
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send 'end of response' marker")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectLabelEntries(ctx *vmselectRequestCtx) error {
|
||||
vmselectLabelEntriesRequests.Inc()
|
||||
|
||||
// Read request
|
||||
accountID, projectID, err := ctx.readAccountIDProjectID()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Perform the request
|
||||
labelEntries, err := s.storage.SearchTagEntries(accountID, projectID, *maxTagKeysPerSearch, *maxTagValuesPerSearch, ctx.deadline)
|
||||
if err != nil {
|
||||
return ctx.writeErrorMessage(err)
|
||||
}
|
||||
|
||||
// Send an empty error message to vmselect.
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send empty error message: %w", err)
|
||||
}
|
||||
|
||||
// Send labelEntries to vmselect
|
||||
for i := range labelEntries {
|
||||
e := &labelEntries[i]
|
||||
label := e.Key
|
||||
if label == "" {
|
||||
// Do this substitution in order to prevent clashing with 'end of response' marker.
|
||||
label = "__name__"
|
||||
}
|
||||
if err := ctx.writeString(label); err != nil {
|
||||
return fmt.Errorf("cannot write label %q: %w", label, err)
|
||||
}
|
||||
if err := writeLabelValues(ctx, e.Values); err != nil {
|
||||
return fmt.Errorf("cannot write label values for %q: %w", label, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Send 'end of response' marker
|
||||
if err := ctx.writeString(""); err != nil {
|
||||
return fmt.Errorf("cannot send 'end of response' marker")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) processVMSelectSeriesCount(ctx *vmselectRequestCtx) error {
|
||||
vmselectSeriesCountRequests.Inc()
|
||||
|
||||
|
@ -1191,19 +1062,16 @@ func checkTimeRange(s *storage.Storage, tr storage.TimeRange) error {
|
|||
}
|
||||
|
||||
var (
|
||||
vmselectRegisterMetricNamesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="register_metric_names"}`)
|
||||
vmselectDeleteMetricsRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="delete_metrics"}`)
|
||||
vmselectLabelsOnTimeRangeRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="labels_on_time_range"}`)
|
||||
vmselectLabelsRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="labels"}`)
|
||||
vmselectLabelValuesOnTimeRangeRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="label_values_on_time_range"}`)
|
||||
vmselectLabelValuesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="label_values"}`)
|
||||
vmselectTagValueSuffixesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tag_value_suffixes"}`)
|
||||
vmselectLabelEntriesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="label_entries"}`)
|
||||
vmselectSeriesCountRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="series_count"}`)
|
||||
vmselectTSDBStatusRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tsdb_status"}`)
|
||||
vmselectTSDBStatusWithFiltersRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tsdb_status_with_filters"}`)
|
||||
vmselectSearchMetricNamesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="search_metric_names"}`)
|
||||
vmselectSearchRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="search"}`)
|
||||
vmselectRegisterMetricNamesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="register_metric_names"}`)
|
||||
vmselectDeleteMetricsRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="delete_metrics"}`)
|
||||
vmselectLabelNamesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="label_names"}`)
|
||||
vmselectLabelValuesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="label_values"}`)
|
||||
vmselectTagValueSuffixesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tag_value_suffixes"}`)
|
||||
vmselectSeriesCountRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="series_count"}`)
|
||||
vmselectTSDBStatusRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tsdb_status"}`)
|
||||
vmselectTSDBStatusWithFiltersRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="tsdb_status_with_filters"}`)
|
||||
vmselectSearchMetricNamesRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="search_metric_names"}`)
|
||||
vmselectSearchRequests = metrics.NewCounter(`vm_vmselect_rpc_requests_total{name="search"}`)
|
||||
|
||||
vmselectMetricBlocksRead = metrics.NewCounter(`vm_vmselect_metric_blocks_read_total`)
|
||||
vmselectMetricRowsRead = metrics.NewCounter(`vm_vmselect_metric_rows_read_total`)
|
||||
|
|
|
@ -23,7 +23,8 @@ The following tip changes can be tested by building VictoriaMetrics components f
|
|||
* FEATURE: add support of `lowercase` and `uppercase` relabeling actions in the same way as [Prometheus 2.36.0 does](https://github.com/prometheus/prometheus/releases/tag/v2.36.0). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2664).
|
||||
* FEATURE: add ability to change the `indexdb` rotation timezone offset via `-retentionTimezoneOffset` command-line flag. Previously it was performed at 4am UTC time. This could lead to performance degradation in the middle of the day when VictoriaMetrics runs in time zones located too far from UTC. Thanks to @cnych for [the pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/2574).
|
||||
* FEATURE: limit the number of background merge threads on systems with big number of CPU cores by default. This increases the max size of parts, which can be created during background merge when `-storageDataPath` directory has limited free disk space. This may improve on-disk data compression efficiency and query performance. The limits can be tuned if needed with `-smallMergeConcurrency` and `-bigMergeConcurrency` command-line flags. See [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/2673).
|
||||
* FEATURE: accept optional `limit` query arg at [/api/v1/labels](https://prometheus.io/docs/prometheus/latest/querying/api/#getting-label-names) and [/api/v1/label_values](https://prometheus.io/docs/prometheus/latest/querying/api/#querying-label-values) for limiting the numbef of sample entries returned from these endpoints. See [these docs](https://docs.victoriametrics.com/#prometheus-querying-api-enhancements).
|
||||
* FEATURE: accept optional `limit` query arg at [/api/v1/labels](https://prometheus.io/docs/prometheus/latest/querying/api/#getting-label-names) and [/api/v1/label/.../values](https://prometheus.io/docs/prometheus/latest/querying/api/#querying-label-values) for limiting the numbef of sample entries returned from these endpoints. See [these docs](https://docs.victoriametrics.com/#prometheus-querying-api-enhancements).
|
||||
* FEATURE: optimize performance for [/api/v1/labels](https://prometheus.io/docs/prometheus/latest/querying/api/#getting-label-names) and [/api/v1/label/.../values](https://prometheus.io/docs/prometheus/latest/querying/api/#querying-label-values) endpoints when `match[]`, `extra_label` or `extra_filters[]` query args are passed to these endpoints.
|
||||
* FEATURE: [vmalert](https://docs.victoriametrics.com/vmalert.html): support `limit` param per-group for limiting number of produced samples per each rule. Thanks to @Howie59 for [implementation](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/2676).
|
||||
* FEATURE: [vmalert](https://docs.victoriametrics.com/vmalert.html): remove dependency on Internet access at [web API pages](https://docs.victoriametrics.com/vmalert.html#web). Previously the functionality and the layout of these pages was broken without Internet access. See [shis issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2594).
|
||||
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): implement the `http://vmagent:8429/service-discovery` page in the same way as Prometheus does. This page shows the original labels for all the discovered targets alongside the resulting labels after the relabeling. This simplifies service discovery debugging.
|
||||
|
|
|
@ -275,6 +275,8 @@ By default cardinality explorer analyzes time series for the current date. It pr
|
|||
By default all the time series for the selected date are analyzed. It is possible to narrow down the analysis to series
|
||||
matching the specified [series selector](https://prometheus.io/docs/prometheus/latest/querying/basics/#time-series-selectors).
|
||||
|
||||
Cardinality explorer takes into account [deleted time series](#how-to-delete-time-series), because they stay in the inverted index for up to [-retentionPeriod](#retention). This means that the deleted time series take RAM, CPU, disk IO and disk space for the inverted index in the same way as other time series.
|
||||
|
||||
Cardinality explorer is built on top of [/api/v1/status/tsdb](#tsdb-stats).
|
||||
|
||||
See [cardinality explorer playground](https://play.victoriametrics.com/select/accounting/1/6a716b0f-38bc-4856-90ce-448fd713e3fe/prometheus/graph/#/cardinality).
|
||||
|
@ -617,7 +619,6 @@ Additionally, VictoriaMetrics provides the following handlers:
|
|||
* `/api/v1/series/count` - returns the total number of time series in the database. Some notes:
|
||||
* the handler scans all the inverted index, so it can be slow if the database contains tens of millions of time series;
|
||||
* the handler may count [deleted time series](#how-to-delete-time-series) additionally to normal time series due to internal implementation restrictions;
|
||||
* `/api/v1/labels/count` - returns a list of `label: values_count` entries. It can be used for determining labels with the maximum number of values.
|
||||
* `/api/v1/status/active_queries` - returns a list of currently running queries.
|
||||
* `/api/v1/status/top_queries` - returns the following query lists:
|
||||
* the most frequently executed queries - `topByCount`
|
||||
|
@ -1245,11 +1246,12 @@ and [clustered VictoriaMetrics](https://grafana.com/grafana/dashboards/11176) Gr
|
|||
See more details in [monitoring docs](#monitoring).
|
||||
|
||||
The `merge` process is usually named "compaction", because the resulting `part` size is usually smaller than
|
||||
the sum of the source `parts`. There are following benefits of doing the merge process:
|
||||
the sum of the source `parts` because of better compression rate. The merge process provides the following additional benefits:
|
||||
|
||||
* it improves query performance, since lower number of `parts` are inspected with each query;
|
||||
* it reduces the number of data files, since each `part`contains fixed number of files;
|
||||
* better compression rate for the resulting part.
|
||||
* it improves query performance, since lower number of `parts` are inspected with each query
|
||||
* it reduces the number of data files, since each `part` contains fixed number of files
|
||||
* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling)
|
||||
and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are perfomed during the merge.
|
||||
|
||||
Newly added `parts` either appear in the storage or fail to appear.
|
||||
Storage never contains partially created parts. The same applies to merge process — `parts` are either fully
|
||||
|
@ -1411,7 +1413,7 @@ See the example of alerting rules for VM components [here](https://github.com/Vi
|
|||
VictoriaMetrics returns TSDB stats at `/api/v1/status/tsdb` page in the way similar to Prometheus - see [these Prometheus docs](https://prometheus.io/docs/prometheus/latest/querying/api/#tsdb-stats). VictoriaMetrics accepts the following optional query args at `/api/v1/status/tsdb` page:
|
||||
|
||||
* `topN=N` where `N` is the number of top entries to return in the response. By default top 10 entries are returned.
|
||||
* `date=YYYY-MM-DD` where `YYYY-MM-DD` is the date for collecting the stats. By default the stats is collected for the current day.
|
||||
* `date=YYYY-MM-DD` where `YYYY-MM-DD` is the date for collecting the stats. By default the stats is collected for the current day. Pass `date=1970-01-01` in order to collect global stats across all the days.
|
||||
* `match[]=SELECTOR` where `SELECTOR` is an arbitrary [time series selector](https://prometheus.io/docs/prometheus/latest/querying/basics/#time-series-selectors) for series to take into account during stats calculation. By default all the series are taken into account.
|
||||
* `extra_label=LABEL=VALUE`. See [these docs](#prometheus-querying-api-enhancements) for more details.
|
||||
|
||||
|
|
|
@ -279,6 +279,8 @@ By default cardinality explorer analyzes time series for the current date. It pr
|
|||
By default all the time series for the selected date are analyzed. It is possible to narrow down the analysis to series
|
||||
matching the specified [series selector](https://prometheus.io/docs/prometheus/latest/querying/basics/#time-series-selectors).
|
||||
|
||||
Cardinality explorer takes into account [deleted time series](#how-to-delete-time-series), because they stay in the inverted index for up to [-retentionPeriod](#retention). This means that the deleted time series take RAM, CPU, disk IO and disk space for the inverted index in the same way as other time series.
|
||||
|
||||
Cardinality explorer is built on top of [/api/v1/status/tsdb](#tsdb-stats).
|
||||
|
||||
See [cardinality explorer playground](https://play.victoriametrics.com/select/accounting/1/6a716b0f-38bc-4856-90ce-448fd713e3fe/prometheus/graph/#/cardinality).
|
||||
|
@ -621,7 +623,6 @@ Additionally, VictoriaMetrics provides the following handlers:
|
|||
* `/api/v1/series/count` - returns the total number of time series in the database. Some notes:
|
||||
* the handler scans all the inverted index, so it can be slow if the database contains tens of millions of time series;
|
||||
* the handler may count [deleted time series](#how-to-delete-time-series) additionally to normal time series due to internal implementation restrictions;
|
||||
* `/api/v1/labels/count` - returns a list of `label: values_count` entries. It can be used for determining labels with the maximum number of values.
|
||||
* `/api/v1/status/active_queries` - returns a list of currently running queries.
|
||||
* `/api/v1/status/top_queries` - returns the following query lists:
|
||||
* the most frequently executed queries - `topByCount`
|
||||
|
@ -1249,11 +1250,12 @@ and [clustered VictoriaMetrics](https://grafana.com/grafana/dashboards/11176) Gr
|
|||
See more details in [monitoring docs](#monitoring).
|
||||
|
||||
The `merge` process is usually named "compaction", because the resulting `part` size is usually smaller than
|
||||
the sum of the source `parts`. There are following benefits of doing the merge process:
|
||||
the sum of the source `parts` because of better compression rate. The merge process provides the following additional benefits:
|
||||
|
||||
* it improves query performance, since lower number of `parts` are inspected with each query;
|
||||
* it reduces the number of data files, since each `part`contains fixed number of files;
|
||||
* better compression rate for the resulting part.
|
||||
* it improves query performance, since lower number of `parts` are inspected with each query
|
||||
* it reduces the number of data files, since each `part` contains fixed number of files
|
||||
* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling)
|
||||
and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are perfomed during the merge.
|
||||
|
||||
Newly added `parts` either appear in the storage or fail to appear.
|
||||
Storage never contains partially created parts. The same applies to merge process — `parts` are either fully
|
||||
|
@ -1415,7 +1417,7 @@ See the example of alerting rules for VM components [here](https://github.com/Vi
|
|||
VictoriaMetrics returns TSDB stats at `/api/v1/status/tsdb` page in the way similar to Prometheus - see [these Prometheus docs](https://prometheus.io/docs/prometheus/latest/querying/api/#tsdb-stats). VictoriaMetrics accepts the following optional query args at `/api/v1/status/tsdb` page:
|
||||
|
||||
* `topN=N` where `N` is the number of top entries to return in the response. By default top 10 entries are returned.
|
||||
* `date=YYYY-MM-DD` where `YYYY-MM-DD` is the date for collecting the stats. By default the stats is collected for the current day.
|
||||
* `date=YYYY-MM-DD` where `YYYY-MM-DD` is the date for collecting the stats. By default the stats is collected for the current day. Pass `date=1970-01-01` in order to collect global stats across all the days.
|
||||
* `match[]=SELECTOR` where `SELECTOR` is an arbitrary [time series selector](https://prometheus.io/docs/prometheus/latest/querying/basics/#time-series-selectors) for series to take into account during stats calculation. By default all the series are taken into account.
|
||||
* `extra_label=LABEL=VALUE`. See [these docs](#prometheus-querying-api-enhancements) for more details.
|
||||
|
||||
|
|
|
@ -433,7 +433,7 @@ func marshalTagFiltersKey(dst []byte, tfss []*TagFilters, tr TimeRange, versione
|
|||
}
|
||||
// Round start and end times to per-day granularity according to per-day inverted index.
|
||||
startDate := uint64(tr.MinTimestamp) / msecPerDay
|
||||
endDate := uint64(tr.MaxTimestamp) / msecPerDay
|
||||
endDate := uint64(tr.MaxTimestamp-1) / msecPerDay
|
||||
dst = encoding.MarshalUint64(dst, prefix)
|
||||
dst = encoding.MarshalUint64(dst, startDate)
|
||||
dst = encoding.MarshalUint64(dst, endDate)
|
||||
|
@ -737,50 +737,66 @@ func putIndexItems(ii *indexItems) {
|
|||
|
||||
var indexItemsPool sync.Pool
|
||||
|
||||
// SearchTagKeysOnTimeRange returns all the tag keys on the given tr.
|
||||
func (db *indexDB) SearchTagKeysOnTimeRange(accountID, projectID uint32, tr TimeRange, maxTagKeys int, deadline uint64) ([]string, error) {
|
||||
tks := make(map[string]struct{})
|
||||
// SearchLabelNamesWithFiltersOnTimeRange returns all the label names, which match the given tfss on the given tr.
|
||||
func (db *indexDB) SearchLabelNamesWithFiltersOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, tfss []*TagFilters, tr TimeRange,
|
||||
maxLabelNames, maxMetrics int, deadline uint64) ([]string, error) {
|
||||
qt = qt.NewChild("search for label names: filters=%s, timeRange=%s, maxLabelNames=%d, maxMetrics=%d", tfss, &tr, maxLabelNames, maxMetrics)
|
||||
defer qt.Done()
|
||||
lns := make(map[string]struct{})
|
||||
qtChild := qt.NewChild("search for label names in the current indexdb")
|
||||
is := db.getIndexSearch(accountID, projectID, deadline)
|
||||
err := is.searchTagKeysOnTimeRange(tks, tr, maxTagKeys)
|
||||
err := is.searchLabelNamesWithFiltersOnTimeRange(qtChild, lns, tfss, tr, maxLabelNames, maxMetrics)
|
||||
db.putIndexSearch(is)
|
||||
qtChild.Donef("found %d label names", len(lns))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ok := db.doExtDB(func(extDB *indexDB) {
|
||||
qtChild := qt.NewChild("search for label names in the previous indexdb")
|
||||
lnsLen := len(lns)
|
||||
is := extDB.getIndexSearch(accountID, projectID, deadline)
|
||||
err = is.searchTagKeysOnTimeRange(tks, tr, maxTagKeys)
|
||||
err = is.searchLabelNamesWithFiltersOnTimeRange(qtChild, lns, tfss, tr, maxLabelNames, maxMetrics)
|
||||
extDB.putIndexSearch(is)
|
||||
qtChild.Donef("found %d additional label names", len(lns)-lnsLen)
|
||||
})
|
||||
if ok && err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
keys := make([]string, 0, len(tks))
|
||||
for key := range tks {
|
||||
// Do not skip empty keys, since they are converted to __name__
|
||||
keys = append(keys, key)
|
||||
labelNames := make([]string, 0, len(lns))
|
||||
for labelName := range lns {
|
||||
labelNames = append(labelNames, labelName)
|
||||
}
|
||||
// Do not sort keys, since they must be sorted by vmselect.
|
||||
return keys, nil
|
||||
// Do not sort label names, since they must be sorted by vmselect.
|
||||
qt.Printf("found %d label names in the current and the previous indexdb", len(labelNames))
|
||||
return labelNames, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagKeysOnTimeRange(tks map[string]struct{}, tr TimeRange, maxTagKeys int) error {
|
||||
func (is *indexSearch) searchLabelNamesWithFiltersOnTimeRange(qt *querytracer.Tracer, lns map[string]struct{}, tfss []*TagFilters, tr TimeRange, maxLabelNames, maxMetrics int) error {
|
||||
minDate := uint64(tr.MinTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp) / msecPerDay
|
||||
if minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
return is.searchTagKeys(tks, maxTagKeys)
|
||||
maxDate := uint64(tr.MaxTimestamp-1) / msecPerDay
|
||||
if maxDate == 0 || minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
qtChild := qt.NewChild("search for label names in global index: filters=%s", tfss)
|
||||
err := is.searchLabelNamesWithFiltersOnDate(qtChild, lns, tfss, 0, maxLabelNames, maxMetrics)
|
||||
qtChild.Done()
|
||||
return err
|
||||
}
|
||||
var mu sync.Mutex
|
||||
wg := getWaitGroup()
|
||||
var errGlobal error
|
||||
qt = qt.NewChild("parallel search for label names: filters=%s, timeRange=%s", tfss, &tr)
|
||||
for date := minDate; date <= maxDate; date++ {
|
||||
wg.Add(1)
|
||||
qtChild := qt.NewChild("search for label names: filters=%s, date=%d", tfss, date)
|
||||
go func(date uint64) {
|
||||
defer wg.Done()
|
||||
tksLocal := make(map[string]struct{})
|
||||
defer func() {
|
||||
qtChild.Done()
|
||||
wg.Done()
|
||||
}()
|
||||
lnsLocal := make(map[string]struct{})
|
||||
isLocal := is.db.getIndexSearch(is.accountID, is.projectID, is.deadline)
|
||||
err := isLocal.searchTagKeysOnDate(tksLocal, date, maxTagKeys)
|
||||
err := isLocal.searchLabelNamesWithFiltersOnDate(qtChild, lnsLocal, tfss, date, maxLabelNames, maxMetrics)
|
||||
is.db.putIndexSearch(isLocal)
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
|
@ -791,31 +807,43 @@ func (is *indexSearch) searchTagKeysOnTimeRange(tks map[string]struct{}, tr Time
|
|||
errGlobal = err
|
||||
return
|
||||
}
|
||||
if len(tks) >= maxTagKeys {
|
||||
if len(lns) >= maxLabelNames {
|
||||
return
|
||||
}
|
||||
for k := range tksLocal {
|
||||
tks[k] = struct{}{}
|
||||
for k := range lnsLocal {
|
||||
lns[k] = struct{}{}
|
||||
}
|
||||
}(date)
|
||||
}
|
||||
wg.Wait()
|
||||
putWaitGroup(wg)
|
||||
qt.Done()
|
||||
return errGlobal
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagKeysOnDate(tks map[string]struct{}, date uint64, maxTagKeys int) error {
|
||||
func (is *indexSearch) searchLabelNamesWithFiltersOnDate(qt *querytracer.Tracer, lns map[string]struct{}, tfss []*TagFilters, date uint64, maxLabelNames, maxMetrics int) error {
|
||||
filter, err := is.searchMetricIDsWithFiltersOnDate(qt, tfss, date, maxMetrics)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if filter != nil && filter.Len() == 0 {
|
||||
qt.Printf("found zero label names for filter=%s", tfss)
|
||||
return nil
|
||||
}
|
||||
var prevLabelName []byte
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
nsPrefixExpected := byte(nsPrefixDateTagToMetricIDs)
|
||||
if date == 0 {
|
||||
nsPrefixExpected = nsPrefixTagToMetricIDs
|
||||
}
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
prefix := kb.B
|
||||
ts.Seek(prefix)
|
||||
for len(tks) < maxTagKeys && ts.NextItem() {
|
||||
for len(lns) < maxLabelNames && ts.NextItem() {
|
||||
if loopsPaceLimiter&paceLimiterFastIterationsMask == 0 {
|
||||
if err := checkSearchDeadlineAndPace(is.deadline); err != nil {
|
||||
return err
|
||||
|
@ -826,110 +854,36 @@ func (is *indexSearch) searchTagKeysOnDate(tks map[string]struct{}, date uint64,
|
|||
if !bytes.HasPrefix(item, prefix) {
|
||||
break
|
||||
}
|
||||
if err := mp.Init(item, nsPrefixDateTagToMetricIDs); err != nil {
|
||||
if err := mp.Init(item, nsPrefixExpected); err != nil {
|
||||
return err
|
||||
}
|
||||
if mp.IsDeletedTag(dmis) {
|
||||
continue
|
||||
}
|
||||
key := mp.Tag.Key
|
||||
if !isArtificialTagKey(key) {
|
||||
tks[string(key)] = struct{}{}
|
||||
if mp.GetMatchingSeriesCount(filter) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
// Search for the next tag key.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag key.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
if len(key) > 0 && key[0] == compositeTagKeyPrefix {
|
||||
// skip composite tag entries
|
||||
kb.B = append(kb.B, compositeTagKeyPrefix)
|
||||
} else {
|
||||
kb.B = marshalTagValue(kb.B, key)
|
||||
labelName := mp.Tag.Key
|
||||
if len(labelName) == 0 {
|
||||
labelName = []byte("__name__")
|
||||
}
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
}
|
||||
if err := ts.Error(); err != nil {
|
||||
return fmt.Errorf("error during search for prefix %q: %w", prefix, err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SearchTagKeys returns all the tag keys for the given accountID, projectID.
|
||||
func (db *indexDB) SearchTagKeys(accountID, projectID uint32, maxTagKeys int, deadline uint64) ([]string, error) {
|
||||
tks := make(map[string]struct{})
|
||||
|
||||
is := db.getIndexSearch(accountID, projectID, deadline)
|
||||
err := is.searchTagKeys(tks, maxTagKeys)
|
||||
db.putIndexSearch(is)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ok := db.doExtDB(func(extDB *indexDB) {
|
||||
is := extDB.getIndexSearch(accountID, projectID, deadline)
|
||||
err = is.searchTagKeys(tks, maxTagKeys)
|
||||
extDB.putIndexSearch(is)
|
||||
})
|
||||
if ok && err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
keys := make([]string, 0, len(tks))
|
||||
for key := range tks {
|
||||
// Do not skip empty keys, since they are converted to __name__
|
||||
keys = append(keys, key)
|
||||
}
|
||||
// Do not sort keys, since they must be sorted by vmselect.
|
||||
return keys, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagKeys(tks map[string]struct{}, maxTagKeys int) error {
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
prefix := kb.B
|
||||
ts.Seek(prefix)
|
||||
for len(tks) < maxTagKeys && ts.NextItem() {
|
||||
if loopsPaceLimiter&paceLimiterFastIterationsMask == 0 {
|
||||
if err := checkSearchDeadlineAndPace(is.deadline); err != nil {
|
||||
return err
|
||||
if isArtificialTagKey(labelName) || string(labelName) == string(prevLabelName) {
|
||||
// Search for the next tag key.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag key.
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
if len(labelName) > 0 && labelName[0] == compositeTagKeyPrefix {
|
||||
// skip composite tag entries
|
||||
kb.B = append(kb.B, compositeTagKeyPrefix)
|
||||
} else {
|
||||
kb.B = marshalTagValue(kb.B, labelName)
|
||||
}
|
||||
}
|
||||
loopsPaceLimiter++
|
||||
item := ts.Item
|
||||
if !bytes.HasPrefix(item, prefix) {
|
||||
break
|
||||
}
|
||||
if err := mp.Init(item, nsPrefixTagToMetricIDs); err != nil {
|
||||
return err
|
||||
}
|
||||
if mp.IsDeletedTag(dmis) {
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
continue
|
||||
}
|
||||
key := mp.Tag.Key
|
||||
if !isArtificialTagKey(key) {
|
||||
tks[string(key)] = struct{}{}
|
||||
}
|
||||
|
||||
// Search for the next tag key.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag key.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
if len(key) > 0 && key[0] == compositeTagKeyPrefix {
|
||||
// skip composite tag entries
|
||||
kb.B = append(kb.B, compositeTagKeyPrefix)
|
||||
} else {
|
||||
kb.B = marshalTagValue(kb.B, key)
|
||||
}
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
lns[string(labelName)] = struct{}{}
|
||||
prevLabelName = append(prevLabelName[:0], labelName...)
|
||||
}
|
||||
if err := ts.Error(); err != nil {
|
||||
return fmt.Errorf("error during search for prefix %q: %w", prefix, err)
|
||||
|
@ -937,53 +891,71 @@ func (is *indexSearch) searchTagKeys(tks map[string]struct{}, maxTagKeys int) er
|
|||
return nil
|
||||
}
|
||||
|
||||
// SearchTagValuesOnTimeRange returns all the tag values for the given tagKey on tr.
|
||||
func (db *indexDB) SearchTagValuesOnTimeRange(accountID, projectID uint32, tagKey []byte, tr TimeRange, maxTagValues int, deadline uint64) ([]string, error) {
|
||||
tvs := make(map[string]struct{})
|
||||
// SearchLabelValuesWithFiltersOnTimeRange returns label values for the given labelName, tfss and tr.
|
||||
func (db *indexDB) SearchLabelValuesWithFiltersOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, labelName string, tfss []*TagFilters, tr TimeRange,
|
||||
maxLabelValues, maxMetrics int, deadline uint64) ([]string, error) {
|
||||
qt = qt.NewChild("search for label values: labelName=%q, filters=%s, timeRange=%s, maxLabelNames=%d, maxMetrics=%d", labelName, tfss, &tr, maxLabelValues, maxMetrics)
|
||||
defer qt.Done()
|
||||
lvs := make(map[string]struct{})
|
||||
qtChild := qt.NewChild("search for label values in the current indexdb")
|
||||
is := db.getIndexSearch(accountID, projectID, deadline)
|
||||
err := is.searchTagValuesOnTimeRange(tvs, tagKey, tr, maxTagValues)
|
||||
err := is.searchLabelValuesWithFiltersOnTimeRange(qtChild, lvs, labelName, tfss, tr, maxLabelValues, maxMetrics)
|
||||
db.putIndexSearch(is)
|
||||
qtChild.Donef("found %d label values", len(lvs))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
ok := db.doExtDB(func(extDB *indexDB) {
|
||||
qtChild := qt.NewChild("search for label values in the previous indexdb")
|
||||
lvsLen := len(lvs)
|
||||
is := extDB.getIndexSearch(accountID, projectID, deadline)
|
||||
err = is.searchTagValuesOnTimeRange(tvs, tagKey, tr, maxTagValues)
|
||||
err = is.searchLabelValuesWithFiltersOnTimeRange(qtChild, lvs, labelName, tfss, tr, maxLabelValues, maxMetrics)
|
||||
extDB.putIndexSearch(is)
|
||||
qtChild.Donef("found %d additional label values", len(lvs)-lvsLen)
|
||||
})
|
||||
if ok && err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tagValues := make([]string, 0, len(tvs))
|
||||
for tv := range tvs {
|
||||
if len(tv) == 0 {
|
||||
labelValues := make([]string, 0, len(lvs))
|
||||
for labelValue := range lvs {
|
||||
if len(labelValue) == 0 {
|
||||
// Skip empty values, since they have no any meaning.
|
||||
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/600
|
||||
continue
|
||||
}
|
||||
tagValues = append(tagValues, tv)
|
||||
labelValues = append(labelValues, labelValue)
|
||||
}
|
||||
// Do not sort tagValues, since they must be sorted by vmselect.
|
||||
return tagValues, nil
|
||||
// Do not sort labelValues, since they must be sorted by vmselect.
|
||||
qt.Printf("found %d label values in the current and the previous indexdb", len(labelValues))
|
||||
return labelValues, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagValuesOnTimeRange(tvs map[string]struct{}, tagKey []byte, tr TimeRange, maxTagValues int) error {
|
||||
func (is *indexSearch) searchLabelValuesWithFiltersOnTimeRange(qt *querytracer.Tracer, lvs map[string]struct{}, labelName string, tfss []*TagFilters,
|
||||
tr TimeRange, maxLabelValues, maxMetrics int) error {
|
||||
minDate := uint64(tr.MinTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp) / msecPerDay
|
||||
if minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
return is.searchTagValues(tvs, tagKey, maxTagValues)
|
||||
maxDate := uint64(tr.MaxTimestamp-1) / msecPerDay
|
||||
if maxDate == 0 || minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
qtChild := qt.NewChild("search for label values in global index: labelName=%q, filters=%s", labelName, tfss)
|
||||
err := is.searchLabelValuesWithFiltersOnDate(qtChild, lvs, labelName, tfss, 0, maxLabelValues, maxMetrics)
|
||||
qtChild.Done()
|
||||
return err
|
||||
}
|
||||
var mu sync.Mutex
|
||||
wg := getWaitGroup()
|
||||
var errGlobal error
|
||||
qt = qt.NewChild("parallel search for label values: labelName=%q, filters=%s, timeRange=%s", labelName, tfss, &tr)
|
||||
for date := minDate; date <= maxDate; date++ {
|
||||
wg.Add(1)
|
||||
qtChild := qt.NewChild("search for label names: filters=%s, date=%d", tfss, date)
|
||||
go func(date uint64) {
|
||||
defer wg.Done()
|
||||
tvsLocal := make(map[string]struct{})
|
||||
defer func() {
|
||||
qtChild.Done()
|
||||
wg.Done()
|
||||
}()
|
||||
lvsLocal := make(map[string]struct{})
|
||||
isLocal := is.db.getIndexSearch(is.accountID, is.projectID, is.deadline)
|
||||
err := isLocal.searchTagValuesOnDate(tvsLocal, tagKey, date, maxTagValues)
|
||||
err := isLocal.searchLabelValuesWithFiltersOnDate(qtChild, lvsLocal, labelName, tfss, date, maxLabelValues, maxMetrics)
|
||||
is.db.putIndexSearch(isLocal)
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
|
@ -994,117 +966,50 @@ func (is *indexSearch) searchTagValuesOnTimeRange(tvs map[string]struct{}, tagKe
|
|||
errGlobal = err
|
||||
return
|
||||
}
|
||||
if len(tvs) >= maxTagValues {
|
||||
if len(lvs) >= maxLabelValues {
|
||||
return
|
||||
}
|
||||
for v := range tvsLocal {
|
||||
tvs[v] = struct{}{}
|
||||
for v := range lvsLocal {
|
||||
lvs[v] = struct{}{}
|
||||
}
|
||||
}(date)
|
||||
}
|
||||
wg.Wait()
|
||||
putWaitGroup(wg)
|
||||
qt.Done()
|
||||
return errGlobal
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagValuesOnDate(tvs map[string]struct{}, tagKey []byte, date uint64, maxTagValues int) error {
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
kb.B = marshalTagValue(kb.B, tagKey)
|
||||
prefix := kb.B
|
||||
ts.Seek(prefix)
|
||||
for len(tvs) < maxTagValues && ts.NextItem() {
|
||||
if loopsPaceLimiter&paceLimiterFastIterationsMask == 0 {
|
||||
if err := checkSearchDeadlineAndPace(is.deadline); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
loopsPaceLimiter++
|
||||
item := ts.Item
|
||||
if !bytes.HasPrefix(item, prefix) {
|
||||
break
|
||||
}
|
||||
if err := mp.Init(item, nsPrefixDateTagToMetricIDs); err != nil {
|
||||
return err
|
||||
}
|
||||
if mp.IsDeletedTag(dmis) {
|
||||
continue
|
||||
}
|
||||
if string(mp.Tag.Key) != string(tagKey) {
|
||||
break
|
||||
}
|
||||
tvs[string(mp.Tag.Value)] = struct{}{}
|
||||
if mp.MetricIDsLen() < maxMetricIDsPerRow/2 {
|
||||
// There is no need in searching for the next tag value,
|
||||
// since it is likely it is located in the next row,
|
||||
// because the current row contains incomplete metricIDs set.
|
||||
continue
|
||||
}
|
||||
// Search for the next tag value.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag value.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
kb.B = marshalTagValue(kb.B, mp.Tag.Key)
|
||||
kb.B = marshalTagValue(kb.B, mp.Tag.Value)
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
}
|
||||
if err := ts.Error(); err != nil {
|
||||
return fmt.Errorf("error when searching for tag name prefix %q: %w", prefix, err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SearchTagValues returns all the tag values for the given tagKey
|
||||
func (db *indexDB) SearchTagValues(accountID, projectID uint32, tagKey []byte, maxTagValues int, deadline uint64) ([]string, error) {
|
||||
tvs := make(map[string]struct{})
|
||||
is := db.getIndexSearch(accountID, projectID, deadline)
|
||||
err := is.searchTagValues(tvs, tagKey, maxTagValues)
|
||||
db.putIndexSearch(is)
|
||||
func (is *indexSearch) searchLabelValuesWithFiltersOnDate(qt *querytracer.Tracer, lvs map[string]struct{}, labelName string, tfss []*TagFilters,
|
||||
date uint64, maxLabelValues, maxMetrics int) error {
|
||||
filter, err := is.searchMetricIDsWithFiltersOnDate(qt, tfss, date, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return err
|
||||
}
|
||||
ok := db.doExtDB(func(extDB *indexDB) {
|
||||
is := extDB.getIndexSearch(accountID, projectID, deadline)
|
||||
err = is.searchTagValues(tvs, tagKey, maxTagValues)
|
||||
extDB.putIndexSearch(is)
|
||||
})
|
||||
if ok && err != nil {
|
||||
return nil, err
|
||||
if filter != nil && filter.Len() == 0 {
|
||||
qt.Printf("found zero label values for filter=%s", tfss)
|
||||
return nil
|
||||
}
|
||||
|
||||
tagValues := make([]string, 0, len(tvs))
|
||||
for tv := range tvs {
|
||||
if len(tv) == 0 {
|
||||
// Skip empty values, since they have no any meaning.
|
||||
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/600
|
||||
continue
|
||||
}
|
||||
tagValues = append(tagValues, tv)
|
||||
if labelName == "__name__" {
|
||||
// __name__ label is encoded as empty string in indexdb.
|
||||
labelName = ""
|
||||
}
|
||||
// Do not sort tagValues, since they must be sorted by vmselect.
|
||||
return tagValues, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchTagValues(tvs map[string]struct{}, tagKey []byte, maxTagValues int) error {
|
||||
labelNameBytes := bytesutil.ToUnsafeBytes(labelName)
|
||||
var prevLabelValue []byte
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
kb.B = marshalTagValue(kb.B, tagKey)
|
||||
nsPrefixExpected := byte(nsPrefixDateTagToMetricIDs)
|
||||
if date == 0 {
|
||||
nsPrefixExpected = nsPrefixTagToMetricIDs
|
||||
}
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
kb.B = marshalTagValue(kb.B, labelNameBytes)
|
||||
prefix := kb.B
|
||||
ts.Seek(prefix)
|
||||
for len(tvs) < maxTagValues && ts.NextItem() {
|
||||
for len(lvs) < maxLabelValues && ts.NextItem() {
|
||||
if loopsPaceLimiter&paceLimiterFastIterationsMask == 0 {
|
||||
if err := checkSearchDeadlineAndPace(is.deadline); err != nil {
|
||||
return err
|
||||
|
@ -1115,30 +1020,29 @@ func (is *indexSearch) searchTagValues(tvs map[string]struct{}, tagKey []byte, m
|
|||
if !bytes.HasPrefix(item, prefix) {
|
||||
break
|
||||
}
|
||||
if err := mp.Init(item, nsPrefixTagToMetricIDs); err != nil {
|
||||
if err := mp.Init(item, nsPrefixExpected); err != nil {
|
||||
return err
|
||||
}
|
||||
if mp.IsDeletedTag(dmis) {
|
||||
continue
|
||||
}
|
||||
if string(mp.Tag.Key) != string(tagKey) {
|
||||
break
|
||||
}
|
||||
tvs[string(mp.Tag.Value)] = struct{}{}
|
||||
if mp.MetricIDsLen() < maxMetricIDsPerRow/2 {
|
||||
// There is no need in searching for the next tag value,
|
||||
// since it is likely it is located in the next row,
|
||||
// because the current row contains incomplete metricIDs set.
|
||||
if mp.GetMatchingSeriesCount(filter) == 0 {
|
||||
continue
|
||||
}
|
||||
// Search for the next tag value.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag value.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
kb.B = marshalTagValue(kb.B, mp.Tag.Key)
|
||||
kb.B = marshalTagValue(kb.B, mp.Tag.Value)
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
labelValue := mp.Tag.Value
|
||||
if string(labelValue) == string(prevLabelValue) {
|
||||
// Search for the next tag value.
|
||||
// The last char in kb.B must be tagSeparatorChar.
|
||||
// Just increment it in order to jump to the next tag value.
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
kb.B = marshalTagValue(kb.B, labelNameBytes)
|
||||
kb.B = marshalTagValue(kb.B, labelValue)
|
||||
kb.B[len(kb.B)-1]++
|
||||
ts.Seek(kb.B)
|
||||
continue
|
||||
}
|
||||
lvs[string(labelValue)] = struct{}{}
|
||||
prevLabelValue = append(prevLabelValue[:0], labelValue...)
|
||||
}
|
||||
if err := ts.Error(); err != nil {
|
||||
return fmt.Errorf("error when searching for tag name prefix %q: %w", prefix, err)
|
||||
|
@ -1187,7 +1091,7 @@ func (db *indexDB) SearchTagValueSuffixes(accountID, projectID uint32, tr TimeRa
|
|||
|
||||
func (is *indexSearch) searchTagValueSuffixesForTimeRange(tvss map[string]struct{}, tr TimeRange, tagKey, tagValuePrefix []byte, delimiter byte, maxTagValueSuffixes int) error {
|
||||
minDate := uint64(tr.MinTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp-1) / msecPerDay
|
||||
if minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
return is.searchTagValueSuffixesAll(tvss, tagKey, tagValuePrefix, delimiter, maxTagValueSuffixes)
|
||||
}
|
||||
|
@ -1253,7 +1157,6 @@ func (is *indexSearch) searchTagValueSuffixesForPrefix(tvss map[string]struct{},
|
|||
kb := &is.kb
|
||||
ts := &is.ts
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
dmis := is.db.s.getDeletedMetricIDs()
|
||||
loopsPaceLimiter := 0
|
||||
ts.Seek(prefix)
|
||||
|
@ -1394,23 +1297,14 @@ func (db *indexDB) GetTSDBStatusWithFiltersForDate(qt *querytracer.Tracer, accou
|
|||
|
||||
// getTSDBStatusWithFiltersForDate returns topN entries for tsdb status for the given tfss and the given date.
|
||||
func (is *indexSearch) getTSDBStatusWithFiltersForDate(qt *querytracer.Tracer, tfss []*TagFilters, date uint64, topN, maxMetrics int) (*TSDBStatus, error) {
|
||||
var filter *uint64set.Set
|
||||
if len(tfss) > 0 {
|
||||
tr := TimeRange{
|
||||
MinTimestamp: int64(date) * msecPerDay,
|
||||
MaxTimestamp: int64(date+1)*msecPerDay - 1,
|
||||
}
|
||||
metricIDs, err := is.searchMetricIDsInternal(qt, tfss, tr, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if metricIDs.Len() == 0 {
|
||||
// Nothing found.
|
||||
return &TSDBStatus{}, nil
|
||||
}
|
||||
filter = metricIDs
|
||||
filter, err := is.searchMetricIDsWithFiltersOnDate(qt, tfss, date, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if filter != nil && filter.Len() == 0 {
|
||||
qt.Printf("no matching series for filter=%s", tfss)
|
||||
return &TSDBStatus{}, nil
|
||||
}
|
||||
|
||||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
|
@ -1423,8 +1317,11 @@ func (is *indexSearch) getTSDBStatusWithFiltersForDate(qt *querytracer.Tracer, t
|
|||
nameEqualBytes := []byte("__name__=")
|
||||
|
||||
loopsPaceLimiter := 0
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
nsPrefixExpected := byte(nsPrefixDateTagToMetricIDs)
|
||||
if date == 0 {
|
||||
nsPrefixExpected = nsPrefixTagToMetricIDs
|
||||
}
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
prefix := kb.B
|
||||
ts.Seek(prefix)
|
||||
for ts.NextItem() {
|
||||
|
@ -1438,28 +1335,15 @@ func (is *indexSearch) getTSDBStatusWithFiltersForDate(qt *querytracer.Tracer, t
|
|||
if !bytes.HasPrefix(item, prefix) {
|
||||
break
|
||||
}
|
||||
matchingSeriesCount := 0
|
||||
if filter != nil {
|
||||
if err := mp.Init(item, nsPrefixDateTagToMetricIDs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
mp.ParseMetricIDs()
|
||||
for _, metricID := range mp.MetricIDs {
|
||||
if filter.Has(metricID) {
|
||||
matchingSeriesCount++
|
||||
}
|
||||
}
|
||||
if matchingSeriesCount == 0 {
|
||||
// Skip rows without matching metricIDs.
|
||||
continue
|
||||
}
|
||||
if err := mp.Init(item, nsPrefixExpected); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
tail := item[len(prefix):]
|
||||
var err error
|
||||
tail, tmp, err = unmarshalTagValue(tmp[:0], tail)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot unmarshal tag key from line %q: %w", item, err)
|
||||
matchingSeriesCount := mp.GetMatchingSeriesCount(filter)
|
||||
if matchingSeriesCount == 0 {
|
||||
// Skip rows without matching metricIDs.
|
||||
continue
|
||||
}
|
||||
tmp = append(tmp[:0], mp.Tag.Key...)
|
||||
tagKey := tmp
|
||||
if isArtificialTagKey(tagKey) {
|
||||
// Skip artificially created tag keys.
|
||||
|
@ -1478,17 +1362,8 @@ func (is *indexSearch) getTSDBStatusWithFiltersForDate(qt *querytracer.Tracer, t
|
|||
tmp = tagKey
|
||||
}
|
||||
tmp = append(tmp, '=')
|
||||
tail, tmp, err = unmarshalTagValue(tmp, tail)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot unmarshal tag value from line %q: %w", item, err)
|
||||
}
|
||||
tmp = append(tmp, mp.Tag.Value...)
|
||||
tagKeyValue := tmp
|
||||
if filter == nil {
|
||||
if err := mp.InitOnlyTail(item, tail); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
matchingSeriesCount = mp.MetricIDsLen()
|
||||
}
|
||||
if string(tagKey) == "__name__" {
|
||||
totalSeries += uint64(matchingSeriesCount)
|
||||
}
|
||||
|
@ -2268,6 +2143,25 @@ func matchTagFilters(mn *MetricName, tfs []*tagFilter, kb *bytesutil.ByteBuffer)
|
|||
return true, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchMetricIDsWithFiltersOnDate(qt *querytracer.Tracer, tfss []*TagFilters, date uint64, maxMetrics int) (*uint64set.Set, error) {
|
||||
if len(tfss) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
tr := TimeRange{
|
||||
MinTimestamp: int64(date) * msecPerDay,
|
||||
MaxTimestamp: int64(date+1)*msecPerDay - 1,
|
||||
}
|
||||
if date == 0 {
|
||||
// Search for metricIDs on the whole time range.
|
||||
tr.MaxTimestamp = timestampFromTime(time.Now())
|
||||
}
|
||||
metricIDs, err := is.searchMetricIDsInternal(qt, tfss, tr, maxMetrics)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return metricIDs, nil
|
||||
}
|
||||
|
||||
func (is *indexSearch) searchMetricIDs(qt *querytracer.Tracer, tfss []*TagFilters, tr TimeRange, maxMetrics int) ([]uint64, error) {
|
||||
metricIDs, err := is.searchMetricIDsInternal(qt, tfss, tr, maxMetrics)
|
||||
if err != nil {
|
||||
|
@ -2384,7 +2278,6 @@ func (is *indexSearch) getMetricIDsForTagFilterSlow(tf *tagFilter, f func(metric
|
|||
ts := &is.ts
|
||||
kb := &is.kb
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
var prevMatchingSuffix []byte
|
||||
var prevMatch bool
|
||||
var loopsCount int64
|
||||
|
@ -2492,7 +2385,6 @@ func (is *indexSearch) updateMetricIDsForOrSuffixes(tf *tagFilter, metricIDs *ui
|
|||
func (is *indexSearch) updateMetricIDsForOrSuffix(prefix []byte, metricIDs *uint64set.Set, maxMetrics int, maxLoopsCount int64) (int64, error) {
|
||||
ts := &is.ts
|
||||
mp := &is.mp
|
||||
mp.Reset()
|
||||
var loopsCount int64
|
||||
loopsPaceLimiter := 0
|
||||
ts.Seek(prefix)
|
||||
|
@ -2530,7 +2422,7 @@ const maxDaysForPerDaySearch = 40
|
|||
func (is *indexSearch) tryUpdatingMetricIDsForDateRange(qt *querytracer.Tracer, metricIDs *uint64set.Set, tfs *TagFilters, tr TimeRange, maxMetrics int) error {
|
||||
atomic.AddUint64(&is.db.dateRangeSearchCalls, 1)
|
||||
minDate := uint64(tr.MinTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp) / msecPerDay
|
||||
maxDate := uint64(tr.MaxTimestamp-1) / msecPerDay
|
||||
if minDate > maxDate || maxDate-minDate > maxDaysForPerDaySearch {
|
||||
// Too much dates must be covered. Give up, since it may be slow.
|
||||
return errFallbackToGlobalSearch
|
||||
|
@ -2952,14 +2844,7 @@ func (is *indexSearch) getMetricIDsForDateTagFilter(qt *querytracer.Tracer, tf *
|
|||
}
|
||||
kb := kbPool.Get()
|
||||
defer kbPool.Put(kb)
|
||||
if date != 0 {
|
||||
// Use per-date search.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
} else {
|
||||
// Use global search if date isn't set.
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
}
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
prefix := kb.B
|
||||
kb.B = append(kb.B, tf.prefix[len(commonPrefix):]...)
|
||||
tfNew := *tf
|
||||
|
@ -3029,14 +2914,7 @@ func (is *indexSearch) getMetricIDsForDate(date uint64, maxMetrics int) (*uint64
|
|||
// Extract all the metricIDs from (date, __name__=value)->metricIDs entries.
|
||||
kb := kbPool.Get()
|
||||
defer kbPool.Put(kb)
|
||||
if date != 0 {
|
||||
// Use per-date search
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixDateTagToMetricIDs)
|
||||
kb.B = encoding.MarshalUint64(kb.B, date)
|
||||
} else {
|
||||
// Use global search
|
||||
kb.B = is.marshalCommonPrefix(kb.B[:0], nsPrefixTagToMetricIDs)
|
||||
}
|
||||
kb.B = is.marshalCommonPrefixForDate(kb.B[:0], date)
|
||||
kb.B = marshalTagValue(kb.B, nil)
|
||||
var metricIDs uint64set.Set
|
||||
if err := is.updateMetricIDsForPrefix(kb.B, &metricIDs, maxMetrics); err != nil {
|
||||
|
@ -3111,6 +2989,16 @@ func (is *indexSearch) marshalCommonPrefix(dst []byte, nsPrefix byte) []byte {
|
|||
return marshalCommonPrefix(dst, nsPrefix, is.accountID, is.projectID)
|
||||
}
|
||||
|
||||
func (is *indexSearch) marshalCommonPrefixForDate(dst []byte, date uint64) []byte {
|
||||
if date == 0 {
|
||||
// Global index
|
||||
return is.marshalCommonPrefix(dst, nsPrefixTagToMetricIDs)
|
||||
}
|
||||
// Per-day index
|
||||
dst = is.marshalCommonPrefix(dst, nsPrefixDateTagToMetricIDs)
|
||||
return encoding.MarshalUint64(dst, date)
|
||||
}
|
||||
|
||||
func unmarshalCommonPrefix(src []byte) ([]byte, byte, uint32, uint32, error) {
|
||||
if len(src) < commonPrefixLen {
|
||||
return nil, 0, 0, 0, fmt.Errorf("cannot unmarshal common prefix from %d bytes; need at least %d bytes; data=%X", len(src), commonPrefixLen, src)
|
||||
|
@ -3141,6 +3029,9 @@ type tagToMetricIDsRowParser struct {
|
|||
// MetricIDs contains parsed MetricIDs after ParseMetricIDs call
|
||||
MetricIDs []uint64
|
||||
|
||||
// metricIDsParsed is set to true after ParseMetricIDs call
|
||||
metricIDsParsed bool
|
||||
|
||||
// Tag contains parsed tag after Init call
|
||||
Tag Tag
|
||||
|
||||
|
@ -3154,6 +3045,7 @@ func (mp *tagToMetricIDsRowParser) Reset() {
|
|||
mp.ProjectID = 0
|
||||
mp.Date = 0
|
||||
mp.MetricIDs = mp.MetricIDs[:0]
|
||||
mp.metricIDsParsed = false
|
||||
mp.Tag.Reset()
|
||||
mp.tail = nil
|
||||
}
|
||||
|
@ -3209,6 +3101,7 @@ func (mp *tagToMetricIDsRowParser) InitOnlyTail(b, tail []byte) error {
|
|||
return fmt.Errorf("invalid tail length in the tag->metricIDs row; got %d bytes; must be multiple of 8 bytes", len(tail))
|
||||
}
|
||||
mp.tail = tail
|
||||
mp.metricIDsParsed = false
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -3229,6 +3122,9 @@ func (mp *tagToMetricIDsRowParser) MetricIDsLen() int {
|
|||
|
||||
// ParseMetricIDs parses MetricIDs from mp.tail into mp.MetricIDs.
|
||||
func (mp *tagToMetricIDsRowParser) ParseMetricIDs() {
|
||||
if mp.metricIDsParsed {
|
||||
return
|
||||
}
|
||||
tail := mp.tail
|
||||
mp.MetricIDs = mp.MetricIDs[:0]
|
||||
n := len(tail) / 8
|
||||
|
@ -3248,6 +3144,24 @@ func (mp *tagToMetricIDsRowParser) ParseMetricIDs() {
|
|||
metricIDs[i] = metricID
|
||||
tail = tail[8:]
|
||||
}
|
||||
mp.metricIDsParsed = true
|
||||
}
|
||||
|
||||
// GetMatchingSeriesCount returns the number of series in mp, which match metricIDs from the given filter.
|
||||
//
|
||||
// if filter is empty, then all series in mp are taken into account.
|
||||
func (mp *tagToMetricIDsRowParser) GetMatchingSeriesCount(filter *uint64set.Set) int {
|
||||
if filter == nil {
|
||||
return mp.MetricIDsLen()
|
||||
}
|
||||
mp.ParseMetricIDs()
|
||||
n := 0
|
||||
for _, metricID := range mp.MetricIDs {
|
||||
if filter.Has(metricID) {
|
||||
n++
|
||||
}
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
// IsDeletedTag verifies whether the tag from mp is deleted according to dmis.
|
||||
|
|
|
@ -731,16 +731,16 @@ func testIndexDBGetOrCreateTSIDByName(db *indexDB, accountsCount, projectsCount,
|
|||
}
|
||||
|
||||
func testIndexDBCheckTSIDByName(db *indexDB, mns []MetricName, tsids []TSID, isConcurrent bool) error {
|
||||
hasValue := func(tvs []string, v []byte) bool {
|
||||
for _, tv := range tvs {
|
||||
if string(v) == tv {
|
||||
hasValue := func(lvs []string, v []byte) bool {
|
||||
for _, lv := range lvs {
|
||||
if string(v) == lv {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
allKeys := make(map[accountProjectKey]map[string]bool)
|
||||
allLabelNames := make(map[accountProjectKey]map[string]bool)
|
||||
timeseriesCounters := make(map[accountProjectKey]map[uint64]bool)
|
||||
var tsidCopy TSID
|
||||
var metricNameCopy []byte
|
||||
|
@ -793,44 +793,44 @@ func testIndexDBCheckTSIDByName(db *indexDB, mns []MetricName, tsids []TSID, isC
|
|||
return fmt.Errorf("expecting empty buf when searching for non-existent metricID; got %X", buf)
|
||||
}
|
||||
|
||||
// Test SearchTagValues
|
||||
tvs, err := db.SearchTagValues(mn.AccountID, mn.ProjectID, nil, 1e5, noDeadline)
|
||||
// Test SearchLabelValuesWithFiltersOnTimeRange
|
||||
lvs, err := db.SearchLabelValuesWithFiltersOnTimeRange(nil, mn.AccountID, mn.ProjectID, "__name__", nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues for __name__: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange(labelName=%q): %w", "__name__", err)
|
||||
}
|
||||
if !hasValue(tvs, mn.MetricGroup) {
|
||||
return fmt.Errorf("SearchTagValues couldn't find %q; found %q", mn.MetricGroup, tvs)
|
||||
if !hasValue(lvs, mn.MetricGroup) {
|
||||
return fmt.Errorf("SearchLabelValuesWithFiltersOnTimeRange(labelName=%q): couldn't find %q; found %q", "__name__", mn.MetricGroup, lvs)
|
||||
}
|
||||
apKeys := allKeys[apKey]
|
||||
if apKeys == nil {
|
||||
apKeys = make(map[string]bool)
|
||||
allKeys[apKey] = apKeys
|
||||
labelNames := allLabelNames[apKey]
|
||||
if labelNames == nil {
|
||||
labelNames = make(map[string]bool)
|
||||
allLabelNames[apKey] = labelNames
|
||||
}
|
||||
for i := range mn.Tags {
|
||||
tag := &mn.Tags[i]
|
||||
tvs, err := db.SearchTagValues(mn.AccountID, mn.ProjectID, tag.Key, 1e5, noDeadline)
|
||||
lvs, err := db.SearchLabelValuesWithFiltersOnTimeRange(nil, mn.AccountID, mn.ProjectID, string(tag.Key), nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues for __name__: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange(labelName=%q): %w", tag.Key, err)
|
||||
}
|
||||
if !hasValue(tvs, tag.Value) {
|
||||
return fmt.Errorf("SearchTagValues couldn't find %q=%q; found %q", tag.Key, tag.Value, tvs)
|
||||
if !hasValue(lvs, tag.Value) {
|
||||
return fmt.Errorf("SearchLabelValuesWithFiltersOnTimeRange(labelName=%q): couldn't find %q; found %q", tag.Key, tag.Value, lvs)
|
||||
}
|
||||
apKeys[string(tag.Key)] = true
|
||||
labelNames[string(tag.Key)] = true
|
||||
}
|
||||
}
|
||||
|
||||
// Test SearchTagKeys
|
||||
for k, apKeys := range allKeys {
|
||||
tks, err := db.SearchTagKeys(k.AccountID, k.ProjectID, 1e5, noDeadline)
|
||||
// Test SearchLabelNamesWithFiltersOnTimeRange (empty filters, global time range)
|
||||
for k, labelNames := range allLabelNames {
|
||||
lns, err := db.SearchLabelNamesWithFiltersOnTimeRange(nil, k.AccountID, k.ProjectID, nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeys: %w", err)
|
||||
}
|
||||
if !hasValue(tks, nil) {
|
||||
return fmt.Errorf("cannot find __name__ in %q", tks)
|
||||
if !hasValue(lns, []byte("__name__")) {
|
||||
return fmt.Errorf("cannot find __name__ in %q", lns)
|
||||
}
|
||||
for key := range apKeys {
|
||||
if !hasValue(tks, []byte(key)) {
|
||||
return fmt.Errorf("cannot find %q in %q", key, tks)
|
||||
for labelName := range labelNames {
|
||||
if !hasValue(lns, []byte(labelName)) {
|
||||
return fmt.Errorf("cannot find %q in %q", labelName, lns)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1705,13 +1705,13 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
|||
var metricNameBuf []byte
|
||||
perDayMetricIDs := make(map[uint64]*uint64set.Set)
|
||||
var allMetricIDs uint64set.Set
|
||||
tagKeys := []string{
|
||||
"", "constant", "day", "uniqueid",
|
||||
labelNames := []string{
|
||||
"__name__", "constant", "day", "uniqueid",
|
||||
}
|
||||
tagValues := []string{
|
||||
labelValues := []string{
|
||||
"testMetric",
|
||||
}
|
||||
sort.Strings(tagKeys)
|
||||
sort.Strings(labelNames)
|
||||
for day := 0; day < days; day++ {
|
||||
var tsids []TSID
|
||||
var mns []MetricName
|
||||
|
@ -1789,30 +1789,28 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
|||
t.Fatalf("unexpected metricIDs found;\ngot\n%d\nwant\n%d", metricIDs.AppendTo(nil), allMetricIDs.AppendTo(nil))
|
||||
}
|
||||
|
||||
// Check SearchTagKeysOnTimeRange.
|
||||
tks, err := db.SearchTagKeysOnTimeRange(accountID, projectID, TimeRange{
|
||||
// Check SearchLabelNamesWithFiltersOnTimeRange with the specified time range.
|
||||
tr := TimeRange{
|
||||
MinTimestamp: int64(now) - msecPerDay,
|
||||
MaxTimestamp: int64(now),
|
||||
}, 10000, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchTagKeysOnTimeRange: %s", err)
|
||||
}
|
||||
sort.Strings(tks)
|
||||
if !reflect.DeepEqual(tks, tagKeys) {
|
||||
t.Fatalf("unexpected tagKeys; got\n%s\nwant\n%s", tks, tagKeys)
|
||||
lns, err := db.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, nil, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNamesWithFiltersOnTimeRange(timeRange=%s): %s", &tr, err)
|
||||
}
|
||||
sort.Strings(lns)
|
||||
if !reflect.DeepEqual(lns, labelNames) {
|
||||
t.Fatalf("unexpected labelNames; got\n%s\nwant\n%s", lns, labelNames)
|
||||
}
|
||||
|
||||
// Check SearchTagValuesOnTimeRange.
|
||||
tvs, err := db.SearchTagValuesOnTimeRange(accountID, projectID, []byte(""), TimeRange{
|
||||
MinTimestamp: int64(now) - msecPerDay,
|
||||
MaxTimestamp: int64(now),
|
||||
}, 10000, noDeadline)
|
||||
// Check SearchLabelValuesWithFiltersOnTimeRange with the specified time range.
|
||||
lvs, err := db.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, "", nil, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchTagValuesOnTimeRange: %s", err)
|
||||
t.Fatalf("unexpected error in SearchLabelValuesWithFiltersOnTimeRange(timeRange=%s): %s", &tr, err)
|
||||
}
|
||||
sort.Strings(tvs)
|
||||
if !reflect.DeepEqual(tvs, tagValues) {
|
||||
t.Fatalf("unexpected tagValues; got\n%s\nwant\n%s", tvs, tagValues)
|
||||
sort.Strings(lvs)
|
||||
if !reflect.DeepEqual(lvs, labelValues) {
|
||||
t.Fatalf("unexpected labelValues; got\n%s\nwant\n%s", lvs, labelValues)
|
||||
}
|
||||
|
||||
// Create a filter that will match series that occur across multiple days
|
||||
|
@ -1823,7 +1821,7 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
|||
|
||||
// Perform a search within a day.
|
||||
// This should return the metrics for the day
|
||||
tr := TimeRange{
|
||||
tr = TimeRange{
|
||||
MinTimestamp: int64(now - 2*msecPerHour - 1),
|
||||
MaxTimestamp: int64(now),
|
||||
}
|
||||
|
@ -1835,6 +1833,46 @@ func TestSearchTSIDWithTimeRange(t *testing.T) {
|
|||
t.Fatalf("expected %d time series for current day, got %d time series", metricsPerDay, len(matchedTSIDs))
|
||||
}
|
||||
|
||||
// Check SearchLabelNamesWithFiltersOnTimeRange with the specified filter.
|
||||
lns, err = db.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNamesWithFiltersOnTimeRange(filters=%s): %s", tfs, err)
|
||||
}
|
||||
sort.Strings(lns)
|
||||
if !reflect.DeepEqual(lns, labelNames) {
|
||||
t.Fatalf("unexpected labelNames; got\n%s\nwant\n%s", lns, labelNames)
|
||||
}
|
||||
|
||||
// Check SearchLabelNamesWithFiltersOnTimeRange with the specified filter and time range.
|
||||
lns, err = db.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelNamesWithFiltersOnTimeRange(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
sort.Strings(lns)
|
||||
if !reflect.DeepEqual(lns, labelNames) {
|
||||
t.Fatalf("unexpected labelNames; got\n%s\nwant\n%s", lns, labelNames)
|
||||
}
|
||||
|
||||
// Check SearchLabelValuesWithFiltersOnTimeRange with the specified filter.
|
||||
lvs, err = db.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, "", []*TagFilters{tfs}, TimeRange{}, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValuesWithFiltersOnTimeRange(filters=%s): %s", tfs, err)
|
||||
}
|
||||
sort.Strings(lvs)
|
||||
if !reflect.DeepEqual(lvs, labelValues) {
|
||||
t.Fatalf("unexpected labelValues; got\n%s\nwant\n%s", lvs, labelValues)
|
||||
}
|
||||
|
||||
// Check SearchLabelValuesWithFiltersOnTimeRange with the specified filter and time range.
|
||||
lvs, err = db.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, "", []*TagFilters{tfs}, tr, 10000, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error in SearchLabelValuesWithFiltersOnTimeRange(filters=%s, timeRange=%s): %s", tfs, &tr, err)
|
||||
}
|
||||
sort.Strings(lvs)
|
||||
if !reflect.DeepEqual(lvs, labelValues) {
|
||||
t.Fatalf("unexpected labelValues; got\n%s\nwant\n%s", lvs, labelValues)
|
||||
}
|
||||
|
||||
// Perform a search across all the days, should match all metrics
|
||||
tr = TimeRange{
|
||||
MinTimestamp: int64(now - msecPerDay*days),
|
||||
|
|
|
@ -1335,24 +1335,16 @@ func (s *Storage) DeleteMetrics(tfss []*TagFilters) (int, error) {
|
|||
return deletedCount, nil
|
||||
}
|
||||
|
||||
// SearchTagKeysOnTimeRange searches for tag keys on tr.
|
||||
func (s *Storage) SearchTagKeysOnTimeRange(accountID, projectID uint32, tr TimeRange, maxTagKeys int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchTagKeysOnTimeRange(accountID, projectID, tr, maxTagKeys, deadline)
|
||||
// SearchLabelNamesWithFiltersOnTimeRange searches for label names matching the given tfss on tr.
|
||||
func (s *Storage) SearchLabelNamesWithFiltersOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, tfss []*TagFilters, tr TimeRange,
|
||||
maxLabelNames, maxMetrics int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchLabelNamesWithFiltersOnTimeRange(qt, accountID, projectID, tfss, tr, maxLabelNames, maxMetrics, deadline)
|
||||
}
|
||||
|
||||
// SearchTagKeys searches for tag keys for the given (accountID, projectID).
|
||||
func (s *Storage) SearchTagKeys(accountID, projectID uint32, maxTagKeys int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchTagKeys(accountID, projectID, maxTagKeys, deadline)
|
||||
}
|
||||
|
||||
// SearchTagValuesOnTimeRange searches for tag values for the given tagKey on tr.
|
||||
func (s *Storage) SearchTagValuesOnTimeRange(accountID, projectID uint32, tagKey []byte, tr TimeRange, maxTagValues int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchTagValuesOnTimeRange(accountID, projectID, tagKey, tr, maxTagValues, deadline)
|
||||
}
|
||||
|
||||
// SearchTagValues searches for tag values for the given tagKey in (accountID, projectID).
|
||||
func (s *Storage) SearchTagValues(accountID, projectID uint32, tagKey []byte, maxTagValues int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchTagValues(accountID, projectID, tagKey, maxTagValues, deadline)
|
||||
// SearchLabelValuesWithFiltersOnTimeRange searches for label values for the given labelName, filters and tr.
|
||||
func (s *Storage) SearchLabelValuesWithFiltersOnTimeRange(qt *querytracer.Tracer, accountID, projectID uint32, labelName string, tfss []*TagFilters,
|
||||
tr TimeRange, maxLabelValues, maxMetrics int, deadline uint64) ([]string, error) {
|
||||
return s.idb().SearchLabelValuesWithFiltersOnTimeRange(qt, accountID, projectID, labelName, tfss, tr, maxLabelValues, maxMetrics, deadline)
|
||||
}
|
||||
|
||||
// SearchTagValueSuffixes returns all the tag value suffixes for the given tagKey and tagValuePrefix on the given tr.
|
||||
|
@ -1541,39 +1533,6 @@ func getRegexpPartsForGraphiteQuery(q string) ([]string, string) {
|
|||
}
|
||||
}
|
||||
|
||||
// SearchTagEntries returns a list of (tagName -> tagValues) for (accountID, projectID).
|
||||
func (s *Storage) SearchTagEntries(accountID, projectID uint32, maxTagKeys, maxTagValues int, deadline uint64) ([]TagEntry, error) {
|
||||
idb := s.idb()
|
||||
keys, err := idb.SearchTagKeys(accountID, projectID, maxTagKeys, deadline)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot search tag keys: %w", err)
|
||||
}
|
||||
|
||||
// Sort keys for faster seeks below
|
||||
sort.Strings(keys)
|
||||
|
||||
tes := make([]TagEntry, len(keys))
|
||||
for i, key := range keys {
|
||||
values, err := idb.SearchTagValues(accountID, projectID, []byte(key), maxTagValues, deadline)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("cannot search values for tag %q: %w", key, err)
|
||||
}
|
||||
te := &tes[i]
|
||||
te.Key = key
|
||||
te.Values = values
|
||||
}
|
||||
return tes, nil
|
||||
}
|
||||
|
||||
// TagEntry contains (tagName -> tagValues) mapping
|
||||
type TagEntry struct {
|
||||
// Key is tagName
|
||||
Key string
|
||||
|
||||
// Values contains all the values for Key.
|
||||
Values []string
|
||||
}
|
||||
|
||||
// GetSeriesCount returns the approximate number of unique time series for the given (accountID, projectID).
|
||||
//
|
||||
// It includes the deleted series too and may count the same series
|
||||
|
|
|
@ -538,13 +538,13 @@ func TestStorageDeleteMetrics(t *testing.T) {
|
|||
t.Fatalf("cannot open storage: %s", err)
|
||||
}
|
||||
|
||||
// Verify no tag keys exist
|
||||
tks, err := s.SearchTagKeys(0, 0, 1e5, noDeadline)
|
||||
// Verify no label names exist
|
||||
lns, err := s.SearchLabelNamesWithFiltersOnTimeRange(nil, 0, 0, nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in SearchTagKeys at the start: %s", err)
|
||||
t.Fatalf("error in SearchLabelNamesWithFiltersOnTimeRange() at the start: %s", err)
|
||||
}
|
||||
if len(tks) != 0 {
|
||||
t.Fatalf("found non-empty tag keys at the start: %q", tks)
|
||||
if len(lns) != 0 {
|
||||
t.Fatalf("found non-empty tag keys at the start: %q", lns)
|
||||
}
|
||||
|
||||
t.Run("serial", func(t *testing.T) {
|
||||
|
@ -590,12 +590,12 @@ func TestStorageDeleteMetrics(t *testing.T) {
|
|||
})
|
||||
|
||||
// Verify no more tag keys exist
|
||||
tks, err = s.SearchTagKeys(0, 0, 1e5, noDeadline)
|
||||
lns, err = s.SearchLabelNamesWithFiltersOnTimeRange(nil, 0, 0, nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
t.Fatalf("error in SearchTagKeys after the test: %s", err)
|
||||
t.Fatalf("error in SearchLabelNamesWithFiltersOnTimeRange after the test: %s", err)
|
||||
}
|
||||
if len(tks) != 0 {
|
||||
t.Fatalf("found non-empty tag keys after the test: %q", tks)
|
||||
if len(lns) != 0 {
|
||||
t.Fatalf("found non-empty tag keys after the test: %q", lns)
|
||||
}
|
||||
|
||||
s.MustClose()
|
||||
|
@ -612,8 +612,8 @@ func testStorageDeleteMetrics(s *Storage, workerNum int) error {
|
|||
accountID := uint32(workerNum)
|
||||
projectID := uint32(123)
|
||||
|
||||
tksAll := make(map[string]bool)
|
||||
tksAll[""] = true // __name__
|
||||
lnsAll := make(map[string]bool)
|
||||
lnsAll["__name__"] = true
|
||||
for i := 0; i < metricsCount; i++ {
|
||||
var mrs []MetricRow
|
||||
var mn MetricName
|
||||
|
@ -627,7 +627,7 @@ func testStorageDeleteMetrics(s *Storage, workerNum int) error {
|
|||
{workerTag, []byte("foobar")},
|
||||
}
|
||||
for i := range mn.Tags {
|
||||
tksAll[string(mn.Tags[i].Key)] = true
|
||||
lnsAll[string(mn.Tags[i].Key)] = true
|
||||
}
|
||||
mn.MetricGroup = []byte(fmt.Sprintf("metric_%d_%d", i, workerNum))
|
||||
metricNameRaw := mn.marshalRaw(nil)
|
||||
|
@ -650,21 +650,21 @@ func testStorageDeleteMetrics(s *Storage, workerNum int) error {
|
|||
s.DebugFlush()
|
||||
|
||||
// Verify tag values exist
|
||||
tvs, err := s.SearchTagValues(accountID, projectID, workerTag, 1e5, noDeadline)
|
||||
tvs, err := s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, string(workerTag), nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues before metrics removal: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange before metrics removal: %w", err)
|
||||
}
|
||||
if len(tvs) == 0 {
|
||||
return fmt.Errorf("unexpected empty number of tag values for workerTag")
|
||||
}
|
||||
|
||||
// Verify tag keys exist
|
||||
tks, err := s.SearchTagKeys(accountID, projectID, 1e5, noDeadline)
|
||||
lns, err := s.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeys before metrics removal: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelNamesWithFiltersOnTimeRange before metrics removal: %w", err)
|
||||
}
|
||||
if err := checkTagKeys(tks, tksAll); err != nil {
|
||||
return fmt.Errorf("unexpected tag keys before metrics removal: %w", err)
|
||||
if err := checkLabelNames(lns, lnsAll); err != nil {
|
||||
return fmt.Errorf("unexpected label names before metrics removal: %w", err)
|
||||
}
|
||||
|
||||
var sr Search
|
||||
|
@ -723,9 +723,9 @@ func testStorageDeleteMetrics(s *Storage, workerNum int) error {
|
|||
if n := metricBlocksCount(tfs); n != 0 {
|
||||
return fmt.Errorf("expecting zero metric blocks after deleting all the metrics; got %d blocks", n)
|
||||
}
|
||||
tvs, err = s.SearchTagValues(accountID, projectID, workerTag, 1e5, noDeadline)
|
||||
tvs, err = s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, string(workerTag), nil, TimeRange{}, 1e5, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues after all the metrics are removed: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange after all the metrics are removed: %w", err)
|
||||
}
|
||||
if len(tvs) != 0 {
|
||||
return fmt.Errorf("found non-empty tag values for %q after metrics removal: %q", workerTag, tvs)
|
||||
|
@ -734,21 +734,21 @@ func testStorageDeleteMetrics(s *Storage, workerNum int) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func checkTagKeys(tks []string, tksExpected map[string]bool) error {
|
||||
if len(tks) < len(tksExpected) {
|
||||
return fmt.Errorf("unexpected number of tag keys found; got %d; want at least %d; tks=%q, tksExpected=%v", len(tks), len(tksExpected), tks, tksExpected)
|
||||
func checkLabelNames(lns []string, lnsExpected map[string]bool) error {
|
||||
if len(lns) < len(lnsExpected) {
|
||||
return fmt.Errorf("unexpected number of label names found; got %d; want at least %d; lns=%q, lnsExpected=%v", len(lns), len(lnsExpected), lns, lnsExpected)
|
||||
}
|
||||
hasItem := func(k string, tks []string) bool {
|
||||
for _, kk := range tks {
|
||||
if k == kk {
|
||||
hasItem := func(s string, lns []string) bool {
|
||||
for _, labelName := range lns {
|
||||
if s == labelName {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
for k := range tksExpected {
|
||||
if !hasItem(k, tks) {
|
||||
return fmt.Errorf("cannot find %q in tag keys %q", k, tks)
|
||||
for labelName := range lnsExpected {
|
||||
if !hasItem(labelName, lns) {
|
||||
return fmt.Errorf("cannot find %q in label names %q", labelName, lns)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
@ -840,32 +840,32 @@ func testStorageRegisterMetricNames(s *Storage) error {
|
|||
// Verify the storage contains the added metric names.
|
||||
s.DebugFlush()
|
||||
|
||||
// Verify that SearchTagKeys returns correct result.
|
||||
tksExpected := []string{
|
||||
"",
|
||||
// Verify that SearchLabelNamesWithFiltersOnTimeRange returns correct result.
|
||||
lnsExpected := []string{
|
||||
"__name__",
|
||||
"add_id",
|
||||
"instance",
|
||||
"job",
|
||||
}
|
||||
tks, err := s.SearchTagKeys(accountID, projectID, 100, noDeadline)
|
||||
lns, err := s.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, nil, TimeRange{}, 100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeys: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelNamesWithFiltersOnTimeRange: %w", err)
|
||||
}
|
||||
sort.Strings(tks)
|
||||
if !reflect.DeepEqual(tks, tksExpected) {
|
||||
return fmt.Errorf("unexpected tag keys returned from SearchTagKeys;\ngot\n%q\nwant\n%q", tks, tksExpected)
|
||||
sort.Strings(lns)
|
||||
if !reflect.DeepEqual(lns, lnsExpected) {
|
||||
return fmt.Errorf("unexpected label names returned from SearchLabelNamesWithFiltersOnTimeRange;\ngot\n%q\nwant\n%q", lns, lnsExpected)
|
||||
}
|
||||
|
||||
// Verify that SearchTagKeys returns empty results for incorrect accountID, projectID
|
||||
tks, err = s.SearchTagKeys(accountID+1, projectID+1, 100, noDeadline)
|
||||
// Verify that SearchLabelNamesWithFiltersOnTimeRange returns empty results for incorrect accountID, projectID
|
||||
lns, err = s.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID+1, projectID+1, nil, TimeRange{}, 100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeys for incorrect accountID, projectID: %w", err)
|
||||
}
|
||||
if len(tks) > 0 {
|
||||
return fmt.Errorf("SearchTagKeys with incorrect accountID, projectID returns unexpected non-empty result:\n%q", tks)
|
||||
if len(lns) > 0 {
|
||||
return fmt.Errorf("SearchTagKeys with incorrect accountID, projectID returns unexpected non-empty result:\n%q", lns)
|
||||
}
|
||||
|
||||
// Verify that SearchTagKeysOnTimeRange returns correct result.
|
||||
// Verify that SearchLabelNamesWithFiltersOnTimeRange with the specified timr range returns correct result.
|
||||
now := timestampFromTime(time.Now())
|
||||
start := now - msecPerDay
|
||||
end := now + 60*1000
|
||||
|
@ -873,36 +873,36 @@ func testStorageRegisterMetricNames(s *Storage) error {
|
|||
MinTimestamp: start,
|
||||
MaxTimestamp: end,
|
||||
}
|
||||
tks, err = s.SearchTagKeysOnTimeRange(accountID, projectID, tr, 100, noDeadline)
|
||||
lns, err = s.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID, projectID, nil, tr, 100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeysOnTimeRange: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelNamesWithFiltersOnTimeRange: %w", err)
|
||||
}
|
||||
sort.Strings(tks)
|
||||
if !reflect.DeepEqual(tks, tksExpected) {
|
||||
return fmt.Errorf("unexpected tag keys returned from SearchTagKeysOnTimeRange;\ngot\n%q\nwant\n%q", tks, tksExpected)
|
||||
sort.Strings(lns)
|
||||
if !reflect.DeepEqual(lns, lnsExpected) {
|
||||
return fmt.Errorf("unexpected label names returned from SearchLabelNamesWithFiltersOnTimeRange;\ngot\n%q\nwant\n%q", lns, lnsExpected)
|
||||
}
|
||||
|
||||
// Verify that SearchTagKeysOnTimeRange returns empty results for incrorrect accountID, projectID
|
||||
tks, err = s.SearchTagKeysOnTimeRange(accountID+1, projectID+1, tr, 100, noDeadline)
|
||||
// Verify that SearchLabelNamesWithFiltersOnTimeRange with the specified time range returns empty results for incrorrect accountID, projectID
|
||||
lns, err = s.SearchLabelNamesWithFiltersOnTimeRange(nil, accountID+1, projectID+1, nil, tr, 100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagKeysOnTimeRange for incorrect accountID, projectID: %w", err)
|
||||
}
|
||||
if len(tks) > 0 {
|
||||
return fmt.Errorf("SearchTagKeysOnTimeRange with incorrect accountID, projectID returns unexpected non-empty result:\n%q", tks)
|
||||
if len(lns) > 0 {
|
||||
return fmt.Errorf("SearchTagKeysOnTimeRange with incorrect accountID, projectID returns unexpected non-empty result:\n%q", lns)
|
||||
}
|
||||
|
||||
// Verify that SearchTagValues returns correct result.
|
||||
addIDs, err := s.SearchTagValues(accountID, projectID, []byte("add_id"), addsCount+100, noDeadline)
|
||||
// Verify that SearchLabelValuesWithFiltersOnTimeRange returns correct result.
|
||||
addIDs, err := s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, "add_id", nil, TimeRange{}, addsCount+100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange: %w", err)
|
||||
}
|
||||
sort.Strings(addIDs)
|
||||
if !reflect.DeepEqual(addIDs, addIDsExpected) {
|
||||
return fmt.Errorf("unexpected tag values returned from SearchTagValues;\ngot\n%q\nwant\n%q", addIDs, addIDsExpected)
|
||||
return fmt.Errorf("unexpected tag values returned from SearchLabelValuesWithFiltersOnTimeRange;\ngot\n%q\nwant\n%q", addIDs, addIDsExpected)
|
||||
}
|
||||
|
||||
// Verify that SearchTagValues return empty results for incorrect accountID, projectID
|
||||
addIDs, err = s.SearchTagValues(accountID+1, projectID+1, []byte("add_id"), addsCount+100, noDeadline)
|
||||
// Verify that SearchLabelValuesWithFiltersOnTimeRange return empty results for incorrect accountID, projectID
|
||||
addIDs, err = s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID+1, projectID+1, "add_id", nil, TimeRange{}, addsCount+100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValues for incorrect accountID, projectID: %w", err)
|
||||
}
|
||||
|
@ -910,23 +910,23 @@ func testStorageRegisterMetricNames(s *Storage) error {
|
|||
return fmt.Errorf("SearchTagValues with incorrect accountID, projectID returns unexpected non-empty result:\n%q", addIDs)
|
||||
}
|
||||
|
||||
// Verify that SearchTagValuesOnTimeRange returns correct result.
|
||||
addIDs, err = s.SearchTagValuesOnTimeRange(accountID, projectID, []byte("add_id"), tr, addsCount+100, noDeadline)
|
||||
// Verify that SearchLabelValuesWithFiltersOnTimeRange with the specified time range returns correct result.
|
||||
addIDs, err = s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID, projectID, "add_id", nil, tr, addsCount+100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValuesOnTimeRange: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange: %w", err)
|
||||
}
|
||||
sort.Strings(addIDs)
|
||||
if !reflect.DeepEqual(addIDs, addIDsExpected) {
|
||||
return fmt.Errorf("unexpected tag values returned from SearchTagValuesOnTimeRange;\ngot\n%q\nwant\n%q", addIDs, addIDsExpected)
|
||||
return fmt.Errorf("unexpected tag values returned from SearchLabelValuesWithFiltersOnTimeRange;\ngot\n%q\nwant\n%q", addIDs, addIDsExpected)
|
||||
}
|
||||
|
||||
// Verify that SearchTagValuesOnTimeRange returns empty results for incorrect accountID, projectID
|
||||
addIDs, err = s.SearchTagValuesOnTimeRange(accountID+1, projectID+1, []byte("addd_id"), tr, addsCount+100, noDeadline)
|
||||
// Verify that SearchLabelValuesWithFiltersOnTimeRange returns empty results for incorrect accountID, projectID
|
||||
addIDs, err = s.SearchLabelValuesWithFiltersOnTimeRange(nil, accountID+1, projectID+1, "addd_id", nil, tr, addsCount+100, 1e9, noDeadline)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in SearchTagValuesOnTimeRange for incorrect accoundID, projectID: %w", err)
|
||||
return fmt.Errorf("error in SearchLabelValuesWithFiltersOnTimeRange for incorrect accoundID, projectID: %w", err)
|
||||
}
|
||||
if len(addIDs) > 0 {
|
||||
return fmt.Errorf("SearchTagValuesOnTimeRange with incorrect accountID, projectID returns unexpected non-empty result:\n%q", addIDs)
|
||||
return fmt.Errorf("SearchLabelValuesWithFiltersOnTimeRange with incorrect accountID, projectID returns unexpected non-empty result:\n%q", addIDs)
|
||||
}
|
||||
|
||||
// Verify that SearchMetricNames returns correct result.
|
||||
|
|
Loading…
Reference in a new issue