diff --git a/.github/ISSUE_TEMPLATE/question.yml b/.github/ISSUE_TEMPLATE/question.yml
new file mode 100644
index 000000000..51ea7b1c0
--- /dev/null
+++ b/.github/ISSUE_TEMPLATE/question.yml
@@ -0,0 +1,32 @@
+name: Question
+description: Ask a question regarding VictoriaMetrics or its components
+labels: [question]
+body:
+ - type: textarea
+ id: describe-the-component
+ attributes:
+ label: Is your question request related to a specific component?
+ placeholder: |
+ VictoriaMetrics, vmagent, vmalert, vmui, etc...
+ validations:
+ required: false
+ - type: textarea
+ id: describe-the-question
+ attributes:
+ label: Describe the question in detail
+ description: |
+ A clear and concise description of the issue and the question.
+ validations:
+ required: true
+ - type: checkboxes
+ id: troubleshooting
+ attributes:
+ label: Troubleshooting docs
+ description: I am familiar with the following troubleshooting docs
+ options:
+ - label: General - https://docs.victoriametrics.com/Troubleshooting.html
+ required: false
+ - label: vmagent - https://docs.victoriametrics.com/vmagent.html#troubleshooting
+ required: false
+ - label: vmalert - https://docs.victoriametrics.com/vmalert.html#troubleshooting
+ required: false
\ No newline at end of file
diff --git a/README.md b/README.md
index 3f54e17f2..edecb87de 100644
--- a/README.md
+++ b/README.md
@@ -126,11 +126,22 @@ See also [articles and slides about VictoriaMetrics from our users](https://docs
## Operation
-### How to start VictoriaMetrics
+### Install
-Just download [VictoriaMetrics executable](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) or [Docker image](https://hub.docker.com/r/victoriametrics/victoria-metrics/) and start it with the desired command-line flags.
+To quickly try VictoriaMetrics, just download [VictoriaMetrics executable](https://github.com/VictoriaMetrics/VictoriaMetrics/releases) or [Docker image](https://hub.docker.com/r/victoriametrics/victoria-metrics/) and start it with the desired command-line flags.
See also [QuickStart guide](https://docs.victoriametrics.com/Quick-Start.html) for additional information.
+VictoriaMetrics can also be installed via these installation methods:
+
+* [Helm charts for single-node and cluster versions of VictoriaMetrics](https://github.com/VictoriaMetrics/helm-charts).
+* [Kubernetes operator for VictoriaMetrics](https://github.com/VictoriaMetrics/operator).
+* [Ansible role for installing cluster VictoriaMetrics (by VictoriaMetrics)](https://github.com/VictoriaMetrics/ansible-playbooks).
+* [Ansible role for installing cluster VictoriaMetrics (by community)](https://github.com/Slapper/ansible-victoriametrics-cluster-role).
+* [Ansible role for installing single-node VictoriaMetrics (by community)](https://github.com/dreamteam-gg/ansible-victoriametrics-role).
+* [Snap package for VictoriaMetrics](https://snapcraft.io/victoriametrics).
+
+### How to start VictoriaMetrics
+
The following command-line flags are used the most:
* `-storageDataPath` - VictoriaMetrics stores all the data in this directory. Default path is `victoria-metrics-data` in the current working directory.
@@ -2039,17 +2050,10 @@ It is safe sharing the collected profiles from security point of view, since the
## Integrations
-* [Helm charts for single-node and cluster versions of VictoriaMetrics](https://github.com/VictoriaMetrics/helm-charts).
-* [Kubernetes operator for VictoriaMetrics](https://github.com/VictoriaMetrics/operator).
-* [netdata](https://github.com/netdata/netdata) can push data into VictoriaMetrics via `Prometheus remote_write API`.
- See [these docs](https://github.com/netdata/netdata#integrations).
* [go-graphite/carbonapi](https://github.com/go-graphite/carbonapi) can use VictoriaMetrics as time series backend.
See [this example](https://github.com/go-graphite/carbonapi/blob/main/cmd/carbonapi/carbonapi.example.victoriametrics.yaml).
-* [Ansible role for installing cluster VictoriaMetrics (by VictoriaMetrics)](https://github.com/VictoriaMetrics/ansible-playbooks).
-* [Ansible role for installing cluster VictoriaMetrics (by community)](https://github.com/Slapper/ansible-victoriametrics-cluster-role).
-* [Ansible role for installing single-node VictoriaMetrics (by community)](https://github.com/dreamteam-gg/ansible-victoriametrics-role).
-
-* [Snap package for VictoriaMetrics](https://snapcraft.io/victoriametrics).
+* [netdata](https://github.com/netdata/netdata) can push data into VictoriaMetrics via `Prometheus remote_write API`.
+ See [these docs](https://github.com/netdata/netdata#integrations).
* [vmalert-cli](https://github.com/aorfanos/vmalert-cli) - a CLI application for managing [vmalert](https://docs.victoriametrics.com/vmalert.html).
## Third-party contributions
@@ -2367,7 +2371,7 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li
-promscrape.seriesLimitPerTarget int
Optional limit on the number of unique time series a single scrape target can expose. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter for more info
-promscrape.streamParse
- Whether to enable stream parsing for metrics obtained from scrape targets. This may be useful for reducing memory usage when millions of metrics are exposed per each scrape target. It is posible to set 'stream_parse: true' individually per each 'scrape_config' section in '-promscrape.config' for fine grained control
+ Whether to enable stream parsing for metrics obtained from scrape targets. This may be useful for reducing memory usage when millions of metrics are exposed per each scrape target. It is possible to set 'stream_parse: true' individually per each 'scrape_config' section in '-promscrape.config' for fine grained control
-promscrape.suppressDuplicateScrapeTargetErrors
Whether to suppress 'duplicate scrape target' errors; see https://docs.victoriametrics.com/vmagent.html#troubleshooting for details
-promscrape.suppressScrapeErrors
diff --git a/app/victoria-metrics/main.go b/app/victoria-metrics/main.go
index 816e576ec..f5be4c345 100644
--- a/app/victoria-metrics/main.go
+++ b/app/victoria-metrics/main.go
@@ -26,7 +26,8 @@ import (
var (
httpListenAddr = flag.String("httpListenAddr", ":8428", "TCP address to listen for http connections. See also -httpListenAddr.useProxyProtocol")
useProxyProtocol = flag.Bool("httpListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -httpListenAddr . "+
- "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt . "+
+ "With enabled proxy protocol http server cannot serve regular /metrics endpoint. Use -pushmetrics.url for metrics pushing")
minScrapeInterval = flag.Duration("dedup.minScrapeInterval", 0, "Leave only the last sample in every time series per each discrete interval "+
"equal to -dedup.minScrapeInterval > 0. See https://docs.victoriametrics.com/#deduplication and https://docs.victoriametrics.com/#downsampling")
dryRun = flag.Bool("dryRun", false, "Whether to check only -promscrape.config and then exit. "+
@@ -61,7 +62,7 @@ func main() {
if err := promscrape.CheckConfig(); err != nil {
logger.Fatalf("error when checking -promscrape.config: %s", err)
}
- logger.Infof("-promscrape.config is ok; exitting with 0 status code")
+ logger.Infof("-promscrape.config is ok; exiting with 0 status code")
return
}
diff --git a/app/vmagent/README.md b/app/vmagent/README.md
index 03e1f6f14..d3727a6d0 100644
--- a/app/vmagent/README.md
+++ b/app/vmagent/README.md
@@ -1398,7 +1398,7 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
-promscrape.seriesLimitPerTarget int
Optional limit on the number of unique time series a single scrape target can expose. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter for more info
-promscrape.streamParse
- Whether to enable stream parsing for metrics obtained from scrape targets. This may be useful for reducing memory usage when millions of metrics are exposed per each scrape target. It is posible to set 'stream_parse: true' individually per each 'scrape_config' section in '-promscrape.config' for fine grained control
+ Whether to enable stream parsing for metrics obtained from scrape targets. This may be useful for reducing memory usage when millions of metrics are exposed per each scrape target. It is possible to set 'stream_parse: true' individually per each 'scrape_config' section in '-promscrape.config' for fine grained control
-promscrape.suppressDuplicateScrapeTargetErrors
Whether to suppress 'duplicate scrape target' errors; see https://docs.victoriametrics.com/vmagent.html#troubleshooting for details
-promscrape.suppressScrapeErrors
diff --git a/app/vmagent/main.go b/app/vmagent/main.go
index 75c4ef13e..9c4a6adda 100644
--- a/app/vmagent/main.go
+++ b/app/vmagent/main.go
@@ -46,7 +46,8 @@ var (
"Set this flag to empty value in order to disable listening on any port. This mode may be useful for running multiple vmagent instances on the same server. "+
"Note that /targets and /metrics pages aren't available if -httpListenAddr=''. See also -httpListenAddr.useProxyProtocol")
useProxyProtocol = flag.Bool("httpListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -httpListenAddr . "+
- "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt . "+
+ "With enabled proxy protocol http server cannot serve regular /metrics endpoint. Use -pushmetrics.url for metrics pushing")
influxListenAddr = flag.String("influxListenAddr", "", "TCP and UDP address to listen for InfluxDB line protocol data. Usually :8089 must be set. Doesn't work if empty. "+
"This flag isn't needed when ingesting data over HTTP - just send it to http://:8429/write . "+
"See also -influxListenAddr.useProxyProtocol")
@@ -98,7 +99,7 @@ func main() {
if err := promscrape.CheckConfig(); err != nil {
logger.Fatalf("error when checking -promscrape.config: %s", err)
}
- logger.Infof("-promscrape.config is ok; exitting with 0 status code")
+ logger.Infof("-promscrape.config is ok; exiting with 0 status code")
return
}
if *dryRun {
@@ -108,7 +109,7 @@ func main() {
if err := promscrape.CheckConfig(); err != nil {
logger.Fatalf("error when checking -promscrape.config: %s", err)
}
- logger.Infof("all the configs are ok; exitting with 0 status code")
+ logger.Infof("all the configs are ok; exiting with 0 status code")
return
}
diff --git a/app/vmagent/remotewrite/client.go b/app/vmagent/remotewrite/client.go
index 0ebff6203..bb5a1e67e 100644
--- a/app/vmagent/remotewrite/client.go
+++ b/app/vmagent/remotewrite/client.go
@@ -147,10 +147,6 @@ func newHTTPClient(argIdx int, remoteWriteURL, sanitizedURL string, fq *persiste
}
c.sendBlock = c.sendBlockHTTP
- return c
-}
-
-func (c *client) init(argIdx, concurrency int, sanitizedURL string) {
useVMProto := forceVMProto.GetOptionalArg(argIdx)
usePromProto := forcePromProto.GetOptionalArg(argIdx)
if useVMProto && usePromProto {
@@ -169,6 +165,10 @@ func (c *client) init(argIdx, concurrency int, sanitizedURL string) {
}
c.useVMProto = useVMProto
+ return c
+}
+
+func (c *client) init(argIdx, concurrency int, sanitizedURL string) {
if bytesPerSec := rateLimit.GetOptionalArgOrDefault(argIdx, 0); bytesPerSec > 0 {
logger.Infof("applying %d bytes per second rate limit for -remoteWrite.url=%q", bytesPerSec, sanitizedURL)
c.rl.perSecondLimit = int64(bytesPerSec)
diff --git a/app/vmalert/group.go b/app/vmalert/group.go
index 7bfda9eee..94b74b27e 100644
--- a/app/vmalert/group.go
+++ b/app/vmalert/group.go
@@ -2,6 +2,7 @@ package main
import (
"context"
+ "errors"
"fmt"
"hash/fnv"
"net/url"
@@ -44,6 +45,9 @@ type Group struct {
// channel accepts new Group obj
// which supposed to update current group
updateCh chan *Group
+ // evalCancel stores the cancel fn for interrupting
+ // rules evaluation. Used on groups update() and close().
+ evalCancel context.CancelFunc
metrics *groupMetrics
}
@@ -233,11 +237,24 @@ func (g *Group) updateWith(newGroup *Group) error {
return nil
}
+// interruptEval interrupts in-flight rules evaluations
+// within the group. It is expected that g.evalCancel
+// will be repopulated after the call.
+func (g *Group) interruptEval() {
+ g.mu.RLock()
+ defer g.mu.RUnlock()
+
+ if g.evalCancel != nil {
+ g.evalCancel()
+ }
+}
+
func (g *Group) close() {
if g.doneCh == nil {
return
}
close(g.doneCh)
+ g.interruptEval()
<-g.finishedCh
g.metrics.iterationDuration.Unregister()
@@ -254,6 +271,26 @@ var skipRandSleepOnGroupStart bool
func (g *Group) start(ctx context.Context, nts func() []notifier.Notifier, rw *remotewrite.Client, rr datasource.QuerierBuilder) {
defer func() { close(g.finishedCh) }()
+ // Spread group rules evaluation over time in order to reduce load on VictoriaMetrics.
+ if !skipRandSleepOnGroupStart {
+ randSleep := uint64(float64(g.Interval) * (float64(g.ID()) / (1 << 64)))
+ sleepOffset := uint64(time.Now().UnixNano()) % uint64(g.Interval)
+ if randSleep < sleepOffset {
+ randSleep += uint64(g.Interval)
+ }
+ randSleep -= sleepOffset
+ sleepTimer := time.NewTimer(time.Duration(randSleep))
+ select {
+ case <-ctx.Done():
+ sleepTimer.Stop()
+ return
+ case <-g.doneCh:
+ sleepTimer.Stop()
+ return
+ case <-sleepTimer.C:
+ }
+ }
+
e := &executor{
rw: rw,
notifiers: nts,
@@ -263,7 +300,7 @@ func (g *Group) start(ctx context.Context, nts func() []notifier.Notifier, rw *r
logger.Infof("group %q started; interval=%v; concurrency=%d", g.Name, g.Interval, g.Concurrency)
- eval := func(ts time.Time) {
+ eval := func(ctx context.Context, ts time.Time) {
g.metrics.iterationTotal.Inc()
start := time.Now()
@@ -285,7 +322,13 @@ func (g *Group) start(ctx context.Context, nts func() []notifier.Notifier, rw *r
g.LastEvaluation = start
}
- eval(evalTS)
+ evalCtx, cancel := context.WithCancel(ctx)
+ g.mu.Lock()
+ g.evalCancel = cancel
+ g.mu.Unlock()
+ defer g.evalCancel()
+
+ eval(evalCtx, evalTS)
t := time.NewTicker(g.Interval)
defer t.Stop()
@@ -309,6 +352,14 @@ func (g *Group) start(ctx context.Context, nts func() []notifier.Notifier, rw *r
return
case ng := <-g.updateCh:
g.mu.Lock()
+
+ // it is expected that g.evalCancel will be evoked
+ // somewhere else to unblock group from the rules evaluation.
+ // we recreate the evalCtx and g.evalCancel, so it can
+ // be called again.
+ evalCtx, cancel = context.WithCancel(ctx)
+ g.evalCancel = cancel
+
err := g.updateWith(ng)
if err != nil {
logger.Errorf("group %q: failed to update: %s", g.Name, err)
@@ -333,7 +384,7 @@ func (g *Group) start(ctx context.Context, nts func() []notifier.Notifier, rw *r
}
evalTS = evalTS.Add((missed + 1) * g.Interval)
- eval(evalTS)
+ eval(evalCtx, evalTS)
}
}
}
@@ -407,6 +458,11 @@ func (e *executor) exec(ctx context.Context, rule Rule, ts time.Time, resolveDur
tss, err := rule.Exec(ctx, ts, limit)
if err != nil {
+ if errors.Is(err, context.Canceled) {
+ // the context can be cancelled on graceful shutdown
+ // or on group update. So no need to handle the error as usual.
+ return nil
+ }
execErrors.Inc()
return fmt.Errorf("rule %q: failed to execute: %w", rule, err)
}
diff --git a/app/vmalert/group_test.go b/app/vmalert/group_test.go
index b20a0a516..29199a4bd 100644
--- a/app/vmalert/group_test.go
+++ b/app/vmalert/group_test.go
@@ -474,3 +474,31 @@ func TestFaultyRW(t *testing.T) {
t.Fatalf("expected to get an error from faulty RW client, got nil instead")
}
}
+
+func TestCloseWithEvalInterruption(t *testing.T) {
+ groups, err := config.Parse([]string{"config/testdata/rules/rules1-good.rules"}, notifier.ValidateTemplates, true)
+ if err != nil {
+ t.Fatalf("failed to parse rules: %s", err)
+ }
+
+ const delay = time.Second * 2
+ fq := &fakeQuerierWithDelay{delay: delay}
+
+ const evalInterval = time.Millisecond
+ g := newGroup(groups[0], fq, evalInterval, nil)
+
+ go g.start(context.Background(), nil, nil, nil)
+
+ time.Sleep(evalInterval * 20)
+
+ go func() {
+ g.close()
+ }()
+
+ deadline := time.Tick(delay / 2)
+ select {
+ case <-deadline:
+ t.Fatalf("deadline for close exceeded")
+ case <-g.finishedCh:
+ }
+}
diff --git a/app/vmalert/helpers_test.go b/app/vmalert/helpers_test.go
index 7ef8bd981..60183aad8 100644
--- a/app/vmalert/helpers_test.go
+++ b/app/vmalert/helpers_test.go
@@ -104,6 +104,24 @@ func (fqr *fakeQuerierWithRegistry) Query(_ context.Context, expr string, _ time
return cp, req, nil
}
+type fakeQuerierWithDelay struct {
+ fakeQuerier
+ delay time.Duration
+}
+
+func (fqd *fakeQuerierWithDelay) Query(ctx context.Context, expr string, ts time.Time) ([]datasource.Metric, *http.Request, error) {
+ timer := time.NewTimer(fqd.delay)
+ select {
+ case <-ctx.Done():
+ case <-timer.C:
+ }
+ return fqd.fakeQuerier.Query(ctx, expr, ts)
+}
+
+func (fqd *fakeQuerierWithDelay) BuildWithParams(_ datasource.QuerierParams) datasource.Querier {
+ return fqd
+}
+
type fakeNotifier struct {
sync.Mutex
alerts []notifier.Alert
diff --git a/app/vmalert/main.go b/app/vmalert/main.go
index 33f3e74e3..8f382a23f 100644
--- a/app/vmalert/main.go
+++ b/app/vmalert/main.go
@@ -57,7 +57,8 @@ absolute path to all .tpl files in root.`)
httpListenAddr = flag.String("httpListenAddr", ":8880", "Address to listen for http connections. See also -httpListenAddr.useProxyProtocol")
useProxyProtocol = flag.Bool("httpListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -httpListenAddr . "+
- "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt . "+
+ "With enabled proxy protocol http server cannot serve regular /metrics endpoint. Use -pushmetrics.url for metrics pushing")
evaluationInterval = flag.Duration("evaluationInterval", time.Minute, "How often to evaluate the rules")
validateTemplates = flag.Bool("rule.validateTemplates", true, "Whether to validate annotation and label templates")
diff --git a/app/vmalert/manager.go b/app/vmalert/manager.go
index 028d8df24..ac1a516f3 100644
--- a/app/vmalert/manager.go
+++ b/app/vmalert/manager.go
@@ -6,7 +6,6 @@ import (
"net/url"
"sort"
"sync"
- "time"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmalert/config"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmalert/datasource"
@@ -87,32 +86,12 @@ func (m *manager) startGroup(ctx context.Context, g *Group, restore bool) error
m.wg.Add(1)
id := g.ID()
go func() {
- // Spread group rules evaluation over time in order to reduce load on VictoriaMetrics.
- if !skipRandSleepOnGroupStart {
- randSleep := uint64(float64(g.Interval) * (float64(g.ID()) / (1 << 64)))
- sleepOffset := uint64(time.Now().UnixNano()) % uint64(g.Interval)
- if randSleep < sleepOffset {
- randSleep += uint64(g.Interval)
- }
- randSleep -= sleepOffset
- sleepTimer := time.NewTimer(time.Duration(randSleep))
- select {
- case <-ctx.Done():
- sleepTimer.Stop()
- return
- case <-g.doneCh:
- sleepTimer.Stop()
- return
- case <-sleepTimer.C:
- }
- }
+ defer m.wg.Done()
if restore {
g.start(ctx, m.notifiers, m.rw, m.rr)
} else {
g.start(ctx, m.notifiers, m.rw, nil)
}
-
- m.wg.Done()
}()
m.groups[id] = g
return nil
@@ -168,6 +147,7 @@ func (m *manager) update(ctx context.Context, groupsCfg []config.Group, restore
}
for _, ng := range groupsRegistry {
if err := m.startGroup(ctx, ng, restore); err != nil {
+ m.groupsMu.Unlock()
return err
}
}
@@ -181,6 +161,7 @@ func (m *manager) update(ctx context.Context, groupsCfg []config.Group, restore
old.updateCh <- new
wg.Done()
}(item.old, item.new)
+ item.old.interruptEval()
}
wg.Wait()
}
diff --git a/app/vmauth/main.go b/app/vmauth/main.go
index f64f56766..4b400c063 100644
--- a/app/vmauth/main.go
+++ b/app/vmauth/main.go
@@ -28,7 +28,8 @@ import (
var (
httpListenAddr = flag.String("httpListenAddr", ":8427", "TCP address to listen for http connections. See also -httpListenAddr.useProxyProtocol")
useProxyProtocol = flag.Bool("httpListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -httpListenAddr . "+
- "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt")
+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt . "+
+ "With enabled proxy protocol http server cannot serve regular /metrics endpoint. Use -pushmetrics.url for metrics pushing")
maxIdleConnsPerBackend = flag.Int("maxIdleConnsPerBackend", 100, "The maximum number of idle connections vmauth can open per each backend host. "+
"See also -maxConcurrentRequests")
responseTimeout = flag.Duration("responseTimeout", 5*time.Minute, "The timeout for receiving a response from backend")
diff --git a/app/vmctl/README.md b/app/vmctl/README.md
index 49275898a..8066c8db0 100644
--- a/app/vmctl/README.md
+++ b/app/vmctl/README.md
@@ -483,6 +483,10 @@ Processing ranges: 8798 / 8798 [████████████████
2022/10/19 16:45:37 Total time: 1m19.406283424s
```
+Migrating big volumes of data may result in remote read client reaching the timeout.
+Consider increasing the value of `--remote-read-http-timeout` (default `5m`) command-line flag when seeing
+timeouts or `context canceled` errors.
+
### Filtering
The filtering consists of two parts: by labels and time.
@@ -733,21 +737,33 @@ or higher.
See `./vmctl vm-native --help` for details and full list of flags.
-In this mode `vmctl` acts as a proxy between two VM instances, where time series filtering is done by "source" (`src`)
-and processing is done by "destination" (`dst`). Because of that, `vmctl` doesn't actually know how much data will be
-processed and can't show the progress bar. It will show the current processing speed and total number of processed bytes:
+Migration in `vm-native` mode takes two steps:
+1. Explore the list of the metrics to migrate via `/api/v1/series` API;
+2. Migrate explored metrics one-by-one.
```
-./vmctl vm-native --vm-native-src-addr=http://localhost:8528 \
- --vm-native-dst-addr=http://localhost:8428 \
- --vm-native-filter-match='{job="vmagent"}' \
- --vm-native-filter-time-start='2020-01-01T20:07:00Z'
+./vmctl vm-native \
+ --vm-native-src-addr=http://127.0.0.1:8481/select/0/prometheus \
+ --vm-native-dst-addr=http://localhost:8428 \
+ --vm-native-filter-time-start='2022-11-20T00:00:00Z' \
+ --vm-native-filter-match='{__name__=~"vm_cache_.*"}'
VictoriaMetrics Native import mode
-Initing export pipe from "http://localhost:8528" with filters:
- filter: match[]={job="vmagent"}
-Initing import process to "http://localhost:8428":
-Total: 336.75 KiB ↖ Speed: 454.46 KiB p/s
-2020/10/13 17:04:59 Total time: 952.143376ms
+
+2023/03/02 09:22:02 Initing import process from "http://127.0.0.1:8481/select/0/prometheus/api/v1/export/native" to "http://localhost:8428/api/v1/import/native" with filter
+ filter: match[]={__name__=~"vm_cache_.*"}
+ start: 2022-11-20T00:00:00Z
+2023/03/02 09:22:02 Exploring metrics...
+Found 9 metrics to import. Continue? [Y/n]
+2023/03/02 09:22:04 Requests to make: 9
+Requests to make: 9 / 9 [███████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████] 100.00%
+2023/03/02 09:22:06 Import finished!
+2023/03/02 09:22:06 VictoriaMetrics importer stats:
+ time spent while importing: 3.632638875s;
+ total bytes: 7.8 MB;
+ bytes/s: 2.1 MB;
+ requests: 9;
+ requests retries: 0;
+2023/03/02 09:22:06 Total time: 3.633127625s
```
Importing tips:
@@ -755,6 +771,7 @@ Importing tips:
1. Migrating big volumes of data may result in reaching the safety limits on `src` side.
Please verify that `-search.maxExportDuration` and `-search.maxExportSeries` were set with
proper values for `src`. If hitting the limits, follow the recommendations [here](https://docs.victoriametrics.com/#how-to-export-data-in-native-format).
+If hitting `the number of matching timeseries exceeds...` error, adjust filters to match less time series or update `-search.maxSeries` command-line flag on vmselect/vmsingle;
2. Migrating all the metrics from one VM to another may collide with existing application metrics
(prefixed with `vm_`) at destination and lead to confusion when using
[official Grafana dashboards](https://grafana.com/orgs/victoriametrics/dashboards).
@@ -766,71 +783,59 @@ Instead, use [relabeling in VictoriaMetrics](https://github.com/VictoriaMetrics/
5. When importing in or from cluster version remember to use correct [URL format](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format)
and specify `accountID` param.
6. When migrating large volumes of data it might be useful to use `--vm-native-step-interval` flag to split single process into smaller steps.
+7. `vmctl` supports `--vm-concurrency` which controls the number of concurrent workers that process the input from source query results.
+Please note that each import request can load up to a single vCPU core on VictoriaMetrics. So try to set it according
+to allocated CPU resources of your VictoriaMetrics installation.
+8. `vmctl` supports `--vm-native-src-headers` and `--vm-native-dst-headers` which defines headers to send with each request
+to the corresponding source address.
+
+In this mode `vmctl` acts as a proxy between two VM instances, where time series filtering is done by "source" (`src`)
+and processing is done by "destination" (`dst`). So no extra memory or CPU resources required on `vmctl` side. Only
+`src` and `dst` resource matter.
#### Using time-based chunking of migration
-It is possible split migration process into set of smaller batches based on time. This is especially useful when migrating large volumes of data as this adds indication of progress and ability to restore process from certain point in case of failure.
+It is possible split migration process into set of smaller batches based on time. This is especially useful when
+migrating large volumes of data as this adds indication of progress and ability to restore process from certain point
+in case of failure.
-To use this you need to specify `--vm-native-step-interval` flag. Supported values are: `month`, `day`, `hour`.
-Note that in order to use this it is required `--vm-native-filter-time-start` to be set to calculate time ranges for export process.
+To use this you need to specify `--vm-native-step-interval` flag. Supported values are: `month`, `day`, `hour`, `minute`.
+Note that in order to use this it is required `--vm-native-filter-time-start` to be set to calculate time ranges for
+export process.
Every range is being processed independently, which means that:
- after range processing is finished all data within range is migrated
-- if process fails on one of stages it is guaranteed that data of prior stages is already written, so it is possible to restart process starting from failed range
+- if process fails on one of stages it is guaranteed that data of prior stages is already written,
+so it is possible to restart process starting from failed range.
-It is recommended using the `month` step when migrating the data over multiple months, since the migration with `day` and `hour` steps may take longer time to complete
-because of additional overhead.
+It is recommended using the `month` step when migrating the data over multiple months,
+since the migration with `day` and `hour` steps may take longer time to complete because of additional overhead.
Usage example:
```console
-./vmctl vm-native
- --vm-native-filter-time-start 2022-06-17T00:07:00Z \
- --vm-native-filter-time-end 2022-10-03T00:07:00Z \
- --vm-native-src-addr http://localhost:8428 \
- --vm-native-dst-addr http://localhost:8528 \
- --vm-native-step-interval=month
+./vmctl vm-native \
+ --vm-native-src-addr=http://127.0.0.1:8481/select/0/prometheus \
+ --vm-native-dst-addr=http://localhost:8428 \
+ --vm-native-filter-time-start='2022-11-20T00:00:00Z' \
+ --vm-native-step-interval=month \
+ --vm-native-filter-match='{__name__=~"vm_cache_.*"}'
VictoriaMetrics Native import mode
-2022/08/30 19:48:24 Processing range 1/5: 2022-06-17T00:07:00Z - 2022-06-30T23:59:59Z
-2022/08/30 19:48:24 Initing export pipe from "http://localhost:8428" with filters:
- filter: match[]={__name__!=""}
- start: 2022-06-17T00:07:00Z
- end: 2022-06-30T23:59:59Z
-Initing import process to "http://localhost:8428":
-2022/08/30 19:48:24 Import finished!
-Total: 16 B ↗ Speed: 28.89 KiB p/s
-2022/08/30 19:48:24 Processing range 2/5: 2022-07-01T00:00:00Z - 2022-07-31T23:59:59Z
-2022/08/30 19:48:24 Initing export pipe from "http://localhost:8428" with filters:
- filter: match[]={__name__!=""}
- start: 2022-07-01T00:00:00Z
- end: 2022-07-31T23:59:59Z
-Initing import process to "http://localhost:8428":
-2022/08/30 19:48:24 Import finished!
-Total: 16 B ↗ Speed: 164.35 KiB p/s
-2022/08/30 19:48:24 Processing range 3/5: 2022-08-01T00:00:00Z - 2022-08-31T23:59:59Z
-2022/08/30 19:48:24 Initing export pipe from "http://localhost:8428" with filters:
- filter: match[]={__name__!=""}
- start: 2022-08-01T00:00:00Z
- end: 2022-08-31T23:59:59Z
-Initing import process to "http://localhost:8428":
-2022/08/30 19:48:24 Import finished!
-Total: 16 B ↗ Speed: 191.42 KiB p/s
-2022/08/30 19:48:24 Processing range 4/5: 2022-09-01T00:00:00Z - 2022-09-30T23:59:59Z
-2022/08/30 19:48:24 Initing export pipe from "http://localhost:8428" with filters:
- filter: match[]={__name__!=""}
- start: 2022-09-01T00:00:00Z
- end: 2022-09-30T23:59:59Z
-Initing import process to "http://localhost:8428":
-2022/08/30 19:48:24 Import finished!
-Total: 16 B ↗ Speed: 141.04 KiB p/s
-2022/08/30 19:48:24 Processing range 5/5: 2022-10-01T00:00:00Z - 2022-10-03T00:07:00Z
-2022/08/30 19:48:24 Initing export pipe from "http://localhost:8428" with filters:
- filter: match[]={__name__!=""}
- start: 2022-10-01T00:00:00Z
- end: 2022-10-03T00:07:00Z
-Initing import process to "http://localhost:8428":
-2022/08/30 19:48:24 Import finished!
-Total: 16 B ↗ Speed: 186.32 KiB p/s
-2022/08/30 19:48:24 Total time: 12.680582ms
+
+2023/03/02 09:18:05 Initing import process from "http://127.0.0.1:8481/select/0/prometheus/api/v1/export/native" to "http://localhost:8428/api/v1/import/native" with filter
+ filter: match[]={__name__=~"vm_cache_.*"}
+ start: 2022-11-20T00:00:00Z
+2023/03/02 09:18:05 Exploring metrics...
+Found 9 metrics to import. Continue? [Y/n]
+2023/03/02 09:18:07 Selected time range will be split into 5 ranges according to "month" step. Requests to make: 45.
+Requests to make: 45 / 45 [█████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████] 100.00%
+2023/03/02 09:18:12 Import finished!
+2023/03/02 09:18:12 VictoriaMetrics importer stats:
+ time spent while importing: 7.111870667s;
+ total bytes: 7.7 MB;
+ bytes/s: 1.1 MB;
+ requests: 45;
+ requests retries: 0;
+2023/03/02 09:18:12 Total time: 7.112405875s
```
#### Cluster-to-cluster migration mode
@@ -842,70 +847,41 @@ Cluster-to-cluster uses `/admin/tenants` endpoint (available starting from [v1.8
To use this mode you need to set `--vm-intercluster` flag to `true`, `--vm-native-src-addr` flag to 'http://vmselect:8481/' and `--vm-native-dst-addr` value to http://vminsert:8480/:
```console
-./bin/vmctl vm-native --vm-intercluster=true --vm-native-src-addr=http://localhost:8481/ --vm-native-dst-addr=http://172.17.0.3:8480/
+ ./vmctl vm-native --vm-native-src-addr=http://127.0.0.1:8481/ \
+ --vm-native-dst-addr=http://127.0.0.1:8480/ \
+ --vm-native-filter-match='{__name__="vm_app_uptime_seconds"}' \
+ --vm-native-filter-time-start='2023-02-01T00:00:00Z' \
+ --vm-native-step-interval=day \
+--vm-intercluster
VictoriaMetrics Native import mode
-2022/12/05 21:20:06 Discovered tenants: [123:1 12812919:1 1289198:1 1289:1283 12:1 1:0 1:1 1:1231231 1:1271727 1:12819 1:281 812891298:1]
-2022/12/05 21:20:06 Initing export pipe from "http://localhost:8481/select/123:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/123:1/prometheus/api/v1/import/native":
-Total: 61.13 MiB ↖ Speed: 2.05 MiB p/s
-Total: 61.13 MiB ↗ Speed: 2.30 MiB p/s
-2022/12/05 21:20:33 Initing export pipe from "http://localhost:8481/select/12812919:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/12812919:1/prometheus/api/v1/import/native":
-Total: 43.14 MiB ↘ Speed: 1.86 MiB p/s
-Total: 43.14 MiB ↙ Speed: 2.36 MiB p/s
-2022/12/05 21:20:51 Initing export pipe from "http://localhost:8481/select/1289198:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1289198:1/prometheus/api/v1/import/native":
-Total: 16.64 MiB ↗ Speed: 2.66 MiB p/s
-Total: 16.64 MiB ↘ Speed: 2.19 MiB p/s
-2022/12/05 21:20:59 Initing export pipe from "http://localhost:8481/select/1289:1283/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1289:1283/prometheus/api/v1/import/native":
-Total: 43.33 MiB ↙ Speed: 1.94 MiB p/s
-Total: 43.33 MiB ↖ Speed: 2.35 MiB p/s
-2022/12/05 21:21:18 Initing export pipe from "http://localhost:8481/select/12:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/12:1/prometheus/api/v1/import/native":
-Total: 63.78 MiB ↙ Speed: 1.96 MiB p/s
-Total: 63.78 MiB ↖ Speed: 2.28 MiB p/s
-2022/12/05 21:21:46 Initing export pipe from "http://localhost:8481/select/1:0/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:0/prometheus/api/v1/import/native":
-2022/12/05 21:21:46 Import finished!
-Total: 330 B ↗ Speed: 3.53 MiB p/s
-2022/12/05 21:21:46 Initing export pipe from "http://localhost:8481/select/1:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:1/prometheus/api/v1/import/native":
-Total: 63.81 MiB ↙ Speed: 1.96 MiB p/s
-Total: 63.81 MiB ↖ Speed: 2.28 MiB p/s
-2022/12/05 21:22:14 Initing export pipe from "http://localhost:8481/select/1:1231231/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:1231231/prometheus/api/v1/import/native":
-Total: 63.84 MiB ↙ Speed: 1.93 MiB p/s
-Total: 63.84 MiB ↖ Speed: 2.29 MiB p/s
-2022/12/05 21:22:42 Initing export pipe from "http://localhost:8481/select/1:1271727/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:1271727/prometheus/api/v1/import/native":
-Total: 54.37 MiB ↘ Speed: 1.90 MiB p/s
-Total: 54.37 MiB ↙ Speed: 2.37 MiB p/s
-2022/12/05 21:23:05 Initing export pipe from "http://localhost:8481/select/1:12819/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:12819/prometheus/api/v1/import/native":
-Total: 17.01 MiB ↙ Speed: 1.75 MiB p/s
-Total: 17.01 MiB ↖ Speed: 2.15 MiB p/s
-2022/12/05 21:23:13 Initing export pipe from "http://localhost:8481/select/1:281/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/1:281/prometheus/api/v1/import/native":
-Total: 63.89 MiB ↘ Speed: 1.90 MiB p/s
-Total: 63.89 MiB ↙ Speed: 2.29 MiB p/s
-2022/12/05 21:23:42 Initing export pipe from "http://localhost:8481/select/812891298:1/prometheus/api/v1/export/native" with filters:
- filter: match[]={__name__!=""}
-Initing import process to "http://172.17.0.3:8480/insert/812891298:1/prometheus/api/v1/import/native":
-Total: 63.84 MiB ↖ Speed: 1.99 MiB p/s
-Total: 63.84 MiB ↗ Speed: 2.26 MiB p/s
-2022/12/05 21:24:10 Total time: 4m4.1466565s
+2023/02/28 10:41:42 Discovering tenants...
+2023/02/28 10:41:42 The following tenants were discovered: [0:0 1:0 2:0 3:0 4:0]
+2023/02/28 10:41:42 Initing import process from "http://127.0.0.1:8481/select/0:0/prometheus/api/v1/export/native" to "http://127.0.0.1:8480/insert/0:0/prometheus/api/v1/import/native" with filter
+ filter: match[]={__name__="vm_app_uptime_seconds"}
+ start: 2023-02-01T00:00:00Z for tenant 0:0
+2023/02/28 10:41:42 Exploring metrics...
+2023/02/28 10:41:42 Found 1 metrics to import
+2023/02/28 10:41:42 Selected time range will be split into 28 ranges according to "day" step.
+Requests to make for tenant 0:0: 28 / 28 [███████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████] 100.00%
+
+2023/02/28 10:41:45 Initing import process from "http://127.0.0.1:8481/select/1:0/prometheus/api/v1/export/native" to "http://127.0.0.1:8480/insert/1:0/prometheus/api/v1/import/native" with filter
+ filter: match[]={__name__="vm_app_uptime_seconds"}
+ start: 2023-02-01T00:00:00Z for tenant 1:0
+2023/02/28 10:41:45 Exploring metrics...
+2023/02/28 10:41:45 Found 1 metrics to import
+2023/02/28 10:41:45 Selected time range will be split into 28 ranges according to "day" step. Requests to make: 28
+Requests to make for tenant 1:0: 28 / 28 [████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████████] 100.00%
+
+...
+
+2023/02/28 10:42:49 Import finished!
+2023/02/28 10:42:49 VictoriaMetrics importer stats:
+ time spent while importing: 1m6.714210417s;
+ total bytes: 39.7 MB;
+ bytes/s: 594.4 kB;
+ requests: 140;
+ requests retries: 0;
+2023/02/28 10:42:49 Total time: 1m7.147971417s
```
## Verifying exported blocks from VictoriaMetrics
@@ -972,6 +948,7 @@ a sign of network issues or VM being overloaded. See the logs during import for
By default `vmctl` waits confirmation from user before starting the import. If this is unwanted
behavior and no user interaction required - pass `-s` flag to enable "silence" mode:
+See below the example of `vm-native` migration process:
```
-s Whether to run in silent mode. If set to true no confirmation prompts will appear. (default: false)
```
diff --git a/app/vmctl/backoff/backoff.go b/app/vmctl/backoff/backoff.go
index beb14304d..431d6a0c4 100644
--- a/app/vmctl/backoff/backoff.go
+++ b/app/vmctl/backoff/backoff.go
@@ -47,7 +47,7 @@ func (b *Backoff) Retry(ctx context.Context, cb retryableFunc) (uint64, error) {
if err == nil {
return attempt, nil
}
- if errors.Is(err, ErrBadRequest) {
+ if errors.Is(err, ErrBadRequest) || errors.Is(err, context.Canceled) {
logger.Errorf("unrecoverable error: %s", err)
return attempt, err // fail fast if not recoverable
}
diff --git a/app/vmctl/flags.go b/app/vmctl/flags.go
index 1f044a7ea..afebd79a8 100644
--- a/app/vmctl/flags.go
+++ b/app/vmctl/flags.go
@@ -328,10 +328,12 @@ const (
vmNativeSrcAddr = "vm-native-src-addr"
vmNativeSrcUser = "vm-native-src-user"
vmNativeSrcPassword = "vm-native-src-password"
+ vmNativeSrcHeaders = "vm-native-src-headers"
vmNativeDstAddr = "vm-native-dst-addr"
vmNativeDstUser = "vm-native-dst-user"
vmNativeDstPassword = "vm-native-dst-password"
+ vmNativeDstHeaders = "vm-native-dst-headers"
)
var (
@@ -344,8 +346,9 @@ var (
Value: `{__name__!=""}`,
},
&cli.StringFlag{
- Name: vmNativeFilterTimeStart,
- Usage: "The time filter may contain either unix timestamp in seconds or RFC3339 values. E.g. '2020-01-01T20:07:00Z'",
+ Name: vmNativeFilterTimeStart,
+ Usage: "The time filter may contain either unix timestamp in seconds or RFC3339 values. E.g. '2020-01-01T20:07:00Z'",
+ Required: true,
},
&cli.StringFlag{
Name: vmNativeFilterTimeEnd,
@@ -372,6 +375,12 @@ var (
Usage: "VictoriaMetrics password for basic auth",
EnvVars: []string{"VM_NATIVE_SRC_PASSWORD"},
},
+ &cli.StringFlag{
+ Name: vmNativeSrcHeaders,
+ Usage: "Optional HTTP headers to send with each request to the corresponding source address. \n" +
+ "For example, --vm-native-src-headers='My-Auth:foobar' would send 'My-Auth: foobar' HTTP header with every request to the corresponding source address. \n" +
+ "Multiple headers must be delimited by '^^': --vm-native-src-headers='header1:value1^^header2:value2'",
+ },
&cli.StringFlag{
Name: vmNativeDstAddr,
Usage: "VictoriaMetrics address to perform import to. \n" +
@@ -389,6 +398,12 @@ var (
Usage: "VictoriaMetrics password for basic auth",
EnvVars: []string{"VM_NATIVE_DST_PASSWORD"},
},
+ &cli.StringFlag{
+ Name: vmNativeDstHeaders,
+ Usage: "Optional HTTP headers to send with each request to the corresponding destination address. \n" +
+ "For example, --vm-native-dst-headers='My-Auth:foobar' would send 'My-Auth: foobar' HTTP header with every request to the corresponding destination address. \n" +
+ "Multiple headers must be delimited by '^^': --vm-native-dst-headers='header1:value1^^header2:value2'",
+ },
&cli.StringSliceFlag{
Name: vmExtraLabel,
Value: nil,
@@ -406,6 +421,11 @@ var (
fmt.Sprintf(" In this mode --%s flag format is: 'http://vmselect:8481/'. --%s flag format is: http://vminsert:8480/. \n", vmNativeSrcAddr, vmNativeDstAddr) +
" TenantID will be appended automatically after discovering tenants from src.",
},
+ &cli.UintFlag{
+ Name: vmConcurrency,
+ Usage: "Number of workers concurrently performing import requests to VM",
+ Value: 2,
+ },
}
)
@@ -485,7 +505,7 @@ var (
},
&cli.DurationFlag{
Name: remoteReadHTTPTimeout,
- Usage: "Timeout defines timeout for HTTP write request to remote storage",
+ Usage: "Timeout defines timeout for HTTP requests made by remote read client",
},
&cli.StringFlag{
Name: remoteReadHeaders,
diff --git a/app/vmctl/main.go b/app/vmctl/main.go
index dbfc332f1..aa5bad045 100644
--- a/app/vmctl/main.go
+++ b/app/vmctl/main.go
@@ -11,6 +11,8 @@ import (
"syscall"
"time"
+ "github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/backoff"
+ "github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/native"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/remoteread"
"github.com/urfave/cli/v2"
@@ -189,7 +191,7 @@ func main() {
{
Name: "vm-native",
Usage: "Migrate time series between VictoriaMetrics installations via native binary format",
- Flags: vmNativeFlags,
+ Flags: mergeFlags(globalFlags, vmNativeFlags),
Action: func(c *cli.Context) error {
fmt.Println("VictoriaMetrics Native import mode")
@@ -200,25 +202,29 @@ func main() {
p := vmNativeProcessor{
rateLimit: c.Int64(vmRateLimit),
interCluster: c.Bool(vmInterCluster),
- filter: filter{
- match: c.String(vmNativeFilterMatch),
- timeStart: c.String(vmNativeFilterTimeStart),
- timeEnd: c.String(vmNativeFilterTimeEnd),
- chunk: c.String(vmNativeStepInterval),
+ filter: native.Filter{
+ Match: c.String(vmNativeFilterMatch),
+ TimeStart: c.String(vmNativeFilterTimeStart),
+ TimeEnd: c.String(vmNativeFilterTimeEnd),
+ Chunk: c.String(vmNativeStepInterval),
},
- src: &vmNativeClient{
- addr: strings.Trim(c.String(vmNativeSrcAddr), "/"),
- user: c.String(vmNativeSrcUser),
- password: c.String(vmNativeSrcPassword),
+ src: &native.Client{
+ Addr: strings.Trim(c.String(vmNativeSrcAddr), "/"),
+ User: c.String(vmNativeSrcUser),
+ Password: c.String(vmNativeSrcPassword),
+ Headers: c.String(vmNativeSrcHeaders),
},
- dst: &vmNativeClient{
- addr: strings.Trim(c.String(vmNativeDstAddr), "/"),
- user: c.String(vmNativeDstUser),
- password: c.String(vmNativeDstPassword),
- extraLabels: c.StringSlice(vmExtraLabel),
+ dst: &native.Client{
+ Addr: strings.Trim(c.String(vmNativeDstAddr), "/"),
+ User: c.String(vmNativeDstUser),
+ Password: c.String(vmNativeDstPassword),
+ ExtraLabels: c.StringSlice(vmExtraLabel),
+ Headers: c.String(vmNativeDstHeaders),
},
+ backoff: backoff.New(),
+ cc: c.Int(vmConcurrency),
}
- return p.run(ctx)
+ return p.run(ctx, c.Bool(globalSilent))
},
},
{
diff --git a/app/vmctl/native/client.go b/app/vmctl/native/client.go
new file mode 100644
index 000000000..ad73224c5
--- /dev/null
+++ b/app/vmctl/native/client.go
@@ -0,0 +1,237 @@
+package native
+
+import (
+ "context"
+ "encoding/json"
+ "fmt"
+ "io"
+ "net/http"
+ "strings"
+)
+
+const (
+ nativeTenantsAddr = "admin/tenants"
+ nativeSeriesAddr = "api/v1/series"
+ nameLabel = "__name__"
+)
+
+// Client is an HTTP client for exporting and importing
+// time series via native protocol.
+type Client struct {
+ Addr string
+ User string
+ Password string
+ ExtraLabels []string
+ Headers string
+}
+
+// LabelValues represents series from api/v1/series response
+type LabelValues map[string]string
+
+// Response represents response from api/v1/series
+type Response struct {
+ Status string `json:"status"`
+ Series []LabelValues `json:"data"`
+}
+
+// Explore finds series by provided filter from api/v1/series
+func (c *Client) Explore(ctx context.Context, f Filter, tenantID string) (map[string]struct{}, error) {
+ url := fmt.Sprintf("%s/%s", c.Addr, nativeSeriesAddr)
+ if tenantID != "" {
+ url = fmt.Sprintf("%s/select/%s/prometheus/%s", c.Addr, tenantID, nativeSeriesAddr)
+ }
+ req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil)
+ if err != nil {
+ return nil, fmt.Errorf("cannot create request to %q: %s", url, err)
+ }
+
+ params := req.URL.Query()
+ if f.TimeStart != "" {
+ params.Set("start", f.TimeStart)
+ }
+ if f.TimeEnd != "" {
+ params.Set("end", f.TimeEnd)
+ }
+ params.Set("match[]", f.Match)
+ req.URL.RawQuery = params.Encode()
+
+ resp, err := c.do(req, http.StatusOK)
+ if err != nil {
+ return nil, fmt.Errorf("series request failed: %s", err)
+ }
+
+ var response Response
+ if err := json.NewDecoder(resp.Body).Decode(&response); err != nil {
+ return nil, fmt.Errorf("cannot decode series response: %s", err)
+ }
+
+ if err := resp.Body.Close(); err != nil {
+ return nil, fmt.Errorf("cannot close series response body: %s", err)
+ }
+ names := make(map[string]struct{})
+ for _, series := range response.Series {
+ // TODO: consider tweaking /api/v1/series API to return metric names only
+ // this could make explore response much lighter.
+ for key, value := range series {
+ if key != nameLabel {
+ continue
+ }
+ if _, ok := names[value]; ok {
+ continue
+ }
+ names[value] = struct{}{}
+ }
+ }
+ return names, nil
+}
+
+// ImportPipe uses pipe reader in request to process data
+func (c *Client) ImportPipe(ctx context.Context, dstURL string, pr *io.PipeReader) error {
+ req, err := http.NewRequestWithContext(ctx, http.MethodPost, dstURL, pr)
+ if err != nil {
+ return fmt.Errorf("cannot create import request to %q: %s", c.Addr, err)
+ }
+
+ parsedHeaders, err := parseHeaders(c.Headers)
+ if err != nil {
+ return err
+ }
+
+ for _, header := range parsedHeaders {
+ req.Header.Set(header.key, header.value)
+ }
+
+ importResp, err := c.do(req, http.StatusNoContent)
+ if err != nil {
+ return fmt.Errorf("import request failed: %s", err)
+ }
+ if err := importResp.Body.Close(); err != nil {
+ return fmt.Errorf("cannot close import response body: %s", err)
+ }
+ return nil
+}
+
+// ExportPipe makes request by provided filter and return io.ReadCloser which can be used to get data
+func (c *Client) ExportPipe(ctx context.Context, url string, f Filter) (io.ReadCloser, error) {
+ req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil)
+ if err != nil {
+ return nil, fmt.Errorf("cannot create request to %q: %s", c.Addr, err)
+ }
+
+ params := req.URL.Query()
+ params.Set("match[]", f.Match)
+ if f.TimeStart != "" {
+ params.Set("start", f.TimeStart)
+ }
+ if f.TimeEnd != "" {
+ params.Set("end", f.TimeEnd)
+ }
+ req.URL.RawQuery = params.Encode()
+
+ // disable compression since it is meaningless for native format
+ req.Header.Set("Accept-Encoding", "identity")
+
+ parsedHeaders, err := parseHeaders(c.Headers)
+ if err != nil {
+ return nil, err
+ }
+
+ for _, header := range parsedHeaders {
+ req.Header.Set(header.key, header.value)
+ }
+
+ resp, err := c.do(req, http.StatusOK)
+ if err != nil {
+ return nil, fmt.Errorf("export request failed: %w", err)
+ }
+ return resp.Body, nil
+}
+
+// GetSourceTenants discovers tenants by provided filter
+func (c *Client) GetSourceTenants(ctx context.Context, f Filter) ([]string, error) {
+ u := fmt.Sprintf("%s/%s", c.Addr, nativeTenantsAddr)
+ req, err := http.NewRequestWithContext(ctx, http.MethodGet, u, nil)
+ if err != nil {
+ return nil, fmt.Errorf("cannot create request to %q: %s", u, err)
+ }
+
+ params := req.URL.Query()
+ if f.TimeStart != "" {
+ params.Set("start", f.TimeStart)
+ }
+ if f.TimeEnd != "" {
+ params.Set("end", f.TimeEnd)
+ }
+ req.URL.RawQuery = params.Encode()
+
+ parsedHeaders, err := parseHeaders(c.Headers)
+ if err != nil {
+ return nil, err
+ }
+
+ for _, header := range parsedHeaders {
+ req.Header.Set(header.key, header.value)
+ }
+
+ resp, err := c.do(req, http.StatusOK)
+ if err != nil {
+ return nil, fmt.Errorf("tenants request failed: %s", err)
+ }
+
+ var r struct {
+ Tenants []string `json:"data"`
+ }
+ if err := json.NewDecoder(resp.Body).Decode(&r); err != nil {
+ return nil, fmt.Errorf("cannot decode tenants response: %s", err)
+ }
+
+ if err := resp.Body.Close(); err != nil {
+ return nil, fmt.Errorf("cannot close tenants response body: %s", err)
+ }
+
+ return r.Tenants, nil
+}
+
+func (c *Client) do(req *http.Request, expSC int) (*http.Response, error) {
+ if c.User != "" {
+ req.SetBasicAuth(c.User, c.Password)
+ }
+ resp, err := http.DefaultClient.Do(req)
+ if err != nil {
+ return nil, fmt.Errorf("unexpected error when performing request: %w", err)
+ }
+
+ if resp.StatusCode != expSC {
+ body, err := io.ReadAll(resp.Body)
+ if err != nil {
+ return nil, fmt.Errorf("failed to read response body for status code %d: %s", resp.StatusCode, err)
+ }
+ return nil, fmt.Errorf("unexpected response code %d: %s", resp.StatusCode, string(body))
+ }
+ return resp, err
+}
+
+type keyValue struct {
+ key string
+ value string
+}
+
+func parseHeaders(headers string) ([]keyValue, error) {
+ if len(headers) == 0 {
+ return nil, nil
+ }
+
+ var headersSplitByDelimiter = strings.Split(headers, "^^")
+
+ kvs := make([]keyValue, len(headersSplitByDelimiter))
+ for i, h := range headersSplitByDelimiter {
+ n := strings.IndexByte(h, ':')
+ if n < 0 {
+ return nil, fmt.Errorf(`missing ':' in header %q; expecting "key: value" format`, h)
+ }
+ kv := &kvs[i]
+ kv.key = strings.TrimSpace(h[:n])
+ kv.value = strings.TrimSpace(h[n+1:])
+ }
+ return kvs, nil
+}
diff --git a/app/vmctl/native/filter.go b/app/vmctl/native/filter.go
new file mode 100644
index 000000000..a038ba3ca
--- /dev/null
+++ b/app/vmctl/native/filter.go
@@ -0,0 +1,22 @@
+package native
+
+import "fmt"
+
+// Filter represents request filter
+type Filter struct {
+ Match string
+ TimeStart string
+ TimeEnd string
+ Chunk string
+}
+
+func (f Filter) String() string {
+ s := fmt.Sprintf("\n\tfilter: match[]=%s", f.Match)
+ if f.TimeStart != "" {
+ s += fmt.Sprintf("\n\tstart: %s", f.TimeStart)
+ }
+ if f.TimeEnd != "" {
+ s += fmt.Sprintf("\n\tend: %s", f.TimeEnd)
+ }
+ return s
+}
diff --git a/app/vmctl/remoteread/remoteread.go b/app/vmctl/remoteread/remoteread.go
index 49a6f9d8c..c27ed799a 100644
--- a/app/vmctl/remoteread/remoteread.go
+++ b/app/vmctl/remoteread/remoteread.go
@@ -21,7 +21,7 @@ import (
)
const (
- defaultReadTimeout = 30 * time.Second
+ defaultReadTimeout = 5 * time.Minute
remoteReadPath = "/api/v1/read"
healthPath = "/-/healthy"
)
diff --git a/app/vmctl/vm_native.go b/app/vmctl/vm_native.go
index b86dc81f7..999fd3b63 100644
--- a/app/vmctl/vm_native.go
+++ b/app/vmctl/vm_native.go
@@ -2,177 +2,125 @@ package main
import (
"context"
- "encoding/json"
"fmt"
"io"
"log"
- "net/http"
+ "sync"
"time"
- "github.com/cheggaaa/pb/v3"
-
+ "github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/backoff"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/limiter"
+ "github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/native"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/stepper"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/vm"
+ "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
+ "github.com/cheggaaa/pb/v3"
)
type vmNativeProcessor struct {
- filter filter
- rateLimit int64
+ filter native.Filter
- dst *vmNativeClient
- src *vmNativeClient
+ dst *native.Client
+ src *native.Client
+ backoff *backoff.Backoff
+
+ s *stats
+ rateLimit int64
interCluster bool
-}
-
-type vmNativeClient struct {
- addr string
- user string
- password string
- extraLabels []string
-}
-
-type filter struct {
- match string
- timeStart string
- timeEnd string
- chunk string
-}
-
-func (f filter) String() string {
- s := fmt.Sprintf("\n\tfilter: match[]=%s", f.match)
- if f.timeStart != "" {
- s += fmt.Sprintf("\n\tstart: %s", f.timeStart)
- }
- if f.timeEnd != "" {
- s += fmt.Sprintf("\n\tend: %s", f.timeEnd)
- }
- return s
+ cc int
}
const (
- nativeExportAddr = "api/v1/export/native"
- nativeImportAddr = "api/v1/import/native"
- nativeTenantsAddr = "admin/tenants"
-
- nativeBarTpl = `Total: {{counters . }} {{ cycle . "↖" "↗" "↘" "↙" }} Speed: {{speed . }} {{string . "suffix"}}`
+ nativeExportAddr = "api/v1/export/native"
+ nativeImportAddr = "api/v1/import/native"
+ nativeBarTpl = `{{ blue "%s:" }} {{ counters . }} {{ bar . "[" "█" (cycle . "█") "▒" "]" }} {{ percent . }}`
)
-func (p *vmNativeProcessor) run(ctx context.Context) error {
- if p.filter.chunk == "" {
- return p.runWithFilter(ctx, p.filter)
+func (p *vmNativeProcessor) run(ctx context.Context, silent bool) error {
+ if p.cc == 0 {
+ p.cc = 1
+ }
+ p.s = &stats{
+ startTime: time.Now(),
}
- startOfRange, err := time.Parse(time.RFC3339, p.filter.timeStart)
+ start, err := time.Parse(time.RFC3339, p.filter.TimeStart)
if err != nil {
- return fmt.Errorf("failed to parse %s, provided: %s, expected format: %s, error: %v", vmNativeFilterTimeStart, p.filter.timeStart, time.RFC3339, err)
+ return fmt.Errorf("failed to parse %s, provided: %s, expected format: %s, error: %w",
+ vmNativeFilterTimeStart, p.filter.TimeStart, time.RFC3339, err)
}
- var endOfRange time.Time
- if p.filter.timeEnd != "" {
- endOfRange, err = time.Parse(time.RFC3339, p.filter.timeEnd)
+ end := time.Now().In(start.Location())
+ if p.filter.TimeEnd != "" {
+ end, err = time.Parse(time.RFC3339, p.filter.TimeEnd)
if err != nil {
- return fmt.Errorf("failed to parse %s, provided: %s, expected format: %s, error: %v", vmNativeFilterTimeEnd, p.filter.timeEnd, time.RFC3339, err)
+ return fmt.Errorf("failed to parse %s, provided: %s, expected format: %s, error: %w",
+ vmNativeFilterTimeEnd, p.filter.TimeEnd, time.RFC3339, err)
}
- } else {
- endOfRange = time.Now()
}
- ranges, err := stepper.SplitDateRange(startOfRange, endOfRange, p.filter.chunk)
- if err != nil {
- return fmt.Errorf("failed to create date ranges for the given time filters: %v", err)
- }
-
- for rangeIdx, r := range ranges {
- formattedStartTime := r[0].Format(time.RFC3339)
- formattedEndTime := r[1].Format(time.RFC3339)
- log.Printf("Processing range %d/%d: %s - %s \n", rangeIdx+1, len(ranges), formattedStartTime, formattedEndTime)
- f := filter{
- match: p.filter.match,
- timeStart: formattedStartTime,
- timeEnd: formattedEndTime,
- }
- err := p.runWithFilter(ctx, f)
-
+ ranges := [][]time.Time{{start, end}}
+ if p.filter.Chunk != "" {
+ ranges, err = stepper.SplitDateRange(start, end, p.filter.Chunk)
if err != nil {
- log.Printf("processing failed for range %d/%d: %s - %s \n", rangeIdx+1, len(ranges), formattedStartTime, formattedEndTime)
- return err
+ return fmt.Errorf("failed to create date ranges for the given time filters: %w", err)
}
}
+
+ tenants := []string{""}
+ if p.interCluster {
+ log.Printf("Discovering tenants...")
+ tenants, err = p.src.GetSourceTenants(ctx, p.filter)
+ if err != nil {
+ return fmt.Errorf("failed to get tenants: %w", err)
+ }
+ question := fmt.Sprintf("The following tenants were discovered: %s.\n Continue?", tenants)
+ if !silent && !prompt(question) {
+ return nil
+ }
+ }
+
+ for _, tenantID := range tenants {
+ err := p.runBackfilling(ctx, tenantID, ranges, silent)
+ if err != nil {
+ return fmt.Errorf("migration failed: %s", err)
+ }
+ }
+
+ log.Println("Import finished!")
+ log.Print(p.s)
+
return nil
}
-func (p *vmNativeProcessor) runWithFilter(ctx context.Context, f filter) error {
- nativeImportAddr, err := vm.AddExtraLabelsToImportPath(nativeImportAddr, p.dst.extraLabels)
+func (p *vmNativeProcessor) do(ctx context.Context, f native.Filter, srcURL, dstURL string) error {
+ retryableFunc := func() error { return p.runSingle(ctx, f, srcURL, dstURL) }
+ attempts, err := p.backoff.Retry(ctx, retryableFunc)
+ p.s.Lock()
+ p.s.retries += attempts
+ p.s.Unlock()
if err != nil {
- return fmt.Errorf("failed to add labels to import path: %s", err)
- }
-
- if !p.interCluster {
- srcURL := fmt.Sprintf("%s/%s", p.src.addr, nativeExportAddr)
- dstURL := fmt.Sprintf("%s/%s", p.dst.addr, nativeImportAddr)
-
- return p.runSingle(ctx, f, srcURL, dstURL)
- }
-
- tenants, err := p.getSourceTenants(ctx, f)
- if err != nil {
- return fmt.Errorf("failed to get source tenants: %s", err)
- }
-
- log.Printf("Discovered tenants: %v", tenants)
- for _, tenant := range tenants {
- // src and dst expected formats: http://vminsert:8480/ and http://vmselect:8481/
- srcURL := fmt.Sprintf("%s/select/%s/prometheus/%s", p.src.addr, tenant, nativeExportAddr)
- dstURL := fmt.Sprintf("%s/insert/%s/prometheus/%s", p.dst.addr, tenant, nativeImportAddr)
-
- if err := p.runSingle(ctx, f, srcURL, dstURL); err != nil {
- return fmt.Errorf("failed to migrate data for tenant %q: %s", tenant, err)
- }
+ return fmt.Errorf("failed to migrate from %s to %s (retry attempts: %d): %w\nwith fileter %s", srcURL, dstURL, attempts, err, f)
}
return nil
}
-func (p *vmNativeProcessor) runSingle(ctx context.Context, f filter, srcURL, dstURL string) error {
- log.Printf("Initing export pipe from %q with filters: %s\n", srcURL, f)
+func (p *vmNativeProcessor) runSingle(ctx context.Context, f native.Filter, srcURL, dstURL string) error {
- exportReader, err := p.exportPipe(ctx, srcURL, f)
+ exportReader, err := p.src.ExportPipe(ctx, srcURL, f)
if err != nil {
- return fmt.Errorf("failed to init export pipe: %s", err)
+ return fmt.Errorf("failed to init export pipe: %w", err)
}
pr, pw := io.Pipe()
- sync := make(chan struct{})
+ done := make(chan struct{})
go func() {
- defer func() { close(sync) }()
- req, err := http.NewRequestWithContext(ctx, http.MethodPost, dstURL, pr)
- if err != nil {
- log.Fatalf("cannot create import request to %q: %s", p.dst.addr, err)
- }
- importResp, err := p.dst.do(req, http.StatusNoContent)
- if err != nil {
- log.Fatalf("import request failed: %s", err)
- }
- if err := importResp.Body.Close(); err != nil {
- log.Fatalf("cannot close import response body: %s", err)
- }
- }()
-
- fmt.Printf("Initing import process to %q:\n", dstURL)
- pool := pb.NewPool()
- bar := pb.ProgressBarTemplate(nativeBarTpl).New(0)
- pool.Add(bar)
- barReader := bar.NewProxyReader(exportReader)
- if err := pool.Start(); err != nil {
- log.Printf("error start process bars pool: %s", err)
- return err
- }
- defer func() {
- bar.Finish()
- if err := pool.Stop(); err != nil {
- fmt.Printf("failed to stop barpool: %+v\n", err)
+ defer func() { close(done) }()
+ if err := p.dst.ImportPipe(ctx, dstURL, pr); err != nil {
+ logger.Errorf("error initialize import pipe: %s", err)
+ return
}
}()
@@ -182,95 +130,176 @@ func (p *vmNativeProcessor) runSingle(ctx context.Context, f filter, srcURL, dst
w = limiter.NewWriteLimiter(pw, rl)
}
- _, err = io.Copy(w, barReader)
+ written, err := io.Copy(w, exportReader)
if err != nil {
- return fmt.Errorf("failed to write into %q: %s", p.dst.addr, err)
+ return fmt.Errorf("failed to write into %q: %s", p.dst.Addr, err)
}
+ p.s.Lock()
+ p.s.bytes += uint64(written)
+ p.s.requests++
+ p.s.Unlock()
+
if err := pw.Close(); err != nil {
return err
}
- <-sync
+ <-done
- log.Println("Import finished!")
return nil
}
-func (p *vmNativeProcessor) getSourceTenants(ctx context.Context, f filter) ([]string, error) {
- u := fmt.Sprintf("%s/%s", p.src.addr, nativeTenantsAddr)
- req, err := http.NewRequestWithContext(ctx, http.MethodGet, u, nil)
+func (p *vmNativeProcessor) runBackfilling(ctx context.Context, tenantID string, ranges [][]time.Time, silent bool) error {
+ exportAddr := nativeExportAddr
+ srcURL := fmt.Sprintf("%s/%s", p.src.Addr, exportAddr)
+
+ importAddr, err := vm.AddExtraLabelsToImportPath(nativeImportAddr, p.dst.ExtraLabels)
if err != nil {
- return nil, fmt.Errorf("cannot create request to %q: %s", u, err)
+ return fmt.Errorf("failed to add labels to import path: %s", err)
+ }
+ dstURL := fmt.Sprintf("%s/%s", p.dst.Addr, importAddr)
+
+ if p.interCluster {
+ srcURL = fmt.Sprintf("%s/select/%s/prometheus/%s", p.src.Addr, tenantID, exportAddr)
+ dstURL = fmt.Sprintf("%s/insert/%s/prometheus/%s", p.dst.Addr, tenantID, importAddr)
}
- params := req.URL.Query()
- if f.timeStart != "" {
- params.Set("start", f.timeStart)
+ barPrefix := "Requests to make"
+ initMessage := "Initing import process from %q to %q with filter %s"
+ initParams := []interface{}{srcURL, dstURL, p.filter.String()}
+ if p.interCluster {
+ barPrefix = fmt.Sprintf("Requests to make for tenant %s", tenantID)
+ initMessage = "Initing import process from %q to %q with filter %s for tenant %s"
+ initParams = []interface{}{srcURL, dstURL, p.filter.String(), tenantID}
}
- if f.timeEnd != "" {
- params.Set("end", f.timeEnd)
- }
- req.URL.RawQuery = params.Encode()
- resp, err := p.src.do(req, http.StatusOK)
+ fmt.Println("") // extra line for better output formatting
+ log.Printf(initMessage, initParams...)
+
+ log.Printf("Exploring metrics...")
+ metrics, err := p.src.Explore(ctx, p.filter, tenantID)
if err != nil {
- return nil, fmt.Errorf("tenants request failed: %s", err)
+ return fmt.Errorf("cannot get metrics from source %s: %w", p.src.Addr, err)
}
- var r struct {
- Tenants []string `json:"data"`
- }
- if err := json.NewDecoder(resp.Body).Decode(&r); err != nil {
- return nil, fmt.Errorf("cannot decode tenants response: %s", err)
+ if len(metrics) == 0 {
+ return fmt.Errorf("no metrics found")
}
- if err := resp.Body.Close(); err != nil {
- return nil, fmt.Errorf("cannot close tenants response body: %s", err)
- }
-
- return r.Tenants, nil
-}
-
-func (p *vmNativeProcessor) exportPipe(ctx context.Context, url string, f filter) (io.ReadCloser, error) {
- req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil)
- if err != nil {
- return nil, fmt.Errorf("cannot create request to %q: %s", p.src.addr, err)
- }
-
- params := req.URL.Query()
- params.Set("match[]", f.match)
- if f.timeStart != "" {
- params.Set("start", f.timeStart)
- }
- if f.timeEnd != "" {
- params.Set("end", f.timeEnd)
- }
- req.URL.RawQuery = params.Encode()
-
- // disable compression since it is meaningless for native format
- req.Header.Set("Accept-Encoding", "identity")
- resp, err := p.src.do(req, http.StatusOK)
- if err != nil {
- return nil, fmt.Errorf("export request failed: %s", err)
- }
- return resp.Body, nil
-}
-
-func (c *vmNativeClient) do(req *http.Request, expSC int) (*http.Response, error) {
- if c.user != "" {
- req.SetBasicAuth(c.user, c.password)
- }
- resp, err := http.DefaultClient.Do(req)
- if err != nil {
- return nil, fmt.Errorf("unexpected error when performing request: %s", err)
- }
-
- if resp.StatusCode != expSC {
- body, err := io.ReadAll(resp.Body)
- if err != nil {
- return nil, fmt.Errorf("failed to read response body for status code %d: %s", resp.StatusCode, err)
+ foundSeriesMsg := fmt.Sprintf("Found %d metrics to import", len(metrics))
+ if !p.interCluster {
+ // do not prompt for intercluster because there could be many tenants,
+ // and we don't want to interrupt the process when moving to the next tenant.
+ question := foundSeriesMsg + ". Continue?"
+ if !silent && !prompt(question) {
+ return nil
}
- return nil, fmt.Errorf("unexpected response code %d: %s", resp.StatusCode, string(body))
+ } else {
+ log.Print(foundSeriesMsg)
}
- return resp, err
+
+ processingMsg := fmt.Sprintf("Requests to make: %d", len(metrics)*len(ranges))
+ if len(ranges) > 1 {
+ processingMsg = fmt.Sprintf("Selected time range will be split into %d ranges according to %q step. %s", len(ranges), p.filter.Chunk, processingMsg)
+ }
+ log.Print(processingMsg)
+
+ var bar *pb.ProgressBar
+ if !silent {
+ bar = pb.ProgressBarTemplate(fmt.Sprintf(nativeBarTpl, barPrefix)).New(len(metrics) * len(ranges))
+ bar.Start()
+ defer bar.Finish()
+ }
+
+ filterCh := make(chan native.Filter)
+ errCh := make(chan error, p.cc)
+
+ var wg sync.WaitGroup
+ for i := 0; i < p.cc; i++ {
+ wg.Add(1)
+ go func() {
+ defer wg.Done()
+ for f := range filterCh {
+ if err := p.do(ctx, f, srcURL, dstURL); err != nil {
+ errCh <- err
+ return
+ }
+ if bar != nil {
+ bar.Increment()
+ }
+ }
+ }()
+ }
+
+ // any error breaks the import
+ for s := range metrics {
+ for _, times := range ranges {
+ select {
+ case <-ctx.Done():
+ return fmt.Errorf("context canceled")
+ case infErr := <-errCh:
+ return fmt.Errorf("native error: %s", infErr)
+ case filterCh <- native.Filter{
+ Match: fmt.Sprintf("{%s=%q}", nameLabel, s),
+ TimeStart: times[0].Format(time.RFC3339),
+ TimeEnd: times[1].Format(time.RFC3339),
+ }:
+ }
+ }
+ }
+
+ close(filterCh)
+ wg.Wait()
+ close(errCh)
+
+ for err := range errCh {
+ return fmt.Errorf("import process failed: %s", err)
+ }
+
+ return nil
+}
+
+// stats represents client statistic
+// when processing data
+type stats struct {
+ sync.Mutex
+ startTime time.Time
+ bytes uint64
+ requests uint64
+ retries uint64
+}
+
+func (s *stats) String() string {
+ s.Lock()
+ defer s.Unlock()
+
+ totalImportDuration := time.Since(s.startTime)
+ totalImportDurationS := totalImportDuration.Seconds()
+ bytesPerS := byteCountSI(0)
+ if s.bytes > 0 && totalImportDurationS > 0 {
+ bytesPerS = byteCountSI(int64(float64(s.bytes) / totalImportDurationS))
+ }
+
+ return fmt.Sprintf("VictoriaMetrics importer stats:\n"+
+ " time spent while importing: %v;\n"+
+ " total bytes: %s;\n"+
+ " bytes/s: %s;\n"+
+ " requests: %d;\n"+
+ " requests retries: %d;",
+ totalImportDuration,
+ byteCountSI(int64(s.bytes)), bytesPerS,
+ s.requests, s.retries)
+}
+
+func byteCountSI(b int64) string {
+ const unit = 1000
+ if b < unit {
+ return fmt.Sprintf("%d B", b)
+ }
+ div, exp := int64(unit), 0
+ for n := b / unit; n >= unit; n /= unit {
+ div *= unit
+ exp++
+ }
+ return fmt.Sprintf("%.1f %cB",
+ float64(b)/float64(div), "kMGTPE"[exp])
}
diff --git a/app/vmctl/vm_native_test.go b/app/vmctl/vm_native_test.go
index b254dc33c..79329d8f5 100644
--- a/app/vmctl/vm_native_test.go
+++ b/app/vmctl/vm_native_test.go
@@ -5,6 +5,7 @@ import (
"testing"
"time"
+ "github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/native"
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmctl/stepper"
)
@@ -27,10 +28,10 @@ const (
func Test_vmNativeProcessor_run(t *testing.T) {
t.Skip()
type fields struct {
- filter filter
+ filter native.Filter
rateLimit int64
- dst *vmNativeClient
- src *vmNativeClient
+ dst *native.Client
+ src *native.Client
}
tests := []struct {
name string
@@ -41,16 +42,16 @@ func Test_vmNativeProcessor_run(t *testing.T) {
{
name: "simulate syscall.SIGINT",
fields: fields{
- filter: filter{
- match: matchFilter,
- timeStart: timeStartFilter,
+ filter: native.Filter{
+ Match: matchFilter,
+ TimeStart: timeStartFilter,
},
rateLimit: 0,
- dst: &vmNativeClient{
- addr: dstAddr,
+ dst: &native.Client{
+ Addr: dstAddr,
},
- src: &vmNativeClient{
- addr: srcAddr,
+ src: &native.Client{
+ Addr: srcAddr,
},
},
closer: func(cancelFunc context.CancelFunc) {
@@ -62,16 +63,16 @@ func Test_vmNativeProcessor_run(t *testing.T) {
{
name: "simulate correct work",
fields: fields{
- filter: filter{
- match: matchFilter,
- timeStart: timeStartFilter,
+ filter: native.Filter{
+ Match: matchFilter,
+ TimeStart: timeStartFilter,
},
rateLimit: 0,
- dst: &vmNativeClient{
- addr: dstAddr,
+ dst: &native.Client{
+ Addr: dstAddr,
},
- src: &vmNativeClient{
- addr: srcAddr,
+ src: &native.Client{
+ Addr: srcAddr,
},
},
closer: func(cancelFunc context.CancelFunc) {},
@@ -80,18 +81,18 @@ func Test_vmNativeProcessor_run(t *testing.T) {
{
name: "simulate correct work with chunking",
fields: fields{
- filter: filter{
- match: matchFilter,
- timeStart: timeStartFilter,
- timeEnd: timeEndFilter,
- chunk: stepper.StepMonth,
+ filter: native.Filter{
+ Match: matchFilter,
+ TimeStart: timeStartFilter,
+ TimeEnd: timeEndFilter,
+ Chunk: stepper.StepMonth,
},
rateLimit: 0,
- dst: &vmNativeClient{
- addr: dstAddr,
+ dst: &native.Client{
+ Addr: dstAddr,
},
- src: &vmNativeClient{
- addr: srcAddr,
+ src: &native.Client{
+ Addr: srcAddr,
},
},
closer: func(cancelFunc context.CancelFunc) {},
@@ -110,7 +111,7 @@ func Test_vmNativeProcessor_run(t *testing.T) {
tt.closer(cancelFn)
- if err := p.run(ctx); (err != nil) != tt.wantErr {
+ if err := p.run(ctx, true); (err != nil) != tt.wantErr {
t.Errorf("run() error = %v, wantErr %v", err, tt.wantErr)
}
})
diff --git a/app/vmselect/vmui/asset-manifest.json b/app/vmselect/vmui/asset-manifest.json
index 3d3f322c5..6f5296b9d 100644
--- a/app/vmselect/vmui/asset-manifest.json
+++ b/app/vmselect/vmui/asset-manifest.json
@@ -1,7 +1,7 @@
{
"files": {
"main.css": "./static/css/main.5c28f4a7.css",
- "main.js": "./static/js/main.0be86920.js",
+ "main.js": "./static/js/main.6eed9ce1.js",
"static/js/27.c1ccfd29.chunk.js": "./static/js/27.c1ccfd29.chunk.js",
"static/media/Lato-Regular.ttf": "./static/media/Lato-Regular.d714fec1633b69a9c2e9.ttf",
"static/media/Lato-Bold.ttf": "./static/media/Lato-Bold.32360ba4b57802daa4d6.ttf",
@@ -9,6 +9,6 @@
},
"entrypoints": [
"static/css/main.5c28f4a7.css",
- "static/js/main.0be86920.js"
+ "static/js/main.6eed9ce1.js"
]
}
\ No newline at end of file
diff --git a/app/vmselect/vmui/index.html b/app/vmselect/vmui/index.html
index f224f22d1..e3a2dc0c0 100644
--- a/app/vmselect/vmui/index.html
+++ b/app/vmselect/vmui/index.html
@@ -1 +1 @@
-VM UI
\ No newline at end of file
+VM UI
\ No newline at end of file
diff --git a/app/vmselect/vmui/static/js/main.0be86920.js b/app/vmselect/vmui/static/js/main.6eed9ce1.js
similarity index 52%
rename from app/vmselect/vmui/static/js/main.0be86920.js
rename to app/vmselect/vmui/static/js/main.6eed9ce1.js
index ad16c1cd0..1837e12da 100644
--- a/app/vmselect/vmui/static/js/main.0be86920.js
+++ b/app/vmselect/vmui/static/js/main.6eed9ce1.js
@@ -1,2 +1,2 @@
-/*! For license information please see main.0be86920.js.LICENSE.txt */
-!function(){var e={680:function(e,t,n){"use strict";var r=n(476),i=n(962),o=i(r("String.prototype.indexOf"));e.exports=function(e,t){var n=r(e,!!t);return"function"===typeof n&&o(e,".prototype.")>-1?i(n):n}},962:function(e,t,n){"use strict";var r=n(199),i=n(476),o=i("%Function.prototype.apply%"),a=i("%Function.prototype.call%"),u=i("%Reflect.apply%",!0)||r.call(a,o),l=i("%Object.getOwnPropertyDescriptor%",!0),c=i("%Object.defineProperty%",!0),s=i("%Math.max%");if(c)try{c({},"a",{value:1})}catch(d){c=null}e.exports=function(e){var t=u(r,a,arguments);if(l&&c){var n=l(t,"length");n.configurable&&c(t,"length",{value:1+s(0,e.length-(arguments.length-1))})}return t};var f=function(){return u(r,o,arguments)};c?c(e.exports,"apply",{value:f}):e.exports.apply=f},123:function(e,t){var n;!function(){"use strict";var r={}.hasOwnProperty;function i(){for(var e=[],t=0;t=t?e:""+Array(t+1-r.length).join(n)+e},y={s:g,z:function(e){var t=-e.utcOffset(),n=Math.abs(t),r=Math.floor(n/60),i=n%60;return(t<=0?"+":"-")+g(r,2,"0")+":"+g(i,2,"0")},m:function e(t,n){if(t.date()1)return e(a[0])}else{var u=t.name;b[u]=t,i=u}return!r&&i&&(_=i),i||!r&&_},x=function(e,t){if(D(e))return e.clone();var n="object"==typeof t?t:{};return n.date=e,n.args=arguments,new C(n)},k=y;k.l=w,k.i=D,k.w=function(e,t){return x(e,{locale:t.$L,utc:t.$u,x:t.$x,$offset:t.$offset})};var C=function(){function m(e){this.$L=w(e.locale,null,!0),this.parse(e)}var g=m.prototype;return g.parse=function(e){this.$d=function(e){var t=e.date,n=e.utc;if(null===t)return new Date(NaN);if(k.u(t))return new Date;if(t instanceof Date)return new Date(t);if("string"==typeof t&&!/Z$/i.test(t)){var r=t.match(p);if(r){var i=r[2]-1||0,o=(r[7]||"0").substring(0,3);return n?new Date(Date.UTC(r[1],i,r[3]||1,r[4]||0,r[5]||0,r[6]||0,o)):new Date(r[1],i,r[3]||1,r[4]||0,r[5]||0,r[6]||0,o)}}return new Date(t)}(e),this.$x=e.x||{},this.init()},g.init=function(){var e=this.$d;this.$y=e.getFullYear(),this.$M=e.getMonth(),this.$D=e.getDate(),this.$W=e.getDay(),this.$H=e.getHours(),this.$m=e.getMinutes(),this.$s=e.getSeconds(),this.$ms=e.getMilliseconds()},g.$utils=function(){return k},g.isValid=function(){return!(this.$d.toString()===h)},g.isSame=function(e,t){var n=x(e);return this.startOf(t)<=n&&n<=this.endOf(t)},g.isAfter=function(e,t){return x(e)=0&&(o[f]=parseInt(s,10))}var d=o[3],h=24===d?0:d,p=o[0]+"-"+o[1]+"-"+o[2]+" "+h+":"+o[4]+":"+o[5]+":000",v=+t;return(i.utc(p).valueOf()-(v-=v%1e3))/6e4},l=r.prototype;l.tz=function(e,t){void 0===e&&(e=o);var n=this.utcOffset(),r=this.toDate(),a=r.toLocaleString("en-US",{timeZone:e}),u=Math.round((r-new Date(a))/1e3/60),l=i(a).$set("millisecond",this.$ms).utcOffset(15*-Math.round(r.getTimezoneOffset()/15)-u,!0);if(t){var c=l.utcOffset();l=l.add(n-c,"minute")}return l.$x.$timezone=e,l},l.offsetName=function(e){var t=this.$x.$timezone||i.tz.guess(),n=a(this.valueOf(),t,{timeZoneName:e}).find((function(e){return"timezonename"===e.type.toLowerCase()}));return n&&n.value};var c=l.startOf;l.startOf=function(e,t){if(!this.$x||!this.$x.$timezone)return c.call(this,e,t);var n=i(this.format("YYYY-MM-DD HH:mm:ss:SSS"));return c.call(n,e,t).tz(this.$x.$timezone,!0)},i.tz=function(e,t,n){var r=n&&t,a=n||t||o,l=u(+i(),a);if("string"!=typeof e)return i(e).tz(a);var c=function(e,t,n){var r=e-60*t*1e3,i=u(r,n);if(t===i)return[r,t];var o=u(r-=60*(i-t)*1e3,n);return i===o?[r,i]:[e-60*Math.min(i,o)*1e3,Math.max(i,o)]}(i.utc(e,r).valueOf(),l,a),s=c[0],f=c[1],d=i(s).utcOffset(f);return d.$x.$timezone=a,d},i.tz.guess=function(){return Intl.DateTimeFormat().resolvedOptions().timeZone},i.tz.setDefault=function(e){o=e}}}()},635:function(e){e.exports=function(){"use strict";var e="minute",t=/[+-]\d\d(?::?\d\d)?/g,n=/([+-]|\d\d)/g;return function(r,i,o){var a=i.prototype;o.utc=function(e){return new i({date:e,utc:!0,args:arguments})},a.utc=function(t){var n=o(this.toDate(),{locale:this.$L,utc:!0});return t?n.add(this.utcOffset(),e):n},a.local=function(){return o(this.toDate(),{locale:this.$L,utc:!1})};var u=a.parse;a.parse=function(e){e.utc&&(this.$u=!0),this.$utils().u(e.$offset)||(this.$offset=e.$offset),u.call(this,e)};var l=a.init;a.init=function(){if(this.$u){var e=this.$d;this.$y=e.getUTCFullYear(),this.$M=e.getUTCMonth(),this.$D=e.getUTCDate(),this.$W=e.getUTCDay(),this.$H=e.getUTCHours(),this.$m=e.getUTCMinutes(),this.$s=e.getUTCSeconds(),this.$ms=e.getUTCMilliseconds()}else l.call(this)};var c=a.utcOffset;a.utcOffset=function(r,i){var o=this.$utils().u;if(o(r))return this.$u?0:o(this.$offset)?c.call(this):this.$offset;if("string"==typeof r&&(r=function(e){void 0===e&&(e="");var r=e.match(t);if(!r)return null;var i=(""+r[0]).match(n)||["-",0,0],o=i[0],a=60*+i[1]+ +i[2];return 0===a?0:"+"===o?a:-a}(r),null===r))return this;var a=Math.abs(r)<=16?60*r:r,u=this;if(i)return u.$offset=a,u.$u=0===r,u;if(0!==r){var l=this.$u?this.toDate().getTimezoneOffset():-1*this.utcOffset();(u=this.local().add(a+l,e)).$offset=a,u.$x.$localOffset=l}else u=this.utc();return u};var s=a.format;a.format=function(e){var t=e||(this.$u?"YYYY-MM-DDTHH:mm:ss[Z]":"");return s.call(this,t)},a.valueOf=function(){var e=this.$utils().u(this.$offset)?0:this.$offset+(this.$x.$localOffset||this.$d.getTimezoneOffset());return this.$d.valueOf()-6e4*e},a.isUTC=function(){return!!this.$u},a.toISOString=function(){return this.toDate().toISOString()},a.toString=function(){return this.toDate().toUTCString()};var f=a.toDate;a.toDate=function(e){return"s"===e&&this.$offset?o(this.format("YYYY-MM-DD HH:mm:ss:SSS")).toDate():f.call(this)};var d=a.diff;a.diff=function(e,t,n){if(e&&this.$u===e.$u)return d.call(this,e,t,n);var r=this.local(),i=o(e).local();return d.call(r,i,t,n)}}}()},781:function(e){"use strict";var t="Function.prototype.bind called on incompatible ",n=Array.prototype.slice,r=Object.prototype.toString,i="[object Function]";e.exports=function(e){var o=this;if("function"!==typeof o||r.call(o)!==i)throw new TypeError(t+o);for(var a,u=n.call(arguments,1),l=function(){if(this instanceof a){var t=o.apply(this,u.concat(n.call(arguments)));return Object(t)===t?t:this}return o.apply(e,u.concat(n.call(arguments)))},c=Math.max(0,o.length-u.length),s=[],f=0;f1&&"boolean"!==typeof t)throw new a('"allowMissing" argument must be a boolean');if(null===k(/^%?[^%]*%?$/,e))throw new i("`%` may not be present anywhere but at the beginning and end of the intrinsic name");var n=E(e),r=n.length>0?n[0]:"",o=S("%"+r+"%",t),u=o.name,c=o.value,s=!1,f=o.alias;f&&(r=f[0],D(n,b([0,1],f)));for(var d=1,h=!0;d=n.length){var y=l(c,p);c=(h=!!y)&&"get"in y&&!("originalValue"in y.get)?y.get:c[p]}else h=_(c,p),c=c[p];h&&!s&&(v[u]=c)}}return c}},520:function(e,t,n){"use strict";var r="undefined"!==typeof Symbol&&Symbol,i=n(541);e.exports=function(){return"function"===typeof r&&("function"===typeof Symbol&&("symbol"===typeof r("foo")&&("symbol"===typeof Symbol("bar")&&i())))}},541:function(e){"use strict";e.exports=function(){if("function"!==typeof Symbol||"function"!==typeof Object.getOwnPropertySymbols)return!1;if("symbol"===typeof Symbol.iterator)return!0;var e={},t=Symbol("test"),n=Object(t);if("string"===typeof t)return!1;if("[object Symbol]"!==Object.prototype.toString.call(t))return!1;if("[object Symbol]"!==Object.prototype.toString.call(n))return!1;for(t in e[t]=42,e)return!1;if("function"===typeof Object.keys&&0!==Object.keys(e).length)return!1;if("function"===typeof Object.getOwnPropertyNames&&0!==Object.getOwnPropertyNames(e).length)return!1;var r=Object.getOwnPropertySymbols(e);if(1!==r.length||r[0]!==t)return!1;if(!Object.prototype.propertyIsEnumerable.call(e,t))return!1;if("function"===typeof Object.getOwnPropertyDescriptor){var i=Object.getOwnPropertyDescriptor(e,t);if(42!==i.value||!0!==i.enumerable)return!1}return!0}},838:function(e,t,n){"use strict";var r=n(199);e.exports=r.call(Function.call,Object.prototype.hasOwnProperty)},936:function(e,t,n){var r=/^\s+|\s+$/g,i=/^[-+]0x[0-9a-f]+$/i,o=/^0b[01]+$/i,a=/^0o[0-7]+$/i,u=parseInt,l="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,c="object"==typeof self&&self&&self.Object===Object&&self,s=l||c||Function("return this")(),f=Object.prototype.toString,d=Math.max,h=Math.min,p=function(){return s.Date.now()};function v(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function m(e){if("number"==typeof e)return e;if(function(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==f.call(e)}(e))return NaN;if(v(e)){var t="function"==typeof e.valueOf?e.valueOf():e;e=v(t)?t+"":t}if("string"!=typeof e)return 0===e?e:+e;e=e.replace(r,"");var n=o.test(e);return n||a.test(e)?u(e.slice(2),n?2:8):i.test(e)?NaN:+e}e.exports=function(e,t,n){var r,i,o,a,u,l,c=0,s=!1,f=!1,g=!0;if("function"!=typeof e)throw new TypeError("Expected a function");function y(t){var n=r,o=i;return r=i=void 0,c=t,a=e.apply(o,n)}function _(e){return c=e,u=setTimeout(D,t),s?y(e):a}function b(e){var n=e-l;return void 0===l||n>=t||n<0||f&&e-c>=o}function D(){var e=p();if(b(e))return w(e);u=setTimeout(D,function(e){var n=t-(e-l);return f?h(n,o-(e-c)):n}(e))}function w(e){return u=void 0,g&&r?y(e):(r=i=void 0,a)}function x(){var e=p(),n=b(e);if(r=arguments,i=this,l=e,n){if(void 0===u)return _(l);if(f)return u=setTimeout(D,t),y(l)}return void 0===u&&(u=setTimeout(D,t)),a}return t=m(t)||0,v(n)&&(s=!!n.leading,o=(f="maxWait"in n)?d(m(n.maxWait)||0,t):o,g="trailing"in n?!!n.trailing:g),x.cancel=function(){void 0!==u&&clearTimeout(u),c=0,r=l=i=u=void 0},x.flush=function(){return void 0===u?a:w(p())},x}},7:function(e,t,n){var r="__lodash_hash_undefined__",i="[object Function]",o="[object GeneratorFunction]",a=/\.|\[(?:[^[\]]*|(["'])(?:(?!\1)[^\\]|\\.)*?\1)\]/,u=/^\w*$/,l=/^\./,c=/[^.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|$))/g,s=/\\(\\)?/g,f=/^\[object .+?Constructor\]$/,d="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,h="object"==typeof self&&self&&self.Object===Object&&self,p=d||h||Function("return this")();var v=Array.prototype,m=Function.prototype,g=Object.prototype,y=p["__core-js_shared__"],_=function(){var e=/[^.]+$/.exec(y&&y.keys&&y.keys.IE_PROTO||"");return e?"Symbol(src)_1."+e:""}(),b=m.toString,D=g.hasOwnProperty,w=g.toString,x=RegExp("^"+b.call(D).replace(/[\\^$.*+?()[\]{}|]/g,"\\$&").replace(/hasOwnProperty|(function).*?(?=\\\()| for .+?(?=\\\])/g,"$1.*?")+"$"),k=p.Symbol,C=v.splice,A=P(p,"Map"),E=P(Object,"create"),S=k?k.prototype:void 0,N=S?S.toString:void 0;function F(e){var t=-1,n=e?e.length:0;for(this.clear();++t-1},M.prototype.set=function(e,t){var n=this.__data__,r=O(n,e);return r<0?n.push([e,t]):n[r][1]=t,this},T.prototype.clear=function(){this.__data__={hash:new F,map:new(A||M),string:new F}},T.prototype.delete=function(e){return I(this,e).delete(e)},T.prototype.get=function(e){return I(this,e).get(e)},T.prototype.has=function(e){return I(this,e).has(e)},T.prototype.set=function(e,t){return I(this,e).set(e,t),this};var R=j((function(e){var t;e=null==(t=e)?"":function(e){if("string"==typeof e)return e;if(U(e))return N?N.call(e):"";var t=e+"";return"0"==t&&1/e==-1/0?"-0":t}(t);var n=[];return l.test(e)&&n.push(""),e.replace(c,(function(e,t,r,i){n.push(r?i.replace(s,"$1"):t||e)})),n}));function z(e){if("string"==typeof e||U(e))return e;var t=e+"";return"0"==t&&1/e==-1/0?"-0":t}function j(e,t){if("function"!=typeof e||t&&"function"!=typeof t)throw new TypeError("Expected a function");var n=function n(){var r=arguments,i=t?t.apply(this,r):r[0],o=n.cache;if(o.has(i))return o.get(i);var a=e.apply(this,r);return n.cache=o.set(i,a),a};return n.cache=new(j.Cache||T),n}j.Cache=T;var $=Array.isArray;function H(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function U(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==w.call(e)}e.exports=function(e,t,n){var r=null==e?void 0:B(e,t);return void 0===r?n:r}},61:function(e,t,n){var r="Expected a function",i=/^\s+|\s+$/g,o=/^[-+]0x[0-9a-f]+$/i,a=/^0b[01]+$/i,u=/^0o[0-7]+$/i,l=parseInt,c="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,s="object"==typeof self&&self&&self.Object===Object&&self,f=c||s||Function("return this")(),d=Object.prototype.toString,h=Math.max,p=Math.min,v=function(){return f.Date.now()};function m(e,t,n){var i,o,a,u,l,c,s=0,f=!1,d=!1,m=!0;if("function"!=typeof e)throw new TypeError(r);function _(t){var n=i,r=o;return i=o=void 0,s=t,u=e.apply(r,n)}function b(e){return s=e,l=setTimeout(w,t),f?_(e):u}function D(e){var n=e-c;return void 0===c||n>=t||n<0||d&&e-s>=a}function w(){var e=v();if(D(e))return x(e);l=setTimeout(w,function(e){var n=t-(e-c);return d?p(n,a-(e-s)):n}(e))}function x(e){return l=void 0,m&&i?_(e):(i=o=void 0,u)}function k(){var e=v(),n=D(e);if(i=arguments,o=this,c=e,n){if(void 0===l)return b(c);if(d)return l=setTimeout(w,t),_(c)}return void 0===l&&(l=setTimeout(w,t)),u}return t=y(t)||0,g(n)&&(f=!!n.leading,a=(d="maxWait"in n)?h(y(n.maxWait)||0,t):a,m="trailing"in n?!!n.trailing:m),k.cancel=function(){void 0!==l&&clearTimeout(l),s=0,i=c=o=l=void 0},k.flush=function(){return void 0===l?u:x(v())},k}function g(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function y(e){if("number"==typeof e)return e;if(function(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==d.call(e)}(e))return NaN;if(g(e)){var t="function"==typeof e.valueOf?e.valueOf():e;e=g(t)?t+"":t}if("string"!=typeof e)return 0===e?e:+e;e=e.replace(i,"");var n=a.test(e);return n||u.test(e)?l(e.slice(2),n?2:8):o.test(e)?NaN:+e}e.exports=function(e,t,n){var i=!0,o=!0;if("function"!=typeof e)throw new TypeError(r);return g(n)&&(i="leading"in n?!!n.leading:i,o="trailing"in n?!!n.trailing:o),m(e,t,{leading:i,maxWait:t,trailing:o})}},154:function(e,t,n){var r="function"===typeof Map&&Map.prototype,i=Object.getOwnPropertyDescriptor&&r?Object.getOwnPropertyDescriptor(Map.prototype,"size"):null,o=r&&i&&"function"===typeof i.get?i.get:null,a=r&&Map.prototype.forEach,u="function"===typeof Set&&Set.prototype,l=Object.getOwnPropertyDescriptor&&u?Object.getOwnPropertyDescriptor(Set.prototype,"size"):null,c=u&&l&&"function"===typeof l.get?l.get:null,s=u&&Set.prototype.forEach,f="function"===typeof WeakMap&&WeakMap.prototype?WeakMap.prototype.has:null,d="function"===typeof WeakSet&&WeakSet.prototype?WeakSet.prototype.has:null,h="function"===typeof WeakRef&&WeakRef.prototype?WeakRef.prototype.deref:null,p=Boolean.prototype.valueOf,v=Object.prototype.toString,m=Function.prototype.toString,g=String.prototype.match,y=String.prototype.slice,_=String.prototype.replace,b=String.prototype.toUpperCase,D=String.prototype.toLowerCase,w=RegExp.prototype.test,x=Array.prototype.concat,k=Array.prototype.join,C=Array.prototype.slice,A=Math.floor,E="function"===typeof BigInt?BigInt.prototype.valueOf:null,S=Object.getOwnPropertySymbols,N="function"===typeof Symbol&&"symbol"===typeof Symbol.iterator?Symbol.prototype.toString:null,F="function"===typeof Symbol&&"object"===typeof Symbol.iterator,M="function"===typeof Symbol&&Symbol.toStringTag&&(typeof Symbol.toStringTag===F||"symbol")?Symbol.toStringTag:null,T=Object.prototype.propertyIsEnumerable,O=("function"===typeof Reflect?Reflect.getPrototypeOf:Object.getPrototypeOf)||([].__proto__===Array.prototype?function(e){return e.__proto__}:null);function B(e,t){if(e===1/0||e===-1/0||e!==e||e&&e>-1e3&&e<1e3||w.call(/e/,t))return t;var n=/[0-9](?=(?:[0-9]{3})+(?![0-9]))/g;if("number"===typeof e){var r=e<0?-A(-e):A(e);if(r!==e){var i=String(r),o=y.call(t,i.length+1);return _.call(i,n,"$&_")+"."+_.call(_.call(o,/([0-9]{3})/g,"$&_"),/_$/,"")}}return _.call(t,n,"$&_")}var L=n(654),I=L.custom,P=H(I)?I:null;function R(e,t,n){var r="double"===(n.quoteStyle||t)?'"':"'";return r+e+r}function z(e){return _.call(String(e),/"/g,""")}function j(e){return"[object Array]"===V(e)&&(!M||!("object"===typeof e&&M in e))}function $(e){return"[object RegExp]"===V(e)&&(!M||!("object"===typeof e&&M in e))}function H(e){if(F)return e&&"object"===typeof e&&e instanceof Symbol;if("symbol"===typeof e)return!0;if(!e||"object"!==typeof e||!N)return!1;try{return N.call(e),!0}catch(t){}return!1}e.exports=function e(t,n,r,i){var u=n||{};if(Y(u,"quoteStyle")&&"single"!==u.quoteStyle&&"double"!==u.quoteStyle)throw new TypeError('option "quoteStyle" must be "single" or "double"');if(Y(u,"maxStringLength")&&("number"===typeof u.maxStringLength?u.maxStringLength<0&&u.maxStringLength!==1/0:null!==u.maxStringLength))throw new TypeError('option "maxStringLength", if provided, must be a positive integer, Infinity, or `null`');var l=!Y(u,"customInspect")||u.customInspect;if("boolean"!==typeof l&&"symbol"!==l)throw new TypeError("option \"customInspect\", if provided, must be `true`, `false`, or `'symbol'`");if(Y(u,"indent")&&null!==u.indent&&"\t"!==u.indent&&!(parseInt(u.indent,10)===u.indent&&u.indent>0))throw new TypeError('option "indent" must be "\\t", an integer > 0, or `null`');if(Y(u,"numericSeparator")&&"boolean"!==typeof u.numericSeparator)throw new TypeError('option "numericSeparator", if provided, must be `true` or `false`');var v=u.numericSeparator;if("undefined"===typeof t)return"undefined";if(null===t)return"null";if("boolean"===typeof t)return t?"true":"false";if("string"===typeof t)return W(t,u);if("number"===typeof t){if(0===t)return 1/0/t>0?"0":"-0";var b=String(t);return v?B(t,b):b}if("bigint"===typeof t){var w=String(t)+"n";return v?B(t,w):w}var A="undefined"===typeof u.depth?5:u.depth;if("undefined"===typeof r&&(r=0),r>=A&&A>0&&"object"===typeof t)return j(t)?"[Array]":"[Object]";var S=function(e,t){var n;if("\t"===e.indent)n="\t";else{if(!("number"===typeof e.indent&&e.indent>0))return null;n=k.call(Array(e.indent+1)," ")}return{base:n,prev:k.call(Array(t+1),n)}}(u,r);if("undefined"===typeof i)i=[];else if(q(i,t)>=0)return"[Circular]";function I(t,n,o){if(n&&(i=C.call(i)).push(n),o){var a={depth:u.depth};return Y(u,"quoteStyle")&&(a.quoteStyle=u.quoteStyle),e(t,a,r+1,i)}return e(t,u,r+1,i)}if("function"===typeof t&&!$(t)){var U=function(e){if(e.name)return e.name;var t=g.call(m.call(e),/^function\s*([\w$]+)/);if(t)return t[1];return null}(t),Q=X(t,I);return"[Function"+(U?": "+U:" (anonymous)")+"]"+(Q.length>0?" { "+k.call(Q,", ")+" }":"")}if(H(t)){var ee=F?_.call(String(t),/^(Symbol\(.*\))_[^)]*$/,"$1"):N.call(t);return"object"!==typeof t||F?ee:G(ee)}if(function(e){if(!e||"object"!==typeof e)return!1;if("undefined"!==typeof HTMLElement&&e instanceof HTMLElement)return!0;return"string"===typeof e.nodeName&&"function"===typeof e.getAttribute}(t)){for(var te="<"+D.call(String(t.nodeName)),ne=t.attributes||[],re=0;re",t.childNodes&&t.childNodes.length&&(te+="..."),te+=""+D.call(String(t.nodeName))+">"}if(j(t)){if(0===t.length)return"[]";var ie=X(t,I);return S&&!function(e){for(var t=0;t=0)return!1;return!0}(ie)?"["+K(ie,S)+"]":"[ "+k.call(ie,", ")+" ]"}if(function(e){return"[object Error]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t)){var oe=X(t,I);return"cause"in Error.prototype||!("cause"in t)||T.call(t,"cause")?0===oe.length?"["+String(t)+"]":"{ ["+String(t)+"] "+k.call(oe,", ")+" }":"{ ["+String(t)+"] "+k.call(x.call("[cause]: "+I(t.cause),oe),", ")+" }"}if("object"===typeof t&&l){if(P&&"function"===typeof t[P]&&L)return L(t,{depth:A-r});if("symbol"!==l&&"function"===typeof t.inspect)return t.inspect()}if(function(e){if(!o||!e||"object"!==typeof e)return!1;try{o.call(e);try{c.call(e)}catch(te){return!0}return e instanceof Map}catch(t){}return!1}(t)){var ae=[];return a&&a.call(t,(function(e,n){ae.push(I(n,t,!0)+" => "+I(e,t))})),Z("Map",o.call(t),ae,S)}if(function(e){if(!c||!e||"object"!==typeof e)return!1;try{c.call(e);try{o.call(e)}catch(t){return!0}return e instanceof Set}catch(n){}return!1}(t)){var ue=[];return s&&s.call(t,(function(e){ue.push(I(e,t))})),Z("Set",c.call(t),ue,S)}if(function(e){if(!f||!e||"object"!==typeof e)return!1;try{f.call(e,f);try{d.call(e,d)}catch(te){return!0}return e instanceof WeakMap}catch(t){}return!1}(t))return J("WeakMap");if(function(e){if(!d||!e||"object"!==typeof e)return!1;try{d.call(e,d);try{f.call(e,f)}catch(te){return!0}return e instanceof WeakSet}catch(t){}return!1}(t))return J("WeakSet");if(function(e){if(!h||!e||"object"!==typeof e)return!1;try{return h.call(e),!0}catch(t){}return!1}(t))return J("WeakRef");if(function(e){return"[object Number]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(I(Number(t)));if(function(e){if(!e||"object"!==typeof e||!E)return!1;try{return E.call(e),!0}catch(t){}return!1}(t))return G(I(E.call(t)));if(function(e){return"[object Boolean]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(p.call(t));if(function(e){return"[object String]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(I(String(t)));if(!function(e){return"[object Date]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t)&&!$(t)){var le=X(t,I),ce=O?O(t)===Object.prototype:t instanceof Object||t.constructor===Object,se=t instanceof Object?"":"null prototype",fe=!ce&&M&&Object(t)===t&&M in t?y.call(V(t),8,-1):se?"Object":"",de=(ce||"function"!==typeof t.constructor?"":t.constructor.name?t.constructor.name+" ":"")+(fe||se?"["+k.call(x.call([],fe||[],se||[]),": ")+"] ":"");return 0===le.length?de+"{}":S?de+"{"+K(le,S)+"}":de+"{ "+k.call(le,", ")+" }"}return String(t)};var U=Object.prototype.hasOwnProperty||function(e){return e in this};function Y(e,t){return U.call(e,t)}function V(e){return v.call(e)}function q(e,t){if(e.indexOf)return e.indexOf(t);for(var n=0,r=e.length;nt.maxStringLength){var n=e.length-t.maxStringLength,r="... "+n+" more character"+(n>1?"s":"");return W(y.call(e,0,t.maxStringLength),t)+r}return R(_.call(_.call(e,/(['\\])/g,"\\$1"),/[\x00-\x1f]/g,Q),"single",t)}function Q(e){var t=e.charCodeAt(0),n={8:"b",9:"t",10:"n",12:"f",13:"r"}[t];return n?"\\"+n:"\\x"+(t<16?"0":"")+b.call(t.toString(16))}function G(e){return"Object("+e+")"}function J(e){return e+" { ? }"}function Z(e,t,n,r){return e+" ("+t+") {"+(r?K(n,r):k.call(n,", "))+"}"}function K(e,t){if(0===e.length)return"";var n="\n"+t.prev+t.base;return n+k.call(e,","+n)+"\n"+t.prev}function X(e,t){var n=j(e),r=[];if(n){r.length=e.length;for(var i=0;i-1?e.split(","):e},c=function(e,t,n,r){if(e){var o=n.allowDots?e.replace(/\.([^.[]+)/g,"[$1]"):e,a=/(\[[^[\]]*])/g,u=n.depth>0&&/(\[[^[\]]*])/.exec(o),c=u?o.slice(0,u.index):o,s=[];if(c){if(!n.plainObjects&&i.call(Object.prototype,c)&&!n.allowPrototypes)return;s.push(c)}for(var f=0;n.depth>0&&null!==(u=a.exec(o))&&f=0;--o){var a,u=e[o];if("[]"===u&&n.parseArrays)a=[].concat(i);else{a=n.plainObjects?Object.create(null):{};var c="["===u.charAt(0)&&"]"===u.charAt(u.length-1)?u.slice(1,-1):u,s=parseInt(c,10);n.parseArrays||""!==c?!isNaN(s)&&u!==c&&String(s)===c&&s>=0&&n.parseArrays&&s<=n.arrayLimit?(a=[])[s]=i:"__proto__"!==c&&(a[c]=i):a={0:i}}i=a}return i}(s,t,n,r)}};e.exports=function(e,t){var n=function(e){if(!e)return a;if(null!==e.decoder&&void 0!==e.decoder&&"function"!==typeof e.decoder)throw new TypeError("Decoder has to be a function.");if("undefined"!==typeof e.charset&&"utf-8"!==e.charset&&"iso-8859-1"!==e.charset)throw new TypeError("The charset option must be either utf-8, iso-8859-1, or undefined");var t="undefined"===typeof e.charset?a.charset:e.charset;return{allowDots:"undefined"===typeof e.allowDots?a.allowDots:!!e.allowDots,allowPrototypes:"boolean"===typeof e.allowPrototypes?e.allowPrototypes:a.allowPrototypes,allowSparse:"boolean"===typeof e.allowSparse?e.allowSparse:a.allowSparse,arrayLimit:"number"===typeof e.arrayLimit?e.arrayLimit:a.arrayLimit,charset:t,charsetSentinel:"boolean"===typeof e.charsetSentinel?e.charsetSentinel:a.charsetSentinel,comma:"boolean"===typeof e.comma?e.comma:a.comma,decoder:"function"===typeof e.decoder?e.decoder:a.decoder,delimiter:"string"===typeof e.delimiter||r.isRegExp(e.delimiter)?e.delimiter:a.delimiter,depth:"number"===typeof e.depth||!1===e.depth?+e.depth:a.depth,ignoreQueryPrefix:!0===e.ignoreQueryPrefix,interpretNumericEntities:"boolean"===typeof e.interpretNumericEntities?e.interpretNumericEntities:a.interpretNumericEntities,parameterLimit:"number"===typeof e.parameterLimit?e.parameterLimit:a.parameterLimit,parseArrays:!1!==e.parseArrays,plainObjects:"boolean"===typeof e.plainObjects?e.plainObjects:a.plainObjects,strictNullHandling:"boolean"===typeof e.strictNullHandling?e.strictNullHandling:a.strictNullHandling}}(t);if(""===e||null===e||"undefined"===typeof e)return n.plainObjects?Object.create(null):{};for(var s="string"===typeof e?function(e,t){var n,c={},s=t.ignoreQueryPrefix?e.replace(/^\?/,""):e,f=t.parameterLimit===1/0?void 0:t.parameterLimit,d=s.split(t.delimiter,f),h=-1,p=t.charset;if(t.charsetSentinel)for(n=0;n-1&&(m=o(m)?[m]:m),i.call(c,v)?c[v]=r.combine(c[v],m):c[v]=m}return c}(e,n):e,f=n.plainObjects?Object.create(null):{},d=Object.keys(s),h=0;h0?C.join(",")||null:void 0}];else if(l(h))B=h;else{var I=Object.keys(C);B=m?I.sort(m):I}for(var P=a&&l(C)&&1===C.length?n+"[]":n,R=0;R0?D+b:""}},837:function(e,t,n){"use strict";var r=n(609),i=Object.prototype.hasOwnProperty,o=Array.isArray,a=function(){for(var e=[],t=0;t<256;++t)e.push("%"+((t<16?"0":"")+t.toString(16)).toUpperCase());return e}(),u=function(e,t){for(var n=t&&t.plainObjects?Object.create(null):{},r=0;r1;){var t=e.pop(),n=t.obj[t.prop];if(o(n)){for(var r=[],i=0;i=48&&s<=57||s>=65&&s<=90||s>=97&&s<=122||o===r.RFC1738&&(40===s||41===s)?l+=u.charAt(c):s<128?l+=a[s]:s<2048?l+=a[192|s>>6]+a[128|63&s]:s<55296||s>=57344?l+=a[224|s>>12]+a[128|s>>6&63]+a[128|63&s]:(c+=1,s=65536+((1023&s)<<10|1023&u.charCodeAt(c)),l+=a[240|s>>18]+a[128|s>>12&63]+a[128|s>>6&63]+a[128|63&s])}return l},isBuffer:function(e){return!(!e||"object"!==typeof e)&&!!(e.constructor&&e.constructor.isBuffer&&e.constructor.isBuffer(e))},isRegExp:function(e){return"[object RegExp]"===Object.prototype.toString.call(e)},maybeMap:function(e,t){if(o(e)){for(var n=[],r=0;r2&&(u.children=arguments.length>3?r.call(arguments,2):n),"function"==typeof e&&null!=e.defaultProps)for(a in e.defaultProps)void 0===u[a]&&(u[a]=e.defaultProps[a]);return v(e,u,i,o,null)}function v(e,t,n,r,a){var u={type:e,props:t,key:n,ref:r,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,__h:null,constructor:void 0,__v:null==a?++o:a};return null==a&&null!=i.vnode&&i.vnode(u),u}function m(){return{current:null}}function g(e){return e.children}function y(e,t){this.props=e,this.context=t}function _(e,t){if(null==t)return e.__?_(e.__,e.__.__k.indexOf(e)+1):null;for(var n;t0?v(m.type,m.props,m.key,m.ref?m.ref:null,m.__v):m)){if(m.__=n,m.__b=n.__b+1,null===(p=w[d])||p&&m.key==p.key&&m.type===p.type)w[d]=void 0;else for(h=0;h2&&(u.children=arguments.length>3?r.call(arguments,2):n),v(e.type,u,i||e.key,o||e.ref,null)}function j(e,t){var n={__c:t="__cC"+l++,__:e,Consumer:function(e,t){return e.children(t)},Provider:function(e){var n,r;return this.getChildContext||(n=[],(r={})[t]=this,this.getChildContext=function(){return r},this.shouldComponentUpdate=function(e){this.props.value!==e.value&&n.some(D)},this.sub=function(e){n.push(e);var t=e.componentWillUnmount;e.componentWillUnmount=function(){n.splice(n.indexOf(e),1),t&&t.call(e)}}),e.children}};return n.Provider.__=n.Consumer.contextType=n}r=s.slice,i={__e:function(e,t,n,r){for(var i,o,a;t=t.__;)if((i=t.__c)&&!i.__)try{if((o=i.constructor)&&null!=o.getDerivedStateFromError&&(i.setState(o.getDerivedStateFromError(e)),a=i.__d),null!=i.componentDidCatch&&(i.componentDidCatch(e,r||{}),a=i.__d),a)return i.__E=i}catch(t){e=t}throw e}},o=0,y.prototype.setState=function(e,t){var n;n=null!=this.__s&&this.__s!==this.state?this.__s:this.__s=d({},this.state),"function"==typeof e&&(e=e(d({},n),this.props)),e&&d(n,e),null!=e&&this.__v&&(t&&this._sb.push(t),D(this))},y.prototype.forceUpdate=function(e){this.__v&&(this.__e=!0,e&&this.__h.push(e),D(this))},y.prototype.render=g,a=[],w.__r=0,l=0;var $,H,U,Y,V=0,q=[],W=[],Q=i.__b,G=i.__r,J=i.diffed,Z=i.__c,K=i.unmount;function X(e,t){i.__h&&i.__h(H,e,V||t),V=0;var n=H.__H||(H.__H={__:[],__h:[]});return e>=n.__.length&&n.__.push({__V:W}),n.__[e]}function ee(e){return V=1,te(ye,e)}function te(e,t,n){var r=X($++,2);if(r.t=e,!r.__c&&(r.__=[n?n(t):ye(void 0,t),function(e){var t=r.__N?r.__N[0]:r.__[0],n=r.t(t,e);t!==n&&(r.__N=[n,r.__[1]],r.__c.setState({}))}],r.__c=H,!H.u)){H.u=!0;var i=H.shouldComponentUpdate;H.shouldComponentUpdate=function(e,t,n){if(!r.__c.__H)return!0;var o=r.__c.__H.__.filter((function(e){return e.__c}));if(o.every((function(e){return!e.__N})))return!i||i.call(this,e,t,n);var a=!1;return o.forEach((function(e){if(e.__N){var t=e.__[0];e.__=e.__N,e.__N=void 0,t!==e.__[0]&&(a=!0)}})),!(!a&&r.__c.props===e)&&(!i||i.call(this,e,t,n))}}return r.__N||r.__}function ne(e,t){var n=X($++,3);!i.__s&&ge(n.__H,t)&&(n.__=e,n.i=t,H.__H.__h.push(n))}function re(e,t){var n=X($++,4);!i.__s&&ge(n.__H,t)&&(n.__=e,n.i=t,H.__h.push(n))}function ie(e){return V=5,ae((function(){return{current:e}}),[])}function oe(e,t,n){V=6,re((function(){return"function"==typeof e?(e(t()),function(){return e(null)}):e?(e.current=t(),function(){return e.current=null}):void 0}),null==n?n:n.concat(e))}function ae(e,t){var n=X($++,7);return ge(n.__H,t)?(n.__V=e(),n.i=t,n.__h=e,n.__V):n.__}function ue(e,t){return V=8,ae((function(){return e}),t)}function le(e){var t=H.context[e.__c],n=X($++,9);return n.c=e,t?(null==n.__&&(n.__=!0,t.sub(H)),t.props.value):e.__}function ce(e,t){i.useDebugValue&&i.useDebugValue(t?t(e):e)}function se(e){var t=X($++,10),n=ee();return t.__=e,H.componentDidCatch||(H.componentDidCatch=function(e,r){t.__&&t.__(e,r),n[1](e)}),[n[0],function(){n[1](void 0)}]}function fe(){var e=X($++,11);if(!e.__){for(var t=H.__v;null!==t&&!t.__m&&null!==t.__;)t=t.__;var n=t.__m||(t.__m=[0,0]);e.__="P"+n[0]+"-"+n[1]++}return e.__}function de(){for(var e;e=q.shift();)if(e.__P&&e.__H)try{e.__H.__h.forEach(ve),e.__H.__h.forEach(me),e.__H.__h=[]}catch(l){e.__H.__h=[],i.__e(l,e.__v)}}i.__b=function(e){H=null,Q&&Q(e)},i.__r=function(e){G&&G(e),$=0;var t=(H=e.__c).__H;t&&(U===H?(t.__h=[],H.__h=[],t.__.forEach((function(e){e.__N&&(e.__=e.__N),e.__V=W,e.__N=e.i=void 0}))):(t.__h.forEach(ve),t.__h.forEach(me),t.__h=[])),U=H},i.diffed=function(e){J&&J(e);var t=e.__c;t&&t.__H&&(t.__H.__h.length&&(1!==q.push(t)&&Y===i.requestAnimationFrame||((Y=i.requestAnimationFrame)||pe)(de)),t.__H.__.forEach((function(e){e.i&&(e.__H=e.i),e.__V!==W&&(e.__=e.__V),e.i=void 0,e.__V=W}))),U=H=null},i.__c=function(e,t){t.some((function(e){try{e.__h.forEach(ve),e.__h=e.__h.filter((function(e){return!e.__||me(e)}))}catch(o){t.some((function(e){e.__h&&(e.__h=[])})),t=[],i.__e(o,e.__v)}})),Z&&Z(e,t)},i.unmount=function(e){K&&K(e);var t,n=e.__c;n&&n.__H&&(n.__H.__.forEach((function(e){try{ve(e)}catch(e){t=e}})),n.__H=void 0,t&&i.__e(t,n.__v))};var he="function"==typeof requestAnimationFrame;function pe(e){var t,n=function(){clearTimeout(r),he&&cancelAnimationFrame(t),setTimeout(e)},r=setTimeout(n,100);he&&(t=requestAnimationFrame(n))}function ve(e){var t=H,n=e.__c;"function"==typeof n&&(e.__c=void 0,n()),H=t}function me(e){var t=H;e.__c=e.__(),H=t}function ge(e,t){return!e||e.length!==t.length||t.some((function(t,n){return t!==e[n]}))}function ye(e,t){return"function"==typeof t?t(e):t}function _e(e,t){for(var n in t)e[n]=t[n];return e}function be(e,t){for(var n in e)if("__source"!==n&&!(n in t))return!0;for(var r in t)if("__source"!==r&&e[r]!==t[r])return!0;return!1}function De(e,t){return e===t&&(0!==e||1/e==1/t)||e!=e&&t!=t}function we(e){this.props=e}function xe(e,t){function n(e){var n=this.props.ref,r=n==e.ref;return!r&&n&&(n.call?n(null):n.current=null),t?!t(this.props,e)||!r:be(this.props,e)}function r(t){return this.shouldComponentUpdate=n,p(e,t)}return r.displayName="Memo("+(e.displayName||e.name)+")",r.prototype.isReactComponent=!0,r.__f=!0,r}(we.prototype=new y).isPureReactComponent=!0,we.prototype.shouldComponentUpdate=function(e,t){return be(this.props,e)||be(this.state,t)};var ke=i.__b;i.__b=function(e){e.type&&e.type.__f&&e.ref&&(e.props.ref=e.ref,e.ref=null),ke&&ke(e)};var Ce="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.forward_ref")||3911;function Ae(e){function t(t){var n=_e({},t);return delete n.ref,e(n,t.ref||null)}return t.$$typeof=Ce,t.render=t,t.prototype.isReactComponent=t.__f=!0,t.displayName="ForwardRef("+(e.displayName||e.name)+")",t}var Ee=function(e,t){return null==e?null:C(C(e).map(t))},Se={map:Ee,forEach:Ee,count:function(e){return e?C(e).length:0},only:function(e){var t=C(e);if(1!==t.length)throw"Children.only";return t[0]},toArray:C},Ne=i.__e;i.__e=function(e,t,n,r){if(e.then)for(var i,o=t;o=o.__;)if((i=o.__c)&&i.__c)return null==t.__e&&(t.__e=n.__e,t.__k=n.__k),i.__c(e,t);Ne(e,t,n,r)};var Fe=i.unmount;function Me(e,t,n){return e&&(e.__c&&e.__c.__H&&(e.__c.__H.__.forEach((function(e){"function"==typeof e.__c&&e.__c()})),e.__c.__H=null),null!=(e=_e({},e)).__c&&(e.__c.__P===n&&(e.__c.__P=t),e.__c=null),e.__k=e.__k&&e.__k.map((function(e){return Me(e,t,n)}))),e}function Te(e,t,n){return e&&(e.__v=null,e.__k=e.__k&&e.__k.map((function(e){return Te(e,t,n)})),e.__c&&e.__c.__P===t&&(e.__e&&n.insertBefore(e.__e,e.__d),e.__c.__e=!0,e.__c.__P=n)),e}function Oe(){this.__u=0,this.t=null,this.__b=null}function Be(e){var t=e.__.__c;return t&&t.__a&&t.__a(e)}function Le(e){var t,n,r;function i(i){if(t||(t=e()).then((function(e){n=e.default||e}),(function(e){r=e})),r)throw r;if(!n)throw t;return p(n,i)}return i.displayName="Lazy",i.__f=!0,i}function Ie(){this.u=null,this.o=null}i.unmount=function(e){var t=e.__c;t&&t.__R&&t.__R(),t&&!0===e.__h&&(e.type=null),Fe&&Fe(e)},(Oe.prototype=new y).__c=function(e,t){var n=t.__c,r=this;null==r.t&&(r.t=[]),r.t.push(n);var i=Be(r.__v),o=!1,a=function(){o||(o=!0,n.__R=null,i?i(u):u())};n.__R=a;var u=function(){if(!--r.__u){if(r.state.__a){var e=r.state.__a;r.__v.__k[0]=Te(e,e.__c.__P,e.__c.__O)}var t;for(r.setState({__a:r.__b=null});t=r.t.pop();)t.forceUpdate()}},l=!0===t.__h;r.__u++||l||r.setState({__a:r.__b=r.__v.__k[0]}),e.then(a,a)},Oe.prototype.componentWillUnmount=function(){this.t=[]},Oe.prototype.render=function(e,t){if(this.__b){if(this.__v.__k){var n=document.createElement("div"),r=this.__v.__k[0].__c;this.__v.__k[0]=Me(this.__b,n,r.__O=r.__P)}this.__b=null}var i=t.__a&&p(g,null,e.fallback);return i&&(i.__h=null),[p(g,null,t.__a?null:e.children),i]};var Pe=function(e,t,n){if(++n[1]===n[0]&&e.o.delete(t),e.props.revealOrder&&("t"!==e.props.revealOrder[0]||!e.o.size))for(n=e.u;n;){for(;n.length>3;)n.pop()();if(n[1]>>1,1),t.i.removeChild(e)}}),P(p(Re,{context:t.context},e.__v),t.l)):t.l&&t.componentWillUnmount()}function je(e,t){var n=p(ze,{__v:e,i:t});return n.containerInfo=t,n}(Ie.prototype=new y).__a=function(e){var t=this,n=Be(t.__v),r=t.o.get(e);return r[0]++,function(i){var o=function(){t.props.revealOrder?(r.push(i),Pe(t,e,r)):i()};n?n(o):o()}},Ie.prototype.render=function(e){this.u=null,this.o=new Map;var t=C(e.children);e.revealOrder&&"b"===e.revealOrder[0]&&t.reverse();for(var n=t.length;n--;)this.o.set(t[n],this.u=[1,0,this.u]);return e.children},Ie.prototype.componentDidUpdate=Ie.prototype.componentDidMount=function(){var e=this;this.o.forEach((function(t,n){Pe(e,n,t)}))};var $e="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.element")||60103,He=/^(?:accent|alignment|arabic|baseline|cap|clip(?!PathU)|color|dominant|fill|flood|font|glyph(?!R)|horiz|image|letter|lighting|marker(?!H|W|U)|overline|paint|pointer|shape|stop|strikethrough|stroke|text(?!L)|transform|underline|unicode|units|v|vector|vert|word|writing|x(?!C))[A-Z]/,Ue="undefined"!=typeof document,Ye=function(e){return("undefined"!=typeof Symbol&&"symbol"==typeof Symbol()?/fil|che|rad/i:/fil|che|ra/i).test(e)};function Ve(e,t,n){return null==t.__k&&(t.textContent=""),P(e,t),"function"==typeof n&&n(),e?e.__c:null}function qe(e,t,n){return R(e,t),"function"==typeof n&&n(),e?e.__c:null}y.prototype.isReactComponent={},["componentWillMount","componentWillReceiveProps","componentWillUpdate"].forEach((function(e){Object.defineProperty(y.prototype,e,{configurable:!0,get:function(){return this["UNSAFE_"+e]},set:function(t){Object.defineProperty(this,e,{configurable:!0,writable:!0,value:t})}})}));var We=i.event;function Qe(){}function Ge(){return this.cancelBubble}function Je(){return this.defaultPrevented}i.event=function(e){return We&&(e=We(e)),e.persist=Qe,e.isPropagationStopped=Ge,e.isDefaultPrevented=Je,e.nativeEvent=e};var Ze,Ke={configurable:!0,get:function(){return this.class}},Xe=i.vnode;i.vnode=function(e){var t=e.type,n=e.props,r=n;if("string"==typeof t){var i=-1===t.indexOf("-");for(var o in r={},n){var a=n[o];Ue&&"children"===o&&"noscript"===t||"value"===o&&"defaultValue"in n&&null==a||("defaultValue"===o&&"value"in n&&null==n.value?o="value":"download"===o&&!0===a?a="":/ondoubleclick/i.test(o)?o="ondblclick":/^onchange(textarea|input)/i.test(o+t)&&!Ye(n.type)?o="oninput":/^onfocus$/i.test(o)?o="onfocusin":/^onblur$/i.test(o)?o="onfocusout":/^on(Ani|Tra|Tou|BeforeInp|Compo)/.test(o)?o=o.toLowerCase():i&&He.test(o)?o=o.replace(/[A-Z0-9]/g,"-$&").toLowerCase():null===a&&(a=void 0),/^oninput$/i.test(o)&&(o=o.toLowerCase(),r[o]&&(o="oninputCapture")),r[o]=a)}"select"==t&&r.multiple&&Array.isArray(r.value)&&(r.value=C(n.children).forEach((function(e){e.props.selected=-1!=r.value.indexOf(e.props.value)}))),"select"==t&&null!=r.defaultValue&&(r.value=C(n.children).forEach((function(e){e.props.selected=r.multiple?-1!=r.defaultValue.indexOf(e.props.value):r.defaultValue==e.props.value}))),e.props=r,n.class!=n.className&&(Ke.enumerable="className"in n,null!=n.className&&(r.class=n.className),Object.defineProperty(r,"className",Ke))}e.$$typeof=$e,Xe&&Xe(e)};var et=i.__r;i.__r=function(e){et&&et(e),Ze=e.__c};var tt={ReactCurrentDispatcher:{current:{readContext:function(e){return Ze.__n[e.__c].props.value}}}},nt="17.0.2";function rt(e){return p.bind(null,e)}function it(e){return!!e&&e.$$typeof===$e}function ot(e){return it(e)?z.apply(null,arguments):e}function at(e){return!!e.__k&&(P(null,e),!0)}function ut(e){return e&&(e.base||1===e.nodeType&&e)||null}var lt=function(e,t){return e(t)},ct=function(e,t){return e(t)},st=g;function ft(e){e()}function dt(e){return e}function ht(){return[!1,ft]}var pt=re;function vt(e,t){var n=t(),r=ee({h:{__:n,v:t}}),i=r[0].h,o=r[1];return re((function(){i.__=n,i.v=t,De(i.__,t())||o({h:i})}),[e,n,t]),ne((function(){return De(i.__,i.v())||o({h:i}),e((function(){De(i.__,i.v())||o({h:i})}))}),[e]),n}var mt,gt={useState:ee,useId:fe,useReducer:te,useEffect:ne,useLayoutEffect:re,useInsertionEffect:pt,useTransition:ht,useDeferredValue:dt,useSyncExternalStore:vt,startTransition:ft,useRef:ie,useImperativeHandle:oe,useMemo:ae,useCallback:ue,useContext:le,useDebugValue:ce,version:"17.0.2",Children:Se,render:Ve,hydrate:qe,unmountComponentAtNode:at,createPortal:je,createElement:p,createContext:j,createFactory:rt,cloneElement:ot,createRef:m,Fragment:g,isValidElement:it,findDOMNode:ut,Component:y,PureComponent:we,memo:xe,forwardRef:Ae,flushSync:ct,unstable_batchedUpdates:lt,StrictMode:st,Suspense:Oe,SuspenseList:Ie,lazy:Le,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:tt},yt=n(658),_t=n.n(yt),bt=n(443),Dt=n.n(bt),wt=n(446),xt=n.n(wt),kt=n(635),Ct=n.n(kt);function At(e){if(Array.isArray(e))return e}function Et(e,t){(null==t||t>e.length)&&(t=e.length);for(var n=0,r=new Array(t);n=e.length?{done:!0}:{done:!1,value:e[r++]}},e:function(e){throw e},f:i}}throw new TypeError("Invalid attempt to iterate non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}var o,a=!0,u=!1;return{s:function(){n=n.call(e)},n:function(){var e=n.next();return a=e.done,e},e:function(e){u=!0,o=e},f:function(){try{a||null==n.return||n.return()}finally{if(u)throw o}}}}function Tt(e){if("undefined"!==typeof Symbol&&null!=e[Symbol.iterator]||null!=e["@@iterator"])return Array.from(e)}function Ot(e){return function(e){if(Array.isArray(e))return Et(e)}(e)||Tt(e)||St(e)||function(){throw new TypeError("Invalid attempt to spread non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}()}function Bt(e,t){if(!(e instanceof t))throw new TypeError("Cannot call a class as a function")}function Lt(e){return Lt="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(e){return typeof e}:function(e){return e&&"function"==typeof Symbol&&e.constructor===Symbol&&e!==Symbol.prototype?"symbol":typeof e},Lt(e)}function It(e){var t=function(e,t){if("object"!==Lt(e)||null===e)return e;var n=e[Symbol.toPrimitive];if(void 0!==n){var r=n.call(e,t||"default");if("object"!==Lt(r))return r;throw new TypeError("@@toPrimitive must return a primitive value.")}return("string"===t?String:Number)(e)}(e,"string");return"symbol"===Lt(t)?t:String(t)}function Pt(e,t){for(var n=0;n=0&&(t.hash=e.substr(n),e=e.substr(0,n));var r=e.indexOf("?");r>=0&&(t.search=e.substr(r),e=e.substr(0,r)),e&&(t.pathname=e)}return t}function nn(e,t,n,r){void 0===r&&(r={});var i=r,o=i.window,a=void 0===o?document.defaultView:o,u=i.v5Compat,l=void 0!==u&&u,c=a.history,s=mt.Pop,f=null,d=h();function h(){return(c.state||{idx:null}).idx}function p(){var e=mt.Pop,t=h();if(null!=t){var n=t-d;s=e,d=t,f&&f({action:s,location:m.location,delta:n})}else Zt(!1,"You are trying to block a POP navigation to a location that was not created by @remix-run/router. The block will fail silently in production, but in general you should do all navigation with the router (instead of using window.history.pushState directly) to avoid this situation.")}function v(e){var t="null"!==a.location.origin?a.location.origin:a.location.href,n="string"===typeof e?e:en(e);return Jt(t,"No window.location.(origin|href) available to create URL for href: "+n),new URL(n,t)}null==d&&(d=0,c.replaceState(Wt({},c.state,{idx:d}),""));var m={get action(){return s},get location(){return e(a,c)},listen:function(e){if(f)throw new Error("A history only accepts one active listener");return a.addEventListener(Gt,p),f=e,function(){a.removeEventListener(Gt,p),f=null}},createHref:function(e){return t(a,e)},createURL:v,encodeLocation:function(e){var t=v(e);return{pathname:t.pathname,search:t.search,hash:t.hash}},push:function(e,t){s=mt.Push;var r=Xt(m.location,e,t);n&&n(r,e);var i=Kt(r,d=h()+1),o=m.createHref(r);try{c.pushState(i,"",o)}catch(u){a.location.assign(o)}l&&f&&f({action:s,location:m.location,delta:1})},replace:function(e,t){s=mt.Replace;var r=Xt(m.location,e,t);n&&n(r,e);var i=Kt(r,d=h()),o=m.createHref(r);c.replaceState(i,"",o),l&&f&&f({action:s,location:m.location,delta:0})},go:function(e){return c.go(e)}};return m}function rn(e,t,n){void 0===n&&(n="/");var r=hn(("string"===typeof t?tn(t):t).pathname||"/",n);if(null==r)return null;var i=on(e);!function(e){e.sort((function(e,t){return e.score!==t.score?t.score-e.score:function(e,t){var n=e.length===t.length&&e.slice(0,-1).every((function(e,n){return e===t[n]}));return n?e[e.length-1]-t[t.length-1]:0}(e.routesMeta.map((function(e){return e.childrenIndex})),t.routesMeta.map((function(e){return e.childrenIndex})))}))}(i);for(var o=null,a=0;null==o&&a0&&(Jt(!0!==e.index,'Index routes must not have child routes. Please remove all child routes from route path "'+u+'".'),on(e.children,t,l,u)),(null!=e.path||e.index)&&t.push({path:u,score:cn(u,e.index),routesMeta:l})};return e.forEach((function(e,t){var n;if(""!==e.path&&null!=(n=e.path)&&n.includes("?")){var r,o=Mt(an(e.path));try{for(o.s();!(r=o.n()).done;){var a=r.value;i(e,t,a)}}catch(u){o.e(u)}finally{o.f()}}else i(e,t)})),t}function an(e){var t=e.split("/");if(0===t.length)return[];var n,r=At(n=t)||Tt(n)||St(n)||Nt(),i=r[0],o=r.slice(1),a=i.endsWith("?"),u=i.replace(/\?$/,"");if(0===o.length)return a?[u,""]:[u];var l=an(o.join("/")),c=[];return c.push.apply(c,Ot(l.map((function(e){return""===e?u:[u,e].join("/")})))),a&&c.push.apply(c,Ot(l)),c.map((function(t){return e.startsWith("/")&&""===t?"/":t}))}!function(e){e.data="data",e.deferred="deferred",e.redirect="redirect",e.error="error"}(Qt||(Qt={}));var un=/^:\w+$/,ln=function(e){return"*"===e};function cn(e,t){var n=e.split("/"),r=n.length;return n.some(ln)&&(r+=-2),t&&(r+=2),n.filter((function(e){return!ln(e)})).reduce((function(e,t){return e+(un.test(t)?3:""===t?1:10)}),r)}function sn(e,t){for(var n=e.routesMeta,r={},i="/",o=[],a=0;a and the router will parse it for you.'}function mn(e){return e.filter((function(e,t){return 0===t||e.route.path&&e.route.path.length>0}))}function gn(e,t,n,r){var i;void 0===r&&(r=!1),"string"===typeof e?i=tn(e):(Jt(!(i=Wt({},e)).pathname||!i.pathname.includes("?"),vn("?","pathname","search",i)),Jt(!i.pathname||!i.pathname.includes("#"),vn("#","pathname","hash",i)),Jt(!i.search||!i.search.includes("#"),vn("#","search","hash",i)));var o,a=""===e||""===i.pathname,u=a?"/":i.pathname;if(r||null==u)o=n;else{var l=t.length-1;if(u.startsWith("..")){for(var c=u.split("/");".."===c[0];)c.shift(),l-=1;i.pathname=c.join("/")}o=l>=0?t[l]:"/"}var s=function(e,t){void 0===t&&(t="/");var n="string"===typeof e?tn(e):e,r=n.pathname,i=n.search,o=void 0===i?"":i,a=n.hash,u=void 0===a?"":a,l=r?r.startsWith("/")?r:function(e,t){var n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach((function(e){".."===e?n.length>1&&n.pop():"."!==e&&n.push(e)})),n.length>1?n.join("/"):"/"}(r,t):t;return{pathname:l,search:bn(o),hash:Dn(u)}}(i,o),f=u&&"/"!==u&&u.endsWith("/"),d=(a||"."===u)&&n.endsWith("/");return s.pathname.endsWith("/")||!f&&!d||(s.pathname+="/"),s}var yn=function(e){return e.join("/").replace(/\/\/+/g,"/")},_n=function(e){return e.replace(/\/+$/,"").replace(/^\/*/,"/")},bn=function(e){return e&&"?"!==e?e.startsWith("?")?e:"?"+e:""},Dn=function(e){return e&&"#"!==e?e.startsWith("#")?e:"#"+e:""};Error;var wn=Rt((function e(t,n,r,i){Bt(this,e),void 0===i&&(i=!1),this.status=t,this.statusText=n||"",this.internal=i,r instanceof Error?(this.data=r.toString(),this.error=r):this.data=r}));function xn(e){return e instanceof wn}var kn=["post","put","patch","delete"],Cn=(new Set(kn),["get"].concat(kn));new Set(Cn),new Set([301,302,303,307,308]),new Set([307,308]),"undefined"!==typeof window&&"undefined"!==typeof window.document&&window.document.createElement;Symbol("deferred");function An(){return An=Object.assign?Object.assign.bind():function(e){for(var t=1;t")))}var Vn,qn,Wn=function(e){jt(n,e);var t=Yt(n);function n(e){var r;return Bt(this,n),(r=t.call(this,e)).state={location:e.location,error:e.error},r}return Rt(n,[{key:"componentDidCatch",value:function(e,t){console.error("React Router caught the following error during render",e,t)}},{key:"render",value:function(){return this.state.error?p(Pn.Provider,{value:this.props.routeContext},p(Rn.Provider,{value:this.state.error,children:this.props.component})):this.props.children}}],[{key:"getDerivedStateFromError",value:function(e){return{error:e}}},{key:"getDerivedStateFromProps",value:function(e,t){return t.location!==e.location?{error:e.error,location:e.location}:{error:e.error||t.error,location:t.location}}}]),n}(y);function Qn(e){var t=e.routeContext,n=e.match,r=e.children,i=le(On);return i&&i.static&&i.staticContext&&n.route.errorElement&&(i.staticContext._deepestRenderedBoundaryId=n.route.id),p(Pn.Provider,{value:t},r)}function Gn(e,t,n){if(void 0===t&&(t=[]),null==e){if(null==n||!n.errors)return null;e=n.matches}var r=e,i=null==n?void 0:n.errors;if(null!=i){var o=r.findIndex((function(e){return e.route.id&&(null==i?void 0:i[e.route.id])}));o>=0||Jt(!1),r=r.slice(0,Math.min(r.length,o+1))}return r.reduceRight((function(e,o,a){var u=o.route.id?null==i?void 0:i[o.route.id]:null,l=n?o.route.errorElement||p(Yn,null):null,c=t.concat(r.slice(0,a+1)),s=function(){return p(Qn,{match:o,routeContext:{outlet:e,matches:c}},u?l:void 0!==o.route.element?o.route.element:e)};return n&&(o.route.errorElement||0===a)?p(Wn,{location:n.location,component:l,error:u,children:s(),routeContext:{outlet:null,matches:c}}):s()}),null)}function Jn(e){var t=le(Bn);return t||Jt(!1),t}function Zn(e){var t=function(e){var t=le(Pn);return t||Jt(!1),t}(),n=t.matches[t.matches.length-1];return n.route.id||Jt(!1),n.route.id}!function(e){e.UseBlocker="useBlocker",e.UseRevalidator="useRevalidator"}(Vn||(Vn={})),function(e){e.UseLoaderData="useLoaderData",e.UseActionData="useActionData",e.UseRouteError="useRouteError",e.UseNavigation="useNavigation",e.UseRouteLoaderData="useRouteLoaderData",e.UseMatches="useMatches",e.UseRevalidator="useRevalidator"}(qn||(qn={}));var Kn;function Xn(e){return function(e){var t=le(Pn).outlet;return t?p(Hn.Provider,{value:e},t):t}(e.context)}function er(e){Jt(!1)}function tr(e){var t=e.basename,n=void 0===t?"/":t,r=e.children,i=void 0===r?null:r,o=e.location,a=e.navigationType,u=void 0===a?mt.Pop:a,l=e.navigator,c=e.static,s=void 0!==c&&c;zn()&&Jt(!1);var f=n.replace(/^\/*/,"/"),d=ae((function(){return{basename:f,navigator:l,static:s}}),[f,l,s]);"string"===typeof o&&(o=tn(o));var h=o,v=h.pathname,m=void 0===v?"/":v,g=h.search,y=void 0===g?"":g,_=h.hash,b=void 0===_?"":_,D=h.state,w=void 0===D?null:D,x=h.key,k=void 0===x?"default":x,C=ae((function(){var e=hn(m,f);return null==e?null:{pathname:e,search:y,hash:b,state:w,key:k}}),[f,m,y,b,w,k]);return null==C?null:p(Ln.Provider,{value:d},p(In.Provider,{children:i,value:{location:C,navigationType:u}}))}function nr(e){var t=e.children,n=e.location,r=le(On);return function(e,t){zn()||Jt(!1);var n,r=le(Ln).navigator,i=le(Bn),o=le(Pn).matches,a=o[o.length-1],u=a?a.params:{},l=(a&&a.pathname,a?a.pathnameBase:"/"),c=(a&&a.route,jn());if(t){var s,f="string"===typeof t?tn(t):t;"/"===l||(null==(s=f.pathname)?void 0:s.startsWith(l))||Jt(!1),n=f}else n=c;var d=n.pathname||"/",h=rn(e,{pathname:"/"===l?d:d.slice(l.length)||"/"}),v=Gn(h&&h.map((function(e){return Object.assign({},e,{params:Object.assign({},u,e.params),pathname:yn([l,r.encodeLocation?r.encodeLocation(e.pathname).pathname:e.pathname]),pathnameBase:"/"===e.pathnameBase?l:yn([l,r.encodeLocation?r.encodeLocation(e.pathnameBase).pathname:e.pathnameBase])})})),o,i||void 0);return t&&v?p(In.Provider,{value:{location:An({pathname:"/",search:"",hash:"",state:null,key:"default"},n),navigationType:mt.Pop}},v):v}(r&&!t?r.router.routes:rr(t),n)}!function(e){e[e.pending=0]="pending",e[e.success=1]="success",e[e.error=2]="error"}(Kn||(Kn={}));new Promise((function(){}));function rr(e,t){void 0===t&&(t=[]);var n=[];return Se.forEach(e,(function(e,r){if(it(e))if(e.type!==g){e.type!==er&&Jt(!1),e.props.index&&e.props.children&&Jt(!1);var i=[].concat(Ot(t),[r]),o={id:e.props.id||i.join("-"),caseSensitive:e.props.caseSensitive,element:e.props.element,index:e.props.index,path:e.props.path,loader:e.props.loader,action:e.props.action,errorElement:e.props.errorElement,hasErrorBoundary:null!=e.props.errorElement,shouldRevalidate:e.props.shouldRevalidate,handle:e.props.handle};e.props.children&&(o.children=rr(e.props.children,i)),n.push(o)}else n.push.apply(n,rr(e.props.children,t))})),n}function ir(){return ir=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0||(i[n]=e[n]);return i}function ar(e){return void 0===e&&(e=""),new URLSearchParams("string"===typeof e||Array.isArray(e)||e instanceof URLSearchParams?e:Object.keys(e).reduce((function(t,n){var r=e[n];return t.concat(Array.isArray(r)?r.map((function(e){return[n,e]})):[[n,r]])}),[]))}var ur=["onClick","relative","reloadDocument","replace","state","target","to","preventScrollReset"],lr=["aria-current","caseSensitive","className","end","style","to","children"];function cr(e){var t=e.basename,n=e.children,r=e.window,i=ie();null==i.current&&(i.current=function(e){return void 0===e&&(e={}),nn((function(e,t){var n=tn(e.location.hash.substr(1)),r=n.pathname,i=void 0===r?"/":r,o=n.search,a=void 0===o?"":o,u=n.hash;return Xt("",{pathname:i,search:a,hash:void 0===u?"":u},t.state&&t.state.usr||null,t.state&&t.state.key||"default")}),(function(e,t){var n=e.document.querySelector("base"),r="";if(n&&n.getAttribute("href")){var i=e.location.href,o=i.indexOf("#");r=-1===o?i:i.slice(0,o)}return r+"#"+("string"===typeof t?t:en(t))}),(function(e,t){Zt("/"===e.pathname.charAt(0),"relative pathnames are not supported in hash history.push("+JSON.stringify(t)+")")}),e)}({window:r,v5Compat:!0}));var o=i.current,a=Ft(ee({action:o.action,location:o.location}),2),u=a[0],l=a[1];return re((function(){return o.listen(l)}),[o]),p(tr,{basename:t,children:n,location:u.location,navigationType:u.action,navigator:o})}var sr=Ae((function(e,t){var n=e.onClick,r=e.relative,i=e.reloadDocument,o=e.replace,a=e.state,u=e.target,l=e.to,c=e.preventScrollReset,s=or(e,ur),f=function(e,t){var n=(void 0===t?{}:t).relative;zn()||Jt(!1);var r=le(Ln),i=r.basename,o=r.navigator,a=Un(e,{relative:n}),u=a.hash,l=a.pathname,c=a.search,s=l;return"/"!==i&&(s="/"===l?i:yn([i,l])),o.createHref({pathname:s,search:c,hash:u})}(l,{relative:r}),d=function(e,t){var n=void 0===t?{}:t,r=n.target,i=n.replace,o=n.state,a=n.preventScrollReset,u=n.relative,l=$n(),c=jn(),s=Un(e,{relative:u});return ue((function(t){if(function(e,t){return 0===e.button&&(!t||"_self"===t)&&!function(e){return!!(e.metaKey||e.altKey||e.ctrlKey||e.shiftKey)}(e)}(t,r)){t.preventDefault();var n=void 0!==i?i:en(c)===en(s);l(e,{replace:n,state:o,preventScrollReset:a,relative:u})}}),[c,l,s,i,o,r,e,a,u])}(l,{replace:o,state:a,target:u,preventScrollReset:c,relative:r});return p("a",ir({},s,{href:f,onClick:i?n:function(e){n&&n(e),e.defaultPrevented||d(e)},ref:t,target:u}))}));var fr=Ae((function(e,t){var n=e["aria-current"],r=void 0===n?"page":n,i=e.caseSensitive,o=void 0!==i&&i,a=e.className,u=void 0===a?"":a,l=e.end,c=void 0!==l&&l,s=e.style,f=e.to,d=e.children,h=or(e,lr),v=Un(f,{relative:h.relative}),m=jn(),g=le(Bn),y=le(Ln).navigator,_=y.encodeLocation?y.encodeLocation(v).pathname:v.pathname,b=m.pathname,D=g&&g.navigation&&g.navigation.location?g.navigation.location.pathname:null;o||(b=b.toLowerCase(),D=D?D.toLowerCase():null,_=_.toLowerCase());var w,x=b===_||!c&&b.startsWith(_)&&"/"===b.charAt(_.length),k=null!=D&&(D===_||!c&&D.startsWith(_)&&"/"===D.charAt(_.length)),C=x?r:void 0;w="function"===typeof u?u({isActive:x,isPending:k}):[u,x?"active":null,k?"pending":null].filter(Boolean).join(" ");var A="function"===typeof s?s({isActive:x,isPending:k}):s;return p(sr,ir({},h,{"aria-current":C,className:w,ref:t,style:A,to:f}),"function"===typeof d?d({isActive:x,isPending:k}):d)}));var dr,hr;function pr(e){var t=ie(ar(e)),n=jn(),r=ae((function(){return function(e,t){var n,r=ar(e),i=Mt(t.keys());try{var o=function(){var e=n.value;r.has(e)||t.getAll(e).forEach((function(t){r.append(e,t)}))};for(i.s();!(n=i.n()).done;)o()}catch(a){i.e(a)}finally{i.f()}return r}(n.search,t.current)}),[n.search]),i=$n(),o=ue((function(e,t){var n=ar("function"===typeof e?e(r):e);i("?"+n,t)}),[i,r]);return[r,o]}(function(e){e.UseScrollRestoration="useScrollRestoration",e.UseSubmitImpl="useSubmitImpl",e.UseFetcher="useFetcher"})(dr||(dr={})),function(e){e.UseFetchers="useFetchers",e.UseScrollRestoration="useScrollRestoration"}(hr||(hr={}));var vr;function mr(e,t,n){return(t=It(t))in e?Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}):e[t]=n,e}function gr(e,t){var n=Object.keys(e);if(Object.getOwnPropertySymbols){var r=Object.getOwnPropertySymbols(e);t&&(r=r.filter((function(t){return Object.getOwnPropertyDescriptor(e,t).enumerable}))),n.push.apply(n,r)}return n}function yr(e){for(var t=1;t=100&&(t=n-n%10),e<100&&e>=10&&(t=n-n%5),e<10&&e>=1&&(t=n),e<1&&e>.01&&(t=Math.round(40*e)/40),di(_t().duration(t||.001,"seconds").asMilliseconds()).replace(/\s/g,"")}(r/Xr),date:si(t||_t()().toDate())}},si=function(e){return _t().tz(e).utc().format(Kr)},fi=function(e){return _t().tz(e).format(Kr)},di=function(e){var t=Math.floor(e%1e3),n=Math.floor(e/1e3%60),r=Math.floor(e/1e3/60%60),i=Math.floor(e/1e3/3600%24),o=Math.floor(e/864e5),a=["d","h","m","s","ms"],u=[o,i,r,n,t].map((function(e,t){return e?"".concat(e).concat(a[t]):""}));return u.filter((function(e){return e})).join(" ")},hi=function(e){var t=_t()(1e3*e);return t.isValid()?t.toDate():new Date},pi=[{title:"Last 5 minutes",duration:"5m"},{title:"Last 15 minutes",duration:"15m"},{title:"Last 30 minutes",duration:"30m",isDefault:!0},{title:"Last 1 hour",duration:"1h"},{title:"Last 3 hours",duration:"3h"},{title:"Last 6 hours",duration:"6h"},{title:"Last 12 hours",duration:"12h"},{title:"Last 24 hours",duration:"24h"},{title:"Last 2 days",duration:"2d"},{title:"Last 7 days",duration:"7d"},{title:"Last 30 days",duration:"30d"},{title:"Last 90 days",duration:"90d"},{title:"Last 180 days",duration:"180d"},{title:"Last 1 year",duration:"1y"},{title:"Yesterday",duration:"1d",until:function(){return _t()().tz().subtract(1,"day").endOf("day").toDate()}},{title:"Today",duration:"1d",until:function(){return _t()().tz().endOf("day").toDate()}}].map((function(e){return yr({id:e.title.replace(/\s/g,"_").toLocaleLowerCase(),until:e.until?e.until:function(){return _t()().tz().toDate()}},e)})),vi=function(e){var t,n=e.relativeTimeId,r=e.defaultDuration,i=e.defaultEndInput,o=null===(t=pi.find((function(e){return e.isDefault})))||void 0===t?void 0:t.id,a=n||Br("g0.relative_time",o),u=pi.find((function(e){return e.id===a}));return{relativeTimeId:u?a:"none",duration:u?u.duration:r,endInput:u?u.until():i}},mi=function(e){var t=_t()().tz(e);return"UTC".concat(t.format("Z"))},gi=function(){var e=arguments.length>0&&void 0!==arguments[0]?arguments[0]:"",t=new RegExp(e,"i");return ri.reduce((function(n,r){var i=(r.match(/^(.*?)\//)||[])[1]||"unknown",o=mi(r),a=o.replace(/UTC|0/,""),u=r.replace(/[/_]/g," "),l={region:r,utc:o,search:"".concat(r," ").concat(o," ").concat(u," ").concat(a)},c=!e||e&&t.test(l.search);return c&&n[i]?n[i].push(l):c&&(n[i]=[l]),n}),{})},yi=function(e){_t().tz.setDefault(e)},_i=Ir("TIMEZONE")||_t().tz.guess();yi(_i);var bi,Di=Br("g0.range_input"),wi=vi({defaultDuration:Di||"1h",defaultEndInput:(bi=Br("g0.end_input",_t()().utc().format(Kr)),_t()(bi).utcOffset(0,!0).toDate()),relativeTimeId:Di?Br("g0.relative_time","none"):void 0}),xi=wi.duration,ki=wi.endInput,Ci=wi.relativeTimeId,Ai={duration:xi,period:ci(xi,ki),relativeTime:Ci,timezone:_i};function Ei(e,t){switch(t.type){case"SET_DURATION":return yr(yr({},e),{},{duration:t.payload,period:ci(t.payload,hi(e.period.end)),relativeTime:"none"});case"SET_RELATIVE_TIME":return yr(yr({},e),{},{duration:t.payload.duration,period:ci(t.payload.duration,t.payload.until),relativeTime:t.payload.id});case"SET_PERIOD":var n=function(e){var t=e.to.valueOf()-e.from.valueOf();return di(t)}(t.payload);return yr(yr({},e),{},{duration:n,period:ci(n,t.payload.to),relativeTime:"none"});case"RUN_QUERY":var r=vi({relativeTimeId:e.relativeTime,defaultDuration:e.duration,defaultEndInput:hi(e.period.end)}),i=r.duration,o=r.endInput;return yr(yr({},e),{},{period:ci(i,o)});case"RUN_QUERY_TO_NOW":return yr(yr({},e),{},{period:ci(e.duration)});case"SET_TIMEZONE":return yi(t.payload),Lr("TIMEZONE",t.payload),yr(yr({},e),{},{timezone:t.payload});default:throw new Error}}var Si=j({}),Ni=function(){return le(Si).state},Fi=function(){return le(Si).dispatch},Mi=function(){var e,t=(null===(e=(window.location.hash.split("?")[1]||"").match(/g\d+\.expr/g))||void 0===e?void 0:e.length)||1;return new Array(t>4?4:t).fill(1).map((function(e,t){return Br("g".concat(t,".expr"),"")}))}(),Ti={query:Mi,queryHistory:Mi.map((function(e){return{index:0,values:[e]}})),autocomplete:Ir("AUTOCOMPLETE")||!1};function Oi(e,t){switch(t.type){case"SET_QUERY":return yr(yr({},e),{},{query:t.payload.map((function(e){return e}))});case"SET_QUERY_HISTORY":return yr(yr({},e),{},{queryHistory:t.payload});case"SET_QUERY_HISTORY_BY_INDEX":return e.queryHistory.splice(t.payload.queryNumber,1,t.payload.value),yr(yr({},e),{},{queryHistory:e.queryHistory});case"TOGGLE_AUTOCOMPLETE":return Lr("AUTOCOMPLETE",!e.autocomplete),yr(yr({},e),{},{autocomplete:!e.autocomplete});default:throw new Error}}var Bi=j({}),Li=function(){return le(Bi).state},Ii=function(){return le(Bi).dispatch},Pi=function(){return Vr("svg",{viewBox:"0 0 74 24",fill:"currentColor",children:[Vr("path",{d:"M6.11767 10.4759C6.47736 10.7556 6.91931 10.909 7.37503 10.9121H7.42681C7.90756 10.9047 8.38832 10.7199 8.67677 10.4685C10.1856 9.18921 14.5568 5.18138 14.5568 5.18138C15.7254 4.09438 12.4637 3.00739 7.42681 3H7.36764C2.3308 3.00739 -0.930935 4.09438 0.237669 5.18138C0.237669 5.18138 4.60884 9.18921 6.11767 10.4759ZM8.67677 12.6424C8.31803 12.9248 7.87599 13.0808 7.41941 13.0861H7.37503C6.91845 13.0808 6.47641 12.9248 6.11767 12.6424C5.0822 11.7551 1.38409 8.42018 0.000989555 7.14832V9.07829C0.000989555 9.29273 0.0823481 9.57372 0.222877 9.70682L0.293316 9.7712L0.293344 9.77122C1.33784 10.7258 4.83903 13.9255 6.11767 15.0161C6.47641 15.2985 6.91845 15.4545 7.37503 15.4597H7.41941C7.90756 15.4449 8.38092 15.2601 8.67677 15.0161C9.9859 13.9069 13.6249 10.572 14.5642 9.70682C14.7121 9.57372 14.7861 9.29273 14.7861 9.07829V7.14832C12.7662 8.99804 10.7297 10.8295 8.67677 12.6424ZM7.41941 17.6263C7.87513 17.6232 8.31708 17.4698 8.67677 17.19C10.7298 15.3746 12.7663 13.5407 14.7861 11.6885V13.6259C14.7861 13.8329 14.7121 14.1139 14.5642 14.247C13.6249 15.1196 9.9859 18.4471 8.67677 19.5563C8.38092 19.8077 7.90756 19.9926 7.41941 20H7.37503C6.91931 19.9968 6.47736 19.8435 6.11767 19.5637C4.91427 18.5373 1.74219 15.6364 0.502294 14.5025C0.393358 14.4029 0.299337 14.3169 0.222877 14.247C0.0823481 14.1139 0.000989555 13.8329 0.000989555 13.6259V11.6885C1.38409 12.953 5.0822 16.2953 6.11767 17.1827C6.47641 17.4651 6.91845 17.6211 7.37503 17.6263H7.41941Z"}),Vr("path",{d:"M34.9996 5L29.1596 19.46H26.7296L20.8896 5H23.0496C23.2829 5 23.4729 5.05667 23.6196 5.17C23.7663 5.28333 23.8763 5.43 23.9496 5.61L27.3596 14.43C27.4729 14.7167 27.5796 15.0333 27.6796 15.38C27.7863 15.72 27.8863 16.0767 27.9796 16.45C28.0596 16.0767 28.1463 15.72 28.2396 15.38C28.3329 15.0333 28.4363 14.7167 28.5496 14.43L31.9396 5.61C31.9929 5.45667 32.0963 5.31667 32.2496 5.19C32.4096 5.06333 32.603 5 32.8297 5H34.9996ZM52.1763 5V19.46H49.8064V10.12C49.8064 9.74667 49.8263 9.34333 49.8663 8.91L45.4963 17.12C45.2897 17.5133 44.973 17.71 44.5463 17.71H44.1663C43.7397 17.71 43.4231 17.5133 43.2164 17.12L38.7963 8.88C38.8163 9.1 38.833 9.31667 38.8463 9.53C38.8597 9.74333 38.8663 9.94 38.8663 10.12V19.46H36.4963V5H38.5263C38.6463 5 38.7497 5.00333 38.8363 5.01C38.923 5.01667 38.9997 5.03333 39.0663 5.06C39.1397 5.08667 39.203 5.13 39.2563 5.19C39.3163 5.25 39.373 5.33 39.4263 5.43L43.7563 13.46C43.8697 13.6733 43.973 13.8933 44.0663 14.12C44.1663 14.3467 44.263 14.58 44.3563 14.82C44.4497 14.5733 44.5464 14.3367 44.6464 14.11C44.7464 13.8767 44.8531 13.6533 44.9664 13.44L49.2363 5.43C49.2897 5.33 49.3463 5.25 49.4063 5.19C49.4663 5.13 49.5297 5.08667 49.5963 5.06C49.6697 5.03333 49.7497 5.01667 49.8363 5.01C49.923 5.00333 50.0264 5 50.1464 5H52.1763ZM61.0626 18.73C61.7426 18.73 62.3492 18.6133 62.8826 18.38C63.4226 18.14 63.8792 17.81 64.2526 17.39C64.6259 16.97 64.9092 16.4767 65.1026 15.91C65.3026 15.3367 65.4026 14.72 65.4026 14.06V5.31H66.4226V14.06C66.4226 14.84 66.2993 15.57 66.0527 16.25C65.806 16.9233 65.4493 17.5133 64.9827 18.02C64.5227 18.52 63.9592 18.9133 63.2926 19.2C62.6326 19.4867 61.8892 19.63 61.0626 19.63C60.2359 19.63 59.4893 19.4867 58.8227 19.2C58.1627 18.9133 57.5992 18.52 57.1326 18.02C56.6726 17.5133 56.3193 16.9233 56.0727 16.25C55.826 15.57 55.7026 14.84 55.7026 14.06V5.31H56.7327V14.05C56.7327 14.71 56.8292 15.3267 57.0226 15.9C57.2226 16.4667 57.506 16.96 57.8727 17.38C58.246 17.8 58.6993 18.13 59.2327 18.37C59.7727 18.61 60.3826 18.73 61.0626 18.73ZM71.4438 19.46H70.4138V5.31H71.4438V19.46Z"})]})},Ri=function(){return Vr("svg",{viewBox:"0 0 15 17",fill:"currentColor",children:Vr("path",{d:"M6.11767 7.47586C6.47736 7.75563 6.91931 7.90898 7.37503 7.91213H7.42681C7.90756 7.90474 8.38832 7.71987 8.67677 7.46846C10.1856 6.18921 14.5568 2.18138 14.5568 2.18138C15.7254 1.09438 12.4637 0.00739 7.42681 0H7.36764C2.3308 0.00739 -0.930935 1.09438 0.237669 2.18138C0.237669 2.18138 4.60884 6.18921 6.11767 7.47586ZM8.67677 9.64243C8.31803 9.92483 7.87599 10.0808 7.41941 10.0861H7.37503C6.91845 10.0808 6.47641 9.92483 6.11767 9.64243C5.0822 8.75513 1.38409 5.42018 0.000989555 4.14832V6.07829C0.000989555 6.29273 0.0823481 6.57372 0.222877 6.70682L0.293316 6.7712L0.293344 6.77122C1.33784 7.72579 4.83903 10.9255 6.11767 12.0161C6.47641 12.2985 6.91845 12.4545 7.37503 12.4597H7.41941C7.90756 12.4449 8.38092 12.2601 8.67677 12.0161C9.9859 10.9069 13.6249 7.57198 14.5642 6.70682C14.7121 6.57372 14.7861 6.29273 14.7861 6.07829V4.14832C12.7662 5.99804 10.7297 7.82949 8.67677 9.64243ZM7.41941 14.6263C7.87513 14.6232 8.31708 14.4698 8.67677 14.19C10.7298 12.3746 12.7663 10.5407 14.7861 8.68853V10.6259C14.7861 10.8329 14.7121 11.1139 14.5642 11.247C13.6249 12.1196 9.9859 15.4471 8.67677 16.5563C8.38092 16.8077 7.90756 16.9926 7.41941 17H7.37503C6.91931 16.9968 6.47736 16.8435 6.11767 16.5637C4.91427 15.5373 1.74219 12.6364 0.502294 11.5025C0.393358 11.4029 0.299337 11.3169 0.222877 11.247C0.0823481 11.1139 0.000989555 10.8329 0.000989555 10.6259V8.68853C1.38409 9.95303 5.0822 13.2953 6.11767 14.1827C6.47641 14.4651 6.91845 14.6211 7.37503 14.6263H7.41941Z"})})},zi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19.14 12.94c.04-.3.06-.61.06-.94 0-.32-.02-.64-.07-.94l2.03-1.58c.18-.14.23-.41.12-.61l-1.92-3.32c-.12-.22-.37-.29-.59-.22l-2.39.96c-.5-.38-1.03-.7-1.62-.94l-.36-2.54c-.04-.24-.24-.41-.48-.41h-3.84c-.24 0-.43.17-.47.41l-.36 2.54c-.59.24-1.13.57-1.62.94l-2.39-.96c-.22-.08-.47 0-.59.22L2.74 8.87c-.12.21-.08.47.12.61l2.03 1.58c-.05.3-.09.63-.09.94s.02.64.07.94l-2.03 1.58c-.18.14-.23.41-.12.61l1.92 3.32c.12.22.37.29.59.22l2.39-.96c.5.38 1.03.7 1.62.94l.36 2.54c.05.24.24.41.48.41h3.84c.24 0 .44-.17.47-.41l.36-2.54c.59-.24 1.13-.56 1.62-.94l2.39.96c.22.08.47 0 .59-.22l1.92-3.32c.12-.22.07-.47-.12-.61l-2.01-1.58zM12 15.6c-1.98 0-3.6-1.62-3.6-3.6s1.62-3.6 3.6-3.6 3.6 1.62 3.6 3.6-1.62 3.6-3.6 3.6z"})})},ji=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 6.41 17.59 5 12 10.59 6.41 5 5 6.41 10.59 12 5 17.59 6.41 19 12 13.41 17.59 19 19 17.59 13.41 12z"})})},$i=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 5V2L8 6l4 4V7c3.31 0 6 2.69 6 6 0 2.97-2.17 5.43-5 5.91v2.02c3.95-.49 7-3.85 7-7.93 0-4.42-3.58-8-8-8zm-6 8c0-1.65.67-3.15 1.76-4.24L6.34 7.34C4.9 8.79 4 10.79 4 13c0 4.08 3.05 7.44 7 7.93v-2.02c-2.83-.48-5-2.94-5-5.91z"})})},Hi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm1 15h-2v-6h2v6zm0-8h-2V7h2v2z"})})},Ui=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M1 21h22L12 2 1 21zm12-3h-2v-2h2v2zm0-4h-2v-4h2v4z"})})},Yi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm1 15h-2v-2h2v2zm0-4h-2V7h2v6z"})})},Vi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm-2 15-5-5 1.41-1.41L10 14.17l7.59-7.59L19 8l-9 9z"})})},qi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 6v3l4-4-4-4v3c-4.42 0-8 3.58-8 8 0 1.57.46 3.03 1.24 4.26L6.7 14.8c-.45-.83-.7-1.79-.7-2.8 0-3.31 2.69-6 6-6zm6.76 1.74L17.3 9.2c.44.84.7 1.79.7 2.8 0 3.31-2.69 6-6 6v-3l-4 4 4 4v-3c4.42 0 8-3.58 8-8 0-1.57-.46-3.03-1.24-4.26z"})})},Wi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M7.41 8.59 12 13.17l4.59-4.58L18 10l-6 6-6-6 1.41-1.41z"})})},Qi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m7 10 5 5 5-5z"})})},Gi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Vr("path",{d:"M11.99 2C6.47 2 2 6.48 2 12s4.47 10 9.99 10C17.52 22 22 17.52 22 12S17.52 2 11.99 2zM12 20c-4.42 0-8-3.58-8-8s3.58-8 8-8 8 3.58 8 8-3.58 8-8 8z"}),Vr("path",{d:"M12.5 7H11v6l5.25 3.15.75-1.23-4.5-2.67z"})]})},Ji=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 3h-1V1h-2v2H7V1H5v2H4c-1.1 0-2 .9-2 2v16c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2V5c0-1.1-.9-2-2-2zm0 18H4V8h16v13z"})})},Zi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m22 5.72-4.6-3.86-1.29 1.53 4.6 3.86L22 5.72zM7.88 3.39 6.6 1.86 2 5.71l1.29 1.53 4.59-3.85zM12.5 8H11v6l4.75 2.85.75-1.23-4-2.37V8zM12 4c-4.97 0-9 4.03-9 9s4.02 9 9 9c4.97 0 9-4.03 9-9s-4.03-9-9-9zm0 16c-3.87 0-7-3.13-7-7s3.13-7 7-7 7 3.13 7 7-3.13 7-7 7z"})})},Ki=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 5H4c-1.1 0-1.99.9-1.99 2L2 17c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2V7c0-1.1-.9-2-2-2zm-9 3h2v2h-2V8zm0 3h2v2h-2v-2zM8 8h2v2H8V8zm0 3h2v2H8v-2zm-1 2H5v-2h2v2zm0-3H5V8h2v2zm9 7H8v-2h8v2zm0-4h-2v-2h2v2zm0-3h-2V8h2v2zm3 3h-2v-2h2v2zm0-3h-2V8h2v2z"})})},Xi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M8 5v14l11-7z"})})},eo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m10 16.5 6-4.5-6-4.5v9zM12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm0 18c-4.41 0-8-3.59-8-8s3.59-8 8-8 8 3.59 8 8-3.59 8-8 8z"})})},to=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m3.5 18.49 6-6.01 4 4L22 6.92l-1.41-1.41-7.09 7.97-4-4L2 16.99z"})})},no=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M10 10.02h5V21h-5zM17 21h3c1.1 0 2-.9 2-2v-9h-5v11zm3-18H5c-1.1 0-2 .9-2 2v3h19V5c0-1.1-.9-2-2-2zM3 19c0 1.1.9 2 2 2h3V10H3v9z"})})},ro=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M9.4 16.6 4.8 12l4.6-4.6L8 6l-6 6 6 6 1.4-1.4zm5.2 0 4.6-4.6-4.6-4.6L16 6l6 6-6 6-1.4-1.4z"})})},io=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M6 19c0 1.1.9 2 2 2h8c1.1 0 2-.9 2-2V7H6v12zM19 4h-3.5l-1-1h-5l-1 1H5v2h14V4z"})})},oo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 13h-6v6h-2v-6H5v-2h6V5h2v6h6v2z"})})},ao=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 13H5v-2h14v2z"})})},uo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M8.9999 14.7854L18.8928 4.8925C19.0803 4.70497 19.3347 4.59961 19.5999 4.59961C19.8651 4.59961 20.1195 4.70497 20.307 4.8925L21.707 6.2925C22.0975 6.68303 22.0975 7.31619 21.707 7.70672L9.70701 19.7067C9.31648 20.0972 8.68332 20.0972 8.2928 19.7067L2.6928 14.1067C2.50526 13.9192 2.3999 13.6648 2.3999 13.3996C2.3999 13.1344 2.50526 12.88 2.6928 12.6925L4.0928 11.2925C4.48332 10.902 5.11648 10.902 5.50701 11.2925L8.9999 14.7854Z"})})},lo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 4.5C7 4.5 2.73 7.61 1 12c1.73 4.39 6 7.5 11 7.5s9.27-3.11 11-7.5c-1.73-4.39-6-7.5-11-7.5zM12 17c-2.76 0-5-2.24-5-5s2.24-5 5-5 5 2.24 5 5-2.24 5-5 5zm0-8c-1.66 0-3 1.34-3 3s1.34 3 3 3 3-1.34 3-3-1.34-3-3-3z"})})},co=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 7c2.76 0 5 2.24 5 5 0 .65-.13 1.26-.36 1.83l2.92 2.92c1.51-1.26 2.7-2.89 3.43-4.75-1.73-4.39-6-7.5-11-7.5-1.4 0-2.74.25-3.98.7l2.16 2.16C10.74 7.13 11.35 7 12 7zM2 4.27l2.28 2.28.46.46C3.08 8.3 1.78 10.02 1 12c1.73 4.39 6 7.5 11 7.5 1.55 0 3.03-.3 4.38-.84l.42.42L19.73 22 21 20.73 3.27 3 2 4.27zM7.53 9.8l1.55 1.55c-.05.21-.08.43-.08.65 0 1.66 1.34 3 3 3 .22 0 .44-.03.65-.08l1.55 1.55c-.67.33-1.41.53-2.2.53-2.76 0-5-2.24-5-5 0-.79.2-1.53.53-2.2zm4.31-.78 3.15 3.15.02-.16c0-1.66-1.34-3-3-3l-.17.01z"})})},so=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M16 1H4c-1.1 0-2 .9-2 2v14h2V3h12V1zm3 4H8c-1.1 0-2 .9-2 2v14c0 1.1.9 2 2 2h11c1.1 0 2-.9 2-2V7c0-1.1-.9-2-2-2zm0 16H8V7h11v14z"})})},fo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 9H4v2h16V9zM4 15h16v-2H4v2z"})})},ho=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M23 8c0 1.1-.9 2-2 2-.18 0-.35-.02-.51-.07l-3.56 3.55c.05.16.07.34.07.52 0 1.1-.9 2-2 2s-2-.9-2-2c0-.18.02-.36.07-.52l-2.55-2.55c-.16.05-.34.07-.52.07s-.36-.02-.52-.07l-4.55 4.56c.05.16.07.33.07.51 0 1.1-.9 2-2 2s-2-.9-2-2 .9-2 2-2c.18 0 .35.02.51.07l4.56-4.55C8.02 9.36 8 9.18 8 9c0-1.1.9-2 2-2s2 .9 2 2c0 .18-.02.36-.07.52l2.55 2.55c.16-.05.34-.07.52-.07s.36.02.52.07l3.55-3.56C19.02 8.35 19 8.18 19 8c0-1.1.9-2 2-2s2 .9 2 2z"})})},po=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Vr("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M21 5C19.89 4.65 18.67 4.5 17.5 4.5C15.55 4.5 13.45 4.9 12 6C10.55 4.9 8.45 4.5 6.5 4.5C5.33 4.5 4.11 4.65 3 5C2.25 5.25 1.6 5.55 1 6V20.6C1 20.85 1.25 21.1 1.5 21.1C1.6 21.1 1.65 21.1 1.75 21.05C3.15 20.3 4.85 20 6.5 20C8.2 20 10.65 20.65 12 21.5C13.35 20.65 15.8 20 17.5 20C19.15 20 20.85 20.3 22.25 21.05C22.35 21.1 22.4 21.1 22.5 21.1C22.75 21.1 23 20.85 23 20.6V6C22.4 5.55 21.75 5.25 21 5ZM21 18.5C19.9 18.15 18.7 18 17.5 18C15.8 18 13.35 18.65 12 19.5C10.65 18.65 8.2 18 6.5 18C5.3 18 4.1 18.15 3 18.5V7C4.1 6.65 5.3 6.5 6.5 6.5C8.2 6.5 10.65 7.15 12 8C13.35 7.15 15.8 6.5 17.5 6.5C18.7 6.5 19.9 6.65 21 7V18.5Z"}),Vr("path",{d:"M17.5 10.5C18.38 10.5 19.23 10.59 20 10.76V9.24C19.21 9.09 18.36 9 17.5 9C15.8 9 14.26 9.29 13 9.83V11.49C14.13 10.85 15.7 10.5 17.5 10.5ZM13 12.49V14.15C14.13 13.51 15.7 13.16 17.5 13.16C18.38 13.16 19.23 13.25 20 13.42V11.9C19.21 11.75 18.36 11.66 17.5 11.66C15.8 11.66 14.26 11.96 13 12.49ZM17.5 14.33C15.8 14.33 14.26 14.62 13 15.16V16.82C14.13 16.18 15.7 15.83 17.5 15.83C18.38 15.83 19.23 15.92 20 16.09V14.57C19.21 14.41 18.36 14.33 17.5 14.33Z"}),Vr("path",{d:"M6.5 10.5C5.62 10.5 4.77 10.59 4 10.76V9.24C4.79 9.09 5.64 9 6.5 9C8.2 9 9.74 9.29 11 9.83V11.49C9.87 10.85 8.3 10.5 6.5 10.5ZM11 12.49V14.15C9.87 13.51 8.3 13.16 6.5 13.16C5.62 13.16 4.77 13.25 4 13.42V11.9C4.79 11.75 5.64 11.66 6.5 11.66C8.2 11.66 9.74 11.96 11 12.49ZM6.5 14.33C8.2 14.33 9.74 14.62 11 15.16V16.82C9.87 16.18 8.3 15.83 6.5 15.83C5.62 15.83 4.77 15.92 4 16.09V14.57C4.79 14.41 5.64 14.33 6.5 14.33Z"})]})},vo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.49 2 2 6.49 2 12s4.49 10 10 10 10-4.49 10-10S17.51 2 12 2zm0 18c-4.41 0-8-3.59-8-8s3.59-8 8-8 8 3.59 8 8-3.59 8-8 8zm3-8c0 1.66-1.34 3-3 3s-3-1.34-3-3 1.34-3 3-3 3 1.34 3 3z"})})},mo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M12 2C6.48 2 2 6.48 2 12C2 17.52 6.48 22 12 22C17.52 22 22 17.52 22 12C22 6.48 17.52 2 12 2ZM12 6C9.79 6 8 7.79 8 10H10C10 8.9 10.9 8 12 8C13.1 8 14 8.9 14 10C14 10.8792 13.4202 11.3236 12.7704 11.8217C11.9421 12.4566 11 13.1787 11 15H13C13 13.9046 13.711 13.2833 14.4408 12.6455C15.21 11.9733 16 11.2829 16 10C16 7.79 14.21 6 12 6ZM13 16V18H11V16H13Z"})})},go=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M4 20h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2s.9 2 2 2zm0-3h2v2H4v-2zM2 6c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2zm4 1H4V5h2v2zm-2 7h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2s.9 2 2 2zm0-3h2v2H4v-2z"})})},yo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 8c1.1 0 2-.9 2-2s-.9-2-2-2-2 .9-2 2 .9 2 2 2zm0 2c-1.1 0-2 .9-2 2s.9 2 2 2 2-.9 2-2-.9-2-2-2zm0 6c-1.1 0-2 .9-2 2s.9 2 2 2 2-.9 2-2-.9-2-2-2z"})})},_o=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M3 17v2h6v-2H3zM3 5v2h10V5H3zm10 16v-2h8v-2h-8v-2h-2v6h2zM7 9v2H3v2h4v2h2V9H7zm14 4v-2H11v2h10zm-6-4h2V7h4V5h-4V3h-2v6z"})})},bo=function(e){var t=Ft(ee({width:0,height:0}),2),n=t[0],r=t[1];return ne((function(){var t=new ResizeObserver((function(e){var t=e[0].contentRect,n=t.width,i=t.height;r({width:n,height:i})}));return e&&t.observe(e),function(){e&&t.unobserve(e)}}),[e]),n},Do=n(123),wo=n.n(Do);function xo(e,t){if(null==e)return{};var n,r,i=function(e,t){if(null==e)return{};var n,r,i={},o=Object.keys(e);for(r=0;r=0||(i[n]=e[n]);return i}(e,t);if(Object.getOwnPropertySymbols){var o=Object.getOwnPropertySymbols(e);for(r=0;r=0||Object.prototype.propertyIsEnumerable.call(e,n)&&(i[n]=e[n])}return i}var ko=["to","isNavLink","children"],Co=function(e){var t=e.to,n=e.isNavLink,r=e.children,i=xo(e,ko);return n?Vr(fr,yr(yr({to:t},i),{},{children:r})):Vr("div",yr(yr({},i),{},{children:r}))},Ao=function(e){var t,n=e.activeItem,r=e.item,i=e.color,o=void 0===i?Rr("color-primary"):i,a=e.activeNavRef,u=e.onChange,l=e.isNavLink;return Vr(Co,{className:wo()(mr({"vm-tabs-item":!0,"vm-tabs-item_active":n===r.value},r.className||"",r.className)),isNavLink:l,to:r.value,style:{color:o},onClick:(t=r.value,function(){u&&u(t)}),ref:n===r.value?a:void 0,children:[r.icon&&Vr("div",{className:wo()({"vm-tabs-item__icon":!0,"vm-tabs-item__icon_single":!r.label}),children:r.icon}),r.label]})},Eo=function(e){var t=e.activeItem,n=e.items,r=e.color,i=void 0===r?Rr("color-primary"):r,o=e.onChange,a=e.indicatorPlacement,u=void 0===a?"bottom":a,l=e.isNavLink,c=bo(document.body),s=ie(null),f=Ft(ee({left:0,width:0,bottom:0}),2),d=f[0],h=f[1];return ne((function(){var e;if((null===(e=s.current)||void 0===e?void 0:e.base)instanceof HTMLElement){var t=s.current.base,n=t.offsetLeft,r=t.offsetWidth,i=t.offsetHeight;h({left:n,width:r,bottom:"top"===u?i-2:0})}}),[c,t,s,n]),Vr("div",{className:"vm-tabs",children:[n.map((function(e){return Vr(Ao,{activeItem:t,item:e,onChange:o,color:i,activeNavRef:s,isNavLink:l},e.value)})),Vr("div",{className:"vm-tabs__indicator",style:yr(yr({},d),{},{borderColor:i})})]})},So=[{value:"chart",icon:Vr(to,{}),label:"Graph",prometheusCode:0},{value:"code",icon:Vr(ro,{}),label:"JSON",prometheusCode:3},{value:"table",icon:Vr(no,{}),label:"Table",prometheusCode:1}],No=function(){var e=Io().displayType,t=Po();return Vr(Eo,{activeItem:e,items:So,onChange:function(n){var r;t({type:"SET_DISPLAY_TYPE",payload:null!==(r=n)&&void 0!==r?r:e})}})},Fo=Br("g0.tab",0),Mo=So.find((function(e){return e.prometheusCode===+Fo||e.value===Fo})),To=Ir("SERIES_LIMITS"),Oo={displayType:(null===Mo||void 0===Mo?void 0:Mo.value)||"chart",nocache:!1,isTracingEnabled:!1,seriesLimits:To?JSON.parse(Ir("SERIES_LIMITS")):Tr,tableCompact:Ir("TABLE_COMPACT")||!1};function Bo(e,t){switch(t.type){case"SET_DISPLAY_TYPE":return yr(yr({},e),{},{displayType:t.payload});case"SET_SERIES_LIMITS":return Lr("SERIES_LIMITS",JSON.stringify(t.payload)),yr(yr({},e),{},{seriesLimits:t.payload});case"TOGGLE_QUERY_TRACING":return yr(yr({},e),{},{isTracingEnabled:!e.isTracingEnabled});case"TOGGLE_NO_CACHE":return yr(yr({},e),{},{nocache:!e.nocache});case"TOGGLE_TABLE_COMPACT":return Lr("TABLE_COMPACT",!e.tableCompact),yr(yr({},e),{},{tableCompact:!e.tableCompact});default:throw new Error}}var Lo=j({}),Io=function(){return le(Lo).state},Po=function(){return le(Lo).dispatch},Ro={customStep:Br("g0.step_input",""),yaxis:{limits:{enable:!1,range:{1:[0,0]}}}};function zo(e,t){switch(t.type){case"TOGGLE_ENABLE_YAXIS_LIMITS":return yr(yr({},e),{},{yaxis:yr(yr({},e.yaxis),{},{limits:yr(yr({},e.yaxis.limits),{},{enable:!e.yaxis.limits.enable})})});case"SET_CUSTOM_STEP":return yr(yr({},e),{},{customStep:t.payload});case"SET_YAXIS_LIMITS":return yr(yr({},e),{},{yaxis:yr(yr({},e.yaxis),{},{limits:yr(yr({},e.yaxis.limits),{},{range:t.payload})})});default:throw new Error}}var jo=j({}),$o=function(){return le(jo).state},Ho=function(){return le(jo).dispatch},Uo={runQuery:0,topN:Br("topN",10),date:Br("date",_t()().tz().format(Jr)),focusLabel:Br("focusLabel",""),match:Br("match",""),extraLabel:Br("extra_label","")};function Yo(e,t){switch(t.type){case"SET_TOP_N":return yr(yr({},e),{},{topN:t.payload});case"SET_DATE":return yr(yr({},e),{},{date:t.payload});case"SET_MATCH":return yr(yr({},e),{},{match:t.payload});case"SET_EXTRA_LABEL":return yr(yr({},e),{},{extraLabel:t.payload});case"SET_FOCUS_LABEL":return yr(yr({},e),{},{focusLabel:t.payload});case"RUN_QUERY":return yr(yr({},e),{},{runQuery:e.runQuery+1});default:throw new Error}}var Vo=j({}),qo=function(){return le(Vo).state},Wo=function(){return le(Vo).dispatch},Qo={topN:Br("topN",null),maxLifetime:Br("maxLifetime",""),runQuery:0};function Go(e,t){switch(t.type){case"SET_TOP_N":return yr(yr({},e),{},{topN:t.payload});case"SET_MAX_LIFE_TIME":return yr(yr({},e),{},{maxLifetime:t.payload});case"SET_RUN_QUERY":return yr(yr({},e),{},{runQuery:e.runQuery+1});default:throw new Error}}var Jo=j({}),Zo=function(){return le(Jo).state},Ko={windows:"Windows",mac:"Mac OS",linux:"Linux"};function Xo(){var e=bo(document.body),t=function(){var e=function(){var e=["Android","webOS","iPhone","iPad","iPod","BlackBerry","Windows Phone"].map((function(e){return navigator.userAgent.match(new RegExp(e,"i"))}));return e.some((function(e){return e}))}(),t=window.innerWidth<500;return e||t},n=Ft(ee(t()),2),r=n[0],i=n[1];return ne((function(){i(t())}),[e]),{isMobile:r}}var ea={success:Vr(Vi,{}),error:Vr(Yi,{}),warning:Vr(Ui,{}),info:Vr(Hi,{})},ta=function(e){var t,n=e.variant,r=e.children,i=Wr().isDarkTheme,o=Xo().isMobile;return Vr("div",{className:wo()((t={"vm-alert":!0},mr(t,"vm-alert_".concat(n),n),mr(t,"vm-alert_dark",i),mr(t,"vm-alert_mobile",o),t)),children:[Vr("div",{className:"vm-alert__icon",children:ea[n||"info"]}),Vr("div",{className:"vm-alert__content",children:r})]})},na=j({showInfoMessage:function(){}}),ra=function(){return le(na)},ia={dashboardsSettings:[],dashboardsLoading:!1,dashboardsError:""};function oa(e,t){switch(t.type){case"SET_DASHBOARDS_SETTINGS":return yr(yr({},e),{},{dashboardsSettings:t.payload});case"SET_DASHBOARDS_LOADING":return yr(yr({},e),{},{dashboardsLoading:t.payload});case"SET_DASHBOARDS_ERROR":return yr(yr({},e),{},{dashboardsError:t.payload});default:throw new Error}}var aa=j({}),ua=function(){return le(aa).state},la=function(){for(var e=arguments.length,t=new Array(e),n=0;nh,v=r.top-20<0,m=r.left+x.width+20>f,g=r.left-20<0;return p&&(r.top=t.top-x.height-u),v&&(r.top=t.height+t.top+u),m&&(r.left=t.right-x.width-l),g&&(r.left=t.left+l),d&&(r.width="".concat(t.width,"px")),r.top<0&&(r.top=20),r}),[n,i,b,t,d]);f&&sa(C,(function(){return D(!1)}),n),ne((function(){if(C.current&&b&&(!v||p)){var e=C.current.getBoundingClientRect(),t=e.right,n=e.width;if(t>window.innerWidth){var r=window.innerWidth-20-n;C.current.style.left=rm,y=r.top-20<0,_=r.left+h.width+20>p,b=r.left-20<0;return g&&(r.top=n.top-h.height-c),y&&(r.top=n.height+n.top+c),_&&(r.left=n.right-h.width-f),b&&(r.left=n.left+f),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[v,o,s,h]),b=function(){"boolean"!==typeof r&&f(!0)},D=function(){f(!1)};return ne((function(){"boolean"===typeof r&&f(r)}),[r]),ne((function(){var e,t=null===v||void 0===v||null===(e=v.current)||void 0===e?void 0:e.base;if(t)return t.addEventListener("mouseenter",b),t.addEventListener("mouseleave",D),function(){t.removeEventListener("mouseenter",b),t.removeEventListener("mouseleave",D)}}),[v]),Vr(g,{children:[Vr(g,{ref:v,children:t}),!l&&s&>.createPortal(Vr("div",{className:"vm-tooltip",ref:m,style:_,children:n}),document.body)]})},ga=(Object.values(Ko).find((function(e){return navigator.userAgent.indexOf(e)>=0}))||"unknown")===Ko.mac?"Cmd":"Ctrl",ya=[{title:"Query",list:[{keys:["Enter"],description:"Run"},{keys:["Shift","Enter"],description:"Multi-line queries"},{keys:[ga,"Arrow Up"],description:"Previous command from the Query history"},{keys:[ga,"Arrow Down"],description:"Next command from the Query history"},{keys:[ga,"Click by 'Eye'"],description:"Toggle multiple queries"}]},{title:"Graph",list:[{keys:[ga,"Scroll Up"],alt:["+"],description:"Zoom in"},{keys:[ga,"Scroll Down"],alt:["-"],description:"Zoom out"},{keys:[ga,"Click and Drag"],description:"Move the graph left/right"}]},{title:"Legend",list:[{keys:["Mouse Click"],description:"Select series"},{keys:[ga,"Mouse Click"],description:"Toggle multiple series"}]}],_a="Shortcut keys",ba=function(e){var t=e.showTitle,n=Ft(ee(!1),2),r=n[0],i=n[1],o=Ar();return Vr(g,{children:[Vr(ma,{open:!0!==t&&void 0,title:_a,placement:"bottom-center",children:Vr(fa,{className:o?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Ki,{}),onClick:function(){i(!0)},children:t&&_a})}),r&&Vr(va,{title:"Shortcut keys",onClose:function(){i(!1)},children:Vr("div",{className:"vm-shortcuts",children:ya.map((function(e){return Vr("div",{className:"vm-shortcuts-section",children:[Vr("h3",{className:"vm-shortcuts-section__title",children:e.title}),Vr("div",{className:"vm-shortcuts-section-list",children:e.list.map((function(e){return Vr("div",{className:"vm-shortcuts-section-list-item",children:[Vr("div",{className:"vm-shortcuts-section-list-item__key",children:[e.keys.map((function(t,n){return Vr(g,{children:[Vr("code",{children:t},t),n!==e.keys.length-1?"+":""]})})),e.alt&&e.alt.map((function(t,n){return Vr(g,{children:["or",Vr("code",{children:t},t),n!==e.alt.length-1?"+":""]})}))]}),Vr("p",{className:"vm-shortcuts-section-list-item__description",children:e.description})]},e.keys.join("+"))}))})]},e.title)}))})})]})},Da=function(e){var t=e.open;return Vr("button",{className:wo()({"vm-menu-burger":!0,"vm-menu-burger_opened":t}),children:Vr("span",{})})},wa=function(e){var t=e.background,n=e.color,r=jn().pathname,i=Xo().isMobile,o=ie(null),a=Ft(ee(!1),2),u=a[0],l=a[1],c=function(){l(!1)};return ne(c,[r]),sa(o,c),Vr("div",{className:"vm-header-sidebar",ref:o,children:[Vr("div",{className:wo()({"vm-header-sidebar-button":!0,"vm-header-sidebar-button_open":u}),onClick:function(){l((function(e){return!e}))},children:Vr(Da,{open:u})}),Vr("div",{className:wo()({"vm-header-sidebar-menu":!0,"vm-header-sidebar-menu_open":u}),children:[Vr("div",{children:Vr(pa,{color:n,background:t,direction:"column"})}),Vr("div",{className:"vm-header-sidebar-menu-settings",children:!i&&Vr(ba,{showTitle:!0})})]})]})},xa=function(e){var t=e.label,n=e.value,r=e.type,i=void 0===r?"text":r,o=e.error,a=void 0===o?"":o,u=e.placeholder,l=e.endIcon,c=e.startIcon,s=e.disabled,f=void 0!==s&&s,d=e.autofocus,h=void 0!==d&&d,p=e.helperText,v=e.inputmode,m=void 0===v?"text":v,g=e.onChange,y=e.onEnter,_=e.onKeyDown,b=e.onFocus,D=e.onBlur,w=Wr().isDarkTheme,x=Xo().isMobile,k=ie(null),C=ie(null),A=ae((function(){return"textarea"===i?C:k}),[i]),E=wo()({"vm-text-field__input":!0,"vm-text-field__input_error":a,"vm-text-field__input_icon-start":c,"vm-text-field__input_disabled":f,"vm-text-field__input_textarea":"textarea"===i}),S=function(e){_&&_(e),"Enter"!==e.key||e.shiftKey||(e.preventDefault(),y&&y())},N=function(e){f||g&&g(e.target.value)};ne((function(){var e;h&&!x&&(null===A||void 0===A||null===(e=A.current)||void 0===e?void 0:e.focus)&&A.current.focus()}),[A,h]);var F=function(){b&&b()},M=function(){D&&D()};return Vr("label",{className:wo()({"vm-text-field":!0,"vm-text-field_textarea":"textarea"===i,"vm-text-field_dark":w}),"data-replicated-value":n,children:[c&&Vr("div",{className:"vm-text-field__icon-start",children:c}),l&&Vr("div",{className:"vm-text-field__icon-end",children:l}),"textarea"===i?Vr("textarea",{className:E,disabled:f,ref:C,value:n,rows:1,inputMode:m,placeholder:u,autoCapitalize:"none",onInput:N,onKeyDown:S,onFocus:F,onBlur:M}):Vr("input",{className:E,disabled:f,ref:k,value:n,type:i,placeholder:u,inputMode:m,autoCapitalize:"none",onInput:N,onKeyDown:S,onFocus:F,onBlur:M}),t&&Vr("span",{className:"vm-text-field__label",children:t}),Vr("span",{className:"vm-text-field__error","data-show":!!a,children:a}),p&&!a&&Vr("span",{className:"vm-text-field__helper-text",children:p})]})},ka=function(e){var t=e.accountIds,n=Ar(),r=Xo().isMobile,i=Wr(),o=i.tenantId,a=i.serverUrl,u=Qr(),l=Fi(),c=Ft(ee(""),2),f=c[0],d=c[1],h=Ft(ee(!1),2),p=h[0],v=h[1],m=ie(null),g=ae((function(){if(!f)return t;try{var e=new RegExp(f,"i");return t.filter((function(t){return e.test(t)})).sort((function(t,n){var r,i;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(i=n.match(e))||void 0===i?void 0:i.index)||0)}))}catch(s){return[]}}),[f,t]),y=ae((function(){return t.length>1&&!0}),[t,a]),_=function(){v((function(e){return!e}))},b=function(){v(!1)},D=function(e){return function(){var t=e;if(u({type:"SET_TENANT_ID",payload:t}),a){var n=Er(a,t);if(n===a)return;u({type:"SET_SERVER",payload:n}),l({type:"RUN_QUERY"})}b()}};return ne((function(){var e=(a.match(/(\/select\/)(\d+|\d.+)(\/)(.+)/)||[])[2];o&&o!==e?D(o)():D(e)()}),[a]),y?Vr("div",{className:"vm-tenant-input",children:[Vr(ma,{title:"Define Tenant ID if you need request to another storage",children:Vr("div",{ref:m,children:r?Vr("div",{className:"vm-mobile-option",onClick:_,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(go,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Tenant ID"}),Vr("span",{className:"vm-mobile-option-text__value",children:o})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(fa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",fullWidth:!0,startIcon:Vr(go,{}),endIcon:Vr("div",{className:wo()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":p}),children:Vr(Wi,{})}),onClick:_,children:o})})}),Vr(da,{open:p,placement:"bottom-right",onClose:b,buttonRef:m,title:r?"Define Tenant ID":void 0,children:Vr("div",{className:wo()({"vm-list vm-tenant-input-list":!0,"vm-list vm-tenant-input-list_mobile":r}),children:[Vr("div",{className:"vm-tenant-input-list__search",children:Vr(xa,{autofocus:!0,label:"Search",value:f,onChange:d,type:"search"})}),g.map((function(e){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":e===o}),onClick:D(e),children:e},e)}))]})})]}):null};var Ca,Aa=function(e){var t=ie();return ne((function(){t.current=e}),[e]),t.current},Ea=function(){var e=Ar(),t=Xo().isMobile,n=$o().customStep,r=Ni().period.step,i=Ho(),o=Ni().period,a=Aa(o.end-o.start),u=Ft(ee(!1),2),l=u[0],c=u[1],s=Ft(ee(n||r),2),f=s[0],d=s[1],h=Ft(ee(""),2),p=h[0],v=h[1],m=ie(null),g=function(){c((function(e){return!e}))},y=function(){c(!1)},_=function(e){var t=e||f||r||"1s",n=(t.match(/[a-zA-Z]+/g)||[]).length?t:"".concat(t,"s");i({type:"SET_CUSTOM_STEP",payload:n}),d(n),v("")},b=function(e){var t=e.match(/[-+]?([0-9]*\.[0-9]+|[0-9]+)/g)||[],n=e.match(/[a-zA-Z]+/g)||[],r=t.length&&t.every((function(e){return parseFloat(e)>0})),i=n.every((function(e){return ii.find((function(t){return t.short===e}))})),o=r&&i;d(e),v(o?"":_r.validStep)};return ne((function(){n&&_(n)}),[n]),ne((function(){!n&&r&&_(r)}),[r]),ne((function(){o.end-o.start!==a&&a&&r&&_(r)}),[o,a,r]),Vr("div",{className:"vm-step-control",ref:m,children:[t?Vr("div",{className:"vm-mobile-option",onClick:g,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(ho,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Step"}),Vr("span",{className:"vm-mobile-option-text__value",children:f})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Query resolution step width",children:Vr(fa,{className:e?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(ho,{}),onClick:g,children:Vr("p",{children:["STEP",Vr("p",{className:"vm-step-control__value",children:f})]})})}),Vr(da,{open:l,placement:"bottom-right",onClose:y,buttonRef:m,title:t?"Query resolution step width":void 0,children:Vr("div",{className:wo()({"vm-step-control-popper":!0,"vm-step-control-popper_mobile":t}),children:[Vr(xa,{autofocus:!0,label:"Step value",value:f,error:p,onChange:b,onEnter:function(){_(),y()},onFocus:function(){document.activeElement instanceof HTMLInputElement&&document.activeElement.select()},onBlur:_,endIcon:Vr(ma,{title:"Set default step value: ".concat(r),children:Vr(fa,{size:"small",variant:"text",color:"primary",startIcon:Vr($i,{}),onClick:function(){var e=r||"1s";b(e),_(e)}})})}),Vr("div",{className:"vm-step-control-popper-info",children:[Vr("code",{children:"step"})," - the ",Vr("a",{className:"vm-link vm-link_colored",href:"https://prometheus.io/docs/prometheus/latest/querying/basics/#time-durations",target:"_blank",rel:"noreferrer",children:"interval"}),"between datapoints, which must be returned from the range query. The ",Vr("code",{children:"query"})," is executed at",Vr("code",{children:"start"}),", ",Vr("code",{children:"start+step"}),", ",Vr("code",{children:"start+2*step"}),", \u2026, ",Vr("code",{children:"end"})," timestamps.",Vr("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/keyConcepts.html#range-query",target:"_blank",rel:"help noreferrer",children:"Read more about Range query"})]})]})})]})},Sa=function(e){var t=e.relativeTime,n=e.setDuration,r=Xo().isMobile;return Vr("div",{className:wo()({"vm-time-duration":!0,"vm-time-duration_mobile":r}),children:pi.map((function(e){var i,o=e.id,a=e.duration,u=e.until,l=e.title;return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":o===t}),onClick:(i={duration:a,until:u(),id:o},function(){n(i)}),children:l||a},o)}))})},Na=function(e){var t=e.viewDate,n=e.displayYears,r=e.onChangeViewDate;return Vr("div",{className:"vm-calendar-header",children:[Vr("div",{className:"vm-calendar-header-left",onClick:e.toggleDisplayYears,children:[Vr("span",{className:"vm-calendar-header-left__date",children:t.format("MMMM YYYY")}),Vr("div",{className:"vm-calendar-header-left__select-year",children:Vr(Qi,{})})]}),!n&&Vr("div",{className:"vm-calendar-header-right",children:[Vr("div",{className:"vm-calendar-header-right__prev",onClick:function(){r(t.subtract(1,"month"))},children:Vr(Wi,{})}),Vr("div",{className:"vm-calendar-header-right__next",onClick:function(){r(t.add(1,"month"))},children:Vr(Wi,{})})]})]})},Fa=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],Ma=function(e){var t=e.viewDate,n=e.selectDate,r=e.onChangeSelectDate,i=_t()().tz().startOf("day"),o=ae((function(){var e=new Array(42).fill(null),n=t.startOf("month"),r=t.endOf("month").diff(n,"day")+1,i=new Array(r).fill(n).map((function(e,t){return e.add(t,"day")})),o=n.day();return e.splice.apply(e,[o,r].concat(Ot(i))),e}),[t]),a=function(e){return function(){e&&r(e)}};return Vr("div",{className:"vm-calendar-body",children:[Fa.map((function(e){return Vr("div",{className:"vm-calendar-body-cell vm-calendar-body-cell_weekday",children:e[0]},e)})),o.map((function(e,t){return Vr("div",{className:wo()({"vm-calendar-body-cell":!0,"vm-calendar-body-cell_day":!0,"vm-calendar-body-cell_day_empty":!e,"vm-calendar-body-cell_day_active":(e&&e.toISOString())===n.startOf("day").toISOString(),"vm-calendar-body-cell_day_today":(e&&e.toISOString())===i.toISOString()}),onClick:a(e),children:e&&e.format("D")},e?e.toISOString():t)}))]})},Ta=function(e){var t=e.viewDate,n=e.onChangeViewDate,r=ae((function(){return t.format("YYYY")}),[t]),i=ae((function(){var e=_t()().subtract(103,"year");return new Array(206).fill(e).map((function(e,t){return e.add(t,"year")}))}),[t]);ne((function(){var e=document.getElementById("vm-calendar-year-".concat(r));e&&e.scrollIntoView({block:"center"})}),[]);return Vr("div",{className:"vm-calendar-years",children:i.map((function(e){return Vr("div",{className:wo()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("YYYY")===r}),id:"vm-calendar-year-".concat(e.format("YYYY")),onClick:(t=e,function(){n(t)}),children:e.format("YYYY")},e.format("YYYY"));var t}))})};!function(e){e[e.hour=0]="hour",e[e.minutes=1]="minutes",e[e.seconds=2]="seconds"}(Ca||(Ca={}));var Oa=function(e){var t=e.selectDate,n=e.onChangeTime,r=e.onClose,i=Wr().isDarkTheme,o=Ft(ee(Ca.hour),2),a=o[0],u=o[1],l=Ft(ee(t.format("HH")),2),c=l[0],s=l[1],f=Ft(ee(t.format("mm")),2),d=f[0],h=f[1],p=Ft(ee(t.format("ss")),2),v=p[0],m=p[1],g=ae((function(){return a===Ca.hour?new Array(24).fill("00").map((function(e,t){return{value:t,degrees:t/12*360,offset:0===t||t>12,title:t?"".concat(t):e}})):new Array(60).fill("00").map((function(e,t){return{value:t,degrees:t/60*360,offset:!1,title:t?"".concat(t):e}}))}),[a,c,d,v]),y=ae((function(){switch(a){case Ca.hour:return+c/12*360;case Ca.minutes:return+d/60*360;case Ca.seconds:return+v/60*360}}),[a,c,d,v]),_=ie(null),b=ie(null),D=ie(null),w=function(e){return function(t){!function(e,t){t.target.select(),u(e)}(e,t)}};return ne((function(){n("".concat(c,":").concat(d,":").concat(v))}),[c,d,v]),ne((function(){s(t.format("HH")),h(t.format("mm")),m(t.format("ss"))}),[t]),ne((function(){_.current&&_.current.focus()}),[]),Vr("div",{className:"vm-calendar-time-picker",children:[Vr("div",{className:"vm-calendar-time-picker-clock",children:[Vr("div",{className:wo()({"vm-calendar-time-picker-clock__arrow":!0,"vm-calendar-time-picker-clock__arrow_offset":a===Ca.hour&&("00"===c||+c>12)}),style:{transform:"rotate(".concat(y,"deg)")}}),g.map((function(e){return Vr("div",{className:wo()({"vm-calendar-time-picker-clock__time":!0,"vm-calendar-time-picker-clock__time_offset":e.offset,"vm-calendar-time-picker-clock__time_hide":g.length>24&&e.value%5}),style:{transform:"rotate(".concat(e.degrees,"deg)")},onClick:(t=e.value,function(){var e=String(t);switch(a){case Ca.hour:s(e),b.current&&b.current.focus();break;case Ca.minutes:h(e),D.current&&D.current.focus();break;case Ca.seconds:m(e),r()}}),children:Vr("span",{style:{transform:"rotate(-".concat(e.degrees,"deg)")},children:e.title})},e.value);var t}))]}),Vr("div",{className:wo()({"vm-calendar-time-picker-fields":!0,"vm-calendar-time-picker-fields_dark":i}),children:[Vr("input",{className:"vm-calendar-time-picker-fields__input",value:c,onChange:function(e){var t=e.target,n=t.value,r=+n>23?"23":n;t.value=r,s(r),n.length>1&&b.current&&b.current.focus()},onFocus:w(Ca.hour),ref:_,type:"number",min:0,max:24}),Vr("span",{children:":"}),Vr("input",{className:"vm-calendar-time-picker-fields__input",value:d,onChange:function(e){var t=e.target,n=t.value,r=+n>59?"59":n;t.value=r,h(r),n.length>1&&D.current&&D.current.focus()},onFocus:w(Ca.minutes),ref:b,type:"number",min:0,max:60}),Vr("span",{children:":"}),Vr("input",{className:"vm-calendar-time-picker-fields__input",value:v,onChange:function(e){var t=e.target,n=t.value,i=+n>59?"59":n;t.value=i,m(i),n.length>1&&D.current&&r()},onFocus:w(Ca.seconds),ref:D,type:"number",min:0,max:60})]})]})},Ba=[{value:"date",icon:Vr(Ji,{})},{value:"time",icon:Vr(Gi,{})}],La=function(e){var t=e.date,n=e.timepicker,r=void 0!==n&&n,i=e.format,o=void 0===i?Zr:i,a=e.onChange,u=e.onClose,l=Ft(ee(!1),2),c=l[0],s=l[1],f=Ft(ee(_t().tz(t)),2),d=f[0],h=f[1],p=Ft(ee(_t().tz(t)),2),v=p[0],m=p[1],y=Ft(ee(Ba[0].value),2),_=y[0],b=y[1],D=Xo().isMobile,w=function(e){h(e),s(!1)};return ne((function(){v.format()!==_t().tz(t).format()&&a(v.format(o))}),[v]),Vr("div",{className:wo()({"vm-calendar":!0,"vm-calendar_mobile":D}),children:["date"===_&&Vr(Na,{viewDate:d,onChangeViewDate:w,toggleDisplayYears:function(){s((function(e){return!e}))},displayYears:c}),"date"===_&&Vr(g,{children:[!c&&Vr(Ma,{viewDate:d,selectDate:v,onChangeSelectDate:function(e){m(e),r&&b("time")}}),c&&Vr(Ta,{viewDate:d,onChangeViewDate:w})]}),"time"===_&&Vr(Oa,{selectDate:v,onChangeTime:function(e){var t=Ft(e.split(":"),3),n=t[0],r=t[1],i=t[2];m((function(e){return e.set("hour",+n).set("minute",+r).set("second",+i)}))},onClose:function(){u&&u()}}),r&&Vr("div",{className:"vm-calendar__tabs",children:Vr(Eo,{activeItem:_,items:Ba,onChange:function(e){b(e)},indicatorPlacement:"top"})})]})},Ia=Ae((function(e,t){var n=e.date,r=e.targetRef,i=e.format,o=void 0===i?Zr:i,a=e.timepicker,u=e.onChange,l=e.label,c=Ft(ee(!1),2),s=c[0],f=c[1],d=ae((function(){return n?_t().tz(n):_t()().tz()}),[n]),h=Xo().isMobile,p=function(){f((function(e){return!e}))},v=function(){f(!1)},m=function(e){"Escape"!==e.key&&"Enter"!==e.key||v()};return ne((function(){var e;return null===(e=r.current)||void 0===e||e.addEventListener("click",p),function(){var e;null===(e=r.current)||void 0===e||e.removeEventListener("click",p)}}),[r]),ne((function(){return window.addEventListener("keyup",m),function(){window.removeEventListener("keyup",m)}}),[]),Vr(g,{children:Vr(da,{open:s,buttonRef:r,placement:"bottom-right",onClose:v,title:h?l:void 0,children:Vr("div",{ref:t,children:Vr(La,{date:d,format:o,timepicker:a,onChange:function(e){a||v(),u(e)},onClose:v})})})})})),Pa=Ia,Ra=function(){var e=Xo().isMobile,t=Wr().isDarkTheme,n=ie(null),r=bo(document.body),i=ae((function(){return r.width>1280}),[r]),o=Ft(ee(),2),a=o[0],u=o[1],l=Ft(ee(),2),c=l[0],s=l[1],f=ae((function(){return _t().tz(c).format(Zr)}),[c]),d=ae((function(){return _t().tz(a).format(Zr)}),[a]),h=Ni(),p=h.period,v=p.end,m=p.start,y=h.relativeTime,_=h.timezone,b=h.duration,D=Fi(),w=Ar(),x=ae((function(){return{region:_,utc:mi(_)}}),[_]);ne((function(){u(fi(hi(v)))}),[_,v]),ne((function(){s(fi(hi(m)))}),[_,m]);var k=function(e){var t=e.duration,n=e.until,r=e.id;D({type:"SET_RELATIVE_TIME",payload:{duration:t,until:n,id:r}}),O(!1)},C=ae((function(){return{start:_t().tz(hi(m)).format(Zr),end:_t().tz(hi(v)).format(Zr)}}),[m,v,_]),A=ae((function(){return y&&"none"!==y?y.replace(/_/g," "):"".concat(C.start," - ").concat(C.end)}),[y,C]),E=ie(null),S=ie(null),N=ie(null),F=ie(null),M=Ft(ee(!1),2),T=M[0],O=M[1],B=ie(null),L=function(){O((function(e){return!e}))},I=function(){O(!1)};return ne((function(){var e=vi({relativeTimeId:y,defaultDuration:b,defaultEndInput:hi(v)});k({id:e.relativeTimeId,duration:e.duration,until:e.endInput})}),[_]),sa(n,(function(t){var n,r;if(!e){var i=t.target,o=(null===E||void 0===E?void 0:E.current)&&E.current.contains(i),a=(null===S||void 0===S?void 0:S.current)&&S.current.contains(i),u=(null===N||void 0===N?void 0:N.current)&&(null===N||void 0===N||null===(n=N.current)||void 0===n?void 0:n.contains(i)),l=(null===F||void 0===F?void 0:F.current)&&(null===F||void 0===F||null===(r=F.current)||void 0===r?void 0:r.contains(i));o||a||u||l||I()}})),Vr(g,{children:[Vr("div",{ref:B,children:e?Vr("div",{className:"vm-mobile-option",onClick:L,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(Gi,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Time range"}),Vr("span",{className:"vm-mobile-option-text__value",children:A})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:i?"Time range controls":A,children:Vr(fa,{className:w?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Gi,{}),onClick:L,children:i&&Vr("span",{children:A})})})}),Vr(da,{open:T,buttonRef:B,placement:"bottom-right",onClose:I,clickOutside:!1,title:e?"Time range controls":"",children:Vr("div",{className:wo()({"vm-time-selector":!0,"vm-time-selector_mobile":e}),ref:n,children:[Vr("div",{className:"vm-time-selector-left",children:[Vr("div",{className:wo()({"vm-time-selector-left-inputs":!0,"vm-time-selector-left-inputs_dark":t}),children:[Vr("div",{className:"vm-time-selector-left-inputs__date",ref:E,children:[Vr("label",{children:"From:"}),Vr("span",{children:f}),Vr(Ji,{}),Vr(Pa,{label:"Date From",ref:N,date:c||"",onChange:function(e){return s(e)},targetRef:E,timepicker:!0})]}),Vr("div",{className:"vm-time-selector-left-inputs__date",ref:S,children:[Vr("label",{children:"To:"}),Vr("span",{children:d}),Vr(Ji,{}),Vr(Pa,{label:"Date To",ref:F,date:a||"",onChange:function(e){return u(e)},targetRef:S,timepicker:!0})]})]}),Vr("div",{className:"vm-time-selector-left-timezone",children:[Vr("div",{className:"vm-time-selector-left-timezone__title",children:x.region}),Vr("div",{className:"vm-time-selector-left-timezone__utc",children:x.utc})]}),Vr(fa,{variant:"text",startIcon:Vr(Zi,{}),onClick:function(){return D({type:"RUN_QUERY_TO_NOW"})},children:"switch to now"}),Vr("div",{className:"vm-time-selector-left__controls",children:[Vr(fa,{color:"error",variant:"outlined",onClick:function(){u(fi(hi(v))),s(fi(hi(m))),O(!1)},children:"Cancel"}),Vr(fa,{color:"primary",onClick:function(){return c&&a&&D({type:"SET_PERIOD",payload:{from:_t().tz(c).toDate(),to:_t().tz(a).toDate()}}),void O(!1)},children:"Apply"})]})]}),Vr(Sa,{relativeTime:y||"",setDuration:k})]})})]})},za=function(){var e=Xo().isMobile,t=Ar(),n=ie(null),r=qo().date,i=Wo(),o=ae((function(){return _t().tz(r).format(Jr)}),[r]);return Vr("div",{children:[Vr("div",{ref:n,children:e?Vr("div",{className:"vm-mobile-option",children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(Ji,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Date control"}),Vr("span",{className:"vm-mobile-option-text__value",children:o})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Date control",children:Vr(fa,{className:t?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Ji,{}),children:o})})}),Vr(Pa,{label:"Date control",date:r||"",format:Jr,onChange:function(e){i({type:"SET_DATE",payload:e})},targetRef:n})]})},ja=[{seconds:0,title:"Off"},{seconds:1,title:"1s"},{seconds:2,title:"2s"},{seconds:5,title:"5s"},{seconds:10,title:"10s"},{seconds:30,title:"30s"},{seconds:60,title:"1m"},{seconds:300,title:"5m"},{seconds:900,title:"15m"},{seconds:1800,title:"30m"},{seconds:3600,title:"1h"},{seconds:7200,title:"2h"}],$a=function(){var e=Xo().isMobile,t=Fi(),n=Ar(),r=Ft(ee(!1),2),i=r[0],o=r[1],a=Ft(ee(ja[0]),2),u=a[0],l=a[1];ne((function(){var e,n=u.seconds;return i?e=setInterval((function(){t({type:"RUN_QUERY"})}),1e3*n):l(ja[0]),function(){e&&clearInterval(e)}}),[u,i]);var c=Ft(ee(!1),2),s=c[0],f=c[1],d=ie(null),h=function(){f((function(e){return!e}))},p=function(e){return function(){!function(e){(i&&!e.seconds||!i&&e.seconds)&&o((function(e){return!e})),l(e),f(!1)}(e)}};return Vr(g,{children:[Vr("div",{className:"vm-execution-controls",children:Vr("div",{className:wo()({"vm-execution-controls-buttons":!0,"vm-execution-controls-buttons_mobile":e,"vm-header-button":!n}),children:[!e&&Vr(ma,{title:"Refresh dashboard",children:Vr(fa,{variant:"contained",color:"primary",onClick:function(){t({type:"RUN_QUERY"})},startIcon:Vr(qi,{})})}),e?Vr("div",{className:"vm-mobile-option",onClick:h,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr($i,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Auto-refresh"}),Vr("span",{className:"vm-mobile-option-text__value",children:u.title})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Auto-refresh control",children:Vr("div",{ref:d,children:Vr(fa,{variant:"contained",color:"primary",fullWidth:!0,endIcon:Vr("div",{className:wo()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":s}),children:Vr(Wi,{})}),onClick:h,children:u.title})})})]})}),Vr(da,{open:s,placement:"bottom-right",onClose:function(){f(!1)},buttonRef:d,title:e?"Auto-refresh duration":void 0,children:Vr("div",{className:wo()({"vm-execution-controls-list":!0,"vm-execution-controls-list_mobile":e}),children:ja.map((function(t){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":e,"vm-list-item_active":t.seconds===u.seconds}),onClick:p(t),children:t.title},t.seconds)}))})})]})},Ha=function(e){var t;try{t=new URL(e)}catch(_){return!1}return"http:"===t.protocol||"https:"===t.protocol},Ua=function(e){var t=e.serverUrl,n=e.onChange,r=e.onEnter,i=e.onBlur,o=Ft(ee(""),2),a=o[0],u=o[1];return Vr(xa,{autofocus:!0,label:"Server URL",value:t,error:a,onChange:function(e){var t=e||"";n(t),u(""),t||u(_r.emptyServer),Ha(t)||u(_r.validServer)},onEnter:r,onBlur:i,inputmode:"url"})},Ya=[{label:"Graph",type:"chart"},{label:"JSON",type:"code"},{label:"Table",type:"table"}],Va=function(e){var t=e.limits,n=e.onChange,r=e.onEnter,i=Xo().isMobile,o=Ft(ee({table:"",chart:"",code:""}),2),a=o[0],u=o[1],l=function(e){return function(r){!function(e,r){var i=e||"";u((function(e){return yr(yr({},e),{},mr({},r,+i<0?_r.positiveNumber:""))})),n(yr(yr({},t),{},mr({},r,i||1/0)))}(r,e)}};return Vr("div",{className:"vm-limits-configurator",children:[Vr("div",{className:"vm-server-configurator__title",children:["Series limits by tabs",Vr(ma,{title:"To disable limits set to 0",children:Vr(fa,{variant:"text",color:"primary",size:"small",startIcon:Vr(Hi,{})})}),Vr("div",{className:"vm-limits-configurator-title__reset",children:Vr(fa,{variant:"text",color:"primary",size:"small",startIcon:Vr($i,{}),onClick:function(){n(Tr)},children:"Reset"})})]}),Vr("div",{className:wo()({"vm-limits-configurator__inputs":!0,"vm-limits-configurator__inputs_mobile":i}),children:Ya.map((function(e){return Vr("div",{children:Vr(xa,{label:e.label,value:t[e.type],error:a[e.type],onChange:l(e.type),onEnter:r,type:"number"})},e.type)}))})]})},qa=function(e){var t=e.defaultExpanded,n=void 0!==t&&t,r=e.onChange,i=e.title,o=e.children,a=Ft(ee(n),2),u=a[0],l=a[1];return ne((function(){r&&r(u)}),[u]),Vr(g,{children:[Vr("header",{className:"vm-accordion-header ".concat(u&&"vm-accordion-header_open"),onClick:function(){l((function(e){return!e}))},children:[i,Vr("div",{className:"vm-accordion-header__arrow ".concat(u&&"vm-accordion-header__arrow_open"),children:Vr(Wi,{})})]}),u&&Vr("section",{className:"vm-accordion-section",children:o},"content")]})},Wa=function(e){var t=e.timezoneState,n=e.onChange,r=Xo().isMobile,i=gi(),o=Ft(ee(!1),2),a=o[0],u=o[1],l=Ft(ee(""),2),c=l[0],f=l[1],d=ie(null),h=ae((function(){if(!c)return i;try{return gi(c)}catch(s){return{}}}),[c,i]),p=ae((function(){return Object.keys(h)}),[h]),v=ae((function(){return{region:_t().tz.guess(),utc:mi(_t().tz.guess())}}),[]),m=ae((function(){return{region:t,utc:mi(t)}}),[t]),g=function(){u(!1)},y=function(e){return function(){!function(e){n(e.region),f(""),g()}(e)}};return Vr("div",{className:"vm-timezones",children:[Vr("div",{className:"vm-server-configurator__title",children:"Time zone"}),Vr("div",{className:"vm-timezones-item vm-timezones-item_selected",onClick:function(){u((function(e){return!e}))},ref:d,children:[Vr("div",{className:"vm-timezones-item__title",children:m.region}),Vr("div",{className:"vm-timezones-item__utc",children:m.utc}),Vr("div",{className:wo()({"vm-timezones-item__icon":!0,"vm-timezones-item__icon_open":a}),children:Vr(Qi,{})})]}),Vr(da,{open:a,buttonRef:d,placement:"bottom-left",onClose:g,fullWidth:!0,title:r?"Time zone":void 0,children:Vr("div",{className:wo()({"vm-timezones-list":!0,"vm-timezones-list_mobile":r}),children:[Vr("div",{className:"vm-timezones-list-header",children:[Vr("div",{className:"vm-timezones-list-header__search",children:Vr(xa,{autofocus:!0,label:"Search",value:c,onChange:function(e){f(e)}})}),Vr("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:y(v),children:[Vr("div",{className:"vm-timezones-item__title",children:["Browser Time (",v.region,")"]}),Vr("div",{className:"vm-timezones-item__utc",children:v.utc})]})]}),p.map((function(e){return Vr("div",{className:"vm-timezones-list-group",children:Vr(qa,{defaultExpanded:!0,title:Vr("div",{className:"vm-timezones-list-group__title",children:e}),children:Vr("div",{className:"vm-timezones-list-group-options",children:h[e]&&h[e].map((function(e){return Vr("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:y(e),children:[Vr("div",{className:"vm-timezones-item__title",children:e.region}),Vr("div",{className:"vm-timezones-item__utc",children:e.utc})]},e.search)}))})})},e)}))]})})]})},Qa=function(e){var t=e.options,n=e.value,r=e.label,i=e.onChange,o=ie(null),a=Ft(ee({width:"0px",left:"0px",borderRadius:"0px"}),2),u=a[0],l=a[1],c=function(e){return function(){i(e)}};return ne((function(){if(o.current){var e=t.findIndex((function(e){return e.value===n})),r=o.current.getBoundingClientRect().width,i=e*r,a="0";0===e&&(a="16px 0 0 16px"),e===t.length-1&&(a="10px",i-=1,a="0 16px 16px 0"),0!==e&&e!==t.length-1&&(r+=1,i-=1),l({width:"".concat(r,"px"),left:"".concat(i,"px"),borderRadius:a})}else l({width:"0px",left:"0px",borderRadius:"0px"})}),[o,n,t]),Vr("div",{className:"vm-toggles",children:[r&&Vr("label",{className:"vm-toggles__label",children:r}),Vr("div",{className:"vm-toggles-group",style:{gridTemplateColumns:"repeat(".concat(t.length,", 1fr)")},children:[u.borderRadius&&Vr("div",{className:"vm-toggles-group__highlight",style:u}),t.map((function(e,t){return Vr("div",{className:wo()({"vm-toggles-group-item":!0,"vm-toggles-group-item_first":0===t,"vm-toggles-group-item_active":e.value===n,"vm-toggles-group-item_icon":e.icon&&e.title}),onClick:c(e.value),ref:e.value===n?o:null,children:[e.icon,e.title]},e.value)}))]})]})},Ga=Object.values(br).map((function(e){return{title:e,value:e}})),Ja=function(){var e=Xo().isMobile,t=Wr().theme,n=Qr();return Vr("div",{className:wo()({"vm-theme-control":!0,"vm-theme-control_mobile":e}),children:[Vr("div",{className:"vm-server-configurator__title",children:"Theme preferences"}),Vr("div",{className:"vm-theme-control__toggle",children:Vr(Qa,{options:Ga,value:t,onChange:function(e){n({type:"SET_THEME",payload:e})}})},"".concat(e))]})},Za="Settings",Ka=function(){var e=Xo().isMobile,t=Ar(),n=Wr().serverUrl,r=Ni().timezone,i=Io().seriesLimits,o=Qr(),a=Fi(),u=Po(),l=Ft(ee(n),2),c=l[0],s=l[1],f=Ft(ee(i),2),d=f[0],h=f[1],p=Ft(ee(r),2),v=p[0],m=p[1],y=Ft(ee(!1),2),_=y[0],b=y[1],D=function(){return b(!0)},w=function(){o({type:"SET_SERVER",payload:c}),a({type:"SET_TIMEZONE",payload:v}),u({type:"SET_SERIES_LIMITS",payload:d})};return ne((function(){n!==c&&s(n)}),[n]),Vr(g,{children:[e?Vr("div",{className:"vm-mobile-option",onClick:D,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(zi,{})}),Vr("div",{className:"vm-mobile-option-text",children:Vr("span",{className:"vm-mobile-option-text__label",children:Za})}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:Za,children:Vr(fa,{className:wo()({"vm-header-button":!t}),variant:"contained",color:"primary",startIcon:Vr(zi,{}),onClick:D})}),_&&Vr(va,{title:Za,onClose:function(){return b(!1)},children:Vr("div",{className:wo()({"vm-server-configurator":!0,"vm-server-configurator_mobile":e}),children:[!t&&Vr("div",{className:"vm-server-configurator__input",children:Vr(Ua,{serverUrl:c,onChange:s,onEnter:w,onBlur:w})}),Vr("div",{className:"vm-server-configurator__input",children:Vr(Va,{limits:d,onChange:h,onEnter:w})}),Vr("div",{className:"vm-server-configurator__input",children:Vr(Wa,{timezoneState:v,onChange:m})}),!t&&Vr("div",{className:"vm-server-configurator__input",children:Vr(Ja,{})})]})})]})};function Xa(){Xa=function(){return e};var e={},t=Object.prototype,n=t.hasOwnProperty,r=Object.defineProperty||function(e,t,n){e[t]=n.value},i="function"==typeof Symbol?Symbol:{},o=i.iterator||"@@iterator",a=i.asyncIterator||"@@asyncIterator",u=i.toStringTag||"@@toStringTag";function l(e,t,n){return Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}),e[t]}try{l({},"")}catch(S){l=function(e,t,n){return e[t]=n}}function c(e,t,n,i){var o=t&&t.prototype instanceof d?t:d,a=Object.create(o.prototype),u=new C(i||[]);return r(a,"_invoke",{value:D(e,n,u)}),a}function s(e,t,n){try{return{type:"normal",arg:e.call(t,n)}}catch(S){return{type:"throw",arg:S}}}e.wrap=c;var f={};function d(){}function h(){}function p(){}var v={};l(v,o,(function(){return this}));var m=Object.getPrototypeOf,g=m&&m(m(A([])));g&&g!==t&&n.call(g,o)&&(v=g);var y=p.prototype=d.prototype=Object.create(v);function _(e){["next","throw","return"].forEach((function(t){l(e,t,(function(e){return this._invoke(t,e)}))}))}function b(e,t){function i(r,o,a,u){var l=s(e[r],e,o);if("throw"!==l.type){var c=l.arg,f=c.value;return f&&"object"==Lt(f)&&n.call(f,"__await")?t.resolve(f.__await).then((function(e){i("next",e,a,u)}),(function(e){i("throw",e,a,u)})):t.resolve(f).then((function(e){c.value=e,a(c)}),(function(e){return i("throw",e,a,u)}))}u(l.arg)}var o;r(this,"_invoke",{value:function(e,n){function r(){return new t((function(t,r){i(e,n,t,r)}))}return o=o?o.then(r,r):r()}})}function D(e,t,n){var r="suspendedStart";return function(i,o){if("executing"===r)throw new Error("Generator is already running");if("completed"===r){if("throw"===i)throw o;return E()}for(n.method=i,n.arg=o;;){var a=n.delegate;if(a){var u=w(a,n);if(u){if(u===f)continue;return u}}if("next"===n.method)n.sent=n._sent=n.arg;else if("throw"===n.method){if("suspendedStart"===r)throw r="completed",n.arg;n.dispatchException(n.arg)}else"return"===n.method&&n.abrupt("return",n.arg);r="executing";var l=s(e,t,n);if("normal"===l.type){if(r=n.done?"completed":"suspendedYield",l.arg===f)continue;return{value:l.arg,done:n.done}}"throw"===l.type&&(r="completed",n.method="throw",n.arg=l.arg)}}}function w(e,t){var n=t.method,r=e.iterator[n];if(void 0===r)return t.delegate=null,"throw"===n&&e.iterator.return&&(t.method="return",t.arg=void 0,w(e,t),"throw"===t.method)||"return"!==n&&(t.method="throw",t.arg=new TypeError("The iterator does not provide a '"+n+"' method")),f;var i=s(r,e.iterator,t.arg);if("throw"===i.type)return t.method="throw",t.arg=i.arg,t.delegate=null,f;var o=i.arg;return o?o.done?(t[e.resultName]=o.value,t.next=e.nextLoc,"return"!==t.method&&(t.method="next",t.arg=void 0),t.delegate=null,f):o:(t.method="throw",t.arg=new TypeError("iterator result is not an object"),t.delegate=null,f)}function x(e){var t={tryLoc:e[0]};1 in e&&(t.catchLoc=e[1]),2 in e&&(t.finallyLoc=e[2],t.afterLoc=e[3]),this.tryEntries.push(t)}function k(e){var t=e.completion||{};t.type="normal",delete t.arg,e.completion=t}function C(e){this.tryEntries=[{tryLoc:"root"}],e.forEach(x,this),this.reset(!0)}function A(e){if(e){var t=e[o];if(t)return t.call(e);if("function"==typeof e.next)return e;if(!isNaN(e.length)){var r=-1,i=function t(){for(;++r=0;--i){var o=this.tryEntries[i],a=o.completion;if("root"===o.tryLoc)return r("end");if(o.tryLoc<=this.prev){var u=n.call(o,"catchLoc"),l=n.call(o,"finallyLoc");if(u&&l){if(this.prev=0;--r){var i=this.tryEntries[r];if(i.tryLoc<=this.prev&&n.call(i,"finallyLoc")&&this.prev=0;--t){var n=this.tryEntries[t];if(n.finallyLoc===e)return this.complete(n.completion,n.afterLoc),k(n),f}},catch:function(e){for(var t=this.tryEntries.length-1;t>=0;--t){var n=this.tryEntries[t];if(n.tryLoc===e){var r=n.completion;if("throw"===r.type){var i=r.arg;k(n)}return i}}throw new Error("illegal catch attempt")},delegateYield:function(e,t,n){return this.delegate={iterator:A(e),resultName:t,nextLoc:n},"next"===this.method&&(this.arg=void 0),f}},e}function eu(e,t,n,r,i,o,a){try{var u=e[o](a),l=u.value}catch(c){return void n(c)}u.done?t(l):Promise.resolve(l).then(r,i)}function tu(e){return function(){var t=this,n=arguments;return new Promise((function(r,i){var o=e.apply(t,n);function a(e){eu(o,r,i,a,u,"next",e)}function u(e){eu(o,r,i,a,u,"throw",e)}a(void 0)}))}}var nu,ru,iu=function(e){var t=e.displaySidebar,n=e.isMobile,r=e.headerSetup,i=e.accountIds;return Vr("div",{className:wo()({"vm-header-controls":!0,"vm-header-controls_mobile":n}),children:[(null===r||void 0===r?void 0:r.tenant)&&Vr(ka,{accountIds:i||[]}),(null===r||void 0===r?void 0:r.stepControl)&&Vr(Ea,{}),(null===r||void 0===r?void 0:r.timeSelector)&&Vr(Ra,{}),(null===r||void 0===r?void 0:r.cardinalityDatePicker)&&Vr(za,{}),(null===r||void 0===r?void 0:r.executionControls)&&Vr($a,{}),Vr(Ka,{}),!t&&Vr(ba,{})]})},ou=function(e){var t=Ar(),n=Ft(ee(!1),2),r=n[0],i=n[1],o=jn().pathname,a=function(){var e=Cr().useTenantID,t=Wr().serverUrl,n=Ft(ee(!1),2),r=n[0],i=n[1],o=Ft(ee(),2),a=o[0],u=o[1],l=Ft(ee([]),2),c=l[0],s=l[1],f=ae((function(){return"".concat(t.replace(/^(.+)(\/select.+)/,"$1"),"/admin/tenants")}),[t]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return i(!0),e.prev=1,e.next=4,fetch(f);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],s(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?u(void 0):u("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&u("".concat(e.t0.name,": ").concat(e.t0.message));case 16:i(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[f]),{accountIds:c,isLoading:r,error:a}}(),u=a.accountIds,l=ae((function(){return(xr[o]||{}).header||{}}),[o]);return e.isMobile?Vr(g,{children:[Vr("div",{children:Vr(fa,{className:wo()({"vm-header-button":!t}),startIcon:Vr(yo,{}),onClick:function(){i((function(e){return!e}))}})}),Vr(va,{title:"Controls",onClose:function(){i(!1)},isOpen:r,className:wo()({"vm-header-controls-modal":!0,"vm-header-controls-modal_open":r}),children:Vr(iu,yr(yr({},e),{},{accountIds:u,headerSetup:l}))})]}):Vr(iu,yr(yr({},e),{},{accountIds:u,headerSetup:l}))},au=function(){var e=Xo().isMobile,t=ae((function(){return window.innerWidth<1e3}),[bo(document.body)]),n=Wr().isDarkTheme,r=Ar(),i=ae((function(){return Rr(n?"color-background-block":"color-primary")}),[n]),o=ae((function(){var e=Cr().headerStyles,t=void 0===e?{}:e,n=t.background,o=void 0===n?r?"#FFF":i:n,a=t.color;return{background:o,color:void 0===a?r?i:"#FFF":a}}),[i]),a=o.background,u=o.color,l=$n(),c=function(){l({pathname:kr.home}),window.location.reload()};return Vr("header",{className:wo()({"vm-header":!0,"vm-header_app":r,"vm-header_dark":n,"vm-header_mobile":e}),style:{background:a,color:u},children:[t?Vr(wa,{background:a,color:u}):Vr(g,{children:[!r&&Vr("div",{className:"vm-header-logo",onClick:c,style:{color:u},children:Vr(Pi,{})}),Vr(pa,{color:u,background:a})]}),e&&Vr("div",{className:"vm-header-logo vm-header-logo_mobile",onClick:c,style:{color:u},children:Vr(Pi,{})}),Vr(ou,{displaySidebar:t,isMobile:e})]})},uu=function(){var e=Xo().isMobile,t="2019-".concat(_t()().format("YYYY"));return Vr("footer",{className:"vm-footer",children:[Vr("a",{className:"vm-link vm-footer__website",target:"_blank",href:"https://victoriametrics.com/",rel:"me noreferrer",children:[Vr(Ri,{}),"victoriametrics.com"]}),Vr("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/#vmui",rel:"help noreferrer",children:[Vr(po,{}),e?"Docs":"Documentation"]}),Vr("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://github.com/VictoriaMetrics/VictoriaMetrics/issues/new/choose",rel:"noreferrer",children:[Vr(vo,{}),e?"New issue":"Create an issue"]}),Vr("div",{className:"vm-footer__copyright",children:["\xa9 ",t," VictoriaMetrics"]})]})},lu=function(){var e=tu(Xa().mark((function e(t){var n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,fetch("./dashboards/".concat(t));case 2:return n=e.sent,e.next=5,n.json();case 5:return r=e.sent,e.abrupt("return",r);case 7:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}(),cu=function(){var e=Ar(),t=Wr().serverUrl,n=le(aa).dispatch,r=Ft(ee(!1),2),i=r[0],o=r[1],a=Ft(ee(""),2),u=a[0],l=a[1],c=Ft(ee([]),2),s=c[0],f=c[1],d=function(){var e=tu(Xa().mark((function e(){var t,n;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:if(null!==(t=window.__VMUI_PREDEFINED_DASHBOARDS__)&&void 0!==t&&t.length){e.next=3;break}return e.abrupt("return",[]);case 3:return e.next=5,Promise.all(t.map(function(){var e=tu(Xa().mark((function e(t){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.abrupt("return",lu(t));case 1:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}()));case 5:n=e.sent,f((function(e){return[].concat(Ot(n),Ot(e))}));case 7:case"end":return e.stop()}}),e)})));return function(){return e.apply(this,arguments)}}(),h=function(){var e=tu(Xa().mark((function e(){var n,r,i;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:if(t){e.next=2;break}return e.abrupt("return");case 2:return l(""),o(!0),e.prev=4,e.next=7,fetch("".concat(t,"/vmui/custom-dashboards"));case 7:return n=e.sent,e.next=10,n.json();case 10:if(r=e.sent,!n.ok){e.next=17;break}(i=r.dashboardsSettings)&&i.length>0&&f((function(e){return[].concat(Ot(e),Ot(i))})),o(!1),e.next=21;break;case 17:return e.next=19,d();case 19:l(r.error),o(!1);case 21:e.next=29;break;case 23:return e.prev=23,e.t0=e.catch(4),o(!1),e.t0 instanceof Error&&l("".concat(e.t0.name,": ").concat(e.t0.message)),e.next=29,d();case 29:case"end":return e.stop()}}),e,null,[[4,23]])})));return function(){return e.apply(this,arguments)}}();return ne((function(){e||(f([]),h())}),[t]),ne((function(){n({type:"SET_DASHBOARDS_SETTINGS",payload:s})}),[s]),ne((function(){n({type:"SET_DASHBOARDS_LOADING",payload:i})}),[i]),ne((function(){n({type:"SET_DASHBOARDS_ERROR",payload:u})}),[u]),{dashboardsSettings:s,isLoading:i,error:u}},su=function(){var e=Ar(),t=Xo().isMobile;cu();var n=jn().pathname;return ne((function(){var e,t="vmui",r=null===(e=xr[n])||void 0===e?void 0:e.title;document.title=r?"".concat(r," - ").concat(t):t}),[n]),Vr("section",{className:"vm-container",children:[Vr(au,{}),Vr("div",{className:wo()({"vm-container-body":!0,"vm-container-body_mobile":t,"vm-container-body_app":e}),children:Vr(Xn,{})}),!e&&Vr(uu,{})]})},fu="u-off",du="u-label",hu="width",pu="height",vu="top",mu="bottom",gu="left",yu="right",_u="#000",bu=_u+"0",Du="mousemove",wu="mousedown",xu="mouseup",ku="mouseenter",Cu="mouseleave",Au="dblclick",Eu="change",Su="dppxchange",Nu="undefined"!=typeof window,Fu=Nu?document:null,Mu=Nu?window:null,Tu=Nu?navigator:null;function Ou(e,t){if(null!=t){var n=e.classList;!n.contains(t)&&n.add(t)}}function Bu(e,t){var n=e.classList;n.contains(t)&&n.remove(t)}function Lu(e,t,n){e.style[t]=n+"px"}function Iu(e,t,n,r){var i=Fu.createElement(e);return null!=t&&Ou(i,t),null!=n&&n.insertBefore(i,r),i}function Pu(e,t){return Iu("div",e,t)}var Ru=new WeakMap;function zu(e,t,n,r,i){var o="translate("+t+"px,"+n+"px)";o!=Ru.get(e)&&(e.style.transform=o,Ru.set(e,o),t<0||n<0||t>r||n>i?Ou(e,fu):Bu(e,fu))}var ju=new WeakMap;function $u(e,t,n){var r=t+n;r!=ju.get(e)&&(ju.set(e,r),e.style.background=t,e.style.borderColor=n)}var Hu=new WeakMap;function Uu(e,t,n,r){var i=t+""+n;i!=Hu.get(e)&&(Hu.set(e,i),e.style.height=n+"px",e.style.width=t+"px",e.style.marginLeft=r?-t/2+"px":0,e.style.marginTop=r?-n/2+"px":0)}var Yu={passive:!0},Vu=yr(yr({},Yu),{},{capture:!0});function qu(e,t,n,r){t.addEventListener(e,n,r?Vu:Yu)}function Wu(e,t,n,r){t.removeEventListener(e,n,r?Vu:Yu)}function Qu(e,t,n,r){var i;n=n||0;for(var o=(r=r||t.length-1)<=2147483647;r-n>1;)t[i=o?n+r>>1:fl((n+r)/2)]=t&&i<=n;i+=r)if(null!=e[i])return i;return-1}function Ju(e,t,n,r){var i=Dl,o=-Dl;if(1==r)i=e[t],o=e[n];else if(-1==r)i=e[n],o=e[t];else for(var a=t;a<=n;a++)null!=e[a]&&(i=pl(i,e[a]),o=vl(o,e[a]));return[i,o]}function Zu(e,t,n){for(var r=Dl,i=-Dl,o=t;o<=n;o++)e[o]>0&&(r=pl(r,e[o]),i=vl(i,e[o]));return[r==Dl?1:r,i==-Dl?10:i]}function Ku(e,t,n,r){var i=gl(e),o=gl(t),a=10==n?yl:_l;e==t&&(-1==i?(e*=n,t/=n):(e/=n,t*=n));var u=1==o?hl:fl,l=(1==i?fl:hl)(a(sl(e))),c=u(a(sl(t))),s=ml(n,l),f=ml(n,c);return l<0&&(s=Ol(s,-l)),c<0&&(f=Ol(f,-c)),r?(e=s*i,t=f*o):(e=Tl(e,s),t=Ml(t,f)),[e,t]}function Xu(e,t,n,r){var i=Ku(e,t,n,r);return 0==e&&(i[0]=0),0==t&&(i[1]=0),i}Nu&&function e(){var t=devicePixelRatio;nu!=t&&(nu=t,ru&&Wu(Eu,ru,e),ru=matchMedia("(min-resolution: ".concat(nu-.001,"dppx) and (max-resolution: ").concat(nu+.001,"dppx)")),qu(Eu,ru,e),Mu.dispatchEvent(new CustomEvent(Su)))}();var el={mode:3,pad:.1},tl={pad:0,soft:null,mode:0},nl={min:tl,max:tl};function rl(e,t,n,r){return Ul(n)?ol(e,t,n):(tl.pad=n,tl.soft=r?0:null,tl.mode=r?3:0,ol(e,t,nl))}function il(e,t){return null==e?t:e}function ol(e,t,n){var r=n.min,i=n.max,o=il(r.pad,0),a=il(i.pad,0),u=il(r.hard,-Dl),l=il(i.hard,Dl),c=il(r.soft,Dl),s=il(i.soft,-Dl),f=il(r.mode,0),d=il(i.mode,0),h=t-e,p=yl(h),v=vl(sl(e),sl(t)),m=yl(v),g=sl(m-p);(h<1e-9||g>10)&&(h=0,0!=e&&0!=t||(h=1e-9,2==f&&c!=Dl&&(o=0),2==d&&s!=-Dl&&(a=0)));var y=h||v||1e3,_=yl(y),b=ml(10,fl(_)),D=Ol(Tl(e-y*(0==h?0==e?.1:1:o),b/10),9),w=e>=c&&(1==f||3==f&&D<=c||2==f&&D>=c)?c:Dl,x=vl(u,D=w?w:pl(w,D)),k=Ol(Ml(t+y*(0==h?0==t?.1:1:a),b/10),9),C=t<=s&&(1==d||3==d&&k>=s||2==d&&k<=s)?s:-Dl,A=pl(l,k>C&&t<=C?C:vl(C,k));return x==A&&0==x&&(A=100),[x,A]}var al=new Intl.NumberFormat(Nu?Tu.language:"en-US"),ul=function(e){return al.format(e)},ll=Math,cl=ll.PI,sl=ll.abs,fl=ll.floor,dl=ll.round,hl=ll.ceil,pl=ll.min,vl=ll.max,ml=ll.pow,gl=ll.sign,yl=ll.log10,_l=ll.log2,bl=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return ll.asinh(e/t)},Dl=1/0;function wl(e){return 1+(0|yl((e^e>>31)-(e>>31)))}function xl(e,t){return dl(e/t)*t}function kl(e,t,n){return pl(vl(e,t),n)}function Cl(e){return"function"==typeof e?e:function(){return e}}var Al=function(e){return e},El=function(e,t){return t},Sl=function(e){return null},Nl=function(e){return!0},Fl=function(e,t){return e==t};function Ml(e,t){return hl(e/t)*t}function Tl(e,t){return fl(e/t)*t}function Ol(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0;if($l(e))return e;var n=Math.pow(10,t),r=e*n*(1+Number.EPSILON);return dl(r)/n}var Bl=new Map;function Ll(e){return((""+e).split(".")[1]||"").length}function Il(e,t,n,r){for(var i=[],o=r.map(Ll),a=t;a=0&&a>=0?0:u)+(a>=o[c]?0:o[c]),d=Ol(s,f);i.push(d),Bl.set(d,f)}return i}var Pl={},Rl=[],zl=[null,null],jl=Array.isArray,$l=Number.isInteger;function Hl(e){return"string"==typeof e}function Ul(e){var t=!1;if(null!=e){var n=e.constructor;t=null==n||n==Object}return t}function Yl(e){return null!=e&&"object"==typeof e}var Vl=Object.getPrototypeOf(Uint8Array);function ql(e){var t,n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:Ul;if(jl(e)){var r=e.find((function(e){return null!=e}));if(jl(r)||n(r)){t=Array(e.length);for(var i=0;io){for(r=a-1;r>=0&&null==e[r];)e[r--]=null;for(r=a+1;r12?t-12:t},AA:function(e){return e.getHours()>=12?"PM":"AM"},aa:function(e){return e.getHours()>=12?"pm":"am"},a:function(e){return e.getHours()>=12?"p":"a"},mm:function(e){return nc(e.getMinutes())},m:function(e){return e.getMinutes()},ss:function(e){return nc(e.getSeconds())},s:function(e){return e.getSeconds()},fff:function(e){return((t=e.getMilliseconds())<10?"00":t<100?"0":"")+t;var t}};function ic(e,t){t=t||tc;for(var n,r=[],i=/\{([a-z]+)\}|[^{]+/gi;n=i.exec(e);)r.push("{"==n[0][0]?rc[n[1]]:n[0]);return function(e){for(var n="",i=0;i=a,v=f>=o&&f=i?i:f,F=_+(fl(c)-fl(g))+Ml(g-_,N);h.push(F);for(var M=t(F),T=M.getHours()+M.getMinutes()/n+M.getSeconds()/r,O=f/r,B=d/u.axes[l]._space;!((F=Ol(F+f,1==e?0:3))>s);)if(O>1){var L=fl(Ol(T+O,6))%24,I=t(F).getHours()-L;I>1&&(I=-1),T=(T+O)%24,Ol(((F-=I*r)-h[h.length-1])/f,3)*B>=.7&&h.push(F)}else h.push(F)}return h}}]}var xc=Ft(wc(1),3),kc=xc[0],Cc=xc[1],Ac=xc[2],Ec=Ft(wc(.001),3),Sc=Ec[0],Nc=Ec[1],Fc=Ec[2];function Mc(e,t){return e.map((function(e){return e.map((function(n,r){return 0==r||8==r||null==n?n:t(1==r||0==e[8]?n:e[1]+n)}))}))}function Tc(e,t){return function(n,r,i,o,a){var u,l,c,s,f,d,h=t.find((function(e){return a>=e[0]}))||t[t.length-1];return r.map((function(t){var n=e(t),r=n.getFullYear(),i=n.getMonth(),o=n.getDate(),a=n.getHours(),p=n.getMinutes(),v=n.getSeconds(),m=r!=u&&h[2]||i!=l&&h[3]||o!=c&&h[4]||a!=s&&h[5]||p!=f&&h[6]||v!=d&&h[7]||h[1];return u=r,l=i,c=o,s=a,f=p,d=v,m(n)}))}}function Oc(e,t,n){return new Date(e,t,n)}function Bc(e,t){return t(e)}Il(2,-53,53,[1]);function Lc(e,t){return function(n,r){return t(e(r))}}var Ic={show:!0,live:!0,isolate:!1,mount:function(){},markers:{show:!0,width:2,stroke:function(e,t){var n=e.series[t];return n.width?n.stroke(e,t):n.points.width?n.points.stroke(e,t):null},fill:function(e,t){return e.series[t].fill(e,t)},dash:"solid"},idx:null,idxs:null,values:[]};var Pc=[0,0];function Rc(e,t,n){return function(e){0==e.button&&n(e)}}function zc(e,t,n){return n}var jc={show:!0,x:!0,y:!0,lock:!1,move:function(e,t,n){return Pc[0]=t,Pc[1]=n,Pc},points:{show:function(e,t){var n=e.cursor.points,r=Pu(),i=n.size(e,t);Lu(r,hu,i),Lu(r,pu,i);var o=i/-2;Lu(r,"marginLeft",o),Lu(r,"marginTop",o);var a=n.width(e,t,i);return a&&Lu(r,"borderWidth",a),r},size:function(e,t){return as(e.series[t].points.width,1)},width:0,stroke:function(e,t){var n=e.series[t].points;return n._stroke||n._fill},fill:function(e,t){var n=e.series[t].points;return n._fill||n._stroke}},bind:{mousedown:Rc,mouseup:Rc,click:Rc,dblclick:Rc,mousemove:zc,mouseleave:zc,mouseenter:zc},drag:{setScale:!0,x:!0,y:!1,dist:0,uni:null,_x:!1,_y:!1},focus:{prox:-1},left:-10,top:-10,idx:null,dataIdx:function(e,t,n){return n},idxs:null},$c={show:!0,stroke:"rgba(0,0,0,0.07)",width:2},Hc=Wl({},$c,{filter:El}),Uc=Wl({},Hc,{size:10}),Yc=Wl({},$c,{show:!1}),Vc='12px system-ui, -apple-system, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"',qc="bold "+Vc,Wc={show:!0,scale:"x",stroke:_u,space:50,gap:5,size:50,labelGap:0,labelSize:30,labelFont:qc,side:2,grid:Hc,ticks:Uc,border:Yc,font:Vc,rotate:0},Qc={show:!0,scale:"x",auto:!1,sorted:1,min:Dl,max:-Dl,idxs:[]};function Gc(e,t,n,r,i){return t.map((function(e){return null==e?"":ul(e)}))}function Jc(e,t,n,r,i,o,a){for(var u=[],l=Bl.get(i)||0,c=n=a?n:Ol(Ml(n,i),l);c<=r;c=Ol(c+i,l))u.push(Object.is(c,-0)?0:c);return u}function Zc(e,t,n,r,i,o,a){var u=[],l=e.scales[e.axes[t].scale].log,c=fl((10==l?yl:_l)(n));i=ml(l,c),c<0&&(i=Ol(i,-c));var s=n;do{u.push(s),(s=Ol(s+i,Bl.get(i)))>=i*l&&(i=s)}while(s<=r);return u}function Kc(e,t,n,r,i,o,a){var u=e.scales[e.axes[t].scale].asinh,l=r>u?Zc(e,t,vl(u,n),r,i):[u],c=r>=0&&n<=0?[0]:[];return(n<-u?Zc(e,t,vl(u,-r),-n,i):[u]).reverse().map((function(e){return-e})).concat(c,l)}var Xc=/./,es=/[12357]/,ts=/[125]/,ns=/1/;function rs(e,t,n,r,i){var o=e.axes[n],a=o.scale,u=e.scales[a];if(3==u.distr&&2==u.log)return t;var l=e.valToPos,c=o._space,s=l(10,a),f=l(9,a)-s>=c?Xc:l(7,a)-s>=c?es:l(5,a)-s>=c?ts:ns;return t.map((function(e){return 4==u.distr&&0==e||f.test(e)?e:null}))}function is(e,t){return null==t?"":ul(t)}var os={show:!0,scale:"y",stroke:_u,space:30,gap:5,size:50,labelGap:0,labelSize:30,labelFont:qc,side:3,grid:Hc,ticks:Uc,border:Yc,font:Vc,rotate:0};function as(e,t){return Ol((3+2*(e||1))*t,3)}var us={scale:null,auto:!0,sorted:0,min:Dl,max:-Dl},ls=function(e,t,n,r,i){return i},cs={show:!0,auto:!0,sorted:0,gaps:ls,alpha:1,facets:[Wl({},us,{scale:"x"}),Wl({},us,{scale:"y"})]},ss={scale:"y",auto:!0,sorted:0,show:!0,spanGaps:!1,gaps:ls,alpha:1,points:{show:function(e,t){var n=e.series[0],r=n.scale,i=n.idxs,o=e._data[0],a=e.valToPos(o[i[0]],r,!0),u=e.valToPos(o[i[1]],r,!0),l=sl(u-a)/(e.series[t].points.space*nu);return i[1]-i[0]<=l},filter:null},values:null,min:Dl,max:-Dl,idxs:[],path:null,clip:null};function fs(e,t,n,r,i){return n/10}var ds={time:!0,auto:!0,distr:1,log:10,asinh:1,min:null,max:null,dir:1,ori:0},hs=Wl({},ds,{time:!1,ori:1}),ps={};function vs(e,t){var n=ps[e];return n||(n={key:e,plots:[],sub:function(e){n.plots.push(e)},unsub:function(e){n.plots=n.plots.filter((function(t){return t!=e}))},pub:function(e,t,r,i,o,a,u){for(var l=0;l0){a=new Path2D;for(var u=0==t?Ss:Ns,l=n,c=0;cs[0]){var f=s[0]-l;f>0&&u(a,l,r,f,r+o),l=s[1]}}var d=n+i-l;d>0&&u(a,l,r,d,r+o)}return a}function Ds(e,t,n,r,i,o,a){for(var u=[],l=e.length,c=1==i?n:r;c>=n&&c<=r;c+=i){if(null===t[c]){var s=c,f=c;if(1==i)for(;++c<=r&&null===t[c];)f=c;else for(;--c>=n&&null===t[c];)f=c;var d=o(e[s]),h=f==s?d:o(e[f]),p=s-i;d=a<=0&&p>=0&&p=0&&v>=0&&v=d&&u.push([d,h])}}return u}function ws(e){return 0==e?Al:1==e?dl:function(t){return xl(t,e)}}function xs(e){var t=0==e?ks:Cs,n=0==e?function(e,t,n,r,i,o){e.arcTo(t,n,r,i,o)}:function(e,t,n,r,i,o){e.arcTo(n,t,i,r,o)},r=0==e?function(e,t,n,r,i){e.rect(t,n,r,i)}:function(e,t,n,r,i){e.rect(n,t,i,r)};return function(e,i,o,a,u){var l=arguments.length>5&&void 0!==arguments[5]?arguments[5]:0;0==l?r(e,i,o,a,u):(l=pl(l,a/2,u/2),t(e,i+l,o),n(e,i+a,o,i+a,o+u,l),n(e,i+a,o+u,i,o+u,l),n(e,i,o+u,i,o,l),n(e,i,o,i+a,o,l),e.closePath())}}var ks=function(e,t,n){e.moveTo(t,n)},Cs=function(e,t,n){e.moveTo(n,t)},As=function(e,t,n){e.lineTo(t,n)},Es=function(e,t,n){e.lineTo(n,t)},Ss=xs(0),Ns=xs(1),Fs=function(e,t,n,r,i,o){e.arc(t,n,r,i,o)},Ms=function(e,t,n,r,i,o){e.arc(n,t,r,i,o)},Ts=function(e,t,n,r,i,o,a){e.bezierCurveTo(t,n,r,i,o,a)},Os=function(e,t,n,r,i,o,a){e.bezierCurveTo(n,t,i,r,a,o)};function Bs(e){return function(e,t,n,r,i){return ms(e,t,(function(t,o,a,u,l,c,s,f,d,h,p){var v,m,g=t.pxRound,y=t.points;0==u.ori?(v=ks,m=Fs):(v=Cs,m=Ms);var _=Ol(y.width*nu,3),b=(y.size-y.width)/2*nu,D=Ol(2*b,3),w=new Path2D,x=new Path2D,k=e.bbox,C=k.left,A=k.top,E=k.width,S=k.height;Ss(x,C-D,A-D,E+2*D,S+2*D);var N=function(e){if(null!=a[e]){var t=g(c(o[e],u,h,f)),n=g(s(a[e],l,p,d));v(w,t+b,n),m(w,t,n,b,0,2*cl)}};if(i)i.forEach(N);else for(var F=n;F<=r;F++)N(F);return{stroke:_>0?w:null,fill:w,clip:x,flags:3}}))}}function Ls(e){return function(t,n,r,i,o,a){r!=i&&(o!=r&&a!=r&&e(t,n,r),o!=i&&a!=i&&e(t,n,i),e(t,n,a))}}var Is=Ls(As),Ps=Ls(Es);function Rs(e){var t=il(null===e||void 0===e?void 0:e.alignGaps,0);return function(e,n,r,i){return ms(e,n,(function(o,a,u,l,c,s,f,d,h,p,v){var m,g,y=o.pxRound,_=function(e){return y(s(e,l,p,d))},b=function(e){return y(f(e,c,v,h))};0==l.ori?(m=As,g=Is):(m=Es,g=Ps);for(var D,w,x,k=l.dir*(0==l.ori?1:-1),C={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:1},A=C.stroke,E=Dl,S=-Dl,N=_(a[1==k?r:i]),F=Gu(u,r,i,1*k),M=Gu(u,r,i,-1*k),T=_(a[F]),O=_(a[M]),B=1==k?r:i;B>=r&&B<=i;B+=k){var L=_(a[B]);L==N?null!=u[B]&&(w=b(u[B]),E==Dl&&(m(A,L,w),D=w),E=pl(w,E),S=vl(w,S)):(E!=Dl&&(g(A,N,E,S,D,w),x=N),null!=u[B]?(m(A,L,w=b(u[B])),E=S=D=w):(E=Dl,S=-Dl),N=L)}E!=Dl&&E!=S&&x!=N&&g(A,N,E,S,D,w);var I=Ft(gs(e,n),2),P=I[0],R=I[1];if(null!=o.fill||0!=P){var z=C.fill=new Path2D(A),j=b(o.fillTo(e,n,o.min,o.max,P));m(z,O,j),m(z,T,j)}if(!o.spanGaps){var $,H=[];($=H).push.apply($,Ot(Ds(a,u,r,i,k,_,t))),C.gaps=H=o.gaps(e,n,r,i,H),C.clip=bs(H,l.ori,d,h,p,v)}return 0!=R&&(C.band=2==R?[_s(e,n,r,i,A,-1),_s(e,n,r,i,A,1)]:_s(e,n,r,i,A,R)),C}))}}function zs(e,t,n,r,i,o){var a=e.length;if(a<2)return null;var u=new Path2D;if(n(u,e[0],t[0]),2==a)r(u,e[1],t[1]);else{for(var l=Array(a),c=Array(a-1),s=Array(a-1),f=Array(a-1),d=0;d0!==c[h]>0?l[h]=0:(l[h]=3*(f[h-1]+f[h])/((2*f[h]+f[h-1])/c[h-1]+(f[h]+2*f[h-1])/c[h]),isFinite(l[h])||(l[h]=0));l[a-1]=c[a-2];for(var p=0;p=i&&o+(l<5?Bl.get(l):0)<=17)return[l,c]}while(++u0?e:t.clamp(r,e,t.min,t.max,t.key)):4==t.distr?bl(e,t.asinh):e)-t._min)/(t._max-t._min)}function a(e,t,n,r){var i=o(e,t);return r+n*(-1==t.dir?1-i:i)}function u(e,t,n,r){var i=o(e,t);return r+n*(-1==t.dir?i:1-i)}function l(e,t,n,r){return 0==t.ori?a(e,t,n,r):u(e,t,n,r)}r.valToPosH=a,r.valToPosV=u;var c=!1;r.status=0;var s=r.root=Pu("uplot");(null!=e.id&&(s.id=e.id),Ou(s,e.class),e.title)&&(Pu("u-title",s).textContent=e.title);var f=Iu("canvas"),d=r.ctx=f.getContext("2d"),h=Pu("u-wrap",s),p=r.under=Pu("u-under",h);h.appendChild(f);var v=r.over=Pu("u-over",h),m=+il((e=ql(e)).pxAlign,1),g=ws(m);(e.plugins||[]).forEach((function(t){t.opts&&(e=t.opts(r,e)||e)}));var y=e.ms||.001,_=r.series=1==i?Ys(e.series||[],Qc,ss,!1):function(e,t){return e.map((function(e,n){return 0==n?null:Wl({},t,e)}))}(e.series||[null],cs),b=r.axes=Ys(e.axes||[],Wc,os,!0),D=r.scales={},w=r.bands=e.bands||[];w.forEach((function(e){e.fill=Cl(e.fill||null),e.dir=il(e.dir,-1)}));var x=2==i?_[1].facets[0].scale:_[0].scale,k={axes:function(){for(var e=function(){var e=b[t];if(!e.show||!e._show)return"continue";var n,i,o=e.side,a=o%2,u=e.stroke(r,t),c=0==o||3==o?-1:1;if(e.label){var s=e.labelGap*c,f=dl((e._lpos+s)*nu);Ke(e.labelFont[0],u,"center",2==o?vu:mu),d.save(),1==a?(n=i=0,d.translate(f,dl(de+pe/2)),d.rotate((3==o?-cl:cl)/2)):(n=dl(fe+he/2),i=f),d.fillText(e.label,n,i),d.restore()}var h=Ft(e._found,2),p=h[0],v=h[1];if(0==v)return"continue";var m=D[e.scale],y=0==a?he:pe,_=0==a?fe:de,w=dl(e.gap*nu),x=e._splits,k=2==m.distr?x.map((function(e){return We[e]})):x,C=2==m.distr?We[x[1]]-We[x[0]]:p,A=e.ticks,E=e.border,S=A.show?dl(A.size*nu):0,N=e._rotate*-cl/180,F=g(e._pos*nu),M=F+(S+w)*c;i=0==a?M:0,n=1==a?M:0,Ke(e.font[0],u,1==e.align?gu:2==e.align?yu:N>0?gu:N<0?yu:0==a?"center":3==o?yu:gu,N||1==a?"middle":2==o?vu:mu);for(var T=1.5*e.font[1],O=x.map((function(e){return g(l(e,m,y,_))})),B=e._values,L=0;L0&&(_.forEach((function(e,n){if(n>0&&e.show&&null==e._paths){var o=2==i?[0,t[n][0].length-1]:function(e){var t=kl(Ye-1,0,Me-1),n=kl(Ve+1,0,Me-1);for(;null==e[t]&&t>0;)t--;for(;null==e[n]&&n0&&e.show){$e!=e.alpha&&(d.globalAlpha=$e=e.alpha),et(t,!1),e._paths&&tt(t,!1),et(t,!0);var n=e._paths?e._paths.gaps:null,i=e.points.show(r,t,Ye,Ve,n),o=e.points.filter(r,t,i,n);(i||o)&&(e.points._paths=e.points.paths(r,t,Ye,Ve,o),tt(t,!0)),1!=$e&&(d.globalAlpha=$e=1),an("drawSeries",t)}})))}},C=(e.drawOrder||["axes","series"]).map((function(e){return k[e]}));function A(t){var n=D[t];if(null==n){var r=(e.scales||Pl)[t]||Pl;if(null!=r.from)A(r.from),D[t]=Wl({},D[r.from],r,{key:t});else{(n=D[t]=Wl({},t==x?ds:hs,r)).key=t;var o=n.time,a=n.range,u=jl(a);if((t!=x||2==i&&!o)&&(!u||null!=a[0]&&null!=a[1]||(a={min:null==a[0]?el:{mode:1,hard:a[0],soft:a[0]},max:null==a[1]?el:{mode:1,hard:a[1],soft:a[1]}},u=!1),!u&&Ul(a))){var l=a;a=function(e,t,n){return null==t?zl:rl(t,n,l)}}n.range=Cl(a||(o?Ws:t==x?3==n.distr?Js:4==n.distr?Ks:qs:3==n.distr?Gs:4==n.distr?Zs:Qs)),n.auto=Cl(!u&&n.auto),n.clamp=Cl(n.clamp||fs),n._min=n._max=null}}}for(var E in A("x"),A("y"),1==i&&_.forEach((function(e){A(e.scale)})),b.forEach((function(e){A(e.scale)})),e.scales)A(E);var S,N,F=D[x],M=F.distr;0==F.ori?(Ou(s,"u-hz"),S=a,N=u):(Ou(s,"u-vt"),S=u,N=a);var T={};for(var O in D){var B=D[O];null==B.min&&null==B.max||(T[O]={min:B.min,max:B.max},B.min=B.max=null)}var L,I=e.tzDate||function(e){return new Date(dl(e/y))},P=e.fmtDate||ic,R=1==y?Ac(I):Fc(I),z=Tc(I,Mc(1==y?Cc:Nc,P)),j=Lc(I,Bc("{YYYY}-{MM}-{DD} {h}:{mm}{aa}",P)),$=[],H=r.legend=Wl({},Ic,e.legend),U=H.show,Y=H.markers;H.idxs=$,Y.width=Cl(Y.width),Y.dash=Cl(Y.dash),Y.stroke=Cl(Y.stroke),Y.fill=Cl(Y.fill);var V,q=[],W=[],Q=!1,G={};if(H.live){var J=_[1]?_[1].values:null;for(var Z in V=(Q=null!=J)?J(r,1,0):{_:0})G[Z]="--"}if(U)if(L=Iu("table","u-legend",s),H.mount(r,L),Q){var K=Iu("tr","u-thead",L);for(var X in Iu("th",null,K),V)Iu("th",du,K).textContent=X}else Ou(L,"u-inline"),H.live&&Ou(L,"u-live");var ee={show:!0},te={show:!1};var ne=new Map;function re(e,t,n){var i=ne.get(t)||{},o=xe.bind[e](r,t,n);o&&(qu(e,t,i[e]=o),ne.set(t,i))}function ie(e,t,n){var r=ne.get(t)||{};for(var i in r)null!=e&&i!=e||(Wu(i,t,r[i]),delete r[i]);null==e&&ne.delete(t)}var oe=0,ae=0,ue=0,le=0,ce=0,se=0,fe=0,de=0,he=0,pe=0;r.bbox={};var ve=!1,me=!1,ge=!1,ye=!1,_e=!1,be=!1;function De(e,t,n){(n||e!=r.width||t!=r.height)&&we(e,t),lt(!1),ge=!0,me=!0,xe.left>=0&&(ye=be=!0),wt()}function we(e,t){r.width=oe=ue=e,r.height=ae=le=t,ce=se=0,function(){var e=!1,t=!1,n=!1,r=!1;b.forEach((function(i,o){if(i.show&&i._show){var a=i.side,u=a%2,l=i._size+(null!=i.label?i.labelSize:0);l>0&&(u?(ue-=l,3==a?(ce+=l,r=!0):n=!0):(le-=l,0==a?(se+=l,e=!0):t=!0))}})),Ne[0]=e,Ne[1]=n,Ne[2]=t,Ne[3]=r,ue-=Ue[1]+Ue[3],ce+=Ue[3],le-=Ue[2]+Ue[0],se+=Ue[0]}(),function(){var e=ce+ue,t=se+le,n=ce,r=se;function i(i,o){switch(i){case 1:return(e+=o)-o;case 2:return(t+=o)-o;case 3:return(n-=o)+o;case 0:return(r-=o)+o}}b.forEach((function(e,t){if(e.show&&e._show){var n=e.side;e._pos=i(n,e._size),null!=e.label&&(e._lpos=i(n,e.labelSize))}}))}();var n=r.bbox;fe=n.left=xl(ce*nu,.5),de=n.top=xl(se*nu,.5),he=n.width=xl(ue*nu,.5),pe=n.height=xl(le*nu,.5)}r.setSize=function(e){De(e.width,e.height)};var xe=r.cursor=Wl({},jc,{drag:{y:2==i}},e.cursor);xe.idxs=$,xe._lock=!1;var ke=xe.points;ke.show=Cl(ke.show),ke.size=Cl(ke.size),ke.stroke=Cl(ke.stroke),ke.width=Cl(ke.width),ke.fill=Cl(ke.fill);var Ce=r.focus=Wl({},e.focus||{alpha:.3},xe.focus),Ae=Ce.prox>=0,Ee=[null];function Se(e,t){if(1==i||t>0){var n=1==i&&D[e.scale].time,o=e.value;e.value=n?Hl(o)?Lc(I,Bc(o,P)):o||j:o||is,e.label=e.label||(n?"Time":"Value")}if(t>0){e.width=null==e.width?1:e.width,e.paths=e.paths||Hs||Sl,e.fillTo=Cl(e.fillTo||ys),e.pxAlign=+il(e.pxAlign,m),e.pxRound=ws(e.pxAlign),e.stroke=Cl(e.stroke||null),e.fill=Cl(e.fill||null),e._stroke=e._fill=e._paths=e._focus=null;var a=as(e.width,1),u=e.points=Wl({},{size:a,width:vl(1,.2*a),stroke:e.stroke,space:2*a,paths:Us,_stroke:null,_fill:null},e.points);u.show=Cl(u.show),u.filter=Cl(u.filter),u.fill=Cl(u.fill),u.stroke=Cl(u.stroke),u.paths=Cl(u.paths),u.pxAlign=e.pxAlign}if(U){var l=function(e,t){if(0==t&&(Q||!H.live||2==i))return zl;var n=[],o=Iu("tr","u-series",L,L.childNodes[t]);Ou(o,e.class),e.show||Ou(o,fu);var a=Iu("th",null,o);if(Y.show){var u=Pu("u-marker",a);if(t>0){var l=Y.width(r,t);l&&(u.style.border=l+"px "+Y.dash(r,t)+" "+Y.stroke(r,t)),u.style.background=Y.fill(r,t)}}var c=Pu(du,a);for(var s in c.textContent=e.label,t>0&&(Y.show||(c.style.color=e.width>0?Y.stroke(r,t):Y.fill(r,t)),re("click",a,(function(t){if(!xe._lock){var n=_.indexOf(e);if((t.ctrlKey||t.metaKey)!=H.isolate){var r=_.some((function(e,t){return t>0&&t!=n&&e.show}));_.forEach((function(e,t){t>0&&Pt(t,r?t==n?ee:te:ee,!0,un.setSeries)}))}else Pt(n,{show:!e.show},!0,un.setSeries)}})),Ae&&re(ku,a,(function(t){xe._lock||Pt(_.indexOf(e),Rt,!0,un.setSeries)}))),V){var f=Iu("td","u-value",o);f.textContent="--",n.push(f)}return[o,n]}(e,t);q.splice(t,0,l[0]),W.splice(t,0,l[1]),H.values.push(null)}if(xe.show){$.splice(t,0,null);var c=function(e,t){if(t>0){var n=xe.points.show(r,t);if(n)return Ou(n,"u-cursor-pt"),Ou(n,e.class),zu(n,-10,-10,ue,le),v.insertBefore(n,Ee[t]),n}}(e,t);c&&Ee.splice(t,0,c)}an("addSeries",t)}r.addSeries=function(e,t){t=null==t?_.length:t,e=1==i?Vs(e,t,Qc,ss):Vs(e,t,null,cs),_.splice(t,0,e),Se(_[t],t)},r.delSeries=function(e){if(_.splice(e,1),U){H.values.splice(e,1),W.splice(e,1);var t=q.splice(e,1)[0];ie(null,t.firstChild),t.remove()}xe.show&&($.splice(e,1),Ee.length>1&&Ee.splice(e,1)[0].remove()),an("delSeries",e)};var Ne=[!1,!1,!1,!1];function Fe(e,t,n,r){var i=Ft(n,4),o=i[0],a=i[1],u=i[2],l=i[3],c=t%2,s=0;return 0==c&&(l||a)&&(s=0==t&&!o||2==t&&!u?dl(Wc.size/3):0),1==c&&(o||u)&&(s=1==t&&!a||3==t&&!l?dl(os.size/2):0),s}var Me,Te,Oe,Be,Le,Ie,Pe,Re,ze,je,$e,He=r.padding=(e.padding||[Fe,Fe,Fe,Fe]).map((function(e){return Cl(il(e,Fe))})),Ue=r._padding=He.map((function(e,t){return e(r,t,Ne,0)})),Ye=null,Ve=null,qe=1==i?_[0].idxs:null,We=null,Qe=!1;function Ge(e,n){if(t=null==e?[]:ql(e,Yl),2==i){Me=0;for(var o=1;o<_.length;o++)Me+=t[o][0].length;r.data=t=e}else if(null==t[0]&&(t[0]=[]),r.data=t.slice(),We=t[0],Me=We.length,2==M){t[0]=Array(Me);for(var a=0;a=0,be=!0,wt()}}function Je(){var e,n;if(Qe=!0,1==i)if(Me>0){if(Ye=qe[0]=0,Ve=qe[1]=Me-1,e=t[0][Ye],n=t[0][Ve],2==M)e=Ye,n=Ve;else if(1==Me)if(3==M){var r=Ft(Ku(e,e,F.log,!1),2);e=r[0],n=r[1]}else if(4==M){var o=Ft(Xu(e,e,F.log,!1),2);e=o[0],n=o[1]}else if(F.time)n=e+dl(86400/y);else{var a=Ft(rl(e,n,.1,!0),2);e=a[0],n=a[1]}}else Ye=qe[0]=e=null,Ve=qe[1]=n=null;It(x,e,n)}function Ze(e,t,n,r,i,o){var a,u,l,c,s;null!==(a=e)&&void 0!==a||(e=bu),null!==(u=n)&&void 0!==u||(n=Rl),null!==(l=r)&&void 0!==l||(r="butt"),null!==(c=i)&&void 0!==c||(i=bu),null!==(s=o)&&void 0!==s||(o="round"),e!=Te&&(d.strokeStyle=Te=e),i!=Oe&&(d.fillStyle=Oe=i),t!=Be&&(d.lineWidth=Be=t),o!=Ie&&(d.lineJoin=Ie=o),r!=Pe&&(d.lineCap=Pe=r),n!=Le&&d.setLineDash(Le=n)}function Ke(e,t,n,r){t!=Oe&&(d.fillStyle=Oe=t),e!=Re&&(d.font=Re=e),n!=ze&&(d.textAlign=ze=n),r!=je&&(d.textBaseline=je=r)}function Xe(e,t,n,i){var o=arguments.length>4&&void 0!==arguments[4]?arguments[4]:0;if(i.length>0&&e.auto(r,Qe)&&(null==t||null==t.min)){var a=il(Ye,0),u=il(Ve,i.length-1),l=null==n.min?3==e.distr?Zu(i,a,u):Ju(i,a,u,o):[n.min,n.max];e.min=pl(e.min,n.min=l[0]),e.max=vl(e.max,n.max=l[1])}}function et(e,t){var n=t?_[e].points:_[e];n._stroke=n.stroke(r,e),n._fill=n.fill(r,e)}function tt(e,n){var i=n?_[e].points:_[e],o=i._stroke,a=i._fill,u=i._paths,l=u.stroke,c=u.fill,s=u.clip,f=u.flags,h=null,p=Ol(i.width*nu,3),v=p%2/2;n&&null==a&&(a=p>0?"#fff":o);var m=1==i.pxAlign;if(m&&d.translate(v,v),!n){var g=fe,y=de,b=he,D=pe,x=p*nu/2;0==i.min&&(D+=x),0==i.max&&(y-=x,D+=x),(h=new Path2D).rect(g,y,b,D)}n?nt(o,p,i.dash,i.cap,a,l,c,f,s):function(e,n,i,o,a,u,l,c,s,f,d){var h=!1;w.forEach((function(p,v){if(p.series[0]==e){var m,g=_[p.series[1]],y=t[p.series[1]],b=(g._paths||Pl).band;jl(b)&&(b=1==p.dir?b[0]:b[1]);var D=null;g.show&&b&&function(e,t,n){for(t=il(t,0),n=il(n,e.length-1);t<=n;){if(null!=e[t])return!0;t++}return!1}(y,Ye,Ve)?(D=p.fill(r,v)||u,m=g._paths.clip):b=null,nt(n,i,o,a,D,l,c,s,f,d,m,b),h=!0}})),h||nt(n,i,o,a,u,l,c,s,f,d)}(e,o,p,i.dash,i.cap,a,l,c,f,h,s),m&&d.translate(-v,-v)}r.setData=Ge;function nt(e,t,n,r,i,o,a,u,l,c,s,f){Ze(e,t,n,r,i),(l||c||f)&&(d.save(),l&&d.clip(l),c&&d.clip(c)),f?3==(3&u)?(d.clip(f),s&&d.clip(s),it(i,a),rt(e,o,t)):2&u?(it(i,a),d.clip(f),rt(e,o,t)):1&u&&(d.save(),d.clip(f),s&&d.clip(s),it(i,a),d.restore(),rt(e,o,t)):(it(i,a),rt(e,o,t)),(l||c||f)&&d.restore()}function rt(e,t,n){n>0&&(t instanceof Map?t.forEach((function(e,t){d.strokeStyle=Te=t,d.stroke(e)})):null!=t&&e&&d.stroke(t))}function it(e,t){t instanceof Map?t.forEach((function(e,t){d.fillStyle=Oe=t,d.fill(e)})):null!=t&&e&&d.fill(t)}function ot(e,t,n,r,i,o,a,u,l,c){var s=a%2/2;1==m&&d.translate(s,s),Ze(u,a,l,c,u),d.beginPath();var f,h,p,v,g=i+(0==r||3==r?-o:o);0==n?(h=i,v=g):(f=i,p=g);for(var y=0;y0&&(t._paths=null,e&&(1==i?(t.min=null,t.max=null):t.facets.forEach((function(e){e.min=null,e.max=null}))))}))}var ct,st,ft,dt,ht,pt,vt,mt,gt,yt,_t,bt,Dt=!1;function wt(){Dt||(Gl(xt),Dt=!0)}function xt(){ve&&(!function(){var e=ql(D,Yl);for(var n in e){var o=e[n],a=T[n];if(null!=a&&null!=a.min)Wl(o,a),n==x&<(!0);else if(n!=x||2==i)if(0==Me&&null==o.from){var u=o.range(r,null,null,n);o.min=u[0],o.max=u[1]}else o.min=Dl,o.max=-Dl}if(Me>0)for(var l in _.forEach((function(n,o){if(1==i){var a=n.scale,u=e[a],l=T[a];if(0==o){var c=u.range(r,u.min,u.max,a);u.min=c[0],u.max=c[1],Ye=Qu(u.min,t[0]),(Ve=Qu(u.max,t[0]))-Ye>1&&(t[0][Ye]u.max&&Ve--),n.min=We[Ye],n.max=We[Ve]}else n.show&&n.auto&&Xe(u,l,n,t[o],n.sorted);n.idxs[0]=Ye,n.idxs[1]=Ve}else if(o>0&&n.show&&n.auto){var s=Ft(n.facets,2),f=s[0],d=s[1],h=f.scale,p=d.scale,v=Ft(t[o],2),m=v[0],g=v[1];Xe(e[h],T[h],f,m,f.sorted),Xe(e[p],T[p],d,g,d.sorted),n.min=d.min,n.max=d.max}})),e){var c=e[l],s=T[l];if(null==c.from&&(null==s||null==s.min)){var f=c.range(r,c.min==Dl?null:c.min,c.max==-Dl?null:c.max,l);c.min=f[0],c.max=f[1]}}for(var d in e){var h=e[d];if(null!=h.from){var p=e[h.from];if(null==p.min)h.min=h.max=null;else{var v=h.range(r,p.min,p.max,d);h.min=v[0],h.max=v[1]}}}var m={},g=!1;for(var y in e){var b=e[y],w=D[y];if(w.min!=b.min||w.max!=b.max){w.min=b.min,w.max=b.max;var k=w.distr;w._min=3==k?yl(w.min):4==k?bl(w.min,w.asinh):w.min,w._max=3==k?yl(w.max):4==k?bl(w.max,w.asinh):w.max,m[y]=g=!0}}if(g){for(var C in _.forEach((function(e,t){2==i?t>0&&m.y&&(e._paths=null):m[e.scale]&&(e._paths=null)})),m)ge=!0,an("setScale",C);xe.show&&xe.left>=0&&(ye=be=!0)}for(var A in T)T[A]=null}(),ve=!1),ge&&(!function(){for(var e=!1,t=0;!e;){var n=at(++t),i=ut(t);(e=3==t||n&&i)||(we(r.width,r.height),me=!0)}}(),ge=!1),me&&(Lu(p,gu,ce),Lu(p,vu,se),Lu(p,hu,ue),Lu(p,pu,le),Lu(v,gu,ce),Lu(v,vu,se),Lu(v,hu,ue),Lu(v,pu,le),Lu(h,hu,oe),Lu(h,pu,ae),f.width=dl(oe*nu),f.height=dl(ae*nu),b.forEach((function(e){var t=e._el,n=e._show,r=e._size,i=e._pos,o=e.side;if(null!=t)if(n){var a=o%2==1;Lu(t,a?"left":"top",i-(3===o||0===o?r:0)),Lu(t,a?"width":"height",r),Lu(t,a?"top":"left",a?se:ce),Lu(t,a?"height":"width",a?le:ue),Bu(t,fu)}else Ou(t,fu)})),Te=Oe=Be=Ie=Pe=Re=ze=je=Le=null,$e=1,Qt(!0),an("setSize"),me=!1),oe>0&&ae>0&&(d.clearRect(0,0,f.width,f.height),an("drawClear"),C.forEach((function(e){return e()})),an("draw")),Ot.show&&_e&&(Lt(Ot),_e=!1),xe.show&&ye&&(qt(null,!0,!1),ye=!1),c||(c=!0,r.status=1,an("ready")),Qe=!1,Dt=!1}function kt(e,n){var i=D[e];if(null==i.from){if(0==Me){var o=i.range(r,n.min,n.max,e);n.min=o[0],n.max=o[1]}if(n.min>n.max){var a=n.min;n.min=n.max,n.max=a}if(Me>1&&null!=n.min&&null!=n.max&&n.max-n.min<1e-16)return;e==x&&2==i.distr&&Me>0&&(n.min=Qu(n.min,t[0]),n.max=Qu(n.max,t[0]),n.min==n.max&&n.max++),T[e]=n,ve=!0,wt()}}r.redraw=function(e,t){ge=t||!1,!1!==e?It(x,F.min,F.max):wt()},r.setScale=kt;var Ct=!1,At=xe.drag,Et=At.x,St=At.y;xe.show&&(xe.x&&(ct=Pu("u-cursor-x",v)),xe.y&&(st=Pu("u-cursor-y",v)),0==F.ori?(ft=ct,dt=st):(ft=st,dt=ct),_t=xe.left,bt=xe.top);var Nt,Mt,Tt,Ot=r.select=Wl({show:!0,over:!0,left:0,width:0,top:0,height:0},e.select),Bt=Ot.show?Pu("u-select",Ot.over?v:p):null;function Lt(e,t){if(Ot.show){for(var n in e)Ot[n]=e[n],n in Zt&&Lu(Bt,n,e[n]);!1!==t&&an("setSelect")}}function It(e,t,n){kt(e,{min:t,max:n})}function Pt(e,t,n,o){null!=t.focus&&function(e){if(e!=Tt){var t=null==e,n=1!=Ce.alpha;_.forEach((function(r,i){var o=t||0==i||i==e;r._focus=t?null:o,n&&function(e,t){_[e].alpha=t,xe.show&&Ee[e]&&(Ee[e].style.opacity=t);U&&q[e]&&(q[e].style.opacity=t)}(i,o?1:Ce.alpha)})),Tt=e,n&&wt()}}(e),null!=t.show&&_.forEach((function(n,r){r>0&&(e==r||null==e)&&(n.show=t.show,function(e,t){var n=_[e],r=U?q[e]:null;n.show?r&&Bu(r,fu):(r&&Ou(r,fu),Ee.length>1&&zu(Ee[e],-10,-10,ue,le))}(r,t.show),It(2==i?n.facets[1].scale:n.scale,null,null),wt())})),!1!==n&&an("setSeries",e,t),o&&sn("setSeries",r,e,t)}r.setSelect=Lt,r.setSeries=Pt,r.addBand=function(e,t){e.fill=Cl(e.fill||null),e.dir=il(e.dir,-1),t=null==t?w.length:t,w.splice(t,0,e)},r.setBand=function(e,t){Wl(w[e],t)},r.delBand=function(e){null==e?w.length=0:w.splice(e,1)};var Rt={focus:!0};function zt(e,t,n){var r=D[t];n&&(e=e/nu-(1==r.ori?se:ce));var i=ue;1==r.ori&&(e=(i=le)-e),-1==r.dir&&(e=i-e);var o=r._min,a=o+(r._max-o)*(e/i),u=r.distr;return 3==u?ml(10,a):4==u?function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return ll.sinh(e)*t}(a,r.asinh):a}function jt(e,t){Lu(Bt,gu,Ot.left=e),Lu(Bt,hu,Ot.width=t)}function $t(e,t){Lu(Bt,vu,Ot.top=e),Lu(Bt,pu,Ot.height=t)}U&&Ae&&qu(Cu,L,(function(e){xe._lock||null!=Tt&&Pt(null,Rt,!0,un.setSeries)})),r.valToIdx=function(e){return Qu(e,t[0])},r.posToIdx=function(e,n){return Qu(zt(e,x,n),t[0],Ye,Ve)},r.posToVal=zt,r.valToPos=function(e,t,n){return 0==D[t].ori?a(e,D[t],n?he:ue,n?fe:0):u(e,D[t],n?pe:le,n?de:0)},r.batch=function(e){e(r),wt()},r.setCursor=function(e,t,n){_t=e.left,bt=e.top,qt(null,t,n)};var Ht=0==F.ori?jt:$t,Ut=1==F.ori?jt:$t;function Yt(e,t){if(null!=e){var n=e.idx;H.idx=n,_.forEach((function(e,t){(t>0||!Q)&&Vt(t,n)}))}U&&H.live&&function(){if(U&&H.live)for(var e=2==i?1:0;e<_.length;e++)if(0!=e||!Q){var t=H.values[e],n=0;for(var r in t)W[e][n++].firstChild.nodeValue=t[r]}}(),be=!1,!1!==t&&an("setLegend")}function Vt(e,n){var i;if(null==n)i=G;else{var o=_[e],a=0==e&&2==M?We:t[e];i=Q?o.values(r,e,n):{_:o.value(r,a[n],e,n)}}H.values[e]=i}function qt(e,n,o){gt=_t,yt=bt;var a,u=Ft(xe.move(r,_t,bt),2);_t=u[0],bt=u[1],xe.show&&(ft&&zu(ft,dl(_t),0,ue,le),dt&&zu(dt,0,dl(bt),ue,le));var l=Ye>Ve;Nt=Dl;var c=0==F.ori?ue:le,s=1==F.ori?ue:le;if(_t<0||0==Me||l){a=null;for(var f=0;f<_.length;f++)f>0&&Ee.length>1&&zu(Ee[f],-10,-10,ue,le);if(Ae&&Pt(null,Rt,!0,null==e&&un.setSeries),H.live){$.fill(null),be=!0;for(var d=0;d<_.length;d++)H.values[d]=G}}else{var h,p;1==i&&(a=Qu(h=zt(0==F.ori?_t:bt,x),t[0],Ye,Ve),p=Ml(S(t[0][a],F,c,0),.5));for(var v=2==i?1:0;v<_.length;v++){var m=_[v],g=$[v],y=1==i?t[v][g]:t[v][1][g],b=xe.dataIdx(r,v,a,h),w=1==i?t[v][b]:t[v][1][b];be=be||w!=y||b!=g,$[v]=b;var k=b==a?p:Ml(S(1==i?t[0][b]:t[v][0][b],F,c,0),.5);if(v>0&&m.show){var C=null==w?-10:Ml(N(w,1==i?D[m.scale]:D[m.facets[1].scale],s,0),.5);if(C>0&&1==i){var A=sl(C-bt);A<=Nt&&(Nt=A,Mt=v)}var E=void 0,M=void 0;if(0==F.ori?(E=k,M=C):(E=C,M=k),be&&Ee.length>1){$u(Ee[v],xe.points.fill(r,v),xe.points.stroke(r,v));var T=void 0,O=void 0,B=void 0,L=void 0,I=!0,P=xe.points.bbox;if(null!=P){I=!1;var R=P(r,v);B=R.left,L=R.top,T=R.width,O=R.height}else B=E,L=M,T=O=xe.points.size(r,v);Uu(Ee[v],T,O,I),zu(Ee[v],B,L,ue,le)}}if(H.live){if(!be||0==v&&Q)continue;Vt(v,b)}}}if(xe.idx=a,xe.left=_t,xe.top=bt,be&&(H.idx=a,Yt()),Ot.show&&Ct)if(null!=e){var z=Ft(un.scales,2),j=z[0],U=z[1],Y=Ft(un.match,2),V=Y[0],q=Y[1],W=Ft(e.cursor.sync.scales,2),J=W[0],Z=W[1],K=e.cursor.drag;if(Et=K._x,St=K._y,Et||St){var X,ee,te,ne,re,ie=e.select,oe=ie.left,ae=ie.top,ce=ie.width,se=ie.height,fe=e.scales[j].ori,de=e.posToVal,he=null!=j&&V(j,J),pe=null!=U&&q(U,Z);he&&Et?(0==fe?(X=oe,ee=ce):(X=ae,ee=se),te=D[j],ne=S(de(X,J),te,c,0),re=S(de(X+ee,J),te,c,0),Ht(pl(ne,re),sl(re-ne))):Ht(0,c),pe&&St?(1==fe?(X=oe,ee=ce):(X=ae,ee=se),te=D[U],ne=N(de(X,Z),te,s,0),re=N(de(X+ee,Z),te,s,0),Ut(pl(ne,re),sl(re-ne))):Ut(0,s)}else Kt()}else{var ve=sl(gt-ht),me=sl(yt-pt);if(1==F.ori){var ge=ve;ve=me,me=ge}Et=At.x&&ve>=At.dist,St=At.y&&me>=At.dist;var ye,_e,De=At.uni;null!=De?Et&&St&&(St=me>=De,(Et=ve>=De)||St||(me>ve?St=!0:Et=!0)):At.x&&At.y&&(Et||St)&&(Et=St=!0),Et&&(0==F.ori?(ye=vt,_e=_t):(ye=mt,_e=bt),Ht(pl(ye,_e),sl(_e-ye)),St||Ut(0,s)),St&&(1==F.ori?(ye=vt,_e=_t):(ye=mt,_e=bt),Ut(pl(ye,_e),sl(_e-ye)),Et||Ht(0,c)),Et||St||(Ht(0,0),Ut(0,0))}if(At._x=Et,At._y=St,null==e){if(o){if(null!=ln){var we=Ft(un.scales,2),ke=we[0],Se=we[1];un.values[0]=null!=ke?zt(0==F.ori?_t:bt,ke):null,un.values[1]=null!=Se?zt(1==F.ori?_t:bt,Se):null}sn(Du,r,_t,bt,ue,le,a)}if(Ae){var Ne=o&&un.setSeries,Fe=Ce.prox;null==Tt?Nt<=Fe&&Pt(Mt,Rt,!0,Ne):Nt>Fe?Pt(null,Rt,!0,Ne):Mt!=Tt&&Pt(Mt,Rt,!0,Ne)}}!1!==n&&an("setCursor")}r.setLegend=Yt;var Wt=null;function Qt(e){!0===e?Wt=null:an("syncRect",Wt=v.getBoundingClientRect())}function Gt(e,t,n,r,i,o,a){xe._lock||Ct&&null!=e&&0==e.movementX&&0==e.movementY||(Jt(e,t,n,r,i,o,a,!1,null!=e),null!=e?qt(null,!0,!0):qt(t,!0,!1))}function Jt(e,t,n,i,o,a,u,c,s){if(null==Wt&&Qt(!1),null!=e)n=e.clientX-Wt.left,i=e.clientY-Wt.top;else{if(n<0||i<0)return _t=-10,void(bt=-10);var f=Ft(un.scales,2),d=f[0],h=f[1],p=t.cursor.sync,v=Ft(p.values,2),m=v[0],g=v[1],y=Ft(p.scales,2),_=y[0],b=y[1],w=Ft(un.match,2),x=w[0],k=w[1],C=t.axes[0].side%2==1,A=0==F.ori?ue:le,E=1==F.ori?ue:le,S=C?a:o,N=C?o:a,M=C?i:n,T=C?n:i;if(n=null!=_?x(d,_)?l(m,D[d],A,0):-10:A*(M/S),i=null!=b?k(h,b)?l(g,D[h],E,0):-10:E*(T/N),1==F.ori){var O=n;n=i,i=O}}if(s&&((n<=1||n>=ue-1)&&(n=xl(n,ue)),(i<=1||i>=le-1)&&(i=xl(i,le))),c){ht=n,pt=i;var B=Ft(xe.move(r,n,i),2);vt=B[0],mt=B[1]}else _t=n,bt=i}var Zt={width:0,height:0,left:0,top:0};function Kt(){Lt(Zt,!1)}function Xt(e,t,n,i,o,a,u){Ct=!0,Et=St=At._x=At._y=!1,Jt(e,t,n,i,o,a,0,!0,!1),null!=e&&(re(xu,Fu,en),sn(wu,r,vt,mt,ue,le,null))}function en(e,t,n,i,o,a,u){Ct=At._x=At._y=!1,Jt(e,t,n,i,o,a,0,!1,!0);var l=Ot.left,c=Ot.top,s=Ot.width,f=Ot.height,d=s>0||f>0;if(d&&Lt(Ot),At.setScale&&d){var h=l,p=s,v=c,m=f;if(1==F.ori&&(h=c,p=f,v=l,m=s),Et&&It(x,zt(h,x),zt(h+p,x)),St)for(var g in D){var y=D[g];g!=x&&null==y.from&&y.min!=Dl&&It(g,zt(v+m,g),zt(v,g))}Kt()}else xe.lock&&(xe._lock=!xe._lock,xe._lock||qt(null,!0,!1));null!=e&&(ie(xu,Fu),sn(xu,r,_t,bt,ue,le,null))}function tn(e,t,n,i,o,a,u){Je(),Kt(),null!=e&&sn(Au,r,_t,bt,ue,le,null)}function nn(){b.forEach(tf),De(r.width,r.height,!0)}qu(Su,Mu,nn);var rn={};rn.mousedown=Xt,rn.mousemove=Gt,rn.mouseup=en,rn.dblclick=tn,rn.setSeries=function(e,t,n,r){Pt(n,r,!0,!1)},xe.show&&(re(wu,v,Xt),re(Du,v,Gt),re(ku,v,Qt),re(Cu,v,(function(e,t,n,r,i,o,a){if(!xe._lock){var u=Ct;if(Ct){var l,c,s=!0,f=!0;0==F.ori?(l=Et,c=St):(l=St,c=Et),l&&c&&(s=_t<=10||_t>=ue-10,f=bt<=10||bt>=le-10),l&&s&&(_t=_t=3&&10==i.log?rs:El)),e.font=ef(e.font),e.labelFont=ef(e.labelFont),e._size=e.size(r,null,t,0),e._space=e._rotate=e._incrs=e._found=e._splits=e._values=null,e._size>0&&(Ne[t]=!0,e._el=Pu("u-axis",h))}})),n?n instanceof HTMLElement?(n.appendChild(s),fn()):n(r,fn):fn(),r}nf.assign=Wl,nf.fmtNum=ul,nf.rangeNum=rl,nf.rangeLog=Ku,nf.rangeAsinh=Xu,nf.orient=ms,nf.pxRatio=nu,nf.join=function(e,t){for(var n=new Set,r=0;r=a&&O<=u;O+=S){var B=s[O];if(null!=B){var L=x(c[O]),I=k(B);1==t?C(E,L,N):C(E,M,I),C(E,L,I),N=I,M=L}}var P=M;i&&1==t&&C(E,P=D+w,N);var R=Ft(gs(e,o),2),z=R[0],j=R[1];if(null!=l.fill||0!=z){var $=A.fill=new Path2D(E),H=k(l.fillTo(e,o,l.min,l.max,z));C($,P,H),C($,T,H)}if(!l.spanGaps){var U,Y=[];(U=Y).push.apply(U,Ot(Ds(c,s,a,u,S,x,r)));var V=l.width*nu/2,q=n||1==t?V:-V,W=n||-1==t?-V:V;Y.forEach((function(e){e[0]+=q,e[1]+=W})),A.gaps=Y=l.gaps(e,o,a,u,Y),A.clip=bs(Y,f.ori,v,m,g,y)}return 0!=j&&(A.band=2==j?[_s(e,o,a,u,E,-1),_s(e,o,a,u,E,1)]:_s(e,o,a,u,E,j)),A}))}},rf.bars=function(e){var t=il((e=e||Pl).size,[.6,Dl,1]),n=e.align||0,r=(e.gap||0)*nu,i=il(e.radius,0),o=1-t[0],a=il(t[1],Dl)*nu,u=il(t[2],1)*nu,l=il(e.disp,Pl),c=il(e.each,(function(e){})),s=l.fill,f=l.stroke;return function(e,t,d,h){return ms(e,t,(function(p,v,m,g,y,_,b,D,w,x,k){var C,A,E=p.pxRound,S=g.dir*(0==g.ori?1:-1),N=y.dir*(1==y.ori?1:-1),F=0==g.ori?Ss:Ns,M=0==g.ori?c:function(e,t,n,r,i,o,a){c(e,t,n,i,r,a,o)},T=Ft(gs(e,t),2),O=T[0],B=T[1],L=3==y.distr?1==O?y.max:y.min:0,I=b(L,y,k,w),P=E(p.width*nu),R=!1,z=null,j=null,$=null,H=null;null==s||0!=P&&null==f||(R=!0,z=s.values(e,t,d,h),j=new Map,new Set(z).forEach((function(e){null!=e&&j.set(e,new Path2D)})),P>0&&($=f.values(e,t,d,h),H=new Map,new Set($).forEach((function(e){null!=e&&H.set(e,new Path2D)}))));var U=l.x0,Y=l.size;if(null!=U&&null!=Y){v=U.values(e,t,d,h),2==U.unit&&(v=v.map((function(t){return e.posToVal(D+t*x,g.key,!0)})));var V=Y.values(e,t,d,h);A=E((A=2==Y.unit?V[0]*x:_(V[0],g,x,D)-_(0,g,x,D))-P),C=1==S?-P/2:A+P/2}else{var q=x;if(v.length>1)for(var W=null,Q=0,G=1/0;Q=d&&ie<=h;ie+=S){var oe=m[ie];if(void 0!==oe){var ae=_(2!=g.distr||null!=l?v[ie]:ie,g,x,D),ue=b(il(oe,L),y,k,w);null!=re&&null!=oe&&(I=b(re[ie],y,k,w));var le=E(ae-C),ce=E(vl(ue,I)),se=E(pl(ue,I)),fe=ce-se,de=i*A;null!=oe&&(R?(P>0&&null!=$[ie]&&F(H.get($[ie]),le,se+fl(P/2),A,vl(0,fe-P),de),null!=z[ie]&&F(j.get(z[ie]),le,se+fl(P/2),A,vl(0,fe-P),de)):F(X,le,se+fl(P/2),A,vl(0,fe-P),de),M(e,t,ie,le-P/2,se,A+P,fe)),0!=B&&(N*B==1?(ce=se,se=Z):(se=ce,ce=Z),F(ee,le-P/2,se,A+P,vl(0,fe=ce-se),0))}}return P>0&&(K.stroke=R?H:X),K.fill=R?j:X,K}))}},rf.spline=function(e){return function(e,t){var n=il(null===t||void 0===t?void 0:t.alignGaps,0);return function(t,r,i,o){return ms(t,r,(function(a,u,l,c,s,f,d,h,p,v,m){var g,y,_,b=a.pxRound,D=function(e){return b(f(e,c,v,h))},w=function(e){return b(d(e,s,m,p))};0==c.ori?(g=ks,_=As,y=Ts):(g=Cs,_=Es,y=Os);var x=c.dir*(0==c.ori?1:-1);i=Gu(l,i,o,1),o=Gu(l,i,o,-1);for(var k=D(u[1==x?i:o]),C=k,A=[],E=[],S=1==x?i:o;S>=i&&S<=o;S+=x)if(null!=l[S]){var N=D(u[S]);A.push(C=N),E.push(w(l[S]))}var F={stroke:e(A,E,g,_,y,b),fill:null,clip:null,band:null,gaps:null,flags:1},M=F.stroke,T=Ft(gs(t,r),2),O=T[0],B=T[1];if(null!=a.fill||0!=O){var L=F.fill=new Path2D(M),I=w(a.fillTo(t,r,a.min,a.max,O));_(L,C,I),_(L,k,I)}if(!a.spanGaps){var P,R=[];(P=R).push.apply(P,Ot(Ds(u,l,i,o,x,D,n))),F.gaps=R=a.gaps(t,r,i,o,R),F.clip=bs(R,c.ori,h,p,v,m)}return 0!=B&&(F.band=2==B?[_s(t,r,i,o,M,-1),_s(t,r,i,o,M,1)]:_s(t,r,i,o,M,B)),F}))}}(zs,e)};var of,af={legend:{show:!1},cursor:{drag:{x:!0,y:!1},focus:{prox:30},points:{size:5.6,width:1.4},bind:{click:function(){return null},dblclick:function(){return null}}}},uf=function(e,t,n){if(void 0===e||null===e)return"";n=n||0,t=t||0;var r=Math.abs(n-t);if(isNaN(r)||0==r)return Math.abs(e)>=1e3?e.toLocaleString("en-US"):e.toString();var i=3+Math.floor(1+Math.log10(Math.max(Math.abs(t),Math.abs(n)))-Math.log10(r));return(isNaN(i)||i>20)&&(i=20),e.toLocaleString("en-US",{minimumSignificantDigits:i,maximumSignificantDigits:i})},lf=function(e,t,n,r){var i,o=e.axes[n];if(r>1)return o._size||60;var a=6+((null===o||void 0===o||null===(i=o.ticks)||void 0===i?void 0:i.size)||0)+(o.gap||0),u=(null!==t&&void 0!==t?t:[]).reduce((function(e,t){return t.length>e.length?t:e}),"");return""!=u&&(a+=function(e,t){var n=document.createElement("span");n.innerText=e,n.style.cssText="position: absolute; z-index: -1; pointer-events: none; opacity: 0; font: ".concat(t),document.body.appendChild(n);var r=n.offsetWidth;return n.remove(),r}(u,"10px Arial")),Math.ceil(a)},cf=function(e){var t=e.e,n=e.factor,r=void 0===n?.85:n,i=e.u,o=e.setPanning,a=e.setPlotScale;t.preventDefault();var u=t instanceof MouseEvent;o(!0);var l=u?t.clientX:t.touches[0].clientX,c=i.posToVal(1,"x")-i.posToVal(0,"x"),s=i.scales.x.min||0,f=i.scales.x.max||0,d=function(e){var t=e instanceof MouseEvent;if(t||!(e.touches.length>1)){e.preventDefault();var n=t?e.clientX:e.touches[0].clientX,o=c*((n-l)*r);a({u:i,min:s-o,max:f-o})}},h=function e(){o(!1),document.removeEventListener("mousemove",d),document.removeEventListener("mouseup",e),document.removeEventListener("touchmove",d),document.removeEventListener("touchend",e)};document.addEventListener("mousemove",d),document.addEventListener("mouseup",h),document.addEventListener("touchmove",d),document.addEventListener("touchend",h)},sf=function(e){for(var t=e.length,n=-1/0;t--;){var r=e[t];Number.isFinite(r)&&r>n&&(n=r)}return Number.isFinite(n)?n:null},ff=function(e){for(var t=e.length,n=1/0;t--;){var r=e[t];Number.isFinite(r)&&r2&&void 0!==arguments[2]?arguments[2]:"",r=t[0],i=t[t.length-1];return n?t.map((function(e){return"".concat(uf(e,r,i)," ").concat(n)})):t.map((function(e){return uf(e,r,i)}))}(e,n,t)}};return e?Number(e)%2?n:yr(yr({},n),{},{side:1}):{space:80,values:df,stroke:Rr("color-text")}}))},pf=function(e,t){if(null==e||null==t)return[-1,1];var n=.02*(Math.abs(t-e)||Math.abs(e)||1);return[e-n,t+n]},vf=n(61),mf=n.n(vf),gf=function(e){var t,n,r,i=e.u,o=e.id,a=e.unit,u=void 0===a?"":a,l=e.metrics,c=e.series,s=e.yRange,f=e.tooltipIdx,d=e.tooltipOffset,h=e.isSticky,p=e.onClose,v=ie(null),m=Ft(ee({top:-999,left:-999}),2),y=m[0],_=m[1],b=Ft(ee(!1),2),D=b[0],w=b[1],x=Ft(ee(!1),2),k=x[0],C=x[1],A=Ft(ee(f.seriesIdx),2),E=A[0],S=A[1],N=Ft(ee(f.dataIdx),2),F=N[0],M=N[1],T=ae((function(){return i.root.querySelector(".u-wrap")}),[i]),O=Mr()(i,["data",E,F],0),B=uf(O,Mr()(s,[0]),Mr()(s,[1])),L=i.data[0][F],I=_t()(1e3*L).tz().format("YYYY-MM-DD HH:mm:ss:SSS (Z)"),P=(null===(t=c[E])||void 0===t?void 0:t.stroke)+"",R=(null===(n=c[E])||void 0===n?void 0:n.calculations)||{},z=new Set(l.map((function(e){return e.group}))),j=z.size>1,$=(null===(r=l[E-1])||void 0===r?void 0:r.group)||0,H=ae((function(){var e,t=(null===(e=l[E-1])||void 0===e?void 0:e.metric)||{},n=Object.keys(t).filter((function(e){return"__name__"!=e})),r=n.map((function(e){return"".concat(e,"=").concat(JSON.stringify(t[e]))})),i=t.__name__||"";return r.length>0&&(i+="{"+r.join(",")+"}"),i}),[l,E]),U=function(e){if(D){var t=e.clientX,n=e.clientY;_({top:n,left:t})}},Y=function(){w(!1)};return ne((function(){var e;if(v.current){var t=i.valToPos(O||0,(null===(e=c[E])||void 0===e?void 0:e.scale)||"1"),n=i.valToPos(L,"x"),r=v.current.getBoundingClientRect(),o=r.width,a=r.height,u=i.over.getBoundingClientRect(),l=n+o>=u.width?o+20:0,s=t+a>=u.height?a+20:0,f={top:t+d.top+10-s,left:n+d.left+10-l};f.left<0&&(f.left=20),f.top<0&&(f.top=20),_(f)}}),[i,O,L,E,d,v]),ne((function(){S(f.seriesIdx),M(f.dataIdx)}),[f]),ne((function(){return D&&(document.addEventListener("mousemove",U),document.addEventListener("mouseup",Y)),function(){document.removeEventListener("mousemove",U),document.removeEventListener("mouseup",Y)}}),[D]),!T||f.seriesIdx<0||f.dataIdx<0?null:gt.createPortal(Vr("div",{className:wo()({"vm-chart-tooltip":!0,"vm-chart-tooltip_sticky":h,"vm-chart-tooltip_moved":k}),ref:v,style:y,children:[Vr("div",{className:"vm-chart-tooltip-header",children:[Vr("div",{className:"vm-chart-tooltip-header__date",children:[j&&Vr("div",{children:["Query ",$]}),I]}),h&&Vr(g,{children:[Vr(fa,{className:"vm-chart-tooltip-header__drag",variant:"text",size:"small",startIcon:Vr(fo,{}),onMouseDown:function(e){C(!0),w(!0);var t=e.clientX,n=e.clientY;_({top:n,left:t})}}),Vr(fa,{className:"vm-chart-tooltip-header__close",variant:"text",size:"small",startIcon:Vr(ji,{}),onClick:function(){p&&p(o)}})]})]}),Vr("div",{className:"vm-chart-tooltip-data",children:[Vr("div",{className:"vm-chart-tooltip-data__marker",style:{background:P}}),Vr("div",{children:[Vr("b",{children:[B,u]}),Vr("br",{}),"median:",Vr("b",{children:R.median}),", min:",Vr("b",{children:R.min}),", max:",Vr("b",{children:R.max})]})]}),Vr("div",{className:"vm-chart-tooltip-info",children:H})]}),T)};!function(e){e.xRange="xRange",e.yRange="yRange",e.data="data"}(of||(of={}));var yf=function(e){var t=e.data,n=e.series,r=e.metrics,i=void 0===r?[]:r,o=e.period,a=e.yaxis,u=e.unit,l=e.setPeriod,c=e.container,s=e.height,f=Wr().isDarkTheme,d=ie(null),h=Ft(ee(!1),2),v=h[0],m=h[1],g=Ft(ee({min:o.start,max:o.end}),2),y=g[0],_=g[1],b=Ft(ee([0,1]),2),D=b[0],w=b[1],x=Ft(ee(),2),k=x[0],C=x[1],A=Ft(ee(0),2),E=A[0],S=A[1],N=bo(c),F=Ft(ee(!1),2),M=F[0],T=F[1],O=Ft(ee({seriesIdx:-1,dataIdx:-1}),2),B=O[0],L=O[1],I=Ft(ee({left:0,top:0}),2),P=I[0],R=I[1],z=Ft(ee([]),2),j=z[0],$=z[1],H=ae((function(){return"".concat(B.seriesIdx,"_").concat(B.dataIdx)}),[B]),U=ue(mf()((function(e){var t=e.min,n=e.max;l({from:_t()(1e3*t).toDate(),to:_t()(1e3*n).toDate()})}),500),[]),Y=function(e){var t=e.u,n=e.min,r=e.max,i=1e3*(r-n);iti||(t.setScale("x",{min:n,max:r}),_({min:n,max:r}),U({min:n,max:r}))},V=function(e){var t=e.target,n=e.ctrlKey,r=e.metaKey,i=e.key,o=t instanceof HTMLInputElement||t instanceof HTMLTextAreaElement;if(k&&!o){var a="+"===i||"="===i;if(("-"===i||a)&&!n&&!r){e.preventDefault();var u=(y.max-y.min)/10*(a?1:-1);Y({u:k,min:y.min+u,max:y.max-u})}}},q=function(){var e="".concat(B.seriesIdx,"_").concat(B.dataIdx),t={id:e,unit:u,series:n,metrics:i,yRange:D,tooltipIdx:B,tooltipOffset:P};if(!j.find((function(t){return t.id===e}))){var r=JSON.parse(JSON.stringify(t));$((function(e){return[].concat(Ot(e),[r])}))}},W=function(e){$((function(t){return t.filter((function(t){return t.id!==e}))}))},Q=function(){return[y.min,y.max]},G=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,r=arguments.length>3?arguments[3]:void 0;return"1"==r&&w([t,n]),a.limits.enable?a.limits.range[r]:pf(t,n)},J=yr(yr({},af),{},{tzDate:function(e){return _t()(fi(hi(e))).local().toDate()},series:n,axes:hf([{},{scale:"1"}],u),scales:yr({},function(){var e={x:{range:Q}},t=Object.keys(a.limits.range);return(t.length?t:["1"]).forEach((function(t){e[t]={range:function(e){var n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,r=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1;return G(e,n,r,t)}}})),e}()),width:N.width||400,height:s||500,plugins:[{hooks:{ready:function(e){var t=.9;R({left:parseFloat(e.over.style.left),top:parseFloat(e.over.style.top)}),e.over.addEventListener("mousedown",(function(n){var r=n.ctrlKey,i=n.metaKey;0===n.button&&(r||i)&&cf({u:e,e:n,setPanning:m,setPlotScale:Y,factor:t})})),e.over.addEventListener("touchstart",(function(n){cf({u:e,e:n,setPanning:m,setPlotScale:Y,factor:t})})),e.over.addEventListener("wheel",(function(n){if(n.ctrlKey||n.metaKey){n.preventDefault();var r=e.over.getBoundingClientRect().width,i=e.cursor.left&&e.cursor.left>0?e.cursor.left:0,o=e.posToVal(i,"x"),a=(e.scales.x.max||0)-(e.scales.x.min||0),u=n.deltaY<0?a*t:a/t,l=o-i/r*u,c=l+u;e.batch((function(){return Y({u:e,min:l,max:c})}))}}))},setCursor:function(e){var t,n=null!==(t=e.cursor.idx)&&void 0!==t?t:-1;L((function(e){return yr(yr({},e),{},{dataIdx:n})}))},setSeries:function(e,t){var n=null!==t&&void 0!==t?t:-1;L((function(e){return yr(yr({},e),{},{seriesIdx:n})}))}}}],hooks:{setSelect:[function(e){var t=e.posToVal(e.select.left,"x"),n=e.posToVal(e.select.left+e.select.width,"x");Y({u:e,min:t,max:n})}]}}),Z=function(e){if(k){switch(e){case of.xRange:k.scales.x.range=Q;break;case of.yRange:Object.keys(a.limits.range).forEach((function(e){k.scales[e]&&(k.scales[e].range=function(t){var n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,r=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1;return G(t,n,r,e)})}));break;case of.data:k.setData(t)}v||k.redraw()}};ne((function(){return _({min:o.start,max:o.end})}),[o]),ne((function(){if($([]),L({seriesIdx:-1,dataIdx:-1}),d.current){var e=new nf(J,t,d.current);return C(e),_({min:o.start,max:o.end}),e.destroy}}),[d.current,n,N,s,f]),ne((function(){return window.addEventListener("keydown",V),function(){window.removeEventListener("keydown",V)}}),[y]);var K=function(e){if(2===e.touches.length){e.preventDefault();var t=e.touches[0].clientX-e.touches[1].clientX,n=e.touches[0].clientY-e.touches[1].clientY;S(Math.sqrt(t*t+n*n))}},X=function(e){if(2===e.touches.length&&k){e.preventDefault();var t=e.touches[0].clientX-e.touches[1].clientX,n=e.touches[0].clientY-e.touches[1].clientY,r=Math.sqrt(t*t+n*n),i=E-r,o=k.scales.x.max||y.max,a=k.scales.x.min||y.min,u=(o-a)/50*(i>0?-1:1);k.batch((function(){return Y({u:k,min:a+u,max:o-u})}))}};return ne((function(){return window.addEventListener("touchmove",X),window.addEventListener("touchstart",K),function(){window.removeEventListener("touchmove",X),window.removeEventListener("touchstart",K)}}),[k,E]),ne((function(){return Z(of.data)}),[t]),ne((function(){return Z(of.xRange)}),[y]),ne((function(){return Z(of.yRange)}),[a]),ne((function(){var e=-1!==B.dataIdx&&-1!==B.seriesIdx;return T(e),e&&window.addEventListener("click",q),function(){window.removeEventListener("click",q)}}),[B,j]),Vr("div",{className:wo()({"vm-line-chart":!0,"vm-line-chart_panning":v}),style:{minWidth:"".concat(N.width||400,"px"),minHeight:"".concat(s||500,"px")},children:[Vr("div",{className:"vm-line-chart__u-plot",ref:d}),k&&M&&Vr(gf,{unit:u,u:k,series:n,metrics:i,yRange:D,tooltipIdx:B,tooltipOffset:P,id:H}),k&&j.map((function(e){return p(gf,yr(yr({},e),{},{isSticky:!0,u:k,key:e.id,onClose:W}))}))]})},_f=function(e){var t=e.legend,n=e.onChange,r=Ft(ee(""),2),i=r[0],o=r[1],a=ae((function(){return function(e){var t=Object.keys(e.freeFormFields).filter((function(e){return"__name__"!==e}));return t.map((function(t){var n="".concat(t,"=").concat(JSON.stringify(e.freeFormFields[t]));return{id:"".concat(e.label,".").concat(n),freeField:n,key:t}}))}(t)}),[t]),u=t.calculations,l=function(){var e=tu(Xa().mark((function e(t,n){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(t);case 2:o(n),setTimeout((function(){return o("")}),2e3);case 4:case"end":return e.stop()}}),e)})));return function(t,n){return e.apply(this,arguments)}}();return Vr("div",{className:wo()({"vm-legend-item":!0,"vm-legend-row":!0,"vm-legend-item_hide":!t.checked}),onClick:function(e){return function(t){n(e,t.ctrlKey||t.metaKey)}}(t),children:[Vr("div",{className:"vm-legend-item__marker",style:{backgroundColor:t.color}}),Vr("div",{className:"vm-legend-item-info",children:Vr("span",{className:"vm-legend-item-info__label",children:[t.freeFormFields.__name__,"{",a.map((function(e,t){return Vr(ma,{open:i===e.id,title:"copied!",placement:"top-center",children:Vr("span",{className:"vm-legend-item-info__free-fields",onClick:(n=e.freeField,r=e.id,function(e){e.stopPropagation(),l(n,r)}),title:"copy to clipboard",children:[e.freeField,t+11;return Vr(g,{children:Vr("div",{className:"vm-legend",children:i.map((function(e){return Vr("div",{className:"vm-legend-group",children:[Vr("div",{className:"vm-legend-group-title",children:[o&&Vr("span",{className:"vm-legend-group-title__count",children:["Query ",e,": "]}),Vr("span",{className:"vm-legend-group-title__query",children:n[e-1]})]}),Vr("div",{children:t.filter((function(t){return t.group===e})).map((function(e){return Vr(_f,{legend:e,onChange:r},e.label)}))})]},e)}))})})},Df=["__name__"],wf=function(e,t){var n=!(arguments.length>2&&void 0!==arguments[2])||arguments[2],r=e.metric,i=r.__name__,o=xo(r,Df),a=t||"".concat(n?"[Query ".concat(e.group,"] "):"").concat(i||"");return 0==Object.keys(o).length?a||"value":"".concat(a,"{").concat(Object.entries(o).map((function(e){return"".concat(e[0],"=").concat(JSON.stringify(e[1]))})).join(", "),"}")},xf=function(e){switch(e){case"NaN":return NaN;case"Inf":case"+Inf":return 1/0;case"-Inf":return-1/0;default:return parseFloat(e)}},kf=["#e54040","#32a9dc","#2ee329","#7126a1","#e38f0f","#3d811a","#ffea00","#2d2d2d","#da42a6","#a44e0c"],Cf=function(e){var t=16777215,n=1,r=0,i=1;if(e.length>0)for(var o=0;or&&(r=e[o].charCodeAt(0)),i=parseInt(String(t/r)),n=(n+e[o].charCodeAt(0)*i*49979693)%t;var a=(n*e.length%t).toString(16);return a=a.padEnd(6,a),"#".concat(a)},Af=function(){var e={};return function(t,n,r){var i=wf(t,r[t.group-1]),o=Object.keys(e).length;o>1]}(a),s=function(e){for(var t=e.length;t--;){var n=e[t];if(Number.isFinite(n))return n}}(a);return{label:i,freeFormFields:t.metric,width:1.4,stroke:e[i]||Cf(i),show:!Sf(i,n),scale:"1",points:{size:4.2,width:1.4},calculations:{min:uf(u,u,l),max:uf(l,u,l),median:uf(c,u,l),last:uf(s,u,l)}}}},Ef=function(e,t){return{group:t,label:e.label||"",color:e.stroke,checked:e.show||!1,freeFormFields:e.freeFormFields,calculations:e.calculations}},Sf=function(e,t){return t.includes("".concat(e))},Nf=function(e){var t=e.data,n=void 0===t?[]:t,r=e.period,i=e.customStep,o=e.query,a=e.yaxis,u=e.unit,l=e.showLegend,c=void 0===l||l,s=e.setYaxisLimits,f=e.setPeriod,d=e.alias,h=void 0===d?[]:d,p=e.fullWidth,v=void 0===p||p,m=e.height,g=Xo().isMobile,y=Ni().timezone,_=ae((function(){return i||r.step||"1s"}),[r.step,i]),b=ue(Af(),[n]),D=Ft(ee([[]]),2),w=D[0],x=D[1],k=Ft(ee([]),2),C=k[0],A=k[1],E=Ft(ee([]),2),S=E[0],N=E[1],F=Ft(ee([]),2),M=F[0],T=F[1],O=function(e){var t=function(e){var t={},n=Object.values(e).flat(),r=ff(n),i=sf(n);return t[1]=pf(r,i),t}(e);s(t)};ne((function(){var e=[],t={},i=[],o=[{}];null===n||void 0===n||n.forEach((function(n){var r=b(n,M,h);o.push(r),i.push(Ef(r,n.group));var a,u=t[n.group]||[],l=Mt(n.values);try{for(l.s();!(a=l.n()).done;){var c=a.value;e.push(c[0]),u.push(xf(c[1]))}}catch(s){l.e(s)}finally{l.f()}t[n.group]=u}));var a=function(e,t,n){for(var r=li(t)||1,i=Array.from(new Set(e)).sort((function(e,t){return e-t})),o=n.start,a=ai(n.end+r),u=0,l=[];o<=a;){for(;u=i.length||i[u]>o)&&l.push(o)}for(;l.length<2;)l.push(o),o=ai(o+r);return l}(e,_,r),u=n.map((function(e){var t,n=[],r=e.values,i=r.length,o=0,u=Mt(a);try{for(u.s();!(t=u.n()).done;){for(var l=t.value;o1e10*h?n.map((function(){return f})):n}));u.unshift(a),O(t),x(u),A(o),N(i)}),[n,y]),ne((function(){var e=[],t=[{}];null===n||void 0===n||n.forEach((function(n){var r=b(n,M,h);t.push(r),e.push(Ef(r,n.group))})),A(t),N(e)}),[M]);var B=ie(null);return Vr("div",{className:wo()({"vm-graph-view":!0,"vm-graph-view_full-width":v,"vm-graph-view_full-width_mobile":v&&g}),ref:B,children:[(null===B||void 0===B?void 0:B.current)&&Vr(yf,{data:w,series:C,metrics:n,period:r,yaxis:a,unit:u,setPeriod:f,container:null===B||void 0===B?void 0:B.current,height:m}),c&&Vr(bf,{labels:S,query:o,onChange:function(e,t){T(function(e){var t=e.hideSeries,n=e.legend,r=e.metaKey,i=e.series,o=n.label,a=Sf(o,t),u=i.map((function(e){return e.label||""}));return r?a?t.filter((function(e){return e!==o})):[].concat(Ot(t),[o]):t.length?a?Ot(u.filter((function(e){return e!==o}))):[]:Ot(u.filter((function(e){return e!==o})))}({hideSeries:M,legend:e,metaKey:t,series:C}))}})]})},Ff=function(e){var t=e.value,n=e.options,r=e.anchor,i=e.disabled,o=e.maxWords,a=void 0===o?1:o,u=e.minLength,l=void 0===u?2:u,c=e.fullWidth,f=e.selected,d=e.noOptionsText,h=e.label,p=e.disabledFullScreen,v=e.onSelect,m=e.onOpenAutocomplete,g=Xo().isMobile,y=ie(null),_=Ft(ee(!1),2),b=_[0],D=_[1],w=Ft(ee(-1),2),x=w[0],k=w[1],C=ae((function(){if(!b)return[];try{var e=new RegExp(String(t),"i");return n.filter((function(n){return e.test(n)&&n!==t})).sort((function(t,n){var r,i;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(i=n.match(e))||void 0===i?void 0:i.index)||0)}))}catch(s){return[]}}),[b,n,t]),A=ae((function(){return d&&!C.length}),[d,C]),E=function(){D(!1)},S=function(e){var t=e.key,n=e.ctrlKey,r=e.metaKey,i=e.shiftKey,o=n||r||i,a=C.length;if("ArrowUp"===t&&!o&&a&&(e.preventDefault(),k((function(e){return e<=0?0:e-1}))),"ArrowDown"===t&&!o&&a){e.preventDefault();var u=C.length-1;k((function(e){return e>=u?u:e+1}))}if("Enter"===t){var l=C[x];l&&v(l),f||E()}"Escape"===t&&E()};return ne((function(){var e=(t.match(/[a-zA-Z_:.][a-zA-Z0-9_:.]*/gm)||[]).length;D(t.length>l&&e<=a)}),[t]),ne((function(){return function(){if(y.current){var e=y.current.childNodes[x];null!==e&&void 0!==e&&e.scrollIntoView&&e.scrollIntoView({block:"center"})}}(),window.addEventListener("keydown",S),function(){window.removeEventListener("keydown",S)}}),[x,C]),ne((function(){k(-1)}),[C]),ne((function(){m&&m(b)}),[b]),Vr(da,{open:b,buttonRef:r,placement:"bottom-left",onClose:E,fullWidth:c,title:g?h:void 0,disabledFullScreen:p,children:Vr("div",{className:wo()({"vm-autocomplete":!0,"vm-autocomplete_mobile":g&&!p}),ref:y,children:[A&&Vr("div",{className:"vm-autocomplete__no-options",children:d}),C.map((function(e,t){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":g,"vm-list-item_active":t===x,"vm-list-item_multiselect":f,"vm-list-item_multiselect_selected":null===f||void 0===f?void 0:f.includes(e)}),id:"$autocomplete$".concat(e),onClick:(n=e,function(){i||(v(n),f||E())}),children:[(null===f||void 0===f?void 0:f.includes(e))&&Vr(uo,{}),Vr("span",{children:e})]},e);var n}))]})})},Mf=function(e){var t=e.value,n=e.onChange,r=e.onEnter,i=e.onArrowUp,o=e.onArrowDown,a=e.autocomplete,u=e.error,l=e.options,c=e.label,s=e.disabled,f=void 0!==s&&s,d=Ft(ee(!1),2),h=d[0],p=d[1],v=ie(null);return Vr("div",{className:"vm-query-editor",ref:v,children:[Vr(xa,{value:t,label:c,type:"textarea",autofocus:!!t,error:u,onKeyDown:function(e){var t=e.key,n=e.ctrlKey,a=e.metaKey,u=e.shiftKey,l=n||a,c="ArrowDown"===t,s="Enter"===t;"ArrowUp"===t&&l&&(e.preventDefault(),i()),c&&l&&(e.preventDefault(),o()),!s||u||h||r()},onChange:n,disabled:f,inputmode:"search"}),a&&Vr(Ff,{disabledFullScreen:!0,value:t,options:l,anchor:v,onSelect:function(e){n(e)},onOpenAutocomplete:p})]})},Tf=function(e){var t,n=e.value,r=void 0!==n&&n,i=e.disabled,o=void 0!==i&&i,a=e.label,u=e.color,l=void 0===u?"secondary":u,c=e.fullWidth,s=e.onChange;return Vr("div",{className:wo()((mr(t={"vm-switch":!0,"vm-switch_full-width":c,"vm-switch_disabled":o,"vm-switch_active":r},"vm-switch_".concat(l,"_active"),r),mr(t,"vm-switch_".concat(l),l),t)),onClick:function(){o||s(!r)},children:[Vr("div",{className:"vm-switch-track",children:Vr("div",{className:"vm-switch-track__thumb"})}),a&&Vr("span",{className:"vm-switch__label",children:a})]})},Of=function(e){var t=e.isMobile,n=Li().autocomplete,r=Ii(),i=Io(),o=i.nocache,a=i.isTracingEnabled,u=Po();return Vr("div",{className:wo()({"vm-additional-settings":!0,"vm-additional-settings_mobile":t}),children:[Vr(Tf,{label:"Autocomplete",value:n,onChange:function(){r({type:"TOGGLE_AUTOCOMPLETE"})},fullWidth:t}),Vr(Tf,{label:"Disable cache",value:o,onChange:function(){u({type:"TOGGLE_NO_CACHE"})},fullWidth:t}),Vr(Tf,{label:"Trace query",value:a,onChange:function(){u({type:"TOGGLE_QUERY_TRACING"})},fullWidth:t})]})},Bf=function(){var e=Xo().isMobile,t=Ft(ee(!1),2),n=t[0],r=t[1],i=ie(null);return e?Vr(g,{children:[Vr("div",{ref:i,children:Vr(fa,{variant:"outlined",startIcon:Vr(_o,{}),onClick:function(){r((function(e){return!e}))}})}),Vr(da,{open:n,buttonRef:i,placement:"bottom-left",onClose:function(){r(!1)},title:"Query settings",children:Vr(Of,{isMobile:e})})]}):Vr(Of,{})},Lf=function(e,t){return e.length===t.length&&e.every((function(e,n){return e===t[n]}))},If=function(e){var t=e.error,n=e.queryOptions,r=e.onHideQuery,i=Xo().isMobile,o=Li(),a=o.query,u=o.queryHistory,l=o.autocomplete,c=Ii(),s=Fi(),f=Ft(ee(a||[]),2),d=f[0],h=f[1],p=Ft(ee([]),2),v=p[0],m=p[1],g=Aa(d),y=function(){c({type:"SET_QUERY_HISTORY",payload:d.map((function(e,t){var n=u[t]||{values:[]},r=e===n.values[n.values.length-1];return{index:n.values.length-Number(r),values:!r&&e?[].concat(Ot(n.values),[e]):n.values}}))}),c({type:"SET_QUERY",payload:d}),s({type:"RUN_QUERY"})},_=function(e,t){h((function(n){return n.map((function(n,r){return r===t?e:n}))}))},b=function(e,t){return function(){!function(e,t){var n=u[t],r=n.index,i=n.values,o=r+e;o<0||o>=i.length||(_(i[o]||"",t),c({type:"SET_QUERY_HISTORY_BY_INDEX",payload:{value:{values:i,index:o},queryNumber:t}}))}(e,t)}},D=function(e){return function(t){_(t,e)}},w=function(e){return function(){var t;t=e,h((function(e){return e.filter((function(e,n){return n!==t}))})),m((function(t){return t.includes(e)?t.filter((function(t){return t!==e})):t.map((function(t){return t>e?t-1:t}))}))}},x=function(e){return function(t){!function(e,t){var n=e.ctrlKey,r=e.metaKey;if(n||r){var i=d.map((function(e,t){return t})).filter((function(e){return e!==t}));m((function(e){return Lf(i,e)?[]:i}))}else m((function(e){return e.includes(t)?e.filter((function(e){return e!==t})):[].concat(Ot(e),[t])}))}(t,e)}};return ne((function(){g&&d.length1&&Vr(ma,{title:"Remove Query",children:Vr("div",{className:"vm-query-configurator-list-row__button",children:Vr(fa,{variant:"text",color:"error",startIcon:Vr(io,{}),onClick:w(r)})})})]},r)}))}),Vr("div",{className:"vm-query-configurator-settings",children:[Vr(Bf,{}),Vr("div",{className:"vm-query-configurator-settings__buttons",children:[d.length<4&&Vr(fa,{variant:"outlined",onClick:function(){h((function(e){return[].concat(Ot(e),[""])}))},startIcon:Vr(oo,{}),children:"Add Query"}),Vr(fa,{variant:"contained",onClick:y,startIcon:Vr(Xi,{}),children:i?"Execute":"Execute Query"})]})]})]})};function Pf(e){var t,n,r,i=2;for("undefined"!=typeof Symbol&&(n=Symbol.asyncIterator,r=Symbol.iterator);i--;){if(n&&null!=(t=e[n]))return t.call(e);if(r&&null!=(t=e[r]))return new Rf(t.call(e));n="@@asyncIterator",r="@@iterator"}throw new TypeError("Object is not async iterable")}function Rf(e){function t(e){if(Object(e)!==e)return Promise.reject(new TypeError(e+" is not an object."));var t=e.done;return Promise.resolve(e.value).then((function(e){return{value:e,done:t}}))}return Rf=function(e){this.s=e,this.n=e.next},Rf.prototype={s:null,n:null,next:function(){return t(this.n.apply(this.s,arguments))},return:function(e){var n=this.s.return;return void 0===n?Promise.resolve({value:e,done:!0}):t(n.apply(this.s,arguments))},throw:function(e){var n=this.s.return;return void 0===n?Promise.reject(e):t(n.apply(this.s,arguments))}},new Rf(e)}var zf=n(936),jf=n.n(zf),$f=0,Hf=function(){function e(t,n){Bt(this,e),this.tracing=void 0,this.query=void 0,this.tracingChildren=void 0,this.originalTracing=void 0,this.id=void 0,this.tracing=t,this.originalTracing=JSON.parse(JSON.stringify(t)),this.query=n,this.id=$f++;var r=t.children||[];this.tracingChildren=r.map((function(t){return new e(t,n)}))}return Rt(e,[{key:"queryValue",get:function(){return this.query}},{key:"idValue",get:function(){return this.id}},{key:"children",get:function(){return this.tracingChildren}},{key:"message",get:function(){return this.tracing.message}},{key:"duration",get:function(){return this.tracing.duration_msec}},{key:"JSON",get:function(){return JSON.stringify(this.tracing,null,2)}},{key:"originalJSON",get:function(){return JSON.stringify(this.originalTracing,null,2)}},{key:"setTracing",value:function(t){var n=this;this.tracing=t;var r=t.children||[];this.tracingChildren=r.map((function(t){return new e(t,n.query)}))}},{key:"setQuery",value:function(e){this.query=e}},{key:"resetTracing",value:function(){this.tracing=this.originalTracing}}]),e}(),Uf=function(e){var t=e.predefinedQuery,n=e.visible,r=e.display,i=e.customStep,o=e.hideQuery,a=e.showAllSeries,u=Li().query,l=Ni().period,c=Io(),s=c.displayType,f=c.nocache,d=c.isTracingEnabled,h=c.seriesLimits,p=Wr().serverUrl,v=Ft(ee(!1),2),m=v[0],g=v[1],y=Ft(ee(),2),_=y[0],b=y[1],D=Ft(ee(),2),w=D[0],x=D[1],k=Ft(ee(),2),C=k[0],A=k[1],E=Ft(ee(),2),S=E[0],N=E[1],F=Ft(ee(),2),M=F[0],T=F[1],O=Ft(ee([]),2),B=O[0],L=O[1];ne((function(){S&&(b(void 0),x(void 0),A(void 0))}),[S]);var I=function(){var e=tu(Xa().mark((function e(t){var n,r,i,o,a,u,l,c,s,f,d,h,p,v,m,y,_,D,w,k,C,E,S,F,M;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:n=t.fetchUrl,r=t.fetchQueue,i=t.displayType,o=t.query,a=t.stateSeriesLimits,u=t.showAllSeries,l=t.hideQuery,c=new AbortController,L([].concat(Ot(r),[c])),e.prev=3,s="chart"===i,f=u?1/0:a[i],d=[],h=[],p=1,v=0,m=!1,y=!1,e.prev=12,D=Pf(n);case 14:return e.next=16,D.next();case 16:if(!(m=!(w=e.sent).done)){e.next=32;break}if(k=w.value,!(null===l||void 0===l?void 0:l.includes(p-1))){e.next=22;break}return p++,e.abrupt("continue",29);case 22:return e.next=24,fetch(k,{signal:c.signal});case 24:return C=e.sent,e.next=27,C.json();case 27:E=e.sent,C.ok?(N(void 0),E.trace&&(S=new Hf(E.trace,o[p-1]),h.push(S)),F=f-d.length,E.data.result.slice(0,F).forEach((function(e){e.group=p,d.push(e)})),v+=E.data.result.length,p++):N("".concat(E.errorType,"\r\n").concat(null===E||void 0===E?void 0:E.error));case 29:m=!1,e.next=14;break;case 32:e.next=38;break;case 34:e.prev=34,e.t0=e.catch(12),y=!0,_=e.t0;case 38:if(e.prev=38,e.prev=39,!m||null==D.return){e.next=43;break}return e.next=43,D.return();case 43:if(e.prev=43,!y){e.next=46;break}throw _;case 46:return e.finish(43);case 47:return e.finish(38);case 48:M="Showing ".concat(f," series out of ").concat(v," series due to performance reasons. Please narrow down the query, so it returns less series"),T(v>f?M:""),s?b(d):x(d),A(h),e.next=57;break;case 54:e.prev=54,e.t1=e.catch(3),e.t1 instanceof Error&&"AbortError"!==e.t1.name&&N("".concat(e.t1.name,": ").concat(e.t1.message));case 57:g(!1);case 58:case"end":return e.stop()}}),e,null,[[3,54],[12,34,38,48],[39,,43,47]])})));return function(t){return e.apply(this,arguments)}}(),P=ue(jf()(I,300),[]),R=ae((function(){var e=null!==t&&void 0!==t?t:u,n="chart"===(r||s);if(l)if(p)if(e.every((function(e){return!e.trim()})))N(_r.validQuery);else{if(Ha(p)){var o=yr({},l);return o.step=i,e.map((function(e){return n?function(e,t,n,r,i){return"".concat(e,"/api/v1/query_range?query=").concat(encodeURIComponent(t),"&start=").concat(n.start,"&end=").concat(n.end,"&step=").concat(n.step).concat(r?"&nocache=1":"").concat(i?"&trace=1":"")}(p,e,o,f,d):function(e,t,n,r){return"".concat(e,"/api/v1/query?query=").concat(encodeURIComponent(t),"&time=").concat(n.end,"&step=").concat(n.step).concat(r?"&trace=1":"")}(p,e,o,d)}))}N(_r.validServer)}else N(_r.emptyServer)}),[p,l,s,i,o]),z=Ft(ee([]),2),j=z[0],$=z[1];return ne((function(){var e=R===j&&!!t;n&&null!==R&&void 0!==R&&R.length&&!e&&(g(!0),P({fetchUrl:R,fetchQueue:B,displayType:r||s,query:null!==t&&void 0!==t?t:u,stateSeriesLimits:h,showAllSeries:a,hideQuery:o}),$(R))}),[R,n,h,a]),ne((function(){var e=B.slice(0,-1);e.length&&(e.map((function(e){return e.abort()})),L(B.filter((function(e){return!e.signal.aborted}))))}),[B]),{fetchUrl:R,isLoading:m,graphData:_,liveData:w,error:S,warning:M,traces:C}},Yf=function(e){var t=e.data,n=ra().showInfoMessage,r=ae((function(){return JSON.stringify(t,null,2)}),[t]);return Vr("div",{className:"vm-json-view",children:[Vr("div",{className:"vm-json-view__copy",children:Vr(fa,{variant:"outlined",onClick:function(){navigator.clipboard.writeText(r),n({text:"Formatted JSON has been copied",type:"success"})},children:"Copy JSON"})}),Vr("pre",{className:"vm-json-view__code",children:Vr("code",{children:r})})]})},Vf=function(e){var t=e.yaxis,n=e.setYaxisLimits,r=e.toggleEnableLimits,i=Xo().isMobile,o=ae((function(){return Object.keys(t.limits.range)}),[t.limits.range]),a=ue(jf()((function(e,r,i){var o=t.limits.range;o[r][i]=+e,o[r][0]===o[r][1]||o[r][0]>o[r][1]||n(o)}),500),[t.limits.range]),u=function(e,t){return function(n){a(n,e,t)}};return Vr("div",{className:wo()({"vm-axes-limits":!0,"vm-axes-limits_mobile":i}),children:[Vr(Tf,{value:t.limits.enable,onChange:r,label:"Fix the limits for y-axis",fullWidth:i}),Vr("div",{className:"vm-axes-limits-list",children:o.map((function(e){return Vr("div",{className:"vm-axes-limits-list__inputs",children:[Vr(xa,{label:"Min ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][0],onChange:u(e,0)}),Vr(xa,{label:"Max ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][1],onChange:u(e,1)})]},e)}))})]})},qf="Axes settings",Wf=function(e){var t=e.yaxis,n=e.setYaxisLimits,r=e.toggleEnableLimits,i=ie(null),o=Ft(ee(!1),2),a=o[0],u=o[1],l=ie(null);return Vr("div",{className:"vm-graph-settings",children:[Vr(ma,{title:qf,children:Vr("div",{ref:l,children:Vr(fa,{variant:"text",startIcon:Vr(zi,{}),onClick:function(){u((function(e){return!e}))}})})}),Vr(da,{open:a,buttonRef:l,placement:"bottom-right",onClose:function(){u(!1)},title:qf,children:Vr("div",{className:"vm-graph-settings-popper",ref:i,children:Vr("div",{className:"vm-graph-settings-popper__body",children:Vr(Vf,{yaxis:t,setYaxisLimits:n,toggleEnableLimits:r})})})})]})},Qf=function(e){var t=e.containerStyles,n=void 0===t?{}:t,r=e.message,i=Wr().isDarkTheme;return Vr("div",{className:wo()({"vm-spinner":!0,"vm-spinner_dark":i}),style:n&&{},children:[Vr("div",{className:"half-circle-spinner",children:[Vr("div",{className:"circle circle-1"}),Vr("div",{className:"circle circle-2"})]}),r&&Vr("div",{className:"vm-spinner__message",children:r})]})},Gf=function(){var e=Wr().serverUrl,t=Ft(ee([]),2),n=t[0],r=t[1],i=function(){var t=tu(Xa().mark((function t(){var n,i,o;return Xa().wrap((function(t){for(;;)switch(t.prev=t.next){case 0:if(e){t.next=2;break}return t.abrupt("return");case 2:return n="".concat(e,"/api/v1/label/__name__/values"),t.prev=3,t.next=6,fetch(n);case 6:return i=t.sent,t.next=9,i.json();case 9:o=t.sent,i.ok&&r(o.data),t.next=16;break;case 13:t.prev=13,t.t0=t.catch(3),console.error(t.t0);case 16:case"end":return t.stop()}}),t,null,[[3,13]])})));return function(){return t.apply(this,arguments)}}();return ne((function(){i()}),[e]),{queryOptions:n}},Jf=function(e){var t=e.value;return Vr("div",{className:"vm-line-progress",children:[Vr("div",{className:"vm-line-progress-track",children:Vr("div",{className:"vm-line-progress-track__thumb",style:{width:"".concat(t,"%")}})}),Vr("span",{children:[t.toFixed(2),"%"]})]})},Zf=function e(t){var n,r=t.trace,i=t.totalMsec,o=Wr().isDarkTheme,a=Xo().isMobile,u=Ft(ee({}),2),l=u[0],c=u[1],s=r.children&&!!r.children.length,f=r.duration/i*100;return Vr("div",{className:wo()({"vm-nested-nav":!0,"vm-nested-nav_dark":o,"vm-nested-nav_mobile":a}),children:[Vr("div",{className:"vm-nested-nav-header",onClick:(n=r.idValue,function(){c((function(e){return yr(yr({},e),{},mr({},n,!e[n]))}))}),children:[s&&Vr("div",{className:wo()({"vm-nested-nav-header__icon":!0,"vm-nested-nav-header__icon_open":l[r.idValue]}),children:Vr(Wi,{})}),Vr("div",{className:"vm-nested-nav-header__progress",children:Vr(Jf,{value:f})}),Vr("div",{className:"vm-nested-nav-header__message",children:r.message}),Vr("div",{className:"vm-nested-nav-header__duration",children:"duration: ".concat(r.duration," ms")})]}),l[r.idValue]&&Vr("div",{children:s&&r.children.map((function(t){return Vr(e,{trace:t,totalMsec:i},t.duration)}))})]})},Kf=function(e){var t=e.editable,n=void 0!==t&&t,r=e.defaultTile,i=void 0===r?"JSON":r,o=e.displayTitle,a=void 0===o||o,u=e.defaultJson,l=void 0===u?"":u,c=e.resetValue,f=void 0===c?"":c,d=e.onClose,h=e.onUpload,p=ra().showInfoMessage,v=Xo().isMobile,m=Ft(ee(l),2),g=m[0],y=m[1],_=Ft(ee(i),2),b=_[0],D=_[1],w=Ft(ee(""),2),x=w[0],k=w[1],C=Ft(ee(""),2),A=C[0],E=C[1],S=ae((function(){try{var e=JSON.parse(g),t=e.trace||e;return t.duration_msec?(new Hf(t,""),""):_r.traceNotFound}catch(s){return s instanceof Error?s.message:"Unknown error"}}),[g]),N=function(){var e=tu(Xa().mark((function e(){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(g);case 2:p({text:"Formatted JSON has been copied",type:"success"});case 3:case"end":return e.stop()}}),e)})));return function(){return e.apply(this,arguments)}}(),F=function(){E(S),b.trim()||k(_r.emptyTitle),S||x||(h(g,b),d())};return Vr("div",{className:wo()({"vm-json-form":!0,"vm-json-form_one-field":!a,"vm-json-form_one-field_mobile":!a&&v,"vm-json-form_mobile":v}),children:[a&&Vr(xa,{value:b,label:"Title",error:x,onEnter:F,onChange:function(e){D(e)}}),Vr(xa,{value:g,label:"JSON",type:"textarea",error:A,autofocus:!0,onChange:function(e){E(""),y(e)},disabled:!n}),Vr("div",{className:"vm-json-form-footer",children:[Vr("div",{className:"vm-json-form-footer__controls",children:[Vr(fa,{variant:"outlined",startIcon:Vr(so,{}),onClick:N,children:"Copy JSON"}),f&&Vr(fa,{variant:"text",startIcon:Vr($i,{}),onClick:function(){y(f)},children:"Reset JSON"})]}),Vr("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Vr(fa,{variant:"outlined",color:"error",onClick:d,children:"Cancel"}),Vr(fa,{variant:"contained",onClick:F,children:"apply"})]})]})]})},Xf=function(e){var t=e.traces,n=e.jsonEditor,r=void 0!==n&&n,i=e.onDeleteClick,o=Xo().isMobile,a=Ft(ee(null),2),u=a[0],l=a[1],c=function(){l(null)};if(!t.length)return Vr(ta,{variant:"info",children:"Please re-run the query to see results of the tracing"});var f=function(e){return function(){i(e)}};return Vr(g,{children:[Vr("div",{className:"vm-tracings-view",children:t.map((function(e){return Vr("div",{className:"vm-tracings-view-trace vm-block vm-block_empty-padding",children:[Vr("div",{className:"vm-tracings-view-trace-header",children:[Vr("h3",{className:"vm-tracings-view-trace-header-title",children:["Trace for ",Vr("b",{className:"vm-tracings-view-trace-header-title__query",children:e.queryValue})]}),Vr(ma,{title:"Open JSON",children:Vr(fa,{variant:"text",startIcon:Vr(ro,{}),onClick:(t=e,function(){l(t)})})}),Vr(ma,{title:"Remove trace",children:Vr(fa,{variant:"text",color:"error",startIcon:Vr(io,{}),onClick:f(e)})})]}),Vr("nav",{className:wo()({"vm-tracings-view-trace__nav":!0,"vm-tracings-view-trace__nav_mobile":o}),children:Vr(Zf,{trace:e,totalMsec:e.duration})})]},e.idValue);var t}))}),u&&Vr(va,{title:u.queryValue,onClose:c,children:Vr(Kf,{editable:r,displayTitle:r,defaultTile:u.queryValue,defaultJson:u.JSON,resetValue:u.originalJSON,onClose:c,onUpload:function(e,t){if(r&&u)try{u.setTracing(JSON.parse(e)),u.setQuery(t),l(null)}catch(s){console.error(s)}}})})]})},ed=function(e,t){return ae((function(){var n={};e.forEach((function(e){return Object.entries(e.metric).forEach((function(e){return n[e[0]]?n[e[0]].options.add(e[1]):n[e[0]]={options:new Set([e[1]])}}))}));var r=Object.entries(n).map((function(e){return{key:e[0],variations:e[1].options.size}})).sort((function(e,t){return e.variations-t.variations}));return t?r.filter((function(e){return t.includes(e.key)})):r}),[e,t])},td=function(e){var t,n=e.checked,r=void 0!==n&&n,i=e.disabled,o=void 0!==i&&i,a=e.label,u=e.color,l=void 0===u?"secondary":u,c=e.onChange;return Vr("div",{className:wo()((mr(t={"vm-checkbox":!0,"vm-checkbox_disabled":o,"vm-checkbox_active":r},"vm-checkbox_".concat(l,"_active"),r),mr(t,"vm-checkbox_".concat(l),l),t)),onClick:function(){o||c(!r)},children:[Vr("div",{className:"vm-checkbox-track",children:Vr("div",{className:"vm-checkbox-track__thumb",children:Vr(uo,{})})}),a&&Vr("span",{className:"vm-checkbox__label",children:a})]})},nd="Table settings",rd=function(e){var t=e.data,n=e.defaultColumns,r=void 0===n?[]:n,i=e.onChange,o=Xo().isMobile,a=Io().tableCompact,u=Po(),l=ed(t),c=ie(null),s=Ft(ee(!1),2),f=s[0],d=s[1],h=ae((function(){return!l.length}),[l]),p=function(e){return function(){!function(e){i(r.includes(e)?r.filter((function(t){return t!==e})):[].concat(Ot(r),[e]))}(e)}};return ne((function(){var e=l.map((function(e){return e.key}));Lf(e,r)||i(e)}),[l]),Vr("div",{className:"vm-table-settings",children:[Vr(ma,{title:nd,children:Vr("div",{ref:c,children:Vr(fa,{variant:"text",startIcon:Vr(zi,{}),onClick:function(){d((function(e){return!e}))},disabled:h})})}),Vr(da,{open:f,onClose:function(){d(!1)},placement:"bottom-right",buttonRef:c,title:nd,children:Vr("div",{className:wo()({"vm-table-settings-popper":!0,"vm-table-settings-popper_mobile":o}),children:[Vr("div",{className:"vm-table-settings-popper-list vm-table-settings-popper-list_first",children:Vr(Tf,{label:"Compact view",value:a,onChange:function(){u({type:"TOGGLE_TABLE_COMPACT"})}})}),Vr("div",{className:"vm-table-settings-popper-list",children:[Vr("div",{className:"vm-table-settings-popper-list-header",children:[Vr("h3",{className:"vm-table-settings-popper-list-header__title",children:"Display columns"}),Vr(ma,{title:"Reset to default",children:Vr(fa,{color:"primary",variant:"text",size:"small",onClick:function(){d(!1),i(l.map((function(e){return e.key})))},startIcon:Vr($i,{})})})]}),l.map((function(e){return Vr("div",{className:"vm-table-settings-popper-list__item",children:Vr(td,{checked:r.includes(e.key),onChange:p(e.key),label:e.key,disabled:a})},e.key)}))]})]})})]})};function id(e){return function(e,t){return Object.fromEntries(Object.entries(e).filter(t))}(e,(function(e){return!!e[1]}))}var od=["__name__"],ad=function(e){var t=e.data,n=e.displayColumns,r=ra().showInfoMessage,i=Xo().isMobile,o=Io().tableCompact,a=bo(document.body),u=ie(null),l=Ft(ee(0),2),c=l[0],s=l[1],f=Ft(ee(0),2),d=f[0],h=f[1],p=Ft(ee(""),2),v=p[0],m=p[1],g=Ft(ee("asc"),2),y=g[0],_=g[1],b=o?ed([{group:0,metric:{Data:"Data"}}],["Data"]):ed(t,n),D=function(e){var t=e.__name__,n=xo(e,od);return t||Object.keys(n).length?"".concat(t," ").concat(JSON.stringify(n)):""},w=new Set(null===t||void 0===t?void 0:t.map((function(e){return e.group}))),x=w.size>1,k=ae((function(){var e=null===t||void 0===t?void 0:t.map((function(e){return{metadata:b.map((function(t){return o?wf(e,"",x):e.metric[t.key]||"-"})),value:e.value?e.value[1]:"-",values:e.values?e.values.map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(r," @").concat(n)})):[],copyValue:D(e.metric)}})),n="Value"===v,r=b.findIndex((function(e){return e.key===v}));return n||-1!==r?e.sort((function(e,t){var i=n?Number(e.value):e.metadata[r],o=n?Number(t.value):t.metadata[r];return("asc"===y?io)?-1:1})):e}),[b,t,v,y,o]),C=ae((function(){return k.some((function(e){return e.copyValue}))}),[k]),A=function(){var e=tu(Xa().mark((function e(t){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(t);case 2:r({text:"Row has been copied",type:"success"});case 3:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}(),E=function(e){return function(){!function(e){_((function(t){return"asc"===t&&v===e?"desc":"asc"})),m(e)}(e)}},S=function(){if(u.current){var e=u.current.getBoundingClientRect().top;h(e<0?window.scrollY-c:0)}};return ne((function(){return window.addEventListener("scroll",S),function(){window.removeEventListener("scroll",S)}}),[u,c,a]),ne((function(){if(u.current){var e=u.current.getBoundingClientRect().top;s(e+window.scrollY)}}),[u,a]),k.length?Vr("div",{className:wo()({"vm-table-view":!0,"vm-table-view_mobile":i}),children:Vr("table",{className:"vm-table",ref:u,children:[Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",style:{transform:"translateY(".concat(d,"px)")},children:[b.map((function(e,t){return Vr("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:E(e.key),children:Vr("div",{className:"vm-table-cell__content",children:[e.key,Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":v===e.key,"vm-table__sort-icon_desc":"desc"===y&&v===e.key}),children:Vr(Qi,{})})]})},t)})),Vr("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_right vm-table-cell_sort",onClick:E("Value"),children:Vr("div",{className:"vm-table-cell__content",children:[Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":"Value"===v,"vm-table__sort-icon_desc":"desc"===y}),children:Vr(Qi,{})}),"Value"]})}),C&&Vr("td",{className:"vm-table-cell vm-table-cell_header"})]})}),Vr("tbody",{className:"vm-table-body",children:k.map((function(e,t){return Vr("tr",{className:"vm-table__row",children:[e.metadata.map((function(e,n){return Vr("td",{className:wo()({"vm-table-cell vm-table-cell_no-wrap":!0,"vm-table-cell_gray":k[t-1]&&k[t-1].metadata[n]===e}),children:e},n)})),Vr("td",{className:"vm-table-cell vm-table-cell_right vm-table-cell_no-wrap",children:e.values.length?e.values.map((function(e){return Vr("p",{children:e},e)})):e.value}),C&&Vr("td",{className:"vm-table-cell vm-table-cell_right",children:e.copyValue&&Vr("div",{className:"vm-table-cell__content",children:Vr(ma,{title:"Copy row",children:Vr(fa,{variant:"text",color:"gray",size:"small",startIcon:Vr(so,{}),onClick:(n=e.copyValue,function(){A(n)})})})})})]},t);var n}))})]})}):Vr(ta,{variant:"warning",children:"No data to show"})},ud=function(){var e=Io(),t=e.displayType,n=e.isTracingEnabled,r=Li().query,i=Ni().period,o=Fi(),a=Xo().isMobile;!function(){var e=Wr().tenantId,t=Io().displayType,n=Li().query,r=Ni(),i=r.duration,o=r.relativeTime,a=r.period,u=a.date,l=a.step,c=$o().customStep,s=Ft(pr(),2)[1],f=function(){var r={};n.forEach((function(n,a){var s,f="g".concat(a);r["".concat(f,".expr")]=n,r["".concat(f,".range_input")]=i,r["".concat(f,".end_input")]=u,r["".concat(f,".tab")]=(null===(s=So.find((function(e){return e.value===t})))||void 0===s?void 0:s.prometheusCode)||0,r["".concat(f,".relative_time")]=o,r["".concat(f,".tenantID")]=e,l!==c&&c&&(r["".concat(f,".step_input")]=c)})),s(id(r))};ne(f,[e,t,n,i,o,u,l,c]),ne(f,[])}();var u=Ft(ee(),2),l=u[0],c=u[1],s=Ft(ee([]),2),f=s[0],d=s[1],h=Ft(ee([]),2),p=h[0],v=h[1],m=Ft(ee(!1),2),g=m[0],y=m[1],_=$o(),b=_.customStep,D=_.yaxis,w=Ho(),x=Gf().queryOptions,k=Uf({visible:!0,customStep:b,hideQuery:p,showAllSeries:g}),C=k.isLoading,A=k.liveData,E=k.graphData,S=k.error,N=k.warning,F=k.traces,M=function(e){w({type:"SET_YAXIS_LIMITS",payload:e})};return ne((function(){F&&d([].concat(Ot(f),Ot(F)))}),[F]),ne((function(){d([])}),[t]),ne((function(){y(!1)}),[r]),Vr("div",{className:wo()({"vm-custom-panel":!0,"vm-custom-panel_mobile":a}),children:[Vr(If,{error:S,queryOptions:x,onHideQuery:function(e){v(e)}}),n&&Vr("div",{className:"vm-custom-panel__trace",children:Vr(Xf,{traces:f,onDeleteClick:function(e){var t=f.filter((function(t){return t.idValue!==e.idValue}));d(Ot(t))}})}),C&&Vr(Qf,{}),S&&Vr(ta,{variant:"error",children:S}),N&&Vr(ta,{variant:"warning",children:Vr("div",{className:wo()({"vm-custom-panel__warning":!0,"vm-custom-panel__warning_mobile":a}),children:[Vr("p",{children:N}),Vr(fa,{color:"warning",variant:"outlined",onClick:function(){y(!0)},children:"Show all"})]})}),Vr("div",{className:wo()({"vm-custom-panel-body":!0,"vm-custom-panel-body_mobile":a,"vm-block":!0,"vm-block_mobile":a}),children:[Vr("div",{className:"vm-custom-panel-body-header",children:[Vr(No,{}),"chart"===t&&Vr(Wf,{yaxis:D,setYaxisLimits:M,toggleEnableLimits:function(){w({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})}}),"table"===t&&Vr(rd,{data:A||[],defaultColumns:l,onChange:c})]}),E&&i&&"chart"===t&&Vr(Nf,{data:E,period:i,customStep:b,query:r,yaxis:D,setYaxisLimits:M,setPeriod:function(e){var t=e.from,n=e.to;o({type:"SET_PERIOD",payload:{from:t,to:n}})},height:a?.5*window.innerHeight:500}),A&&"code"===t&&Vr(Yf,{data:A}),A&&"table"===t&&Vr(ad,{data:A,displayColumns:l})]})]})};function ld(){return{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}}var cd={async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1};var sd=/[&<>"']/,fd=new RegExp(sd.source,"g"),dd=/[<>"']|&(?!(#\d{1,7}|#[Xx][a-fA-F0-9]{1,6}|\w+);)/,hd=new RegExp(dd.source,"g"),pd={"&":"&","<":"<",">":">",'"':""","'":"'"},vd=function(e){return pd[e]};function md(e,t){if(t){if(sd.test(e))return e.replace(fd,vd)}else if(dd.test(e))return e.replace(hd,vd);return e}var gd=/&(#(?:\d+)|(?:#x[0-9A-Fa-f]+)|(?:\w+));?/gi;function yd(e){return e.replace(gd,(function(e,t){return"colon"===(t=t.toLowerCase())?":":"#"===t.charAt(0)?"x"===t.charAt(1)?String.fromCharCode(parseInt(t.substring(2),16)):String.fromCharCode(+t.substring(1)):""}))}var _d=/(^|[^\[])\^/g;function bd(e,t){e="string"===typeof e?e:e.source,t=t||"";var n={replace:function(t,r){return r=(r=r.source||r).replace(_d,"$1"),e=e.replace(t,r),n},getRegex:function(){return new RegExp(e,t)}};return n}var Dd=/[^\w:]/g,wd=/^$|^[a-z][a-z0-9+.-]*:|^[?#]/i;function xd(e,t,n){if(e){var r;try{r=decodeURIComponent(yd(n)).replace(Dd,"").toLowerCase()}catch(s){return null}if(0===r.indexOf("javascript:")||0===r.indexOf("vbscript:")||0===r.indexOf("data:"))return null}t&&!wd.test(n)&&(n=function(e,t){kd[" "+e]||(Cd.test(e)?kd[" "+e]=e+"/":kd[" "+e]=Md(e,"/",!0));e=kd[" "+e];var n=-1===e.indexOf(":");return"//"===t.substring(0,2)?n?t:e.replace(Ad,"$1")+t:"/"===t.charAt(0)?n?t:e.replace(Ed,"$1")+t:e+t}(t,n));try{n=encodeURI(n).replace(/%25/g,"%")}catch(s){return null}return n}var kd={},Cd=/^[^:]+:\/*[^/]*$/,Ad=/^([^:]+:)[\s\S]*$/,Ed=/^([^:]+:\/*[^/]*)[\s\S]*$/;var Sd={exec:function(){}};function Nd(e){for(var t,n,r=1;r=0&&"\\"===n[i];)r=!r;return r?"|":" |"})).split(/ \|/),r=0;if(n[0].trim()||n.shift(),n.length>0&&!n[n.length-1].trim()&&n.pop(),n.length>t)n.splice(t);else for(;n.length1;)1&t&&(n+=e),t>>=1,e+=e;return n+e}function Bd(e,t,n,r){var i=t.href,o=t.title?md(t.title):null,a=e[1].replace(/\\([\[\]])/g,"$1");if("!"!==e[0].charAt(0)){r.state.inLink=!0;var u={type:"link",raw:n,href:i,title:o,text:a,tokens:r.inlineTokens(a)};return r.state.inLink=!1,u}return{type:"image",raw:n,href:i,title:o,text:md(a)}}var Ld=function(){function e(t){Bt(this,e),this.options=t||cd}return Rt(e,[{key:"space",value:function(e){var t=this.rules.block.newline.exec(e);if(t&&t[0].length>0)return{type:"space",raw:t[0]}}},{key:"code",value:function(e){var t=this.rules.block.code.exec(e);if(t){var n=t[0].replace(/^ {1,4}/gm,"");return{type:"code",raw:t[0],codeBlockStyle:"indented",text:this.options.pedantic?n:Md(n,"\n")}}}},{key:"fences",value:function(e){var t=this.rules.block.fences.exec(e);if(t){var n=t[0],r=function(e,t){var n=e.match(/^(\s+)(?:```)/);if(null===n)return t;var r=n[1];return t.split("\n").map((function(e){var t=e.match(/^\s+/);return null===t?e:Ft(t,1)[0].length>=r.length?e.slice(r.length):e})).join("\n")}(n,t[3]||"");return{type:"code",raw:n,lang:t[2]?t[2].trim().replace(this.rules.inline._escapes,"$1"):t[2],text:r}}}},{key:"heading",value:function(e){var t=this.rules.block.heading.exec(e);if(t){var n=t[2].trim();if(/#$/.test(n)){var r=Md(n,"#");this.options.pedantic?n=r.trim():r&&!/ $/.test(r)||(n=r.trim())}return{type:"heading",raw:t[0],depth:t[1].length,text:n,tokens:this.lexer.inline(n)}}}},{key:"hr",value:function(e){var t=this.rules.block.hr.exec(e);if(t)return{type:"hr",raw:t[0]}}},{key:"blockquote",value:function(e){var t=this.rules.block.blockquote.exec(e);if(t){var n=t[0].replace(/^ *>[ \t]?/gm,""),r=this.lexer.state.top;this.lexer.state.top=!0;var i=this.lexer.blockTokens(n);return this.lexer.state.top=r,{type:"blockquote",raw:t[0],tokens:i,text:n}}}},{key:"list",value:function(e){var t=this.rules.block.list.exec(e);if(t){var n,r,i,o,a,u,l,c,s,f,d,h,p=t[1].trim(),v=p.length>1,m={type:"list",raw:"",ordered:v,start:v?+p.slice(0,-1):"",loose:!1,items:[]};p=v?"\\d{1,9}\\".concat(p.slice(-1)):"\\".concat(p),this.options.pedantic&&(p=v?p:"[*+-]");for(var g=new RegExp("^( {0,3}".concat(p,")((?:[\t ][^\\n]*)?(?:\\n|$))"));e&&(h=!1,t=g.exec(e))&&!this.rules.block.hr.test(e);){if(n=t[0],e=e.substring(n.length),c=t[2].split("\n",1)[0].replace(/^\t+/,(function(e){return" ".repeat(3*e.length)})),s=e.split("\n",1)[0],this.options.pedantic?(o=2,d=c.trimLeft()):(o=(o=t[2].search(/[^ ]/))>4?1:o,d=c.slice(o),o+=t[1].length),u=!1,!c&&/^ *$/.test(s)&&(n+=s+"\n",e=e.substring(s.length+1),h=!0),!h)for(var y=new RegExp("^ {0,".concat(Math.min(3,o-1),"}(?:[*+-]|\\d{1,9}[.)])((?:[ \t][^\\n]*)?(?:\\n|$))")),_=new RegExp("^ {0,".concat(Math.min(3,o-1),"}((?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$)")),b=new RegExp("^ {0,".concat(Math.min(3,o-1),"}(?:```|~~~)")),D=new RegExp("^ {0,".concat(Math.min(3,o-1),"}#"));e&&(s=f=e.split("\n",1)[0],this.options.pedantic&&(s=s.replace(/^ {1,4}(?=( {4})*[^ ])/g," ")),!b.test(s))&&!D.test(s)&&!y.test(s)&&!_.test(e);){if(s.search(/[^ ]/)>=o||!s.trim())d+="\n"+s.slice(o);else{if(u)break;if(c.search(/[^ ]/)>=4)break;if(b.test(c))break;if(D.test(c))break;if(_.test(c))break;d+="\n"+s}u||s.trim()||(u=!0),n+=f+"\n",e=e.substring(f.length+1),c=s.slice(o)}m.loose||(l?m.loose=!0:/\n *\n *$/.test(n)&&(l=!0)),this.options.gfm&&(r=/^\[[ xX]\] /.exec(d))&&(i="[ ] "!==r[0],d=d.replace(/^\[[ xX]\] +/,"")),m.items.push({type:"list_item",raw:n,task:!!r,checked:i,loose:!1,text:d}),m.raw+=n}m.items[m.items.length-1].raw=n.trimRight(),m.items[m.items.length-1].text=d.trimRight(),m.raw=m.raw.trimRight();var w=m.items.length;for(a=0;a0&&x.some((function(e){return/\n.*\n/.test(e.raw)}));m.loose=k}if(m.loose)for(a=0;a$/,"$1").replace(this.rules.inline._escapes,"$1"):"",i=t[3]?t[3].substring(1,t[3].length-1).replace(this.rules.inline._escapes,"$1"):t[3];return{type:"def",tag:n,raw:t[0],href:r,title:i}}}},{key:"table",value:function(e){var t=this.rules.block.table.exec(e);if(t){var n={type:"table",header:Fd(t[1]).map((function(e){return{text:e}})),align:t[2].replace(/^ *|\| *$/g,"").split(/ *\| */),rows:t[3]&&t[3].trim()?t[3].replace(/\n[ \t]*$/,"").split("\n"):[]};if(n.header.length===n.align.length){n.raw=t[0];var r,i,o,a,u=n.align.length;for(r=0;r/i.test(t[0])&&(this.lexer.state.inLink=!1),!this.lexer.state.inRawBlock&&/^<(pre|code|kbd|script)(\s|>)/i.test(t[0])?this.lexer.state.inRawBlock=!0:this.lexer.state.inRawBlock&&/^<\/(pre|code|kbd|script)(\s|>)/i.test(t[0])&&(this.lexer.state.inRawBlock=!1),{type:this.options.sanitize?"text":"html",raw:t[0],inLink:this.lexer.state.inLink,inRawBlock:this.lexer.state.inRawBlock,text:this.options.sanitize?this.options.sanitizer?this.options.sanitizer(t[0]):md(t[0]):t[0]}}},{key:"link",value:function(e){var t=this.rules.inline.link.exec(e);if(t){var n=t[2].trim();if(!this.options.pedantic&&/^$/.test(n))return;var r=Md(n.slice(0,-1),"\\");if((n.length-r.length)%2===0)return}else{var i=function(e,t){if(-1===e.indexOf(t[1]))return-1;for(var n=e.length,r=0,i=0;i-1){var o=(0===t[0].indexOf("!")?5:4)+t[1].length+i;t[2]=t[2].substring(0,i),t[0]=t[0].substring(0,o).trim(),t[3]=""}}var a=t[2],u="";if(this.options.pedantic){var l=/^([^'"]*[^\s])\s+(['"])(.*)\2/.exec(a);l&&(a=l[1],u=l[3])}else u=t[3]?t[3].slice(1,-1):"";return a=a.trim(),/^$/.test(n)?a.slice(1):a.slice(1,-1)),Bd(t,{href:a?a.replace(this.rules.inline._escapes,"$1"):a,title:u?u.replace(this.rules.inline._escapes,"$1"):u},t[0],this.lexer)}}},{key:"reflink",value:function(e,t){var n;if((n=this.rules.inline.reflink.exec(e))||(n=this.rules.inline.nolink.exec(e))){var r=(n[2]||n[1]).replace(/\s+/g," ");if(!(r=t[r.toLowerCase()])){var i=n[0].charAt(0);return{type:"text",raw:i,text:i}}return Bd(n,r,n[0],this.lexer)}}},{key:"emStrong",value:function(e,t){var n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"",r=this.rules.inline.emStrong.lDelim.exec(e);if(r&&(!r[3]||!n.match(/(?:[0-9A-Za-z\xAA\xB2\xB3\xB5\xB9\xBA\xBC-\xBE\xC0-\xD6\xD8-\xF6\xF8-\u02C1\u02C6-\u02D1\u02E0-\u02E4\u02EC\u02EE\u0370-\u0374\u0376\u0377\u037A-\u037D\u037F\u0386\u0388-\u038A\u038C\u038E-\u03A1\u03A3-\u03F5\u03F7-\u0481\u048A-\u052F\u0531-\u0556\u0559\u0560-\u0588\u05D0-\u05EA\u05EF-\u05F2\u0620-\u064A\u0660-\u0669\u066E\u066F\u0671-\u06D3\u06D5\u06E5\u06E6\u06EE-\u06FC\u06FF\u0710\u0712-\u072F\u074D-\u07A5\u07B1\u07C0-\u07EA\u07F4\u07F5\u07FA\u0800-\u0815\u081A\u0824\u0828\u0840-\u0858\u0860-\u086A\u0870-\u0887\u0889-\u088E\u08A0-\u08C9\u0904-\u0939\u093D\u0950\u0958-\u0961\u0966-\u096F\u0971-\u0980\u0985-\u098C\u098F\u0990\u0993-\u09A8\u09AA-\u09B0\u09B2\u09B6-\u09B9\u09BD\u09CE\u09DC\u09DD\u09DF-\u09E1\u09E6-\u09F1\u09F4-\u09F9\u09FC\u0A05-\u0A0A\u0A0F\u0A10\u0A13-\u0A28\u0A2A-\u0A30\u0A32\u0A33\u0A35\u0A36\u0A38\u0A39\u0A59-\u0A5C\u0A5E\u0A66-\u0A6F\u0A72-\u0A74\u0A85-\u0A8D\u0A8F-\u0A91\u0A93-\u0AA8\u0AAA-\u0AB0\u0AB2\u0AB3\u0AB5-\u0AB9\u0ABD\u0AD0\u0AE0\u0AE1\u0AE6-\u0AEF\u0AF9\u0B05-\u0B0C\u0B0F\u0B10\u0B13-\u0B28\u0B2A-\u0B30\u0B32\u0B33\u0B35-\u0B39\u0B3D\u0B5C\u0B5D\u0B5F-\u0B61\u0B66-\u0B6F\u0B71-\u0B77\u0B83\u0B85-\u0B8A\u0B8E-\u0B90\u0B92-\u0B95\u0B99\u0B9A\u0B9C\u0B9E\u0B9F\u0BA3\u0BA4\u0BA8-\u0BAA\u0BAE-\u0BB9\u0BD0\u0BE6-\u0BF2\u0C05-\u0C0C\u0C0E-\u0C10\u0C12-\u0C28\u0C2A-\u0C39\u0C3D\u0C58-\u0C5A\u0C5D\u0C60\u0C61\u0C66-\u0C6F\u0C78-\u0C7E\u0C80\u0C85-\u0C8C\u0C8E-\u0C90\u0C92-\u0CA8\u0CAA-\u0CB3\u0CB5-\u0CB9\u0CBD\u0CDD\u0CDE\u0CE0\u0CE1\u0CE6-\u0CEF\u0CF1\u0CF2\u0D04-\u0D0C\u0D0E-\u0D10\u0D12-\u0D3A\u0D3D\u0D4E\u0D54-\u0D56\u0D58-\u0D61\u0D66-\u0D78\u0D7A-\u0D7F\u0D85-\u0D96\u0D9A-\u0DB1\u0DB3-\u0DBB\u0DBD\u0DC0-\u0DC6\u0DE6-\u0DEF\u0E01-\u0E30\u0E32\u0E33\u0E40-\u0E46\u0E50-\u0E59\u0E81\u0E82\u0E84\u0E86-\u0E8A\u0E8C-\u0EA3\u0EA5\u0EA7-\u0EB0\u0EB2\u0EB3\u0EBD\u0EC0-\u0EC4\u0EC6\u0ED0-\u0ED9\u0EDC-\u0EDF\u0F00\u0F20-\u0F33\u0F40-\u0F47\u0F49-\u0F6C\u0F88-\u0F8C\u1000-\u102A\u103F-\u1049\u1050-\u1055\u105A-\u105D\u1061\u1065\u1066\u106E-\u1070\u1075-\u1081\u108E\u1090-\u1099\u10A0-\u10C5\u10C7\u10CD\u10D0-\u10FA\u10FC-\u1248\u124A-\u124D\u1250-\u1256\u1258\u125A-\u125D\u1260-\u1288\u128A-\u128D\u1290-\u12B0\u12B2-\u12B5\u12B8-\u12BE\u12C0\u12C2-\u12C5\u12C8-\u12D6\u12D8-\u1310\u1312-\u1315\u1318-\u135A\u1369-\u137C\u1380-\u138F\u13A0-\u13F5\u13F8-\u13FD\u1401-\u166C\u166F-\u167F\u1681-\u169A\u16A0-\u16EA\u16EE-\u16F8\u1700-\u1711\u171F-\u1731\u1740-\u1751\u1760-\u176C\u176E-\u1770\u1780-\u17B3\u17D7\u17DC\u17E0-\u17E9\u17F0-\u17F9\u1810-\u1819\u1820-\u1878\u1880-\u1884\u1887-\u18A8\u18AA\u18B0-\u18F5\u1900-\u191E\u1946-\u196D\u1970-\u1974\u1980-\u19AB\u19B0-\u19C9\u19D0-\u19DA\u1A00-\u1A16\u1A20-\u1A54\u1A80-\u1A89\u1A90-\u1A99\u1AA7\u1B05-\u1B33\u1B45-\u1B4C\u1B50-\u1B59\u1B83-\u1BA0\u1BAE-\u1BE5\u1C00-\u1C23\u1C40-\u1C49\u1C4D-\u1C7D\u1C80-\u1C88\u1C90-\u1CBA\u1CBD-\u1CBF\u1CE9-\u1CEC\u1CEE-\u1CF3\u1CF5\u1CF6\u1CFA\u1D00-\u1DBF\u1E00-\u1F15\u1F18-\u1F1D\u1F20-\u1F45\u1F48-\u1F4D\u1F50-\u1F57\u1F59\u1F5B\u1F5D\u1F5F-\u1F7D\u1F80-\u1FB4\u1FB6-\u1FBC\u1FBE\u1FC2-\u1FC4\u1FC6-\u1FCC\u1FD0-\u1FD3\u1FD6-\u1FDB\u1FE0-\u1FEC\u1FF2-\u1FF4\u1FF6-\u1FFC\u2070\u2071\u2074-\u2079\u207F-\u2089\u2090-\u209C\u2102\u2107\u210A-\u2113\u2115\u2119-\u211D\u2124\u2126\u2128\u212A-\u212D\u212F-\u2139\u213C-\u213F\u2145-\u2149\u214E\u2150-\u2189\u2460-\u249B\u24EA-\u24FF\u2776-\u2793\u2C00-\u2CE4\u2CEB-\u2CEE\u2CF2\u2CF3\u2CFD\u2D00-\u2D25\u2D27\u2D2D\u2D30-\u2D67\u2D6F\u2D80-\u2D96\u2DA0-\u2DA6\u2DA8-\u2DAE\u2DB0-\u2DB6\u2DB8-\u2DBE\u2DC0-\u2DC6\u2DC8-\u2DCE\u2DD0-\u2DD6\u2DD8-\u2DDE\u2E2F\u3005-\u3007\u3021-\u3029\u3031-\u3035\u3038-\u303C\u3041-\u3096\u309D-\u309F\u30A1-\u30FA\u30FC-\u30FF\u3105-\u312F\u3131-\u318E\u3192-\u3195\u31A0-\u31BF\u31F0-\u31FF\u3220-\u3229\u3248-\u324F\u3251-\u325F\u3280-\u3289\u32B1-\u32BF\u3400-\u4DBF\u4E00-\uA48C\uA4D0-\uA4FD\uA500-\uA60C\uA610-\uA62B\uA640-\uA66E\uA67F-\uA69D\uA6A0-\uA6EF\uA717-\uA71F\uA722-\uA788\uA78B-\uA7CA\uA7D0\uA7D1\uA7D3\uA7D5-\uA7D9\uA7F2-\uA801\uA803-\uA805\uA807-\uA80A\uA80C-\uA822\uA830-\uA835\uA840-\uA873\uA882-\uA8B3\uA8D0-\uA8D9\uA8F2-\uA8F7\uA8FB\uA8FD\uA8FE\uA900-\uA925\uA930-\uA946\uA960-\uA97C\uA984-\uA9B2\uA9CF-\uA9D9\uA9E0-\uA9E4\uA9E6-\uA9FE\uAA00-\uAA28\uAA40-\uAA42\uAA44-\uAA4B\uAA50-\uAA59\uAA60-\uAA76\uAA7A\uAA7E-\uAAAF\uAAB1\uAAB5\uAAB6\uAAB9-\uAABD\uAAC0\uAAC2\uAADB-\uAADD\uAAE0-\uAAEA\uAAF2-\uAAF4\uAB01-\uAB06\uAB09-\uAB0E\uAB11-\uAB16\uAB20-\uAB26\uAB28-\uAB2E\uAB30-\uAB5A\uAB5C-\uAB69\uAB70-\uABE2\uABF0-\uABF9\uAC00-\uD7A3\uD7B0-\uD7C6\uD7CB-\uD7FB\uF900-\uFA6D\uFA70-\uFAD9\uFB00-\uFB06\uFB13-\uFB17\uFB1D\uFB1F-\uFB28\uFB2A-\uFB36\uFB38-\uFB3C\uFB3E\uFB40\uFB41\uFB43\uFB44\uFB46-\uFBB1\uFBD3-\uFD3D\uFD50-\uFD8F\uFD92-\uFDC7\uFDF0-\uFDFB\uFE70-\uFE74\uFE76-\uFEFC\uFF10-\uFF19\uFF21-\uFF3A\uFF41-\uFF5A\uFF66-\uFFBE\uFFC2-\uFFC7\uFFCA-\uFFCF\uFFD2-\uFFD7\uFFDA-\uFFDC]|\uD800[\uDC00-\uDC0B\uDC0D-\uDC26\uDC28-\uDC3A\uDC3C\uDC3D\uDC3F-\uDC4D\uDC50-\uDC5D\uDC80-\uDCFA\uDD07-\uDD33\uDD40-\uDD78\uDD8A\uDD8B\uDE80-\uDE9C\uDEA0-\uDED0\uDEE1-\uDEFB\uDF00-\uDF23\uDF2D-\uDF4A\uDF50-\uDF75\uDF80-\uDF9D\uDFA0-\uDFC3\uDFC8-\uDFCF\uDFD1-\uDFD5]|\uD801[\uDC00-\uDC9D\uDCA0-\uDCA9\uDCB0-\uDCD3\uDCD8-\uDCFB\uDD00-\uDD27\uDD30-\uDD63\uDD70-\uDD7A\uDD7C-\uDD8A\uDD8C-\uDD92\uDD94\uDD95\uDD97-\uDDA1\uDDA3-\uDDB1\uDDB3-\uDDB9\uDDBB\uDDBC\uDE00-\uDF36\uDF40-\uDF55\uDF60-\uDF67\uDF80-\uDF85\uDF87-\uDFB0\uDFB2-\uDFBA]|\uD802[\uDC00-\uDC05\uDC08\uDC0A-\uDC35\uDC37\uDC38\uDC3C\uDC3F-\uDC55\uDC58-\uDC76\uDC79-\uDC9E\uDCA7-\uDCAF\uDCE0-\uDCF2\uDCF4\uDCF5\uDCFB-\uDD1B\uDD20-\uDD39\uDD80-\uDDB7\uDDBC-\uDDCF\uDDD2-\uDE00\uDE10-\uDE13\uDE15-\uDE17\uDE19-\uDE35\uDE40-\uDE48\uDE60-\uDE7E\uDE80-\uDE9F\uDEC0-\uDEC7\uDEC9-\uDEE4\uDEEB-\uDEEF\uDF00-\uDF35\uDF40-\uDF55\uDF58-\uDF72\uDF78-\uDF91\uDFA9-\uDFAF]|\uD803[\uDC00-\uDC48\uDC80-\uDCB2\uDCC0-\uDCF2\uDCFA-\uDD23\uDD30-\uDD39\uDE60-\uDE7E\uDE80-\uDEA9\uDEB0\uDEB1\uDF00-\uDF27\uDF30-\uDF45\uDF51-\uDF54\uDF70-\uDF81\uDFB0-\uDFCB\uDFE0-\uDFF6]|\uD804[\uDC03-\uDC37\uDC52-\uDC6F\uDC71\uDC72\uDC75\uDC83-\uDCAF\uDCD0-\uDCE8\uDCF0-\uDCF9\uDD03-\uDD26\uDD36-\uDD3F\uDD44\uDD47\uDD50-\uDD72\uDD76\uDD83-\uDDB2\uDDC1-\uDDC4\uDDD0-\uDDDA\uDDDC\uDDE1-\uDDF4\uDE00-\uDE11\uDE13-\uDE2B\uDE3F\uDE40\uDE80-\uDE86\uDE88\uDE8A-\uDE8D\uDE8F-\uDE9D\uDE9F-\uDEA8\uDEB0-\uDEDE\uDEF0-\uDEF9\uDF05-\uDF0C\uDF0F\uDF10\uDF13-\uDF28\uDF2A-\uDF30\uDF32\uDF33\uDF35-\uDF39\uDF3D\uDF50\uDF5D-\uDF61]|\uD805[\uDC00-\uDC34\uDC47-\uDC4A\uDC50-\uDC59\uDC5F-\uDC61\uDC80-\uDCAF\uDCC4\uDCC5\uDCC7\uDCD0-\uDCD9\uDD80-\uDDAE\uDDD8-\uDDDB\uDE00-\uDE2F\uDE44\uDE50-\uDE59\uDE80-\uDEAA\uDEB8\uDEC0-\uDEC9\uDF00-\uDF1A\uDF30-\uDF3B\uDF40-\uDF46]|\uD806[\uDC00-\uDC2B\uDCA0-\uDCF2\uDCFF-\uDD06\uDD09\uDD0C-\uDD13\uDD15\uDD16\uDD18-\uDD2F\uDD3F\uDD41\uDD50-\uDD59\uDDA0-\uDDA7\uDDAA-\uDDD0\uDDE1\uDDE3\uDE00\uDE0B-\uDE32\uDE3A\uDE50\uDE5C-\uDE89\uDE9D\uDEB0-\uDEF8]|\uD807[\uDC00-\uDC08\uDC0A-\uDC2E\uDC40\uDC50-\uDC6C\uDC72-\uDC8F\uDD00-\uDD06\uDD08\uDD09\uDD0B-\uDD30\uDD46\uDD50-\uDD59\uDD60-\uDD65\uDD67\uDD68\uDD6A-\uDD89\uDD98\uDDA0-\uDDA9\uDEE0-\uDEF2\uDF02\uDF04-\uDF10\uDF12-\uDF33\uDF50-\uDF59\uDFB0\uDFC0-\uDFD4]|\uD808[\uDC00-\uDF99]|\uD809[\uDC00-\uDC6E\uDC80-\uDD43]|\uD80B[\uDF90-\uDFF0]|[\uD80C\uD81C-\uD820\uD822\uD840-\uD868\uD86A-\uD86C\uD86F-\uD872\uD874-\uD879\uD880-\uD883\uD885-\uD887][\uDC00-\uDFFF]|\uD80D[\uDC00-\uDC2F\uDC41-\uDC46]|\uD811[\uDC00-\uDE46]|\uD81A[\uDC00-\uDE38\uDE40-\uDE5E\uDE60-\uDE69\uDE70-\uDEBE\uDEC0-\uDEC9\uDED0-\uDEED\uDF00-\uDF2F\uDF40-\uDF43\uDF50-\uDF59\uDF5B-\uDF61\uDF63-\uDF77\uDF7D-\uDF8F]|\uD81B[\uDE40-\uDE96\uDF00-\uDF4A\uDF50\uDF93-\uDF9F\uDFE0\uDFE1\uDFE3]|\uD821[\uDC00-\uDFF7]|\uD823[\uDC00-\uDCD5\uDD00-\uDD08]|\uD82B[\uDFF0-\uDFF3\uDFF5-\uDFFB\uDFFD\uDFFE]|\uD82C[\uDC00-\uDD22\uDD32\uDD50-\uDD52\uDD55\uDD64-\uDD67\uDD70-\uDEFB]|\uD82F[\uDC00-\uDC6A\uDC70-\uDC7C\uDC80-\uDC88\uDC90-\uDC99]|\uD834[\uDEC0-\uDED3\uDEE0-\uDEF3\uDF60-\uDF78]|\uD835[\uDC00-\uDC54\uDC56-\uDC9C\uDC9E\uDC9F\uDCA2\uDCA5\uDCA6\uDCA9-\uDCAC\uDCAE-\uDCB9\uDCBB\uDCBD-\uDCC3\uDCC5-\uDD05\uDD07-\uDD0A\uDD0D-\uDD14\uDD16-\uDD1C\uDD1E-\uDD39\uDD3B-\uDD3E\uDD40-\uDD44\uDD46\uDD4A-\uDD50\uDD52-\uDEA5\uDEA8-\uDEC0\uDEC2-\uDEDA\uDEDC-\uDEFA\uDEFC-\uDF14\uDF16-\uDF34\uDF36-\uDF4E\uDF50-\uDF6E\uDF70-\uDF88\uDF8A-\uDFA8\uDFAA-\uDFC2\uDFC4-\uDFCB\uDFCE-\uDFFF]|\uD837[\uDF00-\uDF1E\uDF25-\uDF2A]|\uD838[\uDC30-\uDC6D\uDD00-\uDD2C\uDD37-\uDD3D\uDD40-\uDD49\uDD4E\uDE90-\uDEAD\uDEC0-\uDEEB\uDEF0-\uDEF9]|\uD839[\uDCD0-\uDCEB\uDCF0-\uDCF9\uDFE0-\uDFE6\uDFE8-\uDFEB\uDFED\uDFEE\uDFF0-\uDFFE]|\uD83A[\uDC00-\uDCC4\uDCC7-\uDCCF\uDD00-\uDD43\uDD4B\uDD50-\uDD59]|\uD83B[\uDC71-\uDCAB\uDCAD-\uDCAF\uDCB1-\uDCB4\uDD01-\uDD2D\uDD2F-\uDD3D\uDE00-\uDE03\uDE05-\uDE1F\uDE21\uDE22\uDE24\uDE27\uDE29-\uDE32\uDE34-\uDE37\uDE39\uDE3B\uDE42\uDE47\uDE49\uDE4B\uDE4D-\uDE4F\uDE51\uDE52\uDE54\uDE57\uDE59\uDE5B\uDE5D\uDE5F\uDE61\uDE62\uDE64\uDE67-\uDE6A\uDE6C-\uDE72\uDE74-\uDE77\uDE79-\uDE7C\uDE7E\uDE80-\uDE89\uDE8B-\uDE9B\uDEA1-\uDEA3\uDEA5-\uDEA9\uDEAB-\uDEBB]|\uD83C[\uDD00-\uDD0C]|\uD83E[\uDFF0-\uDFF9]|\uD869[\uDC00-\uDEDF\uDF00-\uDFFF]|\uD86D[\uDC00-\uDF39\uDF40-\uDFFF]|\uD86E[\uDC00-\uDC1D\uDC20-\uDFFF]|\uD873[\uDC00-\uDEA1\uDEB0-\uDFFF]|\uD87A[\uDC00-\uDFE0]|\uD87E[\uDC00-\uDE1D]|\uD884[\uDC00-\uDF4A\uDF50-\uDFFF]|\uD888[\uDC00-\uDFAF])/))){var i=r[1]||r[2]||"";if(!i||i&&(""===n||this.rules.inline.punctuation.exec(n))){var o,a,u=r[0].length-1,l=u,c=0,s="*"===r[0][0]?this.rules.inline.emStrong.rDelimAst:this.rules.inline.emStrong.rDelimUnd;for(s.lastIndex=0,t=t.slice(-1*e.length+u);null!=(r=s.exec(t));)if(o=r[1]||r[2]||r[3]||r[4]||r[5]||r[6])if(a=o.length,r[3]||r[4])l+=a;else if(!((r[5]||r[6])&&u%3)||(u+a)%3){if(!((l-=a)>0)){a=Math.min(a,a+l+c);var f=e.slice(0,u+r.index+(r[0].length-o.length)+a);if(Math.min(u,a)%2){var d=f.slice(1,-1);return{type:"em",raw:f,text:d,tokens:this.lexer.inlineTokens(d)}}var h=f.slice(2,-2);return{type:"strong",raw:f,text:h,tokens:this.lexer.inlineTokens(h)}}}else c+=a}}}},{key:"codespan",value:function(e){var t=this.rules.inline.code.exec(e);if(t){var n=t[2].replace(/\n/g," "),r=/[^ ]/.test(n),i=/^ /.test(n)&&/ $/.test(n);return r&&i&&(n=n.substring(1,n.length-1)),n=md(n,!0),{type:"codespan",raw:t[0],text:n}}}},{key:"br",value:function(e){var t=this.rules.inline.br.exec(e);if(t)return{type:"br",raw:t[0]}}},{key:"del",value:function(e){var t=this.rules.inline.del.exec(e);if(t)return{type:"del",raw:t[0],text:t[2],tokens:this.lexer.inlineTokens(t[2])}}},{key:"autolink",value:function(e,t){var n,r,i=this.rules.inline.autolink.exec(e);if(i)return r="@"===i[2]?"mailto:"+(n=md(this.options.mangle?t(i[1]):i[1])):n=md(i[1]),{type:"link",raw:i[0],text:n,href:r,tokens:[{type:"text",raw:n,text:n}]}}},{key:"url",value:function(e,t){var n;if(n=this.rules.inline.url.exec(e)){var r,i;if("@"===n[2])i="mailto:"+(r=md(this.options.mangle?t(n[0]):n[0]));else{var o;do{o=n[0],n[0]=this.rules.inline._backpedal.exec(n[0])[0]}while(o!==n[0]);r=md(n[0]),i="www."===n[1]?"http://"+n[0]:n[0]}return{type:"link",raw:n[0],text:r,href:i,tokens:[{type:"text",raw:r,text:r}]}}}},{key:"inlineText",value:function(e,t){var n,r=this.rules.inline.text.exec(e);if(r)return n=this.lexer.state.inRawBlock?this.options.sanitize?this.options.sanitizer?this.options.sanitizer(r[0]):md(r[0]):r[0]:md(this.options.smartypants?t(r[0]):r[0]),{type:"text",raw:r[0],text:n}}}]),e}(),Id={newline:/^(?: *(?:\n|$))+/,code:/^( {4}[^\n]+(?:\n(?: *(?:\n|$))*)?)+/,fences:/^ {0,3}(`{3,}(?=[^`\n]*\n)|~{3,})([^\n]*)\n(?:|([\s\S]*?)\n)(?: {0,3}\1[~`]* *(?=\n|$)|$)/,hr:/^ {0,3}((?:-[\t ]*){3,}|(?:_[ \t]*){3,}|(?:\*[ \t]*){3,})(?:\n+|$)/,heading:/^ {0,3}(#{1,6})(?=\s|$)(.*)(?:\n+|$)/,blockquote:/^( {0,3}> ?(paragraph|[^\n]*)(?:\n|$))+/,list:/^( {0,3}bull)([ \t][^\n]+?)?(?:\n|$)/,html:"^ {0,3}(?:<(script|pre|style|textarea)[\\s>][\\s\\S]*?(?:\\1>[^\\n]*\\n+|$)|comment[^\\n]*(\\n+|$)|<\\?[\\s\\S]*?(?:\\?>\\n*|$)|\\n*|$)|\\n*|$)|?(tag)(?: +|\\n|/?>)[\\s\\S]*?(?:(?:\\n *)+\\n|$)|<(?!script|pre|style|textarea)([a-z][\\w-]*)(?:attribute)*? */?>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$)|(?!script|pre|style|textarea)[a-z][\\w-]*\\s*>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$))",def:/^ {0,3}\[(label)\]: *(?:\n *)?([^<\s][^\s]*|<.*?>)(?:(?: +(?:\n *)?| *\n *)(title))? *(?:\n+|$)/,table:Sd,lheading:/^((?:.|\n(?!\n))+?)\n {0,3}(=+|-+) *(?:\n+|$)/,_paragraph:/^([^\n]+(?:\n(?!hr|heading|lheading|blockquote|fences|list|html|table| +\n)[^\n]+)*)/,text:/^[^\n]+/,_label:/(?!\s*\])(?:\\.|[^\[\]\\])+/,_title:/(?:"(?:\\"?|[^"\\])*"|'[^'\n]*(?:\n[^'\n]+)*\n?'|\([^()]*\))/};Id.def=bd(Id.def).replace("label",Id._label).replace("title",Id._title).getRegex(),Id.bullet=/(?:[*+-]|\d{1,9}[.)])/,Id.listItemStart=bd(/^( *)(bull) */).replace("bull",Id.bullet).getRegex(),Id.list=bd(Id.list).replace(/bull/g,Id.bullet).replace("hr","\\n+(?=\\1?(?:(?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$))").replace("def","\\n+(?="+Id.def.source+")").getRegex(),Id._tag="address|article|aside|base|basefont|blockquote|body|caption|center|col|colgroup|dd|details|dialog|dir|div|dl|dt|fieldset|figcaption|figure|footer|form|frame|frameset|h[1-6]|head|header|hr|html|iframe|legend|li|link|main|menu|menuitem|meta|nav|noframes|ol|optgroup|option|p|param|section|source|summary|table|tbody|td|tfoot|th|thead|title|tr|track|ul",Id._comment=/|$)/,Id.html=bd(Id.html,"i").replace("comment",Id._comment).replace("tag",Id._tag).replace("attribute",/ +[a-zA-Z:_][\w.:-]*(?: *= *"[^"\n]*"| *= *'[^'\n]*'| *= *[^\s"'=<>`]+)?/).getRegex(),Id.paragraph=bd(Id._paragraph).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("|table","").replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.blockquote=bd(Id.blockquote).replace("paragraph",Id.paragraph).getRegex(),Id.normal=Nd({},Id),Id.gfm=Nd({},Id.normal,{table:"^ *([^\\n ].*\\|.*)\\n {0,3}(?:\\| *)?(:?-+:? *(?:\\| *:?-+:? *)*)(?:\\| *)?(?:\\n((?:(?! *\\n|hr|heading|blockquote|code|fences|list|html).*(?:\\n|$))*)\\n*|$)"}),Id.gfm.table=bd(Id.gfm.table).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("blockquote"," {0,3}>").replace("code"," {4}[^\\n]").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.gfm.paragraph=bd(Id._paragraph).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("table",Id.gfm.table).replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.pedantic=Nd({},Id.normal,{html:bd("^ *(?:comment *(?:\\n|\\s*$)|<(tag)[\\s\\S]+?\\1> *(?:\\n{2,}|\\s*$)|\\s]*)*?/?> *(?:\\n{2,}|\\s*$))").replace("comment",Id._comment).replace(/tag/g,"(?!(?:a|em|strong|small|s|cite|q|dfn|abbr|data|time|code|var|samp|kbd|sub|sup|i|b|u|mark|ruby|rt|rp|bdi|bdo|span|br|wbr|ins|del|img)\\b)\\w+(?!:|[^\\w\\s@]*@)\\b").getRegex(),def:/^ *\[([^\]]+)\]: *([^\s>]+)>?(?: +(["(][^\n]+[")]))? *(?:\n+|$)/,heading:/^(#{1,6})(.*)(?:\n+|$)/,fences:Sd,lheading:/^(.+?)\n {0,3}(=+|-+) *(?:\n+|$)/,paragraph:bd(Id.normal._paragraph).replace("hr",Id.hr).replace("heading"," *#{1,6} *[^\n]").replace("lheading",Id.lheading).replace("blockquote"," {0,3}>").replace("|fences","").replace("|list","").replace("|html","").getRegex()});var Pd={escape:/^\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/,autolink:/^<(scheme:[^\s\x00-\x1f<>]*|email)>/,url:Sd,tag:"^comment|^[a-zA-Z][\\w:-]*\\s*>|^<[a-zA-Z][\\w-]*(?:attribute)*?\\s*/?>|^<\\?[\\s\\S]*?\\?>|^|^",link:/^!?\[(label)\]\(\s*(href)(?:\s+(title))?\s*\)/,reflink:/^!?\[(label)\]\[(ref)\]/,nolink:/^!?\[(ref)\](?:\[\])?/,reflinkSearch:"reflink|nolink(?!\\()",emStrong:{lDelim:/^(?:\*+(?:([punct_])|[^\s*]))|^_+(?:([punct*])|([^\s_]))/,rDelimAst:/^(?:[^_*\\]|\\.)*?\_\_(?:[^_*\\]|\\.)*?\*(?:[^_*\\]|\\.)*?(?=\_\_)|(?:[^*\\]|\\.)+(?=[^*])|[punct_](\*+)(?=[\s]|$)|(?:[^punct*_\s\\]|\\.)(\*+)(?=[punct_\s]|$)|[punct_\s](\*+)(?=[^punct*_\s])|[\s](\*+)(?=[punct_])|[punct_](\*+)(?=[punct_])|(?:[^punct*_\s\\]|\\.)(\*+)(?=[^punct*_\s])/,rDelimUnd:/^(?:[^_*\\]|\\.)*?\*\*(?:[^_*\\]|\\.)*?\_(?:[^_*\\]|\\.)*?(?=\*\*)|(?:[^_\\]|\\.)+(?=[^_])|[punct*](\_+)(?=[\s]|$)|(?:[^punct*_\s\\]|\\.)(\_+)(?=[punct*\s]|$)|[punct*\s](\_+)(?=[^punct*_\s])|[\s](\_+)(?=[punct*])|[punct*](\_+)(?=[punct*])/},code:/^(`+)([^`]|[^`][\s\S]*?[^`])\1(?!`)/,br:/^( {2,}|\\)\n(?!\s*$)/,del:Sd,text:/^(`+|[^`])(?:(?= {2,}\n)|[\s\S]*?(?:(?=[\\.5&&(n="x"+n.toString(16)),r+=""+n+";";return r}Pd._punctuation="!\"#$%&'()+\\-.,/:;<=>?@\\[\\]`^{|}~",Pd.punctuation=bd(Pd.punctuation).replace(/punctuation/g,Pd._punctuation).getRegex(),Pd.blockSkip=/\[[^\]]*?\]\([^\)]*?\)|`[^`]*?`|<[^>]*?>/g,Pd.escapedEmSt=/(?:^|[^\\])(?:\\\\)*\\[*_]/g,Pd._comment=bd(Id._comment).replace("(?:--\x3e|$)","--\x3e").getRegex(),Pd.emStrong.lDelim=bd(Pd.emStrong.lDelim).replace(/punct/g,Pd._punctuation).getRegex(),Pd.emStrong.rDelimAst=bd(Pd.emStrong.rDelimAst,"g").replace(/punct/g,Pd._punctuation).getRegex(),Pd.emStrong.rDelimUnd=bd(Pd.emStrong.rDelimUnd,"g").replace(/punct/g,Pd._punctuation).getRegex(),Pd._escapes=/\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/g,Pd._scheme=/[a-zA-Z][a-zA-Z0-9+.-]{1,31}/,Pd._email=/[a-zA-Z0-9.!#$%&'*+/=?^_`{|}~-]+(@)[a-zA-Z0-9](?:[a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?(?:\.[a-zA-Z0-9](?:[a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?)+(?![-_])/,Pd.autolink=bd(Pd.autolink).replace("scheme",Pd._scheme).replace("email",Pd._email).getRegex(),Pd._attribute=/\s+[a-zA-Z:_][\w.:-]*(?:\s*=\s*"[^"]*"|\s*=\s*'[^']*'|\s*=\s*[^\s"'=<>`]+)?/,Pd.tag=bd(Pd.tag).replace("comment",Pd._comment).replace("attribute",Pd._attribute).getRegex(),Pd._label=/(?:\[(?:\\.|[^\[\]\\])*\]|\\.|`[^`]*`|[^\[\]\\`])*?/,Pd._href=/<(?:\\.|[^\n<>\\])+>|[^\s\x00-\x1f]*/,Pd._title=/"(?:\\"?|[^"\\])*"|'(?:\\'?|[^'\\])*'|\((?:\\\)?|[^)\\])*\)/,Pd.link=bd(Pd.link).replace("label",Pd._label).replace("href",Pd._href).replace("title",Pd._title).getRegex(),Pd.reflink=bd(Pd.reflink).replace("label",Pd._label).replace("ref",Id._label).getRegex(),Pd.nolink=bd(Pd.nolink).replace("ref",Id._label).getRegex(),Pd.reflinkSearch=bd(Pd.reflinkSearch,"g").replace("reflink",Pd.reflink).replace("nolink",Pd.nolink).getRegex(),Pd.normal=Nd({},Pd),Pd.pedantic=Nd({},Pd.normal,{strong:{start:/^__|\*\*/,middle:/^__(?=\S)([\s\S]*?\S)__(?!_)|^\*\*(?=\S)([\s\S]*?\S)\*\*(?!\*)/,endAst:/\*\*(?!\*)/g,endUnd:/__(?!_)/g},em:{start:/^_|\*/,middle:/^()\*(?=\S)([\s\S]*?\S)\*(?!\*)|^_(?=\S)([\s\S]*?\S)_(?!_)/,endAst:/\*(?!\*)/g,endUnd:/_(?!_)/g},link:bd(/^!?\[(label)\]\((.*?)\)/).replace("label",Pd._label).getRegex(),reflink:bd(/^!?\[(label)\]\s*\[([^\]]*)\]/).replace("label",Pd._label).getRegex()}),Pd.gfm=Nd({},Pd.normal,{escape:bd(Pd.escape).replace("])","~|])").getRegex(),_extended_email:/[A-Za-z0-9._+-]+(@)[a-zA-Z0-9-_]+(?:\.[a-zA-Z0-9-_]*[a-zA-Z0-9])+(?![-_])/,url:/^((?:ftp|https?):\/\/|www\.)(?:[a-zA-Z0-9\-]+\.?)+[^\s<]*|^email/,_backpedal:/(?:[^?!.,:;*_'"~()&]+|\([^)]*\)|&(?![a-zA-Z0-9]+;$)|[?!.,:;*_'"~)]+(?!$))+/,del:/^(~~?)(?=[^\s~])([\s\S]*?[^\s~])\1(?=[^~]|$)/,text:/^([`~]+|[^`~])(?:(?= {2,}\n)|(?=[a-zA-Z0-9.!#$%&'*+\/=?_`{\|}~-]+@)|[\s\S]*?(?:(?=[\\1&&void 0!==arguments[1]?arguments[1]:[];e=this.options.pedantic?e.replace(/\t/g," ").replace(/^ +$/gm,""):e.replace(/^( *)(\t+)/gm,(function(e,t,n){return t+" ".repeat(n.length)}));for(var u=function(){if(o.options.extensions&&o.options.extensions.block&&o.options.extensions.block.some((function(n){return!!(t=n.call({lexer:o},e,a))&&(e=e.substring(t.raw.length),a.push(t),!0)})))return"continue";if(t=o.tokenizer.space(e))return e=e.substring(t.raw.length),1===t.raw.length&&a.length>0?a[a.length-1].raw+="\n":a.push(t),"continue";if(t=o.tokenizer.code(e))return e=e.substring(t.raw.length),!(n=a[a.length-1])||"paragraph"!==n.type&&"text"!==n.type?a.push(t):(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue[o.inlineQueue.length-1].src=n.text),"continue";if(t=o.tokenizer.fences(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.heading(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.hr(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.blockquote(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.list(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.html(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.def(e))return e=e.substring(t.raw.length),!(n=a[a.length-1])||"paragraph"!==n.type&&"text"!==n.type?o.tokens.links[t.tag]||(o.tokens.links[t.tag]={href:t.href,title:t.title}):(n.raw+="\n"+t.raw,n.text+="\n"+t.raw,o.inlineQueue[o.inlineQueue.length-1].src=n.text),"continue";if(t=o.tokenizer.table(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.lheading(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(r=e,o.options.extensions&&o.options.extensions.startBlock){var u,l=1/0,c=e.slice(1);o.options.extensions.startBlock.forEach((function(e){"number"===typeof(u=e.call({lexer:this},c))&&u>=0&&(l=Math.min(l,u))})),l<1/0&&l>=0&&(r=e.substring(0,l+1))}if(o.state.top&&(t=o.tokenizer.paragraph(r)))return n=a[a.length-1],i&&"paragraph"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue.pop(),o.inlineQueue[o.inlineQueue.length-1].src=n.text):a.push(t),i=r.length!==e.length,e=e.substring(t.raw.length),"continue";if(t=o.tokenizer.text(e))return e=e.substring(t.raw.length),(n=a[a.length-1])&&"text"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue.pop(),o.inlineQueue[o.inlineQueue.length-1].src=n.text):a.push(t),"continue";if(e){var s="Infinite loop on byte: "+e.charCodeAt(0);if(o.options.silent)return console.error(s),"break";throw new Error(s)}};e;){var l=u();if("continue"!==l&&"break"===l)break}return this.state.top=!0,a}},{key:"inline",value:function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[];return this.inlineQueue.push({src:e,tokens:t}),t}},{key:"inlineTokens",value:function(e){var t,n,r,i,o,a,u=this,l=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[],c=e;if(this.tokens.links){var s=Object.keys(this.tokens.links);if(s.length>0)for(;null!=(i=this.tokenizer.rules.inline.reflinkSearch.exec(c));)s.includes(i[0].slice(i[0].lastIndexOf("[")+1,-1))&&(c=c.slice(0,i.index)+"["+Od("a",i[0].length-2)+"]"+c.slice(this.tokenizer.rules.inline.reflinkSearch.lastIndex))}for(;null!=(i=this.tokenizer.rules.inline.blockSkip.exec(c));)c=c.slice(0,i.index)+"["+Od("a",i[0].length-2)+"]"+c.slice(this.tokenizer.rules.inline.blockSkip.lastIndex);for(;null!=(i=this.tokenizer.rules.inline.escapedEmSt.exec(c));)c=c.slice(0,i.index+i[0].length-2)+"++"+c.slice(this.tokenizer.rules.inline.escapedEmSt.lastIndex),this.tokenizer.rules.inline.escapedEmSt.lastIndex--;for(var f=function(){if(o||(a=""),o=!1,u.options.extensions&&u.options.extensions.inline&&u.options.extensions.inline.some((function(n){return!!(t=n.call({lexer:u},e,l))&&(e=e.substring(t.raw.length),l.push(t),!0)})))return"continue";if(t=u.tokenizer.escape(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.tag(e))return e=e.substring(t.raw.length),(n=l[l.length-1])&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(t=u.tokenizer.link(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.reflink(e,u.tokens.links))return e=e.substring(t.raw.length),(n=l[l.length-1])&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(t=u.tokenizer.emStrong(e,c,a))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.codespan(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.br(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.del(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.autolink(e,zd))return e=e.substring(t.raw.length),l.push(t),"continue";if(!u.state.inLink&&(t=u.tokenizer.url(e,zd)))return e=e.substring(t.raw.length),l.push(t),"continue";if(r=e,u.options.extensions&&u.options.extensions.startInline){var i,s=1/0,f=e.slice(1);u.options.extensions.startInline.forEach((function(e){"number"===typeof(i=e.call({lexer:this},f))&&i>=0&&(s=Math.min(s,i))})),s<1/0&&s>=0&&(r=e.substring(0,s+1))}if(t=u.tokenizer.inlineText(r,Rd))return e=e.substring(t.raw.length),"_"!==t.raw.slice(-1)&&(a=t.raw.slice(-1)),o=!0,(n=l[l.length-1])&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(e){var d="Infinite loop on byte: "+e.charCodeAt(0);if(u.options.silent)return console.error(d),"break";throw new Error(d)}};e;){var d=f();if("continue"!==d&&"break"===d)break}return l}}],[{key:"rules",get:function(){return{block:Id,inline:Pd}}},{key:"lex",value:function(t,n){return new e(n).lex(t)}},{key:"lexInline",value:function(t,n){return new e(n).inlineTokens(t)}}]),e}(),$d=function(){function e(t){Bt(this,e),this.options=t||cd}return Rt(e,[{key:"code",value:function(e,t,n){var r=(t||"").match(/\S*/)[0];if(this.options.highlight){var i=this.options.highlight(e,r);null!=i&&i!==e&&(n=!0,e=i)}return e=e.replace(/\n$/,"")+"\n",r?''+(n?e:md(e,!0))+"
\n":""+(n?e:md(e,!0))+"
\n"}},{key:"blockquote",value:function(e){return"\n".concat(e,"
\n")}},{key:"html",value:function(e){return e}},{key:"heading",value:function(e,t,n,r){if(this.options.headerIds){var i=this.options.headerPrefix+r.slug(n);return"').concat(e,"\n")}return"").concat(e,"\n")}},{key:"hr",value:function(){return this.options.xhtml?"
\n":"
\n"}},{key:"list",value:function(e,t,n){var r=t?"ol":"ul";return"<"+r+(t&&1!==n?' start="'+n+'"':"")+">\n"+e+""+r+">\n"}},{key:"listitem",value:function(e){return"".concat(e,"\n")}},{key:"checkbox",value:function(e){return" "}},{key:"paragraph",value:function(e){return"".concat(e,"
\n")}},{key:"table",value:function(e,t){return t&&(t="".concat(t,"")),"\n"}},{key:"tablerow",value:function(e){return"\n".concat(e,"
\n")}},{key:"tablecell",value:function(e,t){var n=t.header?"th":"td";return(t.align?"<".concat(n,' align="').concat(t.align,'">'):"<".concat(n,">"))+e+"".concat(n,">\n")}},{key:"strong",value:function(e){return"".concat(e,"")}},{key:"em",value:function(e){return"".concat(e,"")}},{key:"codespan",value:function(e){return"".concat(e,"
")}},{key:"br",value:function(){return this.options.xhtml?"
":"
"}},{key:"del",value:function(e){return"".concat(e,"")}},{key:"link",value:function(e,t,n){if(null===(e=xd(this.options.sanitize,this.options.baseUrl,e)))return n;var r='"+n+""}},{key:"image",value:function(e,t,n){if(null===(e=xd(this.options.sanitize,this.options.baseUrl,e)))return n;var r='":">"}},{key:"text",value:function(e){return e}}]),e}(),Hd=function(){function e(){Bt(this,e)}return Rt(e,[{key:"strong",value:function(e){return e}},{key:"em",value:function(e){return e}},{key:"codespan",value:function(e){return e}},{key:"del",value:function(e){return e}},{key:"html",value:function(e){return e}},{key:"text",value:function(e){return e}},{key:"link",value:function(e,t,n){return""+n}},{key:"image",value:function(e,t,n){return""+n}},{key:"br",value:function(){return""}}]),e}(),Ud=function(){function e(){Bt(this,e),this.seen={}}return Rt(e,[{key:"serialize",value:function(e){return e.toLowerCase().trim().replace(/<[!\/a-z].*?>/gi,"").replace(/[\u2000-\u206F\u2E00-\u2E7F\\'!"#$%&()*+,./:;<=>?@[\]^`{|}~]/g,"").replace(/\s/g,"-")}},{key:"getNextSafeSlug",value:function(e,t){var n=e,r=0;if(this.seen.hasOwnProperty(n)){r=this.seen[e];do{n=e+"-"+ ++r}while(this.seen.hasOwnProperty(n))}return t||(this.seen[e]=r,this.seen[n]=0),n}},{key:"slug",value:function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:{},n=this.serialize(e);return this.getNextSafeSlug(n,t.dryrun)}}]),e}(),Yd=function(){function e(t){Bt(this,e),this.options=t||cd,this.options.renderer=this.options.renderer||new $d,this.renderer=this.options.renderer,this.renderer.options=this.options,this.textRenderer=new Hd,this.slugger=new Ud}return Rt(e,[{key:"parse",value:function(e){var t,n,r,i,o,a,u,l,c,s,f,d,h,p,v,m,g,y,_,b=!(arguments.length>1&&void 0!==arguments[1])||arguments[1],D="",w=e.length;for(t=0;t0&&"paragraph"===v.tokens[0].type?(v.tokens[0].text=y+" "+v.tokens[0].text,v.tokens[0].tokens&&v.tokens[0].tokens.length>0&&"text"===v.tokens[0].tokens[0].type&&(v.tokens[0].tokens[0].text=y+" "+v.tokens[0].tokens[0].text)):v.tokens.unshift({type:"text",text:y}):p+=y),p+=this.parse(v.tokens,h),c+=this.renderer.listitem(p,g,m);D+=this.renderer.list(c,f,d);continue;case"html":D+=this.renderer.html(s.text);continue;case"paragraph":D+=this.renderer.paragraph(this.parseInline(s.tokens));continue;case"text":for(c=s.tokens?this.parseInline(s.tokens):s.text;t+1An error occurred:
"+md(e.message+"",!0)+"
";throw e}try{var l=jd.lex(e,t);if(t.walkTokens){if(t.async)return Promise.all(Vd.walkTokens(l,t.walkTokens)).then((function(){return Yd.parse(l,t)})).catch(u);Vd.walkTokens(l,t.walkTokens)}return Yd.parse(l,t)}catch(s){u(s)}}Vd.options=Vd.setOptions=function(e){var t;return Nd(Vd.defaults,e),t=Vd.defaults,cd=t,Vd},Vd.getDefaults=ld,Vd.defaults=cd,Vd.use=function(){for(var e=Vd.defaults.extensions||{renderers:{},childTokens:{}},t=arguments.length,n=new Array(t),r=0;rAn error occurred:"+md(s.message+"",!0)+"
";throw s}},Vd.Parser=Yd,Vd.parser=Yd.parse,Vd.Renderer=$d,Vd.TextRenderer=Hd,Vd.Lexer=jd,Vd.lexer=jd.lex,Vd.Tokenizer=Ld,Vd.Slugger=Ud,Vd.parse=Vd;Vd.options,Vd.setOptions,Vd.use,Vd.walkTokens,Vd.parseInline,Yd.parse,jd.lex;var qd=function(e){var t=e.title,n=e.description,r=e.unit,i=e.expr,o=e.showLegend,a=e.filename,u=e.alias,l=Xo().isMobile,c=Ni().period,s=$o().customStep,f=Fi(),d=ie(null),h=Ft(ee(!1),2),p=h[0],v=h[1],m=Ft(ee({limits:{enable:!1,range:{1:[0,0]}}}),2),y=m[0],_=m[1],b=ae((function(){return Array.isArray(i)&&i.every((function(e){return e}))}),[i]),D=Uf({predefinedQuery:b?i:[],display:"chart",visible:p,customStep:s}),w=D.isLoading,x=D.graphData,k=D.error,C=D.warning,A=function(e){var t=yr({},y);t.limits.range=e,_(t)};if(ne((function(){var e=new IntersectionObserver((function(e){e.forEach((function(e){return v(e.isIntersecting)}))}),{threshold:.1});return d.current&&e.observe(d.current),function(){d.current&&e.unobserve(d.current)}}),[d]),!b)return Vr(ta,{variant:"error",children:[Vr("code",{children:'"expr"'})," not found. Check the configuration file ",Vr("b",{children:a}),"."]});var E=function(){return Vr("div",{className:"vm-predefined-panel-header__description vm-default-styles",children:[n&&Vr(g,{children:[Vr("div",{children:[Vr("span",{children:"Description:"}),Vr("div",{dangerouslySetInnerHTML:{__html:Vd.parse(n)}})]}),Vr("hr",{})]}),Vr("div",{children:[Vr("span",{children:"Queries:"}),Vr("div",{children:i.map((function(e,t){return Vr("div",{children:e},"".concat(t,"_").concat(e))}))})]})]})};return Vr("div",{className:"vm-predefined-panel",ref:d,children:[Vr("div",{className:"vm-predefined-panel-header",children:[Vr(ma,{title:Vr(E,{}),children:Vr("div",{className:"vm-predefined-panel-header__info",children:Vr(Hi,{})})}),Vr("h3",{className:"vm-predefined-panel-header__title",children:t||""}),Vr(Wf,{yaxis:y,setYaxisLimits:A,toggleEnableLimits:function(){var e=yr({},y);e.limits.enable=!e.limits.enable,_(e)}})]}),Vr("div",{className:"vm-predefined-panel-body",children:[w&&Vr(Qf,{}),k&&Vr(ta,{variant:"error",children:k}),C&&Vr(ta,{variant:"warning",children:C}),x&&Vr(Nf,{data:x,period:c,customStep:s,query:i,yaxis:y,unit:r,alias:u,showLegend:o,setYaxisLimits:A,setPeriod:function(e){var t=e.from,n=e.to;f({type:"SET_PERIOD",payload:{from:t,to:n}})},fullWidth:!1,height:l?.5*window.innerHeight:500})]})]})},Wd=function(e){var t=e.index,n=e.title,r=e.panels,i=e.filename,o=bo(document.body),a=ae((function(){return o.width/12}),[o]),u=Ft(ee(!t),2),l=u[0],c=u[1],s=Ft(ee([]),2),f=s[0],d=s[1];ne((function(){d(r&&r.map((function(e){return e.width||12})))}),[r]);var h=Ft(ee({start:0,target:0,enable:!1}),2),p=h[0],v=h[1],m=function(e){if(p.enable){var t=p.start,n=Math.ceil((t-e.clientX)/a);if(!(Math.abs(n)>=12)){var r=f.map((function(e,t){return e-(t===p.target?n:0)}));d(r)}}},g=function(){v(yr(yr({},p),{},{enable:!1}))},y=function(e){return function(t){!function(e,t){v({start:e.clientX,target:t,enable:!0})}(t,e)}};return ne((function(){return window.addEventListener("mousemove",m),window.addEventListener("mouseup",g),function(){window.removeEventListener("mousemove",m),window.removeEventListener("mouseup",g)}}),[p]),Vr("div",{className:"vm-predefined-dashboard",children:Vr(qa,{defaultExpanded:l,onChange:function(e){return c(e)},title:Vr((function(){return Vr("div",{className:wo()({"vm-predefined-dashboard-header":!0,"vm-predefined-dashboard-header_open":l}),children:[(n||i)&&Vr("span",{className:"vm-predefined-dashboard-header__title",children:n||"".concat(t+1,". ").concat(i)}),r&&Vr("span",{className:"vm-predefined-dashboard-header__count",children:["(",r.length," panels)"]})]})}),{}),children:Vr("div",{className:"vm-predefined-dashboard-panels",children:Array.isArray(r)&&r.length?r.map((function(e,t){return Vr("div",{className:"vm-predefined-dashboard-panels-panel vm-block vm-block_empty-padding",style:{gridColumn:"span ".concat(f[t])},children:[Vr(qd,{title:e.title,description:e.description,unit:e.unit,expr:e.expr,alias:e.alias,filename:i,showLegend:e.showLegend}),Vr("button",{className:"vm-predefined-dashboard-panels-panel__resizer",onMouseDown:y(t)})]},t)})):Vr("div",{className:"vm-predefined-dashboard-panels-panel__alert",children:Vr(ta,{variant:"error",children:[Vr("code",{children:'"panels"'})," not found. Check the configuration file ",Vr("b",{children:i}),"."]})})})})})},Qd=function(){!function(){var e=Ni(),t=e.duration,n=e.relativeTime,r=e.period.date,i=$o().customStep,o=Ft(pr(),2)[1],a=function(){var e,a=id((mr(e={},"g0.range_input",t),mr(e,"g0.end_input",r),mr(e,"g0.step_input",i),mr(e,"g0.relative_time",n),e));o(a)};ne(a,[t,n,r,i]),ne(a,[])}();var e=Xo().isMobile,t=ua(),n=t.dashboardsSettings,r=t.dashboardsLoading,i=t.dashboardsError,o=Ft(ee(0),2),a=o[0],u=o[1],l=ae((function(){return n.map((function(e,t){return{label:e.title||"",value:t}}))}),[n]),c=ae((function(){return n[a]||{}}),[n,a]),s=ae((function(){return null===c||void 0===c?void 0:c.rows}),[c]),f=ae((function(){return c.title||c.filename||""}),[c]),d=ae((function(){return Array.isArray(s)&&!!s.length}),[s]),h=function(e){return function(){!function(e){u(e)}(e)}};return Vr("div",{className:"vm-predefined-panels",children:[r&&Vr(Qf,{}),i&&Vr(ta,{variant:"error",children:i}),!n.length&&Vr(ta,{variant:"info",children:"Dashboards not found"}),l.length>1&&Vr("div",{className:wo()({"vm-predefined-panels-tabs":!0,"vm-block":!0,"vm-block_mobile":e}),children:l.map((function(e){return Vr("div",{className:wo()({"vm-predefined-panels-tabs__tab":!0,"vm-predefined-panels-tabs__tab_active":e.value==a}),onClick:h(e.value),children:e.label},e.value)}))}),Vr("div",{className:"vm-predefined-panels__dashboards",children:[d&&s.map((function(e,t){return Vr(Wd,{index:t,filename:f,title:e.title,panels:e.panels},"".concat(a,"_").concat(t))})),!!n.length&&!d&&Vr(ta,{variant:"error",children:[Vr("code",{children:'"rows"'})," not found. Check the configuration file ",Vr("b",{children:f}),"."]})]})]})},Gd=function(e,t){var n=t.match?"&match[]="+encodeURIComponent(t.match):"",r=t.focusLabel?"&focusLabel="+encodeURIComponent(t.focusLabel):"";return"".concat(e,"/api/v1/status/tsdb?topN=").concat(t.topN,"&date=").concat(t.date).concat(n).concat(r)},Jd=function(){function e(){Bt(this,e),this.tsdbStatus=void 0,this.tabsNames=void 0,this.tsdbStatus=this.defaultTSDBStatus,this.tabsNames=["table","graph"]}return Rt(e,[{key:"tsdbStatusData",get:function(){return this.tsdbStatus},set:function(e){this.tsdbStatus=e}},{key:"defaultTSDBStatus",get:function(){return{totalSeries:0,totalLabelValuePairs:0,seriesCountByMetricName:[],seriesCountByLabelName:[],seriesCountByFocusLabelValue:[],seriesCountByLabelValuePair:[],labelValueCountByLabelName:[]}}},{key:"keys",value:function(e){var t=[];return e&&(t=t.concat("seriesCountByFocusLabelValue")),t=t.concat("seriesCountByMetricName","seriesCountByLabelName","seriesCountByLabelValuePair","labelValueCountByLabelName"),t}},{key:"defaultState",get:function(){var e=this;return this.keys("job").reduce((function(t,n){return yr(yr({},t),{},{tabs:yr(yr({},t.tabs),{},mr({},n,e.tabsNames)),containerRefs:yr(yr({},t.containerRefs),{},mr({},n,ie(null))),defaultActiveTab:yr(yr({},t.defaultActiveTab),{},mr({},n,0))})}),{tabs:{},containerRefs:{},defaultActiveTab:{}})}},{key:"sectionsTitles",value:function(e){return{seriesCountByMetricName:"Metric names with the highest number of series",seriesCountByLabelName:"Labels with the highest number of series",seriesCountByFocusLabelValue:'Values for "'.concat(e,'" label with the highest number of series'),seriesCountByLabelValuePair:"Label=value pairs with the highest number of series",labelValueCountByLabelName:"Labels with the highest number of unique values"}}},{key:"tablesHeaders",get:function(){return{seriesCountByMetricName:Zd,seriesCountByLabelName:Kd,seriesCountByFocusLabelValue:Xd,seriesCountByLabelValuePair:eh,labelValueCountByLabelName:th}}},{key:"totalSeries",value:function(e){return"labelValueCountByLabelName"===e?-1:this.tsdbStatus.totalSeries}}]),e}(),Zd=[{id:"name",label:"Metric name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],Kd=[{id:"name",label:"Label name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],Xd=[{id:"name",label:"Label value"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{disablePadding:!1,id:"action",label:"Action",numeric:!1}],eh=[{id:"name",label:"Label=value pair"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],th=[{id:"name",label:"Label name"},{id:"value",label:"Number of unique values"},{id:"action",label:"Action"}],nh={seriesCountByMetricName:function(e,t){return rh("__name__",t)},seriesCountByLabelName:function(e,t){return"{".concat(t,'!=""}')},seriesCountByFocusLabelValue:function(e,t){return rh(e,t)},seriesCountByLabelValuePair:function(e,t){var n=t.split("="),r=n[0],i=n.slice(1).join("=");return rh(r,i)},labelValueCountByLabelName:function(e,t){return"{".concat(t,'!=""}')}},rh=function(e,t){return e?"{"+e+"="+JSON.stringify(t)+"}":""},ih=function(e){var t=e.topN,n=e.error,r=e.query,i=e.onSetHistory,o=e.onRunQuery,a=e.onSetQuery,u=e.onTopNChange,l=e.onFocusLabelChange,c=e.totalSeries,s=e.totalLabelValuePairs,f=e.date,d=e.match,h=e.focusLabel,p=Li().autocomplete,v=Ii(),m=Xo().isMobile,g=Gf().queryOptions,y=ae((function(){return t<1?"Number must be bigger than zero":""}),[t]);return Vr("div",{className:wo()({"vm-cardinality-configurator":!0,"vm-block":!0,"vm-block_mobile":m}),children:[Vr("div",{className:"vm-cardinality-configurator-controls",children:[Vr("div",{className:"vm-cardinality-configurator-controls__query",children:Vr(Mf,{value:r,autocomplete:p,options:g,error:n,onArrowUp:function(){i(-1)},onArrowDown:function(){i(1)},onEnter:o,onChange:a,label:"Time series selector"})}),Vr("div",{className:"vm-cardinality-configurator-controls__item",children:Vr(xa,{label:"Number of entries per table",type:"number",value:t,error:y,onChange:u})}),Vr("div",{className:"vm-cardinality-configurator-controls__item",children:Vr(xa,{label:"Focus label",type:"text",value:h||"",onChange:l,endIcon:Vr(ma,{title:Vr("div",{children:[Vr("p",{children:"To identify values with the highest number of series for the selected label."}),Vr("p",{children:"Adds a table showing the series with the highest number of series."})]}),children:Vr(Hi,{})})})})]}),Vr("div",{className:"vm-cardinality-configurator-additional",children:Vr(Tf,{label:"Autocomplete",value:p,onChange:function(){v({type:"TOGGLE_AUTOCOMPLETE"})}})}),Vr("div",{className:wo()({"vm-cardinality-configurator-bottom":!0,"vm-cardinality-configurator-bottom_mobile":m}),children:[Vr("div",{className:"vm-cardinality-configurator-bottom__info",children:["Analyzed ",Vr("b",{children:c})," series with ",Vr("b",{children:s}),' "label=value" pairs at ',Vr("b",{children:f}),d&&Vr("span",{children:[" for series selector ",Vr("b",{children:d})]}),". Show top ",t," entries per table."]}),Vr("div",{className:"vm-cardinality-configurator-bottom__docs",children:[Vr("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/#cardinality-explorer",rel:"help noreferrer",children:[Vr(po,{}),"Documentation"]}),Vr("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://victoriametrics.com/blog/cardinality-explorer/",rel:"help noreferrer",children:[Vr(mo,{}),"Example of using"]})]}),Vr(fa,{startIcon:Vr(Xi,{}),onClick:o,fullWidth:!0,children:"Execute Query"})]})]})};function oh(e){var t=e.order,n=e.orderBy,r=e.onRequestSort,i=e.headerCells;return Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",children:i.map((function(e){return Vr("th",{className:wo()({"vm-table-cell vm-table-cell_header":!0,"vm-table-cell_sort":"action"!==e.id&&"percentage"!==e.id,"vm-table-cell_right":"action"===e.id}),onClick:(i=e.id,function(e){r(e,i)}),children:Vr("div",{className:"vm-table-cell__content",children:[e.label,"action"!==e.id&&"percentage"!==e.id&&Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":n===e.id,"vm-table__sort-icon_desc":"desc"===t&&n===e.id}),children:Vr(Qi,{})})]})},e.id);var i}))})})}function ah(e,t,n){return t[n]e[n]?1:0}function uh(e,t){return"desc"===e?function(e,n){return ah(e,n,t)}:function(e,n){return-ah(e,n,t)}}function lh(e,t){var n=e.map((function(e,t){return[e,t]}));return n.sort((function(e,n){var r=t(e[0],n[0]);return 0!==r?r:e[1]-n[1]})),n.map((function(e){return e[0]}))}var ch=function(e){var t=e.rows,n=e.headerCells,r=e.defaultSortColumn,i=e.tableCells,o=Ft(ee("desc"),2),a=o[0],u=o[1],l=Ft(ee(r),2),c=l[0],s=l[1],f=Ft(ee([]),2),d=f[0],h=f[1],p=function(e){return function(){var t=d.indexOf(e),n=[];-1===t?n=n.concat(d,e):0===t?n=n.concat(d.slice(1)):t===d.length-1?n=n.concat(d.slice(0,-1)):t>0&&(n=n.concat(d.slice(0,t),d.slice(t+1))),h(n)}},v=lh(t,uh(a,c));return Vr("table",{className:"vm-table",children:[Vr(oh,{numSelected:d.length,order:a,orderBy:c,onSelectAllClick:function(e){if(e.target.checked){var n=t.map((function(e){return e.name}));h(n)}else h([])},onRequestSort:function(e,t){u(c===t&&"asc"===a?"desc":"asc"),s(t)},rowCount:t.length,headerCells:n}),Vr("tbody",{className:"vm-table-header",children:v.map((function(e){return Vr("tr",{className:wo()({"vm-table__row":!0,"vm-table__row_selected":(t=e.name,-1!==d.indexOf(t))}),onClick:p(e.name),children:i(e)},e.name);var t}))})]})},sh=function(e){var t=e.row,n=e.totalSeries,r=e.onActionClick,i=n>0?t.value/n*100:-1;return Vr(g,{children:[Vr("td",{className:"vm-table-cell",children:t.name},t.name),Vr("td",{className:"vm-table-cell",children:t.value},t.value),i>0&&Vr("td",{className:"vm-table-cell",children:Vr(Jf,{value:i})},t.progressValue),Vr("td",{className:"vm-table-cell vm-table-cell_right",children:Vr("div",{className:"vm-table-cell__content",children:Vr(ma,{title:"Filter by ".concat(t.name),children:Vr(fa,{variant:"text",size:"small",onClick:function(){r(t.name)},children:Vr(eo,{})})})})},"action")]})},fh=function(e){var t=e.data,n=e.container,r=e.configs,i=Wr().isDarkTheme,o=ie(null),a=Ft(ee(),2),u=a[0],l=a[1],c=bo(n),s=yr(yr({},r),{},{width:c.width||400});return ne((function(){if(o.current){var e=new nf(s,t,o.current);return l(e),e.destroy}}),[o.current,c,i]),ne((function(){u&&u.setData(t)}),[t]),Vr("div",{style:{height:"100%"},children:Vr("div",{ref:o})})},dh=function(e,t){return Math.round(e*(t=Math.pow(10,t)))/t},hh=1,ph=function(e,t,n,r){return dh(t+e*(n+r),6)},vh=function(e,t,n,r,i){var o=1-t,a=n===hh?o/(e-1):2===n?o/e:3===n?o/(e+1):0;(isNaN(a)||a===1/0)&&(a=0);var u=n===hh?0:2===n?a/2:3===n?a:0,l=t/e,c=dh(l,6);if(null==r)for(var s=0;s=n&&e<=i&&t>=r&&t<=o};function gh(e,t,n,r,i){var o=this;o.x=e,o.y=t,o.w=n,o.h=r,o.l=i||0,o.o=[],o.q=null}var yh={split:function(){var e=this,t=e.x,n=e.y,r=e.w/2,i=e.h/2,o=e.l+1;e.q=[new gh(t+r,n,r,i,o),new gh(t,n,r,i,o),new gh(t,n+i,r,i,o),new gh(t+r,n+i,r,i,o)]},quads:function(e,t,n,r,i){var o=this,a=o.q,u=o.x+o.w/2,l=o.y+o.h/2,c=tu,d=t+r>l;c&&f&&i(a[0]),s&&c&&i(a[1]),s&&d&&i(a[2]),f&&d&&i(a[3])},add:function(e){var t=this;if(null!=t.q)t.quads(e.x,e.y,e.w,e.h,(function(t){t.add(e)}));else{var n=t.o;if(n.push(e),n.length>10&&t.l<4){t.split();for(var r=function(){var e=n[i];t.quads(e.x,e.y,e.w,e.h,(function(t){t.add(e)}))},i=0;i=0?"left":"right",e.ctx.textBaseline=1===s?"middle":i[n]>=0?"bottom":"top",e.ctx.fillText(i[n],f,y)}}))})),e.ctx.restore()}function b(e,t,n){return[0,nf.rangeNum(0,n,.05,!0)[1]]}return{hooks:{drawClear:function(t){var n;if((g=g||new gh(0,0,t.bbox.width,t.bbox.height)).clear(),t.series.forEach((function(e){e._paths=null})),l=d?[null].concat(m(t.data.length-1-o.length,t.data[0].length)):2===t.series.length?[null].concat(m(t.data[0].length,1)):[null].concat(function(e,t){var n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:h,r=Array.from({length:t},(function(){return{offs:Array(e).fill(0),size:Array(e).fill(0)}}));return vh(e,n,p,null,(function(e,n,i){vh(t,1,v,null,(function(t,o,a){r[t].offs[e]=n+i*o,r[t].size[e]=i*a}))})),r}(t.data[0].length,t.data.length-1-o.length,1===t.data[0].length?1:h)),null!=(null===(n=e.disp)||void 0===n?void 0:n.fill)){c=[null];for(var r=1;r0&&!o.includes(t)&&nf.assign(e,{paths:y,points:{show:_}})}))}}}((_h=[1],bh=0,Dh=1,wh=0,xh=function(e,t){return{stroke:e,fill:t}}({unit:3,values:function(e){return e.data[1].map((function(e,t){return 0!==t?"#33BB55":"#F79420"}))}},{unit:3,values:function(e){return e.data[1].map((function(e,t){return 0!==t?"#33BB55":"#F79420"}))}}),{which:_h,ori:bh,dir:Dh,radius:wh,disp:xh}))]},Ch=function(e){var t=e.rows,n=e.activeTab,r=e.onChange,i=e.tabs,o=e.chartContainer,a=e.totalSeries,u=e.tabId,l=e.onActionClick,c=e.sectionTitle,s=e.tableHeaderCells,f=Xo().isMobile,d=ae((function(){return i.map((function(e,t){return{value:String(t),label:e,icon:Vr(0===t?no:to,{})}}))}),[i]);return Vr("div",{className:wo()({"vm-metrics-content":!0,"vm-metrics-content_mobile":f,"vm-block":!0,"vm-block_mobile":f}),children:[Vr("div",{className:"vm-metrics-content-header vm-section-header",children:[Vr("h5",{className:wo()({"vm-section-header__title":!0,"vm-section-header__title_mobile":f}),children:c}),Vr("div",{className:"vm-section-header__tabs",children:Vr(Eo,{activeItem:String(n),items:d,onChange:function(e){r(e,u)}})})]}),Vr("div",{ref:o,className:wo()({"vm-metrics-content__table":!0,"vm-metrics-content__table_mobile":f}),children:[0===n&&Vr(ch,{rows:t,headerCells:s,defaultSortColumn:"value",tableCells:function(e){return Vr(sh,{row:e,totalSeries:a,onActionClick:l})}}),1===n&&Vr(fh,{data:[t.map((function(e){return e.name})),t.map((function(e){return e.value})),t.map((function(e,t){return t%12==0?1:t%10==0?2:0}))],container:(null===o||void 0===o?void 0:o.current)||null,configs:kh})]})]})},Ah=function(){var e=Xo().isMobile,t=qo(),n=t.topN,r=t.match,i=t.date,o=t.focusLabel,a=Wo();!function(){var e=qo(),t=e.topN,n=e.match,r=e.date,i=e.focusLabel,o=e.extraLabel,a=Ft(pr(),2)[1],u=function(){var e=id({topN:t,date:r,match:n,extraLabel:o,focusLabel:i});a(e)};ne(u,[t,n,r,i,o]),ne(u,[])}();var u=Ft(ee(r||""),2),l=u[0],c=u[1],s=Ft(ee(0),2),f=s[0],d=s[1],h=Ft(ee([]),2),p=h[0],v=h[1],m=function(){var e=new Jd,t=qo(),n=t.topN,r=t.extraLabel,i=t.match,o=t.date,a=t.runQuery,u=t.focusLabel,l=Wr().serverUrl,c=Ft(ee(!1),2),s=c[0],f=c[1],d=Ft(ee(),2),h=d[0],p=d[1],v=Ft(ee(e.defaultTSDBStatus),2),m=v[0],g=v[1];ne((function(){h&&(g(e.defaultTSDBStatus),f(!1))}),[h]);var y=function(){var t=tu(Xa().mark((function t(n){var r,i,o,a;return Xa().wrap((function(t){for(;;)switch(t.prev=t.next){case 0:if(l){t.next=2;break}return t.abrupt("return");case 2:return p(""),f(!0),g(e.defaultTSDBStatus),r=Gd(l,n),t.prev=6,t.next=9,fetch(r);case 9:return i=t.sent,t.next=12,i.json();case 12:o=t.sent,i.ok?(a=o.data,g(yr({},a)),f(!1)):(p(o.error),g(e.defaultTSDBStatus),f(!1)),t.next=20;break;case 16:t.prev=16,t.t0=t.catch(6),f(!1),t.t0 instanceof Error&&p("".concat(t.t0.name,": ").concat(t.t0.message));case 20:case"end":return t.stop()}}),t,null,[[6,16]])})));return function(e){return t.apply(this,arguments)}}();return ne((function(){y({topN:n,extraLabel:r,match:i,date:o,focusLabel:u})}),[l,a,o]),e.tsdbStatusData=m,{isLoading:s,appConfigurator:e,error:h}}(),g=m.isLoading,y=m.appConfigurator,_=m.error,b=Ft(ee(y.defaultState.defaultActiveTab),2),D=b[0],w=b[1],x=y.tsdbStatusData,k=y.defaultState,C=y.tablesHeaders,A=function(e,t){w(yr(yr({},D),{},mr({},t,+e)))};return Vr("div",{className:wo()({"vm-cardinality-panel":!0,"vm-cardinality-panel_mobile":e}),children:[g&&Vr(Qf,{message:"Please wait while cardinality stats is calculated. \n This may take some time if the db contains big number of time series."}),Vr(ih,{error:"",query:l,topN:n,date:i,match:r,totalSeries:x.totalSeries,totalLabelValuePairs:x.totalLabelValuePairs,focusLabel:o,onRunQuery:function(){v((function(e){return[].concat(Ot(e),[l])})),d((function(e){return e+1})),a({type:"SET_MATCH",payload:l}),a({type:"RUN_QUERY"})},onSetQuery:c,onSetHistory:function(e){var t=f+e;t<0||t>=p.length||(d(t),c(p[t]))},onTopNChange:function(e){a({type:"SET_TOP_N",payload:+e})},onFocusLabelChange:function(e){a({type:"SET_FOCUS_LABEL",payload:e})}}),_&&Vr(ta,{variant:"error",children:_}),y.keys(o).map((function(e){return Vr(Ch,{sectionTitle:y.sectionsTitles(o)[e],activeTab:D[e],rows:x[e],onChange:A,onActionClick:(t=e,function(e){var n=nh[t](o,e);c(n),v((function(e){return[].concat(Ot(e),[n])})),d((function(e){return e+1})),a({type:"SET_MATCH",payload:n});var r="";"labelValueCountByLabelName"!==t&&"seriesCountByLabelName"!=t||(r=e),a({type:"SET_FOCUS_LABEL",payload:r}),a({type:"RUN_QUERY"})}),tabs:k.tabs[e],chartContainer:k.containerRefs[e],totalSeries:y.totalSeries(e),tabId:e,tableHeaderCells:C[e]},e);var t}))]})},Eh=function(e){var t=e.rows,n=e.columns,r=Ft(ee(e.defaultOrderBy||"count"),2),i=r[0],o=r[1],a=Ft(ee("desc"),2),u=a[0],l=a[1],c=ae((function(){return lh(t,uh(u,i))}),[t,i,u]),s=function(e){return function(){var t;t=e,l((function(e){return"asc"===e&&i===t?"desc":"asc"})),o(t)}};return Vr("table",{className:"vm-table",children:[Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",children:n.map((function(e){return Vr("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:s(e.key),children:Vr("div",{className:"vm-table-cell__content",children:[e.title||e.key,Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":i===e.key,"vm-table__sort-icon_desc":"desc"===u&&i===e.key}),children:Vr(Qi,{})})]})},e.key)}))})}),Vr("tbody",{className:"vm-table-body",children:c.map((function(e,t){return Vr("tr",{className:"vm-table__row",children:n.map((function(t){return Vr("td",{className:"vm-table-cell",children:e[t.key]||"-"},t.key)}))},t)}))})]})},Sh=["table","JSON"].map((function(e,t){return{value:String(t),label:e,icon:Vr(0===t?no:ro,{})}})),Nh=function(e){var t=e.rows,n=e.title,r=e.columns,i=e.defaultOrderBy,o=Xo().isMobile,a=Ft(ee(0),2),u=a[0],l=a[1];return Vr("div",{className:wo()({"vm-top-queries-panel":!0,"vm-block":!0,"vm-block_mobile":o}),children:[Vr("div",{className:wo()({"vm-top-queries-panel-header":!0,"vm-section-header":!0,"vm-top-queries-panel-header_mobile":o}),children:[Vr("h5",{className:wo()({"vm-section-header__title":!0,"vm-section-header__title_mobile":o}),children:n}),Vr("div",{className:"vm-section-header__tabs",children:Vr(Eo,{activeItem:String(u),items:Sh,onChange:function(e){l(+e)}})})]}),Vr("div",{className:wo()({"vm-top-queries-panel__table":!0,"vm-top-queries-panel__table_mobile":o}),children:[0===u&&Vr(Eh,{rows:t,columns:r,defaultOrderBy:i}),1===u&&Vr(Yf,{data:t})]})]})},Fh=function(){var e=Xo().isMobile,t=function(){var e=Wr().serverUrl,t=Zo(),n=t.topN,r=t.maxLifetime,i=t.runQuery,o=Ft(ee(null),2),a=o[0],u=o[1],l=Ft(ee(!1),2),c=l[0],s=l[1],f=Ft(ee(),2),d=f[0],h=f[1],p=ae((function(){return function(e,t,n){return"".concat(e,"/api/v1/status/top_queries?topN=").concat(t||"","&maxLifetime=").concat(n||"")}(e,n,r)}),[e,n,r]),v=function(){var e=tu(Xa().mark((function e(){var t,n;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return s(!0),e.prev=1,e.next=4,fetch(p);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,t.ok&&["topByAvgDuration","topByCount","topBySumDuration"].forEach((function(e){var t=n[e];Array.isArray(t)&&t.forEach((function(e){return e.timeRangeHours=+(e.timeRangeSeconds/3600).toFixed(2)}))})),u(t.ok?n:null),h(String(n.error||"")),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&"AbortError"!==e.t0.name&&h("".concat(e.t0.name,": ").concat(e.t0.message));case 16:s(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();return ne((function(){v()}),[i]),{data:a,error:d,loading:c}}(),n=t.data,r=t.error,i=t.loading,o=Zo(),a=o.topN,u=o.maxLifetime,l=le(Jo).dispatch;!function(){var e=Zo(),t=e.topN,n=e.maxLifetime,r=Ft(pr(),2)[1],i=function(){var e=id({topN:String(t),maxLifetime:n});r(e)};ne(i,[t,n]),ne(i,[])}();var c=ae((function(){var e=u.trim().split(" ").reduce((function(e,t){var n=ui(t);return n?yr(yr({},e),n):yr({},e)}),{});return!!_t().duration(e).asMilliseconds()}),[u]),s=ae((function(){return!!a&&a<1}),[a]),f=ae((function(){return s?"Number must be bigger than zero":""}),[s]),d=ae((function(){return c?"":"Invalid duration value"}),[c]),h=function(e){if(!n)return e;var t=n[e];return"number"===typeof t?uf(t,t,t):t||e},p=function(){l({type:"SET_RUN_QUERY"})},v=function(e){"Enter"===e.key&&p()};return ne((function(){n&&(a||l({type:"SET_TOP_N",payload:+n.topN}),u||l({type:"SET_MAX_LIFE_TIME",payload:n.maxLifetime}))}),[n]),Vr("div",{className:wo()({"vm-top-queries":!0,"vm-top-queries_mobile":e}),children:[i&&Vr(Qf,{containerStyles:{height:"500px"}}),Vr("div",{className:wo()({"vm-top-queries-controls":!0,"vm-block":!0,"vm-block_mobile":e}),children:[Vr("div",{className:"vm-top-queries-controls-fields",children:[Vr("div",{className:"vm-top-queries-controls-fields__item",children:Vr(xa,{label:"Max lifetime",value:u,error:d,helperText:"For example ".concat("30ms, 15s, 3d4h, 1y2w"),onChange:function(e){l({type:"SET_MAX_LIFE_TIME",payload:e})},onKeyDown:v})}),Vr("div",{className:"vm-top-queries-controls-fields__item",children:Vr(xa,{label:"Number of returned queries",type:"number",value:a||"",error:f,onChange:function(e){l({type:"SET_TOP_N",payload:+e})},onKeyDown:v})})]}),Vr("div",{className:wo()({"vm-top-queries-controls-bottom":!0,"vm-top-queries-controls-bottom_mobile":e}),children:[Vr("div",{className:"vm-top-queries-controls-bottom__info",children:["VictoriaMetrics tracks the last\xa0",Vr(ma,{title:"search.queryStats.lastQueriesCount",children:Vr("b",{children:h("search.queryStats.lastQueriesCount")})}),"\xa0queries with durations at least\xa0",Vr(ma,{title:"search.queryStats.minQueryDuration",children:Vr("b",{children:h("search.queryStats.minQueryDuration")})})]}),Vr("div",{className:"vm-top-queries-controls-bottom__button",children:Vr(fa,{startIcon:Vr(Xi,{}),onClick:p,children:"Execute"})})]})]}),r&&Vr(ta,{variant:"error",children:r}),n&&Vr(g,{children:Vr("div",{className:"vm-top-queries-panels",children:[Vr(Nh,{rows:n.topByCount,title:"Most frequently executed queries",columns:[{key:"query"},{key:"timeRangeHours",title:"time range, hours"},{key:"count"}]}),Vr(Nh,{rows:n.topByAvgDuration,title:"Most heavy queries",columns:[{key:"query"},{key:"avgDurationSeconds",title:"avg duration, seconds"},{key:"timeRangeHours",title:"time range, hours"},{key:"count"}],defaultOrderBy:"avgDurationSeconds"}),Vr(Nh,{rows:n.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"})]})})]})},Mh={"color-primary":"#589DF6","color-secondary":"#316eca","color-error":"#e5534b","color-warning":"#c69026","color-info":"#539bf5","color-success":"#57ab5a","color-background-body":"#22272e","color-background-block":"#2d333b","color-background-tooltip":"rgba(22, 22, 22, 0.8)","color-text":"#cdd9e5","color-text-secondary":"#768390","color-text-disabled":"#636e7b","box-shadow":"rgba(0, 0, 0, 0.16) 1px 2px 6px","box-shadow-popper":"rgba(0, 0, 0, 0.2) 0px 2px 8px 0px","border-divider":"1px solid rgba(99, 110, 123, 0.5)","color-hover-black":"rgba(0, 0, 0, 0.12)"},Th={"color-primary":"#3F51B5","color-secondary":"#E91E63","color-error":"#FD080E","color-warning":"#FF8308","color-info":"#03A9F4","color-success":"#4CAF50","color-background-body":"#FEFEFF","color-background-block":"#FFFFFF","color-background-tooltip":"rgba(97,97,97, 0.92)","color-text":"#110f0f","color-text-secondary":"#706F6F","color-text-disabled":"#A09F9F","box-shadow":"rgba(0, 0, 0, 0.08) 1px 2px 6px","box-shadow-popper":"rgba(0, 0, 0, 0.1) 0px 2px 8px 0px","border-divider":"1px solid rgba(0, 0, 0, 0.15)","color-hover-black":"rgba(0, 0, 0, 0.06)"},Oh=function(){var e=Ft(ee(jr()),2),t=e[0],n=e[1],r=function(e){n(e.matches)};return ne((function(){var e=window.matchMedia("(prefers-color-scheme: dark)");return e.addEventListener("change",r),function(){return e.removeEventListener("change",r)}}),[]),t},Bh=["primary","secondary","error","warning","info","success"],Lh=function(e){var t,n=e.onLoaded,r=Ar(),i=Cr().palette,o=void 0===i?{}:i,a=Wr().theme,u=Oh(),l=Qr(),c=bo(document.body),s=Ft(ee((mr(t={},br.dark,Mh),mr(t,br.light,Th),mr(t,br.system,jr()?Mh:Th),t)),2),f=s[0],d=s[1],h=function(){var e=window,t=e.innerWidth,n=e.innerHeight,r=document.documentElement,i=r.clientWidth,o=r.clientHeight;zr("scrollbar-width","".concat(t-i,"px")),zr("scrollbar-height","".concat(n-o,"px")),zr("vh","".concat(.01*n,"px"))},p=function(){Bh.forEach((function(e,t){var r=function(e){var t=e.replace("#","").trim();if(3===t.length&&(t=t[0]+t[0]+t[1]+t[1]+t[2]+t[2]),6!==t.length)throw new Error("Invalid HEX color.");return(299*parseInt(t.slice(0,2),16)+587*parseInt(t.slice(2,4),16)+114*parseInt(t.slice(4,6),16))/1e3>=128?"#000000":"#FFFFFF"}(Rr("color-".concat(e)));zr("".concat(e,"-text"),r),t===Bh.length-1&&(l({type:"SET_DARK_THEME"}),n(!0))}))},v=function(){var e=Ir("THEME")||br.system,t=f[e];Object.entries(t).forEach((function(e){var t=Ft(e,2),n=t[0],r=t[1];zr(n,r)})),p(),r&&(Bh.forEach((function(e){var t=o[e];t&&zr("color-".concat(e),t)})),p())};return ne((function(){h(),v()}),[f]),ne(h,[c]),ne((function(){var e=jr()?Mh:Th;f[br.system]!==e?d((function(t){return yr(yr({},t),{},mr({},br.system,e))})):v()}),[a,u]),ne((function(){r&&l({type:"SET_THEME",payload:br.light})}),[]),null},Ih=function(){var e=Ft(ee(!1),2),t=e[0],n=e[1],r=Ft(ee([]),2),i=r[0],o=r[1],a=Ft(ee([]),2),u=a[0],l=a[1],c=ae((function(){return!!i.length}),[i]),f=Ft(pr(),2)[1],d=function(){n(!0)},h=function(){n(!1)},p=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:"";l((function(n){return[{filename:t,text:": ".concat(e.message)}].concat(Ot(n))}))},v=function(e,t){try{var n=JSON.parse(e),r=n.trace||n;if(!r.duration_msec)return void p(new Error(_r.traceNotFound),t);var i=new Hf(r,t);o((function(e){return[i].concat(Ot(e))}))}catch(s){s instanceof Error&&p(s,t)}},m=function(e){l([]),Array.from(e.target.files||[]).map((function(e){var t=new FileReader,n=(null===e||void 0===e?void 0:e.name)||"";t.onload=function(e){var t,r=String(null===(t=e.target)||void 0===t?void 0:t.result);v(r,n)},t.readAsText(e)})),e.target.value=""},g=function(e){return function(){!function(e){l((function(t){return t.filter((function(t,n){return n!==e}))}))}(e)}};ne((function(){f({})}),[]);var y=function(){return Vr("div",{className:"vm-trace-page-controls",children:[Vr(fa,{variant:"outlined",onClick:d,children:"Paste JSON"}),Vr(ma,{title:"The file must contain tracing information in JSON format",children:Vr(fa,{children:["Upload Files",Vr("input",{id:"json",type:"file",accept:"application/json",multiple:!0,title:" ",onChange:m})]})})]})};return Vr("div",{className:"vm-trace-page",children:[Vr("div",{className:"vm-trace-page-header",children:[Vr("div",{className:"vm-trace-page-header-errors",children:u.map((function(e,t){return Vr("div",{className:"vm-trace-page-header-errors-item",children:[Vr(ta,{variant:"error",children:[Vr("b",{className:"vm-trace-page-header-errors-item__filename",children:e.filename}),Vr("span",{children:e.text})]}),Vr(fa,{className:"vm-trace-page-header-errors-item__close",startIcon:Vr(ji,{}),variant:"text",color:"error",onClick:g(t)})]},"".concat(e,"_").concat(t))}))}),Vr("div",{children:c&&Vr(y,{})})]}),c&&Vr("div",{children:Vr(Xf,{jsonEditor:!0,traces:i,onDeleteClick:function(e){var t=i.filter((function(t){return t.idValue!==e.idValue}));o(Ot(t))}})}),!c&&Vr("div",{className:"vm-trace-page-preview",children:[Vr("p",{className:"vm-trace-page-preview__text",children:["Please, upload file with JSON response content.","\n","The file must contain tracing information in JSON format.","\n","In order to use tracing please refer to the doc:\xa0",Vr("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/#query-tracing",target:"_blank",rel:"help noreferrer",children:"https://docs.victoriametrics.com/#query-tracing"}),"\n","Tracing graph will be displayed after file upload."]}),Vr(y,{})]}),t&&Vr(va,{title:"Paste JSON",onClose:h,children:Vr(Kf,{editable:!0,displayTitle:!0,defaultTile:"JSON ".concat(i.length+1),onClose:h,onUpload:v})})]})},Ph=function(e){var t=Wr().serverUrl,n=Ni().period,r=Ft(ee([]),2),i=r[0],o=r[1],a=Ft(ee(!1),2),u=a[0],l=a[1],c=Ft(ee(),2),s=c[0],f=c[1],d=ae((function(){return function(e,t,n){var r="{job=".concat(JSON.stringify(n),"}");return"".concat(e,"/api/v1/label/instance/values?match[]=").concat(encodeURIComponent(r),"&start=").concat(t.start,"&end=").concat(t.end)}(t,n,e)}),[t,n,e]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return l(!0),e.prev=1,e.next=4,fetch(d);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],o(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?f(void 0):f("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&f("".concat(e.t0.name,": ").concat(e.t0.message));case 16:l(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[d]),{instances:i,isLoading:u,error:s}},Rh=function(e,t){var n=Wr().serverUrl,r=Ni().period,i=Ft(ee([]),2),o=i[0],a=i[1],u=Ft(ee(!1),2),l=u[0],c=u[1],s=Ft(ee(),2),f=s[0],d=s[1],h=ae((function(){return function(e,t,n,r){var i=Object.entries({job:n,instance:r}).filter((function(e){return e[1]})).map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(n,"=").concat(JSON.stringify(r))})).join(","),o="{".concat(i,"}");return"".concat(e,"/api/v1/label/__name__/values?match[]=").concat(encodeURIComponent(o),"&start=").concat(t.start,"&end=").concat(t.end)}(n,r,e,t)}),[n,r,e,t]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return c(!0),e.prev=1,e.next=4,fetch(h);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],a(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?d(void 0):d("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&d("".concat(e.t0.name,": ").concat(e.t0.message));case 16:c(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[h]),{names:o,isLoading:l,error:f}},zh=function(e){var t=e.name,n=e.job,r=e.instance,i=e.rateEnabled,o=e.isBucket,a=e.height,u=Xo().isMobile,l=$o(),c=l.customStep,s=l.yaxis,f=Ni().period,d=Ho(),h=Fi(),p=Ft(ee(!1),2),v=p[0],m=p[1],g=ae((function(){var e=Object.entries({job:n,instance:r}).filter((function(e){return e[1]})).map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(n,"=").concat(JSON.stringify(r))}));e.push("__name__=".concat(JSON.stringify(t))),"node_cpu_seconds_total"==t&&e.push('mode!="idle"');var a="{".concat(e.join(","),"}");if(o)return r?'\nlabel_map(\n histogram_quantiles("__name__", 0.5, 0.95, 0.99, sum(rate('.concat(a,')) by (vmrange, le)),\n "__name__",\n "0.5", "q50",\n "0.95", "q95",\n "0.99", "q99",\n)'):"\nwith (q = histogram_quantile(0.95, sum(rate(".concat(a,')) by (instance, vmrange, le))) (\n alias(min(q), "q95min"),\n alias(max(q), "q95max"),\n alias(avg(q), "q95avg"),\n)');var u=i?"rollup_rate(".concat(a,")"):"rollup(".concat(a,")");return"\nwith (q = ".concat(u,') (\n alias(min(label_match(q, "rollup", "min")), "min"),\n alias(max(label_match(q, "rollup", "max")), "max"),\n alias(avg(label_match(q, "rollup", "avg")), "avg"),\n)')}),[t,n,r,i,o]),y=Uf({predefinedQuery:[g],visible:!0,customStep:c,showAllSeries:v}),_=y.isLoading,b=y.graphData,D=y.error,w=y.warning;return Vr("div",{className:wo()({"vm-explore-metrics-graph":!0,"vm-explore-metrics-graph_mobile":u}),children:[_&&Vr(Qf,{}),D&&Vr(ta,{variant:"error",children:D}),w&&Vr(ta,{variant:"warning",children:Vr("div",{className:"vm-explore-metrics-graph__warning",children:[Vr("p",{children:w}),Vr(fa,{color:"warning",variant:"outlined",onClick:function(){m(!0)},children:"Show all"})]})}),b&&f&&Vr(Nf,{data:b,period:f,customStep:c,query:[g],yaxis:s,setYaxisLimits:function(e){d({type:"SET_YAXIS_LIMITS",payload:e})},setPeriod:function(e){var t=e.from,n=e.to;h({type:"SET_PERIOD",payload:{from:t,to:n}})},showLegend:!1,height:a})]})},jh=function(e){var t=e.name,n=e.index,r=e.length,i=e.isBucket,o=e.rateEnabled,a=e.onChangeRate,u=e.onRemoveItem,l=e.onChangeOrder,c=Xo().isMobile,s=Ft(ee(!1),2),f=s[0],d=s[1],h=function(){u(t)},p=function(){l(t,n,n+1)},v=function(){l(t,n,n-1)};return Vr("div",c?{className:"vm-explore-metrics-item-header vm-explore-metrics-item-header_mobile",children:[Vr("div",{className:"vm-explore-metrics-item-header__name",children:t}),Vr(fa,{variant:"text",size:"small",startIcon:Vr(yo,{}),onClick:function(){d(!0)}}),f&&Vr(va,{title:t,onClose:function(){d(!1)},children:Vr("div",{className:"vm-explore-metrics-item-header-modal",children:[Vr("div",{className:"vm-explore-metrics-item-header-modal-order",children:[Vr(fa,{startIcon:Vr(ao,{}),variant:"outlined",onClick:v,disabled:0===n}),Vr("p",{children:["position:",Vr("span",{className:"vm-explore-metrics-item-header-modal-order__index",children:["#",n+1]})]}),Vr(fa,{endIcon:Vr(oo,{}),variant:"outlined",onClick:p,disabled:n===r-1})]}),!i&&Vr("div",{className:"vm-explore-metrics-item-header-modal__rate",children:[Vr(Tf,{label:Vr("span",{children:["enable ",Vr("code",{children:"rate()"})]}),value:o,onChange:a,fullWidth:!0}),Vr("p",{children:"calculates the average per-second speed of metrics change"})]}),Vr(fa,{startIcon:Vr(ji,{}),color:"error",variant:"outlined",onClick:h,fullWidth:!0,children:"Remove graph"})]})})]}:{className:"vm-explore-metrics-item-header",children:[Vr("div",{className:"vm-explore-metrics-item-header-order",children:[Vr(ma,{title:"move graph up",children:Vr(fa,{className:"vm-explore-metrics-item-header-order__up",startIcon:Vr(Wi,{}),variant:"text",color:"gray",size:"small",onClick:v})}),Vr("div",{className:"vm-explore-metrics-item-header__index",children:["#",n+1]}),Vr(ma,{title:"move graph down",children:Vr(fa,{className:"vm-explore-metrics-item-header-order__down",startIcon:Vr(Wi,{}),variant:"text",color:"gray",size:"small",onClick:p})})]}),Vr("div",{className:"vm-explore-metrics-item-header__name",children:t}),!i&&Vr("div",{className:"vm-explore-metrics-item-header__rate",children:Vr(ma,{title:"calculates the average per-second speed of metric's change",children:Vr(Tf,{label:Vr("span",{children:["enable ",Vr("code",{children:"rate()"})]}),value:o,onChange:a})})}),Vr("div",{className:"vm-explore-metrics-item-header__close",children:Vr(ma,{title:"close graph",children:Vr(fa,{startIcon:Vr(ji,{}),variant:"text",color:"gray",size:"small",onClick:h})})})]})},$h=function(e){var t=e.name,n=e.job,r=e.instance,i=e.index,o=e.length,a=e.size,u=e.onRemoveItem,l=e.onChangeOrder,c=ae((function(){return/_sum?|_total?|_count?/.test(t)}),[t]),s=ae((function(){return/_bucket?/.test(t)}),[t]),f=Ft(ee(c),2),d=f[0],h=f[1],p=bo(document.body),v=ae(a.height,[a,p]);return ne((function(){h(c)}),[n]),Vr("div",{className:"vm-explore-metrics-item vm-block vm-block_empty-padding",children:[Vr(jh,{name:t,index:i,length:o,isBucket:s,rateEnabled:d,size:a.id,onChangeRate:h,onRemoveItem:u,onChangeOrder:l}),Vr(zh,{name:t,job:n,instance:r,rateEnabled:d,isBucket:s,height:v},"".concat(t,"_").concat(n,"_").concat(r,"_").concat(d))]})},Hh=function(e){var t=e.value,n=e.list,r=e.label,i=e.placeholder,o=e.noOptionsText,a=e.clearable,u=void 0!==a&&a,l=e.searchable,c=void 0!==l&&l,s=e.autofocus,f=e.onChange,d=Wr().isDarkTheme,h=Xo().isMobile,p=Ft(ee(""),2),v=p[0],m=p[1],g=ie(null),y=Ft(ee(!1),2),_=y[0],b=y[1],D=ie(null),w=ae((function(){return Array.isArray(t)}),[t]),x=ae((function(){return Array.isArray(t)?t:void 0}),[w,t]),k=ae((function(){return _?v:Array.isArray(t)?"":t}),[t,v,_,w]),C=ae((function(){return _?v||"(.+)":""}),[v,_]),A=function(){D.current&&D.current.blur()},E=function(e){f(e),w||(b(!1),A()),w&&D.current&&D.current.focus()},S=function(e){return function(t){E(e),t.stopPropagation()}},N=function(e){D.current!==e.target&&b(!1)};return ne((function(){m(""),_&&D.current&&D.current.focus(),_||A()}),[_,D]),ne((function(){s&&D.current&&!h&&D.current.focus()}),[s,D]),ne((function(){return window.addEventListener("keyup",N),function(){window.removeEventListener("keyup",N)}}),[]),Vr("div",{className:wo()({"vm-select":!0,"vm-select_dark":d}),children:[Vr("div",{className:"vm-select-input",onClick:function(e){e.target instanceof HTMLInputElement||b((function(e){return!e}))},ref:g,children:[Vr("div",{className:"vm-select-input-content",children:[!h&&x&&x.map((function(e){return Vr("div",{className:"vm-select-input-content__selected",children:[Vr("span",{children:e}),Vr("div",{onClick:S(e),children:Vr(ji,{})})]},e)})),h&&!(null===x||void 0===x||!x.length)&&Vr("span",{className:"vm-select-input-content__counter",children:["selected ",x.length]}),!h||h&&(!x||!(null!==x&&void 0!==x&&x.length))&&Vr("input",{value:k,type:"text",placeholder:i,onInput:function(e){m(e.target.value)},onFocus:function(){b(!0)},ref:D,readOnly:h||!c})]}),r&&Vr("span",{className:"vm-text-field__label",children:r}),u&&t&&Vr("div",{className:"vm-select-input__icon",onClick:S(""),children:Vr(ji,{})}),Vr("div",{className:wo()({"vm-select-input__icon":!0,"vm-select-input__icon_open":_}),children:Vr(Qi,{})})]}),Vr(Ff,{label:r,value:C,options:n,anchor:g,selected:x,maxWords:10,minLength:0,fullWidth:!0,noOptionsText:o,onSelect:E,onOpenAutocomplete:b})]})},Uh=Or.map((function(e){return e.id})),Yh=function(e){var t=e.jobs,n=e.instances,r=e.names,i=e.job,o=e.instance,a=e.size,u=e.selectedMetrics,l=e.onChangeJob,c=e.onChangeInstance,s=e.onToggleMetric,f=e.onChangeSize,d=ae((function(){return i?"":"No instances. Please select job"}),[i]),h=ae((function(){return i?"":"No metric names. Please select job"}),[i]),p=Xo().isMobile;return Vr("div",{className:wo()({"vm-explore-metrics-header":!0,"vm-explore-metrics-header_mobile":p,"vm-block":!0,"vm-block_mobile":p}),children:[Vr("div",{className:"vm-explore-metrics-header__job",children:Vr(Hh,{value:i,list:t,label:"Job",placeholder:"Please select job",onChange:l,autofocus:!i,searchable:!0})}),Vr("div",{className:"vm-explore-metrics-header__instance",children:Vr(Hh,{value:o,list:n,label:"Instance",placeholder:"Please select instance",onChange:c,noOptionsText:d,clearable:!0,searchable:!0})}),Vr("div",{className:"vm-explore-metrics-header__size",children:Vr(Hh,{label:"Size graphs",value:a,list:Uh,onChange:f})}),Vr("div",{className:"vm-explore-metrics-header-metrics",children:Vr(Hh,{label:"Metrics",value:u,list:r,placeholder:"Search metric name",onChange:s,noOptionsText:h,clearable:!0,searchable:!0})})]})},Vh=Br("job",""),qh=Br("instance",""),Wh=Br("metrics",""),Qh=Br("size",""),Gh=Or.find((function(e){return Qh?e.id===Qh:e.isDefault}))||Or[0],Jh=function(){var e=Ft(ee(Vh),2),t=e[0],n=e[1],r=Ft(ee(qh),2),i=r[0],o=r[1],a=Ft(ee(Wh?Wh.split("&"):[]),2),u=a[0],l=a[1],c=Ft(ee(Gh),2),s=c[0],f=c[1];!function(e){var t=e.job,n=e.instance,r=e.metrics,i=e.size,o=Ni(),a=o.duration,u=o.relativeTime,l=o.period.date,c=$o().customStep,s=Ft(pr(),2)[1],f=function(){var e,o=id((mr(e={},"g0.range_input",a),mr(e,"g0.end_input",l),mr(e,"g0.step_input",c),mr(e,"g0.relative_time",u),mr(e,"size",i),mr(e,"job",t),mr(e,"instance",n),mr(e,"metrics",r),e));s(o)};ne(f,[a,u,l,c,t,n,r,i]),ne(f,[])}({job:t,instance:i,metrics:u.join("&"),size:s.id});var d=function(){var e=Wr().serverUrl,t=Ni().period,n=Ft(ee([]),2),r=n[0],i=n[1],o=Ft(ee(!1),2),a=o[0],u=o[1],l=Ft(ee(),2),c=l[0],s=l[1],f=ae((function(){return function(e,t){return"".concat(e,"/api/v1/label/job/values?start=").concat(t.start,"&end=").concat(t.end)}(e,t)}),[e,t]);return ne((function(){var e=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return u(!0),e.prev=1,e.next=4,fetch(f);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],i(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?s(void 0):s("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&s("".concat(e.t0.name,": ").concat(e.t0.message));case 16:u(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();e().catch(console.error)}),[f]),{jobs:r,isLoading:a,error:c}}(),h=d.jobs,p=d.isLoading,v=d.error,m=Ph(t),g=m.instances,y=m.isLoading,_=m.error,b=Rh(t,i),D=b.names,w=b.isLoading,x=b.error,k=ae((function(){return p||y||w}),[p,y,w]),C=ae((function(){return v||_||x}),[v,_,x]),A=function(e){l(e?function(t){return t.includes(e)?t.filter((function(t){return t!==e})):[].concat(Ot(t),[e])}:[])},E=function(e,t,n){var r=n>u.length-1;n<0||r||l((function(e){var r=Ot(e),i=Ft(r.splice(t,1),1)[0];return r.splice(n,0,i),r}))};return ne((function(){i&&g.length&&!g.includes(i)&&o("")}),[g,i]),Vr("div",{className:"vm-explore-metrics",children:[Vr(Yh,{jobs:h,instances:g,names:D,job:t,size:s.id,instance:i,selectedMetrics:u,onChangeJob:n,onChangeSize:function(e){var t=Or.find((function(t){return t.id===e}));t&&f(t)},onChangeInstance:o,onToggleMetric:A}),k&&Vr(Qf,{}),C&&Vr(ta,{variant:"error",children:C}),!t&&Vr(ta,{variant:"info",children:"Please select job to see list of metric names."}),t&&!u.length&&Vr(ta,{variant:"info",children:"Please select metric names to see the graphs."}),Vr("div",{className:"vm-explore-metrics-body",children:u.map((function(e,n){return Vr($h,{name:e,job:t,instance:i,index:n,length:u.length,size:s,onRemoveItem:A,onChangeOrder:E},e)}))})]})},Zh=function(){var e=ra().showInfoMessage,n=function(t){return function(){var n;n=t,navigator.clipboard.writeText("<".concat(n,"/>")),e({text:"<".concat(n,"/> has been copied"),type:"success"})}};return Vr("div",{className:"vm-preview-icons",children:Object.entries(t).map((function(e){var t=Ft(e,2),r=t[0],i=t[1];return Vr("div",{className:"vm-preview-icons-item",onClick:n(r),children:[Vr("div",{className:"vm-preview-icons-item__svg",children:i()}),Vr("div",{className:"vm-preview-icons-item__name",children:"<".concat(r,"/>")})]},r)}))})},Kh=function(){var e=Ft(ee(!1),2),t=e[0],n=e[1];return Vr(g,{children:Vr(cr,{children:Vr(la,{children:Vr(g,{children:[Vr(Lh,{onLoaded:n}),t&&Vr(nr,{children:Vr(er,{path:"/",element:Vr(su,{}),children:[Vr(er,{path:kr.home,element:Vr(ud,{})}),Vr(er,{path:kr.metrics,element:Vr(Jh,{})}),Vr(er,{path:kr.cardinality,element:Vr(Ah,{})}),Vr(er,{path:kr.topQueries,element:Vr(Fh,{})}),Vr(er,{path:kr.trace,element:Vr(Ih,{})}),Vr(er,{path:kr.dashboards,element:Vr(Qd,{})}),Vr(er,{path:kr.icons,element:Vr(Zh,{})})]})})]})})})})},Xh=function(e){e&&n.e(27).then(n.bind(n,27)).then((function(t){var n=t.getCLS,r=t.getFID,i=t.getFCP,o=t.getLCP,a=t.getTTFB;n(e),r(e),i(e),o(e),a(e)}))},ep=document.getElementById("root");ep&&Ve(Vr(Kh,{}),ep),Xh()}()}();
\ No newline at end of file
+/*! For license information please see main.6eed9ce1.js.LICENSE.txt */
+!function(){var e={680:function(e,t,n){"use strict";var r=n(476),i=n(962),o=i(r("String.prototype.indexOf"));e.exports=function(e,t){var n=r(e,!!t);return"function"===typeof n&&o(e,".prototype.")>-1?i(n):n}},962:function(e,t,n){"use strict";var r=n(199),i=n(476),o=i("%Function.prototype.apply%"),a=i("%Function.prototype.call%"),u=i("%Reflect.apply%",!0)||r.call(a,o),l=i("%Object.getOwnPropertyDescriptor%",!0),c=i("%Object.defineProperty%",!0),s=i("%Math.max%");if(c)try{c({},"a",{value:1})}catch(d){c=null}e.exports=function(e){var t=u(r,a,arguments);if(l&&c){var n=l(t,"length");n.configurable&&c(t,"length",{value:1+s(0,e.length-(arguments.length-1))})}return t};var f=function(){return u(r,o,arguments)};c?c(e.exports,"apply",{value:f}):e.exports.apply=f},123:function(e,t){var n;!function(){"use strict";var r={}.hasOwnProperty;function i(){for(var e=[],t=0;t=t?e:""+Array(t+1-r.length).join(n)+e},y={s:g,z:function(e){var t=-e.utcOffset(),n=Math.abs(t),r=Math.floor(n/60),i=n%60;return(t<=0?"+":"-")+g(r,2,"0")+":"+g(i,2,"0")},m:function e(t,n){if(t.date()1)return e(a[0])}else{var u=t.name;b[u]=t,i=u}return!r&&i&&(_=i),i||!r&&_},x=function(e,t){if(D(e))return e.clone();var n="object"==typeof t?t:{};return n.date=e,n.args=arguments,new C(n)},k=y;k.l=w,k.i=D,k.w=function(e,t){return x(e,{locale:t.$L,utc:t.$u,x:t.$x,$offset:t.$offset})};var C=function(){function m(e){this.$L=w(e.locale,null,!0),this.parse(e)}var g=m.prototype;return g.parse=function(e){this.$d=function(e){var t=e.date,n=e.utc;if(null===t)return new Date(NaN);if(k.u(t))return new Date;if(t instanceof Date)return new Date(t);if("string"==typeof t&&!/Z$/i.test(t)){var r=t.match(p);if(r){var i=r[2]-1||0,o=(r[7]||"0").substring(0,3);return n?new Date(Date.UTC(r[1],i,r[3]||1,r[4]||0,r[5]||0,r[6]||0,o)):new Date(r[1],i,r[3]||1,r[4]||0,r[5]||0,r[6]||0,o)}}return new Date(t)}(e),this.$x=e.x||{},this.init()},g.init=function(){var e=this.$d;this.$y=e.getFullYear(),this.$M=e.getMonth(),this.$D=e.getDate(),this.$W=e.getDay(),this.$H=e.getHours(),this.$m=e.getMinutes(),this.$s=e.getSeconds(),this.$ms=e.getMilliseconds()},g.$utils=function(){return k},g.isValid=function(){return!(this.$d.toString()===h)},g.isSame=function(e,t){var n=x(e);return this.startOf(t)<=n&&n<=this.endOf(t)},g.isAfter=function(e,t){return x(e)=0&&(o[f]=parseInt(s,10))}var d=o[3],h=24===d?0:d,p=o[0]+"-"+o[1]+"-"+o[2]+" "+h+":"+o[4]+":"+o[5]+":000",v=+t;return(i.utc(p).valueOf()-(v-=v%1e3))/6e4},l=r.prototype;l.tz=function(e,t){void 0===e&&(e=o);var n=this.utcOffset(),r=this.toDate(),a=r.toLocaleString("en-US",{timeZone:e}),u=Math.round((r-new Date(a))/1e3/60),l=i(a).$set("millisecond",this.$ms).utcOffset(15*-Math.round(r.getTimezoneOffset()/15)-u,!0);if(t){var c=l.utcOffset();l=l.add(n-c,"minute")}return l.$x.$timezone=e,l},l.offsetName=function(e){var t=this.$x.$timezone||i.tz.guess(),n=a(this.valueOf(),t,{timeZoneName:e}).find((function(e){return"timezonename"===e.type.toLowerCase()}));return n&&n.value};var c=l.startOf;l.startOf=function(e,t){if(!this.$x||!this.$x.$timezone)return c.call(this,e,t);var n=i(this.format("YYYY-MM-DD HH:mm:ss:SSS"));return c.call(n,e,t).tz(this.$x.$timezone,!0)},i.tz=function(e,t,n){var r=n&&t,a=n||t||o,l=u(+i(),a);if("string"!=typeof e)return i(e).tz(a);var c=function(e,t,n){var r=e-60*t*1e3,i=u(r,n);if(t===i)return[r,t];var o=u(r-=60*(i-t)*1e3,n);return i===o?[r,i]:[e-60*Math.min(i,o)*1e3,Math.max(i,o)]}(i.utc(e,r).valueOf(),l,a),s=c[0],f=c[1],d=i(s).utcOffset(f);return d.$x.$timezone=a,d},i.tz.guess=function(){return Intl.DateTimeFormat().resolvedOptions().timeZone},i.tz.setDefault=function(e){o=e}}}()},635:function(e){e.exports=function(){"use strict";var e="minute",t=/[+-]\d\d(?::?\d\d)?/g,n=/([+-]|\d\d)/g;return function(r,i,o){var a=i.prototype;o.utc=function(e){return new i({date:e,utc:!0,args:arguments})},a.utc=function(t){var n=o(this.toDate(),{locale:this.$L,utc:!0});return t?n.add(this.utcOffset(),e):n},a.local=function(){return o(this.toDate(),{locale:this.$L,utc:!1})};var u=a.parse;a.parse=function(e){e.utc&&(this.$u=!0),this.$utils().u(e.$offset)||(this.$offset=e.$offset),u.call(this,e)};var l=a.init;a.init=function(){if(this.$u){var e=this.$d;this.$y=e.getUTCFullYear(),this.$M=e.getUTCMonth(),this.$D=e.getUTCDate(),this.$W=e.getUTCDay(),this.$H=e.getUTCHours(),this.$m=e.getUTCMinutes(),this.$s=e.getUTCSeconds(),this.$ms=e.getUTCMilliseconds()}else l.call(this)};var c=a.utcOffset;a.utcOffset=function(r,i){var o=this.$utils().u;if(o(r))return this.$u?0:o(this.$offset)?c.call(this):this.$offset;if("string"==typeof r&&(r=function(e){void 0===e&&(e="");var r=e.match(t);if(!r)return null;var i=(""+r[0]).match(n)||["-",0,0],o=i[0],a=60*+i[1]+ +i[2];return 0===a?0:"+"===o?a:-a}(r),null===r))return this;var a=Math.abs(r)<=16?60*r:r,u=this;if(i)return u.$offset=a,u.$u=0===r,u;if(0!==r){var l=this.$u?this.toDate().getTimezoneOffset():-1*this.utcOffset();(u=this.local().add(a+l,e)).$offset=a,u.$x.$localOffset=l}else u=this.utc();return u};var s=a.format;a.format=function(e){var t=e||(this.$u?"YYYY-MM-DDTHH:mm:ss[Z]":"");return s.call(this,t)},a.valueOf=function(){var e=this.$utils().u(this.$offset)?0:this.$offset+(this.$x.$localOffset||this.$d.getTimezoneOffset());return this.$d.valueOf()-6e4*e},a.isUTC=function(){return!!this.$u},a.toISOString=function(){return this.toDate().toISOString()},a.toString=function(){return this.toDate().toUTCString()};var f=a.toDate;a.toDate=function(e){return"s"===e&&this.$offset?o(this.format("YYYY-MM-DD HH:mm:ss:SSS")).toDate():f.call(this)};var d=a.diff;a.diff=function(e,t,n){if(e&&this.$u===e.$u)return d.call(this,e,t,n);var r=this.local(),i=o(e).local();return d.call(r,i,t,n)}}}()},781:function(e){"use strict";var t="Function.prototype.bind called on incompatible ",n=Array.prototype.slice,r=Object.prototype.toString,i="[object Function]";e.exports=function(e){var o=this;if("function"!==typeof o||r.call(o)!==i)throw new TypeError(t+o);for(var a,u=n.call(arguments,1),l=function(){if(this instanceof a){var t=o.apply(this,u.concat(n.call(arguments)));return Object(t)===t?t:this}return o.apply(e,u.concat(n.call(arguments)))},c=Math.max(0,o.length-u.length),s=[],f=0;f1&&"boolean"!==typeof t)throw new a('"allowMissing" argument must be a boolean');if(null===k(/^%?[^%]*%?$/,e))throw new i("`%` may not be present anywhere but at the beginning and end of the intrinsic name");var n=E(e),r=n.length>0?n[0]:"",o=S("%"+r+"%",t),u=o.name,c=o.value,s=!1,f=o.alias;f&&(r=f[0],D(n,b([0,1],f)));for(var d=1,h=!0;d=n.length){var y=l(c,p);c=(h=!!y)&&"get"in y&&!("originalValue"in y.get)?y.get:c[p]}else h=_(c,p),c=c[p];h&&!s&&(v[u]=c)}}return c}},520:function(e,t,n){"use strict";var r="undefined"!==typeof Symbol&&Symbol,i=n(541);e.exports=function(){return"function"===typeof r&&("function"===typeof Symbol&&("symbol"===typeof r("foo")&&("symbol"===typeof Symbol("bar")&&i())))}},541:function(e){"use strict";e.exports=function(){if("function"!==typeof Symbol||"function"!==typeof Object.getOwnPropertySymbols)return!1;if("symbol"===typeof Symbol.iterator)return!0;var e={},t=Symbol("test"),n=Object(t);if("string"===typeof t)return!1;if("[object Symbol]"!==Object.prototype.toString.call(t))return!1;if("[object Symbol]"!==Object.prototype.toString.call(n))return!1;for(t in e[t]=42,e)return!1;if("function"===typeof Object.keys&&0!==Object.keys(e).length)return!1;if("function"===typeof Object.getOwnPropertyNames&&0!==Object.getOwnPropertyNames(e).length)return!1;var r=Object.getOwnPropertySymbols(e);if(1!==r.length||r[0]!==t)return!1;if(!Object.prototype.propertyIsEnumerable.call(e,t))return!1;if("function"===typeof Object.getOwnPropertyDescriptor){var i=Object.getOwnPropertyDescriptor(e,t);if(42!==i.value||!0!==i.enumerable)return!1}return!0}},838:function(e,t,n){"use strict";var r=n(199);e.exports=r.call(Function.call,Object.prototype.hasOwnProperty)},936:function(e,t,n){var r=/^\s+|\s+$/g,i=/^[-+]0x[0-9a-f]+$/i,o=/^0b[01]+$/i,a=/^0o[0-7]+$/i,u=parseInt,l="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,c="object"==typeof self&&self&&self.Object===Object&&self,s=l||c||Function("return this")(),f=Object.prototype.toString,d=Math.max,h=Math.min,p=function(){return s.Date.now()};function v(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function m(e){if("number"==typeof e)return e;if(function(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==f.call(e)}(e))return NaN;if(v(e)){var t="function"==typeof e.valueOf?e.valueOf():e;e=v(t)?t+"":t}if("string"!=typeof e)return 0===e?e:+e;e=e.replace(r,"");var n=o.test(e);return n||a.test(e)?u(e.slice(2),n?2:8):i.test(e)?NaN:+e}e.exports=function(e,t,n){var r,i,o,a,u,l,c=0,s=!1,f=!1,g=!0;if("function"!=typeof e)throw new TypeError("Expected a function");function y(t){var n=r,o=i;return r=i=void 0,c=t,a=e.apply(o,n)}function _(e){return c=e,u=setTimeout(D,t),s?y(e):a}function b(e){var n=e-l;return void 0===l||n>=t||n<0||f&&e-c>=o}function D(){var e=p();if(b(e))return w(e);u=setTimeout(D,function(e){var n=t-(e-l);return f?h(n,o-(e-c)):n}(e))}function w(e){return u=void 0,g&&r?y(e):(r=i=void 0,a)}function x(){var e=p(),n=b(e);if(r=arguments,i=this,l=e,n){if(void 0===u)return _(l);if(f)return u=setTimeout(D,t),y(l)}return void 0===u&&(u=setTimeout(D,t)),a}return t=m(t)||0,v(n)&&(s=!!n.leading,o=(f="maxWait"in n)?d(m(n.maxWait)||0,t):o,g="trailing"in n?!!n.trailing:g),x.cancel=function(){void 0!==u&&clearTimeout(u),c=0,r=l=i=u=void 0},x.flush=function(){return void 0===u?a:w(p())},x}},7:function(e,t,n){var r="__lodash_hash_undefined__",i="[object Function]",o="[object GeneratorFunction]",a=/\.|\[(?:[^[\]]*|(["'])(?:(?!\1)[^\\]|\\.)*?\1)\]/,u=/^\w*$/,l=/^\./,c=/[^.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|$))/g,s=/\\(\\)?/g,f=/^\[object .+?Constructor\]$/,d="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,h="object"==typeof self&&self&&self.Object===Object&&self,p=d||h||Function("return this")();var v=Array.prototype,m=Function.prototype,g=Object.prototype,y=p["__core-js_shared__"],_=function(){var e=/[^.]+$/.exec(y&&y.keys&&y.keys.IE_PROTO||"");return e?"Symbol(src)_1."+e:""}(),b=m.toString,D=g.hasOwnProperty,w=g.toString,x=RegExp("^"+b.call(D).replace(/[\\^$.*+?()[\]{}|]/g,"\\$&").replace(/hasOwnProperty|(function).*?(?=\\\()| for .+?(?=\\\])/g,"$1.*?")+"$"),k=p.Symbol,C=v.splice,A=P(p,"Map"),E=P(Object,"create"),S=k?k.prototype:void 0,N=S?S.toString:void 0;function F(e){var t=-1,n=e?e.length:0;for(this.clear();++t-1},M.prototype.set=function(e,t){var n=this.__data__,r=O(n,e);return r<0?n.push([e,t]):n[r][1]=t,this},T.prototype.clear=function(){this.__data__={hash:new F,map:new(A||M),string:new F}},T.prototype.delete=function(e){return I(this,e).delete(e)},T.prototype.get=function(e){return I(this,e).get(e)},T.prototype.has=function(e){return I(this,e).has(e)},T.prototype.set=function(e,t){return I(this,e).set(e,t),this};var R=j((function(e){var t;e=null==(t=e)?"":function(e){if("string"==typeof e)return e;if(U(e))return N?N.call(e):"";var t=e+"";return"0"==t&&1/e==-1/0?"-0":t}(t);var n=[];return l.test(e)&&n.push(""),e.replace(c,(function(e,t,r,i){n.push(r?i.replace(s,"$1"):t||e)})),n}));function z(e){if("string"==typeof e||U(e))return e;var t=e+"";return"0"==t&&1/e==-1/0?"-0":t}function j(e,t){if("function"!=typeof e||t&&"function"!=typeof t)throw new TypeError("Expected a function");var n=function n(){var r=arguments,i=t?t.apply(this,r):r[0],o=n.cache;if(o.has(i))return o.get(i);var a=e.apply(this,r);return n.cache=o.set(i,a),a};return n.cache=new(j.Cache||T),n}j.Cache=T;var $=Array.isArray;function H(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function U(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==w.call(e)}e.exports=function(e,t,n){var r=null==e?void 0:B(e,t);return void 0===r?n:r}},61:function(e,t,n){var r="Expected a function",i=/^\s+|\s+$/g,o=/^[-+]0x[0-9a-f]+$/i,a=/^0b[01]+$/i,u=/^0o[0-7]+$/i,l=parseInt,c="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,s="object"==typeof self&&self&&self.Object===Object&&self,f=c||s||Function("return this")(),d=Object.prototype.toString,h=Math.max,p=Math.min,v=function(){return f.Date.now()};function m(e,t,n){var i,o,a,u,l,c,s=0,f=!1,d=!1,m=!0;if("function"!=typeof e)throw new TypeError(r);function _(t){var n=i,r=o;return i=o=void 0,s=t,u=e.apply(r,n)}function b(e){return s=e,l=setTimeout(w,t),f?_(e):u}function D(e){var n=e-c;return void 0===c||n>=t||n<0||d&&e-s>=a}function w(){var e=v();if(D(e))return x(e);l=setTimeout(w,function(e){var n=t-(e-c);return d?p(n,a-(e-s)):n}(e))}function x(e){return l=void 0,m&&i?_(e):(i=o=void 0,u)}function k(){var e=v(),n=D(e);if(i=arguments,o=this,c=e,n){if(void 0===l)return b(c);if(d)return l=setTimeout(w,t),_(c)}return void 0===l&&(l=setTimeout(w,t)),u}return t=y(t)||0,g(n)&&(f=!!n.leading,a=(d="maxWait"in n)?h(y(n.maxWait)||0,t):a,m="trailing"in n?!!n.trailing:m),k.cancel=function(){void 0!==l&&clearTimeout(l),s=0,i=c=o=l=void 0},k.flush=function(){return void 0===l?u:x(v())},k}function g(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function y(e){if("number"==typeof e)return e;if(function(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&"[object Symbol]"==d.call(e)}(e))return NaN;if(g(e)){var t="function"==typeof e.valueOf?e.valueOf():e;e=g(t)?t+"":t}if("string"!=typeof e)return 0===e?e:+e;e=e.replace(i,"");var n=a.test(e);return n||u.test(e)?l(e.slice(2),n?2:8):o.test(e)?NaN:+e}e.exports=function(e,t,n){var i=!0,o=!0;if("function"!=typeof e)throw new TypeError(r);return g(n)&&(i="leading"in n?!!n.leading:i,o="trailing"in n?!!n.trailing:o),m(e,t,{leading:i,maxWait:t,trailing:o})}},154:function(e,t,n){var r="function"===typeof Map&&Map.prototype,i=Object.getOwnPropertyDescriptor&&r?Object.getOwnPropertyDescriptor(Map.prototype,"size"):null,o=r&&i&&"function"===typeof i.get?i.get:null,a=r&&Map.prototype.forEach,u="function"===typeof Set&&Set.prototype,l=Object.getOwnPropertyDescriptor&&u?Object.getOwnPropertyDescriptor(Set.prototype,"size"):null,c=u&&l&&"function"===typeof l.get?l.get:null,s=u&&Set.prototype.forEach,f="function"===typeof WeakMap&&WeakMap.prototype?WeakMap.prototype.has:null,d="function"===typeof WeakSet&&WeakSet.prototype?WeakSet.prototype.has:null,h="function"===typeof WeakRef&&WeakRef.prototype?WeakRef.prototype.deref:null,p=Boolean.prototype.valueOf,v=Object.prototype.toString,m=Function.prototype.toString,g=String.prototype.match,y=String.prototype.slice,_=String.prototype.replace,b=String.prototype.toUpperCase,D=String.prototype.toLowerCase,w=RegExp.prototype.test,x=Array.prototype.concat,k=Array.prototype.join,C=Array.prototype.slice,A=Math.floor,E="function"===typeof BigInt?BigInt.prototype.valueOf:null,S=Object.getOwnPropertySymbols,N="function"===typeof Symbol&&"symbol"===typeof Symbol.iterator?Symbol.prototype.toString:null,F="function"===typeof Symbol&&"object"===typeof Symbol.iterator,M="function"===typeof Symbol&&Symbol.toStringTag&&(typeof Symbol.toStringTag===F||"symbol")?Symbol.toStringTag:null,T=Object.prototype.propertyIsEnumerable,O=("function"===typeof Reflect?Reflect.getPrototypeOf:Object.getPrototypeOf)||([].__proto__===Array.prototype?function(e){return e.__proto__}:null);function B(e,t){if(e===1/0||e===-1/0||e!==e||e&&e>-1e3&&e<1e3||w.call(/e/,t))return t;var n=/[0-9](?=(?:[0-9]{3})+(?![0-9]))/g;if("number"===typeof e){var r=e<0?-A(-e):A(e);if(r!==e){var i=String(r),o=y.call(t,i.length+1);return _.call(i,n,"$&_")+"."+_.call(_.call(o,/([0-9]{3})/g,"$&_"),/_$/,"")}}return _.call(t,n,"$&_")}var L=n(654),I=L.custom,P=H(I)?I:null;function R(e,t,n){var r="double"===(n.quoteStyle||t)?'"':"'";return r+e+r}function z(e){return _.call(String(e),/"/g,""")}function j(e){return"[object Array]"===V(e)&&(!M||!("object"===typeof e&&M in e))}function $(e){return"[object RegExp]"===V(e)&&(!M||!("object"===typeof e&&M in e))}function H(e){if(F)return e&&"object"===typeof e&&e instanceof Symbol;if("symbol"===typeof e)return!0;if(!e||"object"!==typeof e||!N)return!1;try{return N.call(e),!0}catch(t){}return!1}e.exports=function e(t,n,r,i){var u=n||{};if(Y(u,"quoteStyle")&&"single"!==u.quoteStyle&&"double"!==u.quoteStyle)throw new TypeError('option "quoteStyle" must be "single" or "double"');if(Y(u,"maxStringLength")&&("number"===typeof u.maxStringLength?u.maxStringLength<0&&u.maxStringLength!==1/0:null!==u.maxStringLength))throw new TypeError('option "maxStringLength", if provided, must be a positive integer, Infinity, or `null`');var l=!Y(u,"customInspect")||u.customInspect;if("boolean"!==typeof l&&"symbol"!==l)throw new TypeError("option \"customInspect\", if provided, must be `true`, `false`, or `'symbol'`");if(Y(u,"indent")&&null!==u.indent&&"\t"!==u.indent&&!(parseInt(u.indent,10)===u.indent&&u.indent>0))throw new TypeError('option "indent" must be "\\t", an integer > 0, or `null`');if(Y(u,"numericSeparator")&&"boolean"!==typeof u.numericSeparator)throw new TypeError('option "numericSeparator", if provided, must be `true` or `false`');var v=u.numericSeparator;if("undefined"===typeof t)return"undefined";if(null===t)return"null";if("boolean"===typeof t)return t?"true":"false";if("string"===typeof t)return W(t,u);if("number"===typeof t){if(0===t)return 1/0/t>0?"0":"-0";var b=String(t);return v?B(t,b):b}if("bigint"===typeof t){var w=String(t)+"n";return v?B(t,w):w}var A="undefined"===typeof u.depth?5:u.depth;if("undefined"===typeof r&&(r=0),r>=A&&A>0&&"object"===typeof t)return j(t)?"[Array]":"[Object]";var S=function(e,t){var n;if("\t"===e.indent)n="\t";else{if(!("number"===typeof e.indent&&e.indent>0))return null;n=k.call(Array(e.indent+1)," ")}return{base:n,prev:k.call(Array(t+1),n)}}(u,r);if("undefined"===typeof i)i=[];else if(q(i,t)>=0)return"[Circular]";function I(t,n,o){if(n&&(i=C.call(i)).push(n),o){var a={depth:u.depth};return Y(u,"quoteStyle")&&(a.quoteStyle=u.quoteStyle),e(t,a,r+1,i)}return e(t,u,r+1,i)}if("function"===typeof t&&!$(t)){var U=function(e){if(e.name)return e.name;var t=g.call(m.call(e),/^function\s*([\w$]+)/);if(t)return t[1];return null}(t),Q=X(t,I);return"[Function"+(U?": "+U:" (anonymous)")+"]"+(Q.length>0?" { "+k.call(Q,", ")+" }":"")}if(H(t)){var ee=F?_.call(String(t),/^(Symbol\(.*\))_[^)]*$/,"$1"):N.call(t);return"object"!==typeof t||F?ee:G(ee)}if(function(e){if(!e||"object"!==typeof e)return!1;if("undefined"!==typeof HTMLElement&&e instanceof HTMLElement)return!0;return"string"===typeof e.nodeName&&"function"===typeof e.getAttribute}(t)){for(var te="<"+D.call(String(t.nodeName)),ne=t.attributes||[],re=0;re",t.childNodes&&t.childNodes.length&&(te+="..."),te+=""+D.call(String(t.nodeName))+">"}if(j(t)){if(0===t.length)return"[]";var ie=X(t,I);return S&&!function(e){for(var t=0;t=0)return!1;return!0}(ie)?"["+K(ie,S)+"]":"[ "+k.call(ie,", ")+" ]"}if(function(e){return"[object Error]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t)){var oe=X(t,I);return"cause"in Error.prototype||!("cause"in t)||T.call(t,"cause")?0===oe.length?"["+String(t)+"]":"{ ["+String(t)+"] "+k.call(oe,", ")+" }":"{ ["+String(t)+"] "+k.call(x.call("[cause]: "+I(t.cause),oe),", ")+" }"}if("object"===typeof t&&l){if(P&&"function"===typeof t[P]&&L)return L(t,{depth:A-r});if("symbol"!==l&&"function"===typeof t.inspect)return t.inspect()}if(function(e){if(!o||!e||"object"!==typeof e)return!1;try{o.call(e);try{c.call(e)}catch(te){return!0}return e instanceof Map}catch(t){}return!1}(t)){var ae=[];return a&&a.call(t,(function(e,n){ae.push(I(n,t,!0)+" => "+I(e,t))})),Z("Map",o.call(t),ae,S)}if(function(e){if(!c||!e||"object"!==typeof e)return!1;try{c.call(e);try{o.call(e)}catch(t){return!0}return e instanceof Set}catch(n){}return!1}(t)){var ue=[];return s&&s.call(t,(function(e){ue.push(I(e,t))})),Z("Set",c.call(t),ue,S)}if(function(e){if(!f||!e||"object"!==typeof e)return!1;try{f.call(e,f);try{d.call(e,d)}catch(te){return!0}return e instanceof WeakMap}catch(t){}return!1}(t))return J("WeakMap");if(function(e){if(!d||!e||"object"!==typeof e)return!1;try{d.call(e,d);try{f.call(e,f)}catch(te){return!0}return e instanceof WeakSet}catch(t){}return!1}(t))return J("WeakSet");if(function(e){if(!h||!e||"object"!==typeof e)return!1;try{return h.call(e),!0}catch(t){}return!1}(t))return J("WeakRef");if(function(e){return"[object Number]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(I(Number(t)));if(function(e){if(!e||"object"!==typeof e||!E)return!1;try{return E.call(e),!0}catch(t){}return!1}(t))return G(I(E.call(t)));if(function(e){return"[object Boolean]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(p.call(t));if(function(e){return"[object String]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t))return G(I(String(t)));if(!function(e){return"[object Date]"===V(e)&&(!M||!("object"===typeof e&&M in e))}(t)&&!$(t)){var le=X(t,I),ce=O?O(t)===Object.prototype:t instanceof Object||t.constructor===Object,se=t instanceof Object?"":"null prototype",fe=!ce&&M&&Object(t)===t&&M in t?y.call(V(t),8,-1):se?"Object":"",de=(ce||"function"!==typeof t.constructor?"":t.constructor.name?t.constructor.name+" ":"")+(fe||se?"["+k.call(x.call([],fe||[],se||[]),": ")+"] ":"");return 0===le.length?de+"{}":S?de+"{"+K(le,S)+"}":de+"{ "+k.call(le,", ")+" }"}return String(t)};var U=Object.prototype.hasOwnProperty||function(e){return e in this};function Y(e,t){return U.call(e,t)}function V(e){return v.call(e)}function q(e,t){if(e.indexOf)return e.indexOf(t);for(var n=0,r=e.length;nt.maxStringLength){var n=e.length-t.maxStringLength,r="... "+n+" more character"+(n>1?"s":"");return W(y.call(e,0,t.maxStringLength),t)+r}return R(_.call(_.call(e,/(['\\])/g,"\\$1"),/[\x00-\x1f]/g,Q),"single",t)}function Q(e){var t=e.charCodeAt(0),n={8:"b",9:"t",10:"n",12:"f",13:"r"}[t];return n?"\\"+n:"\\x"+(t<16?"0":"")+b.call(t.toString(16))}function G(e){return"Object("+e+")"}function J(e){return e+" { ? }"}function Z(e,t,n,r){return e+" ("+t+") {"+(r?K(n,r):k.call(n,", "))+"}"}function K(e,t){if(0===e.length)return"";var n="\n"+t.prev+t.base;return n+k.call(e,","+n)+"\n"+t.prev}function X(e,t){var n=j(e),r=[];if(n){r.length=e.length;for(var i=0;i-1?e.split(","):e},c=function(e,t,n,r){if(e){var o=n.allowDots?e.replace(/\.([^.[]+)/g,"[$1]"):e,a=/(\[[^[\]]*])/g,u=n.depth>0&&/(\[[^[\]]*])/.exec(o),c=u?o.slice(0,u.index):o,s=[];if(c){if(!n.plainObjects&&i.call(Object.prototype,c)&&!n.allowPrototypes)return;s.push(c)}for(var f=0;n.depth>0&&null!==(u=a.exec(o))&&f=0;--o){var a,u=e[o];if("[]"===u&&n.parseArrays)a=[].concat(i);else{a=n.plainObjects?Object.create(null):{};var c="["===u.charAt(0)&&"]"===u.charAt(u.length-1)?u.slice(1,-1):u,s=parseInt(c,10);n.parseArrays||""!==c?!isNaN(s)&&u!==c&&String(s)===c&&s>=0&&n.parseArrays&&s<=n.arrayLimit?(a=[])[s]=i:"__proto__"!==c&&(a[c]=i):a={0:i}}i=a}return i}(s,t,n,r)}};e.exports=function(e,t){var n=function(e){if(!e)return a;if(null!==e.decoder&&void 0!==e.decoder&&"function"!==typeof e.decoder)throw new TypeError("Decoder has to be a function.");if("undefined"!==typeof e.charset&&"utf-8"!==e.charset&&"iso-8859-1"!==e.charset)throw new TypeError("The charset option must be either utf-8, iso-8859-1, or undefined");var t="undefined"===typeof e.charset?a.charset:e.charset;return{allowDots:"undefined"===typeof e.allowDots?a.allowDots:!!e.allowDots,allowPrototypes:"boolean"===typeof e.allowPrototypes?e.allowPrototypes:a.allowPrototypes,allowSparse:"boolean"===typeof e.allowSparse?e.allowSparse:a.allowSparse,arrayLimit:"number"===typeof e.arrayLimit?e.arrayLimit:a.arrayLimit,charset:t,charsetSentinel:"boolean"===typeof e.charsetSentinel?e.charsetSentinel:a.charsetSentinel,comma:"boolean"===typeof e.comma?e.comma:a.comma,decoder:"function"===typeof e.decoder?e.decoder:a.decoder,delimiter:"string"===typeof e.delimiter||r.isRegExp(e.delimiter)?e.delimiter:a.delimiter,depth:"number"===typeof e.depth||!1===e.depth?+e.depth:a.depth,ignoreQueryPrefix:!0===e.ignoreQueryPrefix,interpretNumericEntities:"boolean"===typeof e.interpretNumericEntities?e.interpretNumericEntities:a.interpretNumericEntities,parameterLimit:"number"===typeof e.parameterLimit?e.parameterLimit:a.parameterLimit,parseArrays:!1!==e.parseArrays,plainObjects:"boolean"===typeof e.plainObjects?e.plainObjects:a.plainObjects,strictNullHandling:"boolean"===typeof e.strictNullHandling?e.strictNullHandling:a.strictNullHandling}}(t);if(""===e||null===e||"undefined"===typeof e)return n.plainObjects?Object.create(null):{};for(var s="string"===typeof e?function(e,t){var n,c={},s=t.ignoreQueryPrefix?e.replace(/^\?/,""):e,f=t.parameterLimit===1/0?void 0:t.parameterLimit,d=s.split(t.delimiter,f),h=-1,p=t.charset;if(t.charsetSentinel)for(n=0;n-1&&(m=o(m)?[m]:m),i.call(c,v)?c[v]=r.combine(c[v],m):c[v]=m}return c}(e,n):e,f=n.plainObjects?Object.create(null):{},d=Object.keys(s),h=0;h0?C.join(",")||null:void 0}];else if(l(h))B=h;else{var I=Object.keys(C);B=m?I.sort(m):I}for(var P=a&&l(C)&&1===C.length?n+"[]":n,R=0;R0?D+b:""}},837:function(e,t,n){"use strict";var r=n(609),i=Object.prototype.hasOwnProperty,o=Array.isArray,a=function(){for(var e=[],t=0;t<256;++t)e.push("%"+((t<16?"0":"")+t.toString(16)).toUpperCase());return e}(),u=function(e,t){for(var n=t&&t.plainObjects?Object.create(null):{},r=0;r1;){var t=e.pop(),n=t.obj[t.prop];if(o(n)){for(var r=[],i=0;i=48&&s<=57||s>=65&&s<=90||s>=97&&s<=122||o===r.RFC1738&&(40===s||41===s)?l+=u.charAt(c):s<128?l+=a[s]:s<2048?l+=a[192|s>>6]+a[128|63&s]:s<55296||s>=57344?l+=a[224|s>>12]+a[128|s>>6&63]+a[128|63&s]:(c+=1,s=65536+((1023&s)<<10|1023&u.charCodeAt(c)),l+=a[240|s>>18]+a[128|s>>12&63]+a[128|s>>6&63]+a[128|63&s])}return l},isBuffer:function(e){return!(!e||"object"!==typeof e)&&!!(e.constructor&&e.constructor.isBuffer&&e.constructor.isBuffer(e))},isRegExp:function(e){return"[object RegExp]"===Object.prototype.toString.call(e)},maybeMap:function(e,t){if(o(e)){for(var n=[],r=0;r2&&(u.children=arguments.length>3?r.call(arguments,2):n),"function"==typeof e&&null!=e.defaultProps)for(a in e.defaultProps)void 0===u[a]&&(u[a]=e.defaultProps[a]);return v(e,u,i,o,null)}function v(e,t,n,r,a){var u={type:e,props:t,key:n,ref:r,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,__h:null,constructor:void 0,__v:null==a?++o:a};return null==a&&null!=i.vnode&&i.vnode(u),u}function m(){return{current:null}}function g(e){return e.children}function y(e,t){this.props=e,this.context=t}function _(e,t){if(null==t)return e.__?_(e.__,e.__.__k.indexOf(e)+1):null;for(var n;t0?v(m.type,m.props,m.key,m.ref?m.ref:null,m.__v):m)){if(m.__=n,m.__b=n.__b+1,null===(p=w[d])||p&&m.key==p.key&&m.type===p.type)w[d]=void 0;else for(h=0;h2&&(u.children=arguments.length>3?r.call(arguments,2):n),v(e.type,u,i||e.key,o||e.ref,null)}function j(e,t){var n={__c:t="__cC"+l++,__:e,Consumer:function(e,t){return e.children(t)},Provider:function(e){var n,r;return this.getChildContext||(n=[],(r={})[t]=this,this.getChildContext=function(){return r},this.shouldComponentUpdate=function(e){this.props.value!==e.value&&n.some(D)},this.sub=function(e){n.push(e);var t=e.componentWillUnmount;e.componentWillUnmount=function(){n.splice(n.indexOf(e),1),t&&t.call(e)}}),e.children}};return n.Provider.__=n.Consumer.contextType=n}r=s.slice,i={__e:function(e,t,n,r){for(var i,o,a;t=t.__;)if((i=t.__c)&&!i.__)try{if((o=i.constructor)&&null!=o.getDerivedStateFromError&&(i.setState(o.getDerivedStateFromError(e)),a=i.__d),null!=i.componentDidCatch&&(i.componentDidCatch(e,r||{}),a=i.__d),a)return i.__E=i}catch(t){e=t}throw e}},o=0,y.prototype.setState=function(e,t){var n;n=null!=this.__s&&this.__s!==this.state?this.__s:this.__s=d({},this.state),"function"==typeof e&&(e=e(d({},n),this.props)),e&&d(n,e),null!=e&&this.__v&&(t&&this._sb.push(t),D(this))},y.prototype.forceUpdate=function(e){this.__v&&(this.__e=!0,e&&this.__h.push(e),D(this))},y.prototype.render=g,a=[],w.__r=0,l=0;var $,H,U,Y,V=0,q=[],W=[],Q=i.__b,G=i.__r,J=i.diffed,Z=i.__c,K=i.unmount;function X(e,t){i.__h&&i.__h(H,e,V||t),V=0;var n=H.__H||(H.__H={__:[],__h:[]});return e>=n.__.length&&n.__.push({__V:W}),n.__[e]}function ee(e){return V=1,te(ye,e)}function te(e,t,n){var r=X($++,2);if(r.t=e,!r.__c&&(r.__=[n?n(t):ye(void 0,t),function(e){var t=r.__N?r.__N[0]:r.__[0],n=r.t(t,e);t!==n&&(r.__N=[n,r.__[1]],r.__c.setState({}))}],r.__c=H,!H.u)){H.u=!0;var i=H.shouldComponentUpdate;H.shouldComponentUpdate=function(e,t,n){if(!r.__c.__H)return!0;var o=r.__c.__H.__.filter((function(e){return e.__c}));if(o.every((function(e){return!e.__N})))return!i||i.call(this,e,t,n);var a=!1;return o.forEach((function(e){if(e.__N){var t=e.__[0];e.__=e.__N,e.__N=void 0,t!==e.__[0]&&(a=!0)}})),!(!a&&r.__c.props===e)&&(!i||i.call(this,e,t,n))}}return r.__N||r.__}function ne(e,t){var n=X($++,3);!i.__s&&ge(n.__H,t)&&(n.__=e,n.i=t,H.__H.__h.push(n))}function re(e,t){var n=X($++,4);!i.__s&&ge(n.__H,t)&&(n.__=e,n.i=t,H.__h.push(n))}function ie(e){return V=5,ae((function(){return{current:e}}),[])}function oe(e,t,n){V=6,re((function(){return"function"==typeof e?(e(t()),function(){return e(null)}):e?(e.current=t(),function(){return e.current=null}):void 0}),null==n?n:n.concat(e))}function ae(e,t){var n=X($++,7);return ge(n.__H,t)?(n.__V=e(),n.i=t,n.__h=e,n.__V):n.__}function ue(e,t){return V=8,ae((function(){return e}),t)}function le(e){var t=H.context[e.__c],n=X($++,9);return n.c=e,t?(null==n.__&&(n.__=!0,t.sub(H)),t.props.value):e.__}function ce(e,t){i.useDebugValue&&i.useDebugValue(t?t(e):e)}function se(e){var t=X($++,10),n=ee();return t.__=e,H.componentDidCatch||(H.componentDidCatch=function(e,r){t.__&&t.__(e,r),n[1](e)}),[n[0],function(){n[1](void 0)}]}function fe(){var e=X($++,11);if(!e.__){for(var t=H.__v;null!==t&&!t.__m&&null!==t.__;)t=t.__;var n=t.__m||(t.__m=[0,0]);e.__="P"+n[0]+"-"+n[1]++}return e.__}function de(){for(var e;e=q.shift();)if(e.__P&&e.__H)try{e.__H.__h.forEach(ve),e.__H.__h.forEach(me),e.__H.__h=[]}catch(l){e.__H.__h=[],i.__e(l,e.__v)}}i.__b=function(e){H=null,Q&&Q(e)},i.__r=function(e){G&&G(e),$=0;var t=(H=e.__c).__H;t&&(U===H?(t.__h=[],H.__h=[],t.__.forEach((function(e){e.__N&&(e.__=e.__N),e.__V=W,e.__N=e.i=void 0}))):(t.__h.forEach(ve),t.__h.forEach(me),t.__h=[])),U=H},i.diffed=function(e){J&&J(e);var t=e.__c;t&&t.__H&&(t.__H.__h.length&&(1!==q.push(t)&&Y===i.requestAnimationFrame||((Y=i.requestAnimationFrame)||pe)(de)),t.__H.__.forEach((function(e){e.i&&(e.__H=e.i),e.__V!==W&&(e.__=e.__V),e.i=void 0,e.__V=W}))),U=H=null},i.__c=function(e,t){t.some((function(e){try{e.__h.forEach(ve),e.__h=e.__h.filter((function(e){return!e.__||me(e)}))}catch(o){t.some((function(e){e.__h&&(e.__h=[])})),t=[],i.__e(o,e.__v)}})),Z&&Z(e,t)},i.unmount=function(e){K&&K(e);var t,n=e.__c;n&&n.__H&&(n.__H.__.forEach((function(e){try{ve(e)}catch(e){t=e}})),n.__H=void 0,t&&i.__e(t,n.__v))};var he="function"==typeof requestAnimationFrame;function pe(e){var t,n=function(){clearTimeout(r),he&&cancelAnimationFrame(t),setTimeout(e)},r=setTimeout(n,100);he&&(t=requestAnimationFrame(n))}function ve(e){var t=H,n=e.__c;"function"==typeof n&&(e.__c=void 0,n()),H=t}function me(e){var t=H;e.__c=e.__(),H=t}function ge(e,t){return!e||e.length!==t.length||t.some((function(t,n){return t!==e[n]}))}function ye(e,t){return"function"==typeof t?t(e):t}function _e(e,t){for(var n in t)e[n]=t[n];return e}function be(e,t){for(var n in e)if("__source"!==n&&!(n in t))return!0;for(var r in t)if("__source"!==r&&e[r]!==t[r])return!0;return!1}function De(e,t){return e===t&&(0!==e||1/e==1/t)||e!=e&&t!=t}function we(e){this.props=e}function xe(e,t){function n(e){var n=this.props.ref,r=n==e.ref;return!r&&n&&(n.call?n(null):n.current=null),t?!t(this.props,e)||!r:be(this.props,e)}function r(t){return this.shouldComponentUpdate=n,p(e,t)}return r.displayName="Memo("+(e.displayName||e.name)+")",r.prototype.isReactComponent=!0,r.__f=!0,r}(we.prototype=new y).isPureReactComponent=!0,we.prototype.shouldComponentUpdate=function(e,t){return be(this.props,e)||be(this.state,t)};var ke=i.__b;i.__b=function(e){e.type&&e.type.__f&&e.ref&&(e.props.ref=e.ref,e.ref=null),ke&&ke(e)};var Ce="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.forward_ref")||3911;function Ae(e){function t(t){var n=_e({},t);return delete n.ref,e(n,t.ref||null)}return t.$$typeof=Ce,t.render=t,t.prototype.isReactComponent=t.__f=!0,t.displayName="ForwardRef("+(e.displayName||e.name)+")",t}var Ee=function(e,t){return null==e?null:C(C(e).map(t))},Se={map:Ee,forEach:Ee,count:function(e){return e?C(e).length:0},only:function(e){var t=C(e);if(1!==t.length)throw"Children.only";return t[0]},toArray:C},Ne=i.__e;i.__e=function(e,t,n,r){if(e.then)for(var i,o=t;o=o.__;)if((i=o.__c)&&i.__c)return null==t.__e&&(t.__e=n.__e,t.__k=n.__k),i.__c(e,t);Ne(e,t,n,r)};var Fe=i.unmount;function Me(e,t,n){return e&&(e.__c&&e.__c.__H&&(e.__c.__H.__.forEach((function(e){"function"==typeof e.__c&&e.__c()})),e.__c.__H=null),null!=(e=_e({},e)).__c&&(e.__c.__P===n&&(e.__c.__P=t),e.__c=null),e.__k=e.__k&&e.__k.map((function(e){return Me(e,t,n)}))),e}function Te(e,t,n){return e&&(e.__v=null,e.__k=e.__k&&e.__k.map((function(e){return Te(e,t,n)})),e.__c&&e.__c.__P===t&&(e.__e&&n.insertBefore(e.__e,e.__d),e.__c.__e=!0,e.__c.__P=n)),e}function Oe(){this.__u=0,this.t=null,this.__b=null}function Be(e){var t=e.__.__c;return t&&t.__a&&t.__a(e)}function Le(e){var t,n,r;function i(i){if(t||(t=e()).then((function(e){n=e.default||e}),(function(e){r=e})),r)throw r;if(!n)throw t;return p(n,i)}return i.displayName="Lazy",i.__f=!0,i}function Ie(){this.u=null,this.o=null}i.unmount=function(e){var t=e.__c;t&&t.__R&&t.__R(),t&&!0===e.__h&&(e.type=null),Fe&&Fe(e)},(Oe.prototype=new y).__c=function(e,t){var n=t.__c,r=this;null==r.t&&(r.t=[]),r.t.push(n);var i=Be(r.__v),o=!1,a=function(){o||(o=!0,n.__R=null,i?i(u):u())};n.__R=a;var u=function(){if(!--r.__u){if(r.state.__a){var e=r.state.__a;r.__v.__k[0]=Te(e,e.__c.__P,e.__c.__O)}var t;for(r.setState({__a:r.__b=null});t=r.t.pop();)t.forceUpdate()}},l=!0===t.__h;r.__u++||l||r.setState({__a:r.__b=r.__v.__k[0]}),e.then(a,a)},Oe.prototype.componentWillUnmount=function(){this.t=[]},Oe.prototype.render=function(e,t){if(this.__b){if(this.__v.__k){var n=document.createElement("div"),r=this.__v.__k[0].__c;this.__v.__k[0]=Me(this.__b,n,r.__O=r.__P)}this.__b=null}var i=t.__a&&p(g,null,e.fallback);return i&&(i.__h=null),[p(g,null,t.__a?null:e.children),i]};var Pe=function(e,t,n){if(++n[1]===n[0]&&e.o.delete(t),e.props.revealOrder&&("t"!==e.props.revealOrder[0]||!e.o.size))for(n=e.u;n;){for(;n.length>3;)n.pop()();if(n[1]>>1,1),t.i.removeChild(e)}}),P(p(Re,{context:t.context},e.__v),t.l)):t.l&&t.componentWillUnmount()}function je(e,t){var n=p(ze,{__v:e,i:t});return n.containerInfo=t,n}(Ie.prototype=new y).__a=function(e){var t=this,n=Be(t.__v),r=t.o.get(e);return r[0]++,function(i){var o=function(){t.props.revealOrder?(r.push(i),Pe(t,e,r)):i()};n?n(o):o()}},Ie.prototype.render=function(e){this.u=null,this.o=new Map;var t=C(e.children);e.revealOrder&&"b"===e.revealOrder[0]&&t.reverse();for(var n=t.length;n--;)this.o.set(t[n],this.u=[1,0,this.u]);return e.children},Ie.prototype.componentDidUpdate=Ie.prototype.componentDidMount=function(){var e=this;this.o.forEach((function(t,n){Pe(e,n,t)}))};var $e="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.element")||60103,He=/^(?:accent|alignment|arabic|baseline|cap|clip(?!PathU)|color|dominant|fill|flood|font|glyph(?!R)|horiz|image|letter|lighting|marker(?!H|W|U)|overline|paint|pointer|shape|stop|strikethrough|stroke|text(?!L)|transform|underline|unicode|units|v|vector|vert|word|writing|x(?!C))[A-Z]/,Ue="undefined"!=typeof document,Ye=function(e){return("undefined"!=typeof Symbol&&"symbol"==typeof Symbol()?/fil|che|rad/i:/fil|che|ra/i).test(e)};function Ve(e,t,n){return null==t.__k&&(t.textContent=""),P(e,t),"function"==typeof n&&n(),e?e.__c:null}function qe(e,t,n){return R(e,t),"function"==typeof n&&n(),e?e.__c:null}y.prototype.isReactComponent={},["componentWillMount","componentWillReceiveProps","componentWillUpdate"].forEach((function(e){Object.defineProperty(y.prototype,e,{configurable:!0,get:function(){return this["UNSAFE_"+e]},set:function(t){Object.defineProperty(this,e,{configurable:!0,writable:!0,value:t})}})}));var We=i.event;function Qe(){}function Ge(){return this.cancelBubble}function Je(){return this.defaultPrevented}i.event=function(e){return We&&(e=We(e)),e.persist=Qe,e.isPropagationStopped=Ge,e.isDefaultPrevented=Je,e.nativeEvent=e};var Ze,Ke={configurable:!0,get:function(){return this.class}},Xe=i.vnode;i.vnode=function(e){var t=e.type,n=e.props,r=n;if("string"==typeof t){var i=-1===t.indexOf("-");for(var o in r={},n){var a=n[o];Ue&&"children"===o&&"noscript"===t||"value"===o&&"defaultValue"in n&&null==a||("defaultValue"===o&&"value"in n&&null==n.value?o="value":"download"===o&&!0===a?a="":/ondoubleclick/i.test(o)?o="ondblclick":/^onchange(textarea|input)/i.test(o+t)&&!Ye(n.type)?o="oninput":/^onfocus$/i.test(o)?o="onfocusin":/^onblur$/i.test(o)?o="onfocusout":/^on(Ani|Tra|Tou|BeforeInp|Compo)/.test(o)?o=o.toLowerCase():i&&He.test(o)?o=o.replace(/[A-Z0-9]/g,"-$&").toLowerCase():null===a&&(a=void 0),/^oninput$/i.test(o)&&(o=o.toLowerCase(),r[o]&&(o="oninputCapture")),r[o]=a)}"select"==t&&r.multiple&&Array.isArray(r.value)&&(r.value=C(n.children).forEach((function(e){e.props.selected=-1!=r.value.indexOf(e.props.value)}))),"select"==t&&null!=r.defaultValue&&(r.value=C(n.children).forEach((function(e){e.props.selected=r.multiple?-1!=r.defaultValue.indexOf(e.props.value):r.defaultValue==e.props.value}))),e.props=r,n.class!=n.className&&(Ke.enumerable="className"in n,null!=n.className&&(r.class=n.className),Object.defineProperty(r,"className",Ke))}e.$$typeof=$e,Xe&&Xe(e)};var et=i.__r;i.__r=function(e){et&&et(e),Ze=e.__c};var tt={ReactCurrentDispatcher:{current:{readContext:function(e){return Ze.__n[e.__c].props.value}}}},nt="17.0.2";function rt(e){return p.bind(null,e)}function it(e){return!!e&&e.$$typeof===$e}function ot(e){return it(e)?z.apply(null,arguments):e}function at(e){return!!e.__k&&(P(null,e),!0)}function ut(e){return e&&(e.base||1===e.nodeType&&e)||null}var lt=function(e,t){return e(t)},ct=function(e,t){return e(t)},st=g;function ft(e){e()}function dt(e){return e}function ht(){return[!1,ft]}var pt=re;function vt(e,t){var n=t(),r=ee({h:{__:n,v:t}}),i=r[0].h,o=r[1];return re((function(){i.__=n,i.v=t,De(i.__,t())||o({h:i})}),[e,n,t]),ne((function(){return De(i.__,i.v())||o({h:i}),e((function(){De(i.__,i.v())||o({h:i})}))}),[e]),n}var mt,gt={useState:ee,useId:fe,useReducer:te,useEffect:ne,useLayoutEffect:re,useInsertionEffect:pt,useTransition:ht,useDeferredValue:dt,useSyncExternalStore:vt,startTransition:ft,useRef:ie,useImperativeHandle:oe,useMemo:ae,useCallback:ue,useContext:le,useDebugValue:ce,version:"17.0.2",Children:Se,render:Ve,hydrate:qe,unmountComponentAtNode:at,createPortal:je,createElement:p,createContext:j,createFactory:rt,cloneElement:ot,createRef:m,Fragment:g,isValidElement:it,findDOMNode:ut,Component:y,PureComponent:we,memo:xe,forwardRef:Ae,flushSync:ct,unstable_batchedUpdates:lt,StrictMode:st,Suspense:Oe,SuspenseList:Ie,lazy:Le,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:tt},yt=n(658),_t=n.n(yt),bt=n(443),Dt=n.n(bt),wt=n(446),xt=n.n(wt),kt=n(635),Ct=n.n(kt);function At(e){if(Array.isArray(e))return e}function Et(e,t){(null==t||t>e.length)&&(t=e.length);for(var n=0,r=new Array(t);n=e.length?{done:!0}:{done:!1,value:e[r++]}},e:function(e){throw e},f:i}}throw new TypeError("Invalid attempt to iterate non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}var o,a=!0,u=!1;return{s:function(){n=n.call(e)},n:function(){var e=n.next();return a=e.done,e},e:function(e){u=!0,o=e},f:function(){try{a||null==n.return||n.return()}finally{if(u)throw o}}}}function Tt(e){if("undefined"!==typeof Symbol&&null!=e[Symbol.iterator]||null!=e["@@iterator"])return Array.from(e)}function Ot(e){return function(e){if(Array.isArray(e))return Et(e)}(e)||Tt(e)||St(e)||function(){throw new TypeError("Invalid attempt to spread non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}()}function Bt(e,t){if(!(e instanceof t))throw new TypeError("Cannot call a class as a function")}function Lt(e){return Lt="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(e){return typeof e}:function(e){return e&&"function"==typeof Symbol&&e.constructor===Symbol&&e!==Symbol.prototype?"symbol":typeof e},Lt(e)}function It(e){var t=function(e,t){if("object"!==Lt(e)||null===e)return e;var n=e[Symbol.toPrimitive];if(void 0!==n){var r=n.call(e,t||"default");if("object"!==Lt(r))return r;throw new TypeError("@@toPrimitive must return a primitive value.")}return("string"===t?String:Number)(e)}(e,"string");return"symbol"===Lt(t)?t:String(t)}function Pt(e,t){for(var n=0;n=0&&(t.hash=e.substr(n),e=e.substr(0,n));var r=e.indexOf("?");r>=0&&(t.search=e.substr(r),e=e.substr(0,r)),e&&(t.pathname=e)}return t}function nn(e,t,n,r){void 0===r&&(r={});var i=r,o=i.window,a=void 0===o?document.defaultView:o,u=i.v5Compat,l=void 0!==u&&u,c=a.history,s=mt.Pop,f=null,d=h();function h(){return(c.state||{idx:null}).idx}function p(){var e=mt.Pop,t=h();if(null!=t){var n=t-d;s=e,d=t,f&&f({action:s,location:m.location,delta:n})}else Zt(!1,"You are trying to block a POP navigation to a location that was not created by @remix-run/router. The block will fail silently in production, but in general you should do all navigation with the router (instead of using window.history.pushState directly) to avoid this situation.")}function v(e){var t="null"!==a.location.origin?a.location.origin:a.location.href,n="string"===typeof e?e:en(e);return Jt(t,"No window.location.(origin|href) available to create URL for href: "+n),new URL(n,t)}null==d&&(d=0,c.replaceState(Wt({},c.state,{idx:d}),""));var m={get action(){return s},get location(){return e(a,c)},listen:function(e){if(f)throw new Error("A history only accepts one active listener");return a.addEventListener(Gt,p),f=e,function(){a.removeEventListener(Gt,p),f=null}},createHref:function(e){return t(a,e)},createURL:v,encodeLocation:function(e){var t=v(e);return{pathname:t.pathname,search:t.search,hash:t.hash}},push:function(e,t){s=mt.Push;var r=Xt(m.location,e,t);n&&n(r,e);var i=Kt(r,d=h()+1),o=m.createHref(r);try{c.pushState(i,"",o)}catch(u){a.location.assign(o)}l&&f&&f({action:s,location:m.location,delta:1})},replace:function(e,t){s=mt.Replace;var r=Xt(m.location,e,t);n&&n(r,e);var i=Kt(r,d=h()),o=m.createHref(r);c.replaceState(i,"",o),l&&f&&f({action:s,location:m.location,delta:0})},go:function(e){return c.go(e)}};return m}function rn(e,t,n){void 0===n&&(n="/");var r=hn(("string"===typeof t?tn(t):t).pathname||"/",n);if(null==r)return null;var i=on(e);!function(e){e.sort((function(e,t){return e.score!==t.score?t.score-e.score:function(e,t){var n=e.length===t.length&&e.slice(0,-1).every((function(e,n){return e===t[n]}));return n?e[e.length-1]-t[t.length-1]:0}(e.routesMeta.map((function(e){return e.childrenIndex})),t.routesMeta.map((function(e){return e.childrenIndex})))}))}(i);for(var o=null,a=0;null==o&&a0&&(Jt(!0!==e.index,'Index routes must not have child routes. Please remove all child routes from route path "'+u+'".'),on(e.children,t,l,u)),(null!=e.path||e.index)&&t.push({path:u,score:cn(u,e.index),routesMeta:l})};return e.forEach((function(e,t){var n;if(""!==e.path&&null!=(n=e.path)&&n.includes("?")){var r,o=Mt(an(e.path));try{for(o.s();!(r=o.n()).done;){var a=r.value;i(e,t,a)}}catch(u){o.e(u)}finally{o.f()}}else i(e,t)})),t}function an(e){var t=e.split("/");if(0===t.length)return[];var n,r=At(n=t)||Tt(n)||St(n)||Nt(),i=r[0],o=r.slice(1),a=i.endsWith("?"),u=i.replace(/\?$/,"");if(0===o.length)return a?[u,""]:[u];var l=an(o.join("/")),c=[];return c.push.apply(c,Ot(l.map((function(e){return""===e?u:[u,e].join("/")})))),a&&c.push.apply(c,Ot(l)),c.map((function(t){return e.startsWith("/")&&""===t?"/":t}))}!function(e){e.data="data",e.deferred="deferred",e.redirect="redirect",e.error="error"}(Qt||(Qt={}));var un=/^:\w+$/,ln=function(e){return"*"===e};function cn(e,t){var n=e.split("/"),r=n.length;return n.some(ln)&&(r+=-2),t&&(r+=2),n.filter((function(e){return!ln(e)})).reduce((function(e,t){return e+(un.test(t)?3:""===t?1:10)}),r)}function sn(e,t){for(var n=e.routesMeta,r={},i="/",o=[],a=0;a and the router will parse it for you.'}function mn(e){return e.filter((function(e,t){return 0===t||e.route.path&&e.route.path.length>0}))}function gn(e,t,n,r){var i;void 0===r&&(r=!1),"string"===typeof e?i=tn(e):(Jt(!(i=Wt({},e)).pathname||!i.pathname.includes("?"),vn("?","pathname","search",i)),Jt(!i.pathname||!i.pathname.includes("#"),vn("#","pathname","hash",i)),Jt(!i.search||!i.search.includes("#"),vn("#","search","hash",i)));var o,a=""===e||""===i.pathname,u=a?"/":i.pathname;if(r||null==u)o=n;else{var l=t.length-1;if(u.startsWith("..")){for(var c=u.split("/");".."===c[0];)c.shift(),l-=1;i.pathname=c.join("/")}o=l>=0?t[l]:"/"}var s=function(e,t){void 0===t&&(t="/");var n="string"===typeof e?tn(e):e,r=n.pathname,i=n.search,o=void 0===i?"":i,a=n.hash,u=void 0===a?"":a,l=r?r.startsWith("/")?r:function(e,t){var n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach((function(e){".."===e?n.length>1&&n.pop():"."!==e&&n.push(e)})),n.length>1?n.join("/"):"/"}(r,t):t;return{pathname:l,search:bn(o),hash:Dn(u)}}(i,o),f=u&&"/"!==u&&u.endsWith("/"),d=(a||"."===u)&&n.endsWith("/");return s.pathname.endsWith("/")||!f&&!d||(s.pathname+="/"),s}var yn=function(e){return e.join("/").replace(/\/\/+/g,"/")},_n=function(e){return e.replace(/\/+$/,"").replace(/^\/*/,"/")},bn=function(e){return e&&"?"!==e?e.startsWith("?")?e:"?"+e:""},Dn=function(e){return e&&"#"!==e?e.startsWith("#")?e:"#"+e:""};Error;var wn=Rt((function e(t,n,r,i){Bt(this,e),void 0===i&&(i=!1),this.status=t,this.statusText=n||"",this.internal=i,r instanceof Error?(this.data=r.toString(),this.error=r):this.data=r}));function xn(e){return e instanceof wn}var kn=["post","put","patch","delete"],Cn=(new Set(kn),["get"].concat(kn));new Set(Cn),new Set([301,302,303,307,308]),new Set([307,308]),"undefined"!==typeof window&&"undefined"!==typeof window.document&&window.document.createElement;Symbol("deferred");function An(){return An=Object.assign?Object.assign.bind():function(e){for(var t=1;t")))}var Vn,qn,Wn=function(e){jt(n,e);var t=Yt(n);function n(e){var r;return Bt(this,n),(r=t.call(this,e)).state={location:e.location,error:e.error},r}return Rt(n,[{key:"componentDidCatch",value:function(e,t){console.error("React Router caught the following error during render",e,t)}},{key:"render",value:function(){return this.state.error?p(Pn.Provider,{value:this.props.routeContext},p(Rn.Provider,{value:this.state.error,children:this.props.component})):this.props.children}}],[{key:"getDerivedStateFromError",value:function(e){return{error:e}}},{key:"getDerivedStateFromProps",value:function(e,t){return t.location!==e.location?{error:e.error,location:e.location}:{error:e.error||t.error,location:t.location}}}]),n}(y);function Qn(e){var t=e.routeContext,n=e.match,r=e.children,i=le(On);return i&&i.static&&i.staticContext&&n.route.errorElement&&(i.staticContext._deepestRenderedBoundaryId=n.route.id),p(Pn.Provider,{value:t},r)}function Gn(e,t,n){if(void 0===t&&(t=[]),null==e){if(null==n||!n.errors)return null;e=n.matches}var r=e,i=null==n?void 0:n.errors;if(null!=i){var o=r.findIndex((function(e){return e.route.id&&(null==i?void 0:i[e.route.id])}));o>=0||Jt(!1),r=r.slice(0,Math.min(r.length,o+1))}return r.reduceRight((function(e,o,a){var u=o.route.id?null==i?void 0:i[o.route.id]:null,l=n?o.route.errorElement||p(Yn,null):null,c=t.concat(r.slice(0,a+1)),s=function(){return p(Qn,{match:o,routeContext:{outlet:e,matches:c}},u?l:void 0!==o.route.element?o.route.element:e)};return n&&(o.route.errorElement||0===a)?p(Wn,{location:n.location,component:l,error:u,children:s(),routeContext:{outlet:null,matches:c}}):s()}),null)}function Jn(e){var t=le(Bn);return t||Jt(!1),t}function Zn(e){var t=function(e){var t=le(Pn);return t||Jt(!1),t}(),n=t.matches[t.matches.length-1];return n.route.id||Jt(!1),n.route.id}!function(e){e.UseBlocker="useBlocker",e.UseRevalidator="useRevalidator"}(Vn||(Vn={})),function(e){e.UseLoaderData="useLoaderData",e.UseActionData="useActionData",e.UseRouteError="useRouteError",e.UseNavigation="useNavigation",e.UseRouteLoaderData="useRouteLoaderData",e.UseMatches="useMatches",e.UseRevalidator="useRevalidator"}(qn||(qn={}));var Kn;function Xn(e){return function(e){var t=le(Pn).outlet;return t?p(Hn.Provider,{value:e},t):t}(e.context)}function er(e){Jt(!1)}function tr(e){var t=e.basename,n=void 0===t?"/":t,r=e.children,i=void 0===r?null:r,o=e.location,a=e.navigationType,u=void 0===a?mt.Pop:a,l=e.navigator,c=e.static,s=void 0!==c&&c;zn()&&Jt(!1);var f=n.replace(/^\/*/,"/"),d=ae((function(){return{basename:f,navigator:l,static:s}}),[f,l,s]);"string"===typeof o&&(o=tn(o));var h=o,v=h.pathname,m=void 0===v?"/":v,g=h.search,y=void 0===g?"":g,_=h.hash,b=void 0===_?"":_,D=h.state,w=void 0===D?null:D,x=h.key,k=void 0===x?"default":x,C=ae((function(){var e=hn(m,f);return null==e?null:{pathname:e,search:y,hash:b,state:w,key:k}}),[f,m,y,b,w,k]);return null==C?null:p(Ln.Provider,{value:d},p(In.Provider,{children:i,value:{location:C,navigationType:u}}))}function nr(e){var t=e.children,n=e.location,r=le(On);return function(e,t){zn()||Jt(!1);var n,r=le(Ln).navigator,i=le(Bn),o=le(Pn).matches,a=o[o.length-1],u=a?a.params:{},l=(a&&a.pathname,a?a.pathnameBase:"/"),c=(a&&a.route,jn());if(t){var s,f="string"===typeof t?tn(t):t;"/"===l||(null==(s=f.pathname)?void 0:s.startsWith(l))||Jt(!1),n=f}else n=c;var d=n.pathname||"/",h=rn(e,{pathname:"/"===l?d:d.slice(l.length)||"/"}),v=Gn(h&&h.map((function(e){return Object.assign({},e,{params:Object.assign({},u,e.params),pathname:yn([l,r.encodeLocation?r.encodeLocation(e.pathname).pathname:e.pathname]),pathnameBase:"/"===e.pathnameBase?l:yn([l,r.encodeLocation?r.encodeLocation(e.pathnameBase).pathname:e.pathnameBase])})})),o,i||void 0);return t&&v?p(In.Provider,{value:{location:An({pathname:"/",search:"",hash:"",state:null,key:"default"},n),navigationType:mt.Pop}},v):v}(r&&!t?r.router.routes:rr(t),n)}!function(e){e[e.pending=0]="pending",e[e.success=1]="success",e[e.error=2]="error"}(Kn||(Kn={}));new Promise((function(){}));function rr(e,t){void 0===t&&(t=[]);var n=[];return Se.forEach(e,(function(e,r){if(it(e))if(e.type!==g){e.type!==er&&Jt(!1),e.props.index&&e.props.children&&Jt(!1);var i=[].concat(Ot(t),[r]),o={id:e.props.id||i.join("-"),caseSensitive:e.props.caseSensitive,element:e.props.element,index:e.props.index,path:e.props.path,loader:e.props.loader,action:e.props.action,errorElement:e.props.errorElement,hasErrorBoundary:null!=e.props.errorElement,shouldRevalidate:e.props.shouldRevalidate,handle:e.props.handle};e.props.children&&(o.children=rr(e.props.children,i)),n.push(o)}else n.push.apply(n,rr(e.props.children,t))})),n}function ir(){return ir=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0||(i[n]=e[n]);return i}function ar(e){return void 0===e&&(e=""),new URLSearchParams("string"===typeof e||Array.isArray(e)||e instanceof URLSearchParams?e:Object.keys(e).reduce((function(t,n){var r=e[n];return t.concat(Array.isArray(r)?r.map((function(e){return[n,e]})):[[n,r]])}),[]))}var ur=["onClick","relative","reloadDocument","replace","state","target","to","preventScrollReset"],lr=["aria-current","caseSensitive","className","end","style","to","children"];function cr(e){var t=e.basename,n=e.children,r=e.window,i=ie();null==i.current&&(i.current=function(e){return void 0===e&&(e={}),nn((function(e,t){var n=tn(e.location.hash.substr(1)),r=n.pathname,i=void 0===r?"/":r,o=n.search,a=void 0===o?"":o,u=n.hash;return Xt("",{pathname:i,search:a,hash:void 0===u?"":u},t.state&&t.state.usr||null,t.state&&t.state.key||"default")}),(function(e,t){var n=e.document.querySelector("base"),r="";if(n&&n.getAttribute("href")){var i=e.location.href,o=i.indexOf("#");r=-1===o?i:i.slice(0,o)}return r+"#"+("string"===typeof t?t:en(t))}),(function(e,t){Zt("/"===e.pathname.charAt(0),"relative pathnames are not supported in hash history.push("+JSON.stringify(t)+")")}),e)}({window:r,v5Compat:!0}));var o=i.current,a=Ft(ee({action:o.action,location:o.location}),2),u=a[0],l=a[1];return re((function(){return o.listen(l)}),[o]),p(tr,{basename:t,children:n,location:u.location,navigationType:u.action,navigator:o})}var sr=Ae((function(e,t){var n=e.onClick,r=e.relative,i=e.reloadDocument,o=e.replace,a=e.state,u=e.target,l=e.to,c=e.preventScrollReset,s=or(e,ur),f=function(e,t){var n=(void 0===t?{}:t).relative;zn()||Jt(!1);var r=le(Ln),i=r.basename,o=r.navigator,a=Un(e,{relative:n}),u=a.hash,l=a.pathname,c=a.search,s=l;return"/"!==i&&(s="/"===l?i:yn([i,l])),o.createHref({pathname:s,search:c,hash:u})}(l,{relative:r}),d=function(e,t){var n=void 0===t?{}:t,r=n.target,i=n.replace,o=n.state,a=n.preventScrollReset,u=n.relative,l=$n(),c=jn(),s=Un(e,{relative:u});return ue((function(t){if(function(e,t){return 0===e.button&&(!t||"_self"===t)&&!function(e){return!!(e.metaKey||e.altKey||e.ctrlKey||e.shiftKey)}(e)}(t,r)){t.preventDefault();var n=void 0!==i?i:en(c)===en(s);l(e,{replace:n,state:o,preventScrollReset:a,relative:u})}}),[c,l,s,i,o,r,e,a,u])}(l,{replace:o,state:a,target:u,preventScrollReset:c,relative:r});return p("a",ir({},s,{href:f,onClick:i?n:function(e){n&&n(e),e.defaultPrevented||d(e)},ref:t,target:u}))}));var fr=Ae((function(e,t){var n=e["aria-current"],r=void 0===n?"page":n,i=e.caseSensitive,o=void 0!==i&&i,a=e.className,u=void 0===a?"":a,l=e.end,c=void 0!==l&&l,s=e.style,f=e.to,d=e.children,h=or(e,lr),v=Un(f,{relative:h.relative}),m=jn(),g=le(Bn),y=le(Ln).navigator,_=y.encodeLocation?y.encodeLocation(v).pathname:v.pathname,b=m.pathname,D=g&&g.navigation&&g.navigation.location?g.navigation.location.pathname:null;o||(b=b.toLowerCase(),D=D?D.toLowerCase():null,_=_.toLowerCase());var w,x=b===_||!c&&b.startsWith(_)&&"/"===b.charAt(_.length),k=null!=D&&(D===_||!c&&D.startsWith(_)&&"/"===D.charAt(_.length)),C=x?r:void 0;w="function"===typeof u?u({isActive:x,isPending:k}):[u,x?"active":null,k?"pending":null].filter(Boolean).join(" ");var A="function"===typeof s?s({isActive:x,isPending:k}):s;return p(sr,ir({},h,{"aria-current":C,className:w,ref:t,style:A,to:f}),"function"===typeof d?d({isActive:x,isPending:k}):d)}));var dr,hr;function pr(e){var t=ie(ar(e)),n=jn(),r=ae((function(){return function(e,t){var n,r=ar(e),i=Mt(t.keys());try{var o=function(){var e=n.value;r.has(e)||t.getAll(e).forEach((function(t){r.append(e,t)}))};for(i.s();!(n=i.n()).done;)o()}catch(a){i.e(a)}finally{i.f()}return r}(n.search,t.current)}),[n.search]),i=$n(),o=ue((function(e,t){var n=ar("function"===typeof e?e(r):e);i("?"+n,t)}),[i,r]);return[r,o]}(function(e){e.UseScrollRestoration="useScrollRestoration",e.UseSubmitImpl="useSubmitImpl",e.UseFetcher="useFetcher"})(dr||(dr={})),function(e){e.UseFetchers="useFetchers",e.UseScrollRestoration="useScrollRestoration"}(hr||(hr={}));var vr;function mr(e,t,n){return(t=It(t))in e?Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}):e[t]=n,e}function gr(e,t){var n=Object.keys(e);if(Object.getOwnPropertySymbols){var r=Object.getOwnPropertySymbols(e);t&&(r=r.filter((function(t){return Object.getOwnPropertyDescriptor(e,t).enumerable}))),n.push.apply(n,r)}return n}function yr(e){for(var t=1;t=100&&(t=n-n%10),e<100&&e>=10&&(t=n-n%5),e<10&&e>=1&&(t=n),e<1&&e>.01&&(t=Math.round(40*e)/40),di(_t().duration(t||.001,"seconds").asMilliseconds()).replace(/\s/g,"")}(r/Xr),date:si(t||_t()().toDate())}},si=function(e){return _t().tz(e).utc().format(Kr)},fi=function(e){return _t().tz(e).format(Kr)},di=function(e){var t=Math.floor(e%1e3),n=Math.floor(e/1e3%60),r=Math.floor(e/1e3/60%60),i=Math.floor(e/1e3/3600%24),o=Math.floor(e/864e5),a=["d","h","m","s","ms"],u=[o,i,r,n,t].map((function(e,t){return e?"".concat(e).concat(a[t]):""}));return u.filter((function(e){return e})).join(" ")},hi=function(e){var t=_t()(1e3*e);return t.isValid()?t.toDate():new Date},pi=[{title:"Last 5 minutes",duration:"5m"},{title:"Last 15 minutes",duration:"15m"},{title:"Last 30 minutes",duration:"30m",isDefault:!0},{title:"Last 1 hour",duration:"1h"},{title:"Last 3 hours",duration:"3h"},{title:"Last 6 hours",duration:"6h"},{title:"Last 12 hours",duration:"12h"},{title:"Last 24 hours",duration:"24h"},{title:"Last 2 days",duration:"2d"},{title:"Last 7 days",duration:"7d"},{title:"Last 30 days",duration:"30d"},{title:"Last 90 days",duration:"90d"},{title:"Last 180 days",duration:"180d"},{title:"Last 1 year",duration:"1y"},{title:"Yesterday",duration:"1d",until:function(){return _t()().tz().subtract(1,"day").endOf("day").toDate()}},{title:"Today",duration:"1d",until:function(){return _t()().tz().endOf("day").toDate()}}].map((function(e){return yr({id:e.title.replace(/\s/g,"_").toLocaleLowerCase(),until:e.until?e.until:function(){return _t()().tz().toDate()}},e)})),vi=function(e){var t,n=e.relativeTimeId,r=e.defaultDuration,i=e.defaultEndInput,o=null===(t=pi.find((function(e){return e.isDefault})))||void 0===t?void 0:t.id,a=n||Br("g0.relative_time",o),u=pi.find((function(e){return e.id===a}));return{relativeTimeId:u?a:"none",duration:u?u.duration:r,endInput:u?u.until():i}},mi=function(e){var t=_t()().tz(e);return"UTC".concat(t.format("Z"))},gi=function(){var e=arguments.length>0&&void 0!==arguments[0]?arguments[0]:"",t=new RegExp(e,"i");return ri.reduce((function(n,r){var i=(r.match(/^(.*?)\//)||[])[1]||"unknown",o=mi(r),a=o.replace(/UTC|0/,""),u=r.replace(/[/_]/g," "),l={region:r,utc:o,search:"".concat(r," ").concat(o," ").concat(u," ").concat(a)},c=!e||e&&t.test(l.search);return c&&n[i]?n[i].push(l):c&&(n[i]=[l]),n}),{})},yi=function(e){_t().tz.setDefault(e)},_i=Ir("TIMEZONE")||_t().tz.guess();yi(_i);var bi,Di=Br("g0.range_input"),wi=vi({defaultDuration:Di||"1h",defaultEndInput:(bi=Br("g0.end_input",_t()().utc().format(Kr)),_t()(bi).utcOffset(0,!0).toDate()),relativeTimeId:Di?Br("g0.relative_time","none"):void 0}),xi=wi.duration,ki=wi.endInput,Ci=wi.relativeTimeId,Ai={duration:xi,period:ci(xi,ki),relativeTime:Ci,timezone:_i};function Ei(e,t){switch(t.type){case"SET_DURATION":return yr(yr({},e),{},{duration:t.payload,period:ci(t.payload,hi(e.period.end)),relativeTime:"none"});case"SET_RELATIVE_TIME":return yr(yr({},e),{},{duration:t.payload.duration,period:ci(t.payload.duration,t.payload.until),relativeTime:t.payload.id});case"SET_PERIOD":var n=function(e){var t=e.to.valueOf()-e.from.valueOf();return di(t)}(t.payload);return yr(yr({},e),{},{duration:n,period:ci(n,t.payload.to),relativeTime:"none"});case"RUN_QUERY":var r=vi({relativeTimeId:e.relativeTime,defaultDuration:e.duration,defaultEndInput:hi(e.period.end)}),i=r.duration,o=r.endInput;return yr(yr({},e),{},{period:ci(i,o)});case"RUN_QUERY_TO_NOW":return yr(yr({},e),{},{period:ci(e.duration)});case"SET_TIMEZONE":return yi(t.payload),Lr("TIMEZONE",t.payload),yr(yr({},e),{},{timezone:t.payload});default:throw new Error}}var Si=j({}),Ni=function(){return le(Si).state},Fi=function(){return le(Si).dispatch},Mi=function(){var e,t=(null===(e=(window.location.hash.split("?")[1]||"").match(/g\d+\.expr/g))||void 0===e?void 0:e.length)||1;return new Array(t>4?4:t).fill(1).map((function(e,t){return Br("g".concat(t,".expr"),"")}))}(),Ti={query:Mi,queryHistory:Mi.map((function(e){return{index:0,values:[e]}})),autocomplete:Ir("AUTOCOMPLETE")||!1};function Oi(e,t){switch(t.type){case"SET_QUERY":return yr(yr({},e),{},{query:t.payload.map((function(e){return e}))});case"SET_QUERY_HISTORY":return yr(yr({},e),{},{queryHistory:t.payload});case"SET_QUERY_HISTORY_BY_INDEX":return e.queryHistory.splice(t.payload.queryNumber,1,t.payload.value),yr(yr({},e),{},{queryHistory:e.queryHistory});case"TOGGLE_AUTOCOMPLETE":return Lr("AUTOCOMPLETE",!e.autocomplete),yr(yr({},e),{},{autocomplete:!e.autocomplete});default:throw new Error}}var Bi=j({}),Li=function(){return le(Bi).state},Ii=function(){return le(Bi).dispatch},Pi=function(){return Vr("svg",{viewBox:"0 0 74 24",fill:"currentColor",children:[Vr("path",{d:"M6.11767 10.4759C6.47736 10.7556 6.91931 10.909 7.37503 10.9121H7.42681C7.90756 10.9047 8.38832 10.7199 8.67677 10.4685C10.1856 9.18921 14.5568 5.18138 14.5568 5.18138C15.7254 4.09438 12.4637 3.00739 7.42681 3H7.36764C2.3308 3.00739 -0.930935 4.09438 0.237669 5.18138C0.237669 5.18138 4.60884 9.18921 6.11767 10.4759ZM8.67677 12.6424C8.31803 12.9248 7.87599 13.0808 7.41941 13.0861H7.37503C6.91845 13.0808 6.47641 12.9248 6.11767 12.6424C5.0822 11.7551 1.38409 8.42018 0.000989555 7.14832V9.07829C0.000989555 9.29273 0.0823481 9.57372 0.222877 9.70682L0.293316 9.7712L0.293344 9.77122C1.33784 10.7258 4.83903 13.9255 6.11767 15.0161C6.47641 15.2985 6.91845 15.4545 7.37503 15.4597H7.41941C7.90756 15.4449 8.38092 15.2601 8.67677 15.0161C9.9859 13.9069 13.6249 10.572 14.5642 9.70682C14.7121 9.57372 14.7861 9.29273 14.7861 9.07829V7.14832C12.7662 8.99804 10.7297 10.8295 8.67677 12.6424ZM7.41941 17.6263C7.87513 17.6232 8.31708 17.4698 8.67677 17.19C10.7298 15.3746 12.7663 13.5407 14.7861 11.6885V13.6259C14.7861 13.8329 14.7121 14.1139 14.5642 14.247C13.6249 15.1196 9.9859 18.4471 8.67677 19.5563C8.38092 19.8077 7.90756 19.9926 7.41941 20H7.37503C6.91931 19.9968 6.47736 19.8435 6.11767 19.5637C4.91427 18.5373 1.74219 15.6364 0.502294 14.5025C0.393358 14.4029 0.299337 14.3169 0.222877 14.247C0.0823481 14.1139 0.000989555 13.8329 0.000989555 13.6259V11.6885C1.38409 12.953 5.0822 16.2953 6.11767 17.1827C6.47641 17.4651 6.91845 17.6211 7.37503 17.6263H7.41941Z"}),Vr("path",{d:"M34.9996 5L29.1596 19.46H26.7296L20.8896 5H23.0496C23.2829 5 23.4729 5.05667 23.6196 5.17C23.7663 5.28333 23.8763 5.43 23.9496 5.61L27.3596 14.43C27.4729 14.7167 27.5796 15.0333 27.6796 15.38C27.7863 15.72 27.8863 16.0767 27.9796 16.45C28.0596 16.0767 28.1463 15.72 28.2396 15.38C28.3329 15.0333 28.4363 14.7167 28.5496 14.43L31.9396 5.61C31.9929 5.45667 32.0963 5.31667 32.2496 5.19C32.4096 5.06333 32.603 5 32.8297 5H34.9996ZM52.1763 5V19.46H49.8064V10.12C49.8064 9.74667 49.8263 9.34333 49.8663 8.91L45.4963 17.12C45.2897 17.5133 44.973 17.71 44.5463 17.71H44.1663C43.7397 17.71 43.4231 17.5133 43.2164 17.12L38.7963 8.88C38.8163 9.1 38.833 9.31667 38.8463 9.53C38.8597 9.74333 38.8663 9.94 38.8663 10.12V19.46H36.4963V5H38.5263C38.6463 5 38.7497 5.00333 38.8363 5.01C38.923 5.01667 38.9997 5.03333 39.0663 5.06C39.1397 5.08667 39.203 5.13 39.2563 5.19C39.3163 5.25 39.373 5.33 39.4263 5.43L43.7563 13.46C43.8697 13.6733 43.973 13.8933 44.0663 14.12C44.1663 14.3467 44.263 14.58 44.3563 14.82C44.4497 14.5733 44.5464 14.3367 44.6464 14.11C44.7464 13.8767 44.8531 13.6533 44.9664 13.44L49.2363 5.43C49.2897 5.33 49.3463 5.25 49.4063 5.19C49.4663 5.13 49.5297 5.08667 49.5963 5.06C49.6697 5.03333 49.7497 5.01667 49.8363 5.01C49.923 5.00333 50.0264 5 50.1464 5H52.1763ZM61.0626 18.73C61.7426 18.73 62.3492 18.6133 62.8826 18.38C63.4226 18.14 63.8792 17.81 64.2526 17.39C64.6259 16.97 64.9092 16.4767 65.1026 15.91C65.3026 15.3367 65.4026 14.72 65.4026 14.06V5.31H66.4226V14.06C66.4226 14.84 66.2993 15.57 66.0527 16.25C65.806 16.9233 65.4493 17.5133 64.9827 18.02C64.5227 18.52 63.9592 18.9133 63.2926 19.2C62.6326 19.4867 61.8892 19.63 61.0626 19.63C60.2359 19.63 59.4893 19.4867 58.8227 19.2C58.1627 18.9133 57.5992 18.52 57.1326 18.02C56.6726 17.5133 56.3193 16.9233 56.0727 16.25C55.826 15.57 55.7026 14.84 55.7026 14.06V5.31H56.7327V14.05C56.7327 14.71 56.8292 15.3267 57.0226 15.9C57.2226 16.4667 57.506 16.96 57.8727 17.38C58.246 17.8 58.6993 18.13 59.2327 18.37C59.7727 18.61 60.3826 18.73 61.0626 18.73ZM71.4438 19.46H70.4138V5.31H71.4438V19.46Z"})]})},Ri=function(){return Vr("svg",{viewBox:"0 0 15 17",fill:"currentColor",children:Vr("path",{d:"M6.11767 7.47586C6.47736 7.75563 6.91931 7.90898 7.37503 7.91213H7.42681C7.90756 7.90474 8.38832 7.71987 8.67677 7.46846C10.1856 6.18921 14.5568 2.18138 14.5568 2.18138C15.7254 1.09438 12.4637 0.00739 7.42681 0H7.36764C2.3308 0.00739 -0.930935 1.09438 0.237669 2.18138C0.237669 2.18138 4.60884 6.18921 6.11767 7.47586ZM8.67677 9.64243C8.31803 9.92483 7.87599 10.0808 7.41941 10.0861H7.37503C6.91845 10.0808 6.47641 9.92483 6.11767 9.64243C5.0822 8.75513 1.38409 5.42018 0.000989555 4.14832V6.07829C0.000989555 6.29273 0.0823481 6.57372 0.222877 6.70682L0.293316 6.7712L0.293344 6.77122C1.33784 7.72579 4.83903 10.9255 6.11767 12.0161C6.47641 12.2985 6.91845 12.4545 7.37503 12.4597H7.41941C7.90756 12.4449 8.38092 12.2601 8.67677 12.0161C9.9859 10.9069 13.6249 7.57198 14.5642 6.70682C14.7121 6.57372 14.7861 6.29273 14.7861 6.07829V4.14832C12.7662 5.99804 10.7297 7.82949 8.67677 9.64243ZM7.41941 14.6263C7.87513 14.6232 8.31708 14.4698 8.67677 14.19C10.7298 12.3746 12.7663 10.5407 14.7861 8.68853V10.6259C14.7861 10.8329 14.7121 11.1139 14.5642 11.247C13.6249 12.1196 9.9859 15.4471 8.67677 16.5563C8.38092 16.8077 7.90756 16.9926 7.41941 17H7.37503C6.91931 16.9968 6.47736 16.8435 6.11767 16.5637C4.91427 15.5373 1.74219 12.6364 0.502294 11.5025C0.393358 11.4029 0.299337 11.3169 0.222877 11.247C0.0823481 11.1139 0.000989555 10.8329 0.000989555 10.6259V8.68853C1.38409 9.95303 5.0822 13.2953 6.11767 14.1827C6.47641 14.4651 6.91845 14.6211 7.37503 14.6263H7.41941Z"})})},zi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19.14 12.94c.04-.3.06-.61.06-.94 0-.32-.02-.64-.07-.94l2.03-1.58c.18-.14.23-.41.12-.61l-1.92-3.32c-.12-.22-.37-.29-.59-.22l-2.39.96c-.5-.38-1.03-.7-1.62-.94l-.36-2.54c-.04-.24-.24-.41-.48-.41h-3.84c-.24 0-.43.17-.47.41l-.36 2.54c-.59.24-1.13.57-1.62.94l-2.39-.96c-.22-.08-.47 0-.59.22L2.74 8.87c-.12.21-.08.47.12.61l2.03 1.58c-.05.3-.09.63-.09.94s.02.64.07.94l-2.03 1.58c-.18.14-.23.41-.12.61l1.92 3.32c.12.22.37.29.59.22l2.39-.96c.5.38 1.03.7 1.62.94l.36 2.54c.05.24.24.41.48.41h3.84c.24 0 .44-.17.47-.41l.36-2.54c.59-.24 1.13-.56 1.62-.94l2.39.96c.22.08.47 0 .59-.22l1.92-3.32c.12-.22.07-.47-.12-.61l-2.01-1.58zM12 15.6c-1.98 0-3.6-1.62-3.6-3.6s1.62-3.6 3.6-3.6 3.6 1.62 3.6 3.6-1.62 3.6-3.6 3.6z"})})},ji=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 6.41 17.59 5 12 10.59 6.41 5 5 6.41 10.59 12 5 17.59 6.41 19 12 13.41 17.59 19 19 17.59 13.41 12z"})})},$i=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 5V2L8 6l4 4V7c3.31 0 6 2.69 6 6 0 2.97-2.17 5.43-5 5.91v2.02c3.95-.49 7-3.85 7-7.93 0-4.42-3.58-8-8-8zm-6 8c0-1.65.67-3.15 1.76-4.24L6.34 7.34C4.9 8.79 4 10.79 4 13c0 4.08 3.05 7.44 7 7.93v-2.02c-2.83-.48-5-2.94-5-5.91z"})})},Hi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm1 15h-2v-6h2v6zm0-8h-2V7h2v2z"})})},Ui=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M1 21h22L12 2 1 21zm12-3h-2v-2h2v2zm0-4h-2v-4h2v4z"})})},Yi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm1 15h-2v-2h2v2zm0-4h-2V7h2v6z"})})},Vi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm-2 15-5-5 1.41-1.41L10 14.17l7.59-7.59L19 8l-9 9z"})})},qi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 6v3l4-4-4-4v3c-4.42 0-8 3.58-8 8 0 1.57.46 3.03 1.24 4.26L6.7 14.8c-.45-.83-.7-1.79-.7-2.8 0-3.31 2.69-6 6-6zm6.76 1.74L17.3 9.2c.44.84.7 1.79.7 2.8 0 3.31-2.69 6-6 6v-3l-4 4 4 4v-3c4.42 0 8-3.58 8-8 0-1.57-.46-3.03-1.24-4.26z"})})},Wi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M7.41 8.59 12 13.17l4.59-4.58L18 10l-6 6-6-6 1.41-1.41z"})})},Qi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m7 10 5 5 5-5z"})})},Gi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Vr("path",{d:"M11.99 2C6.47 2 2 6.48 2 12s4.47 10 9.99 10C17.52 22 22 17.52 22 12S17.52 2 11.99 2zM12 20c-4.42 0-8-3.58-8-8s3.58-8 8-8 8 3.58 8 8-3.58 8-8 8z"}),Vr("path",{d:"M12.5 7H11v6l5.25 3.15.75-1.23-4.5-2.67z"})]})},Ji=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 3h-1V1h-2v2H7V1H5v2H4c-1.1 0-2 .9-2 2v16c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2V5c0-1.1-.9-2-2-2zm0 18H4V8h16v13z"})})},Zi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m22 5.72-4.6-3.86-1.29 1.53 4.6 3.86L22 5.72zM7.88 3.39 6.6 1.86 2 5.71l1.29 1.53 4.59-3.85zM12.5 8H11v6l4.75 2.85.75-1.23-4-2.37V8zM12 4c-4.97 0-9 4.03-9 9s4.02 9 9 9c4.97 0 9-4.03 9-9s-4.03-9-9-9zm0 16c-3.87 0-7-3.13-7-7s3.13-7 7-7 7 3.13 7 7-3.13 7-7 7z"})})},Ki=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 5H4c-1.1 0-1.99.9-1.99 2L2 17c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2V7c0-1.1-.9-2-2-2zm-9 3h2v2h-2V8zm0 3h2v2h-2v-2zM8 8h2v2H8V8zm0 3h2v2H8v-2zm-1 2H5v-2h2v2zm0-3H5V8h2v2zm9 7H8v-2h8v2zm0-4h-2v-2h2v2zm0-3h-2V8h2v2zm3 3h-2v-2h2v2zm0-3h-2V8h2v2z"})})},Xi=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M8 5v14l11-7z"})})},eo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m10 16.5 6-4.5-6-4.5v9zM12 2C6.48 2 2 6.48 2 12s4.48 10 10 10 10-4.48 10-10S17.52 2 12 2zm0 18c-4.41 0-8-3.59-8-8s3.59-8 8-8 8 3.59 8 8-3.59 8-8 8z"})})},to=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"m3.5 18.49 6-6.01 4 4L22 6.92l-1.41-1.41-7.09 7.97-4-4L2 16.99z"})})},no=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M10 10.02h5V21h-5zM17 21h3c1.1 0 2-.9 2-2v-9h-5v11zm3-18H5c-1.1 0-2 .9-2 2v3h19V5c0-1.1-.9-2-2-2zM3 19c0 1.1.9 2 2 2h3V10H3v9z"})})},ro=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M9.4 16.6 4.8 12l4.6-4.6L8 6l-6 6 6 6 1.4-1.4zm5.2 0 4.6-4.6-4.6-4.6L16 6l6 6-6 6-1.4-1.4z"})})},io=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M6 19c0 1.1.9 2 2 2h8c1.1 0 2-.9 2-2V7H6v12zM19 4h-3.5l-1-1h-5l-1 1H5v2h14V4z"})})},oo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 13h-6v6h-2v-6H5v-2h6V5h2v6h6v2z"})})},ao=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M19 13H5v-2h14v2z"})})},uo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M8.9999 14.7854L18.8928 4.8925C19.0803 4.70497 19.3347 4.59961 19.5999 4.59961C19.8651 4.59961 20.1195 4.70497 20.307 4.8925L21.707 6.2925C22.0975 6.68303 22.0975 7.31619 21.707 7.70672L9.70701 19.7067C9.31648 20.0972 8.68332 20.0972 8.2928 19.7067L2.6928 14.1067C2.50526 13.9192 2.3999 13.6648 2.3999 13.3996C2.3999 13.1344 2.50526 12.88 2.6928 12.6925L4.0928 11.2925C4.48332 10.902 5.11648 10.902 5.50701 11.2925L8.9999 14.7854Z"})})},lo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 4.5C7 4.5 2.73 7.61 1 12c1.73 4.39 6 7.5 11 7.5s9.27-3.11 11-7.5c-1.73-4.39-6-7.5-11-7.5zM12 17c-2.76 0-5-2.24-5-5s2.24-5 5-5 5 2.24 5 5-2.24 5-5 5zm0-8c-1.66 0-3 1.34-3 3s1.34 3 3 3 3-1.34 3-3-1.34-3-3-3z"})})},co=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 7c2.76 0 5 2.24 5 5 0 .65-.13 1.26-.36 1.83l2.92 2.92c1.51-1.26 2.7-2.89 3.43-4.75-1.73-4.39-6-7.5-11-7.5-1.4 0-2.74.25-3.98.7l2.16 2.16C10.74 7.13 11.35 7 12 7zM2 4.27l2.28 2.28.46.46C3.08 8.3 1.78 10.02 1 12c1.73 4.39 6 7.5 11 7.5 1.55 0 3.03-.3 4.38-.84l.42.42L19.73 22 21 20.73 3.27 3 2 4.27zM7.53 9.8l1.55 1.55c-.05.21-.08.43-.08.65 0 1.66 1.34 3 3 3 .22 0 .44-.03.65-.08l1.55 1.55c-.67.33-1.41.53-2.2.53-2.76 0-5-2.24-5-5 0-.79.2-1.53.53-2.2zm4.31-.78 3.15 3.15.02-.16c0-1.66-1.34-3-3-3l-.17.01z"})})},so=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M16 1H4c-1.1 0-2 .9-2 2v14h2V3h12V1zm3 4H8c-1.1 0-2 .9-2 2v14c0 1.1.9 2 2 2h11c1.1 0 2-.9 2-2V7c0-1.1-.9-2-2-2zm0 16H8V7h11v14z"})})},fo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M20 9H4v2h16V9zM4 15h16v-2H4v2z"})})},ho=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M23 8c0 1.1-.9 2-2 2-.18 0-.35-.02-.51-.07l-3.56 3.55c.05.16.07.34.07.52 0 1.1-.9 2-2 2s-2-.9-2-2c0-.18.02-.36.07-.52l-2.55-2.55c-.16.05-.34.07-.52.07s-.36-.02-.52-.07l-4.55 4.56c.05.16.07.33.07.51 0 1.1-.9 2-2 2s-2-.9-2-2 .9-2 2-2c.18 0 .35.02.51.07l4.56-4.55C8.02 9.36 8 9.18 8 9c0-1.1.9-2 2-2s2 .9 2 2c0 .18-.02.36-.07.52l2.55 2.55c.16-.05.34-.07.52-.07s.36.02.52.07l3.55-3.56C19.02 8.35 19 8.18 19 8c0-1.1.9-2 2-2s2 .9 2 2z"})})},po=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Vr("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M21 5C19.89 4.65 18.67 4.5 17.5 4.5C15.55 4.5 13.45 4.9 12 6C10.55 4.9 8.45 4.5 6.5 4.5C5.33 4.5 4.11 4.65 3 5C2.25 5.25 1.6 5.55 1 6V20.6C1 20.85 1.25 21.1 1.5 21.1C1.6 21.1 1.65 21.1 1.75 21.05C3.15 20.3 4.85 20 6.5 20C8.2 20 10.65 20.65 12 21.5C13.35 20.65 15.8 20 17.5 20C19.15 20 20.85 20.3 22.25 21.05C22.35 21.1 22.4 21.1 22.5 21.1C22.75 21.1 23 20.85 23 20.6V6C22.4 5.55 21.75 5.25 21 5ZM21 18.5C19.9 18.15 18.7 18 17.5 18C15.8 18 13.35 18.65 12 19.5C10.65 18.65 8.2 18 6.5 18C5.3 18 4.1 18.15 3 18.5V7C4.1 6.65 5.3 6.5 6.5 6.5C8.2 6.5 10.65 7.15 12 8C13.35 7.15 15.8 6.5 17.5 6.5C18.7 6.5 19.9 6.65 21 7V18.5Z"}),Vr("path",{d:"M17.5 10.5C18.38 10.5 19.23 10.59 20 10.76V9.24C19.21 9.09 18.36 9 17.5 9C15.8 9 14.26 9.29 13 9.83V11.49C14.13 10.85 15.7 10.5 17.5 10.5ZM13 12.49V14.15C14.13 13.51 15.7 13.16 17.5 13.16C18.38 13.16 19.23 13.25 20 13.42V11.9C19.21 11.75 18.36 11.66 17.5 11.66C15.8 11.66 14.26 11.96 13 12.49ZM17.5 14.33C15.8 14.33 14.26 14.62 13 15.16V16.82C14.13 16.18 15.7 15.83 17.5 15.83C18.38 15.83 19.23 15.92 20 16.09V14.57C19.21 14.41 18.36 14.33 17.5 14.33Z"}),Vr("path",{d:"M6.5 10.5C5.62 10.5 4.77 10.59 4 10.76V9.24C4.79 9.09 5.64 9 6.5 9C8.2 9 9.74 9.29 11 9.83V11.49C9.87 10.85 8.3 10.5 6.5 10.5ZM11 12.49V14.15C9.87 13.51 8.3 13.16 6.5 13.16C5.62 13.16 4.77 13.25 4 13.42V11.9C4.79 11.75 5.64 11.66 6.5 11.66C8.2 11.66 9.74 11.96 11 12.49ZM6.5 14.33C8.2 14.33 9.74 14.62 11 15.16V16.82C9.87 16.18 8.3 15.83 6.5 15.83C5.62 15.83 4.77 15.92 4 16.09V14.57C4.79 14.41 5.64 14.33 6.5 14.33Z"})]})},vo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 2C6.49 2 2 6.49 2 12s4.49 10 10 10 10-4.49 10-10S17.51 2 12 2zm0 18c-4.41 0-8-3.59-8-8s3.59-8 8-8 8 3.59 8 8-3.59 8-8 8zm3-8c0 1.66-1.34 3-3 3s-3-1.34-3-3 1.34-3 3-3 3 1.34 3 3z"})})},mo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M12 2C6.48 2 2 6.48 2 12C2 17.52 6.48 22 12 22C17.52 22 22 17.52 22 12C22 6.48 17.52 2 12 2ZM12 6C9.79 6 8 7.79 8 10H10C10 8.9 10.9 8 12 8C13.1 8 14 8.9 14 10C14 10.8792 13.4202 11.3236 12.7704 11.8217C11.9421 12.4566 11 13.1787 11 15H13C13 13.9046 13.711 13.2833 14.4408 12.6455C15.21 11.9733 16 11.2829 16 10C16 7.79 14.21 6 12 6ZM13 16V18H11V16H13Z"})})},go=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M4 20h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2s.9 2 2 2zm0-3h2v2H4v-2zM2 6c0 1.1.9 2 2 2h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2zm4 1H4V5h2v2zm-2 7h16c1.1 0 2-.9 2-2s-.9-2-2-2H4c-1.1 0-2 .9-2 2s.9 2 2 2zm0-3h2v2H4v-2z"})})},yo=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M12 8c1.1 0 2-.9 2-2s-.9-2-2-2-2 .9-2 2 .9 2 2 2zm0 2c-1.1 0-2 .9-2 2s.9 2 2 2 2-.9 2-2-.9-2-2-2zm0 6c-1.1 0-2 .9-2 2s.9 2 2 2 2-.9 2-2-.9-2-2-2z"})})},_o=function(){return Vr("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Vr("path",{d:"M3 17v2h6v-2H3zM3 5v2h10V5H3zm10 16v-2h8v-2h-8v-2h-2v6h2zM7 9v2H3v2h4v2h2V9H7zm14 4v-2H11v2h10zm-6-4h2V7h4V5h-4V3h-2v6z"})})},bo=function(e){var t=Ft(ee({width:0,height:0}),2),n=t[0],r=t[1];return ne((function(){var t=new ResizeObserver((function(e){var t=e[0].contentRect,n=t.width,i=t.height;r({width:n,height:i})}));return e&&t.observe(e),function(){e&&t.unobserve(e)}}),[e]),n},Do=n(123),wo=n.n(Do);function xo(e,t){if(null==e)return{};var n,r,i=function(e,t){if(null==e)return{};var n,r,i={},o=Object.keys(e);for(r=0;r=0||(i[n]=e[n]);return i}(e,t);if(Object.getOwnPropertySymbols){var o=Object.getOwnPropertySymbols(e);for(r=0;r=0||Object.prototype.propertyIsEnumerable.call(e,n)&&(i[n]=e[n])}return i}var ko=["to","isNavLink","children"],Co=function(e){var t=e.to,n=e.isNavLink,r=e.children,i=xo(e,ko);return n?Vr(fr,yr(yr({to:t},i),{},{children:r})):Vr("div",yr(yr({},i),{},{children:r}))},Ao=function(e){var t,n=e.activeItem,r=e.item,i=e.color,o=void 0===i?Rr("color-primary"):i,a=e.activeNavRef,u=e.onChange,l=e.isNavLink;return Vr(Co,{className:wo()(mr({"vm-tabs-item":!0,"vm-tabs-item_active":n===r.value},r.className||"",r.className)),isNavLink:l,to:r.value,style:{color:o},onClick:(t=r.value,function(){u&&u(t)}),ref:n===r.value?a:void 0,children:[r.icon&&Vr("div",{className:wo()({"vm-tabs-item__icon":!0,"vm-tabs-item__icon_single":!r.label}),children:r.icon}),r.label]})},Eo=function(e){var t=e.activeItem,n=e.items,r=e.color,i=void 0===r?Rr("color-primary"):r,o=e.onChange,a=e.indicatorPlacement,u=void 0===a?"bottom":a,l=e.isNavLink,c=bo(document.body),s=ie(null),f=Ft(ee({left:0,width:0,bottom:0}),2),d=f[0],h=f[1];return ne((function(){var e;if((null===(e=s.current)||void 0===e?void 0:e.base)instanceof HTMLElement){var t=s.current.base,n=t.offsetLeft,r=t.offsetWidth,i=t.offsetHeight;h({left:n,width:r,bottom:"top"===u?i-2:0})}}),[c,t,s,n]),Vr("div",{className:"vm-tabs",children:[n.map((function(e){return Vr(Ao,{activeItem:t,item:e,onChange:o,color:i,activeNavRef:s,isNavLink:l},e.value)})),Vr("div",{className:"vm-tabs__indicator",style:yr(yr({},d),{},{borderColor:i})})]})},So=[{value:"chart",icon:Vr(to,{}),label:"Graph",prometheusCode:0},{value:"code",icon:Vr(ro,{}),label:"JSON",prometheusCode:3},{value:"table",icon:Vr(no,{}),label:"Table",prometheusCode:1}],No=function(){var e=Io().displayType,t=Po();return Vr(Eo,{activeItem:e,items:So,onChange:function(n){var r;t({type:"SET_DISPLAY_TYPE",payload:null!==(r=n)&&void 0!==r?r:e})}})},Fo=Br("g0.tab",0),Mo=So.find((function(e){return e.prometheusCode===+Fo||e.value===Fo})),To=Ir("SERIES_LIMITS"),Oo={displayType:(null===Mo||void 0===Mo?void 0:Mo.value)||"chart",nocache:!1,isTracingEnabled:!1,seriesLimits:To?JSON.parse(Ir("SERIES_LIMITS")):Tr,tableCompact:Ir("TABLE_COMPACT")||!1};function Bo(e,t){switch(t.type){case"SET_DISPLAY_TYPE":return yr(yr({},e),{},{displayType:t.payload});case"SET_SERIES_LIMITS":return Lr("SERIES_LIMITS",JSON.stringify(t.payload)),yr(yr({},e),{},{seriesLimits:t.payload});case"TOGGLE_QUERY_TRACING":return yr(yr({},e),{},{isTracingEnabled:!e.isTracingEnabled});case"TOGGLE_NO_CACHE":return yr(yr({},e),{},{nocache:!e.nocache});case"TOGGLE_TABLE_COMPACT":return Lr("TABLE_COMPACT",!e.tableCompact),yr(yr({},e),{},{tableCompact:!e.tableCompact});default:throw new Error}}var Lo=j({}),Io=function(){return le(Lo).state},Po=function(){return le(Lo).dispatch},Ro={customStep:Br("g0.step_input",""),yaxis:{limits:{enable:!1,range:{1:[0,0]}}}};function zo(e,t){switch(t.type){case"TOGGLE_ENABLE_YAXIS_LIMITS":return yr(yr({},e),{},{yaxis:yr(yr({},e.yaxis),{},{limits:yr(yr({},e.yaxis.limits),{},{enable:!e.yaxis.limits.enable})})});case"SET_CUSTOM_STEP":return yr(yr({},e),{},{customStep:t.payload});case"SET_YAXIS_LIMITS":return yr(yr({},e),{},{yaxis:yr(yr({},e.yaxis),{},{limits:yr(yr({},e.yaxis.limits),{},{range:t.payload})})});default:throw new Error}}var jo=j({}),$o=function(){return le(jo).state},Ho=function(){return le(jo).dispatch},Uo={runQuery:0,topN:Br("topN",10),date:Br("date",_t()().tz().format(Jr)),focusLabel:Br("focusLabel",""),match:Br("match",""),extraLabel:Br("extra_label","")};function Yo(e,t){switch(t.type){case"SET_TOP_N":return yr(yr({},e),{},{topN:t.payload});case"SET_DATE":return yr(yr({},e),{},{date:t.payload});case"SET_MATCH":return yr(yr({},e),{},{match:t.payload});case"SET_EXTRA_LABEL":return yr(yr({},e),{},{extraLabel:t.payload});case"SET_FOCUS_LABEL":return yr(yr({},e),{},{focusLabel:t.payload});case"RUN_QUERY":return yr(yr({},e),{},{runQuery:e.runQuery+1});default:throw new Error}}var Vo=j({}),qo=function(){return le(Vo).state},Wo=function(){return le(Vo).dispatch},Qo={topN:Br("topN",null),maxLifetime:Br("maxLifetime",""),runQuery:0};function Go(e,t){switch(t.type){case"SET_TOP_N":return yr(yr({},e),{},{topN:t.payload});case"SET_MAX_LIFE_TIME":return yr(yr({},e),{},{maxLifetime:t.payload});case"SET_RUN_QUERY":return yr(yr({},e),{},{runQuery:e.runQuery+1});default:throw new Error}}var Jo=j({}),Zo=function(){return le(Jo).state},Ko={windows:"Windows",mac:"Mac OS",linux:"Linux"};function Xo(){var e=bo(document.body),t=function(){var e=function(){var e=["Android","webOS","iPhone","iPad","iPod","BlackBerry","Windows Phone"].map((function(e){return navigator.userAgent.match(new RegExp(e,"i"))}));return e.some((function(e){return e}))}(),t=window.innerWidth<500;return e||t},n=Ft(ee(t()),2),r=n[0],i=n[1];return ne((function(){i(t())}),[e]),{isMobile:r}}var ea={success:Vr(Vi,{}),error:Vr(Yi,{}),warning:Vr(Ui,{}),info:Vr(Hi,{})},ta=function(e){var t,n=e.variant,r=e.children,i=Wr().isDarkTheme,o=Xo().isMobile;return Vr("div",{className:wo()((t={"vm-alert":!0},mr(t,"vm-alert_".concat(n),n),mr(t,"vm-alert_dark",i),mr(t,"vm-alert_mobile",o),t)),children:[Vr("div",{className:"vm-alert__icon",children:ea[n||"info"]}),Vr("div",{className:"vm-alert__content",children:r})]})},na=j({showInfoMessage:function(){}}),ra=function(){return le(na)},ia={dashboardsSettings:[],dashboardsLoading:!1,dashboardsError:""};function oa(e,t){switch(t.type){case"SET_DASHBOARDS_SETTINGS":return yr(yr({},e),{},{dashboardsSettings:t.payload});case"SET_DASHBOARDS_LOADING":return yr(yr({},e),{},{dashboardsLoading:t.payload});case"SET_DASHBOARDS_ERROR":return yr(yr({},e),{},{dashboardsError:t.payload});default:throw new Error}}var aa=j({}),ua=function(){return le(aa).state},la=function(){for(var e=arguments.length,t=new Array(e),n=0;nh,v=r.top-20<0,m=r.left+x.width+20>f,g=r.left-20<0;return p&&(r.top=t.top-x.height-u),v&&(r.top=t.height+t.top+u),m&&(r.left=t.right-x.width-l),g&&(r.left=t.left+l),d&&(r.width="".concat(t.width,"px")),r.top<0&&(r.top=20),r}),[n,i,b,t,d]);f&&sa(C,(function(){return D(!1)}),n),ne((function(){if(C.current&&b&&(!v||p)){var e=C.current.getBoundingClientRect(),t=e.right,n=e.width;if(t>window.innerWidth){var r=window.innerWidth-20-n;C.current.style.left=rm,y=r.top-20<0,_=r.left+h.width+20>p,b=r.left-20<0;return g&&(r.top=n.top-h.height-c),y&&(r.top=n.height+n.top+c),_&&(r.left=n.right-h.width-f),b&&(r.left=n.left+f),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[v,o,s,h]),b=function(){"boolean"!==typeof r&&f(!0)},D=function(){f(!1)};return ne((function(){"boolean"===typeof r&&f(r)}),[r]),ne((function(){var e,t=null===v||void 0===v||null===(e=v.current)||void 0===e?void 0:e.base;if(t)return t.addEventListener("mouseenter",b),t.addEventListener("mouseleave",D),function(){t.removeEventListener("mouseenter",b),t.removeEventListener("mouseleave",D)}}),[v]),Vr(g,{children:[Vr(g,{ref:v,children:t}),!l&&s&>.createPortal(Vr("div",{className:"vm-tooltip",ref:m,style:_,children:n}),document.body)]})},ga=(Object.values(Ko).find((function(e){return navigator.userAgent.indexOf(e)>=0}))||"unknown")===Ko.mac?"Cmd":"Ctrl",ya=[{title:"Query",list:[{keys:["Enter"],description:"Run"},{keys:["Shift","Enter"],description:"Multi-line queries"},{keys:[ga,"Arrow Up"],description:"Previous command from the Query history"},{keys:[ga,"Arrow Down"],description:"Next command from the Query history"},{keys:[ga,"Click by 'Eye'"],description:"Toggle multiple queries"}]},{title:"Graph",list:[{keys:[ga,"Scroll Up"],alt:["+"],description:"Zoom in"},{keys:[ga,"Scroll Down"],alt:["-"],description:"Zoom out"},{keys:[ga,"Click and Drag"],description:"Move the graph left/right"}]},{title:"Legend",list:[{keys:["Mouse Click"],description:"Select series"},{keys:[ga,"Mouse Click"],description:"Toggle multiple series"}]}],_a="Shortcut keys",ba=function(e){var t=e.showTitle,n=Ft(ee(!1),2),r=n[0],i=n[1],o=Ar();return Vr(g,{children:[Vr(ma,{open:!0!==t&&void 0,title:_a,placement:"bottom-center",children:Vr(fa,{className:o?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Ki,{}),onClick:function(){i(!0)},children:t&&_a})}),r&&Vr(va,{title:"Shortcut keys",onClose:function(){i(!1)},children:Vr("div",{className:"vm-shortcuts",children:ya.map((function(e){return Vr("div",{className:"vm-shortcuts-section",children:[Vr("h3",{className:"vm-shortcuts-section__title",children:e.title}),Vr("div",{className:"vm-shortcuts-section-list",children:e.list.map((function(e){return Vr("div",{className:"vm-shortcuts-section-list-item",children:[Vr("div",{className:"vm-shortcuts-section-list-item__key",children:[e.keys.map((function(t,n){return Vr(g,{children:[Vr("code",{children:t},t),n!==e.keys.length-1?"+":""]})})),e.alt&&e.alt.map((function(t,n){return Vr(g,{children:["or",Vr("code",{children:t},t),n!==e.alt.length-1?"+":""]})}))]}),Vr("p",{className:"vm-shortcuts-section-list-item__description",children:e.description})]},e.keys.join("+"))}))})]},e.title)}))})})]})},Da=function(e){var t=e.open;return Vr("button",{className:wo()({"vm-menu-burger":!0,"vm-menu-burger_opened":t}),children:Vr("span",{})})},wa=function(e){var t=e.background,n=e.color,r=jn().pathname,i=Xo().isMobile,o=ie(null),a=Ft(ee(!1),2),u=a[0],l=a[1],c=function(){l(!1)};return ne(c,[r]),sa(o,c),Vr("div",{className:"vm-header-sidebar",ref:o,children:[Vr("div",{className:wo()({"vm-header-sidebar-button":!0,"vm-header-sidebar-button_open":u}),onClick:function(){l((function(e){return!e}))},children:Vr(Da,{open:u})}),Vr("div",{className:wo()({"vm-header-sidebar-menu":!0,"vm-header-sidebar-menu_open":u}),children:[Vr("div",{children:Vr(pa,{color:n,background:t,direction:"column"})}),Vr("div",{className:"vm-header-sidebar-menu-settings",children:!i&&Vr(ba,{showTitle:!0})})]})]})},xa=function(e){var t=e.label,n=e.value,r=e.type,i=void 0===r?"text":r,o=e.error,a=void 0===o?"":o,u=e.placeholder,l=e.endIcon,c=e.startIcon,s=e.disabled,f=void 0!==s&&s,d=e.autofocus,h=void 0!==d&&d,p=e.helperText,v=e.inputmode,m=void 0===v?"text":v,g=e.onChange,y=e.onEnter,_=e.onKeyDown,b=e.onFocus,D=e.onBlur,w=Wr().isDarkTheme,x=Xo().isMobile,k=ie(null),C=ie(null),A=ae((function(){return"textarea"===i?C:k}),[i]),E=wo()({"vm-text-field__input":!0,"vm-text-field__input_error":a,"vm-text-field__input_icon-start":c,"vm-text-field__input_disabled":f,"vm-text-field__input_textarea":"textarea"===i}),S=function(e){_&&_(e),"Enter"!==e.key||e.shiftKey||(e.preventDefault(),y&&y())},N=function(e){f||g&&g(e.target.value)};ne((function(){var e;h&&!x&&(null===A||void 0===A||null===(e=A.current)||void 0===e?void 0:e.focus)&&A.current.focus()}),[A,h]);var F=function(){b&&b()},M=function(){D&&D()};return Vr("label",{className:wo()({"vm-text-field":!0,"vm-text-field_textarea":"textarea"===i,"vm-text-field_dark":w}),"data-replicated-value":n,children:[c&&Vr("div",{className:"vm-text-field__icon-start",children:c}),l&&Vr("div",{className:"vm-text-field__icon-end",children:l}),"textarea"===i?Vr("textarea",{className:E,disabled:f,ref:C,value:n,rows:1,inputMode:m,placeholder:u,autoCapitalize:"none",onInput:N,onKeyDown:S,onFocus:F,onBlur:M}):Vr("input",{className:E,disabled:f,ref:k,value:n,type:i,placeholder:u,inputMode:m,autoCapitalize:"none",onInput:N,onKeyDown:S,onFocus:F,onBlur:M}),t&&Vr("span",{className:"vm-text-field__label",children:t}),Vr("span",{className:"vm-text-field__error","data-show":!!a,children:a}),p&&!a&&Vr("span",{className:"vm-text-field__helper-text",children:p})]})},ka=function(e){var t=e.accountIds,n=Ar(),r=Xo().isMobile,i=Wr(),o=i.tenantId,a=i.serverUrl,u=Qr(),l=Fi(),c=Ft(ee(""),2),f=c[0],d=c[1],h=Ft(ee(!1),2),p=h[0],v=h[1],m=ie(null),g=ae((function(){if(!f)return t;try{var e=new RegExp(f,"i");return t.filter((function(t){return e.test(t)})).sort((function(t,n){var r,i;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(i=n.match(e))||void 0===i?void 0:i.index)||0)}))}catch(s){return[]}}),[f,t]),y=ae((function(){return t.length>1&&!0}),[t,a]),_=function(){v((function(e){return!e}))},b=function(){v(!1)},D=function(e){return function(){var t=e;if(u({type:"SET_TENANT_ID",payload:t}),a){var n=Er(a,t);if(n===a)return;u({type:"SET_SERVER",payload:n}),l({type:"RUN_QUERY"})}b()}};return ne((function(){var e=(a.match(/(\/select\/)(\d+|\d.+)(\/)(.+)/)||[])[2];o&&o!==e?D(o)():D(e)()}),[a]),y?Vr("div",{className:"vm-tenant-input",children:[Vr(ma,{title:"Define Tenant ID if you need request to another storage",children:Vr("div",{ref:m,children:r?Vr("div",{className:"vm-mobile-option",onClick:_,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(go,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Tenant ID"}),Vr("span",{className:"vm-mobile-option-text__value",children:o})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(fa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",fullWidth:!0,startIcon:Vr(go,{}),endIcon:Vr("div",{className:wo()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":p}),children:Vr(Wi,{})}),onClick:_,children:o})})}),Vr(da,{open:p,placement:"bottom-right",onClose:b,buttonRef:m,title:r?"Define Tenant ID":void 0,children:Vr("div",{className:wo()({"vm-list vm-tenant-input-list":!0,"vm-list vm-tenant-input-list_mobile":r}),children:[Vr("div",{className:"vm-tenant-input-list__search",children:Vr(xa,{autofocus:!0,label:"Search",value:f,onChange:d,type:"search"})}),g.map((function(e){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":e===o}),onClick:D(e),children:e},e)}))]})})]}):null};var Ca,Aa=function(e){var t=ie();return ne((function(){t.current=e}),[e]),t.current},Ea=function(){var e=Ar(),t=Xo().isMobile,n=$o().customStep,r=Ni().period.step,i=Ho(),o=Ni().period,a=Aa(o.end-o.start),u=Ft(ee(!1),2),l=u[0],c=u[1],s=Ft(ee(n||r),2),f=s[0],d=s[1],h=Ft(ee(""),2),p=h[0],v=h[1],m=ie(null),g=function(){c((function(e){return!e}))},y=function(){c(!1)},_=function(e){var t=e||f||r||"1s",n=(t.match(/[a-zA-Z]+/g)||[]).length?t:"".concat(t,"s");i({type:"SET_CUSTOM_STEP",payload:n}),d(n),v("")},b=function(e){var t=e.match(/[-+]?([0-9]*\.[0-9]+|[0-9]+)/g)||[],n=e.match(/[a-zA-Z]+/g)||[],r=t.length&&t.every((function(e){return parseFloat(e)>0})),i=n.every((function(e){return ii.find((function(t){return t.short===e}))})),o=r&&i;d(e),v(o?"":_r.validStep)};return ne((function(){n&&_(n)}),[n]),ne((function(){!n&&r&&_(r)}),[r]),ne((function(){o.end-o.start!==a&&a&&r&&_(r)}),[o,a,r]),Vr("div",{className:"vm-step-control",ref:m,children:[t?Vr("div",{className:"vm-mobile-option",onClick:g,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(ho,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Step"}),Vr("span",{className:"vm-mobile-option-text__value",children:f})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Query resolution step width",children:Vr(fa,{className:e?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(ho,{}),onClick:g,children:Vr("p",{children:["STEP",Vr("p",{className:"vm-step-control__value",children:f})]})})}),Vr(da,{open:l,placement:"bottom-right",onClose:y,buttonRef:m,title:t?"Query resolution step width":void 0,children:Vr("div",{className:wo()({"vm-step-control-popper":!0,"vm-step-control-popper_mobile":t}),children:[Vr(xa,{autofocus:!0,label:"Step value",value:f,error:p,onChange:b,onEnter:function(){_(),y()},onFocus:function(){document.activeElement instanceof HTMLInputElement&&document.activeElement.select()},onBlur:_,endIcon:Vr(ma,{title:"Set default step value: ".concat(r),children:Vr(fa,{size:"small",variant:"text",color:"primary",startIcon:Vr($i,{}),onClick:function(){var e=r||"1s";b(e),_(e)}})})}),Vr("div",{className:"vm-step-control-popper-info",children:[Vr("code",{children:"step"})," - the ",Vr("a",{className:"vm-link vm-link_colored",href:"https://prometheus.io/docs/prometheus/latest/querying/basics/#time-durations",target:"_blank",rel:"noreferrer",children:"interval"}),"between datapoints, which must be returned from the range query. The ",Vr("code",{children:"query"})," is executed at",Vr("code",{children:"start"}),", ",Vr("code",{children:"start+step"}),", ",Vr("code",{children:"start+2*step"}),", \u2026, ",Vr("code",{children:"end"})," timestamps.",Vr("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/keyConcepts.html#range-query",target:"_blank",rel:"help noreferrer",children:"Read more about Range query"})]})]})})]})},Sa=function(e){var t=e.relativeTime,n=e.setDuration,r=Xo().isMobile;return Vr("div",{className:wo()({"vm-time-duration":!0,"vm-time-duration_mobile":r}),children:pi.map((function(e){var i,o=e.id,a=e.duration,u=e.until,l=e.title;return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":o===t}),onClick:(i={duration:a,until:u(),id:o},function(){n(i)}),children:l||a},o)}))})},Na=function(e){var t=e.viewDate,n=e.displayYears,r=e.onChangeViewDate;return Vr("div",{className:"vm-calendar-header",children:[Vr("div",{className:"vm-calendar-header-left",onClick:e.toggleDisplayYears,children:[Vr("span",{className:"vm-calendar-header-left__date",children:t.format("MMMM YYYY")}),Vr("div",{className:"vm-calendar-header-left__select-year",children:Vr(Qi,{})})]}),!n&&Vr("div",{className:"vm-calendar-header-right",children:[Vr("div",{className:"vm-calendar-header-right__prev",onClick:function(){r(t.subtract(1,"month"))},children:Vr(Wi,{})}),Vr("div",{className:"vm-calendar-header-right__next",onClick:function(){r(t.add(1,"month"))},children:Vr(Wi,{})})]})]})},Fa=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],Ma=function(e){var t=e.viewDate,n=e.selectDate,r=e.onChangeSelectDate,i=_t()().tz().startOf("day"),o=ae((function(){var e=new Array(42).fill(null),n=t.startOf("month"),r=t.endOf("month").diff(n,"day")+1,i=new Array(r).fill(n).map((function(e,t){return e.add(t,"day")})),o=n.day();return e.splice.apply(e,[o,r].concat(Ot(i))),e}),[t]),a=function(e){return function(){e&&r(e)}};return Vr("div",{className:"vm-calendar-body",children:[Fa.map((function(e){return Vr("div",{className:"vm-calendar-body-cell vm-calendar-body-cell_weekday",children:e[0]},e)})),o.map((function(e,t){return Vr("div",{className:wo()({"vm-calendar-body-cell":!0,"vm-calendar-body-cell_day":!0,"vm-calendar-body-cell_day_empty":!e,"vm-calendar-body-cell_day_active":(e&&e.toISOString())===n.startOf("day").toISOString(),"vm-calendar-body-cell_day_today":(e&&e.toISOString())===i.toISOString()}),onClick:a(e),children:e&&e.format("D")},e?e.toISOString():t)}))]})},Ta=function(e){var t=e.viewDate,n=e.onChangeViewDate,r=ae((function(){return t.format("YYYY")}),[t]),i=ae((function(){var e=_t()().subtract(103,"year");return new Array(206).fill(e).map((function(e,t){return e.add(t,"year")}))}),[t]);ne((function(){var e=document.getElementById("vm-calendar-year-".concat(r));e&&e.scrollIntoView({block:"center"})}),[]);return Vr("div",{className:"vm-calendar-years",children:i.map((function(e){return Vr("div",{className:wo()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("YYYY")===r}),id:"vm-calendar-year-".concat(e.format("YYYY")),onClick:(t=e,function(){n(t)}),children:e.format("YYYY")},e.format("YYYY"));var t}))})};!function(e){e[e.hour=0]="hour",e[e.minutes=1]="minutes",e[e.seconds=2]="seconds"}(Ca||(Ca={}));var Oa=function(e){var t=e.selectDate,n=e.onChangeTime,r=e.onClose,i=Wr().isDarkTheme,o=Ft(ee(Ca.hour),2),a=o[0],u=o[1],l=Ft(ee(t.format("HH")),2),c=l[0],s=l[1],f=Ft(ee(t.format("mm")),2),d=f[0],h=f[1],p=Ft(ee(t.format("ss")),2),v=p[0],m=p[1],g=ae((function(){return a===Ca.hour?new Array(24).fill("00").map((function(e,t){return{value:t,degrees:t/12*360,offset:0===t||t>12,title:t?"".concat(t):e}})):new Array(60).fill("00").map((function(e,t){return{value:t,degrees:t/60*360,offset:!1,title:t?"".concat(t):e}}))}),[a,c,d,v]),y=ae((function(){switch(a){case Ca.hour:return+c/12*360;case Ca.minutes:return+d/60*360;case Ca.seconds:return+v/60*360}}),[a,c,d,v]),_=ie(null),b=ie(null),D=ie(null),w=function(e){return function(t){!function(e,t){t.target.select(),u(e)}(e,t)}};return ne((function(){n("".concat(c,":").concat(d,":").concat(v))}),[c,d,v]),ne((function(){s(t.format("HH")),h(t.format("mm")),m(t.format("ss"))}),[t]),ne((function(){_.current&&_.current.focus()}),[]),Vr("div",{className:"vm-calendar-time-picker",children:[Vr("div",{className:"vm-calendar-time-picker-clock",children:[Vr("div",{className:wo()({"vm-calendar-time-picker-clock__arrow":!0,"vm-calendar-time-picker-clock__arrow_offset":a===Ca.hour&&("00"===c||+c>12)}),style:{transform:"rotate(".concat(y,"deg)")}}),g.map((function(e){return Vr("div",{className:wo()({"vm-calendar-time-picker-clock__time":!0,"vm-calendar-time-picker-clock__time_offset":e.offset,"vm-calendar-time-picker-clock__time_hide":g.length>24&&e.value%5}),style:{transform:"rotate(".concat(e.degrees,"deg)")},onClick:(t=e.value,function(){var e=String(t);switch(a){case Ca.hour:s(e),b.current&&b.current.focus();break;case Ca.minutes:h(e),D.current&&D.current.focus();break;case Ca.seconds:m(e),r()}}),children:Vr("span",{style:{transform:"rotate(-".concat(e.degrees,"deg)")},children:e.title})},e.value);var t}))]}),Vr("div",{className:wo()({"vm-calendar-time-picker-fields":!0,"vm-calendar-time-picker-fields_dark":i}),children:[Vr("input",{className:"vm-calendar-time-picker-fields__input",value:c,onChange:function(e){var t=e.target,n=t.value,r=+n>23?"23":n;t.value=r,s(r),n.length>1&&b.current&&b.current.focus()},onFocus:w(Ca.hour),ref:_,type:"number",min:0,max:24}),Vr("span",{children:":"}),Vr("input",{className:"vm-calendar-time-picker-fields__input",value:d,onChange:function(e){var t=e.target,n=t.value,r=+n>59?"59":n;t.value=r,h(r),n.length>1&&D.current&&D.current.focus()},onFocus:w(Ca.minutes),ref:b,type:"number",min:0,max:60}),Vr("span",{children:":"}),Vr("input",{className:"vm-calendar-time-picker-fields__input",value:v,onChange:function(e){var t=e.target,n=t.value,i=+n>59?"59":n;t.value=i,m(i),n.length>1&&D.current&&r()},onFocus:w(Ca.seconds),ref:D,type:"number",min:0,max:60})]})]})},Ba=[{value:"date",icon:Vr(Ji,{})},{value:"time",icon:Vr(Gi,{})}],La=function(e){var t=e.date,n=e.timepicker,r=void 0!==n&&n,i=e.format,o=void 0===i?Zr:i,a=e.onChange,u=e.onClose,l=Ft(ee(!1),2),c=l[0],s=l[1],f=Ft(ee(_t().tz(t)),2),d=f[0],h=f[1],p=Ft(ee(_t().tz(t)),2),v=p[0],m=p[1],y=Ft(ee(Ba[0].value),2),_=y[0],b=y[1],D=Xo().isMobile,w=function(e){h(e),s(!1)};return ne((function(){v.format()!==_t().tz(t).format()&&a(v.format(o))}),[v]),Vr("div",{className:wo()({"vm-calendar":!0,"vm-calendar_mobile":D}),children:["date"===_&&Vr(Na,{viewDate:d,onChangeViewDate:w,toggleDisplayYears:function(){s((function(e){return!e}))},displayYears:c}),"date"===_&&Vr(g,{children:[!c&&Vr(Ma,{viewDate:d,selectDate:v,onChangeSelectDate:function(e){m(e),r&&b("time")}}),c&&Vr(Ta,{viewDate:d,onChangeViewDate:w})]}),"time"===_&&Vr(Oa,{selectDate:v,onChangeTime:function(e){var t=Ft(e.split(":"),3),n=t[0],r=t[1],i=t[2];m((function(e){return e.set("hour",+n).set("minute",+r).set("second",+i)}))},onClose:function(){u&&u()}}),r&&Vr("div",{className:"vm-calendar__tabs",children:Vr(Eo,{activeItem:_,items:Ba,onChange:function(e){b(e)},indicatorPlacement:"top"})})]})},Ia=Ae((function(e,t){var n=e.date,r=e.targetRef,i=e.format,o=void 0===i?Zr:i,a=e.timepicker,u=e.onChange,l=e.label,c=Ft(ee(!1),2),s=c[0],f=c[1],d=ae((function(){return n?_t().tz(n):_t()().tz()}),[n]),h=Xo().isMobile,p=function(){f((function(e){return!e}))},v=function(){f(!1)},m=function(e){"Escape"!==e.key&&"Enter"!==e.key||v()};return ne((function(){var e;return null===(e=r.current)||void 0===e||e.addEventListener("click",p),function(){var e;null===(e=r.current)||void 0===e||e.removeEventListener("click",p)}}),[r]),ne((function(){return window.addEventListener("keyup",m),function(){window.removeEventListener("keyup",m)}}),[]),Vr(g,{children:Vr(da,{open:s,buttonRef:r,placement:"bottom-right",onClose:v,title:h?l:void 0,children:Vr("div",{ref:t,children:Vr(La,{date:d,format:o,timepicker:a,onChange:function(e){a||v(),u(e)},onClose:v})})})})})),Pa=Ia,Ra=function(){var e=Xo().isMobile,t=Wr().isDarkTheme,n=ie(null),r=bo(document.body),i=ae((function(){return r.width>1280}),[r]),o=Ft(ee(),2),a=o[0],u=o[1],l=Ft(ee(),2),c=l[0],s=l[1],f=ae((function(){return _t().tz(c).format(Zr)}),[c]),d=ae((function(){return _t().tz(a).format(Zr)}),[a]),h=Ni(),p=h.period,v=p.end,m=p.start,y=h.relativeTime,_=h.timezone,b=h.duration,D=Fi(),w=Ar(),x=ae((function(){return{region:_,utc:mi(_)}}),[_]);ne((function(){u(fi(hi(v)))}),[_,v]),ne((function(){s(fi(hi(m)))}),[_,m]);var k=function(e){var t=e.duration,n=e.until,r=e.id;D({type:"SET_RELATIVE_TIME",payload:{duration:t,until:n,id:r}}),O(!1)},C=ae((function(){return{start:_t().tz(hi(m)).format(Zr),end:_t().tz(hi(v)).format(Zr)}}),[m,v,_]),A=ae((function(){return y&&"none"!==y?y.replace(/_/g," "):"".concat(C.start," - ").concat(C.end)}),[y,C]),E=ie(null),S=ie(null),N=ie(null),F=ie(null),M=Ft(ee(!1),2),T=M[0],O=M[1],B=ie(null),L=function(){O((function(e){return!e}))},I=function(){O(!1)};return ne((function(){var e=vi({relativeTimeId:y,defaultDuration:b,defaultEndInput:hi(v)});k({id:e.relativeTimeId,duration:e.duration,until:e.endInput})}),[_]),sa(n,(function(t){var n,r;if(!e){var i=t.target,o=(null===E||void 0===E?void 0:E.current)&&E.current.contains(i),a=(null===S||void 0===S?void 0:S.current)&&S.current.contains(i),u=(null===N||void 0===N?void 0:N.current)&&(null===N||void 0===N||null===(n=N.current)||void 0===n?void 0:n.contains(i)),l=(null===F||void 0===F?void 0:F.current)&&(null===F||void 0===F||null===(r=F.current)||void 0===r?void 0:r.contains(i));o||a||u||l||I()}})),Vr(g,{children:[Vr("div",{ref:B,children:e?Vr("div",{className:"vm-mobile-option",onClick:L,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(Gi,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Time range"}),Vr("span",{className:"vm-mobile-option-text__value",children:A})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:i?"Time range controls":A,children:Vr(fa,{className:w?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Gi,{}),onClick:L,children:i&&Vr("span",{children:A})})})}),Vr(da,{open:T,buttonRef:B,placement:"bottom-right",onClose:I,clickOutside:!1,title:e?"Time range controls":"",children:Vr("div",{className:wo()({"vm-time-selector":!0,"vm-time-selector_mobile":e}),ref:n,children:[Vr("div",{className:"vm-time-selector-left",children:[Vr("div",{className:wo()({"vm-time-selector-left-inputs":!0,"vm-time-selector-left-inputs_dark":t}),children:[Vr("div",{className:"vm-time-selector-left-inputs__date",ref:E,children:[Vr("label",{children:"From:"}),Vr("span",{children:f}),Vr(Ji,{}),Vr(Pa,{label:"Date From",ref:N,date:c||"",onChange:function(e){return s(e)},targetRef:E,timepicker:!0})]}),Vr("div",{className:"vm-time-selector-left-inputs__date",ref:S,children:[Vr("label",{children:"To:"}),Vr("span",{children:d}),Vr(Ji,{}),Vr(Pa,{label:"Date To",ref:F,date:a||"",onChange:function(e){return u(e)},targetRef:S,timepicker:!0})]})]}),Vr("div",{className:"vm-time-selector-left-timezone",children:[Vr("div",{className:"vm-time-selector-left-timezone__title",children:x.region}),Vr("div",{className:"vm-time-selector-left-timezone__utc",children:x.utc})]}),Vr(fa,{variant:"text",startIcon:Vr(Zi,{}),onClick:function(){return D({type:"RUN_QUERY_TO_NOW"})},children:"switch to now"}),Vr("div",{className:"vm-time-selector-left__controls",children:[Vr(fa,{color:"error",variant:"outlined",onClick:function(){u(fi(hi(v))),s(fi(hi(m))),O(!1)},children:"Cancel"}),Vr(fa,{color:"primary",onClick:function(){return c&&a&&D({type:"SET_PERIOD",payload:{from:_t().tz(c).toDate(),to:_t().tz(a).toDate()}}),void O(!1)},children:"Apply"})]})]}),Vr(Sa,{relativeTime:y||"",setDuration:k})]})})]})},za=function(){var e=Xo().isMobile,t=Ar(),n=ie(null),r=qo().date,i=Wo(),o=ae((function(){return _t().tz(r).format(Jr)}),[r]);return Vr("div",{children:[Vr("div",{ref:n,children:e?Vr("div",{className:"vm-mobile-option",children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(Ji,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Date control"}),Vr("span",{className:"vm-mobile-option-text__value",children:o})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Date control",children:Vr(fa,{className:t?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Vr(Ji,{}),children:o})})}),Vr(Pa,{label:"Date control",date:r||"",format:Jr,onChange:function(e){i({type:"SET_DATE",payload:e})},targetRef:n})]})},ja=[{seconds:0,title:"Off"},{seconds:1,title:"1s"},{seconds:2,title:"2s"},{seconds:5,title:"5s"},{seconds:10,title:"10s"},{seconds:30,title:"30s"},{seconds:60,title:"1m"},{seconds:300,title:"5m"},{seconds:900,title:"15m"},{seconds:1800,title:"30m"},{seconds:3600,title:"1h"},{seconds:7200,title:"2h"}],$a=function(){var e=Xo().isMobile,t=Fi(),n=Ar(),r=Ft(ee(!1),2),i=r[0],o=r[1],a=Ft(ee(ja[0]),2),u=a[0],l=a[1];ne((function(){var e,n=u.seconds;return i?e=setInterval((function(){t({type:"RUN_QUERY"})}),1e3*n):l(ja[0]),function(){e&&clearInterval(e)}}),[u,i]);var c=Ft(ee(!1),2),s=c[0],f=c[1],d=ie(null),h=function(){f((function(e){return!e}))},p=function(e){return function(){!function(e){(i&&!e.seconds||!i&&e.seconds)&&o((function(e){return!e})),l(e),f(!1)}(e)}};return Vr(g,{children:[Vr("div",{className:"vm-execution-controls",children:Vr("div",{className:wo()({"vm-execution-controls-buttons":!0,"vm-execution-controls-buttons_mobile":e,"vm-header-button":!n}),children:[!e&&Vr(ma,{title:"Refresh dashboard",children:Vr(fa,{variant:"contained",color:"primary",onClick:function(){t({type:"RUN_QUERY"})},startIcon:Vr(qi,{})})}),e?Vr("div",{className:"vm-mobile-option",onClick:h,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr($i,{})}),Vr("div",{className:"vm-mobile-option-text",children:[Vr("span",{className:"vm-mobile-option-text__label",children:"Auto-refresh"}),Vr("span",{className:"vm-mobile-option-text__value",children:u.title})]}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:"Auto-refresh control",children:Vr("div",{ref:d,children:Vr(fa,{variant:"contained",color:"primary",fullWidth:!0,endIcon:Vr("div",{className:wo()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":s}),children:Vr(Wi,{})}),onClick:h,children:u.title})})})]})}),Vr(da,{open:s,placement:"bottom-right",onClose:function(){f(!1)},buttonRef:d,title:e?"Auto-refresh duration":void 0,children:Vr("div",{className:wo()({"vm-execution-controls-list":!0,"vm-execution-controls-list_mobile":e}),children:ja.map((function(t){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":e,"vm-list-item_active":t.seconds===u.seconds}),onClick:p(t),children:t.title},t.seconds)}))})})]})},Ha=function(e){var t;try{t=new URL(e)}catch(_){return!1}return"http:"===t.protocol||"https:"===t.protocol},Ua=function(e){var t=e.serverUrl,n=e.onChange,r=e.onEnter,i=e.onBlur,o=Ft(ee(""),2),a=o[0],u=o[1];return Vr(xa,{autofocus:!0,label:"Server URL",value:t,error:a,onChange:function(e){var t=e||"";n(t),u(""),t||u(_r.emptyServer),Ha(t)||u(_r.validServer)},onEnter:r,onBlur:i,inputmode:"url"})},Ya=[{label:"Graph",type:"chart"},{label:"JSON",type:"code"},{label:"Table",type:"table"}],Va=function(e){var t=e.limits,n=e.onChange,r=e.onEnter,i=Xo().isMobile,o=Ft(ee({table:"",chart:"",code:""}),2),a=o[0],u=o[1],l=function(e){return function(r){!function(e,r){var i=e||"";u((function(e){return yr(yr({},e),{},mr({},r,+i<0?_r.positiveNumber:""))})),n(yr(yr({},t),{},mr({},r,i||1/0)))}(r,e)}};return Vr("div",{className:"vm-limits-configurator",children:[Vr("div",{className:"vm-server-configurator__title",children:["Series limits by tabs",Vr(ma,{title:"To disable limits set to 0",children:Vr(fa,{variant:"text",color:"primary",size:"small",startIcon:Vr(Hi,{})})}),Vr("div",{className:"vm-limits-configurator-title__reset",children:Vr(fa,{variant:"text",color:"primary",size:"small",startIcon:Vr($i,{}),onClick:function(){n(Tr)},children:"Reset"})})]}),Vr("div",{className:wo()({"vm-limits-configurator__inputs":!0,"vm-limits-configurator__inputs_mobile":i}),children:Ya.map((function(e){return Vr("div",{children:Vr(xa,{label:e.label,value:t[e.type],error:a[e.type],onChange:l(e.type),onEnter:r,type:"number"})},e.type)}))})]})},qa=function(e){var t=e.defaultExpanded,n=void 0!==t&&t,r=e.onChange,i=e.title,o=e.children,a=Ft(ee(n),2),u=a[0],l=a[1];return ne((function(){r&&r(u)}),[u]),Vr(g,{children:[Vr("header",{className:"vm-accordion-header ".concat(u&&"vm-accordion-header_open"),onClick:function(){l((function(e){return!e}))},children:[i,Vr("div",{className:"vm-accordion-header__arrow ".concat(u&&"vm-accordion-header__arrow_open"),children:Vr(Wi,{})})]}),u&&Vr("section",{className:"vm-accordion-section",children:o},"content")]})},Wa=function(e){var t=e.timezoneState,n=e.onChange,r=Xo().isMobile,i=gi(),o=Ft(ee(!1),2),a=o[0],u=o[1],l=Ft(ee(""),2),c=l[0],f=l[1],d=ie(null),h=ae((function(){if(!c)return i;try{return gi(c)}catch(s){return{}}}),[c,i]),p=ae((function(){return Object.keys(h)}),[h]),v=ae((function(){return{region:_t().tz.guess(),utc:mi(_t().tz.guess())}}),[]),m=ae((function(){return{region:t,utc:mi(t)}}),[t]),g=function(){u(!1)},y=function(e){return function(){!function(e){n(e.region),f(""),g()}(e)}};return Vr("div",{className:"vm-timezones",children:[Vr("div",{className:"vm-server-configurator__title",children:"Time zone"}),Vr("div",{className:"vm-timezones-item vm-timezones-item_selected",onClick:function(){u((function(e){return!e}))},ref:d,children:[Vr("div",{className:"vm-timezones-item__title",children:m.region}),Vr("div",{className:"vm-timezones-item__utc",children:m.utc}),Vr("div",{className:wo()({"vm-timezones-item__icon":!0,"vm-timezones-item__icon_open":a}),children:Vr(Qi,{})})]}),Vr(da,{open:a,buttonRef:d,placement:"bottom-left",onClose:g,fullWidth:!0,title:r?"Time zone":void 0,children:Vr("div",{className:wo()({"vm-timezones-list":!0,"vm-timezones-list_mobile":r}),children:[Vr("div",{className:"vm-timezones-list-header",children:[Vr("div",{className:"vm-timezones-list-header__search",children:Vr(xa,{autofocus:!0,label:"Search",value:c,onChange:function(e){f(e)}})}),Vr("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:y(v),children:[Vr("div",{className:"vm-timezones-item__title",children:["Browser Time (",v.region,")"]}),Vr("div",{className:"vm-timezones-item__utc",children:v.utc})]})]}),p.map((function(e){return Vr("div",{className:"vm-timezones-list-group",children:Vr(qa,{defaultExpanded:!0,title:Vr("div",{className:"vm-timezones-list-group__title",children:e}),children:Vr("div",{className:"vm-timezones-list-group-options",children:h[e]&&h[e].map((function(e){return Vr("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:y(e),children:[Vr("div",{className:"vm-timezones-item__title",children:e.region}),Vr("div",{className:"vm-timezones-item__utc",children:e.utc})]},e.search)}))})})},e)}))]})})]})},Qa=function(e){var t=e.options,n=e.value,r=e.label,i=e.onChange,o=ie(null),a=Ft(ee({width:"0px",left:"0px",borderRadius:"0px"}),2),u=a[0],l=a[1],c=function(e){return function(){i(e)}};return ne((function(){if(o.current){var e=t.findIndex((function(e){return e.value===n})),r=o.current.getBoundingClientRect().width,i=e*r,a="0";0===e&&(a="16px 0 0 16px"),e===t.length-1&&(a="10px",i-=1,a="0 16px 16px 0"),0!==e&&e!==t.length-1&&(r+=1,i-=1),l({width:"".concat(r,"px"),left:"".concat(i,"px"),borderRadius:a})}else l({width:"0px",left:"0px",borderRadius:"0px"})}),[o,n,t]),Vr("div",{className:"vm-toggles",children:[r&&Vr("label",{className:"vm-toggles__label",children:r}),Vr("div",{className:"vm-toggles-group",style:{gridTemplateColumns:"repeat(".concat(t.length,", 1fr)")},children:[u.borderRadius&&Vr("div",{className:"vm-toggles-group__highlight",style:u}),t.map((function(e,t){return Vr("div",{className:wo()({"vm-toggles-group-item":!0,"vm-toggles-group-item_first":0===t,"vm-toggles-group-item_active":e.value===n,"vm-toggles-group-item_icon":e.icon&&e.title}),onClick:c(e.value),ref:e.value===n?o:null,children:[e.icon,e.title]},e.value)}))]})]})},Ga=Object.values(br).map((function(e){return{title:e,value:e}})),Ja=function(){var e=Xo().isMobile,t=Wr().theme,n=Qr();return Vr("div",{className:wo()({"vm-theme-control":!0,"vm-theme-control_mobile":e}),children:[Vr("div",{className:"vm-server-configurator__title",children:"Theme preferences"}),Vr("div",{className:"vm-theme-control__toggle",children:Vr(Qa,{options:Ga,value:t,onChange:function(e){n({type:"SET_THEME",payload:e})}})},"".concat(e))]})},Za="Settings",Ka=function(){var e=Xo().isMobile,t=Ar(),n=Wr().serverUrl,r=Ni().timezone,i=Io().seriesLimits,o=Qr(),a=Fi(),u=Po(),l=Ft(ee(n),2),c=l[0],s=l[1],f=Ft(ee(i),2),d=f[0],h=f[1],p=Ft(ee(r),2),v=p[0],m=p[1],y=Ft(ee(!1),2),_=y[0],b=y[1],D=function(){return b(!0)},w=function(){o({type:"SET_SERVER",payload:c}),a({type:"SET_TIMEZONE",payload:v}),u({type:"SET_SERIES_LIMITS",payload:d})};return ne((function(){n!==c&&s(n)}),[n]),Vr(g,{children:[e?Vr("div",{className:"vm-mobile-option",onClick:D,children:[Vr("span",{className:"vm-mobile-option__icon",children:Vr(zi,{})}),Vr("div",{className:"vm-mobile-option-text",children:Vr("span",{className:"vm-mobile-option-text__label",children:Za})}),Vr("span",{className:"vm-mobile-option__arrow",children:Vr(Wi,{})})]}):Vr(ma,{title:Za,children:Vr(fa,{className:wo()({"vm-header-button":!t}),variant:"contained",color:"primary",startIcon:Vr(zi,{}),onClick:D})}),_&&Vr(va,{title:Za,onClose:function(){return b(!1)},children:Vr("div",{className:wo()({"vm-server-configurator":!0,"vm-server-configurator_mobile":e}),children:[!t&&Vr("div",{className:"vm-server-configurator__input",children:Vr(Ua,{serverUrl:c,onChange:s,onEnter:w,onBlur:w})}),Vr("div",{className:"vm-server-configurator__input",children:Vr(Va,{limits:d,onChange:h,onEnter:w})}),Vr("div",{className:"vm-server-configurator__input",children:Vr(Wa,{timezoneState:v,onChange:m})}),!t&&Vr("div",{className:"vm-server-configurator__input",children:Vr(Ja,{})})]})})]})};function Xa(){Xa=function(){return e};var e={},t=Object.prototype,n=t.hasOwnProperty,r=Object.defineProperty||function(e,t,n){e[t]=n.value},i="function"==typeof Symbol?Symbol:{},o=i.iterator||"@@iterator",a=i.asyncIterator||"@@asyncIterator",u=i.toStringTag||"@@toStringTag";function l(e,t,n){return Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}),e[t]}try{l({},"")}catch(S){l=function(e,t,n){return e[t]=n}}function c(e,t,n,i){var o=t&&t.prototype instanceof d?t:d,a=Object.create(o.prototype),u=new C(i||[]);return r(a,"_invoke",{value:D(e,n,u)}),a}function s(e,t,n){try{return{type:"normal",arg:e.call(t,n)}}catch(S){return{type:"throw",arg:S}}}e.wrap=c;var f={};function d(){}function h(){}function p(){}var v={};l(v,o,(function(){return this}));var m=Object.getPrototypeOf,g=m&&m(m(A([])));g&&g!==t&&n.call(g,o)&&(v=g);var y=p.prototype=d.prototype=Object.create(v);function _(e){["next","throw","return"].forEach((function(t){l(e,t,(function(e){return this._invoke(t,e)}))}))}function b(e,t){function i(r,o,a,u){var l=s(e[r],e,o);if("throw"!==l.type){var c=l.arg,f=c.value;return f&&"object"==Lt(f)&&n.call(f,"__await")?t.resolve(f.__await).then((function(e){i("next",e,a,u)}),(function(e){i("throw",e,a,u)})):t.resolve(f).then((function(e){c.value=e,a(c)}),(function(e){return i("throw",e,a,u)}))}u(l.arg)}var o;r(this,"_invoke",{value:function(e,n){function r(){return new t((function(t,r){i(e,n,t,r)}))}return o=o?o.then(r,r):r()}})}function D(e,t,n){var r="suspendedStart";return function(i,o){if("executing"===r)throw new Error("Generator is already running");if("completed"===r){if("throw"===i)throw o;return E()}for(n.method=i,n.arg=o;;){var a=n.delegate;if(a){var u=w(a,n);if(u){if(u===f)continue;return u}}if("next"===n.method)n.sent=n._sent=n.arg;else if("throw"===n.method){if("suspendedStart"===r)throw r="completed",n.arg;n.dispatchException(n.arg)}else"return"===n.method&&n.abrupt("return",n.arg);r="executing";var l=s(e,t,n);if("normal"===l.type){if(r=n.done?"completed":"suspendedYield",l.arg===f)continue;return{value:l.arg,done:n.done}}"throw"===l.type&&(r="completed",n.method="throw",n.arg=l.arg)}}}function w(e,t){var n=t.method,r=e.iterator[n];if(void 0===r)return t.delegate=null,"throw"===n&&e.iterator.return&&(t.method="return",t.arg=void 0,w(e,t),"throw"===t.method)||"return"!==n&&(t.method="throw",t.arg=new TypeError("The iterator does not provide a '"+n+"' method")),f;var i=s(r,e.iterator,t.arg);if("throw"===i.type)return t.method="throw",t.arg=i.arg,t.delegate=null,f;var o=i.arg;return o?o.done?(t[e.resultName]=o.value,t.next=e.nextLoc,"return"!==t.method&&(t.method="next",t.arg=void 0),t.delegate=null,f):o:(t.method="throw",t.arg=new TypeError("iterator result is not an object"),t.delegate=null,f)}function x(e){var t={tryLoc:e[0]};1 in e&&(t.catchLoc=e[1]),2 in e&&(t.finallyLoc=e[2],t.afterLoc=e[3]),this.tryEntries.push(t)}function k(e){var t=e.completion||{};t.type="normal",delete t.arg,e.completion=t}function C(e){this.tryEntries=[{tryLoc:"root"}],e.forEach(x,this),this.reset(!0)}function A(e){if(e){var t=e[o];if(t)return t.call(e);if("function"==typeof e.next)return e;if(!isNaN(e.length)){var r=-1,i=function t(){for(;++r=0;--i){var o=this.tryEntries[i],a=o.completion;if("root"===o.tryLoc)return r("end");if(o.tryLoc<=this.prev){var u=n.call(o,"catchLoc"),l=n.call(o,"finallyLoc");if(u&&l){if(this.prev=0;--r){var i=this.tryEntries[r];if(i.tryLoc<=this.prev&&n.call(i,"finallyLoc")&&this.prev=0;--t){var n=this.tryEntries[t];if(n.finallyLoc===e)return this.complete(n.completion,n.afterLoc),k(n),f}},catch:function(e){for(var t=this.tryEntries.length-1;t>=0;--t){var n=this.tryEntries[t];if(n.tryLoc===e){var r=n.completion;if("throw"===r.type){var i=r.arg;k(n)}return i}}throw new Error("illegal catch attempt")},delegateYield:function(e,t,n){return this.delegate={iterator:A(e),resultName:t,nextLoc:n},"next"===this.method&&(this.arg=void 0),f}},e}function eu(e,t,n,r,i,o,a){try{var u=e[o](a),l=u.value}catch(c){return void n(c)}u.done?t(l):Promise.resolve(l).then(r,i)}function tu(e){return function(){var t=this,n=arguments;return new Promise((function(r,i){var o=e.apply(t,n);function a(e){eu(o,r,i,a,u,"next",e)}function u(e){eu(o,r,i,a,u,"throw",e)}a(void 0)}))}}var nu,ru,iu=function(e){var t=e.displaySidebar,n=e.isMobile,r=e.headerSetup,i=e.accountIds;return Vr("div",{className:wo()({"vm-header-controls":!0,"vm-header-controls_mobile":n}),children:[(null===r||void 0===r?void 0:r.tenant)&&Vr(ka,{accountIds:i||[]}),(null===r||void 0===r?void 0:r.stepControl)&&Vr(Ea,{}),(null===r||void 0===r?void 0:r.timeSelector)&&Vr(Ra,{}),(null===r||void 0===r?void 0:r.cardinalityDatePicker)&&Vr(za,{}),(null===r||void 0===r?void 0:r.executionControls)&&Vr($a,{}),Vr(Ka,{}),!t&&Vr(ba,{})]})},ou=function(e){var t=Ar(),n=Ft(ee(!1),2),r=n[0],i=n[1],o=jn().pathname,a=function(){var e=Cr().useTenantID,t=Wr().serverUrl,n=Ft(ee(!1),2),r=n[0],i=n[1],o=Ft(ee(),2),a=o[0],u=o[1],l=Ft(ee([]),2),c=l[0],s=l[1],f=ae((function(){return"".concat(t.replace(/^(.+)(\/select.+)/,"$1"),"/admin/tenants")}),[t]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return i(!0),e.prev=1,e.next=4,fetch(f);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],s(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?u(void 0):u("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&u("".concat(e.t0.name,": ").concat(e.t0.message));case 16:i(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[f]),{accountIds:c,isLoading:r,error:a}}(),u=a.accountIds,l=ae((function(){return(xr[o]||{}).header||{}}),[o]);return e.isMobile?Vr(g,{children:[Vr("div",{children:Vr(fa,{className:wo()({"vm-header-button":!t}),startIcon:Vr(yo,{}),onClick:function(){i((function(e){return!e}))}})}),Vr(va,{title:"Controls",onClose:function(){i(!1)},isOpen:r,className:wo()({"vm-header-controls-modal":!0,"vm-header-controls-modal_open":r}),children:Vr(iu,yr(yr({},e),{},{accountIds:u,headerSetup:l}))})]}):Vr(iu,yr(yr({},e),{},{accountIds:u,headerSetup:l}))},au=function(){var e=Xo().isMobile,t=ae((function(){return window.innerWidth<1e3}),[bo(document.body)]),n=Wr().isDarkTheme,r=Ar(),i=ae((function(){return Rr(n?"color-background-block":"color-primary")}),[n]),o=ae((function(){var e=Cr().headerStyles,t=void 0===e?{}:e,n=t.background,o=void 0===n?r?"#FFF":i:n,a=t.color;return{background:o,color:void 0===a?r?i:"#FFF":a}}),[i]),a=o.background,u=o.color,l=$n(),c=function(){l({pathname:kr.home}),window.location.reload()};return Vr("header",{className:wo()({"vm-header":!0,"vm-header_app":r,"vm-header_dark":n,"vm-header_mobile":e}),style:{background:a,color:u},children:[t?Vr(wa,{background:a,color:u}):Vr(g,{children:[!r&&Vr("div",{className:"vm-header-logo",onClick:c,style:{color:u},children:Vr(Pi,{})}),Vr(pa,{color:u,background:a})]}),e&&Vr("div",{className:"vm-header-logo vm-header-logo_mobile",onClick:c,style:{color:u},children:Vr(Pi,{})}),Vr(ou,{displaySidebar:t,isMobile:e})]})},uu=function(){var e=Xo().isMobile,t="2019-".concat(_t()().format("YYYY"));return Vr("footer",{className:"vm-footer",children:[Vr("a",{className:"vm-link vm-footer__website",target:"_blank",href:"https://victoriametrics.com/",rel:"me noreferrer",children:[Vr(Ri,{}),"victoriametrics.com"]}),Vr("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/#vmui",rel:"help noreferrer",children:[Vr(po,{}),e?"Docs":"Documentation"]}),Vr("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://github.com/VictoriaMetrics/VictoriaMetrics/issues/new/choose",rel:"noreferrer",children:[Vr(vo,{}),e?"New issue":"Create an issue"]}),Vr("div",{className:"vm-footer__copyright",children:["\xa9 ",t," VictoriaMetrics"]})]})},lu=function(){var e=tu(Xa().mark((function e(t){var n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,fetch("./dashboards/".concat(t));case 2:return n=e.sent,e.next=5,n.json();case 5:return r=e.sent,e.abrupt("return",r);case 7:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}(),cu=function(){var e=Ar(),t=Wr().serverUrl,n=le(aa).dispatch,r=Ft(ee(!1),2),i=r[0],o=r[1],a=Ft(ee(""),2),u=a[0],l=a[1],c=Ft(ee([]),2),s=c[0],f=c[1],d=function(){var e=tu(Xa().mark((function e(){var t,n;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:if(null!==(t=window.__VMUI_PREDEFINED_DASHBOARDS__)&&void 0!==t&&t.length){e.next=3;break}return e.abrupt("return",[]);case 3:return e.next=5,Promise.all(t.map(function(){var e=tu(Xa().mark((function e(t){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.abrupt("return",lu(t));case 1:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}()));case 5:n=e.sent,f((function(e){return[].concat(Ot(n),Ot(e))}));case 7:case"end":return e.stop()}}),e)})));return function(){return e.apply(this,arguments)}}(),h=function(){var e=tu(Xa().mark((function e(){var n,r,i;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:if(t){e.next=2;break}return e.abrupt("return");case 2:return l(""),o(!0),e.prev=4,e.next=7,fetch("".concat(t,"/vmui/custom-dashboards"));case 7:return n=e.sent,e.next=10,n.json();case 10:if(r=e.sent,!n.ok){e.next=17;break}(i=r.dashboardsSettings)&&i.length>0&&f((function(e){return[].concat(Ot(e),Ot(i))})),o(!1),e.next=21;break;case 17:return e.next=19,d();case 19:l(r.error),o(!1);case 21:e.next=29;break;case 23:return e.prev=23,e.t0=e.catch(4),o(!1),e.t0 instanceof Error&&l("".concat(e.t0.name,": ").concat(e.t0.message)),e.next=29,d();case 29:case"end":return e.stop()}}),e,null,[[4,23]])})));return function(){return e.apply(this,arguments)}}();return ne((function(){e||(f([]),h())}),[t]),ne((function(){n({type:"SET_DASHBOARDS_SETTINGS",payload:s})}),[s]),ne((function(){n({type:"SET_DASHBOARDS_LOADING",payload:i})}),[i]),ne((function(){n({type:"SET_DASHBOARDS_ERROR",payload:u})}),[u]),{dashboardsSettings:s,isLoading:i,error:u}},su=function(){var e=Ar(),t=Xo().isMobile,n=jn().pathname,r=Ft(pr(),2),i=r[0],o=r[1];cu();return ne((function(){var e,t="vmui",r=null===(e=xr[n])||void 0===e?void 0:e.title;document.title=r?"".concat(r," - ").concat(t):t}),[n]),ne((function(){var e=window.location.search;if(e){var t=Nr().parse(e,{ignoreQueryPrefix:!0});Object.entries(t).forEach((function(e){var t=Ft(e,2),n=t[0],r=t[1];i.set(n,r),o(i)})),window.location.search=""}window.location.replace(window.location.href.replace(/\/\?#\//,"/#/"))}),[]),Vr("section",{className:"vm-container",children:[Vr(au,{}),Vr("div",{className:wo()({"vm-container-body":!0,"vm-container-body_mobile":t,"vm-container-body_app":e}),children:Vr(Xn,{})}),!e&&Vr(uu,{})]})},fu="u-off",du="u-label",hu="width",pu="height",vu="top",mu="bottom",gu="left",yu="right",_u="#000",bu=_u+"0",Du="mousemove",wu="mousedown",xu="mouseup",ku="mouseenter",Cu="mouseleave",Au="dblclick",Eu="change",Su="dppxchange",Nu="undefined"!=typeof window,Fu=Nu?document:null,Mu=Nu?window:null,Tu=Nu?navigator:null;function Ou(e,t){if(null!=t){var n=e.classList;!n.contains(t)&&n.add(t)}}function Bu(e,t){var n=e.classList;n.contains(t)&&n.remove(t)}function Lu(e,t,n){e.style[t]=n+"px"}function Iu(e,t,n,r){var i=Fu.createElement(e);return null!=t&&Ou(i,t),null!=n&&n.insertBefore(i,r),i}function Pu(e,t){return Iu("div",e,t)}var Ru=new WeakMap;function zu(e,t,n,r,i){var o="translate("+t+"px,"+n+"px)";o!=Ru.get(e)&&(e.style.transform=o,Ru.set(e,o),t<0||n<0||t>r||n>i?Ou(e,fu):Bu(e,fu))}var ju=new WeakMap;function $u(e,t,n){var r=t+n;r!=ju.get(e)&&(ju.set(e,r),e.style.background=t,e.style.borderColor=n)}var Hu=new WeakMap;function Uu(e,t,n,r){var i=t+""+n;i!=Hu.get(e)&&(Hu.set(e,i),e.style.height=n+"px",e.style.width=t+"px",e.style.marginLeft=r?-t/2+"px":0,e.style.marginTop=r?-n/2+"px":0)}var Yu={passive:!0},Vu=yr(yr({},Yu),{},{capture:!0});function qu(e,t,n,r){t.addEventListener(e,n,r?Vu:Yu)}function Wu(e,t,n,r){t.removeEventListener(e,n,r?Vu:Yu)}function Qu(e,t,n,r){var i;n=n||0;for(var o=(r=r||t.length-1)<=2147483647;r-n>1;)t[i=o?n+r>>1:fl((n+r)/2)]=t&&i<=n;i+=r)if(null!=e[i])return i;return-1}function Ju(e,t,n,r){var i=Dl,o=-Dl;if(1==r)i=e[t],o=e[n];else if(-1==r)i=e[n],o=e[t];else for(var a=t;a<=n;a++)null!=e[a]&&(i=pl(i,e[a]),o=vl(o,e[a]));return[i,o]}function Zu(e,t,n){for(var r=Dl,i=-Dl,o=t;o<=n;o++)e[o]>0&&(r=pl(r,e[o]),i=vl(i,e[o]));return[r==Dl?1:r,i==-Dl?10:i]}function Ku(e,t,n,r){var i=gl(e),o=gl(t),a=10==n?yl:_l;e==t&&(-1==i?(e*=n,t/=n):(e/=n,t*=n));var u=1==o?hl:fl,l=(1==i?fl:hl)(a(sl(e))),c=u(a(sl(t))),s=ml(n,l),f=ml(n,c);return l<0&&(s=Ol(s,-l)),c<0&&(f=Ol(f,-c)),r?(e=s*i,t=f*o):(e=Tl(e,s),t=Ml(t,f)),[e,t]}function Xu(e,t,n,r){var i=Ku(e,t,n,r);return 0==e&&(i[0]=0),0==t&&(i[1]=0),i}Nu&&function e(){var t=devicePixelRatio;nu!=t&&(nu=t,ru&&Wu(Eu,ru,e),ru=matchMedia("(min-resolution: ".concat(nu-.001,"dppx) and (max-resolution: ").concat(nu+.001,"dppx)")),qu(Eu,ru,e),Mu.dispatchEvent(new CustomEvent(Su)))}();var el={mode:3,pad:.1},tl={pad:0,soft:null,mode:0},nl={min:tl,max:tl};function rl(e,t,n,r){return Ul(n)?ol(e,t,n):(tl.pad=n,tl.soft=r?0:null,tl.mode=r?3:0,ol(e,t,nl))}function il(e,t){return null==e?t:e}function ol(e,t,n){var r=n.min,i=n.max,o=il(r.pad,0),a=il(i.pad,0),u=il(r.hard,-Dl),l=il(i.hard,Dl),c=il(r.soft,Dl),s=il(i.soft,-Dl),f=il(r.mode,0),d=il(i.mode,0),h=t-e,p=yl(h),v=vl(sl(e),sl(t)),m=yl(v),g=sl(m-p);(h<1e-9||g>10)&&(h=0,0!=e&&0!=t||(h=1e-9,2==f&&c!=Dl&&(o=0),2==d&&s!=-Dl&&(a=0)));var y=h||v||1e3,_=yl(y),b=ml(10,fl(_)),D=Ol(Tl(e-y*(0==h?0==e?.1:1:o),b/10),9),w=e>=c&&(1==f||3==f&&D<=c||2==f&&D>=c)?c:Dl,x=vl(u,D=w?w:pl(w,D)),k=Ol(Ml(t+y*(0==h?0==t?.1:1:a),b/10),9),C=t<=s&&(1==d||3==d&&k>=s||2==d&&k<=s)?s:-Dl,A=pl(l,k>C&&t<=C?C:vl(C,k));return x==A&&0==x&&(A=100),[x,A]}var al=new Intl.NumberFormat(Nu?Tu.language:"en-US"),ul=function(e){return al.format(e)},ll=Math,cl=ll.PI,sl=ll.abs,fl=ll.floor,dl=ll.round,hl=ll.ceil,pl=ll.min,vl=ll.max,ml=ll.pow,gl=ll.sign,yl=ll.log10,_l=ll.log2,bl=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return ll.asinh(e/t)},Dl=1/0;function wl(e){return 1+(0|yl((e^e>>31)-(e>>31)))}function xl(e,t){return dl(e/t)*t}function kl(e,t,n){return pl(vl(e,t),n)}function Cl(e){return"function"==typeof e?e:function(){return e}}var Al=function(e){return e},El=function(e,t){return t},Sl=function(e){return null},Nl=function(e){return!0},Fl=function(e,t){return e==t};function Ml(e,t){return hl(e/t)*t}function Tl(e,t){return fl(e/t)*t}function Ol(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0;if($l(e))return e;var n=Math.pow(10,t),r=e*n*(1+Number.EPSILON);return dl(r)/n}var Bl=new Map;function Ll(e){return((""+e).split(".")[1]||"").length}function Il(e,t,n,r){for(var i=[],o=r.map(Ll),a=t;a=0&&a>=0?0:u)+(a>=o[c]?0:o[c]),d=Ol(s,f);i.push(d),Bl.set(d,f)}return i}var Pl={},Rl=[],zl=[null,null],jl=Array.isArray,$l=Number.isInteger;function Hl(e){return"string"==typeof e}function Ul(e){var t=!1;if(null!=e){var n=e.constructor;t=null==n||n==Object}return t}function Yl(e){return null!=e&&"object"==typeof e}var Vl=Object.getPrototypeOf(Uint8Array);function ql(e){var t,n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:Ul;if(jl(e)){var r=e.find((function(e){return null!=e}));if(jl(r)||n(r)){t=Array(e.length);for(var i=0;io){for(r=a-1;r>=0&&null==e[r];)e[r--]=null;for(r=a+1;r12?t-12:t},AA:function(e){return e.getHours()>=12?"PM":"AM"},aa:function(e){return e.getHours()>=12?"pm":"am"},a:function(e){return e.getHours()>=12?"p":"a"},mm:function(e){return nc(e.getMinutes())},m:function(e){return e.getMinutes()},ss:function(e){return nc(e.getSeconds())},s:function(e){return e.getSeconds()},fff:function(e){return((t=e.getMilliseconds())<10?"00":t<100?"0":"")+t;var t}};function ic(e,t){t=t||tc;for(var n,r=[],i=/\{([a-z]+)\}|[^{]+/gi;n=i.exec(e);)r.push("{"==n[0][0]?rc[n[1]]:n[0]);return function(e){for(var n="",i=0;i=a,v=f>=o&&f=i?i:f,F=_+(fl(c)-fl(g))+Ml(g-_,N);h.push(F);for(var M=t(F),T=M.getHours()+M.getMinutes()/n+M.getSeconds()/r,O=f/r,B=d/u.axes[l]._space;!((F=Ol(F+f,1==e?0:3))>s);)if(O>1){var L=fl(Ol(T+O,6))%24,I=t(F).getHours()-L;I>1&&(I=-1),T=(T+O)%24,Ol(((F-=I*r)-h[h.length-1])/f,3)*B>=.7&&h.push(F)}else h.push(F)}return h}}]}var xc=Ft(wc(1),3),kc=xc[0],Cc=xc[1],Ac=xc[2],Ec=Ft(wc(.001),3),Sc=Ec[0],Nc=Ec[1],Fc=Ec[2];function Mc(e,t){return e.map((function(e){return e.map((function(n,r){return 0==r||8==r||null==n?n:t(1==r||0==e[8]?n:e[1]+n)}))}))}function Tc(e,t){return function(n,r,i,o,a){var u,l,c,s,f,d,h=t.find((function(e){return a>=e[0]}))||t[t.length-1];return r.map((function(t){var n=e(t),r=n.getFullYear(),i=n.getMonth(),o=n.getDate(),a=n.getHours(),p=n.getMinutes(),v=n.getSeconds(),m=r!=u&&h[2]||i!=l&&h[3]||o!=c&&h[4]||a!=s&&h[5]||p!=f&&h[6]||v!=d&&h[7]||h[1];return u=r,l=i,c=o,s=a,f=p,d=v,m(n)}))}}function Oc(e,t,n){return new Date(e,t,n)}function Bc(e,t){return t(e)}Il(2,-53,53,[1]);function Lc(e,t){return function(n,r){return t(e(r))}}var Ic={show:!0,live:!0,isolate:!1,mount:function(){},markers:{show:!0,width:2,stroke:function(e,t){var n=e.series[t];return n.width?n.stroke(e,t):n.points.width?n.points.stroke(e,t):null},fill:function(e,t){return e.series[t].fill(e,t)},dash:"solid"},idx:null,idxs:null,values:[]};var Pc=[0,0];function Rc(e,t,n){return function(e){0==e.button&&n(e)}}function zc(e,t,n){return n}var jc={show:!0,x:!0,y:!0,lock:!1,move:function(e,t,n){return Pc[0]=t,Pc[1]=n,Pc},points:{show:function(e,t){var n=e.cursor.points,r=Pu(),i=n.size(e,t);Lu(r,hu,i),Lu(r,pu,i);var o=i/-2;Lu(r,"marginLeft",o),Lu(r,"marginTop",o);var a=n.width(e,t,i);return a&&Lu(r,"borderWidth",a),r},size:function(e,t){return as(e.series[t].points.width,1)},width:0,stroke:function(e,t){var n=e.series[t].points;return n._stroke||n._fill},fill:function(e,t){var n=e.series[t].points;return n._fill||n._stroke}},bind:{mousedown:Rc,mouseup:Rc,click:Rc,dblclick:Rc,mousemove:zc,mouseleave:zc,mouseenter:zc},drag:{setScale:!0,x:!0,y:!1,dist:0,uni:null,_x:!1,_y:!1},focus:{prox:-1},left:-10,top:-10,idx:null,dataIdx:function(e,t,n){return n},idxs:null},$c={show:!0,stroke:"rgba(0,0,0,0.07)",width:2},Hc=Wl({},$c,{filter:El}),Uc=Wl({},Hc,{size:10}),Yc=Wl({},$c,{show:!1}),Vc='12px system-ui, -apple-system, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"',qc="bold "+Vc,Wc={show:!0,scale:"x",stroke:_u,space:50,gap:5,size:50,labelGap:0,labelSize:30,labelFont:qc,side:2,grid:Hc,ticks:Uc,border:Yc,font:Vc,rotate:0},Qc={show:!0,scale:"x",auto:!1,sorted:1,min:Dl,max:-Dl,idxs:[]};function Gc(e,t,n,r,i){return t.map((function(e){return null==e?"":ul(e)}))}function Jc(e,t,n,r,i,o,a){for(var u=[],l=Bl.get(i)||0,c=n=a?n:Ol(Ml(n,i),l);c<=r;c=Ol(c+i,l))u.push(Object.is(c,-0)?0:c);return u}function Zc(e,t,n,r,i,o,a){var u=[],l=e.scales[e.axes[t].scale].log,c=fl((10==l?yl:_l)(n));i=ml(l,c),c<0&&(i=Ol(i,-c));var s=n;do{u.push(s),(s=Ol(s+i,Bl.get(i)))>=i*l&&(i=s)}while(s<=r);return u}function Kc(e,t,n,r,i,o,a){var u=e.scales[e.axes[t].scale].asinh,l=r>u?Zc(e,t,vl(u,n),r,i):[u],c=r>=0&&n<=0?[0]:[];return(n<-u?Zc(e,t,vl(u,-r),-n,i):[u]).reverse().map((function(e){return-e})).concat(c,l)}var Xc=/./,es=/[12357]/,ts=/[125]/,ns=/1/;function rs(e,t,n,r,i){var o=e.axes[n],a=o.scale,u=e.scales[a];if(3==u.distr&&2==u.log)return t;var l=e.valToPos,c=o._space,s=l(10,a),f=l(9,a)-s>=c?Xc:l(7,a)-s>=c?es:l(5,a)-s>=c?ts:ns;return t.map((function(e){return 4==u.distr&&0==e||f.test(e)?e:null}))}function is(e,t){return null==t?"":ul(t)}var os={show:!0,scale:"y",stroke:_u,space:30,gap:5,size:50,labelGap:0,labelSize:30,labelFont:qc,side:3,grid:Hc,ticks:Uc,border:Yc,font:Vc,rotate:0};function as(e,t){return Ol((3+2*(e||1))*t,3)}var us={scale:null,auto:!0,sorted:0,min:Dl,max:-Dl},ls=function(e,t,n,r,i){return i},cs={show:!0,auto:!0,sorted:0,gaps:ls,alpha:1,facets:[Wl({},us,{scale:"x"}),Wl({},us,{scale:"y"})]},ss={scale:"y",auto:!0,sorted:0,show:!0,spanGaps:!1,gaps:ls,alpha:1,points:{show:function(e,t){var n=e.series[0],r=n.scale,i=n.idxs,o=e._data[0],a=e.valToPos(o[i[0]],r,!0),u=e.valToPos(o[i[1]],r,!0),l=sl(u-a)/(e.series[t].points.space*nu);return i[1]-i[0]<=l},filter:null},values:null,min:Dl,max:-Dl,idxs:[],path:null,clip:null};function fs(e,t,n,r,i){return n/10}var ds={time:!0,auto:!0,distr:1,log:10,asinh:1,min:null,max:null,dir:1,ori:0},hs=Wl({},ds,{time:!1,ori:1}),ps={};function vs(e,t){var n=ps[e];return n||(n={key:e,plots:[],sub:function(e){n.plots.push(e)},unsub:function(e){n.plots=n.plots.filter((function(t){return t!=e}))},pub:function(e,t,r,i,o,a,u){for(var l=0;l0){a=new Path2D;for(var u=0==t?Ss:Ns,l=n,c=0;cs[0]){var f=s[0]-l;f>0&&u(a,l,r,f,r+o),l=s[1]}}var d=n+i-l;d>0&&u(a,l,r,d,r+o)}return a}function Ds(e,t,n,r,i,o,a){for(var u=[],l=e.length,c=1==i?n:r;c>=n&&c<=r;c+=i){if(null===t[c]){var s=c,f=c;if(1==i)for(;++c<=r&&null===t[c];)f=c;else for(;--c>=n&&null===t[c];)f=c;var d=o(e[s]),h=f==s?d:o(e[f]),p=s-i;d=a<=0&&p>=0&&p=0&&v>=0&&v=d&&u.push([d,h])}}return u}function ws(e){return 0==e?Al:1==e?dl:function(t){return xl(t,e)}}function xs(e){var t=0==e?ks:Cs,n=0==e?function(e,t,n,r,i,o){e.arcTo(t,n,r,i,o)}:function(e,t,n,r,i,o){e.arcTo(n,t,i,r,o)},r=0==e?function(e,t,n,r,i){e.rect(t,n,r,i)}:function(e,t,n,r,i){e.rect(n,t,i,r)};return function(e,i,o,a,u){var l=arguments.length>5&&void 0!==arguments[5]?arguments[5]:0;0==l?r(e,i,o,a,u):(l=pl(l,a/2,u/2),t(e,i+l,o),n(e,i+a,o,i+a,o+u,l),n(e,i+a,o+u,i,o+u,l),n(e,i,o+u,i,o,l),n(e,i,o,i+a,o,l),e.closePath())}}var ks=function(e,t,n){e.moveTo(t,n)},Cs=function(e,t,n){e.moveTo(n,t)},As=function(e,t,n){e.lineTo(t,n)},Es=function(e,t,n){e.lineTo(n,t)},Ss=xs(0),Ns=xs(1),Fs=function(e,t,n,r,i,o){e.arc(t,n,r,i,o)},Ms=function(e,t,n,r,i,o){e.arc(n,t,r,i,o)},Ts=function(e,t,n,r,i,o,a){e.bezierCurveTo(t,n,r,i,o,a)},Os=function(e,t,n,r,i,o,a){e.bezierCurveTo(n,t,i,r,a,o)};function Bs(e){return function(e,t,n,r,i){return ms(e,t,(function(t,o,a,u,l,c,s,f,d,h,p){var v,m,g=t.pxRound,y=t.points;0==u.ori?(v=ks,m=Fs):(v=Cs,m=Ms);var _=Ol(y.width*nu,3),b=(y.size-y.width)/2*nu,D=Ol(2*b,3),w=new Path2D,x=new Path2D,k=e.bbox,C=k.left,A=k.top,E=k.width,S=k.height;Ss(x,C-D,A-D,E+2*D,S+2*D);var N=function(e){if(null!=a[e]){var t=g(c(o[e],u,h,f)),n=g(s(a[e],l,p,d));v(w,t+b,n),m(w,t,n,b,0,2*cl)}};if(i)i.forEach(N);else for(var F=n;F<=r;F++)N(F);return{stroke:_>0?w:null,fill:w,clip:x,flags:3}}))}}function Ls(e){return function(t,n,r,i,o,a){r!=i&&(o!=r&&a!=r&&e(t,n,r),o!=i&&a!=i&&e(t,n,i),e(t,n,a))}}var Is=Ls(As),Ps=Ls(Es);function Rs(e){var t=il(null===e||void 0===e?void 0:e.alignGaps,0);return function(e,n,r,i){return ms(e,n,(function(o,a,u,l,c,s,f,d,h,p,v){var m,g,y=o.pxRound,_=function(e){return y(s(e,l,p,d))},b=function(e){return y(f(e,c,v,h))};0==l.ori?(m=As,g=Is):(m=Es,g=Ps);for(var D,w,x,k=l.dir*(0==l.ori?1:-1),C={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:1},A=C.stroke,E=Dl,S=-Dl,N=_(a[1==k?r:i]),F=Gu(u,r,i,1*k),M=Gu(u,r,i,-1*k),T=_(a[F]),O=_(a[M]),B=1==k?r:i;B>=r&&B<=i;B+=k){var L=_(a[B]);L==N?null!=u[B]&&(w=b(u[B]),E==Dl&&(m(A,L,w),D=w),E=pl(w,E),S=vl(w,S)):(E!=Dl&&(g(A,N,E,S,D,w),x=N),null!=u[B]?(m(A,L,w=b(u[B])),E=S=D=w):(E=Dl,S=-Dl),N=L)}E!=Dl&&E!=S&&x!=N&&g(A,N,E,S,D,w);var I=Ft(gs(e,n),2),P=I[0],R=I[1];if(null!=o.fill||0!=P){var z=C.fill=new Path2D(A),j=b(o.fillTo(e,n,o.min,o.max,P));m(z,O,j),m(z,T,j)}if(!o.spanGaps){var $,H=[];($=H).push.apply($,Ot(Ds(a,u,r,i,k,_,t))),C.gaps=H=o.gaps(e,n,r,i,H),C.clip=bs(H,l.ori,d,h,p,v)}return 0!=R&&(C.band=2==R?[_s(e,n,r,i,A,-1),_s(e,n,r,i,A,1)]:_s(e,n,r,i,A,R)),C}))}}function zs(e,t,n,r,i,o){var a=e.length;if(a<2)return null;var u=new Path2D;if(n(u,e[0],t[0]),2==a)r(u,e[1],t[1]);else{for(var l=Array(a),c=Array(a-1),s=Array(a-1),f=Array(a-1),d=0;d0!==c[h]>0?l[h]=0:(l[h]=3*(f[h-1]+f[h])/((2*f[h]+f[h-1])/c[h-1]+(f[h]+2*f[h-1])/c[h]),isFinite(l[h])||(l[h]=0));l[a-1]=c[a-2];for(var p=0;p=i&&o+(l<5?Bl.get(l):0)<=17)return[l,c]}while(++u0?e:t.clamp(r,e,t.min,t.max,t.key)):4==t.distr?bl(e,t.asinh):e)-t._min)/(t._max-t._min)}function a(e,t,n,r){var i=o(e,t);return r+n*(-1==t.dir?1-i:i)}function u(e,t,n,r){var i=o(e,t);return r+n*(-1==t.dir?i:1-i)}function l(e,t,n,r){return 0==t.ori?a(e,t,n,r):u(e,t,n,r)}r.valToPosH=a,r.valToPosV=u;var c=!1;r.status=0;var s=r.root=Pu("uplot");(null!=e.id&&(s.id=e.id),Ou(s,e.class),e.title)&&(Pu("u-title",s).textContent=e.title);var f=Iu("canvas"),d=r.ctx=f.getContext("2d"),h=Pu("u-wrap",s),p=r.under=Pu("u-under",h);h.appendChild(f);var v=r.over=Pu("u-over",h),m=+il((e=ql(e)).pxAlign,1),g=ws(m);(e.plugins||[]).forEach((function(t){t.opts&&(e=t.opts(r,e)||e)}));var y=e.ms||.001,_=r.series=1==i?Ys(e.series||[],Qc,ss,!1):function(e,t){return e.map((function(e,n){return 0==n?null:Wl({},t,e)}))}(e.series||[null],cs),b=r.axes=Ys(e.axes||[],Wc,os,!0),D=r.scales={},w=r.bands=e.bands||[];w.forEach((function(e){e.fill=Cl(e.fill||null),e.dir=il(e.dir,-1)}));var x=2==i?_[1].facets[0].scale:_[0].scale,k={axes:function(){for(var e=function(){var e=b[t];if(!e.show||!e._show)return"continue";var n,i,o=e.side,a=o%2,u=e.stroke(r,t),c=0==o||3==o?-1:1;if(e.label){var s=e.labelGap*c,f=dl((e._lpos+s)*nu);Ke(e.labelFont[0],u,"center",2==o?vu:mu),d.save(),1==a?(n=i=0,d.translate(f,dl(de+pe/2)),d.rotate((3==o?-cl:cl)/2)):(n=dl(fe+he/2),i=f),d.fillText(e.label,n,i),d.restore()}var h=Ft(e._found,2),p=h[0],v=h[1];if(0==v)return"continue";var m=D[e.scale],y=0==a?he:pe,_=0==a?fe:de,w=dl(e.gap*nu),x=e._splits,k=2==m.distr?x.map((function(e){return We[e]})):x,C=2==m.distr?We[x[1]]-We[x[0]]:p,A=e.ticks,E=e.border,S=A.show?dl(A.size*nu):0,N=e._rotate*-cl/180,F=g(e._pos*nu),M=F+(S+w)*c;i=0==a?M:0,n=1==a?M:0,Ke(e.font[0],u,1==e.align?gu:2==e.align?yu:N>0?gu:N<0?yu:0==a?"center":3==o?yu:gu,N||1==a?"middle":2==o?vu:mu);for(var T=1.5*e.font[1],O=x.map((function(e){return g(l(e,m,y,_))})),B=e._values,L=0;L0&&(_.forEach((function(e,n){if(n>0&&e.show&&null==e._paths){var o=2==i?[0,t[n][0].length-1]:function(e){var t=kl(Ye-1,0,Me-1),n=kl(Ve+1,0,Me-1);for(;null==e[t]&&t>0;)t--;for(;null==e[n]&&n0&&e.show){$e!=e.alpha&&(d.globalAlpha=$e=e.alpha),et(t,!1),e._paths&&tt(t,!1),et(t,!0);var n=e._paths?e._paths.gaps:null,i=e.points.show(r,t,Ye,Ve,n),o=e.points.filter(r,t,i,n);(i||o)&&(e.points._paths=e.points.paths(r,t,Ye,Ve,o),tt(t,!0)),1!=$e&&(d.globalAlpha=$e=1),an("drawSeries",t)}})))}},C=(e.drawOrder||["axes","series"]).map((function(e){return k[e]}));function A(t){var n=D[t];if(null==n){var r=(e.scales||Pl)[t]||Pl;if(null!=r.from)A(r.from),D[t]=Wl({},D[r.from],r,{key:t});else{(n=D[t]=Wl({},t==x?ds:hs,r)).key=t;var o=n.time,a=n.range,u=jl(a);if((t!=x||2==i&&!o)&&(!u||null!=a[0]&&null!=a[1]||(a={min:null==a[0]?el:{mode:1,hard:a[0],soft:a[0]},max:null==a[1]?el:{mode:1,hard:a[1],soft:a[1]}},u=!1),!u&&Ul(a))){var l=a;a=function(e,t,n){return null==t?zl:rl(t,n,l)}}n.range=Cl(a||(o?Ws:t==x?3==n.distr?Js:4==n.distr?Ks:qs:3==n.distr?Gs:4==n.distr?Zs:Qs)),n.auto=Cl(!u&&n.auto),n.clamp=Cl(n.clamp||fs),n._min=n._max=null}}}for(var E in A("x"),A("y"),1==i&&_.forEach((function(e){A(e.scale)})),b.forEach((function(e){A(e.scale)})),e.scales)A(E);var S,N,F=D[x],M=F.distr;0==F.ori?(Ou(s,"u-hz"),S=a,N=u):(Ou(s,"u-vt"),S=u,N=a);var T={};for(var O in D){var B=D[O];null==B.min&&null==B.max||(T[O]={min:B.min,max:B.max},B.min=B.max=null)}var L,I=e.tzDate||function(e){return new Date(dl(e/y))},P=e.fmtDate||ic,R=1==y?Ac(I):Fc(I),z=Tc(I,Mc(1==y?Cc:Nc,P)),j=Lc(I,Bc("{YYYY}-{MM}-{DD} {h}:{mm}{aa}",P)),$=[],H=r.legend=Wl({},Ic,e.legend),U=H.show,Y=H.markers;H.idxs=$,Y.width=Cl(Y.width),Y.dash=Cl(Y.dash),Y.stroke=Cl(Y.stroke),Y.fill=Cl(Y.fill);var V,q=[],W=[],Q=!1,G={};if(H.live){var J=_[1]?_[1].values:null;for(var Z in V=(Q=null!=J)?J(r,1,0):{_:0})G[Z]="--"}if(U)if(L=Iu("table","u-legend",s),H.mount(r,L),Q){var K=Iu("tr","u-thead",L);for(var X in Iu("th",null,K),V)Iu("th",du,K).textContent=X}else Ou(L,"u-inline"),H.live&&Ou(L,"u-live");var ee={show:!0},te={show:!1};var ne=new Map;function re(e,t,n){var i=ne.get(t)||{},o=xe.bind[e](r,t,n);o&&(qu(e,t,i[e]=o),ne.set(t,i))}function ie(e,t,n){var r=ne.get(t)||{};for(var i in r)null!=e&&i!=e||(Wu(i,t,r[i]),delete r[i]);null==e&&ne.delete(t)}var oe=0,ae=0,ue=0,le=0,ce=0,se=0,fe=0,de=0,he=0,pe=0;r.bbox={};var ve=!1,me=!1,ge=!1,ye=!1,_e=!1,be=!1;function De(e,t,n){(n||e!=r.width||t!=r.height)&&we(e,t),lt(!1),ge=!0,me=!0,xe.left>=0&&(ye=be=!0),wt()}function we(e,t){r.width=oe=ue=e,r.height=ae=le=t,ce=se=0,function(){var e=!1,t=!1,n=!1,r=!1;b.forEach((function(i,o){if(i.show&&i._show){var a=i.side,u=a%2,l=i._size+(null!=i.label?i.labelSize:0);l>0&&(u?(ue-=l,3==a?(ce+=l,r=!0):n=!0):(le-=l,0==a?(se+=l,e=!0):t=!0))}})),Ne[0]=e,Ne[1]=n,Ne[2]=t,Ne[3]=r,ue-=Ue[1]+Ue[3],ce+=Ue[3],le-=Ue[2]+Ue[0],se+=Ue[0]}(),function(){var e=ce+ue,t=se+le,n=ce,r=se;function i(i,o){switch(i){case 1:return(e+=o)-o;case 2:return(t+=o)-o;case 3:return(n-=o)+o;case 0:return(r-=o)+o}}b.forEach((function(e,t){if(e.show&&e._show){var n=e.side;e._pos=i(n,e._size),null!=e.label&&(e._lpos=i(n,e.labelSize))}}))}();var n=r.bbox;fe=n.left=xl(ce*nu,.5),de=n.top=xl(se*nu,.5),he=n.width=xl(ue*nu,.5),pe=n.height=xl(le*nu,.5)}r.setSize=function(e){De(e.width,e.height)};var xe=r.cursor=Wl({},jc,{drag:{y:2==i}},e.cursor);xe.idxs=$,xe._lock=!1;var ke=xe.points;ke.show=Cl(ke.show),ke.size=Cl(ke.size),ke.stroke=Cl(ke.stroke),ke.width=Cl(ke.width),ke.fill=Cl(ke.fill);var Ce=r.focus=Wl({},e.focus||{alpha:.3},xe.focus),Ae=Ce.prox>=0,Ee=[null];function Se(e,t){if(1==i||t>0){var n=1==i&&D[e.scale].time,o=e.value;e.value=n?Hl(o)?Lc(I,Bc(o,P)):o||j:o||is,e.label=e.label||(n?"Time":"Value")}if(t>0){e.width=null==e.width?1:e.width,e.paths=e.paths||Hs||Sl,e.fillTo=Cl(e.fillTo||ys),e.pxAlign=+il(e.pxAlign,m),e.pxRound=ws(e.pxAlign),e.stroke=Cl(e.stroke||null),e.fill=Cl(e.fill||null),e._stroke=e._fill=e._paths=e._focus=null;var a=as(e.width,1),u=e.points=Wl({},{size:a,width:vl(1,.2*a),stroke:e.stroke,space:2*a,paths:Us,_stroke:null,_fill:null},e.points);u.show=Cl(u.show),u.filter=Cl(u.filter),u.fill=Cl(u.fill),u.stroke=Cl(u.stroke),u.paths=Cl(u.paths),u.pxAlign=e.pxAlign}if(U){var l=function(e,t){if(0==t&&(Q||!H.live||2==i))return zl;var n=[],o=Iu("tr","u-series",L,L.childNodes[t]);Ou(o,e.class),e.show||Ou(o,fu);var a=Iu("th",null,o);if(Y.show){var u=Pu("u-marker",a);if(t>0){var l=Y.width(r,t);l&&(u.style.border=l+"px "+Y.dash(r,t)+" "+Y.stroke(r,t)),u.style.background=Y.fill(r,t)}}var c=Pu(du,a);for(var s in c.textContent=e.label,t>0&&(Y.show||(c.style.color=e.width>0?Y.stroke(r,t):Y.fill(r,t)),re("click",a,(function(t){if(!xe._lock){var n=_.indexOf(e);if((t.ctrlKey||t.metaKey)!=H.isolate){var r=_.some((function(e,t){return t>0&&t!=n&&e.show}));_.forEach((function(e,t){t>0&&Pt(t,r?t==n?ee:te:ee,!0,un.setSeries)}))}else Pt(n,{show:!e.show},!0,un.setSeries)}})),Ae&&re(ku,a,(function(t){xe._lock||Pt(_.indexOf(e),Rt,!0,un.setSeries)}))),V){var f=Iu("td","u-value",o);f.textContent="--",n.push(f)}return[o,n]}(e,t);q.splice(t,0,l[0]),W.splice(t,0,l[1]),H.values.push(null)}if(xe.show){$.splice(t,0,null);var c=function(e,t){if(t>0){var n=xe.points.show(r,t);if(n)return Ou(n,"u-cursor-pt"),Ou(n,e.class),zu(n,-10,-10,ue,le),v.insertBefore(n,Ee[t]),n}}(e,t);c&&Ee.splice(t,0,c)}an("addSeries",t)}r.addSeries=function(e,t){t=null==t?_.length:t,e=1==i?Vs(e,t,Qc,ss):Vs(e,t,null,cs),_.splice(t,0,e),Se(_[t],t)},r.delSeries=function(e){if(_.splice(e,1),U){H.values.splice(e,1),W.splice(e,1);var t=q.splice(e,1)[0];ie(null,t.firstChild),t.remove()}xe.show&&($.splice(e,1),Ee.length>1&&Ee.splice(e,1)[0].remove()),an("delSeries",e)};var Ne=[!1,!1,!1,!1];function Fe(e,t,n,r){var i=Ft(n,4),o=i[0],a=i[1],u=i[2],l=i[3],c=t%2,s=0;return 0==c&&(l||a)&&(s=0==t&&!o||2==t&&!u?dl(Wc.size/3):0),1==c&&(o||u)&&(s=1==t&&!a||3==t&&!l?dl(os.size/2):0),s}var Me,Te,Oe,Be,Le,Ie,Pe,Re,ze,je,$e,He=r.padding=(e.padding||[Fe,Fe,Fe,Fe]).map((function(e){return Cl(il(e,Fe))})),Ue=r._padding=He.map((function(e,t){return e(r,t,Ne,0)})),Ye=null,Ve=null,qe=1==i?_[0].idxs:null,We=null,Qe=!1;function Ge(e,n){if(t=null==e?[]:ql(e,Yl),2==i){Me=0;for(var o=1;o<_.length;o++)Me+=t[o][0].length;r.data=t=e}else if(null==t[0]&&(t[0]=[]),r.data=t.slice(),We=t[0],Me=We.length,2==M){t[0]=Array(Me);for(var a=0;a=0,be=!0,wt()}}function Je(){var e,n;if(Qe=!0,1==i)if(Me>0){if(Ye=qe[0]=0,Ve=qe[1]=Me-1,e=t[0][Ye],n=t[0][Ve],2==M)e=Ye,n=Ve;else if(1==Me)if(3==M){var r=Ft(Ku(e,e,F.log,!1),2);e=r[0],n=r[1]}else if(4==M){var o=Ft(Xu(e,e,F.log,!1),2);e=o[0],n=o[1]}else if(F.time)n=e+dl(86400/y);else{var a=Ft(rl(e,n,.1,!0),2);e=a[0],n=a[1]}}else Ye=qe[0]=e=null,Ve=qe[1]=n=null;It(x,e,n)}function Ze(e,t,n,r,i,o){var a,u,l,c,s;null!==(a=e)&&void 0!==a||(e=bu),null!==(u=n)&&void 0!==u||(n=Rl),null!==(l=r)&&void 0!==l||(r="butt"),null!==(c=i)&&void 0!==c||(i=bu),null!==(s=o)&&void 0!==s||(o="round"),e!=Te&&(d.strokeStyle=Te=e),i!=Oe&&(d.fillStyle=Oe=i),t!=Be&&(d.lineWidth=Be=t),o!=Ie&&(d.lineJoin=Ie=o),r!=Pe&&(d.lineCap=Pe=r),n!=Le&&d.setLineDash(Le=n)}function Ke(e,t,n,r){t!=Oe&&(d.fillStyle=Oe=t),e!=Re&&(d.font=Re=e),n!=ze&&(d.textAlign=ze=n),r!=je&&(d.textBaseline=je=r)}function Xe(e,t,n,i){var o=arguments.length>4&&void 0!==arguments[4]?arguments[4]:0;if(i.length>0&&e.auto(r,Qe)&&(null==t||null==t.min)){var a=il(Ye,0),u=il(Ve,i.length-1),l=null==n.min?3==e.distr?Zu(i,a,u):Ju(i,a,u,o):[n.min,n.max];e.min=pl(e.min,n.min=l[0]),e.max=vl(e.max,n.max=l[1])}}function et(e,t){var n=t?_[e].points:_[e];n._stroke=n.stroke(r,e),n._fill=n.fill(r,e)}function tt(e,n){var i=n?_[e].points:_[e],o=i._stroke,a=i._fill,u=i._paths,l=u.stroke,c=u.fill,s=u.clip,f=u.flags,h=null,p=Ol(i.width*nu,3),v=p%2/2;n&&null==a&&(a=p>0?"#fff":o);var m=1==i.pxAlign;if(m&&d.translate(v,v),!n){var g=fe,y=de,b=he,D=pe,x=p*nu/2;0==i.min&&(D+=x),0==i.max&&(y-=x,D+=x),(h=new Path2D).rect(g,y,b,D)}n?nt(o,p,i.dash,i.cap,a,l,c,f,s):function(e,n,i,o,a,u,l,c,s,f,d){var h=!1;w.forEach((function(p,v){if(p.series[0]==e){var m,g=_[p.series[1]],y=t[p.series[1]],b=(g._paths||Pl).band;jl(b)&&(b=1==p.dir?b[0]:b[1]);var D=null;g.show&&b&&function(e,t,n){for(t=il(t,0),n=il(n,e.length-1);t<=n;){if(null!=e[t])return!0;t++}return!1}(y,Ye,Ve)?(D=p.fill(r,v)||u,m=g._paths.clip):b=null,nt(n,i,o,a,D,l,c,s,f,d,m,b),h=!0}})),h||nt(n,i,o,a,u,l,c,s,f,d)}(e,o,p,i.dash,i.cap,a,l,c,f,h,s),m&&d.translate(-v,-v)}r.setData=Ge;function nt(e,t,n,r,i,o,a,u,l,c,s,f){Ze(e,t,n,r,i),(l||c||f)&&(d.save(),l&&d.clip(l),c&&d.clip(c)),f?3==(3&u)?(d.clip(f),s&&d.clip(s),it(i,a),rt(e,o,t)):2&u?(it(i,a),d.clip(f),rt(e,o,t)):1&u&&(d.save(),d.clip(f),s&&d.clip(s),it(i,a),d.restore(),rt(e,o,t)):(it(i,a),rt(e,o,t)),(l||c||f)&&d.restore()}function rt(e,t,n){n>0&&(t instanceof Map?t.forEach((function(e,t){d.strokeStyle=Te=t,d.stroke(e)})):null!=t&&e&&d.stroke(t))}function it(e,t){t instanceof Map?t.forEach((function(e,t){d.fillStyle=Oe=t,d.fill(e)})):null!=t&&e&&d.fill(t)}function ot(e,t,n,r,i,o,a,u,l,c){var s=a%2/2;1==m&&d.translate(s,s),Ze(u,a,l,c,u),d.beginPath();var f,h,p,v,g=i+(0==r||3==r?-o:o);0==n?(h=i,v=g):(f=i,p=g);for(var y=0;y0&&(t._paths=null,e&&(1==i?(t.min=null,t.max=null):t.facets.forEach((function(e){e.min=null,e.max=null}))))}))}var ct,st,ft,dt,ht,pt,vt,mt,gt,yt,_t,bt,Dt=!1;function wt(){Dt||(Gl(xt),Dt=!0)}function xt(){ve&&(!function(){var e=ql(D,Yl);for(var n in e){var o=e[n],a=T[n];if(null!=a&&null!=a.min)Wl(o,a),n==x&<(!0);else if(n!=x||2==i)if(0==Me&&null==o.from){var u=o.range(r,null,null,n);o.min=u[0],o.max=u[1]}else o.min=Dl,o.max=-Dl}if(Me>0)for(var l in _.forEach((function(n,o){if(1==i){var a=n.scale,u=e[a],l=T[a];if(0==o){var c=u.range(r,u.min,u.max,a);u.min=c[0],u.max=c[1],Ye=Qu(u.min,t[0]),(Ve=Qu(u.max,t[0]))-Ye>1&&(t[0][Ye]u.max&&Ve--),n.min=We[Ye],n.max=We[Ve]}else n.show&&n.auto&&Xe(u,l,n,t[o],n.sorted);n.idxs[0]=Ye,n.idxs[1]=Ve}else if(o>0&&n.show&&n.auto){var s=Ft(n.facets,2),f=s[0],d=s[1],h=f.scale,p=d.scale,v=Ft(t[o],2),m=v[0],g=v[1];Xe(e[h],T[h],f,m,f.sorted),Xe(e[p],T[p],d,g,d.sorted),n.min=d.min,n.max=d.max}})),e){var c=e[l],s=T[l];if(null==c.from&&(null==s||null==s.min)){var f=c.range(r,c.min==Dl?null:c.min,c.max==-Dl?null:c.max,l);c.min=f[0],c.max=f[1]}}for(var d in e){var h=e[d];if(null!=h.from){var p=e[h.from];if(null==p.min)h.min=h.max=null;else{var v=h.range(r,p.min,p.max,d);h.min=v[0],h.max=v[1]}}}var m={},g=!1;for(var y in e){var b=e[y],w=D[y];if(w.min!=b.min||w.max!=b.max){w.min=b.min,w.max=b.max;var k=w.distr;w._min=3==k?yl(w.min):4==k?bl(w.min,w.asinh):w.min,w._max=3==k?yl(w.max):4==k?bl(w.max,w.asinh):w.max,m[y]=g=!0}}if(g){for(var C in _.forEach((function(e,t){2==i?t>0&&m.y&&(e._paths=null):m[e.scale]&&(e._paths=null)})),m)ge=!0,an("setScale",C);xe.show&&xe.left>=0&&(ye=be=!0)}for(var A in T)T[A]=null}(),ve=!1),ge&&(!function(){for(var e=!1,t=0;!e;){var n=at(++t),i=ut(t);(e=3==t||n&&i)||(we(r.width,r.height),me=!0)}}(),ge=!1),me&&(Lu(p,gu,ce),Lu(p,vu,se),Lu(p,hu,ue),Lu(p,pu,le),Lu(v,gu,ce),Lu(v,vu,se),Lu(v,hu,ue),Lu(v,pu,le),Lu(h,hu,oe),Lu(h,pu,ae),f.width=dl(oe*nu),f.height=dl(ae*nu),b.forEach((function(e){var t=e._el,n=e._show,r=e._size,i=e._pos,o=e.side;if(null!=t)if(n){var a=o%2==1;Lu(t,a?"left":"top",i-(3===o||0===o?r:0)),Lu(t,a?"width":"height",r),Lu(t,a?"top":"left",a?se:ce),Lu(t,a?"height":"width",a?le:ue),Bu(t,fu)}else Ou(t,fu)})),Te=Oe=Be=Ie=Pe=Re=ze=je=Le=null,$e=1,Qt(!0),an("setSize"),me=!1),oe>0&&ae>0&&(d.clearRect(0,0,f.width,f.height),an("drawClear"),C.forEach((function(e){return e()})),an("draw")),Ot.show&&_e&&(Lt(Ot),_e=!1),xe.show&&ye&&(qt(null,!0,!1),ye=!1),c||(c=!0,r.status=1,an("ready")),Qe=!1,Dt=!1}function kt(e,n){var i=D[e];if(null==i.from){if(0==Me){var o=i.range(r,n.min,n.max,e);n.min=o[0],n.max=o[1]}if(n.min>n.max){var a=n.min;n.min=n.max,n.max=a}if(Me>1&&null!=n.min&&null!=n.max&&n.max-n.min<1e-16)return;e==x&&2==i.distr&&Me>0&&(n.min=Qu(n.min,t[0]),n.max=Qu(n.max,t[0]),n.min==n.max&&n.max++),T[e]=n,ve=!0,wt()}}r.redraw=function(e,t){ge=t||!1,!1!==e?It(x,F.min,F.max):wt()},r.setScale=kt;var Ct=!1,At=xe.drag,Et=At.x,St=At.y;xe.show&&(xe.x&&(ct=Pu("u-cursor-x",v)),xe.y&&(st=Pu("u-cursor-y",v)),0==F.ori?(ft=ct,dt=st):(ft=st,dt=ct),_t=xe.left,bt=xe.top);var Nt,Mt,Tt,Ot=r.select=Wl({show:!0,over:!0,left:0,width:0,top:0,height:0},e.select),Bt=Ot.show?Pu("u-select",Ot.over?v:p):null;function Lt(e,t){if(Ot.show){for(var n in e)Ot[n]=e[n],n in Zt&&Lu(Bt,n,e[n]);!1!==t&&an("setSelect")}}function It(e,t,n){kt(e,{min:t,max:n})}function Pt(e,t,n,o){null!=t.focus&&function(e){if(e!=Tt){var t=null==e,n=1!=Ce.alpha;_.forEach((function(r,i){var o=t||0==i||i==e;r._focus=t?null:o,n&&function(e,t){_[e].alpha=t,xe.show&&Ee[e]&&(Ee[e].style.opacity=t);U&&q[e]&&(q[e].style.opacity=t)}(i,o?1:Ce.alpha)})),Tt=e,n&&wt()}}(e),null!=t.show&&_.forEach((function(n,r){r>0&&(e==r||null==e)&&(n.show=t.show,function(e,t){var n=_[e],r=U?q[e]:null;n.show?r&&Bu(r,fu):(r&&Ou(r,fu),Ee.length>1&&zu(Ee[e],-10,-10,ue,le))}(r,t.show),It(2==i?n.facets[1].scale:n.scale,null,null),wt())})),!1!==n&&an("setSeries",e,t),o&&sn("setSeries",r,e,t)}r.setSelect=Lt,r.setSeries=Pt,r.addBand=function(e,t){e.fill=Cl(e.fill||null),e.dir=il(e.dir,-1),t=null==t?w.length:t,w.splice(t,0,e)},r.setBand=function(e,t){Wl(w[e],t)},r.delBand=function(e){null==e?w.length=0:w.splice(e,1)};var Rt={focus:!0};function zt(e,t,n){var r=D[t];n&&(e=e/nu-(1==r.ori?se:ce));var i=ue;1==r.ori&&(e=(i=le)-e),-1==r.dir&&(e=i-e);var o=r._min,a=o+(r._max-o)*(e/i),u=r.distr;return 3==u?ml(10,a):4==u?function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return ll.sinh(e)*t}(a,r.asinh):a}function jt(e,t){Lu(Bt,gu,Ot.left=e),Lu(Bt,hu,Ot.width=t)}function $t(e,t){Lu(Bt,vu,Ot.top=e),Lu(Bt,pu,Ot.height=t)}U&&Ae&&qu(Cu,L,(function(e){xe._lock||null!=Tt&&Pt(null,Rt,!0,un.setSeries)})),r.valToIdx=function(e){return Qu(e,t[0])},r.posToIdx=function(e,n){return Qu(zt(e,x,n),t[0],Ye,Ve)},r.posToVal=zt,r.valToPos=function(e,t,n){return 0==D[t].ori?a(e,D[t],n?he:ue,n?fe:0):u(e,D[t],n?pe:le,n?de:0)},r.batch=function(e){e(r),wt()},r.setCursor=function(e,t,n){_t=e.left,bt=e.top,qt(null,t,n)};var Ht=0==F.ori?jt:$t,Ut=1==F.ori?jt:$t;function Yt(e,t){if(null!=e){var n=e.idx;H.idx=n,_.forEach((function(e,t){(t>0||!Q)&&Vt(t,n)}))}U&&H.live&&function(){if(U&&H.live)for(var e=2==i?1:0;e<_.length;e++)if(0!=e||!Q){var t=H.values[e],n=0;for(var r in t)W[e][n++].firstChild.nodeValue=t[r]}}(),be=!1,!1!==t&&an("setLegend")}function Vt(e,n){var i;if(null==n)i=G;else{var o=_[e],a=0==e&&2==M?We:t[e];i=Q?o.values(r,e,n):{_:o.value(r,a[n],e,n)}}H.values[e]=i}function qt(e,n,o){gt=_t,yt=bt;var a,u=Ft(xe.move(r,_t,bt),2);_t=u[0],bt=u[1],xe.show&&(ft&&zu(ft,dl(_t),0,ue,le),dt&&zu(dt,0,dl(bt),ue,le));var l=Ye>Ve;Nt=Dl;var c=0==F.ori?ue:le,s=1==F.ori?ue:le;if(_t<0||0==Me||l){a=null;for(var f=0;f<_.length;f++)f>0&&Ee.length>1&&zu(Ee[f],-10,-10,ue,le);if(Ae&&Pt(null,Rt,!0,null==e&&un.setSeries),H.live){$.fill(null),be=!0;for(var d=0;d<_.length;d++)H.values[d]=G}}else{var h,p;1==i&&(a=Qu(h=zt(0==F.ori?_t:bt,x),t[0],Ye,Ve),p=Ml(S(t[0][a],F,c,0),.5));for(var v=2==i?1:0;v<_.length;v++){var m=_[v],g=$[v],y=1==i?t[v][g]:t[v][1][g],b=xe.dataIdx(r,v,a,h),w=1==i?t[v][b]:t[v][1][b];be=be||w!=y||b!=g,$[v]=b;var k=b==a?p:Ml(S(1==i?t[0][b]:t[v][0][b],F,c,0),.5);if(v>0&&m.show){var C=null==w?-10:Ml(N(w,1==i?D[m.scale]:D[m.facets[1].scale],s,0),.5);if(C>0&&1==i){var A=sl(C-bt);A<=Nt&&(Nt=A,Mt=v)}var E=void 0,M=void 0;if(0==F.ori?(E=k,M=C):(E=C,M=k),be&&Ee.length>1){$u(Ee[v],xe.points.fill(r,v),xe.points.stroke(r,v));var T=void 0,O=void 0,B=void 0,L=void 0,I=!0,P=xe.points.bbox;if(null!=P){I=!1;var R=P(r,v);B=R.left,L=R.top,T=R.width,O=R.height}else B=E,L=M,T=O=xe.points.size(r,v);Uu(Ee[v],T,O,I),zu(Ee[v],B,L,ue,le)}}if(H.live){if(!be||0==v&&Q)continue;Vt(v,b)}}}if(xe.idx=a,xe.left=_t,xe.top=bt,be&&(H.idx=a,Yt()),Ot.show&&Ct)if(null!=e){var z=Ft(un.scales,2),j=z[0],U=z[1],Y=Ft(un.match,2),V=Y[0],q=Y[1],W=Ft(e.cursor.sync.scales,2),J=W[0],Z=W[1],K=e.cursor.drag;if(Et=K._x,St=K._y,Et||St){var X,ee,te,ne,re,ie=e.select,oe=ie.left,ae=ie.top,ce=ie.width,se=ie.height,fe=e.scales[j].ori,de=e.posToVal,he=null!=j&&V(j,J),pe=null!=U&&q(U,Z);he&&Et?(0==fe?(X=oe,ee=ce):(X=ae,ee=se),te=D[j],ne=S(de(X,J),te,c,0),re=S(de(X+ee,J),te,c,0),Ht(pl(ne,re),sl(re-ne))):Ht(0,c),pe&&St?(1==fe?(X=oe,ee=ce):(X=ae,ee=se),te=D[U],ne=N(de(X,Z),te,s,0),re=N(de(X+ee,Z),te,s,0),Ut(pl(ne,re),sl(re-ne))):Ut(0,s)}else Kt()}else{var ve=sl(gt-ht),me=sl(yt-pt);if(1==F.ori){var ge=ve;ve=me,me=ge}Et=At.x&&ve>=At.dist,St=At.y&&me>=At.dist;var ye,_e,De=At.uni;null!=De?Et&&St&&(St=me>=De,(Et=ve>=De)||St||(me>ve?St=!0:Et=!0)):At.x&&At.y&&(Et||St)&&(Et=St=!0),Et&&(0==F.ori?(ye=vt,_e=_t):(ye=mt,_e=bt),Ht(pl(ye,_e),sl(_e-ye)),St||Ut(0,s)),St&&(1==F.ori?(ye=vt,_e=_t):(ye=mt,_e=bt),Ut(pl(ye,_e),sl(_e-ye)),Et||Ht(0,c)),Et||St||(Ht(0,0),Ut(0,0))}if(At._x=Et,At._y=St,null==e){if(o){if(null!=ln){var we=Ft(un.scales,2),ke=we[0],Se=we[1];un.values[0]=null!=ke?zt(0==F.ori?_t:bt,ke):null,un.values[1]=null!=Se?zt(1==F.ori?_t:bt,Se):null}sn(Du,r,_t,bt,ue,le,a)}if(Ae){var Ne=o&&un.setSeries,Fe=Ce.prox;null==Tt?Nt<=Fe&&Pt(Mt,Rt,!0,Ne):Nt>Fe?Pt(null,Rt,!0,Ne):Mt!=Tt&&Pt(Mt,Rt,!0,Ne)}}!1!==n&&an("setCursor")}r.setLegend=Yt;var Wt=null;function Qt(e){!0===e?Wt=null:an("syncRect",Wt=v.getBoundingClientRect())}function Gt(e,t,n,r,i,o,a){xe._lock||Ct&&null!=e&&0==e.movementX&&0==e.movementY||(Jt(e,t,n,r,i,o,a,!1,null!=e),null!=e?qt(null,!0,!0):qt(t,!0,!1))}function Jt(e,t,n,i,o,a,u,c,s){if(null==Wt&&Qt(!1),null!=e)n=e.clientX-Wt.left,i=e.clientY-Wt.top;else{if(n<0||i<0)return _t=-10,void(bt=-10);var f=Ft(un.scales,2),d=f[0],h=f[1],p=t.cursor.sync,v=Ft(p.values,2),m=v[0],g=v[1],y=Ft(p.scales,2),_=y[0],b=y[1],w=Ft(un.match,2),x=w[0],k=w[1],C=t.axes[0].side%2==1,A=0==F.ori?ue:le,E=1==F.ori?ue:le,S=C?a:o,N=C?o:a,M=C?i:n,T=C?n:i;if(n=null!=_?x(d,_)?l(m,D[d],A,0):-10:A*(M/S),i=null!=b?k(h,b)?l(g,D[h],E,0):-10:E*(T/N),1==F.ori){var O=n;n=i,i=O}}if(s&&((n<=1||n>=ue-1)&&(n=xl(n,ue)),(i<=1||i>=le-1)&&(i=xl(i,le))),c){ht=n,pt=i;var B=Ft(xe.move(r,n,i),2);vt=B[0],mt=B[1]}else _t=n,bt=i}var Zt={width:0,height:0,left:0,top:0};function Kt(){Lt(Zt,!1)}function Xt(e,t,n,i,o,a,u){Ct=!0,Et=St=At._x=At._y=!1,Jt(e,t,n,i,o,a,0,!0,!1),null!=e&&(re(xu,Fu,en),sn(wu,r,vt,mt,ue,le,null))}function en(e,t,n,i,o,a,u){Ct=At._x=At._y=!1,Jt(e,t,n,i,o,a,0,!1,!0);var l=Ot.left,c=Ot.top,s=Ot.width,f=Ot.height,d=s>0||f>0;if(d&&Lt(Ot),At.setScale&&d){var h=l,p=s,v=c,m=f;if(1==F.ori&&(h=c,p=f,v=l,m=s),Et&&It(x,zt(h,x),zt(h+p,x)),St)for(var g in D){var y=D[g];g!=x&&null==y.from&&y.min!=Dl&&It(g,zt(v+m,g),zt(v,g))}Kt()}else xe.lock&&(xe._lock=!xe._lock,xe._lock||qt(null,!0,!1));null!=e&&(ie(xu,Fu),sn(xu,r,_t,bt,ue,le,null))}function tn(e,t,n,i,o,a,u){Je(),Kt(),null!=e&&sn(Au,r,_t,bt,ue,le,null)}function nn(){b.forEach(tf),De(r.width,r.height,!0)}qu(Su,Mu,nn);var rn={};rn.mousedown=Xt,rn.mousemove=Gt,rn.mouseup=en,rn.dblclick=tn,rn.setSeries=function(e,t,n,r){Pt(n,r,!0,!1)},xe.show&&(re(wu,v,Xt),re(Du,v,Gt),re(ku,v,Qt),re(Cu,v,(function(e,t,n,r,i,o,a){if(!xe._lock){var u=Ct;if(Ct){var l,c,s=!0,f=!0;0==F.ori?(l=Et,c=St):(l=St,c=Et),l&&c&&(s=_t<=10||_t>=ue-10,f=bt<=10||bt>=le-10),l&&s&&(_t=_t=3&&10==i.log?rs:El)),e.font=ef(e.font),e.labelFont=ef(e.labelFont),e._size=e.size(r,null,t,0),e._space=e._rotate=e._incrs=e._found=e._splits=e._values=null,e._size>0&&(Ne[t]=!0,e._el=Pu("u-axis",h))}})),n?n instanceof HTMLElement?(n.appendChild(s),fn()):n(r,fn):fn(),r}nf.assign=Wl,nf.fmtNum=ul,nf.rangeNum=rl,nf.rangeLog=Ku,nf.rangeAsinh=Xu,nf.orient=ms,nf.pxRatio=nu,nf.join=function(e,t){for(var n=new Set,r=0;r=a&&O<=u;O+=S){var B=s[O];if(null!=B){var L=x(c[O]),I=k(B);1==t?C(E,L,N):C(E,M,I),C(E,L,I),N=I,M=L}}var P=M;i&&1==t&&C(E,P=D+w,N);var R=Ft(gs(e,o),2),z=R[0],j=R[1];if(null!=l.fill||0!=z){var $=A.fill=new Path2D(E),H=k(l.fillTo(e,o,l.min,l.max,z));C($,P,H),C($,T,H)}if(!l.spanGaps){var U,Y=[];(U=Y).push.apply(U,Ot(Ds(c,s,a,u,S,x,r)));var V=l.width*nu/2,q=n||1==t?V:-V,W=n||-1==t?-V:V;Y.forEach((function(e){e[0]+=q,e[1]+=W})),A.gaps=Y=l.gaps(e,o,a,u,Y),A.clip=bs(Y,f.ori,v,m,g,y)}return 0!=j&&(A.band=2==j?[_s(e,o,a,u,E,-1),_s(e,o,a,u,E,1)]:_s(e,o,a,u,E,j)),A}))}},rf.bars=function(e){var t=il((e=e||Pl).size,[.6,Dl,1]),n=e.align||0,r=(e.gap||0)*nu,i=il(e.radius,0),o=1-t[0],a=il(t[1],Dl)*nu,u=il(t[2],1)*nu,l=il(e.disp,Pl),c=il(e.each,(function(e){})),s=l.fill,f=l.stroke;return function(e,t,d,h){return ms(e,t,(function(p,v,m,g,y,_,b,D,w,x,k){var C,A,E=p.pxRound,S=g.dir*(0==g.ori?1:-1),N=y.dir*(1==y.ori?1:-1),F=0==g.ori?Ss:Ns,M=0==g.ori?c:function(e,t,n,r,i,o,a){c(e,t,n,i,r,a,o)},T=Ft(gs(e,t),2),O=T[0],B=T[1],L=3==y.distr?1==O?y.max:y.min:0,I=b(L,y,k,w),P=E(p.width*nu),R=!1,z=null,j=null,$=null,H=null;null==s||0!=P&&null==f||(R=!0,z=s.values(e,t,d,h),j=new Map,new Set(z).forEach((function(e){null!=e&&j.set(e,new Path2D)})),P>0&&($=f.values(e,t,d,h),H=new Map,new Set($).forEach((function(e){null!=e&&H.set(e,new Path2D)}))));var U=l.x0,Y=l.size;if(null!=U&&null!=Y){v=U.values(e,t,d,h),2==U.unit&&(v=v.map((function(t){return e.posToVal(D+t*x,g.key,!0)})));var V=Y.values(e,t,d,h);A=E((A=2==Y.unit?V[0]*x:_(V[0],g,x,D)-_(0,g,x,D))-P),C=1==S?-P/2:A+P/2}else{var q=x;if(v.length>1)for(var W=null,Q=0,G=1/0;Q=d&&ie<=h;ie+=S){var oe=m[ie];if(void 0!==oe){var ae=_(2!=g.distr||null!=l?v[ie]:ie,g,x,D),ue=b(il(oe,L),y,k,w);null!=re&&null!=oe&&(I=b(re[ie],y,k,w));var le=E(ae-C),ce=E(vl(ue,I)),se=E(pl(ue,I)),fe=ce-se,de=i*A;null!=oe&&(R?(P>0&&null!=$[ie]&&F(H.get($[ie]),le,se+fl(P/2),A,vl(0,fe-P),de),null!=z[ie]&&F(j.get(z[ie]),le,se+fl(P/2),A,vl(0,fe-P),de)):F(X,le,se+fl(P/2),A,vl(0,fe-P),de),M(e,t,ie,le-P/2,se,A+P,fe)),0!=B&&(N*B==1?(ce=se,se=Z):(se=ce,ce=Z),F(ee,le-P/2,se,A+P,vl(0,fe=ce-se),0))}}return P>0&&(K.stroke=R?H:X),K.fill=R?j:X,K}))}},rf.spline=function(e){return function(e,t){var n=il(null===t||void 0===t?void 0:t.alignGaps,0);return function(t,r,i,o){return ms(t,r,(function(a,u,l,c,s,f,d,h,p,v,m){var g,y,_,b=a.pxRound,D=function(e){return b(f(e,c,v,h))},w=function(e){return b(d(e,s,m,p))};0==c.ori?(g=ks,_=As,y=Ts):(g=Cs,_=Es,y=Os);var x=c.dir*(0==c.ori?1:-1);i=Gu(l,i,o,1),o=Gu(l,i,o,-1);for(var k=D(u[1==x?i:o]),C=k,A=[],E=[],S=1==x?i:o;S>=i&&S<=o;S+=x)if(null!=l[S]){var N=D(u[S]);A.push(C=N),E.push(w(l[S]))}var F={stroke:e(A,E,g,_,y,b),fill:null,clip:null,band:null,gaps:null,flags:1},M=F.stroke,T=Ft(gs(t,r),2),O=T[0],B=T[1];if(null!=a.fill||0!=O){var L=F.fill=new Path2D(M),I=w(a.fillTo(t,r,a.min,a.max,O));_(L,C,I),_(L,k,I)}if(!a.spanGaps){var P,R=[];(P=R).push.apply(P,Ot(Ds(u,l,i,o,x,D,n))),F.gaps=R=a.gaps(t,r,i,o,R),F.clip=bs(R,c.ori,h,p,v,m)}return 0!=B&&(F.band=2==B?[_s(t,r,i,o,M,-1),_s(t,r,i,o,M,1)]:_s(t,r,i,o,M,B)),F}))}}(zs,e)};var of,af={legend:{show:!1},cursor:{drag:{x:!0,y:!1},focus:{prox:30},points:{size:5.6,width:1.4},bind:{click:function(){return null},dblclick:function(){return null}}}},uf=function(e,t,n){if(void 0===e||null===e)return"";n=n||0,t=t||0;var r=Math.abs(n-t);if(isNaN(r)||0==r)return Math.abs(e)>=1e3?e.toLocaleString("en-US"):e.toString();var i=3+Math.floor(1+Math.log10(Math.max(Math.abs(t),Math.abs(n)))-Math.log10(r));return(isNaN(i)||i>20)&&(i=20),e.toLocaleString("en-US",{minimumSignificantDigits:i,maximumSignificantDigits:i})},lf=function(e,t,n,r){var i,o=e.axes[n];if(r>1)return o._size||60;var a=6+((null===o||void 0===o||null===(i=o.ticks)||void 0===i?void 0:i.size)||0)+(o.gap||0),u=(null!==t&&void 0!==t?t:[]).reduce((function(e,t){return t.length>e.length?t:e}),"");return""!=u&&(a+=function(e,t){var n=document.createElement("span");n.innerText=e,n.style.cssText="position: absolute; z-index: -1; pointer-events: none; opacity: 0; font: ".concat(t),document.body.appendChild(n);var r=n.offsetWidth;return n.remove(),r}(u,"10px Arial")),Math.ceil(a)},cf=function(e){var t=e.e,n=e.factor,r=void 0===n?.85:n,i=e.u,o=e.setPanning,a=e.setPlotScale;t.preventDefault();var u=t instanceof MouseEvent;o(!0);var l=u?t.clientX:t.touches[0].clientX,c=i.posToVal(1,"x")-i.posToVal(0,"x"),s=i.scales.x.min||0,f=i.scales.x.max||0,d=function(e){var t=e instanceof MouseEvent;if(t||!(e.touches.length>1)){e.preventDefault();var n=t?e.clientX:e.touches[0].clientX,o=c*((n-l)*r);a({u:i,min:s-o,max:f-o})}},h=function e(){o(!1),document.removeEventListener("mousemove",d),document.removeEventListener("mouseup",e),document.removeEventListener("touchmove",d),document.removeEventListener("touchend",e)};document.addEventListener("mousemove",d),document.addEventListener("mouseup",h),document.addEventListener("touchmove",d),document.addEventListener("touchend",h)},sf=function(e){for(var t=e.length,n=-1/0;t--;){var r=e[t];Number.isFinite(r)&&r>n&&(n=r)}return Number.isFinite(n)?n:null},ff=function(e){for(var t=e.length,n=1/0;t--;){var r=e[t];Number.isFinite(r)&&r2&&void 0!==arguments[2]?arguments[2]:"",r=t[0],i=t[t.length-1];return n?t.map((function(e){return"".concat(uf(e,r,i)," ").concat(n)})):t.map((function(e){return uf(e,r,i)}))}(e,n,t)}};return e?Number(e)%2?n:yr(yr({},n),{},{side:1}):{space:80,values:df,stroke:Rr("color-text")}}))},pf=function(e,t){if(null==e||null==t)return[-1,1];var n=.02*(Math.abs(t-e)||Math.abs(e)||1);return[e-n,t+n]},vf=n(61),mf=n.n(vf),gf=function(e){var t,n,r,i=e.u,o=e.id,a=e.unit,u=void 0===a?"":a,l=e.metrics,c=e.series,s=e.yRange,f=e.tooltipIdx,d=e.tooltipOffset,h=e.isSticky,p=e.onClose,v=ie(null),m=Ft(ee({top:-999,left:-999}),2),y=m[0],_=m[1],b=Ft(ee(!1),2),D=b[0],w=b[1],x=Ft(ee(!1),2),k=x[0],C=x[1],A=Ft(ee(f.seriesIdx),2),E=A[0],S=A[1],N=Ft(ee(f.dataIdx),2),F=N[0],M=N[1],T=ae((function(){return i.root.querySelector(".u-wrap")}),[i]),O=Mr()(i,["data",E,F],0),B=uf(O,Mr()(s,[0]),Mr()(s,[1])),L=i.data[0][F],I=_t()(1e3*L).tz().format("YYYY-MM-DD HH:mm:ss:SSS (Z)"),P=(null===(t=c[E])||void 0===t?void 0:t.stroke)+"",R=(null===(n=c[E])||void 0===n?void 0:n.calculations)||{},z=new Set(l.map((function(e){return e.group}))),j=z.size>1,$=(null===(r=l[E-1])||void 0===r?void 0:r.group)||0,H=ae((function(){var e,t=(null===(e=l[E-1])||void 0===e?void 0:e.metric)||{},n=Object.keys(t).filter((function(e){return"__name__"!=e})),r=n.map((function(e){return"".concat(e,"=").concat(JSON.stringify(t[e]))})),i=t.__name__||"";return r.length>0&&(i+="{"+r.join(",")+"}"),i}),[l,E]),U=function(e){if(D){var t=e.clientX,n=e.clientY;_({top:n,left:t})}},Y=function(){w(!1)};return ne((function(){var e;if(v.current){var t=i.valToPos(O||0,(null===(e=c[E])||void 0===e?void 0:e.scale)||"1"),n=i.valToPos(L,"x"),r=v.current.getBoundingClientRect(),o=r.width,a=r.height,u=i.over.getBoundingClientRect(),l=n+o>=u.width?o+20:0,s=t+a>=u.height?a+20:0,f={top:t+d.top+10-s,left:n+d.left+10-l};f.left<0&&(f.left=20),f.top<0&&(f.top=20),_(f)}}),[i,O,L,E,d,v]),ne((function(){S(f.seriesIdx),M(f.dataIdx)}),[f]),ne((function(){return D&&(document.addEventListener("mousemove",U),document.addEventListener("mouseup",Y)),function(){document.removeEventListener("mousemove",U),document.removeEventListener("mouseup",Y)}}),[D]),!T||f.seriesIdx<0||f.dataIdx<0?null:gt.createPortal(Vr("div",{className:wo()({"vm-chart-tooltip":!0,"vm-chart-tooltip_sticky":h,"vm-chart-tooltip_moved":k}),ref:v,style:y,children:[Vr("div",{className:"vm-chart-tooltip-header",children:[Vr("div",{className:"vm-chart-tooltip-header__date",children:[j&&Vr("div",{children:["Query ",$]}),I]}),h&&Vr(g,{children:[Vr(fa,{className:"vm-chart-tooltip-header__drag",variant:"text",size:"small",startIcon:Vr(fo,{}),onMouseDown:function(e){C(!0),w(!0);var t=e.clientX,n=e.clientY;_({top:n,left:t})}}),Vr(fa,{className:"vm-chart-tooltip-header__close",variant:"text",size:"small",startIcon:Vr(ji,{}),onClick:function(){p&&p(o)}})]})]}),Vr("div",{className:"vm-chart-tooltip-data",children:[Vr("div",{className:"vm-chart-tooltip-data__marker",style:{background:P}}),Vr("div",{children:[Vr("b",{children:[B,u]}),Vr("br",{}),"median:",Vr("b",{children:R.median}),", min:",Vr("b",{children:R.min}),", max:",Vr("b",{children:R.max})]})]}),Vr("div",{className:"vm-chart-tooltip-info",children:H})]}),T)};!function(e){e.xRange="xRange",e.yRange="yRange",e.data="data"}(of||(of={}));var yf=function(e){var t=e.data,n=e.series,r=e.metrics,i=void 0===r?[]:r,o=e.period,a=e.yaxis,u=e.unit,l=e.setPeriod,c=e.container,s=e.height,f=Wr().isDarkTheme,d=ie(null),h=Ft(ee(!1),2),v=h[0],m=h[1],g=Ft(ee({min:o.start,max:o.end}),2),y=g[0],_=g[1],b=Ft(ee([0,1]),2),D=b[0],w=b[1],x=Ft(ee(),2),k=x[0],C=x[1],A=Ft(ee(0),2),E=A[0],S=A[1],N=bo(c),F=Ft(ee(!1),2),M=F[0],T=F[1],O=Ft(ee({seriesIdx:-1,dataIdx:-1}),2),B=O[0],L=O[1],I=Ft(ee({left:0,top:0}),2),P=I[0],R=I[1],z=Ft(ee([]),2),j=z[0],$=z[1],H=ae((function(){return"".concat(B.seriesIdx,"_").concat(B.dataIdx)}),[B]),U=ue(mf()((function(e){var t=e.min,n=e.max;l({from:_t()(1e3*t).toDate(),to:_t()(1e3*n).toDate()})}),500),[]),Y=function(e){var t=e.u,n=e.min,r=e.max,i=1e3*(r-n);iti||(t.setScale("x",{min:n,max:r}),_({min:n,max:r}),U({min:n,max:r}))},V=function(e){var t=e.target,n=e.ctrlKey,r=e.metaKey,i=e.key,o=t instanceof HTMLInputElement||t instanceof HTMLTextAreaElement;if(k&&!o){var a="+"===i||"="===i;if(("-"===i||a)&&!n&&!r){e.preventDefault();var u=(y.max-y.min)/10*(a?1:-1);Y({u:k,min:y.min+u,max:y.max-u})}}},q=function(){var e="".concat(B.seriesIdx,"_").concat(B.dataIdx),t={id:e,unit:u,series:n,metrics:i,yRange:D,tooltipIdx:B,tooltipOffset:P};if(!j.find((function(t){return t.id===e}))){var r=JSON.parse(JSON.stringify(t));$((function(e){return[].concat(Ot(e),[r])}))}},W=function(e){$((function(t){return t.filter((function(t){return t.id!==e}))}))},Q=function(){return[y.min,y.max]},G=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,r=arguments.length>3?arguments[3]:void 0;return"1"==r&&w([t,n]),a.limits.enable?a.limits.range[r]:pf(t,n)},J=yr(yr({},af),{},{tzDate:function(e){return _t()(fi(hi(e))).local().toDate()},series:n,axes:hf([{},{scale:"1"}],u),scales:yr({},function(){var e={x:{range:Q}},t=Object.keys(a.limits.range);return(t.length?t:["1"]).forEach((function(t){e[t]={range:function(e){var n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,r=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1;return G(e,n,r,t)}}})),e}()),width:N.width||400,height:s||500,plugins:[{hooks:{ready:function(e){var t=.9;R({left:parseFloat(e.over.style.left),top:parseFloat(e.over.style.top)}),e.over.addEventListener("mousedown",(function(n){var r=n.ctrlKey,i=n.metaKey;0===n.button&&(r||i)&&cf({u:e,e:n,setPanning:m,setPlotScale:Y,factor:t})})),e.over.addEventListener("touchstart",(function(n){cf({u:e,e:n,setPanning:m,setPlotScale:Y,factor:t})})),e.over.addEventListener("wheel",(function(n){if(n.ctrlKey||n.metaKey){n.preventDefault();var r=e.over.getBoundingClientRect().width,i=e.cursor.left&&e.cursor.left>0?e.cursor.left:0,o=e.posToVal(i,"x"),a=(e.scales.x.max||0)-(e.scales.x.min||0),u=n.deltaY<0?a*t:a/t,l=o-i/r*u,c=l+u;e.batch((function(){return Y({u:e,min:l,max:c})}))}}))},setCursor:function(e){var t,n=null!==(t=e.cursor.idx)&&void 0!==t?t:-1;L((function(e){return yr(yr({},e),{},{dataIdx:n})}))},setSeries:function(e,t){var n=null!==t&&void 0!==t?t:-1;L((function(e){return yr(yr({},e),{},{seriesIdx:n})}))}}}],hooks:{setSelect:[function(e){var t=e.posToVal(e.select.left,"x"),n=e.posToVal(e.select.left+e.select.width,"x");Y({u:e,min:t,max:n})}]}}),Z=function(e){if(k){switch(e){case of.xRange:k.scales.x.range=Q;break;case of.yRange:Object.keys(a.limits.range).forEach((function(e){k.scales[e]&&(k.scales[e].range=function(t){var n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,r=arguments.length>2&&void 0!==arguments[2]?arguments[2]:1;return G(t,n,r,e)})}));break;case of.data:k.setData(t)}v||k.redraw()}};ne((function(){return _({min:o.start,max:o.end})}),[o]),ne((function(){if($([]),L({seriesIdx:-1,dataIdx:-1}),d.current){var e=new nf(J,t,d.current);return C(e),_({min:o.start,max:o.end}),e.destroy}}),[d.current,n,N,s,f]),ne((function(){return window.addEventListener("keydown",V),function(){window.removeEventListener("keydown",V)}}),[y]);var K=function(e){if(2===e.touches.length){e.preventDefault();var t=e.touches[0].clientX-e.touches[1].clientX,n=e.touches[0].clientY-e.touches[1].clientY;S(Math.sqrt(t*t+n*n))}},X=function(e){if(2===e.touches.length&&k){e.preventDefault();var t=e.touches[0].clientX-e.touches[1].clientX,n=e.touches[0].clientY-e.touches[1].clientY,r=Math.sqrt(t*t+n*n),i=E-r,o=k.scales.x.max||y.max,a=k.scales.x.min||y.min,u=(o-a)/50*(i>0?-1:1);k.batch((function(){return Y({u:k,min:a+u,max:o-u})}))}};return ne((function(){return window.addEventListener("touchmove",X),window.addEventListener("touchstart",K),function(){window.removeEventListener("touchmove",X),window.removeEventListener("touchstart",K)}}),[k,E]),ne((function(){return Z(of.data)}),[t]),ne((function(){return Z(of.xRange)}),[y]),ne((function(){return Z(of.yRange)}),[a]),ne((function(){var e=-1!==B.dataIdx&&-1!==B.seriesIdx;return T(e),e&&window.addEventListener("click",q),function(){window.removeEventListener("click",q)}}),[B,j]),Vr("div",{className:wo()({"vm-line-chart":!0,"vm-line-chart_panning":v}),style:{minWidth:"".concat(N.width||400,"px"),minHeight:"".concat(s||500,"px")},children:[Vr("div",{className:"vm-line-chart__u-plot",ref:d}),k&&M&&Vr(gf,{unit:u,u:k,series:n,metrics:i,yRange:D,tooltipIdx:B,tooltipOffset:P,id:H}),k&&j.map((function(e){return p(gf,yr(yr({},e),{},{isSticky:!0,u:k,key:e.id,onClose:W}))}))]})},_f=function(e){var t=e.legend,n=e.onChange,r=Ft(ee(""),2),i=r[0],o=r[1],a=ae((function(){return function(e){var t=Object.keys(e.freeFormFields).filter((function(e){return"__name__"!==e}));return t.map((function(t){var n="".concat(t,"=").concat(JSON.stringify(e.freeFormFields[t]));return{id:"".concat(e.label,".").concat(n),freeField:n,key:t}}))}(t)}),[t]),u=t.calculations,l=function(){var e=tu(Xa().mark((function e(t,n){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(t);case 2:o(n),setTimeout((function(){return o("")}),2e3);case 4:case"end":return e.stop()}}),e)})));return function(t,n){return e.apply(this,arguments)}}();return Vr("div",{className:wo()({"vm-legend-item":!0,"vm-legend-row":!0,"vm-legend-item_hide":!t.checked}),onClick:function(e){return function(t){n(e,t.ctrlKey||t.metaKey)}}(t),children:[Vr("div",{className:"vm-legend-item__marker",style:{backgroundColor:t.color}}),Vr("div",{className:"vm-legend-item-info",children:Vr("span",{className:"vm-legend-item-info__label",children:[t.freeFormFields.__name__,"{",a.map((function(e,t){return Vr(ma,{open:i===e.id,title:"copied!",placement:"top-center",children:Vr("span",{className:"vm-legend-item-info__free-fields",onClick:(n=e.freeField,r=e.id,function(e){e.stopPropagation(),l(n,r)}),title:"copy to clipboard",children:[e.freeField,t+11;return Vr(g,{children:Vr("div",{className:"vm-legend",children:i.map((function(e){return Vr("div",{className:"vm-legend-group",children:[Vr("div",{className:"vm-legend-group-title",children:[o&&Vr("span",{className:"vm-legend-group-title__count",children:["Query ",e,": "]}),Vr("span",{className:"vm-legend-group-title__query",children:n[e-1]})]}),Vr("div",{children:t.filter((function(t){return t.group===e})).map((function(e){return Vr(_f,{legend:e,onChange:r},e.label)}))})]},e)}))})})},Df=["__name__"],wf=function(e,t){var n=!(arguments.length>2&&void 0!==arguments[2])||arguments[2],r=e.metric,i=r.__name__,o=xo(r,Df),a=t||"".concat(n?"[Query ".concat(e.group,"] "):"").concat(i||"");return 0==Object.keys(o).length?a||"value":"".concat(a,"{").concat(Object.entries(o).map((function(e){return"".concat(e[0],"=").concat(JSON.stringify(e[1]))})).join(", "),"}")},xf=function(e){switch(e){case"NaN":return NaN;case"Inf":case"+Inf":return 1/0;case"-Inf":return-1/0;default:return parseFloat(e)}},kf=["#e54040","#32a9dc","#2ee329","#7126a1","#e38f0f","#3d811a","#ffea00","#2d2d2d","#da42a6","#a44e0c"],Cf=function(e){var t=16777215,n=1,r=0,i=1;if(e.length>0)for(var o=0;or&&(r=e[o].charCodeAt(0)),i=parseInt(String(t/r)),n=(n+e[o].charCodeAt(0)*i*49979693)%t;var a=(n*e.length%t).toString(16);return a=a.padEnd(6,a),"#".concat(a)},Af=function(){var e={};return function(t,n,r){var i=wf(t,r[t.group-1]),o=Object.keys(e).length;o>1]}(a),s=function(e){for(var t=e.length;t--;){var n=e[t];if(Number.isFinite(n))return n}}(a);return{label:i,freeFormFields:t.metric,width:1.4,stroke:e[i]||Cf(i),show:!Sf(i,n),scale:"1",points:{size:4.2,width:1.4},calculations:{min:uf(u,u,l),max:uf(l,u,l),median:uf(c,u,l),last:uf(s,u,l)}}}},Ef=function(e,t){return{group:t,label:e.label||"",color:e.stroke,checked:e.show||!1,freeFormFields:e.freeFormFields,calculations:e.calculations}},Sf=function(e,t){return t.includes("".concat(e))},Nf=function(e){var t=e.data,n=void 0===t?[]:t,r=e.period,i=e.customStep,o=e.query,a=e.yaxis,u=e.unit,l=e.showLegend,c=void 0===l||l,s=e.setYaxisLimits,f=e.setPeriod,d=e.alias,h=void 0===d?[]:d,p=e.fullWidth,v=void 0===p||p,m=e.height,g=Xo().isMobile,y=Ni().timezone,_=ae((function(){return i||r.step||"1s"}),[r.step,i]),b=ue(Af(),[n]),D=Ft(ee([[]]),2),w=D[0],x=D[1],k=Ft(ee([]),2),C=k[0],A=k[1],E=Ft(ee([]),2),S=E[0],N=E[1],F=Ft(ee([]),2),M=F[0],T=F[1],O=function(e){var t=function(e){var t={},n=Object.values(e).flat(),r=ff(n),i=sf(n);return t[1]=pf(r,i),t}(e);s(t)};ne((function(){var e=[],t={},i=[],o=[{}];null===n||void 0===n||n.forEach((function(n){var r=b(n,M,h);o.push(r),i.push(Ef(r,n.group));var a,u=t[n.group]||[],l=Mt(n.values);try{for(l.s();!(a=l.n()).done;){var c=a.value;e.push(c[0]),u.push(xf(c[1]))}}catch(s){l.e(s)}finally{l.f()}t[n.group]=u}));var a=function(e,t,n){for(var r=li(t)||1,i=Array.from(new Set(e)).sort((function(e,t){return e-t})),o=n.start,a=ai(n.end+r),u=0,l=[];o<=a;){for(;u=i.length||i[u]>o)&&l.push(o)}for(;l.length<2;)l.push(o),o=ai(o+r);return l}(e,_,r),u=n.map((function(e){var t,n=[],r=e.values,i=r.length,o=0,u=Mt(a);try{for(u.s();!(t=u.n()).done;){for(var l=t.value;o1e10*h?n.map((function(){return f})):n}));u.unshift(a),O(t),x(u),A(o),N(i)}),[n,y]),ne((function(){var e=[],t=[{}];null===n||void 0===n||n.forEach((function(n){var r=b(n,M,h);t.push(r),e.push(Ef(r,n.group))})),A(t),N(e)}),[M]);var B=ie(null);return Vr("div",{className:wo()({"vm-graph-view":!0,"vm-graph-view_full-width":v,"vm-graph-view_full-width_mobile":v&&g}),ref:B,children:[(null===B||void 0===B?void 0:B.current)&&Vr(yf,{data:w,series:C,metrics:n,period:r,yaxis:a,unit:u,setPeriod:f,container:null===B||void 0===B?void 0:B.current,height:m}),c&&Vr(bf,{labels:S,query:o,onChange:function(e,t){T(function(e){var t=e.hideSeries,n=e.legend,r=e.metaKey,i=e.series,o=n.label,a=Sf(o,t),u=i.map((function(e){return e.label||""}));return r?a?t.filter((function(e){return e!==o})):[].concat(Ot(t),[o]):t.length?a?Ot(u.filter((function(e){return e!==o}))):[]:Ot(u.filter((function(e){return e!==o})))}({hideSeries:M,legend:e,metaKey:t,series:C}))}})]})},Ff=function(e){var t=e.value,n=e.options,r=e.anchor,i=e.disabled,o=e.maxWords,a=void 0===o?1:o,u=e.minLength,l=void 0===u?2:u,c=e.fullWidth,f=e.selected,d=e.noOptionsText,h=e.label,p=e.disabledFullScreen,v=e.onSelect,m=e.onOpenAutocomplete,g=Xo().isMobile,y=ie(null),_=Ft(ee(!1),2),b=_[0],D=_[1],w=Ft(ee(-1),2),x=w[0],k=w[1],C=ae((function(){if(!b)return[];try{var e=new RegExp(String(t),"i");return n.filter((function(n){return e.test(n)&&n!==t})).sort((function(t,n){var r,i;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(i=n.match(e))||void 0===i?void 0:i.index)||0)}))}catch(s){return[]}}),[b,n,t]),A=ae((function(){return d&&!C.length}),[d,C]),E=function(){D(!1)},S=function(e){var t=e.key,n=e.ctrlKey,r=e.metaKey,i=e.shiftKey,o=n||r||i,a=C.length;if("ArrowUp"===t&&!o&&a&&(e.preventDefault(),k((function(e){return e<=0?0:e-1}))),"ArrowDown"===t&&!o&&a){e.preventDefault();var u=C.length-1;k((function(e){return e>=u?u:e+1}))}if("Enter"===t){var l=C[x];l&&v(l),f||E()}"Escape"===t&&E()};return ne((function(){var e=(t.match(/[a-zA-Z_:.][a-zA-Z0-9_:.]*/gm)||[]).length;D(t.length>l&&e<=a)}),[t]),ne((function(){return function(){if(y.current){var e=y.current.childNodes[x];null!==e&&void 0!==e&&e.scrollIntoView&&e.scrollIntoView({block:"center"})}}(),window.addEventListener("keydown",S),function(){window.removeEventListener("keydown",S)}}),[x,C]),ne((function(){k(-1)}),[C]),ne((function(){m&&m(b)}),[b]),Vr(da,{open:b,buttonRef:r,placement:"bottom-left",onClose:E,fullWidth:c,title:g?h:void 0,disabledFullScreen:p,children:Vr("div",{className:wo()({"vm-autocomplete":!0,"vm-autocomplete_mobile":g&&!p}),ref:y,children:[A&&Vr("div",{className:"vm-autocomplete__no-options",children:d}),C.map((function(e,t){return Vr("div",{className:wo()({"vm-list-item":!0,"vm-list-item_mobile":g,"vm-list-item_active":t===x,"vm-list-item_multiselect":f,"vm-list-item_multiselect_selected":null===f||void 0===f?void 0:f.includes(e)}),id:"$autocomplete$".concat(e),onClick:(n=e,function(){i||(v(n),f||E())}),children:[(null===f||void 0===f?void 0:f.includes(e))&&Vr(uo,{}),Vr("span",{children:e})]},e);var n}))]})})},Mf=function(e){var t=e.value,n=e.onChange,r=e.onEnter,i=e.onArrowUp,o=e.onArrowDown,a=e.autocomplete,u=e.error,l=e.options,c=e.label,s=e.disabled,f=void 0!==s&&s,d=Ft(ee(!1),2),h=d[0],p=d[1],v=ie(null);return Vr("div",{className:"vm-query-editor",ref:v,children:[Vr(xa,{value:t,label:c,type:"textarea",autofocus:!!t,error:u,onKeyDown:function(e){var t=e.key,n=e.ctrlKey,a=e.metaKey,u=e.shiftKey,l=n||a,c="ArrowDown"===t,s="Enter"===t;"ArrowUp"===t&&l&&(e.preventDefault(),i()),c&&l&&(e.preventDefault(),o()),!s||u||h||r()},onChange:n,disabled:f,inputmode:"search"}),a&&Vr(Ff,{disabledFullScreen:!0,value:t,options:l,anchor:v,onSelect:function(e){n(e)},onOpenAutocomplete:p})]})},Tf=function(e){var t,n=e.value,r=void 0!==n&&n,i=e.disabled,o=void 0!==i&&i,a=e.label,u=e.color,l=void 0===u?"secondary":u,c=e.fullWidth,s=e.onChange;return Vr("div",{className:wo()((mr(t={"vm-switch":!0,"vm-switch_full-width":c,"vm-switch_disabled":o,"vm-switch_active":r},"vm-switch_".concat(l,"_active"),r),mr(t,"vm-switch_".concat(l),l),t)),onClick:function(){o||s(!r)},children:[Vr("div",{className:"vm-switch-track",children:Vr("div",{className:"vm-switch-track__thumb"})}),a&&Vr("span",{className:"vm-switch__label",children:a})]})},Of=function(e){var t=e.isMobile,n=Li().autocomplete,r=Ii(),i=Io(),o=i.nocache,a=i.isTracingEnabled,u=Po();return Vr("div",{className:wo()({"vm-additional-settings":!0,"vm-additional-settings_mobile":t}),children:[Vr(Tf,{label:"Autocomplete",value:n,onChange:function(){r({type:"TOGGLE_AUTOCOMPLETE"})},fullWidth:t}),Vr(Tf,{label:"Disable cache",value:o,onChange:function(){u({type:"TOGGLE_NO_CACHE"})},fullWidth:t}),Vr(Tf,{label:"Trace query",value:a,onChange:function(){u({type:"TOGGLE_QUERY_TRACING"})},fullWidth:t})]})},Bf=function(){var e=Xo().isMobile,t=Ft(ee(!1),2),n=t[0],r=t[1],i=ie(null);return e?Vr(g,{children:[Vr("div",{ref:i,children:Vr(fa,{variant:"outlined",startIcon:Vr(_o,{}),onClick:function(){r((function(e){return!e}))}})}),Vr(da,{open:n,buttonRef:i,placement:"bottom-left",onClose:function(){r(!1)},title:"Query settings",children:Vr(Of,{isMobile:e})})]}):Vr(Of,{})},Lf=function(e,t){return e.length===t.length&&e.every((function(e,n){return e===t[n]}))},If=function(e){var t=e.error,n=e.queryOptions,r=e.onHideQuery,i=e.onRunQuery,o=Xo().isMobile,a=Li(),u=a.query,l=a.queryHistory,c=a.autocomplete,s=Ii(),f=Fi(),d=Ft(ee(u||[]),2),h=d[0],p=d[1],v=Ft(ee([]),2),m=v[0],g=v[1],y=Aa(h),_=function(){s({type:"SET_QUERY_HISTORY",payload:h.map((function(e,t){var n=l[t]||{values:[]},r=e===n.values[n.values.length-1];return{index:n.values.length-Number(r),values:!r&&e?[].concat(Ot(n.values),[e]):n.values}}))}),s({type:"SET_QUERY",payload:h}),f({type:"RUN_QUERY"}),i()},b=function(e,t){p((function(n){return n.map((function(n,r){return r===t?e:n}))}))},D=function(e,t){return function(){!function(e,t){var n=l[t],r=n.index,i=n.values,o=r+e;o<0||o>=i.length||(b(i[o]||"",t),s({type:"SET_QUERY_HISTORY_BY_INDEX",payload:{value:{values:i,index:o},queryNumber:t}}))}(e,t)}},w=function(e){return function(t){b(t,e)}},x=function(e){return function(){var t;t=e,p((function(e){return e.filter((function(e,n){return n!==t}))})),g((function(t){return t.includes(e)?t.filter((function(t){return t!==e})):t.map((function(t){return t>e?t-1:t}))}))}},k=function(e){return function(t){!function(e,t){var n=e.ctrlKey,r=e.metaKey;if(n||r){var i=h.map((function(e,t){return t})).filter((function(e){return e!==t}));g((function(e){return Lf(i,e)?[]:i}))}else g((function(e){return e.includes(t)?e.filter((function(e){return e!==t})):[].concat(Ot(e),[t])}))}(t,e)}};return ne((function(){y&&h.length1&&Vr(ma,{title:"Remove Query",children:Vr("div",{className:"vm-query-configurator-list-row__button",children:Vr(fa,{variant:"text",color:"error",startIcon:Vr(io,{}),onClick:x(r)})})})]},r)}))}),Vr("div",{className:"vm-query-configurator-settings",children:[Vr(Bf,{}),Vr("div",{className:"vm-query-configurator-settings__buttons",children:[h.length<4&&Vr(fa,{variant:"outlined",onClick:function(){p((function(e){return[].concat(Ot(e),[""])}))},startIcon:Vr(oo,{}),children:"Add Query"}),Vr(fa,{variant:"contained",onClick:_,startIcon:Vr(Xi,{}),children:o?"Execute":"Execute Query"})]})]})]})};function Pf(e){var t,n,r,i=2;for("undefined"!=typeof Symbol&&(n=Symbol.asyncIterator,r=Symbol.iterator);i--;){if(n&&null!=(t=e[n]))return t.call(e);if(r&&null!=(t=e[r]))return new Rf(t.call(e));n="@@asyncIterator",r="@@iterator"}throw new TypeError("Object is not async iterable")}function Rf(e){function t(e){if(Object(e)!==e)return Promise.reject(new TypeError(e+" is not an object."));var t=e.done;return Promise.resolve(e.value).then((function(e){return{value:e,done:t}}))}return Rf=function(e){this.s=e,this.n=e.next},Rf.prototype={s:null,n:null,next:function(){return t(this.n.apply(this.s,arguments))},return:function(e){var n=this.s.return;return void 0===n?Promise.resolve({value:e,done:!0}):t(n.apply(this.s,arguments))},throw:function(e){var n=this.s.return;return void 0===n?Promise.reject(e):t(n.apply(this.s,arguments))}},new Rf(e)}var zf=n(936),jf=n.n(zf),$f=0,Hf=function(){function e(t,n){Bt(this,e),this.tracing=void 0,this.query=void 0,this.tracingChildren=void 0,this.originalTracing=void 0,this.id=void 0,this.tracing=t,this.originalTracing=JSON.parse(JSON.stringify(t)),this.query=n,this.id=$f++;var r=t.children||[];this.tracingChildren=r.map((function(t){return new e(t,n)}))}return Rt(e,[{key:"queryValue",get:function(){return this.query}},{key:"idValue",get:function(){return this.id}},{key:"children",get:function(){return this.tracingChildren}},{key:"message",get:function(){return this.tracing.message}},{key:"duration",get:function(){return this.tracing.duration_msec}},{key:"JSON",get:function(){return JSON.stringify(this.tracing,null,2)}},{key:"originalJSON",get:function(){return JSON.stringify(this.originalTracing,null,2)}},{key:"setTracing",value:function(t){var n=this;this.tracing=t;var r=t.children||[];this.tracingChildren=r.map((function(t){return new e(t,n.query)}))}},{key:"setQuery",value:function(e){this.query=e}},{key:"resetTracing",value:function(){this.tracing=this.originalTracing}}]),e}(),Uf=function(e){var t=e.predefinedQuery,n=e.visible,r=e.display,i=e.customStep,o=e.hideQuery,a=e.showAllSeries,u=Li().query,l=Ni().period,c=Io(),s=c.displayType,f=c.nocache,d=c.isTracingEnabled,h=c.seriesLimits,p=Wr().serverUrl,v=Ft(ee(!1),2),m=v[0],g=v[1],y=Ft(ee(),2),_=y[0],b=y[1],D=Ft(ee(),2),w=D[0],x=D[1],k=Ft(ee(),2),C=k[0],A=k[1],E=Ft(ee(),2),S=E[0],N=E[1],F=Ft(ee(),2),M=F[0],T=F[1],O=Ft(ee([]),2),B=O[0],L=O[1];ne((function(){S&&(b(void 0),x(void 0),A(void 0))}),[S]);var I=function(){var e=tu(Xa().mark((function e(t){var n,r,i,o,a,u,l,c,s,f,d,h,p,v,m,y,_,D,w,k,C,E,S,F,M;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:n=t.fetchUrl,r=t.fetchQueue,i=t.displayType,o=t.query,a=t.stateSeriesLimits,u=t.showAllSeries,l=t.hideQuery,c=new AbortController,L([].concat(Ot(r),[c])),e.prev=3,s="chart"===i,f=u?1/0:a[i],d=[],h=[],p=1,v=0,m=!1,y=!1,e.prev=12,D=Pf(n);case 14:return e.next=16,D.next();case 16:if(!(m=!(w=e.sent).done)){e.next=32;break}if(k=w.value,!(null===l||void 0===l?void 0:l.includes(p-1))){e.next=22;break}return p++,e.abrupt("continue",29);case 22:return e.next=24,fetch(k,{signal:c.signal});case 24:return C=e.sent,e.next=27,C.json();case 27:E=e.sent,C.ok?(N(void 0),E.trace&&(S=new Hf(E.trace,o[p-1]),h.push(S)),F=f-d.length,E.data.result.slice(0,F).forEach((function(e){e.group=p,d.push(e)})),v+=E.data.result.length,p++):N("".concat(E.errorType,"\r\n").concat(null===E||void 0===E?void 0:E.error));case 29:m=!1,e.next=14;break;case 32:e.next=38;break;case 34:e.prev=34,e.t0=e.catch(12),y=!0,_=e.t0;case 38:if(e.prev=38,e.prev=39,!m||null==D.return){e.next=43;break}return e.next=43,D.return();case 43:if(e.prev=43,!y){e.next=46;break}throw _;case 46:return e.finish(43);case 47:return e.finish(38);case 48:M="Showing ".concat(f," series out of ").concat(v," series due to performance reasons. Please narrow down the query, so it returns less series"),T(v>f?M:""),s?b(d):x(d),A(h),e.next=57;break;case 54:e.prev=54,e.t1=e.catch(3),e.t1 instanceof Error&&"AbortError"!==e.t1.name&&N("".concat(e.t1.name,": ").concat(e.t1.message));case 57:g(!1);case 58:case"end":return e.stop()}}),e,null,[[3,54],[12,34,38,48],[39,,43,47]])})));return function(t){return e.apply(this,arguments)}}(),P=ue(jf()(I,300),[]),R=ae((function(){var e=null!==t&&void 0!==t?t:u,n="chart"===(r||s);if(l)if(p)if(e.every((function(e){return!e.trim()})))N(_r.validQuery);else{if(Ha(p)){var o=yr({},l);return o.step=i,e.map((function(e){return n?function(e,t,n,r,i){return"".concat(e,"/api/v1/query_range?query=").concat(encodeURIComponent(t),"&start=").concat(n.start,"&end=").concat(n.end,"&step=").concat(n.step).concat(r?"&nocache=1":"").concat(i?"&trace=1":"")}(p,e,o,f,d):function(e,t,n,r){return"".concat(e,"/api/v1/query?query=").concat(encodeURIComponent(t),"&time=").concat(n.end,"&step=").concat(n.step).concat(r?"&trace=1":"")}(p,e,o,d)}))}N(_r.validServer)}else N(_r.emptyServer)}),[p,l,s,i,o]),z=Ft(ee([]),2),j=z[0],$=z[1];return ne((function(){var e=R===j&&!!t;n&&null!==R&&void 0!==R&&R.length&&!e&&(g(!0),P({fetchUrl:R,fetchQueue:B,displayType:r||s,query:null!==t&&void 0!==t?t:u,stateSeriesLimits:h,showAllSeries:a,hideQuery:o}),$(R))}),[R,n,h,a]),ne((function(){var e=B.slice(0,-1);e.length&&(e.map((function(e){return e.abort()})),L(B.filter((function(e){return!e.signal.aborted}))))}),[B]),{fetchUrl:R,isLoading:m,graphData:_,liveData:w,error:S,warning:M,traces:C}},Yf=function(e){var t=e.data,n=ra().showInfoMessage,r=ae((function(){return JSON.stringify(t,null,2)}),[t]);return Vr("div",{className:"vm-json-view",children:[Vr("div",{className:"vm-json-view__copy",children:Vr(fa,{variant:"outlined",onClick:function(){navigator.clipboard.writeText(r),n({text:"Formatted JSON has been copied",type:"success"})},children:"Copy JSON"})}),Vr("pre",{className:"vm-json-view__code",children:Vr("code",{children:r})})]})},Vf=function(e){var t=e.yaxis,n=e.setYaxisLimits,r=e.toggleEnableLimits,i=Xo().isMobile,o=ae((function(){return Object.keys(t.limits.range)}),[t.limits.range]),a=ue(jf()((function(e,r,i){var o=t.limits.range;o[r][i]=+e,o[r][0]===o[r][1]||o[r][0]>o[r][1]||n(o)}),500),[t.limits.range]),u=function(e,t){return function(n){a(n,e,t)}};return Vr("div",{className:wo()({"vm-axes-limits":!0,"vm-axes-limits_mobile":i}),children:[Vr(Tf,{value:t.limits.enable,onChange:r,label:"Fix the limits for y-axis",fullWidth:i}),Vr("div",{className:"vm-axes-limits-list",children:o.map((function(e){return Vr("div",{className:"vm-axes-limits-list__inputs",children:[Vr(xa,{label:"Min ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][0],onChange:u(e,0)}),Vr(xa,{label:"Max ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][1],onChange:u(e,1)})]},e)}))})]})},qf="Axes settings",Wf=function(e){var t=e.yaxis,n=e.setYaxisLimits,r=e.toggleEnableLimits,i=ie(null),o=Ft(ee(!1),2),a=o[0],u=o[1],l=ie(null);return Vr("div",{className:"vm-graph-settings",children:[Vr(ma,{title:qf,children:Vr("div",{ref:l,children:Vr(fa,{variant:"text",startIcon:Vr(zi,{}),onClick:function(){u((function(e){return!e}))}})})}),Vr(da,{open:a,buttonRef:l,placement:"bottom-right",onClose:function(){u(!1)},title:qf,children:Vr("div",{className:"vm-graph-settings-popper",ref:i,children:Vr("div",{className:"vm-graph-settings-popper__body",children:Vr(Vf,{yaxis:t,setYaxisLimits:n,toggleEnableLimits:r})})})})]})},Qf=function(e){var t=e.containerStyles,n=void 0===t?{}:t,r=e.message,i=Wr().isDarkTheme;return Vr("div",{className:wo()({"vm-spinner":!0,"vm-spinner_dark":i}),style:n&&{},children:[Vr("div",{className:"half-circle-spinner",children:[Vr("div",{className:"circle circle-1"}),Vr("div",{className:"circle circle-2"})]}),r&&Vr("div",{className:"vm-spinner__message",children:r})]})},Gf=function(){var e=Wr().serverUrl,t=Ft(ee([]),2),n=t[0],r=t[1],i=function(){var t=tu(Xa().mark((function t(){var n,i,o;return Xa().wrap((function(t){for(;;)switch(t.prev=t.next){case 0:if(e){t.next=2;break}return t.abrupt("return");case 2:return n="".concat(e,"/api/v1/label/__name__/values"),t.prev=3,t.next=6,fetch(n);case 6:return i=t.sent,t.next=9,i.json();case 9:o=t.sent,i.ok&&r(o.data),t.next=16;break;case 13:t.prev=13,t.t0=t.catch(3),console.error(t.t0);case 16:case"end":return t.stop()}}),t,null,[[3,13]])})));return function(){return t.apply(this,arguments)}}();return ne((function(){i()}),[e]),{queryOptions:n}},Jf=function(e){var t=e.value;return Vr("div",{className:"vm-line-progress",children:[Vr("div",{className:"vm-line-progress-track",children:Vr("div",{className:"vm-line-progress-track__thumb",style:{width:"".concat(t,"%")}})}),Vr("span",{children:[t.toFixed(2),"%"]})]})},Zf=function e(t){var n,r=t.trace,i=t.totalMsec,o=Wr().isDarkTheme,a=Xo().isMobile,u=Ft(ee({}),2),l=u[0],c=u[1],s=r.children&&!!r.children.length,f=r.duration/i*100;return Vr("div",{className:wo()({"vm-nested-nav":!0,"vm-nested-nav_dark":o,"vm-nested-nav_mobile":a}),children:[Vr("div",{className:"vm-nested-nav-header",onClick:(n=r.idValue,function(){c((function(e){return yr(yr({},e),{},mr({},n,!e[n]))}))}),children:[s&&Vr("div",{className:wo()({"vm-nested-nav-header__icon":!0,"vm-nested-nav-header__icon_open":l[r.idValue]}),children:Vr(Wi,{})}),Vr("div",{className:"vm-nested-nav-header__progress",children:Vr(Jf,{value:f})}),Vr("div",{className:"vm-nested-nav-header__message",children:r.message}),Vr("div",{className:"vm-nested-nav-header__duration",children:"duration: ".concat(r.duration," ms")})]}),l[r.idValue]&&Vr("div",{children:s&&r.children.map((function(t){return Vr(e,{trace:t,totalMsec:i},t.duration)}))})]})},Kf=function(e){var t=e.editable,n=void 0!==t&&t,r=e.defaultTile,i=void 0===r?"JSON":r,o=e.displayTitle,a=void 0===o||o,u=e.defaultJson,l=void 0===u?"":u,c=e.resetValue,f=void 0===c?"":c,d=e.onClose,h=e.onUpload,p=ra().showInfoMessage,v=Xo().isMobile,m=Ft(ee(l),2),g=m[0],y=m[1],_=Ft(ee(i),2),b=_[0],D=_[1],w=Ft(ee(""),2),x=w[0],k=w[1],C=Ft(ee(""),2),A=C[0],E=C[1],S=ae((function(){try{var e=JSON.parse(g),t=e.trace||e;return t.duration_msec?(new Hf(t,""),""):_r.traceNotFound}catch(s){return s instanceof Error?s.message:"Unknown error"}}),[g]),N=function(){var e=tu(Xa().mark((function e(){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(g);case 2:p({text:"Formatted JSON has been copied",type:"success"});case 3:case"end":return e.stop()}}),e)})));return function(){return e.apply(this,arguments)}}(),F=function(){E(S),b.trim()||k(_r.emptyTitle),S||x||(h(g,b),d())};return Vr("div",{className:wo()({"vm-json-form":!0,"vm-json-form_one-field":!a,"vm-json-form_one-field_mobile":!a&&v,"vm-json-form_mobile":v}),children:[a&&Vr(xa,{value:b,label:"Title",error:x,onEnter:F,onChange:function(e){D(e)}}),Vr(xa,{value:g,label:"JSON",type:"textarea",error:A,autofocus:!0,onChange:function(e){E(""),y(e)},disabled:!n}),Vr("div",{className:"vm-json-form-footer",children:[Vr("div",{className:"vm-json-form-footer__controls",children:[Vr(fa,{variant:"outlined",startIcon:Vr(so,{}),onClick:N,children:"Copy JSON"}),f&&Vr(fa,{variant:"text",startIcon:Vr($i,{}),onClick:function(){y(f)},children:"Reset JSON"})]}),Vr("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Vr(fa,{variant:"outlined",color:"error",onClick:d,children:"Cancel"}),Vr(fa,{variant:"contained",onClick:F,children:"apply"})]})]})]})},Xf=function(e){var t=e.traces,n=e.jsonEditor,r=void 0!==n&&n,i=e.onDeleteClick,o=Xo().isMobile,a=Ft(ee(null),2),u=a[0],l=a[1],c=function(){l(null)};if(!t.length)return Vr(ta,{variant:"info",children:"Please re-run the query to see results of the tracing"});var f=function(e){return function(){i(e)}};return Vr(g,{children:[Vr("div",{className:"vm-tracings-view",children:t.map((function(e){return Vr("div",{className:"vm-tracings-view-trace vm-block vm-block_empty-padding",children:[Vr("div",{className:"vm-tracings-view-trace-header",children:[Vr("h3",{className:"vm-tracings-view-trace-header-title",children:["Trace for ",Vr("b",{className:"vm-tracings-view-trace-header-title__query",children:e.queryValue})]}),Vr(ma,{title:"Open JSON",children:Vr(fa,{variant:"text",startIcon:Vr(ro,{}),onClick:(t=e,function(){l(t)})})}),Vr(ma,{title:"Remove trace",children:Vr(fa,{variant:"text",color:"error",startIcon:Vr(io,{}),onClick:f(e)})})]}),Vr("nav",{className:wo()({"vm-tracings-view-trace__nav":!0,"vm-tracings-view-trace__nav_mobile":o}),children:Vr(Zf,{trace:e,totalMsec:e.duration})})]},e.idValue);var t}))}),u&&Vr(va,{title:u.queryValue,onClose:c,children:Vr(Kf,{editable:r,displayTitle:r,defaultTile:u.queryValue,defaultJson:u.JSON,resetValue:u.originalJSON,onClose:c,onUpload:function(e,t){if(r&&u)try{u.setTracing(JSON.parse(e)),u.setQuery(t),l(null)}catch(s){console.error(s)}}})})]})},ed=function(e,t){return ae((function(){var n={};e.forEach((function(e){return Object.entries(e.metric).forEach((function(e){return n[e[0]]?n[e[0]].options.add(e[1]):n[e[0]]={options:new Set([e[1]])}}))}));var r=Object.entries(n).map((function(e){return{key:e[0],variations:e[1].options.size}})).sort((function(e,t){return e.variations-t.variations}));return t?r.filter((function(e){return t.includes(e.key)})):r}),[e,t])},td=function(e){var t,n=e.checked,r=void 0!==n&&n,i=e.disabled,o=void 0!==i&&i,a=e.label,u=e.color,l=void 0===u?"secondary":u,c=e.onChange;return Vr("div",{className:wo()((mr(t={"vm-checkbox":!0,"vm-checkbox_disabled":o,"vm-checkbox_active":r},"vm-checkbox_".concat(l,"_active"),r),mr(t,"vm-checkbox_".concat(l),l),t)),onClick:function(){o||c(!r)},children:[Vr("div",{className:"vm-checkbox-track",children:Vr("div",{className:"vm-checkbox-track__thumb",children:Vr(uo,{})})}),a&&Vr("span",{className:"vm-checkbox__label",children:a})]})},nd="Table settings",rd=function(e){var t=e.data,n=e.defaultColumns,r=void 0===n?[]:n,i=e.onChange,o=Xo().isMobile,a=Io().tableCompact,u=Po(),l=ed(t),c=ie(null),s=Ft(ee(!1),2),f=s[0],d=s[1],h=ae((function(){return!l.length}),[l]),p=function(e){return function(){!function(e){i(r.includes(e)?r.filter((function(t){return t!==e})):[].concat(Ot(r),[e]))}(e)}};return ne((function(){var e=l.map((function(e){return e.key}));Lf(e,r)||i(e)}),[l]),Vr("div",{className:"vm-table-settings",children:[Vr(ma,{title:nd,children:Vr("div",{ref:c,children:Vr(fa,{variant:"text",startIcon:Vr(zi,{}),onClick:function(){d((function(e){return!e}))},disabled:h})})}),Vr(da,{open:f,onClose:function(){d(!1)},placement:"bottom-right",buttonRef:c,title:nd,children:Vr("div",{className:wo()({"vm-table-settings-popper":!0,"vm-table-settings-popper_mobile":o}),children:[Vr("div",{className:"vm-table-settings-popper-list vm-table-settings-popper-list_first",children:Vr(Tf,{label:"Compact view",value:a,onChange:function(){u({type:"TOGGLE_TABLE_COMPACT"})}})}),Vr("div",{className:"vm-table-settings-popper-list",children:[Vr("div",{className:"vm-table-settings-popper-list-header",children:[Vr("h3",{className:"vm-table-settings-popper-list-header__title",children:"Display columns"}),Vr(ma,{title:"Reset to default",children:Vr(fa,{color:"primary",variant:"text",size:"small",onClick:function(){d(!1),i(l.map((function(e){return e.key})))},startIcon:Vr($i,{})})})]}),l.map((function(e){return Vr("div",{className:"vm-table-settings-popper-list__item",children:Vr(td,{checked:r.includes(e.key),onChange:p(e.key),label:e.key,disabled:a})},e.key)}))]})]})})]})};function id(e){return function(e,t){return Object.fromEntries(Object.entries(e).filter(t))}(e,(function(e){return!!e[1]}))}var od=["__name__"],ad=function(e){var t=e.data,n=e.displayColumns,r=ra().showInfoMessage,i=Xo().isMobile,o=Io().tableCompact,a=bo(document.body),u=ie(null),l=Ft(ee(0),2),c=l[0],s=l[1],f=Ft(ee(0),2),d=f[0],h=f[1],p=Ft(ee(""),2),v=p[0],m=p[1],g=Ft(ee("asc"),2),y=g[0],_=g[1],b=o?ed([{group:0,metric:{Data:"Data"}}],["Data"]):ed(t,n),D=function(e){var t=e.__name__,n=xo(e,od);return t||Object.keys(n).length?"".concat(t," ").concat(JSON.stringify(n)):""},w=new Set(null===t||void 0===t?void 0:t.map((function(e){return e.group}))),x=w.size>1,k=ae((function(){var e=null===t||void 0===t?void 0:t.map((function(e){return{metadata:b.map((function(t){return o?wf(e,"",x):e.metric[t.key]||"-"})),value:e.value?e.value[1]:"-",values:e.values?e.values.map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(r," @").concat(n)})):[],copyValue:D(e.metric)}})),n="Value"===v,r=b.findIndex((function(e){return e.key===v}));return n||-1!==r?e.sort((function(e,t){var i=n?Number(e.value):e.metadata[r],o=n?Number(t.value):t.metadata[r];return("asc"===y?io)?-1:1})):e}),[b,t,v,y,o]),C=ae((function(){return k.some((function(e){return e.copyValue}))}),[k]),A=function(){var e=tu(Xa().mark((function e(t){return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return e.next=2,navigator.clipboard.writeText(t);case 2:r({text:"Row has been copied",type:"success"});case 3:case"end":return e.stop()}}),e)})));return function(t){return e.apply(this,arguments)}}(),E=function(e){return function(){!function(e){_((function(t){return"asc"===t&&v===e?"desc":"asc"})),m(e)}(e)}},S=function(){if(u.current){var e=u.current.getBoundingClientRect().top;h(e<0?window.scrollY-c:0)}};return ne((function(){return window.addEventListener("scroll",S),function(){window.removeEventListener("scroll",S)}}),[u,c,a]),ne((function(){if(u.current){var e=u.current.getBoundingClientRect().top;s(e+window.scrollY)}}),[u,a]),k.length?Vr("div",{className:wo()({"vm-table-view":!0,"vm-table-view_mobile":i}),children:Vr("table",{className:"vm-table",ref:u,children:[Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",style:{transform:"translateY(".concat(d,"px)")},children:[b.map((function(e,t){return Vr("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:E(e.key),children:Vr("div",{className:"vm-table-cell__content",children:[e.key,Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":v===e.key,"vm-table__sort-icon_desc":"desc"===y&&v===e.key}),children:Vr(Qi,{})})]})},t)})),Vr("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_right vm-table-cell_sort",onClick:E("Value"),children:Vr("div",{className:"vm-table-cell__content",children:[Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":"Value"===v,"vm-table__sort-icon_desc":"desc"===y}),children:Vr(Qi,{})}),"Value"]})}),C&&Vr("td",{className:"vm-table-cell vm-table-cell_header"})]})}),Vr("tbody",{className:"vm-table-body",children:k.map((function(e,t){return Vr("tr",{className:"vm-table__row",children:[e.metadata.map((function(e,n){return Vr("td",{className:wo()({"vm-table-cell vm-table-cell_no-wrap":!0,"vm-table-cell_gray":k[t-1]&&k[t-1].metadata[n]===e}),children:e},n)})),Vr("td",{className:"vm-table-cell vm-table-cell_right vm-table-cell_no-wrap",children:e.values.length?e.values.map((function(e){return Vr("p",{children:e},e)})):e.value}),C&&Vr("td",{className:"vm-table-cell vm-table-cell_right",children:e.copyValue&&Vr("div",{className:"vm-table-cell__content",children:Vr(ma,{title:"Copy row",children:Vr(fa,{variant:"text",color:"gray",size:"small",startIcon:Vr(so,{}),onClick:(n=e.copyValue,function(){A(n)})})})})})]},t);var n}))})]})}):Vr(ta,{variant:"warning",children:"No data to show"})},ud=function(){var e=Io(),t=e.displayType,n=e.isTracingEnabled,r=Li().query,i=Ni().period,o=Fi(),a=Xo().isMobile;!function(){var e=Wr().tenantId,t=Io().displayType,n=Li().query,r=Ni(),i=r.duration,o=r.relativeTime,a=r.period,u=a.date,l=a.step,c=$o().customStep,s=Ft(pr(),2)[1],f=function(){var r={};n.forEach((function(n,a){var s,f="g".concat(a);r["".concat(f,".expr")]=n,r["".concat(f,".range_input")]=i,r["".concat(f,".end_input")]=u,r["".concat(f,".tab")]=(null===(s=So.find((function(e){return e.value===t})))||void 0===s?void 0:s.prometheusCode)||0,r["".concat(f,".relative_time")]=o,r["".concat(f,".tenantID")]=e,l!==c&&c&&(r["".concat(f,".step_input")]=c)})),s(id(r))};ne(f,[e,t,n,i,o,u,l,c]),ne(f,[])}();var u=Ft(ee(),2),l=u[0],c=u[1],s=Ft(ee([]),2),f=s[0],d=s[1],h=Ft(ee([]),2),p=h[0],v=h[1],m=Ft(ee(!1),2),g=m[0],y=m[1],_=Ft(ee(!r[0]),2),b=_[0],D=_[1],w=$o(),x=w.customStep,k=w.yaxis,C=Ho(),A=Gf().queryOptions,E=Uf({visible:!0,customStep:x,hideQuery:p,showAllSeries:g}),S=E.isLoading,N=E.liveData,F=E.graphData,M=E.error,T=E.warning,O=E.traces,B=function(e){C({type:"SET_YAXIS_LIMITS",payload:e})};return ne((function(){O&&d([].concat(Ot(f),Ot(O)))}),[O]),ne((function(){d([])}),[t]),ne((function(){y(!1)}),[r]),Vr("div",{className:wo()({"vm-custom-panel":!0,"vm-custom-panel_mobile":a}),children:[Vr(If,{error:b?"":M,queryOptions:A,onHideQuery:function(e){v(e)},onRunQuery:function(){D(!1)}}),n&&Vr("div",{className:"vm-custom-panel__trace",children:Vr(Xf,{traces:f,onDeleteClick:function(e){var t=f.filter((function(t){return t.idValue!==e.idValue}));d(Ot(t))}})}),S&&Vr(Qf,{}),!b&&M&&Vr(ta,{variant:"error",children:M}),T&&Vr(ta,{variant:"warning",children:Vr("div",{className:wo()({"vm-custom-panel__warning":!0,"vm-custom-panel__warning_mobile":a}),children:[Vr("p",{children:T}),Vr(fa,{color:"warning",variant:"outlined",onClick:function(){y(!0)},children:"Show all"})]})}),Vr("div",{className:wo()({"vm-custom-panel-body":!0,"vm-custom-panel-body_mobile":a,"vm-block":!0,"vm-block_mobile":a}),children:[Vr("div",{className:"vm-custom-panel-body-header",children:[Vr(No,{}),"chart"===t&&Vr(Wf,{yaxis:k,setYaxisLimits:B,toggleEnableLimits:function(){C({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})}}),"table"===t&&Vr(rd,{data:N||[],defaultColumns:l,onChange:c})]}),F&&i&&"chart"===t&&Vr(Nf,{data:F,period:i,customStep:x,query:r,yaxis:k,setYaxisLimits:B,setPeriod:function(e){var t=e.from,n=e.to;o({type:"SET_PERIOD",payload:{from:t,to:n}})},height:a?.5*window.innerHeight:500}),N&&"code"===t&&Vr(Yf,{data:N}),N&&"table"===t&&Vr(ad,{data:N,displayColumns:l})]})]})};function ld(){return{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}}var cd={async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1};var sd=/[&<>"']/,fd=new RegExp(sd.source,"g"),dd=/[<>"']|&(?!(#\d{1,7}|#[Xx][a-fA-F0-9]{1,6}|\w+);)/,hd=new RegExp(dd.source,"g"),pd={"&":"&","<":"<",">":">",'"':""","'":"'"},vd=function(e){return pd[e]};function md(e,t){if(t){if(sd.test(e))return e.replace(fd,vd)}else if(dd.test(e))return e.replace(hd,vd);return e}var gd=/&(#(?:\d+)|(?:#x[0-9A-Fa-f]+)|(?:\w+));?/gi;function yd(e){return e.replace(gd,(function(e,t){return"colon"===(t=t.toLowerCase())?":":"#"===t.charAt(0)?"x"===t.charAt(1)?String.fromCharCode(parseInt(t.substring(2),16)):String.fromCharCode(+t.substring(1)):""}))}var _d=/(^|[^\[])\^/g;function bd(e,t){e="string"===typeof e?e:e.source,t=t||"";var n={replace:function(t,r){return r=(r=r.source||r).replace(_d,"$1"),e=e.replace(t,r),n},getRegex:function(){return new RegExp(e,t)}};return n}var Dd=/[^\w:]/g,wd=/^$|^[a-z][a-z0-9+.-]*:|^[?#]/i;function xd(e,t,n){if(e){var r;try{r=decodeURIComponent(yd(n)).replace(Dd,"").toLowerCase()}catch(s){return null}if(0===r.indexOf("javascript:")||0===r.indexOf("vbscript:")||0===r.indexOf("data:"))return null}t&&!wd.test(n)&&(n=function(e,t){kd[" "+e]||(Cd.test(e)?kd[" "+e]=e+"/":kd[" "+e]=Md(e,"/",!0));e=kd[" "+e];var n=-1===e.indexOf(":");return"//"===t.substring(0,2)?n?t:e.replace(Ad,"$1")+t:"/"===t.charAt(0)?n?t:e.replace(Ed,"$1")+t:e+t}(t,n));try{n=encodeURI(n).replace(/%25/g,"%")}catch(s){return null}return n}var kd={},Cd=/^[^:]+:\/*[^/]*$/,Ad=/^([^:]+:)[\s\S]*$/,Ed=/^([^:]+:\/*[^/]*)[\s\S]*$/;var Sd={exec:function(){}};function Nd(e){for(var t,n,r=1;r=0&&"\\"===n[i];)r=!r;return r?"|":" |"})).split(/ \|/),r=0;if(n[0].trim()||n.shift(),n.length>0&&!n[n.length-1].trim()&&n.pop(),n.length>t)n.splice(t);else for(;n.length1;)1&t&&(n+=e),t>>=1,e+=e;return n+e}function Bd(e,t,n,r){var i=t.href,o=t.title?md(t.title):null,a=e[1].replace(/\\([\[\]])/g,"$1");if("!"!==e[0].charAt(0)){r.state.inLink=!0;var u={type:"link",raw:n,href:i,title:o,text:a,tokens:r.inlineTokens(a)};return r.state.inLink=!1,u}return{type:"image",raw:n,href:i,title:o,text:md(a)}}var Ld=function(){function e(t){Bt(this,e),this.options=t||cd}return Rt(e,[{key:"space",value:function(e){var t=this.rules.block.newline.exec(e);if(t&&t[0].length>0)return{type:"space",raw:t[0]}}},{key:"code",value:function(e){var t=this.rules.block.code.exec(e);if(t){var n=t[0].replace(/^ {1,4}/gm,"");return{type:"code",raw:t[0],codeBlockStyle:"indented",text:this.options.pedantic?n:Md(n,"\n")}}}},{key:"fences",value:function(e){var t=this.rules.block.fences.exec(e);if(t){var n=t[0],r=function(e,t){var n=e.match(/^(\s+)(?:```)/);if(null===n)return t;var r=n[1];return t.split("\n").map((function(e){var t=e.match(/^\s+/);return null===t?e:Ft(t,1)[0].length>=r.length?e.slice(r.length):e})).join("\n")}(n,t[3]||"");return{type:"code",raw:n,lang:t[2]?t[2].trim().replace(this.rules.inline._escapes,"$1"):t[2],text:r}}}},{key:"heading",value:function(e){var t=this.rules.block.heading.exec(e);if(t){var n=t[2].trim();if(/#$/.test(n)){var r=Md(n,"#");this.options.pedantic?n=r.trim():r&&!/ $/.test(r)||(n=r.trim())}return{type:"heading",raw:t[0],depth:t[1].length,text:n,tokens:this.lexer.inline(n)}}}},{key:"hr",value:function(e){var t=this.rules.block.hr.exec(e);if(t)return{type:"hr",raw:t[0]}}},{key:"blockquote",value:function(e){var t=this.rules.block.blockquote.exec(e);if(t){var n=t[0].replace(/^ *>[ \t]?/gm,""),r=this.lexer.state.top;this.lexer.state.top=!0;var i=this.lexer.blockTokens(n);return this.lexer.state.top=r,{type:"blockquote",raw:t[0],tokens:i,text:n}}}},{key:"list",value:function(e){var t=this.rules.block.list.exec(e);if(t){var n,r,i,o,a,u,l,c,s,f,d,h,p=t[1].trim(),v=p.length>1,m={type:"list",raw:"",ordered:v,start:v?+p.slice(0,-1):"",loose:!1,items:[]};p=v?"\\d{1,9}\\".concat(p.slice(-1)):"\\".concat(p),this.options.pedantic&&(p=v?p:"[*+-]");for(var g=new RegExp("^( {0,3}".concat(p,")((?:[\t ][^\\n]*)?(?:\\n|$))"));e&&(h=!1,t=g.exec(e))&&!this.rules.block.hr.test(e);){if(n=t[0],e=e.substring(n.length),c=t[2].split("\n",1)[0].replace(/^\t+/,(function(e){return" ".repeat(3*e.length)})),s=e.split("\n",1)[0],this.options.pedantic?(o=2,d=c.trimLeft()):(o=(o=t[2].search(/[^ ]/))>4?1:o,d=c.slice(o),o+=t[1].length),u=!1,!c&&/^ *$/.test(s)&&(n+=s+"\n",e=e.substring(s.length+1),h=!0),!h)for(var y=new RegExp("^ {0,".concat(Math.min(3,o-1),"}(?:[*+-]|\\d{1,9}[.)])((?:[ \t][^\\n]*)?(?:\\n|$))")),_=new RegExp("^ {0,".concat(Math.min(3,o-1),"}((?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$)")),b=new RegExp("^ {0,".concat(Math.min(3,o-1),"}(?:```|~~~)")),D=new RegExp("^ {0,".concat(Math.min(3,o-1),"}#"));e&&(s=f=e.split("\n",1)[0],this.options.pedantic&&(s=s.replace(/^ {1,4}(?=( {4})*[^ ])/g," ")),!b.test(s))&&!D.test(s)&&!y.test(s)&&!_.test(e);){if(s.search(/[^ ]/)>=o||!s.trim())d+="\n"+s.slice(o);else{if(u)break;if(c.search(/[^ ]/)>=4)break;if(b.test(c))break;if(D.test(c))break;if(_.test(c))break;d+="\n"+s}u||s.trim()||(u=!0),n+=f+"\n",e=e.substring(f.length+1),c=s.slice(o)}m.loose||(l?m.loose=!0:/\n *\n *$/.test(n)&&(l=!0)),this.options.gfm&&(r=/^\[[ xX]\] /.exec(d))&&(i="[ ] "!==r[0],d=d.replace(/^\[[ xX]\] +/,"")),m.items.push({type:"list_item",raw:n,task:!!r,checked:i,loose:!1,text:d}),m.raw+=n}m.items[m.items.length-1].raw=n.trimRight(),m.items[m.items.length-1].text=d.trimRight(),m.raw=m.raw.trimRight();var w=m.items.length;for(a=0;a0&&x.some((function(e){return/\n.*\n/.test(e.raw)}));m.loose=k}if(m.loose)for(a=0;a$/,"$1").replace(this.rules.inline._escapes,"$1"):"",i=t[3]?t[3].substring(1,t[3].length-1).replace(this.rules.inline._escapes,"$1"):t[3];return{type:"def",tag:n,raw:t[0],href:r,title:i}}}},{key:"table",value:function(e){var t=this.rules.block.table.exec(e);if(t){var n={type:"table",header:Fd(t[1]).map((function(e){return{text:e}})),align:t[2].replace(/^ *|\| *$/g,"").split(/ *\| */),rows:t[3]&&t[3].trim()?t[3].replace(/\n[ \t]*$/,"").split("\n"):[]};if(n.header.length===n.align.length){n.raw=t[0];var r,i,o,a,u=n.align.length;for(r=0;r/i.test(t[0])&&(this.lexer.state.inLink=!1),!this.lexer.state.inRawBlock&&/^<(pre|code|kbd|script)(\s|>)/i.test(t[0])?this.lexer.state.inRawBlock=!0:this.lexer.state.inRawBlock&&/^<\/(pre|code|kbd|script)(\s|>)/i.test(t[0])&&(this.lexer.state.inRawBlock=!1),{type:this.options.sanitize?"text":"html",raw:t[0],inLink:this.lexer.state.inLink,inRawBlock:this.lexer.state.inRawBlock,text:this.options.sanitize?this.options.sanitizer?this.options.sanitizer(t[0]):md(t[0]):t[0]}}},{key:"link",value:function(e){var t=this.rules.inline.link.exec(e);if(t){var n=t[2].trim();if(!this.options.pedantic&&/^$/.test(n))return;var r=Md(n.slice(0,-1),"\\");if((n.length-r.length)%2===0)return}else{var i=function(e,t){if(-1===e.indexOf(t[1]))return-1;for(var n=e.length,r=0,i=0;i-1){var o=(0===t[0].indexOf("!")?5:4)+t[1].length+i;t[2]=t[2].substring(0,i),t[0]=t[0].substring(0,o).trim(),t[3]=""}}var a=t[2],u="";if(this.options.pedantic){var l=/^([^'"]*[^\s])\s+(['"])(.*)\2/.exec(a);l&&(a=l[1],u=l[3])}else u=t[3]?t[3].slice(1,-1):"";return a=a.trim(),/^$/.test(n)?a.slice(1):a.slice(1,-1)),Bd(t,{href:a?a.replace(this.rules.inline._escapes,"$1"):a,title:u?u.replace(this.rules.inline._escapes,"$1"):u},t[0],this.lexer)}}},{key:"reflink",value:function(e,t){var n;if((n=this.rules.inline.reflink.exec(e))||(n=this.rules.inline.nolink.exec(e))){var r=(n[2]||n[1]).replace(/\s+/g," ");if(!(r=t[r.toLowerCase()])){var i=n[0].charAt(0);return{type:"text",raw:i,text:i}}return Bd(n,r,n[0],this.lexer)}}},{key:"emStrong",value:function(e,t){var n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"",r=this.rules.inline.emStrong.lDelim.exec(e);if(r&&(!r[3]||!n.match(/(?:[0-9A-Za-z\xAA\xB2\xB3\xB5\xB9\xBA\xBC-\xBE\xC0-\xD6\xD8-\xF6\xF8-\u02C1\u02C6-\u02D1\u02E0-\u02E4\u02EC\u02EE\u0370-\u0374\u0376\u0377\u037A-\u037D\u037F\u0386\u0388-\u038A\u038C\u038E-\u03A1\u03A3-\u03F5\u03F7-\u0481\u048A-\u052F\u0531-\u0556\u0559\u0560-\u0588\u05D0-\u05EA\u05EF-\u05F2\u0620-\u064A\u0660-\u0669\u066E\u066F\u0671-\u06D3\u06D5\u06E5\u06E6\u06EE-\u06FC\u06FF\u0710\u0712-\u072F\u074D-\u07A5\u07B1\u07C0-\u07EA\u07F4\u07F5\u07FA\u0800-\u0815\u081A\u0824\u0828\u0840-\u0858\u0860-\u086A\u0870-\u0887\u0889-\u088E\u08A0-\u08C9\u0904-\u0939\u093D\u0950\u0958-\u0961\u0966-\u096F\u0971-\u0980\u0985-\u098C\u098F\u0990\u0993-\u09A8\u09AA-\u09B0\u09B2\u09B6-\u09B9\u09BD\u09CE\u09DC\u09DD\u09DF-\u09E1\u09E6-\u09F1\u09F4-\u09F9\u09FC\u0A05-\u0A0A\u0A0F\u0A10\u0A13-\u0A28\u0A2A-\u0A30\u0A32\u0A33\u0A35\u0A36\u0A38\u0A39\u0A59-\u0A5C\u0A5E\u0A66-\u0A6F\u0A72-\u0A74\u0A85-\u0A8D\u0A8F-\u0A91\u0A93-\u0AA8\u0AAA-\u0AB0\u0AB2\u0AB3\u0AB5-\u0AB9\u0ABD\u0AD0\u0AE0\u0AE1\u0AE6-\u0AEF\u0AF9\u0B05-\u0B0C\u0B0F\u0B10\u0B13-\u0B28\u0B2A-\u0B30\u0B32\u0B33\u0B35-\u0B39\u0B3D\u0B5C\u0B5D\u0B5F-\u0B61\u0B66-\u0B6F\u0B71-\u0B77\u0B83\u0B85-\u0B8A\u0B8E-\u0B90\u0B92-\u0B95\u0B99\u0B9A\u0B9C\u0B9E\u0B9F\u0BA3\u0BA4\u0BA8-\u0BAA\u0BAE-\u0BB9\u0BD0\u0BE6-\u0BF2\u0C05-\u0C0C\u0C0E-\u0C10\u0C12-\u0C28\u0C2A-\u0C39\u0C3D\u0C58-\u0C5A\u0C5D\u0C60\u0C61\u0C66-\u0C6F\u0C78-\u0C7E\u0C80\u0C85-\u0C8C\u0C8E-\u0C90\u0C92-\u0CA8\u0CAA-\u0CB3\u0CB5-\u0CB9\u0CBD\u0CDD\u0CDE\u0CE0\u0CE1\u0CE6-\u0CEF\u0CF1\u0CF2\u0D04-\u0D0C\u0D0E-\u0D10\u0D12-\u0D3A\u0D3D\u0D4E\u0D54-\u0D56\u0D58-\u0D61\u0D66-\u0D78\u0D7A-\u0D7F\u0D85-\u0D96\u0D9A-\u0DB1\u0DB3-\u0DBB\u0DBD\u0DC0-\u0DC6\u0DE6-\u0DEF\u0E01-\u0E30\u0E32\u0E33\u0E40-\u0E46\u0E50-\u0E59\u0E81\u0E82\u0E84\u0E86-\u0E8A\u0E8C-\u0EA3\u0EA5\u0EA7-\u0EB0\u0EB2\u0EB3\u0EBD\u0EC0-\u0EC4\u0EC6\u0ED0-\u0ED9\u0EDC-\u0EDF\u0F00\u0F20-\u0F33\u0F40-\u0F47\u0F49-\u0F6C\u0F88-\u0F8C\u1000-\u102A\u103F-\u1049\u1050-\u1055\u105A-\u105D\u1061\u1065\u1066\u106E-\u1070\u1075-\u1081\u108E\u1090-\u1099\u10A0-\u10C5\u10C7\u10CD\u10D0-\u10FA\u10FC-\u1248\u124A-\u124D\u1250-\u1256\u1258\u125A-\u125D\u1260-\u1288\u128A-\u128D\u1290-\u12B0\u12B2-\u12B5\u12B8-\u12BE\u12C0\u12C2-\u12C5\u12C8-\u12D6\u12D8-\u1310\u1312-\u1315\u1318-\u135A\u1369-\u137C\u1380-\u138F\u13A0-\u13F5\u13F8-\u13FD\u1401-\u166C\u166F-\u167F\u1681-\u169A\u16A0-\u16EA\u16EE-\u16F8\u1700-\u1711\u171F-\u1731\u1740-\u1751\u1760-\u176C\u176E-\u1770\u1780-\u17B3\u17D7\u17DC\u17E0-\u17E9\u17F0-\u17F9\u1810-\u1819\u1820-\u1878\u1880-\u1884\u1887-\u18A8\u18AA\u18B0-\u18F5\u1900-\u191E\u1946-\u196D\u1970-\u1974\u1980-\u19AB\u19B0-\u19C9\u19D0-\u19DA\u1A00-\u1A16\u1A20-\u1A54\u1A80-\u1A89\u1A90-\u1A99\u1AA7\u1B05-\u1B33\u1B45-\u1B4C\u1B50-\u1B59\u1B83-\u1BA0\u1BAE-\u1BE5\u1C00-\u1C23\u1C40-\u1C49\u1C4D-\u1C7D\u1C80-\u1C88\u1C90-\u1CBA\u1CBD-\u1CBF\u1CE9-\u1CEC\u1CEE-\u1CF3\u1CF5\u1CF6\u1CFA\u1D00-\u1DBF\u1E00-\u1F15\u1F18-\u1F1D\u1F20-\u1F45\u1F48-\u1F4D\u1F50-\u1F57\u1F59\u1F5B\u1F5D\u1F5F-\u1F7D\u1F80-\u1FB4\u1FB6-\u1FBC\u1FBE\u1FC2-\u1FC4\u1FC6-\u1FCC\u1FD0-\u1FD3\u1FD6-\u1FDB\u1FE0-\u1FEC\u1FF2-\u1FF4\u1FF6-\u1FFC\u2070\u2071\u2074-\u2079\u207F-\u2089\u2090-\u209C\u2102\u2107\u210A-\u2113\u2115\u2119-\u211D\u2124\u2126\u2128\u212A-\u212D\u212F-\u2139\u213C-\u213F\u2145-\u2149\u214E\u2150-\u2189\u2460-\u249B\u24EA-\u24FF\u2776-\u2793\u2C00-\u2CE4\u2CEB-\u2CEE\u2CF2\u2CF3\u2CFD\u2D00-\u2D25\u2D27\u2D2D\u2D30-\u2D67\u2D6F\u2D80-\u2D96\u2DA0-\u2DA6\u2DA8-\u2DAE\u2DB0-\u2DB6\u2DB8-\u2DBE\u2DC0-\u2DC6\u2DC8-\u2DCE\u2DD0-\u2DD6\u2DD8-\u2DDE\u2E2F\u3005-\u3007\u3021-\u3029\u3031-\u3035\u3038-\u303C\u3041-\u3096\u309D-\u309F\u30A1-\u30FA\u30FC-\u30FF\u3105-\u312F\u3131-\u318E\u3192-\u3195\u31A0-\u31BF\u31F0-\u31FF\u3220-\u3229\u3248-\u324F\u3251-\u325F\u3280-\u3289\u32B1-\u32BF\u3400-\u4DBF\u4E00-\uA48C\uA4D0-\uA4FD\uA500-\uA60C\uA610-\uA62B\uA640-\uA66E\uA67F-\uA69D\uA6A0-\uA6EF\uA717-\uA71F\uA722-\uA788\uA78B-\uA7CA\uA7D0\uA7D1\uA7D3\uA7D5-\uA7D9\uA7F2-\uA801\uA803-\uA805\uA807-\uA80A\uA80C-\uA822\uA830-\uA835\uA840-\uA873\uA882-\uA8B3\uA8D0-\uA8D9\uA8F2-\uA8F7\uA8FB\uA8FD\uA8FE\uA900-\uA925\uA930-\uA946\uA960-\uA97C\uA984-\uA9B2\uA9CF-\uA9D9\uA9E0-\uA9E4\uA9E6-\uA9FE\uAA00-\uAA28\uAA40-\uAA42\uAA44-\uAA4B\uAA50-\uAA59\uAA60-\uAA76\uAA7A\uAA7E-\uAAAF\uAAB1\uAAB5\uAAB6\uAAB9-\uAABD\uAAC0\uAAC2\uAADB-\uAADD\uAAE0-\uAAEA\uAAF2-\uAAF4\uAB01-\uAB06\uAB09-\uAB0E\uAB11-\uAB16\uAB20-\uAB26\uAB28-\uAB2E\uAB30-\uAB5A\uAB5C-\uAB69\uAB70-\uABE2\uABF0-\uABF9\uAC00-\uD7A3\uD7B0-\uD7C6\uD7CB-\uD7FB\uF900-\uFA6D\uFA70-\uFAD9\uFB00-\uFB06\uFB13-\uFB17\uFB1D\uFB1F-\uFB28\uFB2A-\uFB36\uFB38-\uFB3C\uFB3E\uFB40\uFB41\uFB43\uFB44\uFB46-\uFBB1\uFBD3-\uFD3D\uFD50-\uFD8F\uFD92-\uFDC7\uFDF0-\uFDFB\uFE70-\uFE74\uFE76-\uFEFC\uFF10-\uFF19\uFF21-\uFF3A\uFF41-\uFF5A\uFF66-\uFFBE\uFFC2-\uFFC7\uFFCA-\uFFCF\uFFD2-\uFFD7\uFFDA-\uFFDC]|\uD800[\uDC00-\uDC0B\uDC0D-\uDC26\uDC28-\uDC3A\uDC3C\uDC3D\uDC3F-\uDC4D\uDC50-\uDC5D\uDC80-\uDCFA\uDD07-\uDD33\uDD40-\uDD78\uDD8A\uDD8B\uDE80-\uDE9C\uDEA0-\uDED0\uDEE1-\uDEFB\uDF00-\uDF23\uDF2D-\uDF4A\uDF50-\uDF75\uDF80-\uDF9D\uDFA0-\uDFC3\uDFC8-\uDFCF\uDFD1-\uDFD5]|\uD801[\uDC00-\uDC9D\uDCA0-\uDCA9\uDCB0-\uDCD3\uDCD8-\uDCFB\uDD00-\uDD27\uDD30-\uDD63\uDD70-\uDD7A\uDD7C-\uDD8A\uDD8C-\uDD92\uDD94\uDD95\uDD97-\uDDA1\uDDA3-\uDDB1\uDDB3-\uDDB9\uDDBB\uDDBC\uDE00-\uDF36\uDF40-\uDF55\uDF60-\uDF67\uDF80-\uDF85\uDF87-\uDFB0\uDFB2-\uDFBA]|\uD802[\uDC00-\uDC05\uDC08\uDC0A-\uDC35\uDC37\uDC38\uDC3C\uDC3F-\uDC55\uDC58-\uDC76\uDC79-\uDC9E\uDCA7-\uDCAF\uDCE0-\uDCF2\uDCF4\uDCF5\uDCFB-\uDD1B\uDD20-\uDD39\uDD80-\uDDB7\uDDBC-\uDDCF\uDDD2-\uDE00\uDE10-\uDE13\uDE15-\uDE17\uDE19-\uDE35\uDE40-\uDE48\uDE60-\uDE7E\uDE80-\uDE9F\uDEC0-\uDEC7\uDEC9-\uDEE4\uDEEB-\uDEEF\uDF00-\uDF35\uDF40-\uDF55\uDF58-\uDF72\uDF78-\uDF91\uDFA9-\uDFAF]|\uD803[\uDC00-\uDC48\uDC80-\uDCB2\uDCC0-\uDCF2\uDCFA-\uDD23\uDD30-\uDD39\uDE60-\uDE7E\uDE80-\uDEA9\uDEB0\uDEB1\uDF00-\uDF27\uDF30-\uDF45\uDF51-\uDF54\uDF70-\uDF81\uDFB0-\uDFCB\uDFE0-\uDFF6]|\uD804[\uDC03-\uDC37\uDC52-\uDC6F\uDC71\uDC72\uDC75\uDC83-\uDCAF\uDCD0-\uDCE8\uDCF0-\uDCF9\uDD03-\uDD26\uDD36-\uDD3F\uDD44\uDD47\uDD50-\uDD72\uDD76\uDD83-\uDDB2\uDDC1-\uDDC4\uDDD0-\uDDDA\uDDDC\uDDE1-\uDDF4\uDE00-\uDE11\uDE13-\uDE2B\uDE3F\uDE40\uDE80-\uDE86\uDE88\uDE8A-\uDE8D\uDE8F-\uDE9D\uDE9F-\uDEA8\uDEB0-\uDEDE\uDEF0-\uDEF9\uDF05-\uDF0C\uDF0F\uDF10\uDF13-\uDF28\uDF2A-\uDF30\uDF32\uDF33\uDF35-\uDF39\uDF3D\uDF50\uDF5D-\uDF61]|\uD805[\uDC00-\uDC34\uDC47-\uDC4A\uDC50-\uDC59\uDC5F-\uDC61\uDC80-\uDCAF\uDCC4\uDCC5\uDCC7\uDCD0-\uDCD9\uDD80-\uDDAE\uDDD8-\uDDDB\uDE00-\uDE2F\uDE44\uDE50-\uDE59\uDE80-\uDEAA\uDEB8\uDEC0-\uDEC9\uDF00-\uDF1A\uDF30-\uDF3B\uDF40-\uDF46]|\uD806[\uDC00-\uDC2B\uDCA0-\uDCF2\uDCFF-\uDD06\uDD09\uDD0C-\uDD13\uDD15\uDD16\uDD18-\uDD2F\uDD3F\uDD41\uDD50-\uDD59\uDDA0-\uDDA7\uDDAA-\uDDD0\uDDE1\uDDE3\uDE00\uDE0B-\uDE32\uDE3A\uDE50\uDE5C-\uDE89\uDE9D\uDEB0-\uDEF8]|\uD807[\uDC00-\uDC08\uDC0A-\uDC2E\uDC40\uDC50-\uDC6C\uDC72-\uDC8F\uDD00-\uDD06\uDD08\uDD09\uDD0B-\uDD30\uDD46\uDD50-\uDD59\uDD60-\uDD65\uDD67\uDD68\uDD6A-\uDD89\uDD98\uDDA0-\uDDA9\uDEE0-\uDEF2\uDF02\uDF04-\uDF10\uDF12-\uDF33\uDF50-\uDF59\uDFB0\uDFC0-\uDFD4]|\uD808[\uDC00-\uDF99]|\uD809[\uDC00-\uDC6E\uDC80-\uDD43]|\uD80B[\uDF90-\uDFF0]|[\uD80C\uD81C-\uD820\uD822\uD840-\uD868\uD86A-\uD86C\uD86F-\uD872\uD874-\uD879\uD880-\uD883\uD885-\uD887][\uDC00-\uDFFF]|\uD80D[\uDC00-\uDC2F\uDC41-\uDC46]|\uD811[\uDC00-\uDE46]|\uD81A[\uDC00-\uDE38\uDE40-\uDE5E\uDE60-\uDE69\uDE70-\uDEBE\uDEC0-\uDEC9\uDED0-\uDEED\uDF00-\uDF2F\uDF40-\uDF43\uDF50-\uDF59\uDF5B-\uDF61\uDF63-\uDF77\uDF7D-\uDF8F]|\uD81B[\uDE40-\uDE96\uDF00-\uDF4A\uDF50\uDF93-\uDF9F\uDFE0\uDFE1\uDFE3]|\uD821[\uDC00-\uDFF7]|\uD823[\uDC00-\uDCD5\uDD00-\uDD08]|\uD82B[\uDFF0-\uDFF3\uDFF5-\uDFFB\uDFFD\uDFFE]|\uD82C[\uDC00-\uDD22\uDD32\uDD50-\uDD52\uDD55\uDD64-\uDD67\uDD70-\uDEFB]|\uD82F[\uDC00-\uDC6A\uDC70-\uDC7C\uDC80-\uDC88\uDC90-\uDC99]|\uD834[\uDEC0-\uDED3\uDEE0-\uDEF3\uDF60-\uDF78]|\uD835[\uDC00-\uDC54\uDC56-\uDC9C\uDC9E\uDC9F\uDCA2\uDCA5\uDCA6\uDCA9-\uDCAC\uDCAE-\uDCB9\uDCBB\uDCBD-\uDCC3\uDCC5-\uDD05\uDD07-\uDD0A\uDD0D-\uDD14\uDD16-\uDD1C\uDD1E-\uDD39\uDD3B-\uDD3E\uDD40-\uDD44\uDD46\uDD4A-\uDD50\uDD52-\uDEA5\uDEA8-\uDEC0\uDEC2-\uDEDA\uDEDC-\uDEFA\uDEFC-\uDF14\uDF16-\uDF34\uDF36-\uDF4E\uDF50-\uDF6E\uDF70-\uDF88\uDF8A-\uDFA8\uDFAA-\uDFC2\uDFC4-\uDFCB\uDFCE-\uDFFF]|\uD837[\uDF00-\uDF1E\uDF25-\uDF2A]|\uD838[\uDC30-\uDC6D\uDD00-\uDD2C\uDD37-\uDD3D\uDD40-\uDD49\uDD4E\uDE90-\uDEAD\uDEC0-\uDEEB\uDEF0-\uDEF9]|\uD839[\uDCD0-\uDCEB\uDCF0-\uDCF9\uDFE0-\uDFE6\uDFE8-\uDFEB\uDFED\uDFEE\uDFF0-\uDFFE]|\uD83A[\uDC00-\uDCC4\uDCC7-\uDCCF\uDD00-\uDD43\uDD4B\uDD50-\uDD59]|\uD83B[\uDC71-\uDCAB\uDCAD-\uDCAF\uDCB1-\uDCB4\uDD01-\uDD2D\uDD2F-\uDD3D\uDE00-\uDE03\uDE05-\uDE1F\uDE21\uDE22\uDE24\uDE27\uDE29-\uDE32\uDE34-\uDE37\uDE39\uDE3B\uDE42\uDE47\uDE49\uDE4B\uDE4D-\uDE4F\uDE51\uDE52\uDE54\uDE57\uDE59\uDE5B\uDE5D\uDE5F\uDE61\uDE62\uDE64\uDE67-\uDE6A\uDE6C-\uDE72\uDE74-\uDE77\uDE79-\uDE7C\uDE7E\uDE80-\uDE89\uDE8B-\uDE9B\uDEA1-\uDEA3\uDEA5-\uDEA9\uDEAB-\uDEBB]|\uD83C[\uDD00-\uDD0C]|\uD83E[\uDFF0-\uDFF9]|\uD869[\uDC00-\uDEDF\uDF00-\uDFFF]|\uD86D[\uDC00-\uDF39\uDF40-\uDFFF]|\uD86E[\uDC00-\uDC1D\uDC20-\uDFFF]|\uD873[\uDC00-\uDEA1\uDEB0-\uDFFF]|\uD87A[\uDC00-\uDFE0]|\uD87E[\uDC00-\uDE1D]|\uD884[\uDC00-\uDF4A\uDF50-\uDFFF]|\uD888[\uDC00-\uDFAF])/))){var i=r[1]||r[2]||"";if(!i||i&&(""===n||this.rules.inline.punctuation.exec(n))){var o,a,u=r[0].length-1,l=u,c=0,s="*"===r[0][0]?this.rules.inline.emStrong.rDelimAst:this.rules.inline.emStrong.rDelimUnd;for(s.lastIndex=0,t=t.slice(-1*e.length+u);null!=(r=s.exec(t));)if(o=r[1]||r[2]||r[3]||r[4]||r[5]||r[6])if(a=o.length,r[3]||r[4])l+=a;else if(!((r[5]||r[6])&&u%3)||(u+a)%3){if(!((l-=a)>0)){a=Math.min(a,a+l+c);var f=e.slice(0,u+r.index+(r[0].length-o.length)+a);if(Math.min(u,a)%2){var d=f.slice(1,-1);return{type:"em",raw:f,text:d,tokens:this.lexer.inlineTokens(d)}}var h=f.slice(2,-2);return{type:"strong",raw:f,text:h,tokens:this.lexer.inlineTokens(h)}}}else c+=a}}}},{key:"codespan",value:function(e){var t=this.rules.inline.code.exec(e);if(t){var n=t[2].replace(/\n/g," "),r=/[^ ]/.test(n),i=/^ /.test(n)&&/ $/.test(n);return r&&i&&(n=n.substring(1,n.length-1)),n=md(n,!0),{type:"codespan",raw:t[0],text:n}}}},{key:"br",value:function(e){var t=this.rules.inline.br.exec(e);if(t)return{type:"br",raw:t[0]}}},{key:"del",value:function(e){var t=this.rules.inline.del.exec(e);if(t)return{type:"del",raw:t[0],text:t[2],tokens:this.lexer.inlineTokens(t[2])}}},{key:"autolink",value:function(e,t){var n,r,i=this.rules.inline.autolink.exec(e);if(i)return r="@"===i[2]?"mailto:"+(n=md(this.options.mangle?t(i[1]):i[1])):n=md(i[1]),{type:"link",raw:i[0],text:n,href:r,tokens:[{type:"text",raw:n,text:n}]}}},{key:"url",value:function(e,t){var n;if(n=this.rules.inline.url.exec(e)){var r,i;if("@"===n[2])i="mailto:"+(r=md(this.options.mangle?t(n[0]):n[0]));else{var o;do{o=n[0],n[0]=this.rules.inline._backpedal.exec(n[0])[0]}while(o!==n[0]);r=md(n[0]),i="www."===n[1]?"http://"+n[0]:n[0]}return{type:"link",raw:n[0],text:r,href:i,tokens:[{type:"text",raw:r,text:r}]}}}},{key:"inlineText",value:function(e,t){var n,r=this.rules.inline.text.exec(e);if(r)return n=this.lexer.state.inRawBlock?this.options.sanitize?this.options.sanitizer?this.options.sanitizer(r[0]):md(r[0]):r[0]:md(this.options.smartypants?t(r[0]):r[0]),{type:"text",raw:r[0],text:n}}}]),e}(),Id={newline:/^(?: *(?:\n|$))+/,code:/^( {4}[^\n]+(?:\n(?: *(?:\n|$))*)?)+/,fences:/^ {0,3}(`{3,}(?=[^`\n]*\n)|~{3,})([^\n]*)\n(?:|([\s\S]*?)\n)(?: {0,3}\1[~`]* *(?=\n|$)|$)/,hr:/^ {0,3}((?:-[\t ]*){3,}|(?:_[ \t]*){3,}|(?:\*[ \t]*){3,})(?:\n+|$)/,heading:/^ {0,3}(#{1,6})(?=\s|$)(.*)(?:\n+|$)/,blockquote:/^( {0,3}> ?(paragraph|[^\n]*)(?:\n|$))+/,list:/^( {0,3}bull)([ \t][^\n]+?)?(?:\n|$)/,html:"^ {0,3}(?:<(script|pre|style|textarea)[\\s>][\\s\\S]*?(?:\\1>[^\\n]*\\n+|$)|comment[^\\n]*(\\n+|$)|<\\?[\\s\\S]*?(?:\\?>\\n*|$)|\\n*|$)|\\n*|$)|?(tag)(?: +|\\n|/?>)[\\s\\S]*?(?:(?:\\n *)+\\n|$)|<(?!script|pre|style|textarea)([a-z][\\w-]*)(?:attribute)*? */?>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$)|(?!script|pre|style|textarea)[a-z][\\w-]*\\s*>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$))",def:/^ {0,3}\[(label)\]: *(?:\n *)?([^<\s][^\s]*|<.*?>)(?:(?: +(?:\n *)?| *\n *)(title))? *(?:\n+|$)/,table:Sd,lheading:/^((?:.|\n(?!\n))+?)\n {0,3}(=+|-+) *(?:\n+|$)/,_paragraph:/^([^\n]+(?:\n(?!hr|heading|lheading|blockquote|fences|list|html|table| +\n)[^\n]+)*)/,text:/^[^\n]+/,_label:/(?!\s*\])(?:\\.|[^\[\]\\])+/,_title:/(?:"(?:\\"?|[^"\\])*"|'[^'\n]*(?:\n[^'\n]+)*\n?'|\([^()]*\))/};Id.def=bd(Id.def).replace("label",Id._label).replace("title",Id._title).getRegex(),Id.bullet=/(?:[*+-]|\d{1,9}[.)])/,Id.listItemStart=bd(/^( *)(bull) */).replace("bull",Id.bullet).getRegex(),Id.list=bd(Id.list).replace(/bull/g,Id.bullet).replace("hr","\\n+(?=\\1?(?:(?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$))").replace("def","\\n+(?="+Id.def.source+")").getRegex(),Id._tag="address|article|aside|base|basefont|blockquote|body|caption|center|col|colgroup|dd|details|dialog|dir|div|dl|dt|fieldset|figcaption|figure|footer|form|frame|frameset|h[1-6]|head|header|hr|html|iframe|legend|li|link|main|menu|menuitem|meta|nav|noframes|ol|optgroup|option|p|param|section|source|summary|table|tbody|td|tfoot|th|thead|title|tr|track|ul",Id._comment=/|$)/,Id.html=bd(Id.html,"i").replace("comment",Id._comment).replace("tag",Id._tag).replace("attribute",/ +[a-zA-Z:_][\w.:-]*(?: *= *"[^"\n]*"| *= *'[^'\n]*'| *= *[^\s"'=<>`]+)?/).getRegex(),Id.paragraph=bd(Id._paragraph).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("|table","").replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.blockquote=bd(Id.blockquote).replace("paragraph",Id.paragraph).getRegex(),Id.normal=Nd({},Id),Id.gfm=Nd({},Id.normal,{table:"^ *([^\\n ].*\\|.*)\\n {0,3}(?:\\| *)?(:?-+:? *(?:\\| *:?-+:? *)*)(?:\\| *)?(?:\\n((?:(?! *\\n|hr|heading|blockquote|code|fences|list|html).*(?:\\n|$))*)\\n*|$)"}),Id.gfm.table=bd(Id.gfm.table).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("blockquote"," {0,3}>").replace("code"," {4}[^\\n]").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.gfm.paragraph=bd(Id._paragraph).replace("hr",Id.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("table",Id.gfm.table).replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html","?(?:tag)(?: +|\\n|/?>)|<(?:script|pre|style|textarea|!--)").replace("tag",Id._tag).getRegex(),Id.pedantic=Nd({},Id.normal,{html:bd("^ *(?:comment *(?:\\n|\\s*$)|<(tag)[\\s\\S]+?\\1> *(?:\\n{2,}|\\s*$)|\\s]*)*?/?> *(?:\\n{2,}|\\s*$))").replace("comment",Id._comment).replace(/tag/g,"(?!(?:a|em|strong|small|s|cite|q|dfn|abbr|data|time|code|var|samp|kbd|sub|sup|i|b|u|mark|ruby|rt|rp|bdi|bdo|span|br|wbr|ins|del|img)\\b)\\w+(?!:|[^\\w\\s@]*@)\\b").getRegex(),def:/^ *\[([^\]]+)\]: *([^\s>]+)>?(?: +(["(][^\n]+[")]))? *(?:\n+|$)/,heading:/^(#{1,6})(.*)(?:\n+|$)/,fences:Sd,lheading:/^(.+?)\n {0,3}(=+|-+) *(?:\n+|$)/,paragraph:bd(Id.normal._paragraph).replace("hr",Id.hr).replace("heading"," *#{1,6} *[^\n]").replace("lheading",Id.lheading).replace("blockquote"," {0,3}>").replace("|fences","").replace("|list","").replace("|html","").getRegex()});var Pd={escape:/^\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/,autolink:/^<(scheme:[^\s\x00-\x1f<>]*|email)>/,url:Sd,tag:"^comment|^[a-zA-Z][\\w:-]*\\s*>|^<[a-zA-Z][\\w-]*(?:attribute)*?\\s*/?>|^<\\?[\\s\\S]*?\\?>|^|^",link:/^!?\[(label)\]\(\s*(href)(?:\s+(title))?\s*\)/,reflink:/^!?\[(label)\]\[(ref)\]/,nolink:/^!?\[(ref)\](?:\[\])?/,reflinkSearch:"reflink|nolink(?!\\()",emStrong:{lDelim:/^(?:\*+(?:([punct_])|[^\s*]))|^_+(?:([punct*])|([^\s_]))/,rDelimAst:/^(?:[^_*\\]|\\.)*?\_\_(?:[^_*\\]|\\.)*?\*(?:[^_*\\]|\\.)*?(?=\_\_)|(?:[^*\\]|\\.)+(?=[^*])|[punct_](\*+)(?=[\s]|$)|(?:[^punct*_\s\\]|\\.)(\*+)(?=[punct_\s]|$)|[punct_\s](\*+)(?=[^punct*_\s])|[\s](\*+)(?=[punct_])|[punct_](\*+)(?=[punct_])|(?:[^punct*_\s\\]|\\.)(\*+)(?=[^punct*_\s])/,rDelimUnd:/^(?:[^_*\\]|\\.)*?\*\*(?:[^_*\\]|\\.)*?\_(?:[^_*\\]|\\.)*?(?=\*\*)|(?:[^_\\]|\\.)+(?=[^_])|[punct*](\_+)(?=[\s]|$)|(?:[^punct*_\s\\]|\\.)(\_+)(?=[punct*\s]|$)|[punct*\s](\_+)(?=[^punct*_\s])|[\s](\_+)(?=[punct*])|[punct*](\_+)(?=[punct*])/},code:/^(`+)([^`]|[^`][\s\S]*?[^`])\1(?!`)/,br:/^( {2,}|\\)\n(?!\s*$)/,del:Sd,text:/^(`+|[^`])(?:(?= {2,}\n)|[\s\S]*?(?:(?=[\\.5&&(n="x"+n.toString(16)),r+=""+n+";";return r}Pd._punctuation="!\"#$%&'()+\\-.,/:;<=>?@\\[\\]`^{|}~",Pd.punctuation=bd(Pd.punctuation).replace(/punctuation/g,Pd._punctuation).getRegex(),Pd.blockSkip=/\[[^\]]*?\]\([^\)]*?\)|`[^`]*?`|<[^>]*?>/g,Pd.escapedEmSt=/(?:^|[^\\])(?:\\\\)*\\[*_]/g,Pd._comment=bd(Id._comment).replace("(?:--\x3e|$)","--\x3e").getRegex(),Pd.emStrong.lDelim=bd(Pd.emStrong.lDelim).replace(/punct/g,Pd._punctuation).getRegex(),Pd.emStrong.rDelimAst=bd(Pd.emStrong.rDelimAst,"g").replace(/punct/g,Pd._punctuation).getRegex(),Pd.emStrong.rDelimUnd=bd(Pd.emStrong.rDelimUnd,"g").replace(/punct/g,Pd._punctuation).getRegex(),Pd._escapes=/\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/g,Pd._scheme=/[a-zA-Z][a-zA-Z0-9+.-]{1,31}/,Pd._email=/[a-zA-Z0-9.!#$%&'*+/=?^_`{|}~-]+(@)[a-zA-Z0-9](?:[a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?(?:\.[a-zA-Z0-9](?:[a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?)+(?![-_])/,Pd.autolink=bd(Pd.autolink).replace("scheme",Pd._scheme).replace("email",Pd._email).getRegex(),Pd._attribute=/\s+[a-zA-Z:_][\w.:-]*(?:\s*=\s*"[^"]*"|\s*=\s*'[^']*'|\s*=\s*[^\s"'=<>`]+)?/,Pd.tag=bd(Pd.tag).replace("comment",Pd._comment).replace("attribute",Pd._attribute).getRegex(),Pd._label=/(?:\[(?:\\.|[^\[\]\\])*\]|\\.|`[^`]*`|[^\[\]\\`])*?/,Pd._href=/<(?:\\.|[^\n<>\\])+>|[^\s\x00-\x1f]*/,Pd._title=/"(?:\\"?|[^"\\])*"|'(?:\\'?|[^'\\])*'|\((?:\\\)?|[^)\\])*\)/,Pd.link=bd(Pd.link).replace("label",Pd._label).replace("href",Pd._href).replace("title",Pd._title).getRegex(),Pd.reflink=bd(Pd.reflink).replace("label",Pd._label).replace("ref",Id._label).getRegex(),Pd.nolink=bd(Pd.nolink).replace("ref",Id._label).getRegex(),Pd.reflinkSearch=bd(Pd.reflinkSearch,"g").replace("reflink",Pd.reflink).replace("nolink",Pd.nolink).getRegex(),Pd.normal=Nd({},Pd),Pd.pedantic=Nd({},Pd.normal,{strong:{start:/^__|\*\*/,middle:/^__(?=\S)([\s\S]*?\S)__(?!_)|^\*\*(?=\S)([\s\S]*?\S)\*\*(?!\*)/,endAst:/\*\*(?!\*)/g,endUnd:/__(?!_)/g},em:{start:/^_|\*/,middle:/^()\*(?=\S)([\s\S]*?\S)\*(?!\*)|^_(?=\S)([\s\S]*?\S)_(?!_)/,endAst:/\*(?!\*)/g,endUnd:/_(?!_)/g},link:bd(/^!?\[(label)\]\((.*?)\)/).replace("label",Pd._label).getRegex(),reflink:bd(/^!?\[(label)\]\s*\[([^\]]*)\]/).replace("label",Pd._label).getRegex()}),Pd.gfm=Nd({},Pd.normal,{escape:bd(Pd.escape).replace("])","~|])").getRegex(),_extended_email:/[A-Za-z0-9._+-]+(@)[a-zA-Z0-9-_]+(?:\.[a-zA-Z0-9-_]*[a-zA-Z0-9])+(?![-_])/,url:/^((?:ftp|https?):\/\/|www\.)(?:[a-zA-Z0-9\-]+\.?)+[^\s<]*|^email/,_backpedal:/(?:[^?!.,:;*_'"~()&]+|\([^)]*\)|&(?![a-zA-Z0-9]+;$)|[?!.,:;*_'"~)]+(?!$))+/,del:/^(~~?)(?=[^\s~])([\s\S]*?[^\s~])\1(?=[^~]|$)/,text:/^([`~]+|[^`~])(?:(?= {2,}\n)|(?=[a-zA-Z0-9.!#$%&'*+\/=?_`{\|}~-]+@)|[\s\S]*?(?:(?=[\\1&&void 0!==arguments[1]?arguments[1]:[];e=this.options.pedantic?e.replace(/\t/g," ").replace(/^ +$/gm,""):e.replace(/^( *)(\t+)/gm,(function(e,t,n){return t+" ".repeat(n.length)}));for(var u=function(){if(o.options.extensions&&o.options.extensions.block&&o.options.extensions.block.some((function(n){return!!(t=n.call({lexer:o},e,a))&&(e=e.substring(t.raw.length),a.push(t),!0)})))return"continue";if(t=o.tokenizer.space(e))return e=e.substring(t.raw.length),1===t.raw.length&&a.length>0?a[a.length-1].raw+="\n":a.push(t),"continue";if(t=o.tokenizer.code(e))return e=e.substring(t.raw.length),!(n=a[a.length-1])||"paragraph"!==n.type&&"text"!==n.type?a.push(t):(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue[o.inlineQueue.length-1].src=n.text),"continue";if(t=o.tokenizer.fences(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.heading(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.hr(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.blockquote(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.list(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.html(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.def(e))return e=e.substring(t.raw.length),!(n=a[a.length-1])||"paragraph"!==n.type&&"text"!==n.type?o.tokens.links[t.tag]||(o.tokens.links[t.tag]={href:t.href,title:t.title}):(n.raw+="\n"+t.raw,n.text+="\n"+t.raw,o.inlineQueue[o.inlineQueue.length-1].src=n.text),"continue";if(t=o.tokenizer.table(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(t=o.tokenizer.lheading(e))return e=e.substring(t.raw.length),a.push(t),"continue";if(r=e,o.options.extensions&&o.options.extensions.startBlock){var u,l=1/0,c=e.slice(1);o.options.extensions.startBlock.forEach((function(e){"number"===typeof(u=e.call({lexer:this},c))&&u>=0&&(l=Math.min(l,u))})),l<1/0&&l>=0&&(r=e.substring(0,l+1))}if(o.state.top&&(t=o.tokenizer.paragraph(r)))return n=a[a.length-1],i&&"paragraph"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue.pop(),o.inlineQueue[o.inlineQueue.length-1].src=n.text):a.push(t),i=r.length!==e.length,e=e.substring(t.raw.length),"continue";if(t=o.tokenizer.text(e))return e=e.substring(t.raw.length),(n=a[a.length-1])&&"text"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,o.inlineQueue.pop(),o.inlineQueue[o.inlineQueue.length-1].src=n.text):a.push(t),"continue";if(e){var s="Infinite loop on byte: "+e.charCodeAt(0);if(o.options.silent)return console.error(s),"break";throw new Error(s)}};e;){var l=u();if("continue"!==l&&"break"===l)break}return this.state.top=!0,a}},{key:"inline",value:function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[];return this.inlineQueue.push({src:e,tokens:t}),t}},{key:"inlineTokens",value:function(e){var t,n,r,i,o,a,u=this,l=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[],c=e;if(this.tokens.links){var s=Object.keys(this.tokens.links);if(s.length>0)for(;null!=(i=this.tokenizer.rules.inline.reflinkSearch.exec(c));)s.includes(i[0].slice(i[0].lastIndexOf("[")+1,-1))&&(c=c.slice(0,i.index)+"["+Od("a",i[0].length-2)+"]"+c.slice(this.tokenizer.rules.inline.reflinkSearch.lastIndex))}for(;null!=(i=this.tokenizer.rules.inline.blockSkip.exec(c));)c=c.slice(0,i.index)+"["+Od("a",i[0].length-2)+"]"+c.slice(this.tokenizer.rules.inline.blockSkip.lastIndex);for(;null!=(i=this.tokenizer.rules.inline.escapedEmSt.exec(c));)c=c.slice(0,i.index+i[0].length-2)+"++"+c.slice(this.tokenizer.rules.inline.escapedEmSt.lastIndex),this.tokenizer.rules.inline.escapedEmSt.lastIndex--;for(var f=function(){if(o||(a=""),o=!1,u.options.extensions&&u.options.extensions.inline&&u.options.extensions.inline.some((function(n){return!!(t=n.call({lexer:u},e,l))&&(e=e.substring(t.raw.length),l.push(t),!0)})))return"continue";if(t=u.tokenizer.escape(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.tag(e))return e=e.substring(t.raw.length),(n=l[l.length-1])&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(t=u.tokenizer.link(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.reflink(e,u.tokens.links))return e=e.substring(t.raw.length),(n=l[l.length-1])&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(t=u.tokenizer.emStrong(e,c,a))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.codespan(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.br(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.del(e))return e=e.substring(t.raw.length),l.push(t),"continue";if(t=u.tokenizer.autolink(e,zd))return e=e.substring(t.raw.length),l.push(t),"continue";if(!u.state.inLink&&(t=u.tokenizer.url(e,zd)))return e=e.substring(t.raw.length),l.push(t),"continue";if(r=e,u.options.extensions&&u.options.extensions.startInline){var i,s=1/0,f=e.slice(1);u.options.extensions.startInline.forEach((function(e){"number"===typeof(i=e.call({lexer:this},f))&&i>=0&&(s=Math.min(s,i))})),s<1/0&&s>=0&&(r=e.substring(0,s+1))}if(t=u.tokenizer.inlineText(r,Rd))return e=e.substring(t.raw.length),"_"!==t.raw.slice(-1)&&(a=t.raw.slice(-1)),o=!0,(n=l[l.length-1])&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t),"continue";if(e){var d="Infinite loop on byte: "+e.charCodeAt(0);if(u.options.silent)return console.error(d),"break";throw new Error(d)}};e;){var d=f();if("continue"!==d&&"break"===d)break}return l}}],[{key:"rules",get:function(){return{block:Id,inline:Pd}}},{key:"lex",value:function(t,n){return new e(n).lex(t)}},{key:"lexInline",value:function(t,n){return new e(n).inlineTokens(t)}}]),e}(),$d=function(){function e(t){Bt(this,e),this.options=t||cd}return Rt(e,[{key:"code",value:function(e,t,n){var r=(t||"").match(/\S*/)[0];if(this.options.highlight){var i=this.options.highlight(e,r);null!=i&&i!==e&&(n=!0,e=i)}return e=e.replace(/\n$/,"")+"\n",r?''+(n?e:md(e,!0))+"
\n":""+(n?e:md(e,!0))+"
\n"}},{key:"blockquote",value:function(e){return"\n".concat(e,"
\n")}},{key:"html",value:function(e){return e}},{key:"heading",value:function(e,t,n,r){if(this.options.headerIds){var i=this.options.headerPrefix+r.slug(n);return"').concat(e,"\n")}return"").concat(e,"\n")}},{key:"hr",value:function(){return this.options.xhtml?"
\n":"
\n"}},{key:"list",value:function(e,t,n){var r=t?"ol":"ul";return"<"+r+(t&&1!==n?' start="'+n+'"':"")+">\n"+e+""+r+">\n"}},{key:"listitem",value:function(e){return"".concat(e,"\n")}},{key:"checkbox",value:function(e){return" "}},{key:"paragraph",value:function(e){return"".concat(e,"
\n")}},{key:"table",value:function(e,t){return t&&(t="".concat(t,"")),"\n"}},{key:"tablerow",value:function(e){return"\n".concat(e,"
\n")}},{key:"tablecell",value:function(e,t){var n=t.header?"th":"td";return(t.align?"<".concat(n,' align="').concat(t.align,'">'):"<".concat(n,">"))+e+"".concat(n,">\n")}},{key:"strong",value:function(e){return"".concat(e,"")}},{key:"em",value:function(e){return"".concat(e,"")}},{key:"codespan",value:function(e){return"".concat(e,"
")}},{key:"br",value:function(){return this.options.xhtml?"
":"
"}},{key:"del",value:function(e){return"".concat(e,"")}},{key:"link",value:function(e,t,n){if(null===(e=xd(this.options.sanitize,this.options.baseUrl,e)))return n;var r='"+n+""}},{key:"image",value:function(e,t,n){if(null===(e=xd(this.options.sanitize,this.options.baseUrl,e)))return n;var r='":">"}},{key:"text",value:function(e){return e}}]),e}(),Hd=function(){function e(){Bt(this,e)}return Rt(e,[{key:"strong",value:function(e){return e}},{key:"em",value:function(e){return e}},{key:"codespan",value:function(e){return e}},{key:"del",value:function(e){return e}},{key:"html",value:function(e){return e}},{key:"text",value:function(e){return e}},{key:"link",value:function(e,t,n){return""+n}},{key:"image",value:function(e,t,n){return""+n}},{key:"br",value:function(){return""}}]),e}(),Ud=function(){function e(){Bt(this,e),this.seen={}}return Rt(e,[{key:"serialize",value:function(e){return e.toLowerCase().trim().replace(/<[!\/a-z].*?>/gi,"").replace(/[\u2000-\u206F\u2E00-\u2E7F\\'!"#$%&()*+,./:;<=>?@[\]^`{|}~]/g,"").replace(/\s/g,"-")}},{key:"getNextSafeSlug",value:function(e,t){var n=e,r=0;if(this.seen.hasOwnProperty(n)){r=this.seen[e];do{n=e+"-"+ ++r}while(this.seen.hasOwnProperty(n))}return t||(this.seen[e]=r,this.seen[n]=0),n}},{key:"slug",value:function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:{},n=this.serialize(e);return this.getNextSafeSlug(n,t.dryrun)}}]),e}(),Yd=function(){function e(t){Bt(this,e),this.options=t||cd,this.options.renderer=this.options.renderer||new $d,this.renderer=this.options.renderer,this.renderer.options=this.options,this.textRenderer=new Hd,this.slugger=new Ud}return Rt(e,[{key:"parse",value:function(e){var t,n,r,i,o,a,u,l,c,s,f,d,h,p,v,m,g,y,_,b=!(arguments.length>1&&void 0!==arguments[1])||arguments[1],D="",w=e.length;for(t=0;t0&&"paragraph"===v.tokens[0].type?(v.tokens[0].text=y+" "+v.tokens[0].text,v.tokens[0].tokens&&v.tokens[0].tokens.length>0&&"text"===v.tokens[0].tokens[0].type&&(v.tokens[0].tokens[0].text=y+" "+v.tokens[0].tokens[0].text)):v.tokens.unshift({type:"text",text:y}):p+=y),p+=this.parse(v.tokens,h),c+=this.renderer.listitem(p,g,m);D+=this.renderer.list(c,f,d);continue;case"html":D+=this.renderer.html(s.text);continue;case"paragraph":D+=this.renderer.paragraph(this.parseInline(s.tokens));continue;case"text":for(c=s.tokens?this.parseInline(s.tokens):s.text;t+1An error occurred:"+md(e.message+"",!0)+"
";throw e}try{var l=jd.lex(e,t);if(t.walkTokens){if(t.async)return Promise.all(Vd.walkTokens(l,t.walkTokens)).then((function(){return Yd.parse(l,t)})).catch(u);Vd.walkTokens(l,t.walkTokens)}return Yd.parse(l,t)}catch(s){u(s)}}Vd.options=Vd.setOptions=function(e){var t;return Nd(Vd.defaults,e),t=Vd.defaults,cd=t,Vd},Vd.getDefaults=ld,Vd.defaults=cd,Vd.use=function(){for(var e=Vd.defaults.extensions||{renderers:{},childTokens:{}},t=arguments.length,n=new Array(t),r=0;rAn error occurred:"+md(s.message+"",!0)+"
";throw s}},Vd.Parser=Yd,Vd.parser=Yd.parse,Vd.Renderer=$d,Vd.TextRenderer=Hd,Vd.Lexer=jd,Vd.lexer=jd.lex,Vd.Tokenizer=Ld,Vd.Slugger=Ud,Vd.parse=Vd;Vd.options,Vd.setOptions,Vd.use,Vd.walkTokens,Vd.parseInline,Yd.parse,jd.lex;var qd=function(e){var t=e.title,n=e.description,r=e.unit,i=e.expr,o=e.showLegend,a=e.filename,u=e.alias,l=Xo().isMobile,c=Ni().period,s=$o().customStep,f=Fi(),d=ie(null),h=Ft(ee(!1),2),p=h[0],v=h[1],m=Ft(ee({limits:{enable:!1,range:{1:[0,0]}}}),2),y=m[0],_=m[1],b=ae((function(){return Array.isArray(i)&&i.every((function(e){return e}))}),[i]),D=Uf({predefinedQuery:b?i:[],display:"chart",visible:p,customStep:s}),w=D.isLoading,x=D.graphData,k=D.error,C=D.warning,A=function(e){var t=yr({},y);t.limits.range=e,_(t)};if(ne((function(){var e=new IntersectionObserver((function(e){e.forEach((function(e){return v(e.isIntersecting)}))}),{threshold:.1});return d.current&&e.observe(d.current),function(){d.current&&e.unobserve(d.current)}}),[d]),!b)return Vr(ta,{variant:"error",children:[Vr("code",{children:'"expr"'})," not found. Check the configuration file ",Vr("b",{children:a}),"."]});var E=function(){return Vr("div",{className:"vm-predefined-panel-header__description vm-default-styles",children:[n&&Vr(g,{children:[Vr("div",{children:[Vr("span",{children:"Description:"}),Vr("div",{dangerouslySetInnerHTML:{__html:Vd.parse(n)}})]}),Vr("hr",{})]}),Vr("div",{children:[Vr("span",{children:"Queries:"}),Vr("div",{children:i.map((function(e,t){return Vr("div",{children:e},"".concat(t,"_").concat(e))}))})]})]})};return Vr("div",{className:"vm-predefined-panel",ref:d,children:[Vr("div",{className:"vm-predefined-panel-header",children:[Vr(ma,{title:Vr(E,{}),children:Vr("div",{className:"vm-predefined-panel-header__info",children:Vr(Hi,{})})}),Vr("h3",{className:"vm-predefined-panel-header__title",children:t||""}),Vr(Wf,{yaxis:y,setYaxisLimits:A,toggleEnableLimits:function(){var e=yr({},y);e.limits.enable=!e.limits.enable,_(e)}})]}),Vr("div",{className:"vm-predefined-panel-body",children:[w&&Vr(Qf,{}),k&&Vr(ta,{variant:"error",children:k}),C&&Vr(ta,{variant:"warning",children:C}),x&&Vr(Nf,{data:x,period:c,customStep:s,query:i,yaxis:y,unit:r,alias:u,showLegend:o,setYaxisLimits:A,setPeriod:function(e){var t=e.from,n=e.to;f({type:"SET_PERIOD",payload:{from:t,to:n}})},fullWidth:!1,height:l?.5*window.innerHeight:500})]})]})},Wd=function(e){var t=e.index,n=e.title,r=e.panels,i=e.filename,o=bo(document.body),a=ae((function(){return o.width/12}),[o]),u=Ft(ee(!t),2),l=u[0],c=u[1],s=Ft(ee([]),2),f=s[0],d=s[1];ne((function(){d(r&&r.map((function(e){return e.width||12})))}),[r]);var h=Ft(ee({start:0,target:0,enable:!1}),2),p=h[0],v=h[1],m=function(e){if(p.enable){var t=p.start,n=Math.ceil((t-e.clientX)/a);if(!(Math.abs(n)>=12)){var r=f.map((function(e,t){return e-(t===p.target?n:0)}));d(r)}}},g=function(){v(yr(yr({},p),{},{enable:!1}))},y=function(e){return function(t){!function(e,t){v({start:e.clientX,target:t,enable:!0})}(t,e)}};return ne((function(){return window.addEventListener("mousemove",m),window.addEventListener("mouseup",g),function(){window.removeEventListener("mousemove",m),window.removeEventListener("mouseup",g)}}),[p]),Vr("div",{className:"vm-predefined-dashboard",children:Vr(qa,{defaultExpanded:l,onChange:function(e){return c(e)},title:Vr((function(){return Vr("div",{className:wo()({"vm-predefined-dashboard-header":!0,"vm-predefined-dashboard-header_open":l}),children:[(n||i)&&Vr("span",{className:"vm-predefined-dashboard-header__title",children:n||"".concat(t+1,". ").concat(i)}),r&&Vr("span",{className:"vm-predefined-dashboard-header__count",children:["(",r.length," panels)"]})]})}),{}),children:Vr("div",{className:"vm-predefined-dashboard-panels",children:Array.isArray(r)&&r.length?r.map((function(e,t){return Vr("div",{className:"vm-predefined-dashboard-panels-panel vm-block vm-block_empty-padding",style:{gridColumn:"span ".concat(f[t])},children:[Vr(qd,{title:e.title,description:e.description,unit:e.unit,expr:e.expr,alias:e.alias,filename:i,showLegend:e.showLegend}),Vr("button",{className:"vm-predefined-dashboard-panels-panel__resizer",onMouseDown:y(t)})]},t)})):Vr("div",{className:"vm-predefined-dashboard-panels-panel__alert",children:Vr(ta,{variant:"error",children:[Vr("code",{children:'"panels"'})," not found. Check the configuration file ",Vr("b",{children:i}),"."]})})})})})},Qd=function(){!function(){var e=Ni(),t=e.duration,n=e.relativeTime,r=e.period.date,i=$o().customStep,o=Ft(pr(),2)[1],a=function(){var e,a=id((mr(e={},"g0.range_input",t),mr(e,"g0.end_input",r),mr(e,"g0.step_input",i),mr(e,"g0.relative_time",n),e));o(a)};ne(a,[t,n,r,i]),ne(a,[])}();var e=Xo().isMobile,t=ua(),n=t.dashboardsSettings,r=t.dashboardsLoading,i=t.dashboardsError,o=Ft(ee(0),2),a=o[0],u=o[1],l=ae((function(){return n.map((function(e,t){return{label:e.title||"",value:t}}))}),[n]),c=ae((function(){return n[a]||{}}),[n,a]),s=ae((function(){return null===c||void 0===c?void 0:c.rows}),[c]),f=ae((function(){return c.title||c.filename||""}),[c]),d=ae((function(){return Array.isArray(s)&&!!s.length}),[s]),h=function(e){return function(){!function(e){u(e)}(e)}};return Vr("div",{className:"vm-predefined-panels",children:[r&&Vr(Qf,{}),i&&Vr(ta,{variant:"error",children:i}),!n.length&&Vr(ta,{variant:"info",children:"Dashboards not found"}),l.length>1&&Vr("div",{className:wo()({"vm-predefined-panels-tabs":!0,"vm-block":!0,"vm-block_mobile":e}),children:l.map((function(e){return Vr("div",{className:wo()({"vm-predefined-panels-tabs__tab":!0,"vm-predefined-panels-tabs__tab_active":e.value==a}),onClick:h(e.value),children:e.label},e.value)}))}),Vr("div",{className:"vm-predefined-panels__dashboards",children:[d&&s.map((function(e,t){return Vr(Wd,{index:t,filename:f,title:e.title,panels:e.panels},"".concat(a,"_").concat(t))})),!!n.length&&!d&&Vr(ta,{variant:"error",children:[Vr("code",{children:'"rows"'})," not found. Check the configuration file ",Vr("b",{children:f}),"."]})]})]})},Gd=function(e,t){var n=t.match?"&match[]="+encodeURIComponent(t.match):"",r=t.focusLabel?"&focusLabel="+encodeURIComponent(t.focusLabel):"";return"".concat(e,"/api/v1/status/tsdb?topN=").concat(t.topN,"&date=").concat(t.date).concat(n).concat(r)},Jd=function(){function e(){Bt(this,e),this.tsdbStatus=void 0,this.tabsNames=void 0,this.tsdbStatus=this.defaultTSDBStatus,this.tabsNames=["table","graph"]}return Rt(e,[{key:"tsdbStatusData",get:function(){return this.tsdbStatus},set:function(e){this.tsdbStatus=e}},{key:"defaultTSDBStatus",get:function(){return{totalSeries:0,totalLabelValuePairs:0,seriesCountByMetricName:[],seriesCountByLabelName:[],seriesCountByFocusLabelValue:[],seriesCountByLabelValuePair:[],labelValueCountByLabelName:[]}}},{key:"keys",value:function(e){var t=[];return e&&(t=t.concat("seriesCountByFocusLabelValue")),t=t.concat("seriesCountByMetricName","seriesCountByLabelName","seriesCountByLabelValuePair","labelValueCountByLabelName"),t}},{key:"defaultState",get:function(){var e=this;return this.keys("job").reduce((function(t,n){return yr(yr({},t),{},{tabs:yr(yr({},t.tabs),{},mr({},n,e.tabsNames)),containerRefs:yr(yr({},t.containerRefs),{},mr({},n,ie(null))),defaultActiveTab:yr(yr({},t.defaultActiveTab),{},mr({},n,0))})}),{tabs:{},containerRefs:{},defaultActiveTab:{}})}},{key:"sectionsTitles",value:function(e){return{seriesCountByMetricName:"Metric names with the highest number of series",seriesCountByLabelName:"Labels with the highest number of series",seriesCountByFocusLabelValue:'Values for "'.concat(e,'" label with the highest number of series'),seriesCountByLabelValuePair:"Label=value pairs with the highest number of series",labelValueCountByLabelName:"Labels with the highest number of unique values"}}},{key:"tablesHeaders",get:function(){return{seriesCountByMetricName:Zd,seriesCountByLabelName:Kd,seriesCountByFocusLabelValue:Xd,seriesCountByLabelValuePair:eh,labelValueCountByLabelName:th}}},{key:"totalSeries",value:function(e){return"labelValueCountByLabelName"===e?-1:this.tsdbStatus.totalSeries}}]),e}(),Zd=[{id:"name",label:"Metric name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],Kd=[{id:"name",label:"Label name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],Xd=[{id:"name",label:"Label value"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{disablePadding:!1,id:"action",label:"Action",numeric:!1}],eh=[{id:"name",label:"Label=value pair"},{id:"value",label:"Number of series"},{id:"percentage",label:"Percent of series"},{id:"action",label:"Action"}],th=[{id:"name",label:"Label name"},{id:"value",label:"Number of unique values"},{id:"action",label:"Action"}],nh={seriesCountByMetricName:function(e,t){return rh("__name__",t)},seriesCountByLabelName:function(e,t){return"{".concat(t,'!=""}')},seriesCountByFocusLabelValue:function(e,t){return rh(e,t)},seriesCountByLabelValuePair:function(e,t){var n=t.split("="),r=n[0],i=n.slice(1).join("=");return rh(r,i)},labelValueCountByLabelName:function(e,t){return"{".concat(t,'!=""}')}},rh=function(e,t){return e?"{"+e+"="+JSON.stringify(t)+"}":""},ih=function(e){var t=e.topN,n=e.error,r=e.query,i=e.onSetHistory,o=e.onRunQuery,a=e.onSetQuery,u=e.onTopNChange,l=e.onFocusLabelChange,c=e.totalSeries,s=e.totalLabelValuePairs,f=e.date,d=e.match,h=e.focusLabel,p=Li().autocomplete,v=Ii(),m=Xo().isMobile,g=Gf().queryOptions,y=ae((function(){return t<1?"Number must be bigger than zero":""}),[t]);return Vr("div",{className:wo()({"vm-cardinality-configurator":!0,"vm-block":!0,"vm-block_mobile":m}),children:[Vr("div",{className:"vm-cardinality-configurator-controls",children:[Vr("div",{className:"vm-cardinality-configurator-controls__query",children:Vr(Mf,{value:r,autocomplete:p,options:g,error:n,onArrowUp:function(){i(-1)},onArrowDown:function(){i(1)},onEnter:o,onChange:a,label:"Time series selector"})}),Vr("div",{className:"vm-cardinality-configurator-controls__item",children:Vr(xa,{label:"Number of entries per table",type:"number",value:t,error:y,onChange:u})}),Vr("div",{className:"vm-cardinality-configurator-controls__item",children:Vr(xa,{label:"Focus label",type:"text",value:h||"",onChange:l,endIcon:Vr(ma,{title:Vr("div",{children:[Vr("p",{children:"To identify values with the highest number of series for the selected label."}),Vr("p",{children:"Adds a table showing the series with the highest number of series."})]}),children:Vr(Hi,{})})})})]}),Vr("div",{className:"vm-cardinality-configurator-additional",children:Vr(Tf,{label:"Autocomplete",value:p,onChange:function(){v({type:"TOGGLE_AUTOCOMPLETE"})}})}),Vr("div",{className:wo()({"vm-cardinality-configurator-bottom":!0,"vm-cardinality-configurator-bottom_mobile":m}),children:[Vr("div",{className:"vm-cardinality-configurator-bottom__info",children:["Analyzed ",Vr("b",{children:c})," series with ",Vr("b",{children:s}),' "label=value" pairs at ',Vr("b",{children:f}),d&&Vr("span",{children:[" for series selector ",Vr("b",{children:d})]}),". Show top ",t," entries per table."]}),Vr("div",{className:"vm-cardinality-configurator-bottom__docs",children:[Vr("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/#cardinality-explorer",rel:"help noreferrer",children:[Vr(po,{}),"Documentation"]}),Vr("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://victoriametrics.com/blog/cardinality-explorer/",rel:"help noreferrer",children:[Vr(mo,{}),"Example of using"]})]}),Vr(fa,{startIcon:Vr(Xi,{}),onClick:o,fullWidth:!0,children:"Execute Query"})]})]})};function oh(e){var t=e.order,n=e.orderBy,r=e.onRequestSort,i=e.headerCells;return Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",children:i.map((function(e){return Vr("th",{className:wo()({"vm-table-cell vm-table-cell_header":!0,"vm-table-cell_sort":"action"!==e.id&&"percentage"!==e.id,"vm-table-cell_right":"action"===e.id}),onClick:(i=e.id,function(e){r(e,i)}),children:Vr("div",{className:"vm-table-cell__content",children:[e.label,"action"!==e.id&&"percentage"!==e.id&&Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":n===e.id,"vm-table__sort-icon_desc":"desc"===t&&n===e.id}),children:Vr(Qi,{})})]})},e.id);var i}))})})}function ah(e,t,n){return t[n]e[n]?1:0}function uh(e,t){return"desc"===e?function(e,n){return ah(e,n,t)}:function(e,n){return-ah(e,n,t)}}function lh(e,t){var n=e.map((function(e,t){return[e,t]}));return n.sort((function(e,n){var r=t(e[0],n[0]);return 0!==r?r:e[1]-n[1]})),n.map((function(e){return e[0]}))}var ch=function(e){var t=e.rows,n=e.headerCells,r=e.defaultSortColumn,i=e.tableCells,o=Ft(ee("desc"),2),a=o[0],u=o[1],l=Ft(ee(r),2),c=l[0],s=l[1],f=Ft(ee([]),2),d=f[0],h=f[1],p=function(e){return function(){var t=d.indexOf(e),n=[];-1===t?n=n.concat(d,e):0===t?n=n.concat(d.slice(1)):t===d.length-1?n=n.concat(d.slice(0,-1)):t>0&&(n=n.concat(d.slice(0,t),d.slice(t+1))),h(n)}},v=lh(t,uh(a,c));return Vr("table",{className:"vm-table",children:[Vr(oh,{numSelected:d.length,order:a,orderBy:c,onSelectAllClick:function(e){if(e.target.checked){var n=t.map((function(e){return e.name}));h(n)}else h([])},onRequestSort:function(e,t){u(c===t&&"asc"===a?"desc":"asc"),s(t)},rowCount:t.length,headerCells:n}),Vr("tbody",{className:"vm-table-header",children:v.map((function(e){return Vr("tr",{className:wo()({"vm-table__row":!0,"vm-table__row_selected":(t=e.name,-1!==d.indexOf(t))}),onClick:p(e.name),children:i(e)},e.name);var t}))})]})},sh=function(e){var t=e.row,n=e.totalSeries,r=e.onActionClick,i=n>0?t.value/n*100:-1;return Vr(g,{children:[Vr("td",{className:"vm-table-cell",children:t.name},t.name),Vr("td",{className:"vm-table-cell",children:t.value},t.value),i>0&&Vr("td",{className:"vm-table-cell",children:Vr(Jf,{value:i})},t.progressValue),Vr("td",{className:"vm-table-cell vm-table-cell_right",children:Vr("div",{className:"vm-table-cell__content",children:Vr(ma,{title:"Filter by ".concat(t.name),children:Vr(fa,{variant:"text",size:"small",onClick:function(){r(t.name)},children:Vr(eo,{})})})})},"action")]})},fh=function(e){var t=e.data,n=e.container,r=e.configs,i=Wr().isDarkTheme,o=ie(null),a=Ft(ee(),2),u=a[0],l=a[1],c=bo(n),s=yr(yr({},r),{},{width:c.width||400});return ne((function(){if(o.current){var e=new nf(s,t,o.current);return l(e),e.destroy}}),[o.current,c,i]),ne((function(){u&&u.setData(t)}),[t]),Vr("div",{style:{height:"100%"},children:Vr("div",{ref:o})})},dh=function(e,t){return Math.round(e*(t=Math.pow(10,t)))/t},hh=1,ph=function(e,t,n,r){return dh(t+e*(n+r),6)},vh=function(e,t,n,r,i){var o=1-t,a=n===hh?o/(e-1):2===n?o/e:3===n?o/(e+1):0;(isNaN(a)||a===1/0)&&(a=0);var u=n===hh?0:2===n?a/2:3===n?a:0,l=t/e,c=dh(l,6);if(null==r)for(var s=0;s=n&&e<=i&&t>=r&&t<=o};function gh(e,t,n,r,i){var o=this;o.x=e,o.y=t,o.w=n,o.h=r,o.l=i||0,o.o=[],o.q=null}var yh={split:function(){var e=this,t=e.x,n=e.y,r=e.w/2,i=e.h/2,o=e.l+1;e.q=[new gh(t+r,n,r,i,o),new gh(t,n,r,i,o),new gh(t,n+i,r,i,o),new gh(t+r,n+i,r,i,o)]},quads:function(e,t,n,r,i){var o=this,a=o.q,u=o.x+o.w/2,l=o.y+o.h/2,c=tu,d=t+r>l;c&&f&&i(a[0]),s&&c&&i(a[1]),s&&d&&i(a[2]),f&&d&&i(a[3])},add:function(e){var t=this;if(null!=t.q)t.quads(e.x,e.y,e.w,e.h,(function(t){t.add(e)}));else{var n=t.o;if(n.push(e),n.length>10&&t.l<4){t.split();for(var r=function(){var e=n[i];t.quads(e.x,e.y,e.w,e.h,(function(t){t.add(e)}))},i=0;i=0?"left":"right",e.ctx.textBaseline=1===s?"middle":i[n]>=0?"bottom":"top",e.ctx.fillText(i[n],f,y)}}))})),e.ctx.restore()}function b(e,t,n){return[0,nf.rangeNum(0,n,.05,!0)[1]]}return{hooks:{drawClear:function(t){var n;if((g=g||new gh(0,0,t.bbox.width,t.bbox.height)).clear(),t.series.forEach((function(e){e._paths=null})),l=d?[null].concat(m(t.data.length-1-o.length,t.data[0].length)):2===t.series.length?[null].concat(m(t.data[0].length,1)):[null].concat(function(e,t){var n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:h,r=Array.from({length:t},(function(){return{offs:Array(e).fill(0),size:Array(e).fill(0)}}));return vh(e,n,p,null,(function(e,n,i){vh(t,1,v,null,(function(t,o,a){r[t].offs[e]=n+i*o,r[t].size[e]=i*a}))})),r}(t.data[0].length,t.data.length-1-o.length,1===t.data[0].length?1:h)),null!=(null===(n=e.disp)||void 0===n?void 0:n.fill)){c=[null];for(var r=1;r0&&!o.includes(t)&&nf.assign(e,{paths:y,points:{show:_}})}))}}}((_h=[1],bh=0,Dh=1,wh=0,xh=function(e,t){return{stroke:e,fill:t}}({unit:3,values:function(e){return e.data[1].map((function(e,t){return 0!==t?"#33BB55":"#F79420"}))}},{unit:3,values:function(e){return e.data[1].map((function(e,t){return 0!==t?"#33BB55":"#F79420"}))}}),{which:_h,ori:bh,dir:Dh,radius:wh,disp:xh}))]},Ch=function(e){var t=e.rows,n=e.activeTab,r=e.onChange,i=e.tabs,o=e.chartContainer,a=e.totalSeries,u=e.tabId,l=e.onActionClick,c=e.sectionTitle,s=e.tableHeaderCells,f=Xo().isMobile,d=ae((function(){return i.map((function(e,t){return{value:String(t),label:e,icon:Vr(0===t?no:to,{})}}))}),[i]);return Vr("div",{className:wo()({"vm-metrics-content":!0,"vm-metrics-content_mobile":f,"vm-block":!0,"vm-block_mobile":f}),children:[Vr("div",{className:"vm-metrics-content-header vm-section-header",children:[Vr("h5",{className:wo()({"vm-section-header__title":!0,"vm-section-header__title_mobile":f}),children:c}),Vr("div",{className:"vm-section-header__tabs",children:Vr(Eo,{activeItem:String(n),items:d,onChange:function(e){r(e,u)}})})]}),Vr("div",{ref:o,className:wo()({"vm-metrics-content__table":!0,"vm-metrics-content__table_mobile":f}),children:[0===n&&Vr(ch,{rows:t,headerCells:s,defaultSortColumn:"value",tableCells:function(e){return Vr(sh,{row:e,totalSeries:a,onActionClick:l})}}),1===n&&Vr(fh,{data:[t.map((function(e){return e.name})),t.map((function(e){return e.value})),t.map((function(e,t){return t%12==0?1:t%10==0?2:0}))],container:(null===o||void 0===o?void 0:o.current)||null,configs:kh})]})]})},Ah=function(){var e=Xo().isMobile,t=qo(),n=t.topN,r=t.match,i=t.date,o=t.focusLabel,a=Wo();!function(){var e=qo(),t=e.topN,n=e.match,r=e.date,i=e.focusLabel,o=e.extraLabel,a=Ft(pr(),2)[1],u=function(){var e=id({topN:t,date:r,match:n,extraLabel:o,focusLabel:i});a(e)};ne(u,[t,n,r,i,o]),ne(u,[])}();var u=Ft(ee(r||""),2),l=u[0],c=u[1],s=Ft(ee(0),2),f=s[0],d=s[1],h=Ft(ee([]),2),p=h[0],v=h[1],m=function(){var e=new Jd,t=qo(),n=t.topN,r=t.extraLabel,i=t.match,o=t.date,a=t.runQuery,u=t.focusLabel,l=Wr().serverUrl,c=Ft(ee(!1),2),s=c[0],f=c[1],d=Ft(ee(),2),h=d[0],p=d[1],v=Ft(ee(e.defaultTSDBStatus),2),m=v[0],g=v[1];ne((function(){h&&(g(e.defaultTSDBStatus),f(!1))}),[h]);var y=function(){var t=tu(Xa().mark((function t(n){var r,i,o,a;return Xa().wrap((function(t){for(;;)switch(t.prev=t.next){case 0:if(l){t.next=2;break}return t.abrupt("return");case 2:return p(""),f(!0),g(e.defaultTSDBStatus),r=Gd(l,n),t.prev=6,t.next=9,fetch(r);case 9:return i=t.sent,t.next=12,i.json();case 12:o=t.sent,i.ok?(a=o.data,g(yr({},a)),f(!1)):(p(o.error),g(e.defaultTSDBStatus),f(!1)),t.next=20;break;case 16:t.prev=16,t.t0=t.catch(6),f(!1),t.t0 instanceof Error&&p("".concat(t.t0.name,": ").concat(t.t0.message));case 20:case"end":return t.stop()}}),t,null,[[6,16]])})));return function(e){return t.apply(this,arguments)}}();return ne((function(){y({topN:n,extraLabel:r,match:i,date:o,focusLabel:u})}),[l,a,o]),e.tsdbStatusData=m,{isLoading:s,appConfigurator:e,error:h}}(),g=m.isLoading,y=m.appConfigurator,_=m.error,b=Ft(ee(y.defaultState.defaultActiveTab),2),D=b[0],w=b[1],x=y.tsdbStatusData,k=y.defaultState,C=y.tablesHeaders,A=function(e,t){w(yr(yr({},D),{},mr({},t,+e)))};return Vr("div",{className:wo()({"vm-cardinality-panel":!0,"vm-cardinality-panel_mobile":e}),children:[g&&Vr(Qf,{message:"Please wait while cardinality stats is calculated. \n This may take some time if the db contains big number of time series."}),Vr(ih,{error:"",query:l,topN:n,date:i,match:r,totalSeries:x.totalSeries,totalLabelValuePairs:x.totalLabelValuePairs,focusLabel:o,onRunQuery:function(){v((function(e){return[].concat(Ot(e),[l])})),d((function(e){return e+1})),a({type:"SET_MATCH",payload:l}),a({type:"RUN_QUERY"})},onSetQuery:c,onSetHistory:function(e){var t=f+e;t<0||t>=p.length||(d(t),c(p[t]))},onTopNChange:function(e){a({type:"SET_TOP_N",payload:+e})},onFocusLabelChange:function(e){a({type:"SET_FOCUS_LABEL",payload:e})}}),_&&Vr(ta,{variant:"error",children:_}),y.keys(o).map((function(e){return Vr(Ch,{sectionTitle:y.sectionsTitles(o)[e],activeTab:D[e],rows:x[e],onChange:A,onActionClick:(t=e,function(e){var n=nh[t](o,e);c(n),v((function(e){return[].concat(Ot(e),[n])})),d((function(e){return e+1})),a({type:"SET_MATCH",payload:n});var r="";"labelValueCountByLabelName"!==t&&"seriesCountByLabelName"!=t||(r=e),a({type:"SET_FOCUS_LABEL",payload:r}),a({type:"RUN_QUERY"})}),tabs:k.tabs[e],chartContainer:k.containerRefs[e],totalSeries:y.totalSeries(e),tabId:e,tableHeaderCells:C[e]},e);var t}))]})},Eh=function(e){var t=e.rows,n=e.columns,r=Ft(ee(e.defaultOrderBy||"count"),2),i=r[0],o=r[1],a=Ft(ee("desc"),2),u=a[0],l=a[1],c=ae((function(){return lh(t,uh(u,i))}),[t,i,u]),s=function(e){return function(){var t;t=e,l((function(e){return"asc"===e&&i===t?"desc":"asc"})),o(t)}};return Vr("table",{className:"vm-table",children:[Vr("thead",{className:"vm-table-header",children:Vr("tr",{className:"vm-table__row vm-table__row_header",children:n.map((function(e){return Vr("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:s(e.key),children:Vr("div",{className:"vm-table-cell__content",children:[e.title||e.key,Vr("div",{className:wo()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":i===e.key,"vm-table__sort-icon_desc":"desc"===u&&i===e.key}),children:Vr(Qi,{})})]})},e.key)}))})}),Vr("tbody",{className:"vm-table-body",children:c.map((function(e,t){return Vr("tr",{className:"vm-table__row",children:n.map((function(t){return Vr("td",{className:"vm-table-cell",children:e[t.key]||"-"},t.key)}))},t)}))})]})},Sh=["table","JSON"].map((function(e,t){return{value:String(t),label:e,icon:Vr(0===t?no:ro,{})}})),Nh=function(e){var t=e.rows,n=e.title,r=e.columns,i=e.defaultOrderBy,o=Xo().isMobile,a=Ft(ee(0),2),u=a[0],l=a[1];return Vr("div",{className:wo()({"vm-top-queries-panel":!0,"vm-block":!0,"vm-block_mobile":o}),children:[Vr("div",{className:wo()({"vm-top-queries-panel-header":!0,"vm-section-header":!0,"vm-top-queries-panel-header_mobile":o}),children:[Vr("h5",{className:wo()({"vm-section-header__title":!0,"vm-section-header__title_mobile":o}),children:n}),Vr("div",{className:"vm-section-header__tabs",children:Vr(Eo,{activeItem:String(u),items:Sh,onChange:function(e){l(+e)}})})]}),Vr("div",{className:wo()({"vm-top-queries-panel__table":!0,"vm-top-queries-panel__table_mobile":o}),children:[0===u&&Vr(Eh,{rows:t,columns:r,defaultOrderBy:i}),1===u&&Vr(Yf,{data:t})]})]})},Fh=function(){var e=Xo().isMobile,t=function(){var e=Wr().serverUrl,t=Zo(),n=t.topN,r=t.maxLifetime,i=t.runQuery,o=Ft(ee(null),2),a=o[0],u=o[1],l=Ft(ee(!1),2),c=l[0],s=l[1],f=Ft(ee(),2),d=f[0],h=f[1],p=ae((function(){return function(e,t,n){return"".concat(e,"/api/v1/status/top_queries?topN=").concat(t||"","&maxLifetime=").concat(n||"")}(e,n,r)}),[e,n,r]),v=function(){var e=tu(Xa().mark((function e(){var t,n;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return s(!0),e.prev=1,e.next=4,fetch(p);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,t.ok&&["topByAvgDuration","topByCount","topBySumDuration"].forEach((function(e){var t=n[e];Array.isArray(t)&&t.forEach((function(e){return e.timeRangeHours=+(e.timeRangeSeconds/3600).toFixed(2)}))})),u(t.ok?n:null),h(String(n.error||"")),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&"AbortError"!==e.t0.name&&h("".concat(e.t0.name,": ").concat(e.t0.message));case 16:s(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();return ne((function(){v()}),[i]),{data:a,error:d,loading:c}}(),n=t.data,r=t.error,i=t.loading,o=Zo(),a=o.topN,u=o.maxLifetime,l=le(Jo).dispatch;!function(){var e=Zo(),t=e.topN,n=e.maxLifetime,r=Ft(pr(),2)[1],i=function(){var e=id({topN:String(t),maxLifetime:n});r(e)};ne(i,[t,n]),ne(i,[])}();var c=ae((function(){var e=u.trim().split(" ").reduce((function(e,t){var n=ui(t);return n?yr(yr({},e),n):yr({},e)}),{});return!!_t().duration(e).asMilliseconds()}),[u]),s=ae((function(){return!!a&&a<1}),[a]),f=ae((function(){return s?"Number must be bigger than zero":""}),[s]),d=ae((function(){return c?"":"Invalid duration value"}),[c]),h=function(e){if(!n)return e;var t=n[e];return"number"===typeof t?uf(t,t,t):t||e},p=function(){l({type:"SET_RUN_QUERY"})},v=function(e){"Enter"===e.key&&p()};return ne((function(){n&&(a||l({type:"SET_TOP_N",payload:+n.topN}),u||l({type:"SET_MAX_LIFE_TIME",payload:n.maxLifetime}))}),[n]),Vr("div",{className:wo()({"vm-top-queries":!0,"vm-top-queries_mobile":e}),children:[i&&Vr(Qf,{containerStyles:{height:"500px"}}),Vr("div",{className:wo()({"vm-top-queries-controls":!0,"vm-block":!0,"vm-block_mobile":e}),children:[Vr("div",{className:"vm-top-queries-controls-fields",children:[Vr("div",{className:"vm-top-queries-controls-fields__item",children:Vr(xa,{label:"Max lifetime",value:u,error:d,helperText:"For example ".concat("30ms, 15s, 3d4h, 1y2w"),onChange:function(e){l({type:"SET_MAX_LIFE_TIME",payload:e})},onKeyDown:v})}),Vr("div",{className:"vm-top-queries-controls-fields__item",children:Vr(xa,{label:"Number of returned queries",type:"number",value:a||"",error:f,onChange:function(e){l({type:"SET_TOP_N",payload:+e})},onKeyDown:v})})]}),Vr("div",{className:wo()({"vm-top-queries-controls-bottom":!0,"vm-top-queries-controls-bottom_mobile":e}),children:[Vr("div",{className:"vm-top-queries-controls-bottom__info",children:["VictoriaMetrics tracks the last\xa0",Vr(ma,{title:"search.queryStats.lastQueriesCount",children:Vr("b",{children:h("search.queryStats.lastQueriesCount")})}),"\xa0queries with durations at least\xa0",Vr(ma,{title:"search.queryStats.minQueryDuration",children:Vr("b",{children:h("search.queryStats.minQueryDuration")})})]}),Vr("div",{className:"vm-top-queries-controls-bottom__button",children:Vr(fa,{startIcon:Vr(Xi,{}),onClick:p,children:"Execute"})})]})]}),r&&Vr(ta,{variant:"error",children:r}),n&&Vr(g,{children:Vr("div",{className:"vm-top-queries-panels",children:[Vr(Nh,{rows:n.topByCount,title:"Most frequently executed queries",columns:[{key:"query"},{key:"timeRangeHours",title:"time range, hours"},{key:"count"}]}),Vr(Nh,{rows:n.topByAvgDuration,title:"Most heavy queries",columns:[{key:"query"},{key:"avgDurationSeconds",title:"avg duration, seconds"},{key:"timeRangeHours",title:"time range, hours"},{key:"count"}],defaultOrderBy:"avgDurationSeconds"}),Vr(Nh,{rows:n.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"})]})})]})},Mh={"color-primary":"#589DF6","color-secondary":"#316eca","color-error":"#e5534b","color-warning":"#c69026","color-info":"#539bf5","color-success":"#57ab5a","color-background-body":"#22272e","color-background-block":"#2d333b","color-background-tooltip":"rgba(22, 22, 22, 0.8)","color-text":"#cdd9e5","color-text-secondary":"#768390","color-text-disabled":"#636e7b","box-shadow":"rgba(0, 0, 0, 0.16) 1px 2px 6px","box-shadow-popper":"rgba(0, 0, 0, 0.2) 0px 2px 8px 0px","border-divider":"1px solid rgba(99, 110, 123, 0.5)","color-hover-black":"rgba(0, 0, 0, 0.12)"},Th={"color-primary":"#3F51B5","color-secondary":"#E91E63","color-error":"#FD080E","color-warning":"#FF8308","color-info":"#03A9F4","color-success":"#4CAF50","color-background-body":"#FEFEFF","color-background-block":"#FFFFFF","color-background-tooltip":"rgba(97,97,97, 0.92)","color-text":"#110f0f","color-text-secondary":"#706F6F","color-text-disabled":"#A09F9F","box-shadow":"rgba(0, 0, 0, 0.08) 1px 2px 6px","box-shadow-popper":"rgba(0, 0, 0, 0.1) 0px 2px 8px 0px","border-divider":"1px solid rgba(0, 0, 0, 0.15)","color-hover-black":"rgba(0, 0, 0, 0.06)"},Oh=function(){var e=Ft(ee(jr()),2),t=e[0],n=e[1],r=function(e){n(e.matches)};return ne((function(){var e=window.matchMedia("(prefers-color-scheme: dark)");return e.addEventListener("change",r),function(){return e.removeEventListener("change",r)}}),[]),t},Bh=["primary","secondary","error","warning","info","success"],Lh=function(e){var t,n=e.onLoaded,r=Ar(),i=Cr().palette,o=void 0===i?{}:i,a=Wr().theme,u=Oh(),l=Qr(),c=bo(document.body),s=Ft(ee((mr(t={},br.dark,Mh),mr(t,br.light,Th),mr(t,br.system,jr()?Mh:Th),t)),2),f=s[0],d=s[1],h=function(){var e=window,t=e.innerWidth,n=e.innerHeight,r=document.documentElement,i=r.clientWidth,o=r.clientHeight;zr("scrollbar-width","".concat(t-i,"px")),zr("scrollbar-height","".concat(n-o,"px")),zr("vh","".concat(.01*n,"px"))},p=function(){Bh.forEach((function(e,t){var r=function(e){var t=e.replace("#","").trim();if(3===t.length&&(t=t[0]+t[0]+t[1]+t[1]+t[2]+t[2]),6!==t.length)throw new Error("Invalid HEX color.");return(299*parseInt(t.slice(0,2),16)+587*parseInt(t.slice(2,4),16)+114*parseInt(t.slice(4,6),16))/1e3>=128?"#000000":"#FFFFFF"}(Rr("color-".concat(e)));zr("".concat(e,"-text"),r),t===Bh.length-1&&(l({type:"SET_DARK_THEME"}),n(!0))}))},v=function(){var e=Ir("THEME")||br.system,t=f[e];Object.entries(t).forEach((function(e){var t=Ft(e,2),n=t[0],r=t[1];zr(n,r)})),p(),r&&(Bh.forEach((function(e){var t=o[e];t&&zr("color-".concat(e),t)})),p())};return ne((function(){h(),v()}),[f]),ne(h,[c]),ne((function(){var e=jr()?Mh:Th;f[br.system]!==e?d((function(t){return yr(yr({},t),{},mr({},br.system,e))})):v()}),[a,u]),ne((function(){r&&l({type:"SET_THEME",payload:br.light})}),[]),null},Ih=function(){var e=Ft(ee(!1),2),t=e[0],n=e[1],r=Ft(ee([]),2),i=r[0],o=r[1],a=Ft(ee([]),2),u=a[0],l=a[1],c=ae((function(){return!!i.length}),[i]),f=Ft(pr(),2)[1],d=function(){n(!0)},h=function(){n(!1)},p=function(e){var t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:"";l((function(n){return[{filename:t,text:": ".concat(e.message)}].concat(Ot(n))}))},v=function(e,t){try{var n=JSON.parse(e),r=n.trace||n;if(!r.duration_msec)return void p(new Error(_r.traceNotFound),t);var i=new Hf(r,t);o((function(e){return[i].concat(Ot(e))}))}catch(s){s instanceof Error&&p(s,t)}},m=function(e){l([]),Array.from(e.target.files||[]).map((function(e){var t=new FileReader,n=(null===e||void 0===e?void 0:e.name)||"";t.onload=function(e){var t,r=String(null===(t=e.target)||void 0===t?void 0:t.result);v(r,n)},t.readAsText(e)})),e.target.value=""},g=function(e){return function(){!function(e){l((function(t){return t.filter((function(t,n){return n!==e}))}))}(e)}};ne((function(){f({})}),[]);var y=function(){return Vr("div",{className:"vm-trace-page-controls",children:[Vr(fa,{variant:"outlined",onClick:d,children:"Paste JSON"}),Vr(ma,{title:"The file must contain tracing information in JSON format",children:Vr(fa,{children:["Upload Files",Vr("input",{id:"json",type:"file",accept:"application/json",multiple:!0,title:" ",onChange:m})]})})]})};return Vr("div",{className:"vm-trace-page",children:[Vr("div",{className:"vm-trace-page-header",children:[Vr("div",{className:"vm-trace-page-header-errors",children:u.map((function(e,t){return Vr("div",{className:"vm-trace-page-header-errors-item",children:[Vr(ta,{variant:"error",children:[Vr("b",{className:"vm-trace-page-header-errors-item__filename",children:e.filename}),Vr("span",{children:e.text})]}),Vr(fa,{className:"vm-trace-page-header-errors-item__close",startIcon:Vr(ji,{}),variant:"text",color:"error",onClick:g(t)})]},"".concat(e,"_").concat(t))}))}),Vr("div",{children:c&&Vr(y,{})})]}),c&&Vr("div",{children:Vr(Xf,{jsonEditor:!0,traces:i,onDeleteClick:function(e){var t=i.filter((function(t){return t.idValue!==e.idValue}));o(Ot(t))}})}),!c&&Vr("div",{className:"vm-trace-page-preview",children:[Vr("p",{className:"vm-trace-page-preview__text",children:["Please, upload file with JSON response content.","\n","The file must contain tracing information in JSON format.","\n","In order to use tracing please refer to the doc:\xa0",Vr("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/#query-tracing",target:"_blank",rel:"help noreferrer",children:"https://docs.victoriametrics.com/#query-tracing"}),"\n","Tracing graph will be displayed after file upload."]}),Vr(y,{})]}),t&&Vr(va,{title:"Paste JSON",onClose:h,children:Vr(Kf,{editable:!0,displayTitle:!0,defaultTile:"JSON ".concat(i.length+1),onClose:h,onUpload:v})})]})},Ph=function(e){var t=Wr().serverUrl,n=Ni().period,r=Ft(ee([]),2),i=r[0],o=r[1],a=Ft(ee(!1),2),u=a[0],l=a[1],c=Ft(ee(),2),s=c[0],f=c[1],d=ae((function(){return function(e,t,n){var r="{job=".concat(JSON.stringify(n),"}");return"".concat(e,"/api/v1/label/instance/values?match[]=").concat(encodeURIComponent(r),"&start=").concat(t.start,"&end=").concat(t.end)}(t,n,e)}),[t,n,e]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return l(!0),e.prev=1,e.next=4,fetch(d);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],o(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?f(void 0):f("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&f("".concat(e.t0.name,": ").concat(e.t0.message));case 16:l(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[d]),{instances:i,isLoading:u,error:s}},Rh=function(e,t){var n=Wr().serverUrl,r=Ni().period,i=Ft(ee([]),2),o=i[0],a=i[1],u=Ft(ee(!1),2),l=u[0],c=u[1],s=Ft(ee(),2),f=s[0],d=s[1],h=ae((function(){return function(e,t,n,r){var i=Object.entries({job:n,instance:r}).filter((function(e){return e[1]})).map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(n,"=").concat(JSON.stringify(r))})).join(","),o="{".concat(i,"}");return"".concat(e,"/api/v1/label/__name__/values?match[]=").concat(encodeURIComponent(o),"&start=").concat(t.start,"&end=").concat(t.end)}(n,r,e,t)}),[n,r,e,t]);return ne((function(){if(e){var t=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return c(!0),e.prev=1,e.next=4,fetch(h);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],a(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?d(void 0):d("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&d("".concat(e.t0.name,": ").concat(e.t0.message));case 16:c(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();t().catch(console.error)}}),[h]),{names:o,isLoading:l,error:f}},zh=function(e){var t=e.name,n=e.job,r=e.instance,i=e.rateEnabled,o=e.isBucket,a=e.height,u=Xo().isMobile,l=$o(),c=l.customStep,s=l.yaxis,f=Ni().period,d=Ho(),h=Fi(),p=Ft(ee(!1),2),v=p[0],m=p[1],g=ae((function(){var e=Object.entries({job:n,instance:r}).filter((function(e){return e[1]})).map((function(e){var t=Ft(e,2),n=t[0],r=t[1];return"".concat(n,"=").concat(JSON.stringify(r))}));e.push("__name__=".concat(JSON.stringify(t))),"node_cpu_seconds_total"==t&&e.push('mode!="idle"');var a="{".concat(e.join(","),"}");if(o)return r?'\nlabel_map(\n histogram_quantiles("__name__", 0.5, 0.95, 0.99, sum(rate('.concat(a,')) by (vmrange, le)),\n "__name__",\n "0.5", "q50",\n "0.95", "q95",\n "0.99", "q99",\n)'):"\nwith (q = histogram_quantile(0.95, sum(rate(".concat(a,')) by (instance, vmrange, le))) (\n alias(min(q), "q95min"),\n alias(max(q), "q95max"),\n alias(avg(q), "q95avg"),\n)');var u=i?"rollup_rate(".concat(a,")"):"rollup(".concat(a,")");return"\nwith (q = ".concat(u,') (\n alias(min(label_match(q, "rollup", "min")), "min"),\n alias(max(label_match(q, "rollup", "max")), "max"),\n alias(avg(label_match(q, "rollup", "avg")), "avg"),\n)')}),[t,n,r,i,o]),y=Uf({predefinedQuery:[g],visible:!0,customStep:c,showAllSeries:v}),_=y.isLoading,b=y.graphData,D=y.error,w=y.warning;return Vr("div",{className:wo()({"vm-explore-metrics-graph":!0,"vm-explore-metrics-graph_mobile":u}),children:[_&&Vr(Qf,{}),D&&Vr(ta,{variant:"error",children:D}),w&&Vr(ta,{variant:"warning",children:Vr("div",{className:"vm-explore-metrics-graph__warning",children:[Vr("p",{children:w}),Vr(fa,{color:"warning",variant:"outlined",onClick:function(){m(!0)},children:"Show all"})]})}),b&&f&&Vr(Nf,{data:b,period:f,customStep:c,query:[g],yaxis:s,setYaxisLimits:function(e){d({type:"SET_YAXIS_LIMITS",payload:e})},setPeriod:function(e){var t=e.from,n=e.to;h({type:"SET_PERIOD",payload:{from:t,to:n}})},showLegend:!1,height:a})]})},jh=function(e){var t=e.name,n=e.index,r=e.length,i=e.isBucket,o=e.rateEnabled,a=e.onChangeRate,u=e.onRemoveItem,l=e.onChangeOrder,c=Xo().isMobile,s=Ft(ee(!1),2),f=s[0],d=s[1],h=function(){u(t)},p=function(){l(t,n,n+1)},v=function(){l(t,n,n-1)};return Vr("div",c?{className:"vm-explore-metrics-item-header vm-explore-metrics-item-header_mobile",children:[Vr("div",{className:"vm-explore-metrics-item-header__name",children:t}),Vr(fa,{variant:"text",size:"small",startIcon:Vr(yo,{}),onClick:function(){d(!0)}}),f&&Vr(va,{title:t,onClose:function(){d(!1)},children:Vr("div",{className:"vm-explore-metrics-item-header-modal",children:[Vr("div",{className:"vm-explore-metrics-item-header-modal-order",children:[Vr(fa,{startIcon:Vr(ao,{}),variant:"outlined",onClick:v,disabled:0===n}),Vr("p",{children:["position:",Vr("span",{className:"vm-explore-metrics-item-header-modal-order__index",children:["#",n+1]})]}),Vr(fa,{endIcon:Vr(oo,{}),variant:"outlined",onClick:p,disabled:n===r-1})]}),!i&&Vr("div",{className:"vm-explore-metrics-item-header-modal__rate",children:[Vr(Tf,{label:Vr("span",{children:["enable ",Vr("code",{children:"rate()"})]}),value:o,onChange:a,fullWidth:!0}),Vr("p",{children:"calculates the average per-second speed of metrics change"})]}),Vr(fa,{startIcon:Vr(ji,{}),color:"error",variant:"outlined",onClick:h,fullWidth:!0,children:"Remove graph"})]})})]}:{className:"vm-explore-metrics-item-header",children:[Vr("div",{className:"vm-explore-metrics-item-header-order",children:[Vr(ma,{title:"move graph up",children:Vr(fa,{className:"vm-explore-metrics-item-header-order__up",startIcon:Vr(Wi,{}),variant:"text",color:"gray",size:"small",onClick:v})}),Vr("div",{className:"vm-explore-metrics-item-header__index",children:["#",n+1]}),Vr(ma,{title:"move graph down",children:Vr(fa,{className:"vm-explore-metrics-item-header-order__down",startIcon:Vr(Wi,{}),variant:"text",color:"gray",size:"small",onClick:p})})]}),Vr("div",{className:"vm-explore-metrics-item-header__name",children:t}),!i&&Vr("div",{className:"vm-explore-metrics-item-header__rate",children:Vr(ma,{title:"calculates the average per-second speed of metric's change",children:Vr(Tf,{label:Vr("span",{children:["enable ",Vr("code",{children:"rate()"})]}),value:o,onChange:a})})}),Vr("div",{className:"vm-explore-metrics-item-header__close",children:Vr(ma,{title:"close graph",children:Vr(fa,{startIcon:Vr(ji,{}),variant:"text",color:"gray",size:"small",onClick:h})})})]})},$h=function(e){var t=e.name,n=e.job,r=e.instance,i=e.index,o=e.length,a=e.size,u=e.onRemoveItem,l=e.onChangeOrder,c=ae((function(){return/_sum?|_total?|_count?/.test(t)}),[t]),s=ae((function(){return/_bucket?/.test(t)}),[t]),f=Ft(ee(c),2),d=f[0],h=f[1],p=bo(document.body),v=ae(a.height,[a,p]);return ne((function(){h(c)}),[n]),Vr("div",{className:"vm-explore-metrics-item vm-block vm-block_empty-padding",children:[Vr(jh,{name:t,index:i,length:o,isBucket:s,rateEnabled:d,size:a.id,onChangeRate:h,onRemoveItem:u,onChangeOrder:l}),Vr(zh,{name:t,job:n,instance:r,rateEnabled:d,isBucket:s,height:v},"".concat(t,"_").concat(n,"_").concat(r,"_").concat(d))]})},Hh=function(e){var t=e.values,n=e.onRemoveItem,r=Xo().isMobile;return r?Vr("span",{className:"vm-select-input-content__counter",children:["selected ",t.length]}):Vr(g,{children:t.map((function(e){return Vr("div",{className:"vm-select-input-content__selected",children:[Vr("span",{children:e}),Vr("div",{onClick:(t=e,function(e){n(t),e.stopPropagation()}),children:Vr(ji,{})})]},e);var t}))})},Uh=function(e){var t=e.value,n=e.list,r=e.label,i=e.placeholder,o=e.noOptionsText,a=e.clearable,u=void 0!==a&&a,l=e.searchable,c=void 0!==l&&l,s=e.autofocus,f=e.onChange,d=Wr().isDarkTheme,h=Xo().isMobile,p=Ft(ee(""),2),v=p[0],m=p[1],g=ie(null),y=Ft(ee(!1),2),_=y[0],b=y[1],D=ie(null),w=Array.isArray(t),x=Array.isArray(t)?t:void 0,k=h&&w&&!(null===x||void 0===x||!x.length),C=ae((function(){return _?v:Array.isArray(t)?"":t}),[t,v,_,w]),A=ae((function(){return _?v||"(.+)":""}),[v,_]),E=function(){D.current&&D.current.blur()},S=function(e){f(e),w||(b(!1),E()),w&&D.current&&D.current.focus()},N=function(e){D.current!==e.target&&b(!1)};return ne((function(){m(""),_&&D.current&&D.current.focus(),_||E()}),[_,D]),ne((function(){s&&D.current&&!h&&D.current.focus()}),[s,D]),ne((function(){return window.addEventListener("keyup",N),function(){window.removeEventListener("keyup",N)}}),[]),Vr("div",{className:wo()({"vm-select":!0,"vm-select_dark":d}),children:[Vr("div",{className:"vm-select-input",onClick:function(e){e.target instanceof HTMLInputElement||b((function(e){return!e}))},ref:g,children:[Vr("div",{className:"vm-select-input-content",children:[!(null===x||void 0===x||!x.length)&&Vr(Hh,{values:x,onRemoveItem:S}),!k&&Vr("input",{value:C,type:"text",placeholder:i,onInput:function(e){m(e.target.value)},onFocus:function(){b(!0)},ref:D,readOnly:h||!c})]}),r&&Vr("span",{className:"vm-text-field__label",children:r}),u&&t&&Vr("div",{className:"vm-select-input__icon",onClick:function(e){return function(t){S(e),t.stopPropagation()}}(""),children:Vr(ji,{})}),Vr("div",{className:wo()({"vm-select-input__icon":!0,"vm-select-input__icon_open":_}),children:Vr(Qi,{})})]}),Vr(Ff,{label:r,value:A,options:n,anchor:g,selected:x,maxWords:10,minLength:0,fullWidth:!0,noOptionsText:o,onSelect:S,onOpenAutocomplete:b})]})},Yh=Or.map((function(e){return e.id})),Vh=function(e){var t=e.jobs,n=e.instances,r=e.names,i=e.job,o=e.instance,a=e.size,u=e.selectedMetrics,l=e.onChangeJob,c=e.onChangeInstance,s=e.onToggleMetric,f=e.onChangeSize,d=ae((function(){return i?"":"No instances. Please select job"}),[i]),h=ae((function(){return i?"":"No metric names. Please select job"}),[i]),p=Xo().isMobile;return Vr("div",{className:wo()({"vm-explore-metrics-header":!0,"vm-explore-metrics-header_mobile":p,"vm-block":!0,"vm-block_mobile":p}),children:[Vr("div",{className:"vm-explore-metrics-header__job",children:Vr(Uh,{value:i,list:t,label:"Job",placeholder:"Please select job",onChange:l,autofocus:!i,searchable:!0})}),Vr("div",{className:"vm-explore-metrics-header__instance",children:Vr(Uh,{value:o,list:n,label:"Instance",placeholder:"Please select instance",onChange:c,noOptionsText:d,clearable:!0,searchable:!0})}),Vr("div",{className:"vm-explore-metrics-header__size",children:Vr(Uh,{label:"Size graphs",value:a,list:Yh,onChange:f})}),Vr("div",{className:"vm-explore-metrics-header-metrics",children:Vr(Uh,{label:"Metrics",value:u,list:r,placeholder:"Search metric name",onChange:s,noOptionsText:h,clearable:!0,searchable:!0})})]})},qh=Br("job",""),Wh=Br("instance",""),Qh=Br("metrics",""),Gh=Br("size",""),Jh=Or.find((function(e){return Gh?e.id===Gh:e.isDefault}))||Or[0],Zh=function(){var e=Ft(ee(qh),2),t=e[0],n=e[1],r=Ft(ee(Wh),2),i=r[0],o=r[1],a=Ft(ee(Qh?Qh.split("&"):[]),2),u=a[0],l=a[1],c=Ft(ee(Jh),2),s=c[0],f=c[1];!function(e){var t=e.job,n=e.instance,r=e.metrics,i=e.size,o=Ni(),a=o.duration,u=o.relativeTime,l=o.period.date,c=$o().customStep,s=Ft(pr(),2)[1],f=function(){var e,o=id((mr(e={},"g0.range_input",a),mr(e,"g0.end_input",l),mr(e,"g0.step_input",c),mr(e,"g0.relative_time",u),mr(e,"size",i),mr(e,"job",t),mr(e,"instance",n),mr(e,"metrics",r),e));s(o)};ne(f,[a,u,l,c,t,n,r,i]),ne(f,[])}({job:t,instance:i,metrics:u.join("&"),size:s.id});var d=function(){var e=Wr().serverUrl,t=Ni().period,n=Ft(ee([]),2),r=n[0],i=n[1],o=Ft(ee(!1),2),a=o[0],u=o[1],l=Ft(ee(),2),c=l[0],s=l[1],f=ae((function(){return function(e,t){return"".concat(e,"/api/v1/label/job/values?start=").concat(t.start,"&end=").concat(t.end)}(e,t)}),[e,t]);return ne((function(){var e=function(){var e=tu(Xa().mark((function e(){var t,n,r;return Xa().wrap((function(e){for(;;)switch(e.prev=e.next){case 0:return u(!0),e.prev=1,e.next=4,fetch(f);case 4:return t=e.sent,e.next=7,t.json();case 7:n=e.sent,r=n.data||[],i(r.sort((function(e,t){return e.localeCompare(t)}))),t.ok?s(void 0):s("".concat(n.errorType,"\r\n").concat(null===n||void 0===n?void 0:n.error)),e.next=16;break;case 13:e.prev=13,e.t0=e.catch(1),e.t0 instanceof Error&&s("".concat(e.t0.name,": ").concat(e.t0.message));case 16:u(!1);case 17:case"end":return e.stop()}}),e,null,[[1,13]])})));return function(){return e.apply(this,arguments)}}();e().catch(console.error)}),[f]),{jobs:r,isLoading:a,error:c}}(),h=d.jobs,p=d.isLoading,v=d.error,m=Ph(t),g=m.instances,y=m.isLoading,_=m.error,b=Rh(t,i),D=b.names,w=b.isLoading,x=b.error,k=ae((function(){return p||y||w}),[p,y,w]),C=ae((function(){return v||_||x}),[v,_,x]),A=function(e){l(e?function(t){return t.includes(e)?t.filter((function(t){return t!==e})):[].concat(Ot(t),[e])}:[])},E=function(e,t,n){var r=n>u.length-1;n<0||r||l((function(e){var r=Ot(e),i=Ft(r.splice(t,1),1)[0];return r.splice(n,0,i),r}))};return ne((function(){i&&g.length&&!g.includes(i)&&o("")}),[g,i]),Vr("div",{className:"vm-explore-metrics",children:[Vr(Vh,{jobs:h,instances:g,names:D,job:t,size:s.id,instance:i,selectedMetrics:u,onChangeJob:n,onChangeSize:function(e){var t=Or.find((function(t){return t.id===e}));t&&f(t)},onChangeInstance:o,onToggleMetric:A}),k&&Vr(Qf,{}),C&&Vr(ta,{variant:"error",children:C}),!t&&Vr(ta,{variant:"info",children:"Please select job to see list of metric names."}),t&&!u.length&&Vr(ta,{variant:"info",children:"Please select metric names to see the graphs."}),Vr("div",{className:"vm-explore-metrics-body",children:u.map((function(e,n){return Vr($h,{name:e,job:t,instance:i,index:n,length:u.length,size:s,onRemoveItem:A,onChangeOrder:E},e)}))})]})},Kh=function(){var e=ra().showInfoMessage,n=function(t){return function(){var n;n=t,navigator.clipboard.writeText("<".concat(n,"/>")),e({text:"<".concat(n,"/> has been copied"),type:"success"})}};return Vr("div",{className:"vm-preview-icons",children:Object.entries(t).map((function(e){var t=Ft(e,2),r=t[0],i=t[1];return Vr("div",{className:"vm-preview-icons-item",onClick:n(r),children:[Vr("div",{className:"vm-preview-icons-item__svg",children:i()}),Vr("div",{className:"vm-preview-icons-item__name",children:"<".concat(r,"/>")})]},r)}))})},Xh=function(){var e=Ft(ee(!1),2),t=e[0],n=e[1];return Vr(g,{children:Vr(cr,{children:Vr(la,{children:Vr(g,{children:[Vr(Lh,{onLoaded:n}),t&&Vr(nr,{children:Vr(er,{path:"/",element:Vr(su,{}),children:[Vr(er,{path:kr.home,element:Vr(ud,{})}),Vr(er,{path:kr.metrics,element:Vr(Zh,{})}),Vr(er,{path:kr.cardinality,element:Vr(Ah,{})}),Vr(er,{path:kr.topQueries,element:Vr(Fh,{})}),Vr(er,{path:kr.trace,element:Vr(Ih,{})}),Vr(er,{path:kr.dashboards,element:Vr(Qd,{})}),Vr(er,{path:kr.icons,element:Vr(Kh,{})})]})})]})})})})},ep=function(e){e&&n.e(27).then(n.bind(n,27)).then((function(t){var n=t.getCLS,r=t.getFID,i=t.getFCP,o=t.getLCP,a=t.getTTFB;n(e),r(e),i(e),o(e),a(e)}))},tp=document.getElementById("root");tp&&Ve(Vr(Xh,{}),tp),ep()}()}();
\ No newline at end of file
diff --git a/app/vmselect/vmui/static/js/main.0be86920.js.LICENSE.txt b/app/vmselect/vmui/static/js/main.6eed9ce1.js.LICENSE.txt
similarity index 100%
rename from app/vmselect/vmui/static/js/main.0be86920.js.LICENSE.txt
rename to app/vmselect/vmui/static/js/main.6eed9ce1.js.LICENSE.txt
diff --git a/app/vmui/packages/vmui/src/components/Layout/Layout.tsx b/app/vmui/packages/vmui/src/components/Layout/Layout.tsx
index 1d89a2984..7a34cac18 100644
--- a/app/vmui/packages/vmui/src/components/Layout/Layout.tsx
+++ b/app/vmui/packages/vmui/src/components/Layout/Layout.tsx
@@ -1,6 +1,7 @@
import Header from "./Header/Header";
import React, { FC, useEffect } from "preact/compat";
-import { Outlet, useLocation } from "react-router-dom";
+import { Outlet, useLocation, useSearchParams } from "react-router-dom";
+import qs from "qs";
import "./style.scss";
import { getAppModeEnable } from "../../utils/app-mode";
import classNames from "classnames";
@@ -12,14 +13,33 @@ import useDeviceDetect from "../../hooks/useDeviceDetect";
const Layout: FC = () => {
const appModeEnable = getAppModeEnable();
const { isMobile } = useDeviceDetect();
+ const { pathname } = useLocation();
+ const [searchParams, setSearchParams] = useSearchParams();
+
useFetchDashboards();
- const { pathname } = useLocation();
- useEffect(() => {
+ const setDocumentTitle = () => {
const defaultTitle = "vmui";
const routeTitle = routerOptions[pathname]?.title;
document.title = routeTitle ? `${routeTitle} - ${defaultTitle}` : defaultTitle;
- }, [pathname]);
+ };
+
+ // for support old links with search params
+ const redirectSearchToHashParams = () => {
+ const { search } = window.location;
+ if (search) {
+ const query = qs.parse(search, { ignoreQueryPrefix: true });
+ Object.entries(query).forEach(([key, value]) => {
+ searchParams.set(key, value as string);
+ setSearchParams(searchParams);
+ });
+ window.location.search = "";
+ }
+ window.location.replace(window.location.href.replace(/\/\?#\//, "/#/"));
+ };
+
+ useEffect(setDocumentTitle, [pathname]);
+ useEffect(redirectSearchToHashParams, []);
return
diff --git a/app/vmui/packages/vmui/src/components/Main/Select/MultipleSelectedValue/MultipleSelectedValue.tsx b/app/vmui/packages/vmui/src/components/Main/Select/MultipleSelectedValue/MultipleSelectedValue.tsx
new file mode 100644
index 000000000..50bb6f8b8
--- /dev/null
+++ b/app/vmui/packages/vmui/src/components/Main/Select/MultipleSelectedValue/MultipleSelectedValue.tsx
@@ -0,0 +1,42 @@
+import React, { FC } from "preact/compat";
+import useDeviceDetect from "../../../../hooks/useDeviceDetect";
+import { CloseIcon } from "../../Icons";
+import { MouseEvent } from "react";
+
+interface MultipleSelectedValueProps {
+ values: string[]
+ onRemoveItem: (val: string) => void
+}
+
+const MultipleSelectedValue: FC = ({ values, onRemoveItem }) => {
+ const { isMobile } = useDeviceDetect();
+
+ const createHandleClick = (value: string) => (e: MouseEvent) => {
+ onRemoveItem(value);
+ e.stopPropagation();
+ };
+
+ if (isMobile) {
+ return (
+
+ selected {values.length}
+
+ );
+ }
+
+ return <>
+ {values.map(item => (
+
+ ))}
+ >;
+};
+
+export default MultipleSelectedValue;
diff --git a/app/vmui/packages/vmui/src/components/Main/Select/Select.tsx b/app/vmui/packages/vmui/src/components/Main/Select/Select.tsx
index f0c45cacd..dba043960 100644
--- a/app/vmui/packages/vmui/src/components/Main/Select/Select.tsx
+++ b/app/vmui/packages/vmui/src/components/Main/Select/Select.tsx
@@ -6,6 +6,7 @@ import Autocomplete from "../Autocomplete/Autocomplete";
import { useAppState } from "../../../state/common/StateContext";
import "./style.scss";
import useDeviceDetect from "../../../hooks/useDeviceDetect";
+import MultipleSelectedValue from "./MultipleSelectedValue/MultipleSelectedValue";
interface SelectProps {
value: string | string[]
@@ -39,8 +40,9 @@ const Select: FC = ({
const inputRef = useRef(null);
- const isMultiple = useMemo(() => Array.isArray(value), [value]);
- const selectedValues = useMemo(() => Array.isArray(value) ? value : undefined, [isMultiple, value]);
+ const isMultiple = Array.isArray(value);
+ const selectedValues = Array.isArray(value) ? value : undefined;
+ const hideInput = isMobile && isMultiple && !!selectedValues?.length;
const textFieldValue = useMemo(() => {
if (openList) return search;
@@ -124,23 +126,13 @@ const Select: FC = ({
ref={autocompleteAnchorEl}
>
- {!isMobile && selectedValues && selectedValues.map(item => (
-
- ))}
- {isMobile && !!selectedValues?.length && (
-
- selected {selectedValues.length}
-
+ {!!selectedValues?.length && (
+
)}
- {!isMobile || (isMobile && (!selectedValues || !selectedValues?.length)) && (
+ {!hideInput && (
void
+ onRunQuery: () => void
}
-const QueryConfigurator: FC
= ({ error, queryOptions, onHideQuery }) => {
+const QueryConfigurator: FC = ({
+ error,
+ queryOptions,
+ onHideQuery,
+ onRunQuery
+}) => {
const { isMobile } = useDeviceDetect();
const { query, queryHistory, autocomplete } = useQueryState();
@@ -45,21 +51,22 @@ const QueryConfigurator: FC = ({ error, queryOptions, on
});
};
- const onRunQuery = () => {
+ const handleRunQuery = () => {
updateHistory();
queryDispatch({ type: "SET_QUERY", payload: stateQuery });
timeDispatch({ type: "RUN_QUERY" });
+ onRunQuery();
};
- const onAddQuery = () => {
+ const handleAddQuery = () => {
setStateQuery(prev => [...prev, ""]);
};
- const onRemoveQuery = (index: number) => {
+ const handleRemoveQuery = (index: number) => {
setStateQuery(prev => prev.filter((q, i) => i !== index));
};
- const onToggleHideQuery = (e: ReactMouseEvent, index: number) => {
+ const handleToggleHideQuery = (e: ReactMouseEvent, index: number) => {
const { ctrlKey, metaKey } = e;
const ctrlMetaKey = ctrlKey || metaKey;
@@ -95,17 +102,17 @@ const QueryConfigurator: FC = ({ error, queryOptions, on
};
const createHandlerRemoveQuery = (i: number) => () => {
- onRemoveQuery(i);
+ handleRemoveQuery(i);
setHideQuery(prev => prev.includes(i) ? prev.filter(n => n !== i) : prev.map(n => n > i ? n - 1: n));
};
const createHandlerHideQuery = (i: number) => (e: ReactMouseEvent) => {
- onToggleHideQuery(e, i);
+ handleToggleHideQuery(e, i);
};
useEffect(() => {
if (prevStateQuery && (stateQuery.length < prevStateQuery.length)) {
- onRunQuery();
+ handleRunQuery();
}
}, [stateQuery]);
@@ -137,7 +144,7 @@ const QueryConfigurator: FC = ({ error, queryOptions, on
error={error}
onArrowUp={createHandlerArrow(-1, i)}
onArrowDown={createHandlerArrow(1, i)}
- onEnter={onRunQuery}
+ onEnter={handleRunQuery}
onChange={createHandlerChangeQuery(i)}
label={`Query ${i + 1}`}
disabled={hideQuery.includes(i)}
@@ -173,7 +180,7 @@ const QueryConfigurator: FC = ({ error, queryOptions, on
{stateQuery.length < MAX_QUERY_FIELDS && (