Merge branch 'public-single-node' into pmm-6401-read-prometheus-data-files

This commit is contained in:
Aliaksandr Valialkin 2022-09-14 17:54:06 +03:00
commit 1c9f5b3580
No known key found for this signature in database
GPG key ID: A72BEC6CD3D0DED1
122 changed files with 4442 additions and 1238 deletions

View file

@ -17,7 +17,7 @@ jobs:
- name: Setup Go
uses: actions/setup-go@main
with:
go-version: 1.18
go-version: 1.19.1
id: go
- name: Code checkout
uses: actions/checkout@master

View file

@ -19,7 +19,7 @@ jobs:
- name: Setup Go
uses: actions/setup-go@main
with:
go-version: 1.18
go-version: 1.19.1
id: go
- name: Code checkout
uses: actions/checkout@master

View file

@ -169,9 +169,7 @@ publish-release:
git checkout $(TAG) && $(MAKE) release publish && \
git checkout $(TAG)-cluster && $(MAKE) release publish && \
git checkout $(TAG)-enterprise && $(MAKE) release publish && \
git checkout $(TAG)-enterprise-cluster && $(MAKE) release publish && \
$(MAKE) github-create-release && \
$(MAKE) github-upload-assets
git checkout $(TAG)-enterprise-cluster && $(MAKE) release publish
release: \
release-victoria-metrics \
@ -366,7 +364,7 @@ benchmark-pure:
vendor-update:
go get -u -d ./lib/...
go get -u -d ./app/...
go mod tidy -compat=1.18
go mod tidy -compat=1.19.1
go mod vendor
app-local:

View file

@ -260,7 +260,10 @@ Prometheus doesn't drop data during VictoriaMetrics restart. See [this article](
VictoriaMetrics provides UI for query troubleshooting and exploration. The UI is available at `http://victoriametrics:8428/vmui`.
The UI allows exploring query results via graphs and tables.
It also provides the ability to [explore cardinality](#cardinality-explorer) and to [investigate query traces](#query-tracing).
It also provides the following features:
- [cardinality explorer](#cardinality-explorer)
- [query tracer](#query-tracing)
- [top queries explorer](#top-queries)
Graphs in vmui support scrolling and zooming:
@ -280,6 +283,13 @@ VMUI allows investigating correlations between two queries on the same graph. Ju
See the [example VMUI at VictoriaMetrics playground](https://play.victoriametrics.com/select/accounting/1/6a716b0f-38bc-4856-90ce-448fd713e3fe/prometheus/graph/?g0.expr=100%20*%20sum(rate(process_cpu_seconds_total))%20by%20(job)&g0.range_input=1d).
## Top queries
[VMUI](#vmui) provides `top queries` tab, which can help determining the following query types:
* the most frequently executed queries;
* queries with the biggest average execution duration;
* queries that took the most summary time for execution.
## Cardinality explorer
@ -329,11 +339,11 @@ See also [vmagent](https://docs.victoriametrics.com/vmagent.html), which can be
VictoriaMetrics accepts data from [DataDog agent](https://docs.datadoghq.com/agent/) or [DogStatsD](https://docs.datadoghq.com/developers/dogstatsd/) via ["submit metrics" API](https://docs.datadoghq.com/api/latest/metrics/#submit-metrics) at `/datadog/api/v1/series` path.
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://victoriametrics-host:8428/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `http://victoriametrics-host:8428/datadog`.
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`.
@ -341,7 +351,7 @@ VictoriaMetrics doesn't check `DD_API_KEY` param, so it can be set to arbitrary
Example of how to send data to VictoriaMetrics via [DataDog "submit metrics"](https://docs.victoriametrics.com/url-examples.html#datadogapiv1series) from command line:
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
```console
echo '
@ -365,7 +375,7 @@ echo '
' | curl -X POST --data-binary @- http://victoriametrics-host:8428/datadog/api/v1/series
```
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -396,7 +406,7 @@ echo '
The imported data can be read via [export API](https://docs.victoriametrics.com/url-examples.html#apiv1export):
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -406,7 +416,7 @@ curl http://victoriametrics-host:8428/api/v1/export -d 'match[]=system.load.1'
</div>
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -769,7 +779,7 @@ to your needs or when testing bugfixes.
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1
2. Run `make victoria-metrics` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `victoria-metrics` binary and puts it into the `bin` folder.
@ -785,7 +795,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1
2. Run `make victoria-metrics-linux-arm` or `make victoria-metrics-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `victoria-metrics-linux-arm` or `victoria-metrics-linux-arm64` binary respectively and puts it into the `bin` folder.
@ -799,7 +809,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
`Pure Go` mode builds only Go code without [cgo](https://golang.org/cmd/cgo/) dependencies.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1
2. Run `make victoria-metrics-pure` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `victoria-metrics-pure` binary and puts it into the `bin` folder.

View file

@ -829,7 +829,7 @@ We recommend using [binary releases](https://github.com/VictoriaMetrics/Victoria
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmagent` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds the `vmagent` binary and puts it into the `bin` folder.
@ -858,7 +858,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmagent-linux-arm` or `make vmagent-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics)
It builds `vmagent-linux-arm` or `vmagent-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -184,6 +184,13 @@ expr: <string>
# as firing once they return.
[ for: <duration> | default = 0s ]
# Whether to print debug information into logs.
# Information includes alerts state changes and requests sent to the datasource.
# Please note, that if rule's query params contain sensitive
# information - it will be printed to logs.
# Is applicable to alerting rules only.
[ debug: <bool> | default = false ]
# Labels to add or overwrite for each alert.
labels:
[ <labelname>: <tmpl_string> ]
@ -504,6 +511,7 @@ or time series modification via [relabeling](https://docs.victoriametrics.com/vm
* `http://<vmalert-addr>/vmalert/api/v1/alert?group_id=<group_id>&alert_id=<alert_id>` - get alert status in JSON format.
Used as alert source in AlertManager.
* `http://<vmalert-addr>/vmalert/alert?group_id=<group_id>&alert_id=<alert_id>` - get alert status in web UI.
* `http://<vmalert-addr>/vmalert/rule?group_id=<group_id>&rule_id=<rule_id>` - get rule status in web UI.
* `http://<vmalert-addr>/metrics` - application metrics.
* `http://<vmalert-addr>/-/reload` - hot configuration reload.
@ -1170,7 +1178,7 @@ spec:
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmalert` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmalert` binary and puts it into the `bin` folder.
@ -1186,7 +1194,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmalert-linux-arm` or `make vmalert-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmalert-linux-arm` or `vmalert-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -6,6 +6,7 @@ import (
"hash/fnv"
"sort"
"strconv"
"strings"
"sync"
"time"
@ -30,24 +31,17 @@ type AlertingRule struct {
GroupID uint64
GroupName string
EvalInterval time.Duration
Debug bool
q datasource.Querier
// guard status fields
mu sync.RWMutex
alertsMu sync.RWMutex
// stores list of active alerts
alerts map[uint64]*notifier.Alert
// stores last moment of time Exec was called
lastExecTime time.Time
// stores the duration of the last Exec call
lastExecDuration time.Duration
// stores last error that happened in Exec func
// resets on every successful Exec
// may be used as Health state
lastExecError error
// stores the number of samples returned during
// the last evaluation
lastExecSamples int
// state stores recent state changes
// during evaluations
state *ruleState
metrics *alertingRuleMetrics
}
@ -71,21 +65,24 @@ func newAlertingRule(qb datasource.QuerierBuilder, group *Group, cfg config.Rule
GroupID: group.ID(),
GroupName: group.Name,
EvalInterval: group.Interval,
Debug: cfg.Debug,
q: qb.BuildWithParams(datasource.QuerierParams{
DataSourceType: group.Type.String(),
EvaluationInterval: group.Interval,
QueryParams: group.Params,
Headers: group.Headers,
Debug: cfg.Debug,
}),
alerts: make(map[uint64]*notifier.Alert),
state: newRuleState(),
metrics: &alertingRuleMetrics{},
}
labels := fmt.Sprintf(`alertname=%q, group=%q, id="%d"`, ar.Name, group.Name, ar.ID())
ar.metrics.pending = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_alerts_pending{%s}`, labels),
func() float64 {
ar.mu.RLock()
defer ar.mu.RUnlock()
ar.alertsMu.RLock()
defer ar.alertsMu.RUnlock()
var num int
for _, a := range ar.alerts {
if a.State == notifier.StatePending {
@ -96,8 +93,8 @@ func newAlertingRule(qb datasource.QuerierBuilder, group *Group, cfg config.Rule
})
ar.metrics.active = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_alerts_firing{%s}`, labels),
func() float64 {
ar.mu.RLock()
defer ar.mu.RUnlock()
ar.alertsMu.RLock()
defer ar.alertsMu.RUnlock()
var num int
for _, a := range ar.alerts {
if a.State == notifier.StateFiring {
@ -108,18 +105,16 @@ func newAlertingRule(qb datasource.QuerierBuilder, group *Group, cfg config.Rule
})
ar.metrics.errors = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_alerting_rules_error{%s}`, labels),
func() float64 {
ar.mu.RLock()
defer ar.mu.RUnlock()
if ar.lastExecError == nil {
e := ar.state.getLast()
if e.err == nil {
return 0
}
return 1
})
ar.metrics.samples = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_alerting_rules_last_evaluation_samples{%s}`, labels),
func() float64 {
ar.mu.RLock()
defer ar.mu.RUnlock()
return float64(ar.lastExecSamples)
e := ar.state.getLast()
return float64(e.samples)
})
return ar
}
@ -143,6 +138,32 @@ func (ar *AlertingRule) ID() uint64 {
return ar.RuleID
}
func (ar *AlertingRule) logDebugf(at time.Time, a *notifier.Alert, format string, args ...interface{}) {
if !ar.Debug {
return
}
prefix := fmt.Sprintf("DEBUG rule %q:%q (%d) at %v: ",
ar.GroupName, ar.Name, ar.RuleID, at.Format(time.RFC3339))
if a != nil {
labelKeys := make([]string, len(a.Labels))
var i int
for k := range a.Labels {
labelKeys[i] = k
i++
}
sort.Strings(labelKeys)
labels := make([]string, len(labelKeys))
for i, l := range labelKeys {
labels[i] = fmt.Sprintf("%s=%q", l, a.Labels[l])
}
labelsStr := strings.Join(labels, ",")
prefix += fmt.Sprintf("alert %d {%s} ", a.ID, labelsStr)
}
msg := fmt.Sprintf(format, args...)
logger.Infof("%s", prefix+msg)
}
type labelSet struct {
// origin labels from series
// used for templating
@ -244,20 +265,31 @@ const resolvedRetention = 15 * time.Minute
func (ar *AlertingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]prompbmarshal.TimeSeries, error) {
start := time.Now()
qMetrics, err := ar.q.Query(ctx, ar.Expr, ts)
ar.mu.Lock()
defer ar.mu.Unlock()
curState := ruleStateEntry{
time: start,
at: ts,
duration: time.Since(start),
samples: len(qMetrics),
err: err,
}
defer func() {
ar.state.add(curState)
}()
ar.alertsMu.Lock()
defer ar.alertsMu.Unlock()
ar.lastExecTime = start
ar.lastExecDuration = time.Since(start)
ar.lastExecError = err
ar.lastExecSamples = len(qMetrics)
if err != nil {
return nil, fmt.Errorf("failed to execute query %q: %w", ar.Expr, err)
}
ar.logDebugf(ts, nil, "query returned %d samples (elapsed: %s)", curState.samples, curState.duration)
for h, a := range ar.alerts {
// cleanup inactive alerts from previous Exec
if a.State == notifier.StateInactive && ts.Sub(a.ResolvedAt) > resolvedRetention {
ar.logDebugf(ts, a, "deleted as inactive")
delete(ar.alerts, h)
}
}
@ -268,14 +300,15 @@ func (ar *AlertingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]pr
for _, m := range qMetrics {
ls, err := ar.toLabels(m, qFn)
if err != nil {
return nil, fmt.Errorf("failed to expand labels: %s", err)
curState.err = fmt.Errorf("failed to expand labels: %s", err)
return nil, curState.err
}
h := hash(ls.processed)
if _, ok := updated[h]; ok {
// duplicate may be caused by extra labels
// conflicting with the metric labels
ar.lastExecError = fmt.Errorf("labels %v: %w", ls.processed, errDuplicate)
return nil, ar.lastExecError
curState.err = fmt.Errorf("labels %v: %w", ls.processed, errDuplicate)
return nil, curState.err
}
updated[h] = struct{}{}
if a, ok := ar.alerts[h]; ok {
@ -285,28 +318,31 @@ func (ar *AlertingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]pr
// back to notifier.StatePending
a.State = notifier.StatePending
a.ActiveAt = ts
ar.logDebugf(ts, a, "INACTIVE => PENDING")
}
if a.Value != m.Values[0] {
// update Value field with latest value
// update Value field with the latest value
a.Value = m.Values[0]
// and re-exec template since Value can be used
// in annotations
a.Annotations, err = a.ExecTemplate(qFn, ls.origin, ar.Annotations)
if err != nil {
return nil, err
curState.err = err
return nil, curState.err
}
}
continue
}
a, err := ar.newAlert(m, ls, ar.lastExecTime, qFn)
a, err := ar.newAlert(m, ls, start, qFn)
if err != nil {
ar.lastExecError = err
return nil, fmt.Errorf("failed to create alert: %w", err)
curState.err = fmt.Errorf("failed to create alert: %w", err)
return nil, curState.err
}
a.ID = h
a.State = notifier.StatePending
a.ActiveAt = ts
ar.alerts[h] = a
ar.logDebugf(ts, a, "created in state PENDING")
}
var numActivePending int
for h, a := range ar.alerts {
@ -317,11 +353,13 @@ func (ar *AlertingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]pr
// alert was in Pending state - it is not
// active anymore
delete(ar.alerts, h)
ar.logDebugf(ts, a, "PENDING => DELETED: is absent in current evaluation round")
continue
}
if a.State == notifier.StateFiring {
a.State = notifier.StateInactive
a.ResolvedAt = ts
ar.logDebugf(ts, a, "FIRING => INACTIVE: is absent in current evaluation round")
}
continue
}
@ -330,11 +368,13 @@ func (ar *AlertingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]pr
a.State = notifier.StateFiring
a.Start = ts
alertsFired.Inc()
ar.logDebugf(ts, a, "PENDING => FIRING: %s since becoming active at %v", ts.Sub(a.ActiveAt), a.ActiveAt)
}
}
if limit > 0 && numActivePending > limit {
ar.alerts = map[uint64]*notifier.Alert{}
return nil, fmt.Errorf("exec exceeded limit of %d with %d alerts", limit, numActivePending)
curState.err = fmt.Errorf("exec exceeded limit of %d with %d alerts", limit, numActivePending)
return nil, curState.err
}
return ar.toTimeSeries(ts.Unix()), nil
}
@ -411,8 +451,8 @@ func (ar *AlertingRule) newAlert(m datasource.Metric, ls *labelSet, start time.T
// AlertAPI generates APIAlert object from alert by its id(hash)
func (ar *AlertingRule) AlertAPI(id uint64) *APIAlert {
ar.mu.RLock()
defer ar.mu.RUnlock()
ar.alertsMu.RLock()
defer ar.alertsMu.RUnlock()
a, ok := ar.alerts[id]
if !ok {
return nil
@ -420,9 +460,10 @@ func (ar *AlertingRule) AlertAPI(id uint64) *APIAlert {
return ar.newAlertAPI(*a)
}
// ToAPI returns Rule representation in form
// of APIRule
// ToAPI returns Rule representation in form of APIRule
// Isn't thread-safe. Call must be protected by AlertingRule mutex.
func (ar *AlertingRule) ToAPI() APIRule {
lastState := ar.state.getLast()
r := APIRule{
Type: "alerting",
DatasourceType: ar.Type.String(),
@ -431,19 +472,20 @@ func (ar *AlertingRule) ToAPI() APIRule {
Duration: ar.For.Seconds(),
Labels: ar.Labels,
Annotations: ar.Annotations,
LastEvaluation: ar.lastExecTime,
EvaluationTime: ar.lastExecDuration.Seconds(),
LastEvaluation: lastState.time,
EvaluationTime: lastState.duration.Seconds(),
Health: "ok",
State: "inactive",
Alerts: ar.AlertsToAPI(),
LastSamples: ar.lastExecSamples,
LastSamples: lastState.samples,
Updates: ar.state.getAll(),
// encode as strings to avoid rounding in JSON
ID: fmt.Sprintf("%d", ar.ID()),
GroupID: fmt.Sprintf("%d", ar.GroupID),
}
if ar.lastExecError != nil {
r.LastError = ar.lastExecError.Error()
if lastState.err != nil {
r.LastError = lastState.err.Error()
r.Health = "err"
}
// satisfy APIRule.State logic
@ -463,14 +505,14 @@ func (ar *AlertingRule) ToAPI() APIRule {
// AlertsToAPI generates list of APIAlert objects from existing alerts
func (ar *AlertingRule) AlertsToAPI() []*APIAlert {
var alerts []*APIAlert
ar.mu.RLock()
ar.alertsMu.RLock()
for _, a := range ar.alerts {
if a.State == notifier.StateInactive {
continue
}
alerts = append(alerts, ar.newAlertAPI(*a))
}
ar.mu.RUnlock()
ar.alertsMu.RUnlock()
return alerts
}

View file

@ -735,6 +735,7 @@ func TestAlertingRule_Template(t *testing.T) {
"description": `{{ $labels.alertname}}: It is {{ $value }} connections for "{{ $labels.instance }}"`,
},
alerts: make(map[uint64]*notifier.Alert),
state: newRuleState(),
},
[]datasource.Metric{
metricWithValueAndLabels(t, 2, "__name__", "first", "instance", "foo", alertNameLabel, "override"),
@ -774,6 +775,7 @@ func TestAlertingRule_Template(t *testing.T) {
"summary": `Alert "{{ $labels.alertname }}({{ $labels.alertgroup }})" for instance {{ $labels.instance }}`,
},
alerts: make(map[uint64]*notifier.Alert),
state: newRuleState(),
},
[]datasource.Metric{
metricWithValueAndLabels(t, 1,
@ -915,5 +917,11 @@ func newTestRuleWithLabels(name string, labels ...string) *AlertingRule {
}
func newTestAlertingRule(name string, waitFor time.Duration) *AlertingRule {
return &AlertingRule{Name: name, alerts: make(map[uint64]*notifier.Alert), For: waitFor, EvalInterval: waitFor}
return &AlertingRule{
Name: name,
For: waitFor,
EvalInterval: waitFor,
alerts: make(map[uint64]*notifier.Alert),
state: newRuleState(),
}
}

View file

@ -113,6 +113,7 @@ type Rule struct {
For *promutils.Duration `yaml:"for,omitempty"`
Labels map[string]string `yaml:"labels,omitempty"`
Annotations map[string]string `yaml:"annotations,omitempty"`
Debug bool `yaml:"debug,omitempty"`
// Catches all undefined fields and must be empty after parsing.
XXX map[string]interface{} `yaml:",inline"`

View file

@ -535,6 +535,21 @@ headers:
rules:
- alert: foo
expr: sum by(job) (up == 1)
`)
})
t.Run("`debug` change", func(t *testing.T) {
f(t, `
name: TestGroup
rules:
- alert: foo
expr: sum by(job) (up == 1)
`, `
name: TestGroup
rules:
- alert: foo
expr: sum by(job) (up == 1)
debug: true
`)
})
}

View file

@ -1,6 +1,6 @@
groups:
- name: TestGroup
interval: 2s
interval: 5s
concurrency: 2
limit: 1000
headers:
@ -11,6 +11,7 @@ groups:
- alert: Conns
expr: sum(vm_tcplistener_conns) by(instance) > 1
for: 3m
debug: true
annotations:
summary: Too high connection number for {{$labels.instance}}
{{ with printf "sum(vm_tcplistener_conns{instance=%q})" .Labels.instance | query }}

View file

@ -23,6 +23,7 @@ type QuerierParams struct {
EvaluationInterval time.Duration
QueryParams url.Values
Headers map[string]string
Debug bool
}
// Metric is the basic entity which should be return by datasource

View file

@ -9,6 +9,7 @@ import (
"strings"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
)
@ -39,6 +40,10 @@ type VMStorage struct {
evaluationInterval time.Duration
extraParams url.Values
extraHeaders []keyValue
// whether to print additional log messages
// for each sent request
debug bool
}
type keyValue struct {
@ -64,6 +69,7 @@ func (s *VMStorage) ApplyParams(params QuerierParams) *VMStorage {
s.dataSourceType = toDatasourceType(params.DataSourceType)
s.evaluationInterval = params.EvaluationInterval
s.extraParams = params.QueryParams
s.debug = params.Debug
if params.Headers != nil {
for key, value := range params.Headers {
kv := keyValue{key: key, value: value}
@ -151,6 +157,9 @@ func (s *VMStorage) QueryRange(ctx context.Context, query string, start, end tim
}
func (s *VMStorage) do(ctx context.Context, req *http.Request) (*http.Response, error) {
if s.debug {
logger.Infof("DEBUG datasource request: executing %s request with params %q", req.Method, req.URL.RawQuery)
}
resp, err := s.c.Do(req.WithContext(ctx))
if err != nil {
return nil, fmt.Errorf("error getting response from %s: %w", req.URL.Redacted(), err)

View file

@ -30,6 +30,23 @@ type manager struct {
groups map[uint64]*Group
}
// RuleAPI generates APIRule object from alert by its ID(hash)
func (m *manager) RuleAPI(gID, rID uint64) (APIRule, error) {
m.groupsMu.RLock()
defer m.groupsMu.RUnlock()
g, ok := m.groups[gID]
if !ok {
return APIRule{}, fmt.Errorf("can't find group with id %d", gID)
}
for _, rule := range g.Rules {
if rule.ID() == rID {
return rule.ToAPI(), nil
}
}
return APIRule{}, fmt.Errorf("can't find rule with id %d in group %q", rID, g.Name)
}
// AlertAPI generates APIAlert object from alert by its ID(hash)
func (m *manager) AlertAPI(gID, aID uint64) (*APIAlert, error) {
m.groupsMu.RLock()
@ -70,9 +87,9 @@ func (m *manager) startGroup(ctx context.Context, group *Group, restore bool) er
err := group.Restore(ctx, m.rr, *remoteReadLookBack, m.labels)
if err != nil {
if !*remoteReadIgnoreRestoreErrors {
return fmt.Errorf("failed to restore state for group %q: %w", group.Name, err)
return fmt.Errorf("failed to restore ruleState for group %q: %w", group.Name, err)
}
logger.Errorf("error while restoring state for group %q: %s", group.Name, err)
logger.Errorf("error while restoring ruleState for group %q: %s", group.Name, err)
}
}

View file

@ -5,7 +5,6 @@ import (
"fmt"
"sort"
"strings"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmalert/config"
@ -27,19 +26,9 @@ type RecordingRule struct {
q datasource.Querier
// guard status fields
mu sync.RWMutex
// stores last moment of time Exec was called
lastExecTime time.Time
// stores the duration of the last Exec call
lastExecDuration time.Duration
// stores last error that happened in Exec func
// resets on every successful Exec
// may be used as Health state
lastExecError error
// stores the number of samples returned during
// the last evaluation
lastExecSamples int
// state stores recent state changes
// during evaluations
state *ruleState
metrics *recordingRuleMetrics
}
@ -69,6 +58,7 @@ func newRecordingRule(qb datasource.QuerierBuilder, group *Group, cfg config.Rul
Labels: cfg.Labels,
GroupID: group.ID(),
metrics: &recordingRuleMetrics{},
state: newRuleState(),
q: qb.BuildWithParams(datasource.QuerierParams{
DataSourceType: group.Type.String(),
EvaluationInterval: group.Interval,
@ -80,18 +70,16 @@ func newRecordingRule(qb datasource.QuerierBuilder, group *Group, cfg config.Rul
labels := fmt.Sprintf(`recording=%q, group=%q, id="%d"`, rr.Name, group.Name, rr.ID())
rr.metrics.errors = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_recording_rules_error{%s}`, labels),
func() float64 {
rr.mu.RLock()
defer rr.mu.RUnlock()
if rr.lastExecError == nil {
e := rr.state.getLast()
if e.err == nil {
return 0
}
return 1
})
rr.metrics.samples = utils.GetOrCreateGauge(fmt.Sprintf(`vmalert_recording_rules_last_evaluation_samples{%s}`, labels),
func() float64 {
rr.mu.RLock()
defer rr.mu.RUnlock()
return float64(rr.lastExecSamples)
e := rr.state.getLast()
return float64(e.samples)
})
return rr
}
@ -126,21 +114,28 @@ func (rr *RecordingRule) ExecRange(ctx context.Context, start, end time.Time) ([
// Exec executes RecordingRule expression via the given Querier.
func (rr *RecordingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]prompbmarshal.TimeSeries, error) {
start := time.Now()
qMetrics, err := rr.q.Query(ctx, rr.Expr, ts)
rr.mu.Lock()
defer rr.mu.Unlock()
curState := ruleStateEntry{
time: start,
at: ts,
duration: time.Since(start),
samples: len(qMetrics),
}
defer func() {
rr.state.add(curState)
}()
rr.lastExecTime = ts
rr.lastExecDuration = time.Since(ts)
rr.lastExecError = err
rr.lastExecSamples = len(qMetrics)
if err != nil {
return nil, fmt.Errorf("failed to execute query %q: %w", rr.Expr, err)
curState.err = fmt.Errorf("failed to execute query %q: %w", rr.Expr, err)
return nil, curState.err
}
numSeries := len(qMetrics)
if limit > 0 && numSeries > limit {
return nil, fmt.Errorf("exec exceeded limit of %d with %d series", limit, numSeries)
curState.err = fmt.Errorf("exec exceeded limit of %d with %d series", limit, numSeries)
return nil, curState.err
}
duplicates := make(map[string]struct{}, len(qMetrics))
@ -149,8 +144,8 @@ func (rr *RecordingRule) Exec(ctx context.Context, ts time.Time, limit int) ([]p
ts := rr.toTimeSeries(r)
key := stringifyLabels(ts)
if _, ok := duplicates[key]; ok {
rr.lastExecError = errDuplicate
return nil, fmt.Errorf("original metric %v; resulting labels %q: %w", r, key, errDuplicate)
curState.err = fmt.Errorf("original metric %v; resulting labels %q: %w", r, key, errDuplicate)
return nil, curState.err
}
duplicates[key] = struct{}{}
tss = append(tss, ts)
@ -205,23 +200,25 @@ func (rr *RecordingRule) UpdateWith(r Rule) error {
// ToAPI returns Rule's representation in form
// of APIRule
func (rr *RecordingRule) ToAPI() APIRule {
lastState := rr.state.getLast()
r := APIRule{
Type: "recording",
DatasourceType: rr.Type.String(),
Name: rr.Name,
Query: rr.Expr,
Labels: rr.Labels,
LastEvaluation: rr.lastExecTime,
EvaluationTime: rr.lastExecDuration.Seconds(),
LastEvaluation: lastState.time,
EvaluationTime: lastState.duration.Seconds(),
Health: "ok",
LastSamples: rr.lastExecSamples,
LastSamples: lastState.samples,
Updates: rr.state.getAll(),
// encode as strings to avoid rounding
ID: fmt.Sprintf("%d", rr.ID()),
GroupID: fmt.Sprintf("%d", rr.GroupID),
}
if rr.lastExecError != nil {
r.LastError = rr.lastExecError.Error()
if lastState.err != nil {
r.LastError = lastState.err.Error()
r.Health = "err"
}
return r

View file

@ -19,7 +19,7 @@ func TestRecordingRule_Exec(t *testing.T) {
expTS []prompbmarshal.TimeSeries
}{
{
&RecordingRule{Name: "foo"},
&RecordingRule{Name: "foo", state: newRuleState()},
[]datasource.Metric{metricWithValueAndLabels(t, 10,
"__name__", "bar",
)},
@ -30,7 +30,7 @@ func TestRecordingRule_Exec(t *testing.T) {
},
},
{
&RecordingRule{Name: "foobarbaz"},
&RecordingRule{Name: "foobarbaz", state: newRuleState()},
[]datasource.Metric{
metricWithValueAndLabels(t, 1, "__name__", "foo", "job", "foo"),
metricWithValueAndLabels(t, 2, "__name__", "bar", "job", "bar"),
@ -52,7 +52,10 @@ func TestRecordingRule_Exec(t *testing.T) {
},
},
{
&RecordingRule{Name: "job:foo", Labels: map[string]string{
&RecordingRule{
Name: "job:foo",
state: newRuleState(),
Labels: map[string]string{
"source": "test",
}},
[]datasource.Metric{
@ -195,7 +198,7 @@ func TestRecordingRuleLimit(t *testing.T) {
metricWithValuesAndLabels(t, []float64{2, 3}, "__name__", "bar", "job", "bar"),
metricWithValuesAndLabels(t, []float64{4, 5, 6}, "__name__", "baz", "job", "baz"),
}
rule := &RecordingRule{Name: "job:foo", Labels: map[string]string{
rule := &RecordingRule{Name: "job:foo", state: newRuleState(), Labels: map[string]string{
"source": "test_limit",
}}
var err error
@ -211,9 +214,13 @@ func TestRecordingRuleLimit(t *testing.T) {
}
func TestRecordingRule_ExecNegative(t *testing.T) {
rr := &RecordingRule{Name: "job:foo", Labels: map[string]string{
rr := &RecordingRule{
Name: "job:foo",
state: newRuleState(),
Labels: map[string]string{
"job": "test",
}}
},
}
fq := &fakeQuerier{}
expErr := "connection reset by peer"

View file

@ -3,6 +3,7 @@ package main
import (
"context"
"errors"
"sync"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
@ -31,3 +32,72 @@ type Rule interface {
}
var errDuplicate = errors.New("result contains metrics with the same labelset after applying rule labels")
type ruleState struct {
sync.RWMutex
entries []ruleStateEntry
cur int
}
type ruleStateEntry struct {
// stores last moment of time rule.Exec was called
time time.Time
// stores the timesteamp with which rule.Exec was called
at time.Time
// stores the duration of the last rule.Exec call
duration time.Duration
// stores last error that happened in Exec func
// resets on every successful Exec
// may be used as Health ruleState
err error
// stores the number of samples returned during
// the last evaluation
samples int
}
const defaultStateEntriesLimit = 20
func newRuleState() *ruleState {
return &ruleState{
entries: make([]ruleStateEntry, defaultStateEntriesLimit),
}
}
func (s *ruleState) getLast() ruleStateEntry {
s.RLock()
defer s.RUnlock()
return s.entries[s.cur]
}
func (s *ruleState) getAll() []ruleStateEntry {
entries := make([]ruleStateEntry, 0)
s.RLock()
defer s.RUnlock()
cur := s.cur
for {
e := s.entries[cur]
if !e.time.IsZero() || !e.at.IsZero() {
entries = append(entries, e)
}
cur--
if cur < 0 {
cur = cap(s.entries) - 1
}
if cur == s.cur {
return entries
}
}
}
func (s *ruleState) add(e ruleStateEntry) {
s.Lock()
defer s.Unlock()
s.cur++
if s.cur > cap(s.entries)-1 {
s.cur = 0
}
s.entries[s.cur] = e
}

81
app/vmalert/rule_test.go Normal file
View file

@ -0,0 +1,81 @@
package main
import (
"sync"
"testing"
"time"
)
func TestRule_state(t *testing.T) {
state := newRuleState()
e := state.getLast()
if !e.at.IsZero() {
t.Fatalf("expected entry to be zero")
}
now := time.Now()
state.add(ruleStateEntry{at: now})
e = state.getLast()
if e.at != now {
t.Fatalf("expected entry at %v to be equal to %v",
e.at, now)
}
time.Sleep(time.Millisecond)
now2 := time.Now()
state.add(ruleStateEntry{at: now2})
e = state.getLast()
if e.at != now2 {
t.Fatalf("expected entry at %v to be equal to %v",
e.at, now2)
}
if len(state.getAll()) != 2 {
t.Fatalf("expected for state to have 2 entries only; got %d",
len(state.getAll()),
)
}
var last time.Time
for i := 0; i < defaultStateEntriesLimit*2; i++ {
last = time.Now()
state.add(ruleStateEntry{at: last})
}
e = state.getLast()
if e.at != last {
t.Fatalf("expected entry at %v to be equal to %v",
e.at, last)
}
if len(state.getAll()) != defaultStateEntriesLimit {
t.Fatalf("expected for state to have %d entries only; got %d",
defaultStateEntriesLimit, len(state.getAll()),
)
}
}
// TestRule_stateConcurrent supposed to test concurrent
// execution of state updates.
// Should be executed with -race flag
func TestRule_stateConcurrent(t *testing.T) {
state := newRuleState()
const workers = 50
const iterations = 100
wg := sync.WaitGroup{}
wg.Add(workers)
for i := 0; i < workers; i++ {
go func() {
defer wg.Done()
for i := 0; i < iterations; i++ {
state.add(ruleStateEntry{at: time.Now()})
state.getAll()
state.getLast()
}
}()
}
wg.Wait()
}

View file

@ -85,6 +85,14 @@ func (rh *requestHandler) handler(w http.ResponseWriter, r *http.Request) bool {
}
WriteAlert(w, r, alert)
return true
case "/vmalert/rule":
rule, err := rh.getRule(r)
if err != nil {
httpserver.Errorf(w, r, "%s", err)
return true
}
WriteRuleDetails(w, r, rule)
return true
case "/vmalert/groups":
WriteListGroups(w, r, rh.groups())
return true
@ -168,8 +176,25 @@ func (rh *requestHandler) handler(w http.ResponseWriter, r *http.Request) bool {
const (
paramGroupID = "group_id"
paramAlertID = "alert_id"
paramRuleID = "rule_id"
)
func (rh *requestHandler) getRule(r *http.Request) (APIRule, error) {
groupID, err := strconv.ParseUint(r.FormValue(paramGroupID), 10, 0)
if err != nil {
return APIRule{}, fmt.Errorf("failed to read %q param: %s", paramGroupID, err)
}
ruleID, err := strconv.ParseUint(r.FormValue(paramRuleID), 10, 0)
if err != nil {
return APIRule{}, fmt.Errorf("failed to read %q param: %s", paramRuleID, err)
}
rule, err := rh.m.RuleAPI(groupID, ruleID)
if err != nil {
return APIRule{}, errResponse(err, http.StatusNotFound)
}
return rule, nil
}
func (rh *requestHandler) getAlert(r *http.Request) (*APIAlert, error) {
groupID, err := strconv.ParseUint(r.FormValue(paramGroupID), 10, 0)
if err != nil {

View file

@ -26,6 +26,7 @@
{% endfunc %}
{% func ListGroups(r *http.Request, groups []APIGroup) %}
{%code prefix := utils.Prefix(r.URL.Path) %}
{%= tpl.Header(r, navItems, "Groups") %}
{% if len(groups) > 0 %}
{%code
@ -85,6 +86,7 @@
{% else %}
<b>record:</b> {%s r.Name %}
{% endif %}
| <span><a target="_blank" href="{%s prefix+r.WebLink() %}">Details</a></span>
</div>
<div class="col-12">
<code><pre>{%s r.Query %}</pre></code>
@ -116,7 +118,7 @@
{% else %}
<div>
<p>No items...</p>
<p>No groups...</p>
</div>
{% endif %}
@ -204,7 +206,7 @@
{% else %}
<div>
<p>No items...</p>
<p>No active alerts...</p>
</div>
{% endif %}
@ -260,7 +262,7 @@
{% else %}
<div>
<p>No items...</p>
<p>No targets...</p>
</div>
{% endif %}
@ -284,7 +286,7 @@
}
sort.Strings(annotationKeys)
%}
<div class="display-6 pb-3 mb-3">{%s alert.Name %}<span class="ms-2 badge {% if alert.State=="firing" %}bg-danger{% else %} bg-warning text-dark{% endif %}">{%s alert.State %}</span></div>
<div class="display-6 pb-3 mb-3">Alert: {%s alert.Name %}<span class="ms-2 badge {% if alert.State=="firing" %}bg-danger{% else %} bg-warning text-dark{% endif %}">{%s alert.State %}</span></div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
@ -354,6 +356,117 @@
{% endfunc %}
{% func RuleDetails(r *http.Request, rule APIRule) %}
{%code prefix := utils.Prefix(r.URL.Path) %}
{%= tpl.Header(r, navItems, "") %}
{%code
var labelKeys []string
for k := range rule.Labels {
labelKeys = append(labelKeys, k)
}
sort.Strings(labelKeys)
var annotationKeys []string
for k := range rule.Annotations {
annotationKeys = append(annotationKeys, k)
}
sort.Strings(annotationKeys)
%}
<div class="display-6 pb-3 mb-3">Rule: {%s rule.Name %}<span class="ms-2 badge {% if rule.Health!="ok" %}bg-danger{% else %} bg-warning text-dark{% endif %}">{%s rule.Health %}</span></div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
Expr
</div>
<div class="col">
<code><pre>{%s rule.Query %}</pre></code>
</div>
</div>
</div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
For
</div>
<div class="col">
{%v rule.Duration %} seconds
</div>
</div>
</div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
Labels
</div>
<div class="col">
{% for _, k := range labelKeys %}
<span class="m-1 badge bg-primary">{%s k %}={%s rule.Labels[k] %}</span>
{% endfor %}
</div>
</div>
</div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
Annotations
</div>
<div class="col">
{% for _, k := range annotationKeys %}
<b>{%s k %}:</b><br>
<p>{%s rule.Annotations[k] %}</p>
{% endfor %}
</div>
</div>
</div>
<div class="container border-bottom p-2">
<div class="row">
<div class="col-2">
Group
</div>
<div class="col">
<a target="_blank" href="{%s prefix %}groups#group-{%s rule.GroupID %}">{%s rule.GroupID %}</a>
</div>
</div>
</div>
<br>
<div class="display-6 pb-3">Last {%d len(rule.Updates) %} updates</span>:</div>
<table class="table table-striped table-hover table-sm">
<thead>
<tr>
<th scope="col" style="width: 20%" title="The time when event was created">Updated at</th>
<th scope="col" style="width: 20%" class="text-center" title="How many samples were returned">Samples</th>
<th scope="col" style="width: 20%" class="text-center" title="How many seconds request took">Duration</th>
<th scope="col" style="width: 20%" class="text-center" title="Time used for rule execution">Executed at</th>
</tr>
</thead>
<tbody>
{% for _, u := range rule.Updates %}
<tr{% if u.err != nil %} class="alert-danger"{% endif %}>
<td>
<span class="badge bg-primary rounded-pill me-3" title="Updated at">{%s u.time.Format(time.RFC3339) %}</span>
</td>
<td class="text-center">{%d u.samples %}</td>
<td class="text-center">{%f.3 u.duration.Seconds() %}s</td>
<td class="text-center">{%s u.at.Format(time.RFC3339) %}</td>
</tr>
</li>
{% if u.err != nil %}
<tr{% if u.err != nil %} class="alert-danger"{% endif %}>
<td colspan="4">
<span class="alert-danger">{%v u.err %}</span>
</td>
</tr>
{% endif %}
{% endfor %}
{%= tpl.Footer(r) %}
{% endfunc %}
{% func badgeState(state string) %}
{%code
badgeClass := "bg-warning text-dark"

File diff suppressed because it is too large Load diff

View file

@ -17,6 +17,7 @@ func TestHandler(t *testing.T) {
alerts: map[uint64]*notifier.Alert{
0: {State: notifier.StateFiring},
},
state: newRuleState(),
}
g := &Group{
Name: "group",
@ -52,6 +53,22 @@ func TestHandler(t *testing.T) {
t.Run("/", func(t *testing.T) {
getResp(ts.URL, nil, 200)
getResp(ts.URL+"/vmalert", nil, 200)
getResp(ts.URL+"/vmalert/alerts", nil, 200)
getResp(ts.URL+"/vmalert/groups", nil, 200)
getResp(ts.URL+"/vmalert/notifiers", nil, 200)
getResp(ts.URL+"/rules", nil, 200)
})
t.Run("/vmalert/rule", func(t *testing.T) {
a := ar.ToAPI()
getResp(ts.URL+"/vmalert/"+a.WebLink(), nil, 200)
})
t.Run("/vmalert/rule?badParam", func(t *testing.T) {
params := fmt.Sprintf("?%s=0&%s=1", paramGroupID, paramRuleID)
getResp(ts.URL+"/vmalert/rule"+params, nil, 404)
params = fmt.Sprintf("?%s=1&%s=0", paramGroupID, paramRuleID)
getResp(ts.URL+"/vmalert/rule"+params, nil, 404)
})
t.Run("/api/v1/alerts", func(t *testing.T) {

View file

@ -5,11 +5,11 @@ import (
"time"
)
// APIAlert represents a notifier.AlertingRule state
// APIAlert represents a notifier.AlertingRule ruleState
// for WEB view
// https://github.com/prometheus/compliance/blob/main/alert_generator/specification.md#get-apiv1rules
type APIAlert struct {
State string `json:"state"`
State string `json:"ruleState"`
Name string `json:"name"`
Value string `json:"value"`
Labels map[string]string `json:"labels,omitempty"`
@ -30,7 +30,7 @@ type APIAlert struct {
// SourceLink contains a link to a system which should show
// why Alert was generated
SourceLink string `json:"source"`
// Restored shows whether Alert's state was restored on restart
// Restored shows whether Alert's ruleState was restored on restart
Restored bool `json:"restored"`
}
@ -86,10 +86,10 @@ type GroupAlerts struct {
// see https://github.com/prometheus/compliance/blob/main/alert_generator/specification.md#get-apiv1rules
type APIRule struct {
// State must be one of these under following scenarios
// "pending": at least 1 alert in the rule in pending state and no other alert in firing state.
// "firing": at least 1 alert in the rule in firing state.
// "inactive": no alert in the rule in firing or pending state.
State string `json:"state"`
// "pending": at least 1 alert in the rule in pending ruleState and no other alert in firing ruleState.
// "firing": at least 1 alert in the rule in firing ruleState.
// "inactive": no alert in the rule in firing or pending ruleState.
State string `json:"ruleState"`
Name string `json:"name"`
// Query represents Rule's `expression` field
Query string `json:"query"`
@ -116,8 +116,17 @@ type APIRule struct {
// Type of the rule: recording or alerting
DatasourceType string `json:"datasourceType"`
LastSamples int `json:"lastSamples"`
// ID is an unique Alert's ID within a group
// ID is a unique Alert's ID within a group
ID string `json:"id"`
// GroupID is an unique Group's ID
GroupID string `json:"group_id"`
// TODO:
Updates []ruleStateEntry `json:"updates"`
}
// WebLink returns a link to the alert which can be used in UI.
func (ar APIRule) WebLink() string {
return fmt.Sprintf("rule?%s=%s&%s=%s",
paramGroupID, ar.GroupID, paramRuleID, ar.ID)
}

View file

@ -167,7 +167,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmauth` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmauth` binary and puts it into the `bin` folder.

View file

@ -276,7 +276,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmbackup` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmbackup` binary and puts it into the `bin` folder.

View file

@ -699,7 +699,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmctl` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmctl` binary and puts it into the `bin` folder.
@ -728,7 +728,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
#### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmctl-linux-arm` or `make vmctl-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmctl-linux-arm` or `vmctl-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -176,7 +176,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmrestore` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmrestore` binary and puts it into the `bin` folder.

View file

@ -99,7 +99,8 @@ func maySortResults(e metricsql.Expr, tss []*timeseries) bool {
case *metricsql.FuncExpr:
switch strings.ToLower(v.Name) {
case "sort", "sort_desc",
"sort_by_label", "sort_by_label_desc":
"sort_by_label", "sort_by_label_desc",
"sort_by_label_numeric", "sort_by_label_numeric_desc":
return false
}
case *metricsql.AggrFuncExpr:

View file

@ -7738,6 +7738,178 @@ func TestExecSuccess(t *testing.T) {
resultExpected := []netstorage.Result{r1, r2, r3, r4}
f(q, resultExpected)
})
t.Run(`sort_by_label_numeric(multiple_labels_only_string)`, func(t *testing.T) {
t.Parallel()
q := `sort_by_label_numeric((
label_set(1, "x", "b", "y", "aa"),
label_set(2, "x", "a", "y", "aa"),
), "y", "x")`
r1 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{2, 2, 2, 2, 2, 2},
Timestamps: timestampsExpected,
}
r1.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("a"),
},
{
Key: []byte("y"),
Value: []byte("aa"),
},
}
r2 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{1, 1, 1, 1, 1, 1},
Timestamps: timestampsExpected,
}
r2.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("b"),
},
{
Key: []byte("y"),
Value: []byte("aa"),
},
}
resultExpected := []netstorage.Result{r1, r2}
f(q, resultExpected)
})
t.Run(`sort_by_label_numeric(multiple_labels_numbers_special_chars)`, func(t *testing.T) {
t.Parallel()
q := `sort_by_label_numeric((
label_set(1, "x", "1:0:2", "y", "1:0:1"),
label_set(2, "x", "1:0:15", "y", "1:0:1"),
), "x", "y")`
r1 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{1, 1, 1, 1, 1, 1},
Timestamps: timestampsExpected,
}
r1.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("1:0:2"),
},
{
Key: []byte("y"),
Value: []byte("1:0:1"),
},
}
r2 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{2, 2, 2, 2, 2, 2},
Timestamps: timestampsExpected,
}
r2.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("1:0:15"),
},
{
Key: []byte("y"),
Value: []byte("1:0:1"),
},
}
resultExpected := []netstorage.Result{r1, r2}
f(q, resultExpected)
})
t.Run(`sort_by_label_numeric_desc(multiple_labels_numbers_special_chars)`, func(t *testing.T) {
t.Parallel()
q := `sort_by_label_numeric_desc((
label_set(1, "x", "1:0:2", "y", "1:0:1"),
label_set(2, "x", "1:0:15", "y", "1:0:1"),
), "x", "y")`
r1 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{2, 2, 2, 2, 2, 2},
Timestamps: timestampsExpected,
}
r1.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("1:0:15"),
},
{
Key: []byte("y"),
Value: []byte("1:0:1"),
},
}
r2 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{1, 1, 1, 1, 1, 1},
Timestamps: timestampsExpected,
}
r2.MetricName.Tags = []storage.Tag{
{
Key: []byte("x"),
Value: []byte("1:0:2"),
},
{
Key: []byte("y"),
Value: []byte("1:0:1"),
},
}
resultExpected := []netstorage.Result{r1, r2}
f(q, resultExpected)
})
t.Run(`sort_by_label_numeric(alias_numbers_with_special_chars)`, func(t *testing.T) {
t.Parallel()
q := `sort_by_label_numeric((
label_set(4, "a", "DS50:1/0/15"),
label_set(1, "a", "DS50:1/0/0"),
label_set(2, "a", "DS50:1/0/1"),
label_set(3, "a", "DS50:1/0/2"),
), "a")`
r1 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{1, 1, 1, 1, 1, 1},
Timestamps: timestampsExpected,
}
r1.MetricName.Tags = []storage.Tag{
{
Key: []byte("a"),
Value: []byte("DS50:1/0/0"),
},
}
r2 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{2, 2, 2, 2, 2, 2},
Timestamps: timestampsExpected,
}
r2.MetricName.Tags = []storage.Tag{
{
Key: []byte("a"),
Value: []byte("DS50:1/0/1"),
},
}
r3 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{3, 3, 3, 3, 3, 3},
Timestamps: timestampsExpected,
}
r3.MetricName.Tags = []storage.Tag{
{
Key: []byte("a"),
Value: []byte("DS50:1/0/2"),
},
}
r4 := netstorage.Result{
MetricName: metricNameExpected,
Values: []float64{4, 4, 4, 4, 4, 4},
Timestamps: timestampsExpected,
}
r4.MetricName.Tags = []storage.Tag{
{
Key: []byte("a"),
Value: []byte("DS50:1/0/15"),
},
}
resultExpected := []netstorage.Result{r1, r2, r3, r4}
f(q, resultExpected)
})
}
func TestExecError(t *testing.T) {
@ -7811,6 +7983,8 @@ func TestExecError(t *testing.T) {
f(`sort_desc()`)
f(`sort_by_label()`)
f(`sort_by_label_desc()`)
f(`sort_by_label_numeric()`)
f(`sort_by_label_numeric_desc()`)
f(`timestamp()`)
f(`timestamp_with_name()`)
f(`vector()`)
@ -7933,6 +8107,7 @@ func TestExecError(t *testing.T) {
f(`round(1, 1 or label_set(2, "xx", "foo"))`)
f(`histogram_quantile(1 or label_set(2, "xx", "foo"), 1)`)
f(`histogram_quantiles("foo", 1 or label_set(2, "xxx", "foo"), 2)`)
f(`sort_by_label_numeric(1, 2)`)
f(`label_set(1, 2, 3)`)
f(`label_set(1, "foo", (label_set(1, "foo", bar") or label_set(2, "xxx", "yy")))`)
f(`label_set(1, "foo", 3)`)

View file

@ -14,6 +14,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/searchutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/decimal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/storage"
"github.com/VictoriaMetrics/metricsql"
)
@ -105,6 +106,8 @@ var transformFuncs = map[string]transformFunc{
"sort": newTransformFuncSort(false),
"sort_by_label": newTransformFuncSortByLabel(false),
"sort_by_label_desc": newTransformFuncSortByLabel(true),
"sort_by_label_numeric": newTransformFuncNumericSort(false),
"sort_by_label_numeric_desc": newTransformFuncNumericSort(true),
"sort_desc": newTransformFuncSort(true),
"sqrt": newTransformFuncOneArg(transformSqrt),
"start": newTransformFuncZeroArgs(transformStart),
@ -1992,6 +1995,130 @@ func newTransformFuncSortByLabel(isDesc bool) transformFunc {
}
}
func newTransformFuncNumericSort(isDesc bool) transformFunc {
return func(tfa *transformFuncArg) ([]*timeseries, error) {
args := tfa.args
if len(args) < 2 {
return nil, fmt.Errorf("expecting at least 2 args; got %d args", len(args))
}
var labels []string
for i, arg := range args[1:] {
label, err := getString(arg, i+1)
if err != nil {
return nil, fmt.Errorf("cannot parse label #%d for sorting: %w", i+1, err)
}
labels = append(labels, label)
}
rvs := args[0]
sort.SliceStable(rvs, func(i, j int) bool {
for _, label := range labels {
a := rvs[i].MetricName.GetTagValue(label)
b := rvs[j].MetricName.GetTagValue(label)
if string(a) == string(b) {
continue
}
aStr := bytesutil.ToUnsafeString(a)
bStr := bytesutil.ToUnsafeString(b)
if isDesc {
return numericLess(bStr, aStr)
}
return numericLess(aStr, bStr)
}
return false
})
return rvs, nil
}
}
func numericLess(a, b string) bool {
for {
if len(b) == 0 {
return false
}
if len(a) == 0 {
return true
}
aPrefix := getNumPrefix(a)
bPrefix := getNumPrefix(b)
a = a[len(aPrefix):]
b = b[len(bPrefix):]
if len(aPrefix) > 0 || len(bPrefix) > 0 {
if len(aPrefix) == 0 {
return false
}
if len(bPrefix) == 0 {
return true
}
aNum := mustParseNum(aPrefix)
bNum := mustParseNum(bPrefix)
if aNum != bNum {
return aNum < bNum
}
}
aPrefix = getNonNumPrefix(a)
bPrefix = getNonNumPrefix(b)
a = a[len(aPrefix):]
b = b[len(bPrefix):]
if aPrefix != bPrefix {
return aPrefix < bPrefix
}
}
}
func getNumPrefix(s string) string {
i := 0
if len(s) > 0 {
switch s[0] {
case '-', '+':
i++
}
}
hasNum := false
hasDot := false
for i < len(s) {
if !isDecimalChar(s[i]) {
if !hasDot && s[i] == '.' {
hasDot = true
i++
continue
}
if !hasNum {
return ""
}
return s[:i]
}
hasNum = true
i++
}
if !hasNum {
return ""
}
return s
}
func getNonNumPrefix(s string) string {
i := 0
for i < len(s) {
if isDecimalChar(s[i]) {
return s[:i]
}
i++
}
return s
}
func isDecimalChar(ch byte) bool {
return ch >= '0' && ch <= '9'
}
func mustParseNum(s string) float64 {
f, err := strconv.ParseFloat(s, 64)
if err != nil {
logger.Panicf("BUG: unexpected error when parsing the number %q: %s", s, err)
}
return f
}
func newTransformFuncSort(isDesc bool) transformFunc {
return func(tfa *transformFuncArg) ([]*timeseries, error) {
args := tfa.args

View file

@ -3,6 +3,7 @@ package promql
import (
"fmt"
"reflect"
"strconv"
"strings"
"testing"
@ -220,3 +221,110 @@ func timeseriesToPromMetrics(tss []*timeseries) string {
}
return strings.Join(a, "\n")
}
func TestGetNumPrefix(t *testing.T) {
f := func(s, prefixExpected string) {
t.Helper()
prefix := getNumPrefix(s)
if prefix != prefixExpected {
t.Fatalf("unexpected getNumPrefix(%q): got %q; want %q", s, prefix, prefixExpected)
}
if len(prefix) > 0 {
if _, err := strconv.ParseFloat(prefix, 64); err != nil {
t.Fatalf("cannot parse num %q: %s", prefix, err)
}
}
}
f("", "")
f("foo", "")
f("-", "")
f(".", "")
f("-.", "")
f("+..", "")
f("1", "1")
f("12", "12")
f("1foo", "1")
f("-123", "-123")
f("-123bar", "-123")
f("+123", "+123")
f("+123.", "+123.")
f("+123..", "+123.")
f("+123.-", "+123.")
f("12.34..", "12.34")
f("-12.34..", "-12.34")
f("-12.-34..", "-12.")
}
func TestNumericLess(t *testing.T) {
f := func(a, b string, want bool) {
t.Helper()
if got := numericLess(a, b); got != want {
t.Fatalf("unexpected numericLess(%q, %q): got %v; want %v", a, b, got, want)
}
}
// empty strings
f("", "", false)
f("", "321", true)
f("321", "", false)
f("", "abc", true)
f("abc", "", false)
f("foo", "123", false)
f("123", "foo", true)
// same length numbers
f("123", "321", true)
f("321", "123", false)
f("123", "123", false)
// same length strings
f("a", "b", true)
f("b", "a", false)
f("a", "a", false)
// identical string prefix
f("foo123", "foo", false)
f("foo", "foo123", true)
f("foo", "foo", false)
// identical num prefix
f("123foo", "123bar", false)
f("123bar", "123foo", true)
f("123bar", "123bar", false)
// numbers with special chars
f("1:0:0", "1:0:2", true)
// numbers with special chars and different number rank
f("1:0:15", "1:0:2", false)
// multiple zeroes"
f("0", "00", false)
// only chars
f("aa", "ab", true)
// strings with different lengths
f("ab", "abc", true)
// multiple zeroes after equal char
f("a0001", "a0000001", false)
// short first string with numbers and highest rank
f("a10", "abcdefgh2", true)
// less as second string
f("a1b", "a01b", false)
// equal strings by length with different number rank
f("a001b01", "a01b001", false)
// different numbers rank
f("a01b001", "a001b01", false)
// different numbers rank
f("a01b001", "a001b01", false)
// highest char and number
f("a1", "a1x", true)
// highest number reverse chars
f("1b", "1ax", false)
// numbers with leading zero
f("082", "83", true)
// numbers with leading zero and chars
f("083a", "9a", false)
f("083a", "94a", true)
// negative number
f("-123", "123", true)
f("-123", "+123", true)
f("-123", "-123", false)
f("123", "-123", false)
// fractional number
f("12.9", "12.56", false)
f("12.56", "12.9", true)
f("12.9", "12.9", false)
}

View file

@ -1,12 +1,12 @@
{
"files": {
"main.css": "./static/css/main.9b22c3e0.css",
"main.js": "./static/js/main.b8df40e9.js",
"main.js": "./static/js/main.8f1a3d7a.js",
"static/js/27.939f971b.chunk.js": "./static/js/27.939f971b.chunk.js",
"index.html": "./index.html"
},
"entrypoints": [
"static/css/main.9b22c3e0.css",
"static/js/main.b8df40e9.js"
"static/js/main.8f1a3d7a.js"
]
}

View file

@ -1 +1 @@
<!doctype html><html lang="en"><head><meta charset="utf-8"/><link rel="icon" href="./favicon.ico"/><meta name="viewport" content="width=device-width,initial-scale=1"/><meta name="theme-color" content="#000000"/><meta name="description" content="VM-UI is a metric explorer for Victoria Metrics"/><link rel="apple-touch-icon" href="./apple-touch-icon.png"/><link rel="icon" type="image/png" sizes="32x32" href="./favicon-32x32.png"><link rel="manifest" href="./manifest.json"/><title>VM UI</title><link rel="stylesheet" href="https://fonts.googleapis.com/css?family=Roboto:300,400,500,700&display=swap"/><script src="./dashboards/index.js" type="module"></script><script defer="defer" src="./static/js/main.b8df40e9.js"></script><link href="./static/css/main.9b22c3e0.css" rel="stylesheet"></head><body><noscript>You need to enable JavaScript to run this app.</noscript><div id="root"></div></body></html>
<!doctype html><html lang="en"><head><meta charset="utf-8"/><link rel="icon" href="./favicon.ico"/><meta name="viewport" content="width=device-width,initial-scale=1"/><meta name="theme-color" content="#000000"/><meta name="description" content="VM-UI is a metric explorer for Victoria Metrics"/><link rel="apple-touch-icon" href="./apple-touch-icon.png"/><link rel="icon" type="image/png" sizes="32x32" href="./favicon-32x32.png"><link rel="manifest" href="./manifest.json"/><title>VM UI</title><link rel="stylesheet" href="https://fonts.googleapis.com/css?family=Roboto:300,400,500,700&display=swap"/><script src="./dashboards/index.js" type="module"></script><script defer="defer" src="./static/js/main.8f1a3d7a.js"></script><link href="./static/css/main.9b22c3e0.css" rel="stylesheet"></head><body><noscript>You need to enable JavaScript to run this app.</noscript><div id="root"></div></body></html>

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View file

@ -5,6 +5,7 @@ import {StateProvider} from "./state/common/StateContext";
import {AuthStateProvider} from "./state/auth/AuthStateContext";
import {GraphStateProvider} from "./state/graph/GraphStateContext";
import {CardinalityStateProvider} from "./state/cardinality/CardinalityStateContext";
import {TopQueriesStateProvider} from "./state/topQueries/TopQueriesStateContext";
import THEME from "./theme/theme";
import { ThemeProvider, StyledEngineProvider } from "@mui/material/styles";
import CssBaseline from "@mui/material/CssBaseline";
@ -16,6 +17,7 @@ import CustomPanel from "./components/CustomPanel/CustomPanel";
import HomeLayout from "./components/Home/HomeLayout";
import DashboardsLayout from "./components/PredefinedPanels/DashboardsLayout";
import CardinalityPanel from "./components/CardinalityPanel/CardinalityPanel";
import TopQueries from "./components/TopQueries/TopQueries";
const App: FC = () => {
@ -30,15 +32,18 @@ const App: FC = () => {
<AuthStateProvider> {/* Auth related info - optionally persisted to Local Storage */}
<GraphStateProvider> {/* Graph settings */}
<CardinalityStateProvider> {/* Cardinality settings */}
<TopQueriesStateProvider> {/* Top Queries settings */}
<SnackbarProvider> {/* Display various snackbars */}
<Routes>
<Route path={"/"} element={<HomeLayout/>}>
<Route path={router.home} element={<CustomPanel/>}/>
<Route path={router.dashboards} element={<DashboardsLayout/>}/>
<Route path={router.cardinality} element={<CardinalityPanel/>} />
<Route path={router.topQueries} element={<TopQueries/>} />
</Route>
</Routes>
</SnackbarProvider>
</TopQueriesStateProvider>
</CardinalityStateProvider>
</GraphStateProvider>
</AuthStateProvider>

View file

@ -0,0 +1,3 @@
export const getTopQueries = (server: string, topN: number | null, maxLifetime?: string) => (
`${server}/api/v1/status/top_queries?topN=${topN || ""}&maxLifetime=${maxLifetime || ""}`
);

View file

@ -3,9 +3,10 @@ import {InstantMetricResult} from "../../../api/types";
import Box from "@mui/material/Box";
import Button from "@mui/material/Button";
import {useSnack} from "../../../contexts/Snackbar";
import {TopQuery} from "../../../types";
export interface JsonViewProps {
data: InstantMetricResult[];
data: InstantMetricResult[] | TopQuery[];
}
const JsonView: FC<JsonViewProps> = ({data}) => {

View file

@ -61,8 +61,26 @@ const Header: FC = () => {
const {date} = useCardinalityState();
const cardinalityDispatch = useCardinalityDispatch();
const {search, pathname} = useLocation();
const navigate = useNavigate();
const {search, pathname} = useLocation();
const routes = [
{
label: "Custom panel",
value: router.home,
},
{
label: "Dashboards",
value: router.dashboards,
},
{
label: "Cardinality",
value: router.cardinality,
},
{
label: "Top queries",
value: router.topQueries,
}
];
const [activeMenu, setActiveMenu] = useState(pathname);
@ -102,13 +120,15 @@ const Header: FC = () => {
<Box sx={{ml: 8}}>
<Tabs value={activeMenu} textColor="inherit" TabIndicatorProps={{style: {background: "white"}}}
onChange={(e, val) => setActiveMenu(val)}>
<Tab label="Custom panel" value={router.home} component={RouterLink} to={`${router.home}${search}`}/>
<Tab label="Dashboards" value={router.dashboards} component={RouterLink} to={`${router.dashboards}${search}`}/>
{routes.map(r => (
<Tab
label="Cardinality"
value={router.cardinality}
key={`${r.label}_${r.value}`}
label={r.label}
value={r.value}
component={RouterLink}
to={`${router.cardinality}${search}`}/>
to={`${r.value}${search}`}
/>
))}
</Tabs>
</Box>
<Box display="flex" gap={1} alignItems="center" ml="auto" mr={0}>

View file

@ -78,7 +78,7 @@ const PredefinedDashboard: FC<PredefinedDashboardProps> = ({index, title, panels
return <Accordion defaultExpanded={!index} sx={{boxShadow: "none"}}>
<AccordionSummary
sx={{px: 3, bgcolor: "rgba(227, 242, 253, 0.6)"}}
sx={{px: 3, bgcolor: "primary.light"}}
aria-controls={`panel${index}-content`}
id={`panel${index}-header`}
expandIcon={<ExpandMoreIcon />}

View file

@ -0,0 +1,165 @@
import React, {ChangeEvent, FC, useEffect, useMemo, KeyboardEvent} from "react";
import Box from "@mui/material/Box";
import {useFetchTopQueries} from "../../hooks/useFetchTopQueries";
import Spinner from "../common/Spinner";
import Alert from "@mui/material/Alert";
import TopQueryPanel from "./TopQueryPanel/TopQueryPanel";
import Tooltip from "@mui/material/Tooltip";
import Typography from "@mui/material/Typography";
import TextField from "@mui/material/TextField";
import {useTopQueriesDispatch, useTopQueriesState} from "../../state/topQueries/TopQueriesStateContext";
import {formatPrettyNumber} from "../../utils/uplot/helpers";
import {isSupportedDuration} from "../../utils/time";
import IconButton from "@mui/material/IconButton";
import PlayCircleOutlineIcon from "@mui/icons-material/PlayCircleOutline";
import dayjs from "dayjs";
import {TopQueryStats} from "../../types";
const exampleDuration = "30ms, 15s, 3d4h, 1y2w";
const TopQueries: FC = () => {
const {data, error, loading} = useFetchTopQueries();
const {topN, maxLifetime} = useTopQueriesState();
const topQueriesDispatch = useTopQueriesDispatch();
const invalidTopN = useMemo(() => !!topN && topN < 1, [topN]);
const maxLifetimeValid = useMemo(() => {
const durItems = maxLifetime.trim().split(" ");
const durObject = durItems.reduce((prev, curr) => {
const dur = isSupportedDuration(curr);
return dur ? {...prev, ...dur} : {...prev};
}, {});
const delta = dayjs.duration(durObject).asMilliseconds();
return !!delta;
}, [maxLifetime]);
const getQueryStatsTitle = (key: keyof TopQueryStats) => {
if (!data) return key;
const value = data[key];
if (typeof value === "number") return formatPrettyNumber(value);
return value || key;
};
const onTopNChange = (e: ChangeEvent<HTMLTextAreaElement|HTMLInputElement>) => {
topQueriesDispatch({type: "SET_TOP_N", payload: +e.target.value});
};
const onMaxLifetimeChange = (e: ChangeEvent<HTMLTextAreaElement|HTMLInputElement>) => {
topQueriesDispatch({type: "SET_MAX_LIFE_TIME", payload: e.target.value});
};
const onApplyQuery = () => {
topQueriesDispatch({type: "SET_RUN_QUERY"});
};
const onKeyDown = (e: KeyboardEvent) => {
if (e.key === "Enter") onApplyQuery();
};
useEffect(() => {
if (!data) return;
if (!topN) topQueriesDispatch({type: "SET_TOP_N", payload: +data.topN});
if (!maxLifetime) topQueriesDispatch({type: "SET_MAX_LIFE_TIME", payload: data.maxLifetime});
}, [data]);
return (
<Box p={4} style={{minHeight: "calc(100vh - 64px)"}}>
{loading && <Spinner isLoading={true} height={"100%"}/>}
<Box boxShadow="rgba(99, 99, 99, 0.2) 0px 2px 8px 0px;" p={4} pb={2} m={-4} mb={4}>
<Box display={"flex"} alignItems={"flex"} mb={2}>
<Box mr={2} flexGrow={1}>
<TextField
fullWidth
label="Max lifetime"
size="medium"
variant="outlined"
value={maxLifetime}
error={!maxLifetimeValid}
helperText={!maxLifetimeValid ? "Invalid duration value" : `For example ${exampleDuration}`}
onChange={onMaxLifetimeChange}
onKeyDown={onKeyDown}
/>
</Box>
<Box mr={2}>
<TextField
fullWidth
label="Number of returned queries"
type="number"
size="medium"
variant="outlined"
value={topN || ""}
error={invalidTopN}
helperText={invalidTopN ? "Number must be bigger than zero" : " "}
onChange={onTopNChange}
onKeyDown={onKeyDown}
/>
</Box>
<Box>
<Tooltip title="Apply">
<IconButton onClick={onApplyQuery} sx={{height: "49px", width: "49px"}}>
<PlayCircleOutlineIcon/>
</IconButton>
</Tooltip>
</Box>
</Box>
<Typography variant="body1" pt={2}>
VictoriaMetrics tracks the last&nbsp;
<Tooltip arrow title={<Typography>search.queryStats.lastQueriesCount</Typography>}>
<b style={{cursor: "default"}}>
{getQueryStatsTitle("search.queryStats.lastQueriesCount")}
</b>
</Tooltip>
&nbsp;queries with durations at least&nbsp;
<Tooltip arrow title={<Typography>search.queryStats.minQueryDuration</Typography>}>
<b style={{cursor: "default"}}>
{getQueryStatsTitle("search.queryStats.minQueryDuration")}
</b>
</Tooltip>
</Typography>
</Box>
{error && <Alert color="error" severity="error" sx={{whiteSpace: "pre-wrap", my: 2}}>{error}</Alert>}
{data && (<>
<Box>
<TopQueryPanel
rows={data.topByCount}
title={"Most frequently executed queries"}
columns={[
{key: "query"},
{key: "timeRangeHours", title: "time range, hours"},
{key: "count"}
]}
/>
<TopQueryPanel
rows={data.topByAvgDuration}
title={"Most heavy queries"}
// columns={["query", "avgDurationSeconds", "timeRangeHours", "count"]}
columns={[
{key: "query"},
{key: "avgDurationSeconds", title: "avg duration, seconds"},
{key: "timeRangeHours", title: "time range, hours"},
{key: "count"}
]}
defaultOrderBy={"avgDurationSeconds"}
/>
<TopQueryPanel
rows={data.topBySumDuration}
title={"Queries with most summary time to execute"}
columns={[
{key: "query"},
{key: "sumDurationSeconds", title: "sum duration, seconds"},
{key: "timeRangeHours", title: "time range, hours"},
{key: "count"}
]}
defaultOrderBy={"sumDurationSeconds"}
/>
</Box>
</>)}
</Box>
);
};
export default TopQueries;

View file

@ -0,0 +1,89 @@
import React, {FC, useState} from "react";
import Box from "@mui/material/Box";
import {TopQuery} from "../../../types";
import Typography from "@mui/material/Typography";
import Accordion from "@mui/material/Accordion";
import AccordionSummary from "@mui/material/AccordionSummary";
import ExpandMoreIcon from "@mui/icons-material/ExpandMore";
import AccordionDetails from "@mui/material/AccordionDetails";
import Tabs from "@mui/material/Tabs";
import Tab from "@mui/material/Tab";
import TableChartIcon from "@mui/icons-material/TableChart";
import CodeIcon from "@mui/icons-material/Code";
import TopQueryTable from "../TopQueryTable/TopQueryTable";
import JsonView from "../../CustomPanel/Views/JsonView";
export interface TopQueryPanelProps {
rows: TopQuery[],
title?: string,
columns: {title?: string, key: (keyof TopQuery)}[],
defaultOrderBy?: keyof TopQuery,
}
const tabs = ["table", "JSON"];
const TopQueryPanel: FC<TopQueryPanelProps> = ({rows, title, columns, defaultOrderBy}) => {
const [activeTab, setActiveTab] = useState(0);
const onChangeTab = (e: React.SyntheticEvent, val: number) => {
setActiveTab(val);
};
return (
<Accordion
defaultExpanded={true}
sx={{
mt: 2,
border: "1px solid",
borderColor: "primary.light",
boxShadow: "none",
"&:before": {
opacity: 0
}
}}
>
<AccordionSummary
sx={{
p: 2,
bgcolor: "primary.light",
minHeight: "64px",
".MuiAccordionSummary-content": { display: "flex", alignItems: "center" },
}}
expandIcon={<ExpandMoreIcon />}
>
<Typography variant="h6" component="h6">
{title}
</Typography>
</AccordionSummary>
<AccordionDetails sx={{p: 0}}>
<Box width={"100%"}>
<Box sx={{ borderBottom: 1, borderColor: "divider" }}>
<Tabs
value={activeTab}
onChange={onChangeTab}
sx={{minHeight: "0", marginBottom: "-1px"}}
>
{tabs.map((title: string, i: number) =>
<Tab
key={title}
label={title}
aria-controls={`tabpanel-${i}`}
id={`${title}_${i}`}
iconPosition={"start"}
sx={{minHeight: "41px"}}
icon={ i === 0 ? <TableChartIcon /> : <CodeIcon /> } />
)}
</Tabs>
</Box>
{activeTab === 0 && <TopQueryTable rows={rows} columns={columns} defaultOrderBy={defaultOrderBy}/>}
{activeTab === 1 && <Box m={2}><JsonView data={rows} /></Box>}
</Box>
</AccordionDetails>
<Box >
</Box>
</Accordion>
);
};
export default TopQueryPanel;

View file

@ -0,0 +1,76 @@
import React, {FC, useState, useMemo} from "react";
import TableContainer from "@mui/material/TableContainer";
import Table from "@mui/material/Table";
import TableHead from "@mui/material/TableHead";
import TableRow from "@mui/material/TableRow";
import TableCell from "@mui/material/TableCell";
import TableBody from "@mui/material/TableBody";
import TableSortLabel from "@mui/material/TableSortLabel";
import {TopQuery} from "../../../types";
import {getComparator, stableSort} from "../../Table/helpers";
import {TopQueryPanelProps} from "../TopQueryPanel/TopQueryPanel";
const TopQueryTable:FC<TopQueryPanelProps> = ({rows, columns, defaultOrderBy}) => {
const [orderBy, setOrderBy] = useState<keyof TopQuery>(defaultOrderBy || "count");
const [orderDir, setOrderDir] = useState<"asc" | "desc">("desc");
const sortedList = useMemo(() => stableSort(rows as [], getComparator(orderDir, orderBy)),
[rows, orderBy, orderDir]);
const onSortHandler = (key: keyof TopQuery) => {
setOrderDir((prev) => prev === "asc" && orderBy === key ? "desc" : "asc");
setOrderBy(key);
};
const createSortHandler = (col: keyof TopQuery) => () => {
onSortHandler(col);
};
return <TableContainer>
<Table
sx={{minWidth: 750}}
aria-labelledby="tableTitle"
>
<TableHead>
<TableRow>
{columns.map((col) => (
<TableCell
key={col.key}
style={{width: "100%"}}
sx={{borderBottomColor: "primary.light", whiteSpace: "nowrap"}}
>
<TableSortLabel
active={orderBy === col.key}
direction={orderDir}
id={col.key}
onClick={createSortHandler(col.key)}
>
{col.title || col.key}
</TableSortLabel>
</TableCell>
))}
</TableRow>
</TableHead>
<TableBody>
{sortedList.map((row, rowIndex) => (
<TableRow key={rowIndex}>
{columns.map((col) => (
<TableCell
key={col.key}
sx={{
borderBottom: rowIndex === rows.length - 1 ? "none" : "",
borderBottomColor: "primary.light"
}}
>
{row[col.key] || "-"}
</TableCell>
))}
</TableRow>
))}
</TableBody>
</Table>
</TableContainer>;
};
export default TopQueryTable;

View file

@ -50,9 +50,10 @@ export const useFetchQuery = ({predefinedQuery, visible, display, customStep}: F
const fetchData = async (fetchUrl: string[], fetchQueue: AbortController[], displayType: DisplayType, query: string[]) => {
const controller = new AbortController();
setFetchQueue([...fetchQueue, controller]);
const isDisplayChart = displayType === "chart";
try {
const responses = await Promise.all(fetchUrl.map(url => fetch(url, {signal: controller.signal})));
const tempData = [];
const tempData: MetricBase[] = [];
const tempTraces: Trace[] = [];
let counter = 1;
for await (const response of responses) {
@ -63,16 +64,16 @@ export const useFetchQuery = ({predefinedQuery, visible, display, customStep}: F
const trace = new Trace(resp.trace, query[counter-1]);
tempTraces.push(trace);
}
tempData.push(...resp.data.result.map((d: MetricBase) => {
resp.data.result.forEach((d: MetricBase) => {
d.group = counter;
return d;
}));
tempData.push(d);
});
counter++;
} else {
setError(`${resp.errorType}\r\n${resp?.error}`);
}
}
displayType === "chart" ? setGraphData(tempData) : setLiveData(tempData);
isDisplayChart ? setGraphData(tempData as MetricResult[]) : setLiveData(tempData as InstantMetricResult[]);
setTraces(tempTraces);
} catch (e) {
if (e instanceof Error && e.name !== "AbortError") {

View file

@ -0,0 +1,58 @@
import { useEffect, useState } from "react";
import {ErrorTypes} from "../types";
import {getAppModeEnable, getAppModeParams} from "../utils/app-mode";
import {useAppState} from "../state/common/StateContext";
import {useMemo} from "preact/compat";
import {getTopQueries} from "../api/top-queries";
import {TopQueriesData} from "../types";
import {useTopQueriesState} from "../state/topQueries/TopQueriesStateContext";
export const useFetchTopQueries = () => {
const appModeEnable = getAppModeEnable();
const {serverURL: appServerUrl} = getAppModeParams();
const {serverUrl} = useAppState();
const {topN, maxLifetime, runQuery} = useTopQueriesState();
const [data, setData] = useState<TopQueriesData | null>(null);
const [loading, setLoading] = useState(false);
const [error, setError] = useState<ErrorTypes | string>();
const server = useMemo(() => appModeEnable ? appServerUrl : serverUrl,
[appModeEnable, serverUrl, appServerUrl]);
const fetchUrl = useMemo(() => getTopQueries(server, topN, maxLifetime), [server, topN, maxLifetime]);
const fetchData = async () => {
setLoading(true);
try {
const response = await fetch(fetchUrl);
const resp = await response.json();
if (response.ok) {
const list = ["topByAvgDuration", "topByCount", "topBySumDuration"] as (keyof TopQueriesData)[];
list.forEach(key => {
const target = resp[key];
if (Array.isArray(target)) {
target.forEach(t => t.timeRangeHours = +(t.timeRangeSeconds/3600).toFixed(2));
}
});
}
setData(response.ok ? resp : null);
setError(String(resp.error || ""));
} catch (e) {
if (e instanceof Error && e.name !== "AbortError") {
setError(`${e.name}: ${e.message}`);
}
}
setLoading(false);
};
useEffect(() => {
fetchData();
}, [runQuery]);
return {
data,
error,
loading
};
};

View file

@ -2,6 +2,7 @@ const router = {
home: "/",
dashboards: "/dashboards",
cardinality: "/cardinality",
topQueries: "/top-queries",
};
export interface RouterOptions {

View file

@ -19,14 +19,14 @@ export const initialPrepopulatedState = Object.entries(initialState)
}), {}) as AppState;
export const StateProvider: FC = ({children}) => {
const location = useLocation();
const {pathname} = useLocation();
const [state, dispatch] = useReducer(reducer, initialPrepopulatedState);
useEffect(() => {
if (location.pathname === router.cardinality) return;
if (pathname !== router.dashboards && pathname !== router.home) return;
setQueryStringValue(state as unknown as Record<string, unknown>);
}, [state, location]);
}, [state, pathname]);
const contextValue = useMemo(() => {
return { state, dispatch };

View file

@ -0,0 +1,35 @@
import React, {createContext, FC, useContext, useEffect, useMemo, useReducer} from "preact/compat";
import {Action, TopQueriesState, initialState, reducer} from "./reducer";
import {Dispatch} from "react";
import {useLocation} from "react-router-dom";
import {setQueryStringValue} from "../../utils/query-string";
import router from "../../router";
type TopQueriesStateContextType = { state: TopQueriesState, dispatch: Dispatch<Action> };
export const TopQueriesStateContext = createContext<TopQueriesStateContextType>({} as TopQueriesStateContextType);
export const useTopQueriesState = (): TopQueriesState => useContext(TopQueriesStateContext).state;
export const useTopQueriesDispatch = (): Dispatch<Action> => useContext(TopQueriesStateContext).dispatch;
export const TopQueriesStateProvider: FC = ({children}) => {
const location = useLocation();
const [state, dispatch] = useReducer(reducer, initialState);
useEffect(() => {
if (location.pathname !== router.topQueries) return;
setQueryStringValue(state as unknown as Record<string, unknown>);
}, [state, location]);
const contextValue = useMemo(() => {
return { state, dispatch };
}, [state, dispatch]);
return <TopQueriesStateContext.Provider value={contextValue}>
{children}
</TopQueriesStateContext.Provider>;
};

View file

@ -0,0 +1,41 @@
import {getQueryStringValue} from "../../utils/query-string";
export interface TopQueriesState {
maxLifetime: string,
topN: number | null,
runQuery: number
}
export type Action =
| { type: "SET_TOP_N", payload: number | null }
| { type: "SET_MAX_LIFE_TIME", payload: string }
| { type: "SET_RUN_QUERY" }
export const initialState: TopQueriesState = {
topN: getQueryStringValue("topN", null) as number,
maxLifetime: getQueryStringValue("maxLifetime", "") as string,
runQuery: 0
};
export function reducer(state: TopQueriesState, action: Action): TopQueriesState {
switch (action.type) {
case "SET_TOP_N":
return {
...state,
topN: action.payload
};
case "SET_MAX_LIFE_TIME":
return {
...state,
maxLifetime: action.payload
};
case "SET_RUN_QUERY":
return {
...state,
runQuery: state.runQuery + 1
};
default:
throw new Error();
}
}

View file

@ -3,7 +3,8 @@ import {createTheme} from "@mui/material/styles";
const THEME = createTheme({
palette: {
primary: {
main: "#3F51B5"
main: "#3F51B5",
light: "#e3f2fd"
},
secondary: {
main: "#F50057"
@ -17,7 +18,7 @@ const THEME = createTheme({
styleOverrides: {
root: {
position: "absolute",
top: "36px",
bottom: "-16px",
left: "2px",
margin: 0,
}

View file

@ -69,3 +69,27 @@ export interface RelativeTimeOption {
title: string,
isDefault?: boolean,
}
export interface TopQuery {
accountID: number
avgDurationSeconds: number
count: number
projectID: number
query: string
timeRangeSeconds: number
sumDurationSeconds: number
timeRangeHours: number
}
export interface TopQueryStats {
"search.queryStats.lastQueriesCount": number
"search.queryStats.minQueryDuration": string
}
export interface TopQueriesData extends TopQueryStats{
maxLifetime: string
topN: string
topByAvgDuration: TopQuery[]
topByCount: TopQuery[]
topBySumDuration: TopQuery[]
}

View file

@ -19,6 +19,10 @@ const stateToUrlParams = {
"match": "match[]",
"extraLabel": "extra_label",
"focusLabel": "focusLabel"
},
[router.topQueries]: {
"topN": "topN",
"maxLifetime": "maxLifetime",
}
};

View file

@ -56,6 +56,7 @@ See also [case studies](https://docs.victoriametrics.com/CaseStudies.html).
* [Superset BI with Victoria Metrics](https://cer6erus.medium.com/superset-bi-with-victoria-metrics-a109d3e91bc6)
* [VictoriaMetrics Source Code Analysis - Bloom filter](https://www.sobyte.net/post/2022-05/victoriametrics-bloomfilter/)
* [How we tried using VictoriaMetrics and Thanos at the same time](https://habr.com/ru/company/sravni/blog/672908/)
* [Prometheus, Grafana, and Kubernetes, Oh My!](https://www.groundcover.com/blog/prometheus-grafana-kubernetes)
## Our articles
@ -68,6 +69,7 @@ See also [case studies](https://docs.victoriametrics.com/CaseStudies.html).
### Benchmarks
* [Grafana Mimir and VictoriaMetrics: performance tests](https://victoriametrics.com/blog/mimir-benchmark/)
* [When size matters — benchmarking VictoriaMetrics vs Timescale and InfluxDB](https://valyala.medium.com/when-size-matters-benchmarking-victoriametrics-vs-timescale-and-influxdb-6035811952d4)
* [High-cardinality TSDB benchmarks: VictoriaMetrics vs TimescaleDB vs InfluxDB](https://valyala.medium.com/high-cardinality-tsdb-benchmarks-victoriametrics-vs-timescaledb-vs-influxdb-13e6ee64dd6b)
* [Insert benchmarks with inch: InfluxDB vs VictoriaMetrics](https://valyala.medium.com/insert-benchmarks-with-inch-influxdb-vs-victoriametrics-e31a41ae2893)

View file

@ -17,13 +17,20 @@ The following tip changes can be tested by building VictoriaMetrics components f
**Update note:** this release changes data format for [/api/v1/export/native](https://docs.victoriametrics.com/#how-to-export-data-in-native-format) in incompatible way, so it cannot be imported into older version of VictoriaMetrics via [/api/v1/import/native](https://docs.victoriametrics.com/#how-to-import-data-in-native-format).
* FEATURE: check the correctess of raw sample timestamps stored on disk when reading them. This reduces the probability of possible silent corruption of the data stored on disk. This should help [this](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2998) and [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3011).
* FEATURE: atomically delete directories with snapshots, parts and partitions at [storage level](https://docs.victoriametrics.com/#storage). Previously such directories can be left in partially deleted state when the deletion operation was interrupted by unclean shutdown. This may result in `cannot open file ...: no such file or directory` error on the next start. The probability of this error was quite high when NFS or EFS was used as persistent storage for VictoriaMetrics data. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3038).
* FEATURE: set the `start` arg to `end - 5 minutes` if isn't passed explicitly to [/api/v1/labels](https://docs.victoriametrics.com/url-examples.html#apiv1labels) and [/api/v1/label/.../values](https://docs.victoriametrics.com/url-examples.html#apiv1labelvalues). See [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/3052).
* FEATURE: [vmctl](https://docs.victoriametrics.com/vmctl.html): add `vm-native-step-interval` command line flag for `vm-native` mode. New option allows splitting the import process into chunks by time interval. This helps migrating data sets with high churn rate and provides better control over the process. See [feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2733).
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): add `top queries` tab, which shows various stats for recently executed queries. See [these docs](https://docs.victoriametrics.com/#top-queries) and [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2707).
* FEATURE: [vmalert](https://docs.victoriametrics.com/vmalert.html): add `debug` mode to the alerting rule settings for printing additional information into logs during evaluation. See `debug` param in [alerting rule config](https://docs.victoriametrics.com/vmalert.html#alerting-rules).
* FEATURE: [vmalert](https://docs.victoriametrics.com/vmalert.html): add experimental feature for displaying last 10 states of the rule (recording or alerting) evaluation. The state is available on the Rule page, which can be opened by clicking on `Details` link next to Rule's name on the `/groups` page.
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): minimize the time needed for reading large responses from scrape targets in [stream parsing mode](https://docs.victoriametrics.com/vmagent.html#stream-parsing-mode). This should reduce scrape durations for such targets as [kube-state-metrics](https://github.com/kubernetes/kube-state-metrics) running in a big Kubernetes cluster.
* FEATURE: [MetricsQL](https://docs.victoriametrics.com/MetricsQL.html): add [sort_by_label_numeric](https://docs.victoriametrics.com/MetricsQL.html#sort_by_label_numeric) and [sort_by_label_numeric_desc](https://docs.victoriametrics.com/MetricsQL.html#sort_by_label_numeric_desc) functions for [numeric sort](https://www.gnu.org/software/coreutils/manual/html_node/Version-sort-is-not-the-same-as-numeric-sort.html) of input time series by the specified labels. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2938).
* BUGFIX: [MetricsQL](https://docs.victoriametrics.com/MetricsQL.html): properly calculate `rate_over_sum(m[d])` as `sum_over_time(m[d])/d`. Previously the `sum_over_time(m[d])` could be improperly divided by smaller than `d` time range. See [rate_over_sum() docs](https://docs.victoriametrics.com/MetricsQL.html#rate_over_sum) and [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3045).
* BUGFIX: [VictoriaMetrics cluster](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html): properly calculate query results at `vmselect`. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3067). The issue has been introduced in [v1.81.0](https://docs.victoriametrics.com/CHANGELOG.html#v1810).
* BUGFIX: [VictoriaMetrics cluster](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html): log clear error when multiple identical `-storageNode` command-line flags are passed to `vmselect` or to `vminsert`. Previously these components were crashed with cryptic panic `metric ... is already registered` in this case. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3076).
* BUGFIX: [vmui](https://docs.victoriametrics.com/#vmui): fix `RangeError: Maximum call stack size exceeded` error when the query returns too many data points at `Table` view. See [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/3092/files).
## [v1.81.2](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.81.2)

7
docs/ExtendedPromQL.md Normal file
View file

@ -0,0 +1,7 @@
---
sort: 100
---
# MetricsQL old page
The page has been moved to [MetricsQL](https://docs.victoriametrics.com/MetricsQL.html).

View file

@ -109,6 +109,8 @@ VictoriaMetrics also [uses less RAM than Thanos components](https://github.com/t
Grafana Mimir is a [Cortex](https://github.com/cortexproject/cortex) fork, so it has the same differences
as Cortex. See [what is the difference between VictoriaMetrics and Cortex](#what-is-the-difference-between-victoriametrics-and-cortex).
See also [Grafana Mimir vs VictoriaMetrics benchmark](https://victoriametrics.com/blog/mimir-benchmark/).
## What is the difference between VictoriaMetrics and [Cortex](https://github.com/cortexproject/cortex)?
VictoriaMetrics is similar to Cortex in the following aspects:

View file

@ -21,7 +21,7 @@ The following functionality is implemented differently in MetricsQL compared to
* MetricsQL removes all the `NaN` values from the output, so some queries like `(-1)^0.5` return empty results in VictoriaMetrics, while returning a series of `NaN` values in Prometheus. Note that Grafana doesn't draw any lines or dots for `NaN` values, so the end result looks the same for both VictoriaMetrics and Prometheus.
* MetricsQL keeps metric names after applying functions, which don't change the meaning of the original time series. For example, [min_over_time(foo)](#min_over_time) or [round(foo)](#round) leaves `foo` metric name in the result. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/674) for details.
Read more about the diffferences between PromQL and MetricsQL in [this article](https://medium.com/@romanhavronenko/victoriametrics-promql-compliance-d4318203f51e).
Read more about the differences between PromQL and MetricsQL in [this article](https://medium.com/@romanhavronenko/victoriametrics-promql-compliance-d4318203f51e).
Other PromQL functionality should work the same in MetricsQL. [File an issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues) if you notice discrepancies between PromQL and MetricsQL results other than mentioned above.
@ -37,7 +37,7 @@ This functionality can be evaluated at [an editable Grafana dashboard](https://p
* [@ modifier](https://prometheus.io/docs/prometheus/latest/querying/basics/#modifier) can be put anywhere in the query. For example, `sum(foo) @ end()` calculates `sum(foo)` at the `end` timestamp of the selected time range `[start ... end]`.
* Arbitrary subexpression can be used as [@ modifier](https://prometheus.io/docs/prometheus/latest/querying/basics/#modifier). For example, `foo @ (end() - 1h)` calculates `foo` at the `end - 1 hour` timestamp on the selected time range `[start ... end]`.
* [offset](https://prometheus.io/docs/prometheus/latest/querying/basics/#offset-modifier), lookbehind window in square brackets and `step` value for [subquery](#subqueries) may refer to the current step aka `$__interval` value from Grafana with `[Ni]` syntax. For instance, `rate(metric[10i] offset 5i)` would return per-second rate over a range covering 10 previous steps with the offset of 5 steps.
* [offset](https://prometheus.io/docs/prometheus/latest/querying/basics/#offset-modifier) may be put anywere in the query. For instance, `sum(foo) offset 24h`.
* [offset](https://prometheus.io/docs/prometheus/latest/querying/basics/#offset-modifier) may be put anywhere in the query. For instance, `sum(foo) offset 24h`.
* Lookbehind window in square brackets and [offset](https://prometheus.io/docs/prometheus/latest/querying/basics/#offset-modifier) may be fractional. For instance, `rate(node_network_receive_bytes_total[1.5m] offset 0.5d)`.
* The duration suffix is optional. The duration is in seconds if the suffix is missing. For example, `rate(m[300] offset 1800)` is equivalent to `rate(m[5m]) offset 30m`.
* The duration can be placed anywhere in the query. For example, `sum_over_time(m[1h]) / 1h` is equivalent to `sum_over_time(m[1h]) / 3600`.
@ -88,7 +88,7 @@ The list of supported rollup functions:
#### aggr_over_time
`aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`. The calculations are perfomed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). `rollup_func*` can contain any rollup function. For instance, `aggr_over_time(("min_over_time", "max_over_time", "rate"), m[d])` would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time) and [rate](#rate) for `m[d]`.
`aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). `rollup_func*` can contain any rollup function. For instance, `aggr_over_time(("min_over_time", "max_over_time", "rate"), m[d])` would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time) and [rate](#rate) for `m[d]`.
#### ascent_over_time
@ -136,7 +136,7 @@ The list of supported rollup functions:
#### delta
`delta(series_selector[d])` calculates the difference between the last sample before the given lookbehind window `d` and the last sample at the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). The behaviour of `delta()` function in MetricsQL is slighly different to the behaviour of `delta()` function in Prometheus. See [this article](https://medium.com/@romanhavronenko/victoriametrics-promql-compliance-d4318203f51e) for details. Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. This function is supported by PromQL. See also [increase](#increase) and [delta_prometheus](#delta_prometheus).
`delta(series_selector[d])` calculates the difference between the last sample before the given lookbehind window `d` and the last sample at the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). The behaviour of `delta()` function in MetricsQL is slightly different to the behaviour of `delta()` function in Prometheus. See [this article](https://medium.com/@romanhavronenko/victoriametrics-promql-compliance-d4318203f51e) for details. Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. This function is supported by PromQL. See also [increase](#increase) and [delta_prometheus](#delta_prometheus).
#### delta_prometheus
@ -272,7 +272,7 @@ The list of supported rollup functions:
#### rate_over_sum
`rate_over_sum(series_selector[d])` calculates per-second rate over the sum of raw samples on the given lookbehind window `d`. The calculations are performed indiviually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
`rate_over_sum(series_selector[d])` calculates per-second rate over the sum of raw samples on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### resets
@ -284,27 +284,27 @@ The list of supported rollup functions:
#### rollup_candlestick
`rollup_candlestick(series_selector[d])` calculates `open`, `high`, `low` and `close` values (aka OHLC) over raw samples on the given lookbehind window `d`. The calculations are perfomed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). This function is useful for financial applications.
`rollup_candlestick(series_selector[d])` calculates `open`, `high`, `low` and `close` values (aka OHLC) over raw samples on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). This function is useful for financial applications.
#### rollup_delta
`rollup_delta(series_selector[d])` calculates differences between adjancent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [rollup_increase](#rollup_increase).
`rollup_delta(series_selector[d])` calculates differences between adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [rollup_increase](#rollup_increase).
#### rollup_deriv
`rollup_deriv(series_selector[d])` calculates per-second derivatives for adjancent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
`rollup_deriv(series_selector[d])` calculates per-second derivatives for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### rollup_increase
`rollup_increase(series_selector[d])` calculates increases for adjancent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [rollup_delta](#rollup_delta).
`rollup_increase(series_selector[d])` calculates increases for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [rollup_delta](#rollup_delta).
#### rollup_rate
`rollup_rate(series_selector[d])` calculates per-second change rates for adjancent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates. The calculations are perfomed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
`rollup_rate(series_selector[d])` calculates per-second change rates for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names.
#### rollup_scrape_interval
`rollup_scrape_interval(series_selector[d])` calculates the interval in seconds between adjancent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval. The calculations are perfomed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [scrape_interval](#scrape_interval).
`rollup_scrape_interval(series_selector[d])` calculates the interval in seconds between adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. See also [scrape_interval](#scrape_interval).
#### scrape_interval
@ -328,7 +328,7 @@ The list of supported rollup functions:
#### stdvar_over_time
`stdvar_over_time(series_selector[d])` calculates stadnard variance over raw samples on the given lookbheind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. This function is supported by PromQL. See also [stddev_over_time](#stddev_over_time).
`stdvar_over_time(series_selector[d])` calculates standard variance over raw samples on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. This function is supported by PromQL. See also [stddev_over_time](#stddev_over_time).
#### sum_over_time
@ -443,7 +443,7 @@ The list of supported transform functions:
#### clamp_min
`clamp_min(q, min)` clamps every pount for every time series returned by `q` with the given `min` value. This function is supported by PromQL. See also [clamp](#clamp) and [clamp_max](#clamp_max).
`clamp_min(q, min)` clamps every point for every time series returned by `q` with the given `min` value. This function is supported by PromQL. See also [clamp](#clamp) and [clamp_max](#clamp_max).
#### cos
@ -483,7 +483,7 @@ The list of supported transform functions:
#### histogram_avg
`histogram_avg(buckets)` calculates the average value for the given `buckets`. It can be used for calculating the average over the given time range across multiple time series. For exmple, `histogram_avg(sum(histogram_over_time(response_time_duration_seconds[5m])) by (vmrange,job))` would return the average response time per each `job` over the last 5 minutes.
`histogram_avg(buckets)` calculates the average value for the given `buckets`. It can be used for calculating the average over the given time range across multiple time series. For example, `histogram_avg(sum(histogram_over_time(response_time_duration_seconds[5m])) by (vmrange,job))` would return the average response time per each `job` over the last 5 minutes.
#### histogram_quantile
@ -571,7 +571,7 @@ The list of supported transform functions:
#### rand_normal
`rand_normal(seed)` returns pesudo-random numbers with [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). Optional `seed` can be used as a seed for pseudo-random number generator. See also [rand](#rand) and [rand_exponential](#rand_exponential).
`rand_normal(seed)` returns pseudo-random numbers with [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). Optional `seed` can be used as a seed for pseudo-random number generator. See also [rand](#rand) and [rand_exponential](#rand_exponential).
#### range_avg
@ -673,6 +673,16 @@ The list of supported transform functions:
`sort_by_label_desc(q, label1, ... labelN)` sorts series in descending order by the given set of labels. For example, `sort_by_label(foo, "bar")` would sort `foo` series by values of the label `bar` in these series. See also [sort_by_label](#sort_by_label).
#### sort_by_label_numeric
`sort_by_label_numeric(q, label1, ... labelN)` sorts series in ascending order by the given set of labels using [numeric sort](https://www.gnu.org/software/coreutils/manual/html_node/Version-sort-is-not-the-same-as-numeric-sort.html). For example, if `foo` series have `bar` label with values `1`, `101`, `15` and `2`, then `sort_by_label_numeric(foo, "bar")` would return series in the following order of `bar` label values: `1`, `2`, `15` and `101`.
See also [sort_by_label_numeric_desc](#sort_by_label_numeric_desc) and [sort_by_label](#sort_by_label).
#### sort_by_label_numeric_desc
`sort_by_label_numeric_desc(q, label1, ... labelN)` sorts series in descending order by the given set of labels using [numeric sort](https://www.gnu.org/software/coreutils/manual/html_node/Version-sort-is-not-the-same-as-numeric-sort.html). For example, if `foo` series have `bar` label with values `1`, `101`, `15` and `2`, then `sort_by_label_numeric(foo, "bar")` would return series in the following order of `bar` label values: `101`, `15`, `2` and `1`.
See also [sort_by_label_numeric](#sort_by_label_numeric) and [sort_by_label_desc](#sort_by_label_desc).
#### sort_desc
`sort_desc(q)` sorts series in descending order by the last point in every time series returned by `q`. This function is supported by PromQL. See also [sort](#sort).
@ -703,7 +713,7 @@ The list of supported transform functions:
#### union
`union(q1, ..., qN)` returns a union of time series returned from `q1`, ..., `qN`. The `union` function name can be skipped - the following queries are quivalent: `union(q1, q2)` and `(q1, q2)`. It is expected that each `q*` query returns time series with unique sets of labels. Otherwise only the first time series out of series with identical set of labels is returned. Use [alias](#alias) and [label_set](#label_set) functions for giving unique labelsets per each `q*` query:
`union(q1, ..., qN)` returns a union of time series returned from `q1`, ..., `qN`. The `union` function name can be skipped - the following queries are equivalent: `union(q1, q2)` and `(q1, q2)`. It is expected that each `q*` query returns time series with unique sets of labels. Otherwise only the first time series out of series with identical set of labels is returned. Use [alias](#alias) and [label_set](#label_set) functions for giving unique labelsets per each `q*` query:
#### vector
@ -764,7 +774,7 @@ sum by (__name__) (
#### label_map
`label_map(q, "label", "src_value1", "dst_value1", ..., "src_valueN", "dst_valueN")` maps `label` values from `src_*` to `dst*` for all the time seires returned by `q`.
`label_map(q, "label", "src_value1", "dst_value1", ..., "src_valueN", "dst_valueN")` maps `label` values from `src_*` to `dst*` for all the time series returned by `q`.
#### label_match
@ -803,7 +813,7 @@ sum by (__name__) (
**Aggregate functions** calculate aggregates over groups of rollup results. Additional details:
* By default a single group is used for aggregation. Multiple independent groups can be set up by specifying grouping labels in `by` and `without` modifiers. For example, `count(up) by (job)` would group rollup results by `job` label value and calculate the [count](#count) aggregate function independently per each group, while `count(up) without (instance)` would group rollup results by all the labels except `instance` before calculating [count](#count) aggregate function independently per each group. Multiple labels can be put in `by` and `without` modifiers.
* If the aggregate function is applied directly to a [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering), then the [default_rollup()](#default_rollup) function is automatically applied before cacluating the aggregate. For example, `count(up)` is implicitly transformed to `count(default_rollup(up[1i]))`.
* If the aggregate function is applied directly to a [series_selector](https://docs.victoriametrics.com/keyConcepts.html#filtering), then the [default_rollup()](#default_rollup) function is automatically applied before calculating the aggregate. For example, `count(up)` is implicitly transformed to `count(default_rollup(up[1i]))`.
* Aggregate functions accept arbitrary number of args. For example, `avg(q1, q2, q3)` would return the average values for every point across time series returned by `q1`, `q2` and `q3`.
* Aggregate functions support optional `limit N` suffix, which can be used for limiting the number of output groups. For example, `sum(x) by (y) limit 3` limits the number of groups for the aggregation to 3. All the other groups are ignored.
@ -849,7 +859,7 @@ The list of supported aggregate functions:
#### count_values
`count_values("label", q)` counts the number of points with the same value and stores the counts in a time series with an additional `label`, wich contains each initial value. The aggregate is calculated individually per each group of points with the same timestamp. This function is supported by PromQL.
`count_values("label", q)` counts the number of points with the same value and stores the counts in a time series with an additional `label`, which contains each initial value. The aggregate is calculated individually per each group of points with the same timestamp. This function is supported by PromQL.
#### distinct
@ -933,11 +943,11 @@ The list of supported aggregate functions:
#### topk_last
`topk_last(k, q, "other_label=other_value")` returns up to `k` time series from `q` with the biggest last values. If an optional `other_label=other_value` arg is set, then the sum of the remaining time series is returned with the given label. For example, `topk_max(3, sum(process_resident_memory_bytes) by (job), "job=other")` would return up to 3 time series with the biggest amaximums plus a time series with `{job="other"}` label with the sum of the remaining series if any. See also [bottomk_last](#bottomk_last).
`topk_last(k, q, "other_label=other_value")` returns up to `k` time series from `q` with the biggest last values. If an optional `other_label=other_value` arg is set, then the sum of the remaining time series is returned with the given label. For example, `topk_max(3, sum(process_resident_memory_bytes) by (job), "job=other")` would return up to 3 time series with the biggest maximums plus a time series with `{job="other"}` label with the sum of the remaining series if any. See also [bottomk_last](#bottomk_last).
#### topk_max
`topk_max(k, q, "other_label=other_value")` returns up to `k` time series from `q` with the biggest maximums. If an optional `other_label=other_value` arg is set, then the sum of the remaining time series is returned with the given label. For example, `topk_max(3, sum(process_resident_memory_bytes) by (job), "job=other")` would return up to 3 time series with the biggest amaximums plus a time series with `{job="other"}` label with the sum of the remaining series if any. See also [bottomk_max](#bottomk_max).
`topk_max(k, q, "other_label=other_value")` returns up to `k` time series from `q` with the biggest maximums. If an optional `other_label=other_value` arg is set, then the sum of the remaining time series is returned with the given label. For example, `topk_max(3, sum(process_resident_memory_bytes) by (job), "job=other")` would return up to 3 time series with the biggest maximums plus a time series with `{job="other"}` label with the sum of the remaining series if any. See also [bottomk_max](#bottomk_max).
#### topk_median

View file

@ -260,7 +260,10 @@ Prometheus doesn't drop data during VictoriaMetrics restart. See [this article](
VictoriaMetrics provides UI for query troubleshooting and exploration. The UI is available at `http://victoriametrics:8428/vmui`.
The UI allows exploring query results via graphs and tables.
It also provides the ability to [explore cardinality](#cardinality-explorer) and to [investigate query traces](#query-tracing).
It also provides the following features:
- [cardinality explorer](#cardinality-explorer)
- [query tracer](#query-tracing)
- [top queries explorer](#top-queries)
Graphs in vmui support scrolling and zooming:
@ -280,6 +283,13 @@ VMUI allows investigating correlations between two queries on the same graph. Ju
See the [example VMUI at VictoriaMetrics playground](https://play.victoriametrics.com/select/accounting/1/6a716b0f-38bc-4856-90ce-448fd713e3fe/prometheus/graph/?g0.expr=100%20*%20sum(rate(process_cpu_seconds_total))%20by%20(job)&g0.range_input=1d).
## Top queries
[VMUI](#vmui) provides `top queries` tab, which can help determining the following query types:
* the most frequently executed queries;
* queries with the biggest average execution duration;
* queries that took the most summary time for execution.
## Cardinality explorer
@ -329,11 +339,11 @@ See also [vmagent](https://docs.victoriametrics.com/vmagent.html), which can be
VictoriaMetrics accepts data from [DataDog agent](https://docs.datadoghq.com/agent/) or [DogStatsD](https://docs.datadoghq.com/developers/dogstatsd/) via ["submit metrics" API](https://docs.datadoghq.com/api/latest/metrics/#submit-metrics) at `/datadog/api/v1/series` path.
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://victoriametrics-host:8428/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `http://victoriametrics-host:8428/datadog`.
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`.
@ -341,7 +351,7 @@ VictoriaMetrics doesn't check `DD_API_KEY` param, so it can be set to arbitrary
Example of how to send data to VictoriaMetrics via [DataDog "submit metrics"](https://docs.victoriametrics.com/url-examples.html#datadogapiv1series) from command line:
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
```console
echo '
@ -365,7 +375,7 @@ echo '
' | curl -X POST --data-binary @- http://victoriametrics-host:8428/datadog/api/v1/series
```
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -396,7 +406,7 @@ echo '
The imported data can be read via [export API](https://docs.victoriametrics.com/url-examples.html#apiv1export):
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -406,7 +416,7 @@ curl http://victoriametrics-host:8428/api/v1/export -d 'match[]=system.load.1'
</div>
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">

View file

@ -264,7 +264,10 @@ Prometheus doesn't drop data during VictoriaMetrics restart. See [this article](
VictoriaMetrics provides UI for query troubleshooting and exploration. The UI is available at `http://victoriametrics:8428/vmui`.
The UI allows exploring query results via graphs and tables.
It also provides the ability to [explore cardinality](#cardinality-explorer) and to [investigate query traces](#query-tracing).
It also provides the following features:
- [cardinality explorer](#cardinality-explorer)
- [query tracer](#query-tracing)
- [top queries explorer](#top-queries)
Graphs in vmui support scrolling and zooming:
@ -284,6 +287,13 @@ VMUI allows investigating correlations between two queries on the same graph. Ju
See the [example VMUI at VictoriaMetrics playground](https://play.victoriametrics.com/select/accounting/1/6a716b0f-38bc-4856-90ce-448fd713e3fe/prometheus/graph/?g0.expr=100%20*%20sum(rate(process_cpu_seconds_total))%20by%20(job)&g0.range_input=1d).
## Top queries
[VMUI](#vmui) provides `top queries` tab, which can help determining the following query types:
* the most frequently executed queries;
* queries with the biggest average execution duration;
* queries that took the most summary time for execution.
## Cardinality explorer
@ -333,11 +343,11 @@ See also [vmagent](https://docs.victoriametrics.com/vmagent.html), which can be
VictoriaMetrics accepts data from [DataDog agent](https://docs.datadoghq.com/agent/) or [DogStatsD](https://docs.datadoghq.com/developers/dogstatsd/) via ["submit metrics" API](https://docs.datadoghq.com/api/latest/metrics/#submit-metrics) at `/datadog/api/v1/series` path.
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://victoriametrics-host:8428/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `http://victoriametrics-host:8428/datadog`.
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
Run DataDog agent with environment variable `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`. Alternatively, set `dd_url` param at [DataDog agent configuration file](https://docs.datadoghq.com/agent/guide/agent-configuration-files/) to `DD_DD_URL=http://vinsert-host:8480/insert/0/datadog`.
@ -345,7 +355,7 @@ VictoriaMetrics doesn't check `DD_API_KEY` param, so it can be set to arbitrary
Example of how to send data to VictoriaMetrics via [DataDog "submit metrics"](https://docs.victoriametrics.com/url-examples.html#datadogapiv1series) from command line:
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
```console
echo '
@ -369,7 +379,7 @@ echo '
' | curl -X POST --data-binary @- http://victoriametrics-host:8428/datadog/api/v1/series
```
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -400,7 +410,7 @@ echo '
The imported data can be read via [export API](https://docs.victoriametrics.com/url-examples.html#apiv1export):
Single-node VictoriaMetrics:
### Single-node VictoriaMetrics:
<div class="with-copy" markdown="1">
@ -410,7 +420,7 @@ curl http://victoriametrics-host:8428/api/v1/export -d 'match[]=system.load.1'
</div>
Cluster version of VictoriaMetrics:
### Cluster version of VictoriaMetrics:
<div class="with-copy" markdown="1">

View file

@ -744,7 +744,7 @@ VMAgentSpec defines the desired state of VMAgent
| extraEnvs | ExtraEnvs that will be added to VMAgent pod | [][v1.EnvVar](https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.22/#envvar-v1-core) | false |
| serviceSpec | ServiceSpec that will be added to vmagent service spec | *[ServiceSpec](#servicespec) | false |
| serviceScrapeSpec | ServiceScrapeSpec that will be added to vmselect VMServiceScrape spec | *[VMServiceScrapeSpec](#vmservicescrapespec) | false |
| shardCount | ShardCount - numbers of shards of VMAgent in this case operator will use 1 deployment/sts per shard with replicas count according to spec.replicas https://victoriametrics.github.io/vmagent.html#scraping-big-number-of-targets | *int | false |
| shardCount | ShardCount - numbers of shards of VMAgent in this case operator will use 1 deployment/sts per shard with replicas count according to spec.replicas https://docs.victoriametrics.com/vmagent.html#scraping-big-number-of-targets | *int | false |
| updateStrategy | UpdateStrategy - overrides default update strategy. works only for deployments, statefulset always use OnDelete. | *[appsv1.DeploymentStrategyType](https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.22/#deploymentstrategy-v1-apps) | false |
| rollingUpdate | RollingUpdate - overrides deployment update params. | *[appsv1.RollingUpdateDeployment](https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.22/#rollingupdatedeployment-v1-apps) | false |
| podDisruptionBudget | PodDisruptionBudget created by operator | *[EmbeddedPodDisruptionBudgetSpec](#embeddedpoddisruptionbudgetspec) | false |

View file

@ -833,7 +833,7 @@ We recommend using [binary releases](https://github.com/VictoriaMetrics/Victoria
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmagent` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds the `vmagent` binary and puts it into the `bin` folder.
@ -862,7 +862,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmagent-linux-arm` or `make vmagent-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics)
It builds `vmagent-linux-arm` or `vmagent-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -188,6 +188,13 @@ expr: <string>
# as firing once they return.
[ for: <duration> | default = 0s ]
# Whether to print debug information into logs.
# Information includes alerts state changes and requests sent to the datasource.
# Please note, that if rule's query params contain sensitive
# information - it will be printed to logs.
# Is applicable to alerting rules only.
[ debug: <bool> | default = false ]
# Labels to add or overwrite for each alert.
labels:
[ <labelname>: <tmpl_string> ]
@ -508,6 +515,7 @@ or time series modification via [relabeling](https://docs.victoriametrics.com/vm
* `http://<vmalert-addr>/vmalert/api/v1/alert?group_id=<group_id>&alert_id=<alert_id>` - get alert status in JSON format.
Used as alert source in AlertManager.
* `http://<vmalert-addr>/vmalert/alert?group_id=<group_id>&alert_id=<alert_id>` - get alert status in web UI.
* `http://<vmalert-addr>/vmalert/rule?group_id=<group_id>&rule_id=<rule_id>` - get rule status in web UI.
* `http://<vmalert-addr>/metrics` - application metrics.
* `http://<vmalert-addr>/-/reload` - hot configuration reload.
@ -1174,7 +1182,7 @@ spec:
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmalert` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmalert` binary and puts it into the `bin` folder.
@ -1190,7 +1198,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmalert-linux-arm` or `make vmalert-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmalert-linux-arm` or `vmalert-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -171,7 +171,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmauth` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmauth` binary and puts it into the `bin` folder.

View file

@ -280,7 +280,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmbackup` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmbackup` binary and puts it into the `bin` folder.

View file

@ -703,7 +703,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmctl` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmctl` binary and puts it into the `bin` folder.
@ -732,7 +732,7 @@ ARM build may run on Raspberry Pi or on [energy-efficient ARM servers](https://b
#### Development ARM build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmctl-linux-arm` or `make vmctl-linux-arm64` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmctl-linux-arm` or `vmctl-linux-arm64` binary respectively and puts it into the `bin` folder.

View file

@ -180,7 +180,7 @@ It is recommended using [binary releases](https://github.com/VictoriaMetrics/Vic
### Development build
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.18.
1. [Install Go](https://golang.org/doc/install). The minimum supported version is Go 1.19.1.
2. Run `make vmrestore` from the root folder of [the repository](https://github.com/VictoriaMetrics/VictoriaMetrics).
It builds `vmrestore` binary and puts it into the `bin` folder.

18
go.mod
View file

@ -1,6 +1,6 @@
module github.com/VictoriaMetrics/VictoriaMetrics
go 1.18
go 1.19
require (
cloud.google.com/go/storage v1.26.0
@ -10,8 +10,8 @@ require (
// like https://github.com/valyala/fasthttp/commit/996610f021ff45fdc98c2ce7884d5fa4e7f9199b
github.com/VictoriaMetrics/fasthttp v1.1.0
github.com/VictoriaMetrics/metrics v1.22.2
github.com/VictoriaMetrics/metricsql v0.44.1
github.com/aws/aws-sdk-go v1.44.93
github.com/VictoriaMetrics/metricsql v0.45.0
github.com/aws/aws-sdk-go v1.44.96
github.com/cespare/xxhash/v2 v2.1.2
github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect
@ -29,15 +29,15 @@ require (
github.com/oklog/ulid v1.3.1
github.com/prometheus/common v0.37.0 // indirect
github.com/prometheus/prometheus v1.8.2-0.20201119142752-3ad25a6dc3d9
github.com/urfave/cli/v2 v2.14.1
github.com/urfave/cli/v2 v2.16.3
github.com/valyala/fastjson v1.6.3
github.com/valyala/fastrand v1.1.0
github.com/valyala/fasttemplate v1.2.1
github.com/valyala/gozstd v1.17.0
github.com/valyala/quicktemplate v1.7.0
golang.org/x/net v0.0.0-20220907135653-1e95f45603a7
golang.org/x/oauth2 v0.0.0-20220822191816-0ebed06d0094
golang.org/x/sys v0.0.0-20220908150016-7ac13a9a928d
golang.org/x/net v0.0.0-20220909164309-bea034e7d591
golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1
golang.org/x/sys v0.0.0-20220913120320-3275c407cedc
google.golang.org/api v0.95.0
gopkg.in/yaml.v2 v2.4.0
)
@ -63,7 +63,7 @@ require (
github.com/prometheus/client_golang v1.13.0 // indirect
github.com/prometheus/client_model v0.2.0 // indirect
github.com/prometheus/procfs v0.8.0 // indirect
github.com/rivo/uniseg v0.3.4 // indirect
github.com/rivo/uniseg v0.4.2 // indirect
github.com/russross/blackfriday/v2 v2.1.0 // indirect
github.com/valyala/bytebufferpool v1.0.0 // indirect
github.com/valyala/histogram v1.2.0 // indirect
@ -75,7 +75,7 @@ require (
golang.org/x/text v0.3.7 // indirect
golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
google.golang.org/appengine v1.6.7 // indirect
google.golang.org/genproto v0.0.0-20220908141613-51c1cc9bc6d0 // indirect
google.golang.org/genproto v0.0.0-20220909194730-69f6226f97e5 // indirect
google.golang.org/grpc v1.49.0 // indirect
google.golang.org/protobuf v1.28.1 // indirect
)

31
go.sum
View file

@ -111,8 +111,8 @@ github.com/VictoriaMetrics/fasthttp v1.1.0/go.mod h1:/7DMcogqd+aaD3G3Hg5kFgoFwlR
github.com/VictoriaMetrics/metrics v1.18.1/go.mod h1:ArjwVz7WpgpegX/JpB0zpNF2h2232kErkEnzH1sxMmA=
github.com/VictoriaMetrics/metrics v1.22.2 h1:A6LsNidYwkAHetxsvNFaUWjtzu5ltdgNEoS6i7Bn+6I=
github.com/VictoriaMetrics/metrics v1.22.2/go.mod h1:rAr/llLpEnAdTehiNlUxKgnjcOuROSzpw0GvjpEbvFc=
github.com/VictoriaMetrics/metricsql v0.44.1 h1:qGoRt0g84uMUscVjS7P3uDZKmjJubWKaIx9v0iHKgck=
github.com/VictoriaMetrics/metricsql v0.44.1/go.mod h1:6pP1ZeLVJHqJrHlF6Ij3gmpQIznSsgktEcZgsAWYel0=
github.com/VictoriaMetrics/metricsql v0.45.0 h1:kVQHnkDJm4qyJ8f5msTclmwqAtlUdPbbEJ7zoa/FTNs=
github.com/VictoriaMetrics/metricsql v0.45.0/go.mod h1:6pP1ZeLVJHqJrHlF6Ij3gmpQIznSsgktEcZgsAWYel0=
github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow=
github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4=
github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g=
@ -148,8 +148,8 @@ github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQ
github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/aws/aws-sdk-go v1.34.28/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48=
github.com/aws/aws-sdk-go v1.35.31/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro=
github.com/aws/aws-sdk-go v1.44.93 h1:hAgd9fuaptBatSft27/5eBMdcA8+cIMqo96/tZ6rKl8=
github.com/aws/aws-sdk-go v1.44.93/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo=
github.com/aws/aws-sdk-go v1.44.96 h1:S9paaqnJ0AJ95t5AB+iK8RM6YNZN0W0Lek1gOVJsEr8=
github.com/aws/aws-sdk-go v1.44.96/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX4oIKwKHZo=
github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
@ -765,8 +765,8 @@ github.com/prometheus/prometheus v1.8.2-0.20201119142752-3ad25a6dc3d9/go.mod h1:
github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/retailnext/hllpp v1.0.1-0.20180308014038-101a6d2f8b52/go.mod h1:RDpi1RftBQPUCDRw6SmxeaREsAaRKnOclghuzp/WRzc=
github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc=
github.com/rivo/uniseg v0.3.4 h1:3Z3Eu6FGHZWSfNKJTOUiPatWwfc7DzJRU04jFUqJODw=
github.com/rivo/uniseg v0.3.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88=
github.com/rivo/uniseg v0.4.2 h1:YwD0ulJSJytLpiaWua0sBDusfsCZohxjxzVTYjwxfV8=
github.com/rivo/uniseg v0.4.2/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88=
github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
@ -828,8 +828,8 @@ github.com/uber/jaeger-client-go v2.25.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMW
github.com/uber/jaeger-lib v2.4.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
github.com/urfave/cli/v2 v2.14.1 h1:0Sx+C9404t2+DPuIJ3UpZFOEFhNG3wPxMj7uZHyZKFA=
github.com/urfave/cli/v2 v2.14.1/go.mod h1:1CNUng3PtjQMtRzJO4FMXBQvkGtuYRxxiR9xMa7jMwI=
github.com/urfave/cli/v2 v2.16.3 h1:gHoFIwpPjoyIMbJp/VFd+/vuD0dAgFK4B6DpEMFJfQk=
github.com/urfave/cli/v2 v2.16.3/go.mod h1:1CNUng3PtjQMtRzJO4FMXBQvkGtuYRxxiR9xMa7jMwI=
github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw=
github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
github.com/valyala/fasthttp v1.30.0/go.mod h1:2rsYD01CKFrjjsvFxx75KlEUNpWNBY9JWD3K/7o2Cus=
@ -1007,8 +1007,8 @@ golang.org/x/net v0.0.0-20220412020605-290c469a71a5/go.mod h1:CfG3xpIq0wQ8r1q4Su
golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
golang.org/x/net v0.0.0-20220907135653-1e95f45603a7 h1:1WGATo9HAhkWMbfyuVU0tEFP88OIkUvwaHFveQPvzCQ=
golang.org/x/net v0.0.0-20220907135653-1e95f45603a7/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk=
golang.org/x/net v0.0.0-20220909164309-bea034e7d591 h1:D0B/7al0LLrVC8aWF4+oxpv/m8bc7ViFfVS8/gXGdqI=
golang.org/x/net v0.0.0-20220909164309-bea034e7d591/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk=
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
@ -1029,8 +1029,9 @@ golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j
golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc=
golang.org/x/oauth2 v0.0.0-20220411215720-9780585627b5/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc=
golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE=
golang.org/x/oauth2 v0.0.0-20220822191816-0ebed06d0094 h1:2o1E+E8TpNLklK9nHiPiK1uzIYrIHt+cQx3ynCwq9V8=
golang.org/x/oauth2 v0.0.0-20220822191816-0ebed06d0094/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg=
golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1 h1:lxqLZaMad/dJHMFZH0NiNpiEZI/nhgWhe4wgzpE+MuA=
golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg=
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
@ -1141,8 +1142,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220624220833-87e55d714810/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220908150016-7ac13a9a928d h1:RoyzQTK76Rktm3p4xyZslc8T8I1tBz4UEjZCzeh57mM=
golang.org/x/sys v0.0.0-20220908150016-7ac13a9a928d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220913120320-3275c407cedc h1:dpclq5m2YrqPGStKmtw7IcNbKLfbIqKXvNxDJKdIKYc=
golang.org/x/sys v0.0.0-20220913120320-3275c407cedc/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
@ -1382,8 +1383,8 @@ google.golang.org/genproto v0.0.0-20220523171625-347a074981d8/go.mod h1:RAyBrSAP
google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
google.golang.org/genproto v0.0.0-20220624142145-8cd45d7dbd1f/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA=
google.golang.org/genproto v0.0.0-20220908141613-51c1cc9bc6d0 h1:bMz0aY2wd9TwUp9M7QfjBWuQqaFD/ZaTtvDpPDCo2Ow=
google.golang.org/genproto v0.0.0-20220908141613-51c1cc9bc6d0/go.mod h1:rQWNQYp1kbHR3+n5cARSTCF5rlJOttUn8yIhRklGAWQ=
google.golang.org/genproto v0.0.0-20220909194730-69f6226f97e5 h1:ngtP8S8JkBWfJACT9cmj5eTkS9tIWPQI5leBz/7Bq/c=
google.golang.org/genproto v0.0.0-20220909194730-69f6226f97e5/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo=
google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM=

View file

@ -10,9 +10,13 @@ import (
"github.com/VictoriaMetrics/metrics"
)
func mustRemoveAll(path string, done func()) {
// MustRemoveAll removes path with all the contents.
//
// It properly fsyncs the parent directory after path removal.
//
// It properly handles NFS issue https://github.com/VictoriaMetrics/VictoriaMetrics/issues/61 .
func MustRemoveAll(path string) {
if tryRemoveAll(path) {
done()
return
}
select {
@ -29,7 +33,6 @@ func mustRemoveAll(path string, done func()) {
for {
time.Sleep(time.Second)
if tryRemoveAll(path) {
done()
return
}
}

View file

@ -8,8 +8,10 @@ import (
"os"
"path/filepath"
"regexp"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/filestream"
@ -193,25 +195,57 @@ func IsEmptyDir(path string) bool {
return false
}
// MustRemoveAll removes path with all the contents.
// MustRemoveDirAtomic removes the given dir atomically.
//
// It properly fsyncs the parent directory after path removal.
// It uses the following algorithm:
//
// It properly handles NFS issue https://github.com/VictoriaMetrics/VictoriaMetrics/issues/61 .
func MustRemoveAll(path string) {
mustRemoveAll(path, func() {})
// 1. Atomically rename the "<dir>" to "<dir>.must-remove.<XYZ>",
// where <XYZ> is an unique number.
// 2. Remove the "<dir>.must-remove.XYZ" in background.
//
// If the process crashes after the step 1, then the directory must be removed
// on the next process start by calling MustRemoveTemporaryDirs on the parent directory.
func MustRemoveDirAtomic(dir string) {
if !IsPathExist(dir) {
return
}
n := atomic.AddUint64(&atomicDirRemoveCounter, 1)
tmpDir := fmt.Sprintf("%s.must-remove.%d", dir, n)
if err := os.Rename(dir, tmpDir); err != nil {
logger.Panicf("FATAL: cannot move %s to %s: %s", dir, tmpDir, err)
}
MustRemoveAll(tmpDir)
parentDir := filepath.Dir(dir)
MustSyncPath(parentDir)
}
// MustRemoveAllWithDoneCallback removes path with all the contents.
var atomicDirRemoveCounter = uint64(time.Now().UnixNano())
// MustRemoveTemporaryDirs removes all the subdirectories with ".must-remove.<XYZ>" suffix.
//
// It properly fsyncs the parent directory after path removal.
//
// done is called after the path is successfully removed.
//
// done may be called after the function returns for NFS path.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/61.
func MustRemoveAllWithDoneCallback(path string, done func()) {
mustRemoveAll(path, done)
// Such directories may be left on unclean shutdown during MustRemoveDirAtomic call.
func MustRemoveTemporaryDirs(dir string) {
d, err := os.Open(dir)
if err != nil {
logger.Panicf("FATAL: cannot open dir %q: %s", dir, err)
}
defer MustClose(d)
fis, err := d.Readdir(-1)
if err != nil {
logger.Panicf("FATAL: cannot read dir %q: %s", dir, err)
}
for _, fi := range fis {
if !IsDirOrSymlink(fi) {
// Skip non-directories
continue
}
dirName := fi.Name()
if strings.Contains(dirName, ".must-remove.") {
fullPath := dir + "/" + dirName
MustRemoveAll(fullPath)
}
}
MustSyncPath(dir)
}
// HardLinkFiles makes hard links for all the files from srcDir in dstDir.

View file

@ -94,7 +94,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
metaindexPath := path + "/metaindex.bin"
metaindexFile, err := filestream.Create(metaindexPath, false)
if err != nil {
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create metaindex file: %w", err)
}
@ -102,7 +102,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
indexFile, err := filestream.Create(indexPath, nocache)
if err != nil {
metaindexFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create index file: %w", err)
}
@ -111,7 +111,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
if err != nil {
metaindexFile.MustClose()
indexFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create items file: %w", err)
}
@ -121,7 +121,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
metaindexFile.MustClose()
indexFile.MustClose()
itemsFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create lens file: %w", err)
}

View file

@ -1059,6 +1059,7 @@ func openParts(path string) ([]*partWrapper, error) {
if err := fs.MkdirAllIfNotExist(path); err != nil {
return nil, err
}
fs.MustRemoveTemporaryDirs(path)
d, err := os.Open(path)
if err != nil {
return nil, fmt.Errorf("cannot open difrectory: %w", err)
@ -1073,13 +1074,13 @@ func openParts(path string) ([]*partWrapper, error) {
}
txnDir := path + "/txn"
fs.MustRemoveAll(txnDir)
fs.MustRemoveDirAtomic(txnDir)
if err := fs.MkdirAllFailIfExist(txnDir); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", txnDir, err)
}
tmpDir := path + "/tmp"
fs.MustRemoveAll(tmpDir)
fs.MustRemoveDirAtomic(tmpDir)
if err := fs.MkdirAllFailIfExist(tmpDir); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", tmpDir, err)
}
@ -1106,7 +1107,7 @@ func openParts(path string) ([]*partWrapper, error) {
if fs.IsEmptyDir(partPath) {
// Remove empty directory, which can be left after unclean shutdown on NFS.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1142
fs.MustRemoveAll(partPath)
fs.MustRemoveDirAtomic(partPath)
continue
}
p, err := openFilePart(partPath)
@ -1277,14 +1278,12 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix, txnPath string) error {
}
// Remove old paths. It is OK if certain paths don't exist.
var removeWG sync.WaitGroup
for _, path := range rmPaths {
path, err := validatePath(pathPrefix, path)
if err != nil {
return fmt.Errorf("invalid path to remove: %w", err)
}
removeWG.Add(1)
fs.MustRemoveAllWithDoneCallback(path, removeWG.Done)
fs.MustRemoveDirAtomic(path)
}
// Move the new part to new directory.
@ -1316,9 +1315,6 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix, txnPath string) error {
pendingTxnDeletionsWG.Add(1)
go func() {
defer pendingTxnDeletionsWG.Done()
// Remove the transaction file only after all the source paths are deleted.
// This is required for NFS mounts. See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/61 .
removeWG.Wait()
if err := os.Remove(txnPath); err != nil {
logger.Errorf("cannot remove transaction file %q: %s", txnPath, err)
}

View file

@ -702,7 +702,7 @@ func (uw *urlWatcher) readObjectUpdateStream(r io.Reader) error {
var we WatchEvent
for {
if err := d.Decode(&we); err != nil {
return err
return fmt.Errorf("cannot parse WatchEvent json response: %s", err)
}
switch we.Type {
case "ADDED", "MODIFIED":

View file

@ -516,19 +516,35 @@ func (sw *scrapeWork) pushData(at *auth.Token, wr *prompbmarshal.WriteRequest) {
}
type streamBodyReader struct {
sr *streamReader
body []byte
bodyLen int
captureBody bool
readOffset int
}
func (sbr *streamBodyReader) Init(sr *streamReader) error {
sbr.body = nil
sbr.bodyLen = 0
sbr.readOffset = 0
// Read the whole response body in memory before parsing it in stream mode.
// This minimizes the time needed for reading response body from scrape target.
startTime := fasttime.UnixTimestamp()
body, err := io.ReadAll(sr)
if err != nil {
d := fasttime.UnixTimestamp() - startTime
return fmt.Errorf("cannot read stream body in %d seconds: %w", d, err)
}
sbr.body = body
sbr.bodyLen = len(body)
return nil
}
func (sbr *streamBodyReader) Read(b []byte) (int, error) {
n, err := sbr.sr.Read(b)
sbr.bodyLen += n
if sbr.captureBody {
sbr.body = append(sbr.body, b[:n]...)
if sbr.readOffset >= len(sbr.body) {
return 0, io.EOF
}
return n, err
n := copy(b, sbr.body[sbr.readOffset:])
sbr.readOffset += n
return n, nil
}
func (sw *scrapeWork) scrapeStream(scrapeTimestamp, realTimestamp int64) error {
@ -536,17 +552,16 @@ func (sw *scrapeWork) scrapeStream(scrapeTimestamp, realTimestamp int64) error {
samplesPostRelabeling := 0
wc := writeRequestCtxPool.Get(sw.prevLabelsLen)
// Do not pool sbr and do not pre-allocate sbr.body in order to reduce memory usage when scraping big responses.
sbr := &streamBodyReader{
captureBody: !*noStaleMarkers,
}
var sbr streamBodyReader
sr, err := sw.GetStreamReader()
if err != nil {
err = fmt.Errorf("cannot read data: %s", err)
} else {
var mu sync.Mutex
sbr.sr = sr
err = parser.ParseStream(sbr, scrapeTimestamp, false, func(rows []parser.Row) error {
err = sbr.Init(sr)
if err == nil {
err = parser.ParseStream(&sbr, scrapeTimestamp, false, func(rows []parser.Row) error {
mu.Lock()
defer mu.Unlock()
samplesScraped += len(rows)
@ -567,6 +582,7 @@ func (sw *scrapeWork) scrapeStream(scrapeTimestamp, realTimestamp int64) error {
wc.resetNoRows()
return nil
}, sw.logError)
}
sr.MustClose()
}
lastScrape := sw.loadLastScrape()

View file

@ -110,7 +110,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
timestampsPath := path + "/timestamps.bin"
timestampsFile, err := filestream.Create(timestampsPath, nocache)
if err != nil {
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create timestamps file: %w", err)
}
@ -118,7 +118,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
valuesFile, err := filestream.Create(valuesPath, nocache)
if err != nil {
timestampsFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create values file: %w", err)
}
@ -127,7 +127,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
if err != nil {
timestampsFile.MustClose()
valuesFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create index file: %w", err)
}
@ -139,7 +139,7 @@ func (bsw *blockStreamWriter) InitFromFilePart(path string, nocache bool, compre
timestampsFile.MustClose()
valuesFile.MustClose()
indexFile.MustClose()
fs.MustRemoveAll(path)
fs.MustRemoveDirAtomic(path)
return fmt.Errorf("cannot create metaindex file: %w", err)
}

View file

@ -308,7 +308,7 @@ func (db *indexDB) decRef() {
}
logger.Infof("dropping indexDB %q", tbPath)
fs.MustRemoveAll(tbPath)
fs.MustRemoveDirAtomic(tbPath)
logger.Infof("indexDB %q has been dropped", tbPath)
}

View file

@ -2080,5 +2080,5 @@ func stopTestStorage(s *Storage) {
s.metricIDCache.Stop()
s.metricNameCache.Stop()
s.tsidCache.Stop()
fs.MustRemoveAll(s.cachePath)
fs.MustRemoveDirAtomic(s.cachePath)
}

View file

@ -236,8 +236,8 @@ func (pt *partition) Drop() {
// Wait until all the pending transaction deletions are finished before removing partition directories.
pendingTxnDeletionsWG.Wait()
fs.MustRemoveAll(pt.smallPartsPath)
fs.MustRemoveAll(pt.bigPartsPath)
fs.MustRemoveDirAtomic(pt.smallPartsPath)
fs.MustRemoveDirAtomic(pt.bigPartsPath)
logger.Infof("partition %q has been dropped", pt.name)
}
@ -1394,17 +1394,14 @@ func (pt *partition) removeStaleParts() {
}
// Physically remove stale parts under snapshotLock in order to provide
// consistent snapshots with partition.CreateSnapshot().
// consistent snapshots with table.CreateSnapshot().
pt.snapshotLock.RLock()
var removeWG sync.WaitGroup
for pw := range m {
logger.Infof("removing part %q, since its data is out of the configured retention (%d secs)", pw.p.path, pt.retentionMsecs/1000)
removeWG.Add(1)
fs.MustRemoveAllWithDoneCallback(pw.p.path, removeWG.Done)
fs.MustRemoveDirAtomic(pw.p.path)
}
removeWG.Wait()
// There is no need in calling fs.MustSyncPath() on pt.smallPartsPath and pt.bigPartsPath,
// since they should be automatically called inside fs.MustRemoveAllWithDoneCallback.
// since they should be automatically called inside fs.MustRemoveDirAtomic().
pt.snapshotLock.RUnlock()
@ -1554,6 +1551,7 @@ func openParts(pathPrefix1, pathPrefix2, path string) ([]*partWrapper, error) {
if err := fs.MkdirAllIfNotExist(path); err != nil {
return nil, err
}
fs.MustRemoveTemporaryDirs(path)
d, err := os.Open(path)
if err != nil {
return nil, fmt.Errorf("cannot open directory %q: %w", path, err)
@ -1568,9 +1566,9 @@ func openParts(pathPrefix1, pathPrefix2, path string) ([]*partWrapper, error) {
}
txnDir := path + "/txn"
fs.MustRemoveAll(txnDir)
fs.MustRemoveDirAtomic(txnDir)
tmpDir := path + "/tmp"
fs.MustRemoveAll(tmpDir)
fs.MustRemoveDirAtomic(tmpDir)
if err := createPartitionDirs(path); err != nil {
return nil, fmt.Errorf("cannot create directories for partition %q: %w", path, err)
}
@ -1596,7 +1594,7 @@ func openParts(pathPrefix1, pathPrefix2, path string) ([]*partWrapper, error) {
if fs.IsEmptyDir(partPath) {
// Remove empty directory, which can be left after unclean shutdown on NFS.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1142
fs.MustRemoveAll(partPath)
fs.MustRemoveDirAtomic(partPath)
continue
}
startTime := time.Now()
@ -1761,14 +1759,12 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix1, pathPrefix2, txnPath str
}
// Remove old paths. It is OK if certain paths don't exist.
var removeWG sync.WaitGroup
for _, path := range rmPaths {
path, err := validatePath(pathPrefix1, pathPrefix2, path)
if err != nil {
return fmt.Errorf("invalid path to remove: %w", err)
}
removeWG.Add(1)
fs.MustRemoveAllWithDoneCallback(path, removeWG.Done)
fs.MustRemoveDirAtomic(path)
}
// Move the new part to new directory.
@ -1797,8 +1793,7 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix1, pathPrefix2, txnPath str
}
} else {
// Just remove srcPath.
removeWG.Add(1)
fs.MustRemoveAllWithDoneCallback(srcPath, removeWG.Done)
fs.MustRemoveDirAtomic(srcPath)
}
// Flush pathPrefix* directory metadata to the underying storage,
@ -1809,12 +1804,9 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix1, pathPrefix2, txnPath str
pendingTxnDeletionsWG.Add(1)
go func() {
defer pendingTxnDeletionsWG.Done()
// Remove the transaction file only after all the source paths are deleted.
// This is required for NFS mounts. See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/61 .
removeWG.Wait()
// There is no need in calling fs.MustSyncPath for pathPrefix* after parts' removal,
// since it is already called by fs.MustRemoveAllWithDoneCallback.
// since it is already called by fs.MustRemoveDirAtomic.
if err := os.Remove(txnPath); err != nil {
logger.Errorf("cannot remove transaction file %q: %s", txnPath, err)

View file

@ -169,11 +169,10 @@ func OpenStorage(path string, retentionMsecs int64, maxHourlySeries, maxDailySer
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1447 for details.
if fs.IsPathExist(s.cachePath + "/reset_cache_on_startup") {
logger.Infof("removing cache directory at %q, since it contains `reset_cache_on_startup` file...", s.cachePath)
wg := getWaitGroup()
wg.Add(1)
fs.MustRemoveAllWithDoneCallback(s.cachePath, wg.Done)
wg.Wait()
putWaitGroup(wg)
// Do not use fs.MustRemoveDirAtomic() here, since the cache directory may be mounted
// to a separate filesystem. In this case the fs.MustRemoveDirAtomic() will fail while
// trying to remove the mount root.
fs.RemoveDirContents(s.cachePath)
logger.Infof("cache directory at %q has been successfully removed", s.cachePath)
}
@ -195,6 +194,7 @@ func OpenStorage(path string, retentionMsecs int64, maxHourlySeries, maxDailySer
if err := fs.MkdirAllIfNotExist(snapshotsPath); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", snapshotsPath, err)
}
fs.MustRemoveTemporaryDirs(snapshotsPath)
// Initialize series cardinality limiter.
if maxHourlySeries > 0 {
@ -239,6 +239,7 @@ func OpenStorage(path string, retentionMsecs int64, maxHourlySeries, maxDailySer
if err := fs.MkdirAllIfNotExist(idbSnapshotsPath); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", idbSnapshotsPath, err)
}
fs.MustRemoveTemporaryDirs(idbSnapshotsPath)
idbCurr, idbPrev, err := s.openIndexDBTables(idbPath)
if err != nil {
return nil, fmt.Errorf("cannot open indexdb tables at %q: %w", idbPath, err)
@ -411,8 +412,8 @@ func (s *Storage) DeleteSnapshot(snapshotName string) error {
s.tb.MustDeleteSnapshot(snapshotName)
idbPath := fmt.Sprintf("%s/indexdb/snapshots/%s", s.path, snapshotName)
fs.MustRemoveAll(idbPath)
fs.MustRemoveAll(snapshotPath)
fs.MustRemoveDirAtomic(idbPath)
fs.MustRemoveDirAtomic(snapshotPath)
logger.Infof("deleted snapshot %q in %.3f seconds", snapshotPath, time.Since(startTime).Seconds())
@ -2449,6 +2450,7 @@ func (s *Storage) openIndexDBTables(path string) (curr, prev *indexDB, err error
if err := fs.MkdirAllIfNotExist(path); err != nil {
return nil, nil, fmt.Errorf("cannot create directory %q: %w", path, err)
}
fs.MustRemoveTemporaryDirs(path)
d, err := os.Open(path)
if err != nil {
@ -2494,7 +2496,7 @@ func (s *Storage) openIndexDBTables(path string) (curr, prev *indexDB, err error
for _, tn := range tableNames[:len(tableNames)-2] {
pathToRemove := path + "/" + tn
logger.Infof("removing obsolete indexdb dir %q...", pathToRemove)
fs.MustRemoveAll(pathToRemove)
fs.MustRemoveDirAtomic(pathToRemove)
logger.Infof("removed obsolete indexdb dir %q", pathToRemove)
}

View file

@ -103,18 +103,23 @@ func openTable(path string, getDeletedMetricIDs func() *uint64set.Set, retention
if err := fs.MkdirAllIfNotExist(smallPartitionsPath); err != nil {
return nil, fmt.Errorf("cannot create directory for small partitions %q: %w", smallPartitionsPath, err)
}
fs.MustRemoveTemporaryDirs(smallPartitionsPath)
smallSnapshotsPath := smallPartitionsPath + "/snapshots"
if err := fs.MkdirAllIfNotExist(smallSnapshotsPath); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", smallSnapshotsPath, err)
}
fs.MustRemoveTemporaryDirs(smallSnapshotsPath)
bigPartitionsPath := path + "/big"
if err := fs.MkdirAllIfNotExist(bigPartitionsPath); err != nil {
return nil, fmt.Errorf("cannot create directory for big partitions %q: %w", bigPartitionsPath, err)
}
fs.MustRemoveTemporaryDirs(bigPartitionsPath)
bigSnapshotsPath := bigPartitionsPath + "/snapshots"
if err := fs.MkdirAllIfNotExist(bigSnapshotsPath); err != nil {
return nil, fmt.Errorf("cannot create %q: %w", bigSnapshotsPath, err)
}
fs.MustRemoveTemporaryDirs(bigSnapshotsPath)
// Open partitions.
pts, err := openPartitions(smallPartitionsPath, bigPartitionsPath, getDeletedMetricIDs, retentionMsecs, isReadOnly)
@ -179,9 +184,9 @@ func (tb *table) CreateSnapshot(snapshotName string) (string, string, error) {
// MustDeleteSnapshot deletes snapshot with the given snapshotName.
func (tb *table) MustDeleteSnapshot(snapshotName string) {
smallDir := fmt.Sprintf("%s/small/snapshots/%s", tb.path, snapshotName)
fs.MustRemoveAll(smallDir)
fs.MustRemoveDirAtomic(smallDir)
bigDir := fmt.Sprintf("%s/big/snapshots/%s", tb.path, snapshotName)
fs.MustRemoveAll(bigDir)
fs.MustRemoveDirAtomic(bigDir)
}
func (tb *table) addPartitionNolock(pt *partition) {

View file

@ -11,5 +11,4 @@
// }
// // Now expr contains parsed MetricsQL as `*Expr` structs.
// // See Parse examples for more details.
//
package metricsql

View file

@ -91,6 +91,8 @@ var transformFuncs = map[string]bool{
"sort": true,
"sort_by_label": true,
"sort_by_label_desc": true,
"sort_by_label_numeric": true,
"sort_by_label_numeric_desc": true,
"sort_desc": true,
"sqrt": true,
"start": true,

View file

@ -6661,12 +6661,21 @@ var awsPartition = partition{
endpointKey{
Region: "ap-northeast-1",
}: endpoint{},
endpointKey{
Region: "ap-northeast-2",
}: endpoint{},
endpointKey{
Region: "ap-south-1",
}: endpoint{},
endpointKey{
Region: "ap-southeast-1",
}: endpoint{},
endpointKey{
Region: "ap-southeast-2",
}: endpoint{},
endpointKey{
Region: "ca-central-1",
}: endpoint{},
endpointKey{
Region: "eu-central-1",
}: endpoint{},
@ -6676,6 +6685,12 @@ var awsPartition = partition{
endpointKey{
Region: "eu-west-1",
}: endpoint{},
endpointKey{
Region: "eu-west-2",
}: endpoint{},
endpointKey{
Region: "eu-west-3",
}: endpoint{},
endpointKey{
Region: "fips-us-east-1",
}: endpoint{
@ -6703,6 +6718,9 @@ var awsPartition = partition{
},
Deprecated: boxedTrue,
},
endpointKey{
Region: "sa-east-1",
}: endpoint{},
endpointKey{
Region: "us-east-1",
}: endpoint{},
@ -6721,6 +6739,9 @@ var awsPartition = partition{
}: endpoint{
Hostname: "devops-guru-fips.us-east-2.amazonaws.com",
},
endpointKey{
Region: "us-west-1",
}: endpoint{},
endpointKey{
Region: "us-west-2",
}: endpoint{},
@ -30127,6 +30148,13 @@ var awsusgovPartition = partition{
},
},
},
"managedblockchain": service{
Endpoints: serviceEndpoints{
endpointKey{
Region: "us-gov-west-1",
}: endpoint{},
},
},
"mediaconvert": service{
Endpoints: serviceEndpoints{
endpointKey{

View file

@ -5,4 +5,4 @@ package aws
const SDKName = "aws-sdk-go"
// SDKVersion is the version of this SDK
const SDKVersion = "1.44.93"
const SDKVersion = "1.44.96"

View file

@ -3,13 +3,13 @@
[![Go Reference](https://pkg.go.dev/badge/github.com/rivo/uniseg.svg)](https://pkg.go.dev/github.com/rivo/uniseg)
[![Go Report](https://img.shields.io/badge/go%20report-A%2B-brightgreen.svg)](https://goreportcard.com/report/github.com/rivo/uniseg)
This Go package implements Unicode Text Segmentation according to [Unicode Standard Annex #29](https://unicode.org/reports/tr29/) and Unicode Line Breaking according to [Unicode Standard Annex #14](https://unicode.org/reports/tr14/) (Unicode version 14.0.0).
This Go package implements Unicode Text Segmentation according to [Unicode Standard Annex #29](https://unicode.org/reports/tr29/), Unicode Line Breaking according to [Unicode Standard Annex #14](https://unicode.org/reports/tr14/) (Unicode version 14.0.0), and monospace font string width calculation similar to [wcwidth](https://man7.org/linux/man-pages/man3/wcwidth.3.html).
## Background
### Grapheme Clusters
In Go, [strings are read-only slices of bytes](https://blog.golang.org/strings). They can be turned into Unicode code points using the `for` loop or by casting: `[]rune(str)`. However, multiple code points may be combined into one user-perceived character or what the Unicode specification calls "grapheme cluster". Here are some examples:
In Go, [strings are read-only slices of bytes](https://go.dev/blog/strings). They can be turned into Unicode code points using the `for` loop or by casting: `[]rune(str)`. However, multiple code points may be combined into one user-perceived character or what the Unicode specification calls "grapheme cluster". Here are some examples:
|String|Bytes (UTF-8)|Code points (runes)|Grapheme clusters|
|-|-|-|-|
@ -31,6 +31,10 @@ Sentence boundaries are often used for triple-click or some other method of sele
Line breaking, also known as word wrapping, is the process of breaking a section of text into lines such that it will fit in the available width of a page, window or other display area. This package provides tools to determine where a string may or may not be broken and where it must be broken (for example after newline characters).
### Monospace Width
Most terminals or text displays / text editors using a monospace font (for example source code editors) use a fixed width for each character. Some characters such as emojis or characters found in Asian and other languages may take up more than one character cell. This package provides tools to determine the number of cells a string will take up when displayed in a monospace font. See [here](https://pkg.go.dev/github.com/rivo/uniseg#hdr-Monospace_Width) for more information.
## Installation
```bash
@ -47,6 +51,14 @@ fmt.Println(n)
// 2
```
### Calculating the Monospace String Width
```go
width := uniseg.StringWidth("🇩🇪🏳️‍🌈!")
fmt.Println(width)
// 5
```
### Using the [`Graphemes`](https://pkg.go.dev/github.com/rivo/uniseg#Graphemes) Class
This is the most convenient method of iterating over grapheme clusters:

77
vendor/github.com/rivo/uniseg/doc.go generated vendored
View file

@ -1,8 +1,9 @@
/*
Package uniseg implements Unicode Text Segmentation and Unicode Line Breaking.
Unicode Text Segmentation conforms to Unicode Standard Annex #29
(https://unicode.org/reports/tr29/) and Unicode Line Breaking conforms to
Unicode Standard Annex #14 (https://unicode.org/reports/tr14/).
Package uniseg implements Unicode Text Segmentation, Unicode Line Breaking, and
string width calculation for monospace fonts. Unicode Text Segmentation conforms
to Unicode Standard Annex #29 (https://unicode.org/reports/tr29/) and Unicode
Line Breaking conforms to Unicode Standard Annex #14
(https://unicode.org/reports/tr14/).
In short, using this package, you can split a string into grapheme clusters
(what people would usually refer to as a "character"), into words, and into
@ -12,8 +13,23 @@ as emojis, combining characters, or characters from Asian, Arabic, Hebrew, or
other languages. Additionally, you can use it to implement line breaking (or
"word wrapping"), that is, to determine where text can be broken over to the
next line when the width of the line is not big enough to fit the entire text.
Finally, you can use it to calculate the display width of a string for monospace
fonts.
Grapheme Clusters
# Getting Started
If you just want to count the number of characters in a string, you can use
[GraphemeClusterCount]. If you want to determine the display width of a string,
you can use [StringWidth]. If you want to iterate over a string, you can use
[Step], [StepString], or the [Graphemes] class (more convenient but less
performant). This will provide you with all information: grapheme clusters,
word boundaries, sentence boundaries, line breaks, and monospace character
widths. The specialized functions [FirstGraphemeCluster],
[FirstGraphemeClusterInString], [FirstWord], [FirstWordInString],
[FirstSentence], and [FirstSentenceInString] can be used if only one type of
information is needed.
# Grapheme Clusters
Consider the rainbow flag emoji: 🏳🌈. On most modern systems, it appears as one
character. But its string representation actually has 14 bytes, so counting
@ -21,11 +37,11 @@ bytes (or using len("🏳️‍🌈")) will not work as expected. Counting runes
either: The flag has 4 Unicode code points, thus 4 runes. The stdlib function
utf8.RuneCountInString("🏳️‍🌈") and len([]rune("🏳️‍🌈")) will both return 4.
The uniseg.GraphemeClusterCount(str) function will return 1 for the rainbow flag
emoji. The Graphemes class and a variety of functions in this package will allow
you to split strings into its grapheme clusters.
The [GraphemeClusterCount] function will return 1 for the rainbow flag emoji.
The Graphemes class and a variety of functions in this package will allow you to
split strings into its grapheme clusters.
Word Boundaries
# Word Boundaries
Word boundaries are used in a number of different contexts. The most familiar
ones are selection (double-click mouse selection), cursor movement ("move to
@ -33,7 +49,7 @@ next word" control-arrow keys), and the dialog option "Whole Word Search" for
search and replace. This package provides methods for determining word
boundaries.
Sentence Boundaries
# Sentence Boundaries
Sentence boundaries are often used for triple-click or some other method of
selecting or iterating through blocks of text that are larger than single words.
@ -41,7 +57,7 @@ They are also used to determine whether words occur within the same sentence in
database queries. This package provides methods for determining sentence
boundaries.
Line Breaking
# Line Breaking
Line breaking, also known as word wrapping, is the process of breaking a section
of text into lines such that it will fit in the available width of a page,
@ -49,5 +65,44 @@ window or other display area. This package provides methods to determine the
positions in a string where a line must be broken, may be broken, or must not be
broken.
# Monospace Width
Monospace width, as referred to in this package, is the width of a string in a
monospace font. This is commonly used in terminal user interfaces or text
displays or editors that don't support proportional fonts. A width of 1
corresponds to a single character cell. The C function [wcwidth()] and its
implementation in other programming languages is in widespread use for the same
purpose. However, there is no standard for the calculation of such widths, and
this package differs from wcwidth() in a number of ways, presumably to generate
more visually pleasing results.
To start, we assume that every code point has a width of 1, with the following
exceptions:
- Code points with grapheme cluster break properties Control, CR, LF, Extend,
and ZWJ have a width of 0.
- U+2E3A, Two-Em Dash, has a width of 3.
- U+2E3B, Three-Em Dash, has a width of 4.
- Characters with the East-Asian Width properties "Fullwidth" (F) and "Wide"
(W) have a width of 2. (Properties "Ambiguous" (A) and "Neutral" (N) both
have a width of 1.)
- Code points with grapheme cluster break property Regional Indicator have a
width of 2.
- Code points with grapheme cluster break property Extended Pictographic have
a width of 2, unless their Emoji Presentation flag is "No", in which case
the width is 1.
For Hangul grapheme clusters composed of conjoining Jamo and for Regional
Indicators (flags), all code points except the first one have a width of 0. For
grapheme clusters starting with an Extended Pictographic, any additional code
point will force a total width of 2, except if the Variation Selector-15
(U+FE0E) is included, in which case the total width is always 1. Grapheme
clusters ending with Variation Selector-16 (U+FE0F) have a width of 2.
Note that whether these widths appear correct depends on your application's
render engine, to which extent it conforms to the Unicode Standard, and its
choice of font.
[wcwidth()]: https://man7.org/linux/man-pages/man3/wcwidth.3.html
*/
package uniseg

View file

@ -4,7 +4,10 @@ package uniseg
// eastAsianWidth are taken from
// https://www.unicode.org/Public/14.0.0/ucd/EastAsianWidth.txt
// on July 25, 2022. See https://www.unicode.org/license.html for the Unicode
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var eastAsianWidth = [][3]int{
{0x0000, 0x001F, prN}, // Cc [32] <control-0000>..<control-001F>

285
vendor/github.com/rivo/uniseg/emojipresentation.go generated vendored Normal file
View file

@ -0,0 +1,285 @@
package uniseg
// Code generated via go generate from gen_properties.go. DO NOT EDIT.
// emojiPresentation are taken from
//
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var emojiPresentation = [][3]int{
{0x231A, 0x231B, prEmojiPresentation}, // E0.6 [2] (⌚..⌛) watch..hourglass done
{0x23E9, 0x23EC, prEmojiPresentation}, // E0.6 [4] (⏩..⏬) fast-forward button..fast down button
{0x23F0, 0x23F0, prEmojiPresentation}, // E0.6 [1] (⏰) alarm clock
{0x23F3, 0x23F3, prEmojiPresentation}, // E0.6 [1] (⏳) hourglass not done
{0x25FD, 0x25FE, prEmojiPresentation}, // E0.6 [2] (◽..◾) white medium-small square..black medium-small square
{0x2614, 0x2615, prEmojiPresentation}, // E0.6 [2] (☔..☕) umbrella with rain drops..hot beverage
{0x2648, 0x2653, prEmojiPresentation}, // E0.6 [12] (♈..♓) Aries..Pisces
{0x267F, 0x267F, prEmojiPresentation}, // E0.6 [1] (♿) wheelchair symbol
{0x2693, 0x2693, prEmojiPresentation}, // E0.6 [1] (⚓) anchor
{0x26A1, 0x26A1, prEmojiPresentation}, // E0.6 [1] (⚡) high voltage
{0x26AA, 0x26AB, prEmojiPresentation}, // E0.6 [2] (⚪..⚫) white circle..black circle
{0x26BD, 0x26BE, prEmojiPresentation}, // E0.6 [2] (⚽..⚾) soccer ball..baseball
{0x26C4, 0x26C5, prEmojiPresentation}, // E0.6 [2] (⛄..⛅) snowman without snow..sun behind cloud
{0x26CE, 0x26CE, prEmojiPresentation}, // E0.6 [1] (⛎) Ophiuchus
{0x26D4, 0x26D4, prEmojiPresentation}, // E0.6 [1] (⛔) no entry
{0x26EA, 0x26EA, prEmojiPresentation}, // E0.6 [1] (⛪) church
{0x26F2, 0x26F3, prEmojiPresentation}, // E0.6 [2] (⛲..⛳) fountain..flag in hole
{0x26F5, 0x26F5, prEmojiPresentation}, // E0.6 [1] (⛵) sailboat
{0x26FA, 0x26FA, prEmojiPresentation}, // E0.6 [1] (⛺) tent
{0x26FD, 0x26FD, prEmojiPresentation}, // E0.6 [1] (⛽) fuel pump
{0x2705, 0x2705, prEmojiPresentation}, // E0.6 [1] (✅) check mark button
{0x270A, 0x270B, prEmojiPresentation}, // E0.6 [2] (✊..✋) raised fist..raised hand
{0x2728, 0x2728, prEmojiPresentation}, // E0.6 [1] (✨) sparkles
{0x274C, 0x274C, prEmojiPresentation}, // E0.6 [1] (❌) cross mark
{0x274E, 0x274E, prEmojiPresentation}, // E0.6 [1] (❎) cross mark button
{0x2753, 0x2755, prEmojiPresentation}, // E0.6 [3] (❓..❕) red question mark..white exclamation mark
{0x2757, 0x2757, prEmojiPresentation}, // E0.6 [1] (❗) red exclamation mark
{0x2795, 0x2797, prEmojiPresentation}, // E0.6 [3] (..➗) plus..divide
{0x27B0, 0x27B0, prEmojiPresentation}, // E0.6 [1] (➰) curly loop
{0x27BF, 0x27BF, prEmojiPresentation}, // E1.0 [1] (➿) double curly loop
{0x2B1B, 0x2B1C, prEmojiPresentation}, // E0.6 [2] (⬛..⬜) black large square..white large square
{0x2B50, 0x2B50, prEmojiPresentation}, // E0.6 [1] (⭐) star
{0x2B55, 0x2B55, prEmojiPresentation}, // E0.6 [1] (⭕) hollow red circle
{0x1F004, 0x1F004, prEmojiPresentation}, // E0.6 [1] (🀄) mahjong red dragon
{0x1F0CF, 0x1F0CF, prEmojiPresentation}, // E0.6 [1] (🃏) joker
{0x1F18E, 0x1F18E, prEmojiPresentation}, // E0.6 [1] (🆎) AB button (blood type)
{0x1F191, 0x1F19A, prEmojiPresentation}, // E0.6 [10] (🆑..🆚) CL button..VS button
{0x1F1E6, 0x1F1FF, prEmojiPresentation}, // E0.0 [26] (🇦..🇿) regional indicator symbol letter a..regional indicator symbol letter z
{0x1F201, 0x1F201, prEmojiPresentation}, // E0.6 [1] (🈁) Japanese “here” button
{0x1F21A, 0x1F21A, prEmojiPresentation}, // E0.6 [1] (🈚) Japanese “free of charge” button
{0x1F22F, 0x1F22F, prEmojiPresentation}, // E0.6 [1] (🈯) Japanese “reserved” button
{0x1F232, 0x1F236, prEmojiPresentation}, // E0.6 [5] (🈲..🈶) Japanese “prohibited” button..Japanese “not free of charge” button
{0x1F238, 0x1F23A, prEmojiPresentation}, // E0.6 [3] (🈸..🈺) Japanese “application” button..Japanese “open for business” button
{0x1F250, 0x1F251, prEmojiPresentation}, // E0.6 [2] (🉐..🉑) Japanese “bargain” button..Japanese “acceptable” button
{0x1F300, 0x1F30C, prEmojiPresentation}, // E0.6 [13] (🌀..🌌) cyclone..milky way
{0x1F30D, 0x1F30E, prEmojiPresentation}, // E0.7 [2] (🌍..🌎) globe showing Europe-Africa..globe showing Americas
{0x1F30F, 0x1F30F, prEmojiPresentation}, // E0.6 [1] (🌏) globe showing Asia-Australia
{0x1F310, 0x1F310, prEmojiPresentation}, // E1.0 [1] (🌐) globe with meridians
{0x1F311, 0x1F311, prEmojiPresentation}, // E0.6 [1] (🌑) new moon
{0x1F312, 0x1F312, prEmojiPresentation}, // E1.0 [1] (🌒) waxing crescent moon
{0x1F313, 0x1F315, prEmojiPresentation}, // E0.6 [3] (🌓..🌕) first quarter moon..full moon
{0x1F316, 0x1F318, prEmojiPresentation}, // E1.0 [3] (🌖..🌘) waning gibbous moon..waning crescent moon
{0x1F319, 0x1F319, prEmojiPresentation}, // E0.6 [1] (🌙) crescent moon
{0x1F31A, 0x1F31A, prEmojiPresentation}, // E1.0 [1] (🌚) new moon face
{0x1F31B, 0x1F31B, prEmojiPresentation}, // E0.6 [1] (🌛) first quarter moon face
{0x1F31C, 0x1F31C, prEmojiPresentation}, // E0.7 [1] (🌜) last quarter moon face
{0x1F31D, 0x1F31E, prEmojiPresentation}, // E1.0 [2] (🌝..🌞) full moon face..sun with face
{0x1F31F, 0x1F320, prEmojiPresentation}, // E0.6 [2] (🌟..🌠) glowing star..shooting star
{0x1F32D, 0x1F32F, prEmojiPresentation}, // E1.0 [3] (🌭..🌯) hot dog..burrito
{0x1F330, 0x1F331, prEmojiPresentation}, // E0.6 [2] (🌰..🌱) chestnut..seedling
{0x1F332, 0x1F333, prEmojiPresentation}, // E1.0 [2] (🌲..🌳) evergreen tree..deciduous tree
{0x1F334, 0x1F335, prEmojiPresentation}, // E0.6 [2] (🌴..🌵) palm tree..cactus
{0x1F337, 0x1F34A, prEmojiPresentation}, // E0.6 [20] (🌷..🍊) tulip..tangerine
{0x1F34B, 0x1F34B, prEmojiPresentation}, // E1.0 [1] (🍋) lemon
{0x1F34C, 0x1F34F, prEmojiPresentation}, // E0.6 [4] (🍌..🍏) banana..green apple
{0x1F350, 0x1F350, prEmojiPresentation}, // E1.0 [1] (🍐) pear
{0x1F351, 0x1F37B, prEmojiPresentation}, // E0.6 [43] (🍑..🍻) peach..clinking beer mugs
{0x1F37C, 0x1F37C, prEmojiPresentation}, // E1.0 [1] (🍼) baby bottle
{0x1F37E, 0x1F37F, prEmojiPresentation}, // E1.0 [2] (🍾..🍿) bottle with popping cork..popcorn
{0x1F380, 0x1F393, prEmojiPresentation}, // E0.6 [20] (🎀..🎓) ribbon..graduation cap
{0x1F3A0, 0x1F3C4, prEmojiPresentation}, // E0.6 [37] (🎠..🏄) carousel horse..person surfing
{0x1F3C5, 0x1F3C5, prEmojiPresentation}, // E1.0 [1] (🏅) sports medal
{0x1F3C6, 0x1F3C6, prEmojiPresentation}, // E0.6 [1] (🏆) trophy
{0x1F3C7, 0x1F3C7, prEmojiPresentation}, // E1.0 [1] (🏇) horse racing
{0x1F3C8, 0x1F3C8, prEmojiPresentation}, // E0.6 [1] (🏈) american football
{0x1F3C9, 0x1F3C9, prEmojiPresentation}, // E1.0 [1] (🏉) rugby football
{0x1F3CA, 0x1F3CA, prEmojiPresentation}, // E0.6 [1] (🏊) person swimming
{0x1F3CF, 0x1F3D3, prEmojiPresentation}, // E1.0 [5] (🏏..🏓) cricket game..ping pong
{0x1F3E0, 0x1F3E3, prEmojiPresentation}, // E0.6 [4] (🏠..🏣) house..Japanese post office
{0x1F3E4, 0x1F3E4, prEmojiPresentation}, // E1.0 [1] (🏤) post office
{0x1F3E5, 0x1F3F0, prEmojiPresentation}, // E0.6 [12] (🏥..🏰) hospital..castle
{0x1F3F4, 0x1F3F4, prEmojiPresentation}, // E1.0 [1] (🏴) black flag
{0x1F3F8, 0x1F407, prEmojiPresentation}, // E1.0 [16] (🏸..🐇) badminton..rabbit
{0x1F408, 0x1F408, prEmojiPresentation}, // E0.7 [1] (🐈) cat
{0x1F409, 0x1F40B, prEmojiPresentation}, // E1.0 [3] (🐉..🐋) dragon..whale
{0x1F40C, 0x1F40E, prEmojiPresentation}, // E0.6 [3] (🐌..🐎) snail..horse
{0x1F40F, 0x1F410, prEmojiPresentation}, // E1.0 [2] (🐏..🐐) ram..goat
{0x1F411, 0x1F412, prEmojiPresentation}, // E0.6 [2] (🐑..🐒) ewe..monkey
{0x1F413, 0x1F413, prEmojiPresentation}, // E1.0 [1] (🐓) rooster
{0x1F414, 0x1F414, prEmojiPresentation}, // E0.6 [1] (🐔) chicken
{0x1F415, 0x1F415, prEmojiPresentation}, // E0.7 [1] (🐕) dog
{0x1F416, 0x1F416, prEmojiPresentation}, // E1.0 [1] (🐖) pig
{0x1F417, 0x1F429, prEmojiPresentation}, // E0.6 [19] (🐗..🐩) boar..poodle
{0x1F42A, 0x1F42A, prEmojiPresentation}, // E1.0 [1] (🐪) camel
{0x1F42B, 0x1F43E, prEmojiPresentation}, // E0.6 [20] (🐫..🐾) two-hump camel..paw prints
{0x1F440, 0x1F440, prEmojiPresentation}, // E0.6 [1] (👀) eyes
{0x1F442, 0x1F464, prEmojiPresentation}, // E0.6 [35] (👂..👤) ear..bust in silhouette
{0x1F465, 0x1F465, prEmojiPresentation}, // E1.0 [1] (👥) busts in silhouette
{0x1F466, 0x1F46B, prEmojiPresentation}, // E0.6 [6] (👦..👫) boy..woman and man holding hands
{0x1F46C, 0x1F46D, prEmojiPresentation}, // E1.0 [2] (👬..👭) men holding hands..women holding hands
{0x1F46E, 0x1F4AC, prEmojiPresentation}, // E0.6 [63] (👮..💬) police officer..speech balloon
{0x1F4AD, 0x1F4AD, prEmojiPresentation}, // E1.0 [1] (💭) thought balloon
{0x1F4AE, 0x1F4B5, prEmojiPresentation}, // E0.6 [8] (💮..💵) white flower..dollar banknote
{0x1F4B6, 0x1F4B7, prEmojiPresentation}, // E1.0 [2] (💶..💷) euro banknote..pound banknote
{0x1F4B8, 0x1F4EB, prEmojiPresentation}, // E0.6 [52] (💸..📫) money with wings..closed mailbox with raised flag
{0x1F4EC, 0x1F4ED, prEmojiPresentation}, // E0.7 [2] (📬..📭) open mailbox with raised flag..open mailbox with lowered flag
{0x1F4EE, 0x1F4EE, prEmojiPresentation}, // E0.6 [1] (📮) postbox
{0x1F4EF, 0x1F4EF, prEmojiPresentation}, // E1.0 [1] (📯) postal horn
{0x1F4F0, 0x1F4F4, prEmojiPresentation}, // E0.6 [5] (📰..📴) newspaper..mobile phone off
{0x1F4F5, 0x1F4F5, prEmojiPresentation}, // E1.0 [1] (📵) no mobile phones
{0x1F4F6, 0x1F4F7, prEmojiPresentation}, // E0.6 [2] (📶..📷) antenna bars..camera
{0x1F4F8, 0x1F4F8, prEmojiPresentation}, // E1.0 [1] (📸) camera with flash
{0x1F4F9, 0x1F4FC, prEmojiPresentation}, // E0.6 [4] (📹..📼) video camera..videocassette
{0x1F4FF, 0x1F502, prEmojiPresentation}, // E1.0 [4] (📿..🔂) prayer beads..repeat single button
{0x1F503, 0x1F503, prEmojiPresentation}, // E0.6 [1] (🔃) clockwise vertical arrows
{0x1F504, 0x1F507, prEmojiPresentation}, // E1.0 [4] (🔄..🔇) counterclockwise arrows button..muted speaker
{0x1F508, 0x1F508, prEmojiPresentation}, // E0.7 [1] (🔈) speaker low volume
{0x1F509, 0x1F509, prEmojiPresentation}, // E1.0 [1] (🔉) speaker medium volume
{0x1F50A, 0x1F514, prEmojiPresentation}, // E0.6 [11] (🔊..🔔) speaker high volume..bell
{0x1F515, 0x1F515, prEmojiPresentation}, // E1.0 [1] (🔕) bell with slash
{0x1F516, 0x1F52B, prEmojiPresentation}, // E0.6 [22] (🔖..🔫) bookmark..water pistol
{0x1F52C, 0x1F52D, prEmojiPresentation}, // E1.0 [2] (🔬..🔭) microscope..telescope
{0x1F52E, 0x1F53D, prEmojiPresentation}, // E0.6 [16] (🔮..🔽) crystal ball..downwards button
{0x1F54B, 0x1F54E, prEmojiPresentation}, // E1.0 [4] (🕋..🕎) kaaba..menorah
{0x1F550, 0x1F55B, prEmojiPresentation}, // E0.6 [12] (🕐..🕛) one oclock..twelve oclock
{0x1F55C, 0x1F567, prEmojiPresentation}, // E0.7 [12] (🕜..🕧) one-thirty..twelve-thirty
{0x1F57A, 0x1F57A, prEmojiPresentation}, // E3.0 [1] (🕺) man dancing
{0x1F595, 0x1F596, prEmojiPresentation}, // E1.0 [2] (🖕..🖖) middle finger..vulcan salute
{0x1F5A4, 0x1F5A4, prEmojiPresentation}, // E3.0 [1] (🖤) black heart
{0x1F5FB, 0x1F5FF, prEmojiPresentation}, // E0.6 [5] (🗻..🗿) mount fuji..moai
{0x1F600, 0x1F600, prEmojiPresentation}, // E1.0 [1] (😀) grinning face
{0x1F601, 0x1F606, prEmojiPresentation}, // E0.6 [6] (😁..😆) beaming face with smiling eyes..grinning squinting face
{0x1F607, 0x1F608, prEmojiPresentation}, // E1.0 [2] (😇..😈) smiling face with halo..smiling face with horns
{0x1F609, 0x1F60D, prEmojiPresentation}, // E0.6 [5] (😉..😍) winking face..smiling face with heart-eyes
{0x1F60E, 0x1F60E, prEmojiPresentation}, // E1.0 [1] (😎) smiling face with sunglasses
{0x1F60F, 0x1F60F, prEmojiPresentation}, // E0.6 [1] (😏) smirking face
{0x1F610, 0x1F610, prEmojiPresentation}, // E0.7 [1] (😐) neutral face
{0x1F611, 0x1F611, prEmojiPresentation}, // E1.0 [1] (😑) expressionless face
{0x1F612, 0x1F614, prEmojiPresentation}, // E0.6 [3] (😒..😔) unamused face..pensive face
{0x1F615, 0x1F615, prEmojiPresentation}, // E1.0 [1] (😕) confused face
{0x1F616, 0x1F616, prEmojiPresentation}, // E0.6 [1] (😖) confounded face
{0x1F617, 0x1F617, prEmojiPresentation}, // E1.0 [1] (😗) kissing face
{0x1F618, 0x1F618, prEmojiPresentation}, // E0.6 [1] (😘) face blowing a kiss
{0x1F619, 0x1F619, prEmojiPresentation}, // E1.0 [1] (😙) kissing face with smiling eyes
{0x1F61A, 0x1F61A, prEmojiPresentation}, // E0.6 [1] (😚) kissing face with closed eyes
{0x1F61B, 0x1F61B, prEmojiPresentation}, // E1.0 [1] (😛) face with tongue
{0x1F61C, 0x1F61E, prEmojiPresentation}, // E0.6 [3] (😜..😞) winking face with tongue..disappointed face
{0x1F61F, 0x1F61F, prEmojiPresentation}, // E1.0 [1] (😟) worried face
{0x1F620, 0x1F625, prEmojiPresentation}, // E0.6 [6] (😠..😥) angry face..sad but relieved face
{0x1F626, 0x1F627, prEmojiPresentation}, // E1.0 [2] (😦..😧) frowning face with open mouth..anguished face
{0x1F628, 0x1F62B, prEmojiPresentation}, // E0.6 [4] (😨..😫) fearful face..tired face
{0x1F62C, 0x1F62C, prEmojiPresentation}, // E1.0 [1] (😬) grimacing face
{0x1F62D, 0x1F62D, prEmojiPresentation}, // E0.6 [1] (😭) loudly crying face
{0x1F62E, 0x1F62F, prEmojiPresentation}, // E1.0 [2] (😮..😯) face with open mouth..hushed face
{0x1F630, 0x1F633, prEmojiPresentation}, // E0.6 [4] (😰..😳) anxious face with sweat..flushed face
{0x1F634, 0x1F634, prEmojiPresentation}, // E1.0 [1] (😴) sleeping face
{0x1F635, 0x1F635, prEmojiPresentation}, // E0.6 [1] (😵) face with crossed-out eyes
{0x1F636, 0x1F636, prEmojiPresentation}, // E1.0 [1] (😶) face without mouth
{0x1F637, 0x1F640, prEmojiPresentation}, // E0.6 [10] (😷..🙀) face with medical mask..weary cat
{0x1F641, 0x1F644, prEmojiPresentation}, // E1.0 [4] (🙁..🙄) slightly frowning face..face with rolling eyes
{0x1F645, 0x1F64F, prEmojiPresentation}, // E0.6 [11] (🙅..🙏) person gesturing NO..folded hands
{0x1F680, 0x1F680, prEmojiPresentation}, // E0.6 [1] (🚀) rocket
{0x1F681, 0x1F682, prEmojiPresentation}, // E1.0 [2] (🚁..🚂) helicopter..locomotive
{0x1F683, 0x1F685, prEmojiPresentation}, // E0.6 [3] (🚃..🚅) railway car..bullet train
{0x1F686, 0x1F686, prEmojiPresentation}, // E1.0 [1] (🚆) train
{0x1F687, 0x1F687, prEmojiPresentation}, // E0.6 [1] (🚇) metro
{0x1F688, 0x1F688, prEmojiPresentation}, // E1.0 [1] (🚈) light rail
{0x1F689, 0x1F689, prEmojiPresentation}, // E0.6 [1] (🚉) station
{0x1F68A, 0x1F68B, prEmojiPresentation}, // E1.0 [2] (🚊..🚋) tram..tram car
{0x1F68C, 0x1F68C, prEmojiPresentation}, // E0.6 [1] (🚌) bus
{0x1F68D, 0x1F68D, prEmojiPresentation}, // E0.7 [1] (🚍) oncoming bus
{0x1F68E, 0x1F68E, prEmojiPresentation}, // E1.0 [1] (🚎) trolleybus
{0x1F68F, 0x1F68F, prEmojiPresentation}, // E0.6 [1] (🚏) bus stop
{0x1F690, 0x1F690, prEmojiPresentation}, // E1.0 [1] (🚐) minibus
{0x1F691, 0x1F693, prEmojiPresentation}, // E0.6 [3] (🚑..🚓) ambulance..police car
{0x1F694, 0x1F694, prEmojiPresentation}, // E0.7 [1] (🚔) oncoming police car
{0x1F695, 0x1F695, prEmojiPresentation}, // E0.6 [1] (🚕) taxi
{0x1F696, 0x1F696, prEmojiPresentation}, // E1.0 [1] (🚖) oncoming taxi
{0x1F697, 0x1F697, prEmojiPresentation}, // E0.6 [1] (🚗) automobile
{0x1F698, 0x1F698, prEmojiPresentation}, // E0.7 [1] (🚘) oncoming automobile
{0x1F699, 0x1F69A, prEmojiPresentation}, // E0.6 [2] (🚙..🚚) sport utility vehicle..delivery truck
{0x1F69B, 0x1F6A1, prEmojiPresentation}, // E1.0 [7] (🚛..🚡) articulated lorry..aerial tramway
{0x1F6A2, 0x1F6A2, prEmojiPresentation}, // E0.6 [1] (🚢) ship
{0x1F6A3, 0x1F6A3, prEmojiPresentation}, // E1.0 [1] (🚣) person rowing boat
{0x1F6A4, 0x1F6A5, prEmojiPresentation}, // E0.6 [2] (🚤..🚥) speedboat..horizontal traffic light
{0x1F6A6, 0x1F6A6, prEmojiPresentation}, // E1.0 [1] (🚦) vertical traffic light
{0x1F6A7, 0x1F6AD, prEmojiPresentation}, // E0.6 [7] (🚧..🚭) construction..no smoking
{0x1F6AE, 0x1F6B1, prEmojiPresentation}, // E1.0 [4] (🚮..🚱) litter in bin sign..non-potable water
{0x1F6B2, 0x1F6B2, prEmojiPresentation}, // E0.6 [1] (🚲) bicycle
{0x1F6B3, 0x1F6B5, prEmojiPresentation}, // E1.0 [3] (🚳..🚵) no bicycles..person mountain biking
{0x1F6B6, 0x1F6B6, prEmojiPresentation}, // E0.6 [1] (🚶) person walking
{0x1F6B7, 0x1F6B8, prEmojiPresentation}, // E1.0 [2] (🚷..🚸) no pedestrians..children crossing
{0x1F6B9, 0x1F6BE, prEmojiPresentation}, // E0.6 [6] (🚹..🚾) mens room..water closet
{0x1F6BF, 0x1F6BF, prEmojiPresentation}, // E1.0 [1] (🚿) shower
{0x1F6C0, 0x1F6C0, prEmojiPresentation}, // E0.6 [1] (🛀) person taking bath
{0x1F6C1, 0x1F6C5, prEmojiPresentation}, // E1.0 [5] (🛁..🛅) bathtub..left luggage
{0x1F6CC, 0x1F6CC, prEmojiPresentation}, // E1.0 [1] (🛌) person in bed
{0x1F6D0, 0x1F6D0, prEmojiPresentation}, // E1.0 [1] (🛐) place of worship
{0x1F6D1, 0x1F6D2, prEmojiPresentation}, // E3.0 [2] (🛑..🛒) stop sign..shopping cart
{0x1F6D5, 0x1F6D5, prEmojiPresentation}, // E12.0 [1] (🛕) hindu temple
{0x1F6D6, 0x1F6D7, prEmojiPresentation}, // E13.0 [2] (🛖..🛗) hut..elevator
{0x1F6DD, 0x1F6DF, prEmojiPresentation}, // E14.0 [3] (🛝..🛟) playground slide..ring buoy
{0x1F6EB, 0x1F6EC, prEmojiPresentation}, // E1.0 [2] (🛫..🛬) airplane departure..airplane arrival
{0x1F6F4, 0x1F6F6, prEmojiPresentation}, // E3.0 [3] (🛴..🛶) kick scooter..canoe
{0x1F6F7, 0x1F6F8, prEmojiPresentation}, // E5.0 [2] (🛷..🛸) sled..flying saucer
{0x1F6F9, 0x1F6F9, prEmojiPresentation}, // E11.0 [1] (🛹) skateboard
{0x1F6FA, 0x1F6FA, prEmojiPresentation}, // E12.0 [1] (🛺) auto rickshaw
{0x1F6FB, 0x1F6FC, prEmojiPresentation}, // E13.0 [2] (🛻..🛼) pickup truck..roller skate
{0x1F7E0, 0x1F7EB, prEmojiPresentation}, // E12.0 [12] (🟠..🟫) orange circle..brown square
{0x1F7F0, 0x1F7F0, prEmojiPresentation}, // E14.0 [1] (🟰) heavy equals sign
{0x1F90C, 0x1F90C, prEmojiPresentation}, // E13.0 [1] (🤌) pinched fingers
{0x1F90D, 0x1F90F, prEmojiPresentation}, // E12.0 [3] (🤍..🤏) white heart..pinching hand
{0x1F910, 0x1F918, prEmojiPresentation}, // E1.0 [9] (🤐..🤘) zipper-mouth face..sign of the horns
{0x1F919, 0x1F91E, prEmojiPresentation}, // E3.0 [6] (🤙..🤞) call me hand..crossed fingers
{0x1F91F, 0x1F91F, prEmojiPresentation}, // E5.0 [1] (🤟) love-you gesture
{0x1F920, 0x1F927, prEmojiPresentation}, // E3.0 [8] (🤠..🤧) cowboy hat face..sneezing face
{0x1F928, 0x1F92F, prEmojiPresentation}, // E5.0 [8] (🤨..🤯) face with raised eyebrow..exploding head
{0x1F930, 0x1F930, prEmojiPresentation}, // E3.0 [1] (🤰) pregnant woman
{0x1F931, 0x1F932, prEmojiPresentation}, // E5.0 [2] (🤱..🤲) breast-feeding..palms up together
{0x1F933, 0x1F93A, prEmojiPresentation}, // E3.0 [8] (🤳..🤺) selfie..person fencing
{0x1F93C, 0x1F93E, prEmojiPresentation}, // E3.0 [3] (🤼..🤾) people wrestling..person playing handball
{0x1F93F, 0x1F93F, prEmojiPresentation}, // E12.0 [1] (🤿) diving mask
{0x1F940, 0x1F945, prEmojiPresentation}, // E3.0 [6] (🥀..🥅) wilted flower..goal net
{0x1F947, 0x1F94B, prEmojiPresentation}, // E3.0 [5] (🥇..🥋) 1st place medal..martial arts uniform
{0x1F94C, 0x1F94C, prEmojiPresentation}, // E5.0 [1] (🥌) curling stone
{0x1F94D, 0x1F94F, prEmojiPresentation}, // E11.0 [3] (🥍..🥏) lacrosse..flying disc
{0x1F950, 0x1F95E, prEmojiPresentation}, // E3.0 [15] (🥐..🥞) croissant..pancakes
{0x1F95F, 0x1F96B, prEmojiPresentation}, // E5.0 [13] (🥟..🥫) dumpling..canned food
{0x1F96C, 0x1F970, prEmojiPresentation}, // E11.0 [5] (🥬..🥰) leafy green..smiling face with hearts
{0x1F971, 0x1F971, prEmojiPresentation}, // E12.0 [1] (🥱) yawning face
{0x1F972, 0x1F972, prEmojiPresentation}, // E13.0 [1] (🥲) smiling face with tear
{0x1F973, 0x1F976, prEmojiPresentation}, // E11.0 [4] (🥳..🥶) partying face..cold face
{0x1F977, 0x1F978, prEmojiPresentation}, // E13.0 [2] (🥷..🥸) ninja..disguised face
{0x1F979, 0x1F979, prEmojiPresentation}, // E14.0 [1] (🥹) face holding back tears
{0x1F97A, 0x1F97A, prEmojiPresentation}, // E11.0 [1] (🥺) pleading face
{0x1F97B, 0x1F97B, prEmojiPresentation}, // E12.0 [1] (🥻) sari
{0x1F97C, 0x1F97F, prEmojiPresentation}, // E11.0 [4] (🥼..🥿) lab coat..flat shoe
{0x1F980, 0x1F984, prEmojiPresentation}, // E1.0 [5] (🦀..🦄) crab..unicorn
{0x1F985, 0x1F991, prEmojiPresentation}, // E3.0 [13] (🦅..🦑) eagle..squid
{0x1F992, 0x1F997, prEmojiPresentation}, // E5.0 [6] (🦒..🦗) giraffe..cricket
{0x1F998, 0x1F9A2, prEmojiPresentation}, // E11.0 [11] (🦘..🦢) kangaroo..swan
{0x1F9A3, 0x1F9A4, prEmojiPresentation}, // E13.0 [2] (🦣..🦤) mammoth..dodo
{0x1F9A5, 0x1F9AA, prEmojiPresentation}, // E12.0 [6] (🦥..🦪) sloth..oyster
{0x1F9AB, 0x1F9AD, prEmojiPresentation}, // E13.0 [3] (🦫..🦭) beaver..seal
{0x1F9AE, 0x1F9AF, prEmojiPresentation}, // E12.0 [2] (🦮..🦯) guide dog..white cane
{0x1F9B0, 0x1F9B9, prEmojiPresentation}, // E11.0 [10] (🦰..🦹) red hair..supervillain
{0x1F9BA, 0x1F9BF, prEmojiPresentation}, // E12.0 [6] (🦺..🦿) safety vest..mechanical leg
{0x1F9C0, 0x1F9C0, prEmojiPresentation}, // E1.0 [1] (🧀) cheese wedge
{0x1F9C1, 0x1F9C2, prEmojiPresentation}, // E11.0 [2] (🧁..🧂) cupcake..salt
{0x1F9C3, 0x1F9CA, prEmojiPresentation}, // E12.0 [8] (🧃..🧊) beverage box..ice
{0x1F9CB, 0x1F9CB, prEmojiPresentation}, // E13.0 [1] (🧋) bubble tea
{0x1F9CC, 0x1F9CC, prEmojiPresentation}, // E14.0 [1] (🧌) troll
{0x1F9CD, 0x1F9CF, prEmojiPresentation}, // E12.0 [3] (🧍..🧏) person standing..deaf person
{0x1F9D0, 0x1F9E6, prEmojiPresentation}, // E5.0 [23] (🧐..🧦) face with monocle..socks
{0x1F9E7, 0x1F9FF, prEmojiPresentation}, // E11.0 [25] (🧧..🧿) red envelope..nazar amulet
{0x1FA70, 0x1FA73, prEmojiPresentation}, // E12.0 [4] (🩰..🩳) ballet shoes..shorts
{0x1FA74, 0x1FA74, prEmojiPresentation}, // E13.0 [1] (🩴) thong sandal
{0x1FA78, 0x1FA7A, prEmojiPresentation}, // E12.0 [3] (🩸..🩺) drop of blood..stethoscope
{0x1FA7B, 0x1FA7C, prEmojiPresentation}, // E14.0 [2] (🩻..🩼) x-ray..crutch
{0x1FA80, 0x1FA82, prEmojiPresentation}, // E12.0 [3] (🪀..🪂) yo-yo..parachute
{0x1FA83, 0x1FA86, prEmojiPresentation}, // E13.0 [4] (🪃..🪆) boomerang..nesting dolls
{0x1FA90, 0x1FA95, prEmojiPresentation}, // E12.0 [6] (🪐..🪕) ringed planet..banjo
{0x1FA96, 0x1FAA8, prEmojiPresentation}, // E13.0 [19] (🪖..🪨) military helmet..rock
{0x1FAA9, 0x1FAAC, prEmojiPresentation}, // E14.0 [4] (🪩..🪬) mirror ball..hamsa
{0x1FAB0, 0x1FAB6, prEmojiPresentation}, // E13.0 [7] (🪰..🪶) fly..feather
{0x1FAB7, 0x1FABA, prEmojiPresentation}, // E14.0 [4] (🪷..🪺) lotus..nest with eggs
{0x1FAC0, 0x1FAC2, prEmojiPresentation}, // E13.0 [3] (🫀..🫂) anatomical heart..people hugging
{0x1FAC3, 0x1FAC5, prEmojiPresentation}, // E14.0 [3] (🫃..🫅) pregnant man..person with crown
{0x1FAD0, 0x1FAD6, prEmojiPresentation}, // E13.0 [7] (🫐..🫖) blueberries..teapot
{0x1FAD7, 0x1FAD9, prEmojiPresentation}, // E14.0 [3] (🫗..🫙) pouring liquid..jar
{0x1FAE0, 0x1FAE7, prEmojiPresentation}, // E14.0 [8] (🫠..🫧) melting face..bubbles
{0x1FAF0, 0x1FAF6, prEmojiPresentation}, // E14.0 [7] (🫰..🫶) hand with index finger and thumb crossed..heart hands
}

View file

@ -4,18 +4,21 @@
// Database auxiliary data files. The command line arguments are as follows:
//
// 1. The name of the Unicode data file (just the filename, without extension).
// Can be "-" (to skip) if the emoji flag is included.
// 2. The name of the locally generated Go file.
// 3. The name of the slice mapping code points to properties.
// 4. The name of the generator, for logging purposes.
// 5. (Optional) Flags, comma-separated. The following flags are available:
// - "emojis": include emoji properties (Extended Pictographic only).
// - "emojis=<property>": include the specified emoji properties (e.g.
// "Extended_Pictographic").
// - "gencat": include general category properties.
//
//go:generate go run gen_properties.go auxiliary/GraphemeBreakProperty graphemeproperties.go graphemeCodePoints graphemes emojis
//go:generate go run gen_properties.go auxiliary/WordBreakProperty wordproperties.go workBreakCodePoints words emojis
//go:generate go run gen_properties.go auxiliary/GraphemeBreakProperty graphemeproperties.go graphemeCodePoints graphemes emojis=Extended_Pictographic
//go:generate go run gen_properties.go auxiliary/WordBreakProperty wordproperties.go workBreakCodePoints words emojis=Extended_Pictographic
//go:generate go run gen_properties.go auxiliary/SentenceBreakProperty sentenceproperties.go sentenceBreakCodePoints sentences
//go:generate go run gen_properties.go LineBreak lineproperties.go lineBreakCodePoints lines gencat
//go:generate go run gen_properties.go EastAsianWidth eastasianwidth.go eastAsianWidth eastasianwidth
//go:generate go run gen_properties.go - emojipresentation.go emojiPresentation emojipresentation emojis=Emoji_Presentation
package main
import (
@ -38,7 +41,7 @@ import (
// We want to test against a specific version rather than the latest. When the
// package is upgraded to a new version, change these to generate new tests.
const (
gbpURL = `https://www.unicode.org/Public/14.0.0/ucd/%s.txt`
propertyURL = `https://www.unicode.org/Public/14.0.0/ucd/%s.txt`
emojiURL = `https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt`
)
@ -55,20 +58,25 @@ func main() {
log.SetFlags(0)
// Parse flags.
flags := make(map[string]struct{})
flags := make(map[string]string)
if len(os.Args) >= 6 {
for _, flag := range strings.Split(os.Args[5], ",") {
flags[flag] = struct{}{}
flagFields := strings.Split(flag, "=")
if len(flagFields) == 1 {
flags[flagFields[0]] = "yes"
} else {
flags[flagFields[0]] = flagFields[1]
}
}
}
// Parse the text file and generate Go source code from it.
var emojis string
if _, ok := flags["emojis"]; ok {
emojis = emojiURL
}
_, includeGeneralCategory := flags["gencat"]
src, err := parse(fmt.Sprintf(gbpURL, os.Args[1]), emojis, includeGeneralCategory)
var mainURL string
if os.Args[1] != "-" {
mainURL = fmt.Sprintf(propertyURL, os.Args[1])
}
src, err := parse(mainURL, flags["emojis"], includeGeneralCategory)
if err != nil {
log.Fatal(err)
}
@ -88,16 +96,23 @@ func main() {
// parse parses the Unicode Properties text files located at the given URLs and
// returns their equivalent Go source code to be used in the uniseg package. If
// "emojiURL" is an empty string, no emoji code points will be included. If
// "emojiProperty" is not an empty string, emoji code points for that emoji
// property (e.g. "Extended_Pictographic") will be included. In those cases, you
// may pass an empty "propertyURL" to skip parsing the main properties file. If
// "includeGeneralCategory" is true, the Unicode General Category property will
// be extracted from the comments and included in the output.
func parse(gbpURL, emojiURL string, includeGeneralCategory bool) (string, error) {
func parse(propertyURL, emojiProperty string, includeGeneralCategory bool) (string, error) {
if propertyURL == "" && emojiProperty == "" {
return "", errors.New("no properties to parse")
}
// Temporary buffer to hold properties.
var properties [][4]string
// Open the first URL.
log.Printf("Parsing %s", gbpURL)
res, err := http.Get(gbpURL)
if propertyURL != "" {
log.Printf("Parsing %s", propertyURL)
res, err := http.Get(propertyURL)
if err != nil {
return "", err
}
@ -126,11 +141,12 @@ func parse(gbpURL, emojiURL string, includeGeneralCategory bool) (string, error)
if err := scanner.Err(); err != nil {
return "", err
}
}
// Open the second URL.
if emojiURL != "" {
if emojiProperty != "" {
log.Printf("Parsing %s", emojiURL)
res, err = http.Get(emojiURL)
res, err := http.Get(emojiURL)
if err != nil {
return "", err
}
@ -138,15 +154,15 @@ func parse(gbpURL, emojiURL string, includeGeneralCategory bool) (string, error)
defer in2.Close()
// Parse it.
scanner = bufio.NewScanner(in2)
num = 0
scanner := bufio.NewScanner(in2)
num := 0
for scanner.Scan() {
num++
line := scanner.Text()
// Skip comments, empty lines, and everything not containing
// "Extended_Pictographic".
if strings.HasPrefix(line, "#") || line == "" || !strings.Contains(line, "Extended_Pictographic") {
if strings.HasPrefix(line, "#") || line == "" || !strings.Contains(line, emojiProperty) {
continue
}
@ -189,7 +205,7 @@ func parse(gbpURL, emojiURL string, includeGeneralCategory bool) (string, error)
// Code generated via go generate from gen_properties.go. DO NOT EDIT.
// ` + os.Args[3] + ` are taken from
// ` + gbpURL + emojiComment + `
// ` + propertyURL + emojiComment + `
// on ` + time.Now().Format("January 2, 2006") + `. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var ` + os.Args[3] + ` = [][` + strconv.Itoa(columns) + `]int{

View file

@ -4,12 +4,14 @@ import "unicode/utf8"
// Graphemes implements an iterator over Unicode grapheme clusters, or
// user-perceived characters. While iterating, it also provides information
// about word boundaries, sentence boundaries, and line breaks.
// about word boundaries, sentence boundaries, line breaks, and monospace
// character widths.
//
// After constructing the class via [NewGraphemes] for a given string "str",
// [Next] is called for every grapheme cluster in a loop until it returns false.
// Inside the loop, information about the grapheme cluster as well as boundary
// information is available via the various methods (see examples below).
// [Graphemes.Next] is called for every grapheme cluster in a loop until it
// returns false. Inside the loop, information about the grapheme cluster as
// well as boundary information and character width is available via the various
// methods (see examples below).
//
// Using this class to iterate over a string is convenient but it is much slower
// than using this package's [Step] or [StepString] functions or any of the
@ -28,18 +30,18 @@ type Graphemes struct {
// string.
offset int
// The current boundary information of the Step() parser.
// The current boundary information of the [Step] parser.
boundaries int
// The current state of the Step() parser.
// The current state of the [Step] parser.
state int
}
// NewGraphemes returns a new grapheme cluster iterator.
func NewGraphemes(s string) *Graphemes {
func NewGraphemes(str string) *Graphemes {
return &Graphemes{
original: s,
remaining: s,
original: str,
remaining: str,
state: -1,
}
}
@ -60,8 +62,8 @@ func (g *Graphemes) Next() bool {
}
// Runes returns a slice of runes (code points) which corresponds to the current
// grapheme cluster. If the iterator is already past the end or [Next] has not
// yet been called, nil is returned.
// grapheme cluster. If the iterator is already past the end or [Graphemes.Next]
// has not yet been called, nil is returned.
func (g *Graphemes) Runes() []rune {
if g.state < 0 {
return nil
@ -70,15 +72,15 @@ func (g *Graphemes) Runes() []rune {
}
// Str returns a substring of the original string which corresponds to the
// current grapheme cluster. If the iterator is already past the end or [Next]
// has not yet been called, an empty string is returned.
// current grapheme cluster. If the iterator is already past the end or
// [Graphemes.Next] has not yet been called, an empty string is returned.
func (g *Graphemes) Str() string {
return g.cluster
}
// Bytes returns a byte slice which corresponds to the current grapheme cluster.
// If the iterator is already past the end or [Next] has not yet been called,
// nil is returned.
// If the iterator is already past the end or [Graphemes.Next] has not yet been
// called, nil is returned.
func (g *Graphemes) Bytes() []byte {
if g.state < 0 {
return nil
@ -90,8 +92,8 @@ func (g *Graphemes) Bytes() []byte {
// positions into the original string. The first returned value "from" indexes
// the first byte and the second returned value "to" indexes the first byte that
// is not included anymore, i.e. str[from:to] is the current grapheme cluster of
// the original string "str". If [Next] has not yet been called, both values are
// 0. If the iterator is already past the end, both values are 1.
// the original string "str". If [Graphemes.Next] has not yet been called, both
// values are 0. If the iterator is already past the end, both values are 1.
func (g *Graphemes) Positions() (int, int) {
if g.state == -1 {
return 0, 0
@ -133,8 +135,16 @@ func (g *Graphemes) LineBreak() int {
return g.boundaries & MaskLine
}
// Width returns the monospace width of the current grapheme cluster.
func (g *Graphemes) Width() int {
if g.state < 0 {
return 0
}
return g.boundaries >> ShiftWidth
}
// Reset puts the iterator into its initial state such that the next call to
// [Next] sets it to the first grapheme cluster again.
// [Graphemes.Next] sets it to the first grapheme cluster again.
func (g *Graphemes) Reset() {
g.state = -1
g.offset = 0
@ -153,6 +163,10 @@ func GraphemeClusterCount(s string) (n int) {
return
}
// The number of bits the grapheme property must be shifted to make place for
// grapheme states.
const shiftGraphemePropState = 4
// FirstGraphemeCluster returns the first grapheme cluster found in the given
// byte slice according to the rules of Unicode Standard Annex #29, Grapheme
// Cluster Boundaries. This function can be called continuously to extract all
@ -168,15 +182,15 @@ func GraphemeClusterCount(s string) (n int) {
// "cluster" byte slice is the sub-slice of the input slice containing the
// identified grapheme cluster.
//
// The returned width is the width of the grapheme cluster for most monospace
// fonts where a value of 1 represents one character cell.
//
// Given an empty byte slice "b", the function returns nil values.
//
// While slightly less convenient than using the Graphemes class, this function
// has much better performance and makes no allocations. It lends itself well to
// large byte slices.
//
// The "reserved" return value is a placeholder for future functionality and may
// be ignored for the time being.
func FirstGraphemeCluster(b []byte, state int) (cluster, rest []byte, reserved, newState int) {
func FirstGraphemeCluster(b []byte, state int) (cluster, rest []byte, width, newState int) {
// An empty byte slice returns nothing.
if len(b) == 0 {
return
@ -185,34 +199,60 @@ func FirstGraphemeCluster(b []byte, state int) (cluster, rest []byte, reserved,
// Extract the first rune.
r, length := utf8.DecodeRune(b)
if len(b) <= length { // If we're already past the end, there is nothing else to parse.
return b, nil, 0, grAny
var prop int
if state < 0 {
prop = property(graphemeCodePoints, r)
} else {
prop = state >> shiftGraphemePropState
}
return b, nil, runeWidth(r, prop), grAny | (prop << shiftGraphemePropState)
}
// If we don't know the state, determine it now.
var firstProp int
if state < 0 {
state, _ = transitionGraphemeState(state, r)
state, firstProp, _ = transitionGraphemeState(state, r)
} else {
firstProp = state >> shiftGraphemePropState
}
width += runeWidth(r, firstProp)
// Transition until we find a boundary.
var boundary bool
for {
var (
prop int
boundary bool
)
r, l := utf8.DecodeRune(b[length:])
state, boundary = transitionGraphemeState(state, r)
state, prop, boundary = transitionGraphemeState(state&maskGraphemeState, r)
if boundary {
return b[:length], b[length:], 0, state
return b[:length], b[length:], width, state | (prop << shiftGraphemePropState)
}
if r == vs16 {
width = 2
} else if firstProp != prExtendedPictographic && firstProp != prRegionalIndicator && firstProp != prL {
width += runeWidth(r, prop)
} else if firstProp == prExtendedPictographic {
if r == vs15 {
width = 1
} else {
width = 2
}
}
length += l
if len(b) <= length {
return b, nil, 0, grAny
return b, nil, width, grAny | (prop << shiftGraphemePropState)
}
}
}
// FirstGraphemeClusterInString is like [FirstGraphemeCluster] but its input and
// outputs are strings.
func FirstGraphemeClusterInString(str string, state int) (cluster, rest string, reserved, newState int) {
func FirstGraphemeClusterInString(str string, state int) (cluster, rest string, width, newState int) {
// An empty string returns nothing.
if len(str) == 0 {
return
@ -221,27 +261,53 @@ func FirstGraphemeClusterInString(str string, state int) (cluster, rest string,
// Extract the first rune.
r, length := utf8.DecodeRuneInString(str)
if len(str) <= length { // If we're already past the end, there is nothing else to parse.
return str, "", 0, grAny
var prop int
if state < 0 {
prop = property(graphemeCodePoints, r)
} else {
prop = state >> shiftGraphemePropState
}
return str, "", runeWidth(r, prop), grAny | (prop << shiftGraphemePropState)
}
// If we don't know the state, determine it now.
var firstProp int
if state < 0 {
state, _ = transitionGraphemeState(state, r)
state, firstProp, _ = transitionGraphemeState(state, r)
} else {
firstProp = state >> shiftGraphemePropState
}
width += runeWidth(r, firstProp)
// Transition until we find a boundary.
var boundary bool
for {
var (
prop int
boundary bool
)
r, l := utf8.DecodeRuneInString(str[length:])
state, boundary = transitionGraphemeState(state, r)
state, prop, boundary = transitionGraphemeState(state&maskGraphemeState, r)
if boundary {
return str[:length], str[length:], 0, state
return str[:length], str[length:], width, state | (prop << shiftGraphemePropState)
}
if r == vs16 {
width = 2
} else if firstProp != prExtendedPictographic && firstProp != prRegionalIndicator && firstProp != prL {
width += runeWidth(r, prop)
} else if firstProp == prExtendedPictographic {
if r == vs15 {
width = 1
} else {
width = 2
}
}
length += l
if len(str) <= length {
return str, "", 0, grAny
return str, "", width, grAny | (prop << shiftGraphemePropState)
}
}
}

View file

@ -7,7 +7,7 @@ package uniseg
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on July 25, 2022. See https://www.unicode.org/license.html for the Unicode
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var graphemeCodePoints = [][3]int{
{0x0000, 0x0009, prControl}, // Cc [10] <control-0000>..<control-0009>

View file

@ -92,22 +92,23 @@ var grTransitions = map[[2]int][3]int{
}
// transitionGraphemeState determines the new state of the grapheme cluster
// parser given the current state and the next code point. It also returns
// whether a cluster boundary was detected.
func transitionGraphemeState(state int, r rune) (newState int, boundary bool) {
// parser given the current state and the next code point. It also returns the
// code point's grapheme property (the value mapped by the [graphemeCodePoints]
// table) and whether a cluster boundary was detected.
func transitionGraphemeState(state int, r rune) (newState, prop int, boundary bool) {
// Determine the property of the next character.
nextProperty := property(graphemeCodePoints, r)
prop = property(graphemeCodePoints, r)
// Find the applicable transition.
transition, ok := grTransitions[[2]int{state, nextProperty}]
transition, ok := grTransitions[[2]int{state, prop}]
if ok {
// We have a specific transition. We'll use it.
return transition[0], transition[1] == grBoundary
return transition[0], prop, transition[1] == grBoundary
}
// No specific transition found. Try the less specific ones.
transAnyProp, okAnyProp := grTransitions[[2]int{state, prAny}]
transAnyState, okAnyState := grTransitions[[2]int{grAny, nextProperty}]
transAnyState, okAnyState := grTransitions[[2]int{grAny, prop}]
if okAnyProp && okAnyState {
// Both apply. We'll use a mix (see comments for grTransitions).
newState = transAnyState[0]
@ -120,7 +121,7 @@ func transitionGraphemeState(state int, r rune) (newState int, boundary bool) {
if okAnyProp {
// We only have a specific state.
return transAnyProp[0], transAnyProp[1] == grBoundary
return transAnyProp[0], prop, transAnyProp[1] == grBoundary
// This branch will probably never be reached because okAnyState will
// always be true given the current transition map. But we keep it here
// for future modifications to the transition map where this may not be
@ -129,9 +130,9 @@ func transitionGraphemeState(state int, r rune) (newState int, boundary bool) {
if okAnyState {
// We only have a specific property.
return transAnyState[0], transAnyState[1] == grBoundary
return transAnyState[0], prop, transAnyState[1] == grBoundary
}
// No known transition. GB999: Any ÷ Any.
return grAny, true
return grAny, prop, true
}

View file

@ -13,7 +13,7 @@ import "unicode/utf8"
//
// The returned "segment" may not be broken into smaller parts, unless no other
// breaking opportunities present themselves, in which case you may break by
// grapheme clusters (using the FirstGraphemeCluster() function to determine the
// grapheme clusters (using the [FirstGraphemeCluster] function to determine the
// grapheme clusters).
//
// The "mustBreak" flag indicates whether you MUST break the line after the
@ -42,7 +42,7 @@ import "unicode/utf8"
//
// Note also that this algorithm may break within grapheme clusters. This is
// addressed in Section 8.2 Example 6 of UAX #14. To avoid this, you can use
// the Step() function instead.
// the [Step] function instead.
func FirstLineSegment(b []byte, state int) (segment, rest []byte, mustBreak bool, newState int) {
// An empty byte slice returns nothing.
if len(b) == 0 {
@ -114,7 +114,9 @@ func FirstLineSegmentInString(str string, state int) (segment, rest string, must
}
// HasTrailingLineBreak returns true if the last rune in the given byte slice is
// one of the hard line break code points as defined in LB4 and LB5 of UAX #14.
// one of the hard line break code points defined in LB4 and LB5 of [UAX #14].
//
// [UAX #14]: https://www.unicode.org/reports/tr14/#Algorithm
func HasTrailingLineBreak(b []byte) bool {
r, _ := utf8.DecodeLastRune(b)
property, _ := propertyWithGenCat(lineBreakCodePoints, r)

View file

@ -4,7 +4,10 @@ package uniseg
// lineBreakCodePoints are taken from
// https://www.unicode.org/Public/14.0.0/ucd/LineBreak.txt
// on July 25, 2022. See https://www.unicode.org/license.html for the Unicode
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var lineBreakCodePoints = [][4]int{
{0x0000, 0x0008, prCM, gcCc}, // [9] <control-0000>..<control-0008>

View file

@ -5,7 +5,7 @@ package uniseg
const (
prXX = 0 // Same as prAny.
prAny = iota // prAny must be 0.
prPrepend
prPrepend // Grapheme properties must come first, to reduce the number of bits stored in the state vector.
prCR
prLF
prControl
@ -86,6 +86,7 @@ const (
prW
prH
prF
prEmojiPresentation
)
// Unicode General Categories. Only the ones needed in the context of this
@ -124,6 +125,12 @@ const (
gcCo
)
// Special code points.
const (
vs15 = 0xfe0e // Variation Selector-15 (text presentation)
vs16 = 0xfe0f // Variation Selector-16 (emoji presentation)
)
// propertySearch performs a binary search on a property slice and returns the
// entry whose range (start = first array element, end = second array element)
// includes r, or an array of 0's if no such entry was found.

View file

@ -4,7 +4,10 @@ package uniseg
// sentenceBreakCodePoints are taken from
// https://www.unicode.org/Public/14.0.0/ucd/auxiliary/SentenceBreakProperty.txt
// on July 25, 2022. See https://www.unicode.org/license.html for the Unicode
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var sentenceBreakCodePoints = [][3]int{
{0x0009, 0x0009, prSp}, // Cc <control-0009>

104
vendor/github.com/rivo/uniseg/step.go generated vendored
View file

@ -2,31 +2,37 @@ package uniseg
import "unicode/utf8"
// The bit masks used to extract boundary information returned by the Step()
// function.
// The bit masks used to extract boundary information returned by [Step].
const (
MaskLine = 3
MaskWord = 4
MaskSentence = 8
)
// The bit positions by which boundary flags are shifted by the Step() function.
// This must correspond to the Mask constants.
// The number of bits to shift the boundary information returned by [Step] to
// obtain the monospace width of the grapheme cluster.
const ShiftWidth = 4
// The bit positions by which boundary flags are shifted by the [Step] function.
// These must correspond to the Mask constants.
const (
shiftWord = 2
shiftSentence = 3
// shiftwWidth is ShiftWidth above. No mask as these are always the remaining bits.
)
// The bit positions by which states are shifted by the Step() function. These
// The bit positions by which states are shifted by the [Step] function. These
// values must ensure state values defined for each of the boundary algorithms
// don't overlap (and that they all still fit in a single int).
// don't overlap (and that they all still fit in a single int). These must
// correspond to the Mask constants.
const (
shiftWordState = 4
shiftSentenceState = 9
shiftLineState = 13
shiftPropState = 21 // No mask as these are always the remaining bits.
)
// The bit mask used to extract the state returned by the Step() function, after
// The bit mask used to extract the state returned by the [Step] function, after
// shifting. These values must correspond to the shift constants.
const (
maskGraphemeState = 0xf
@ -37,10 +43,11 @@ const (
// Step returns the first grapheme cluster (user-perceived character) found in
// the given byte slice. It also returns information about the boundary between
// that grapheme cluster and the one following it. There are three types of
// boundary information: word boundaries, sentence boundaries, and line breaks.
// This function is therefore a combination of FirstGraphemeCluster(),
// FirstWord(), FirstSentence(), and FirstLineSegment().
// that grapheme cluster and the one following it as well as the monospace width
// of the grapheme cluster. There are three types of boundary information: word
// boundaries, sentence boundaries, and line breaks. This function is therefore
// a combination of [FirstGraphemeCluster], [FirstWord], [FirstSentence], and
// [FirstLineSegment].
//
// The "boundaries" return value can be evaluated as follows:
//
@ -54,6 +61,8 @@ const (
// boundary.
// - boundaries&MaskLine == LineCanBreak: You may or may not break the line at
// the boundary.
// - boundaries >> ShiftWidth: The width of the grapheme cluster for most
// monospace fonts where a value of 1 represents one character cell.
//
// This function can be called continuously to extract all grapheme clusters
// from a byte slice, as illustrated in the examples below.
@ -87,14 +96,20 @@ func Step(b []byte, state int) (cluster, rest []byte, boundaries int, newState i
// Extract the first rune.
r, length := utf8.DecodeRune(b)
if len(b) <= length { // If we're already past the end, there is nothing else to parse.
return b, nil, LineMustBreak | (1 << shiftWord) | (1 << shiftSentence), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState)
var prop int
if state < 0 {
prop = property(graphemeCodePoints, r)
} else {
prop = state >> shiftPropState
}
return b, nil, LineMustBreak | (1 << shiftWord) | (1 << shiftSentence) | (runeWidth(r, prop) << ShiftWidth), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState) | (prop << shiftPropState)
}
// If we don't know the state, determine it now.
var graphemeState, wordState, sentenceState, lineState int
var graphemeState, wordState, sentenceState, lineState, firstProp int
remainder := b[length:]
if state < 0 {
graphemeState, _ = transitionGraphemeState(state, r)
graphemeState, firstProp, _ = transitionGraphemeState(state, r)
wordState, _ = transitionWordBreakState(state, r, remainder, "")
sentenceState, _ = transitionSentenceBreakState(state, r, remainder, "")
lineState, _ = transitionLineBreakState(state, r, remainder, "")
@ -103,36 +118,51 @@ func Step(b []byte, state int) (cluster, rest []byte, boundaries int, newState i
wordState = (state >> shiftWordState) & maskWordState
sentenceState = (state >> shiftSentenceState) & maskSentenceState
lineState = (state >> shiftLineState) & maskLineState
firstProp = state >> shiftPropState
}
// Transition until we find a grapheme cluster boundary.
width := runeWidth(r, firstProp)
for {
var (
graphemeBoundary, wordBoundary, sentenceBoundary bool
lineBreak int
lineBreak, prop int
)
for {
r, l := utf8.DecodeRune(remainder)
remainder = b[length+l:]
graphemeState, graphemeBoundary = transitionGraphemeState(graphemeState, r)
graphemeState, prop, graphemeBoundary = transitionGraphemeState(graphemeState, r)
wordState, wordBoundary = transitionWordBreakState(wordState, r, remainder, "")
sentenceState, sentenceBoundary = transitionSentenceBreakState(sentenceState, r, remainder, "")
lineState, lineBreak = transitionLineBreakState(lineState, r, remainder, "")
if graphemeBoundary {
boundary := lineBreak
boundary := lineBreak | (width << ShiftWidth)
if wordBoundary {
boundary |= 1 << shiftWord
}
if sentenceBoundary {
boundary |= 1 << shiftSentence
}
return b[:length], b[length:], boundary, graphemeState | (wordState << shiftWordState) | (sentenceState << shiftSentenceState) | (lineState << shiftLineState)
return b[:length], b[length:], boundary, graphemeState | (wordState << shiftWordState) | (sentenceState << shiftSentenceState) | (lineState << shiftLineState) | (prop << shiftPropState)
}
if r == vs16 {
width = 2
} else if firstProp != prExtendedPictographic && firstProp != prRegionalIndicator && firstProp != prL {
width += runeWidth(r, prop)
} else if firstProp == prExtendedPictographic {
if r == vs15 {
width = 1
} else {
width = 2
}
}
length += l
if len(b) <= length {
return b, nil, LineMustBreak | (1 << shiftWord) | (1 << shiftSentence), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState)
return b, nil, LineMustBreak | (1 << shiftWord) | (1 << shiftSentence) | (width << ShiftWidth), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState) | (prop << shiftPropState)
}
}
}
@ -147,14 +177,15 @@ func StepString(str string, state int) (cluster, rest string, boundaries int, ne
// Extract the first rune.
r, length := utf8.DecodeRuneInString(str)
if len(str) <= length { // If we're already past the end, there is nothing else to parse.
return str, "", LineMustBreak | (1 << shiftWord) | (1 << shiftSentence), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState)
prop := property(graphemeCodePoints, r)
return str, "", LineMustBreak | (1 << shiftWord) | (1 << shiftSentence) | (runeWidth(r, prop) << ShiftWidth), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState)
}
// If we don't know the state, determine it now.
var graphemeState, wordState, sentenceState, lineState int
var graphemeState, wordState, sentenceState, lineState, firstProp int
remainder := str[length:]
if state < 0 {
graphemeState, _ = transitionGraphemeState(state, r)
graphemeState, firstProp, _ = transitionGraphemeState(state, r)
wordState, _ = transitionWordBreakState(state, r, nil, remainder)
sentenceState, _ = transitionSentenceBreakState(state, r, nil, remainder)
lineState, _ = transitionLineBreakState(state, r, nil, remainder)
@ -163,36 +194,51 @@ func StepString(str string, state int) (cluster, rest string, boundaries int, ne
wordState = (state >> shiftWordState) & maskWordState
sentenceState = (state >> shiftSentenceState) & maskSentenceState
lineState = (state >> shiftLineState) & maskLineState
firstProp = state >> shiftPropState
}
// Transition until we find a grapheme cluster boundary.
width := runeWidth(r, firstProp)
for {
var (
graphemeBoundary, wordBoundary, sentenceBoundary bool
lineBreak int
lineBreak, prop int
)
for {
r, l := utf8.DecodeRuneInString(remainder)
remainder = str[length+l:]
graphemeState, graphemeBoundary = transitionGraphemeState(graphemeState, r)
graphemeState, prop, graphemeBoundary = transitionGraphemeState(graphemeState, r)
wordState, wordBoundary = transitionWordBreakState(wordState, r, nil, remainder)
sentenceState, sentenceBoundary = transitionSentenceBreakState(sentenceState, r, nil, remainder)
lineState, lineBreak = transitionLineBreakState(lineState, r, nil, remainder)
if graphemeBoundary {
boundary := lineBreak
boundary := lineBreak | (width << ShiftWidth)
if wordBoundary {
boundary |= 1 << shiftWord
}
if sentenceBoundary {
boundary |= 1 << shiftSentence
}
return str[:length], str[length:], boundary, graphemeState | (wordState << shiftWordState) | (sentenceState << shiftSentenceState) | (lineState << shiftLineState)
return str[:length], str[length:], boundary, graphemeState | (wordState << shiftWordState) | (sentenceState << shiftSentenceState) | (lineState << shiftLineState) | (prop << shiftPropState)
}
if r == vs16 {
width = 2
} else if firstProp != prExtendedPictographic && firstProp != prRegionalIndicator && firstProp != prL {
width += runeWidth(r, prop)
} else if firstProp == prExtendedPictographic {
if r == vs15 {
width = 1
} else {
width = 2
}
}
length += l
if len(str) <= length {
return str, "", LineMustBreak | (1 << shiftWord) | (1 << shiftSentence), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState)
return str, "", LineMustBreak | (1 << shiftWord) | (1 << shiftSentence) | (width << ShiftWidth), grAny | (wbAny << shiftWordState) | (sbAny << shiftSentenceState) | (lbAny << shiftLineState) | (prop << shiftPropState)
}
}
}

54
vendor/github.com/rivo/uniseg/width.go generated vendored Normal file
View file

@ -0,0 +1,54 @@
package uniseg
// runeWidth returns the monospace width for the given rune. The provided
// grapheme property is a value mapped by the [graphemeCodePoints] table.
//
// Every rune has a width of 1, except for runes with the following properties
// (evaluated in this order):
//
// - Control, CR, LF, Extend, ZWJ: Width of 0
// - \u2e3a, TWO-EM DASH: Width of 3
// - \u2e3b, THREE-EM DASH: Width of 4
// - East-Asian width Fullwidth and Wide: Width of 2 (Ambiguous and Neutral
// have a width of 1)
// - Regional Indicator: Width of 2
// - Extended Pictographic: Width of 2, unless Emoji Presentation is "No".
func runeWidth(r rune, graphemeProperty int) int {
switch graphemeProperty {
case prControl, prCR, prLF, prExtend, prZWJ:
return 0
case prRegionalIndicator:
return 2
case prExtendedPictographic:
if property(emojiPresentation, r) == prEmojiPresentation {
return 2
}
return 1
}
switch r {
case 0x2e3a:
return 3
case 0x2e3b:
return 4
}
switch property(eastAsianWidth, r) {
case prW, prF:
return 2
}
return 1
}
// StringWidth returns the monospace width for the given string, that is, the
// number of same-size cells to be occupied by the string.
func StringWidth(s string) (width int) {
state := -1
for len(s) > 0 {
var w int
_, s, w, state = FirstGraphemeClusterInString(s, state)
width += w
}
return
}

View file

@ -7,7 +7,7 @@ package uniseg
// and
// https://unicode.org/Public/14.0.0/ucd/emoji/emoji-data.txt
// ("Extended_Pictographic" only)
// on July 25, 2022. See https://www.unicode.org/license.html for the Unicode
// on September 10, 2022. See https://www.unicode.org/license.html for the Unicode
// license agreement.
var workBreakCodePoints = [][3]int{
{0x000A, 0x000A, prLF}, // Cc <control-000A>
@ -624,8 +624,8 @@ var workBreakCodePoints = [][3]int{
{0x212A, 0x212D, prALetter}, // L& [4] KELVIN SIGN..BLACK-LETTER CAPITAL C
{0x212F, 0x2134, prALetter}, // L& [6] SCRIPT SMALL E..SCRIPT SMALL O
{0x2135, 0x2138, prALetter}, // Lo [4] ALEF SYMBOL..DALET SYMBOL
{0x2139, 0x2139, prALetter}, // L& INFORMATION SOURCE
{0x2139, 0x2139, prExtendedPictographic}, // E0.6 [1] () information
{0x2139, 0x2139, prALetter}, // L& INFORMATION SOURCE
{0x213C, 0x213F, prALetter}, // L& [4] DOUBLE-STRUCK SMALL PI..DOUBLE-STRUCK CAPITAL PI
{0x2145, 0x2149, prALetter}, // L& [5] DOUBLE-STRUCK ITALIC CAPITAL D..DOUBLE-STRUCK ITALIC SMALL J
{0x214E, 0x214E, prALetter}, // L& TURNED SMALL F

Some files were not shown because too many files have changed in this diff Show more