diff --git a/README.md b/README.md index f4693e146..784919a2d 100644 --- a/README.md +++ b/README.md @@ -86,6 +86,7 @@ VictoriaMetrics has the following prominent features: * [Prometheus exposition format](#how-to-import-data-in-prometheus-exposition-format). * [InfluxDB line protocol](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) over HTTP, TCP and UDP. * [Graphite plaintext protocol](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) with [tags](https://graphite.readthedocs.io/en/latest/tags.html#carbon). + * [Statsd plaintext protocol](#how-to-send-data-from-statsd-compatible-clients) * [OpenTSDB put message](#sending-data-via-telnet-put-protocol). * [HTTP OpenTSDB /api/put requests](#sending-opentsdb-data-via-http-apiput-requests). * [JSON line format](#how-to-import-data-in-json-line-format). @@ -701,6 +702,45 @@ The `/api/v1/export` endpoint should return the following response: {"metric":{"__name__":"measurement_field2","tag1":"value1","tag2":"value2"},"values":[1.23],"timestamps":[1695902762311]} ``` +## How to send data from Statsd-compatible clients + +VictoriaMetrics supports extended statsd protocol with tags. Also it does not support sampling and metric types(it will be ignored). +Enable Statsd receiver in VictoriaMetrics by setting `-statsdListenAddr` command line flag. For instance, +the following command will enable Statsd receiver in VictoriaMetrics on TCP and UDP port `8125`: + +```console +/path/to/victoria-metrics-prod -statsdListenAddr=:8125 +``` + +Example for writing data with Statsd plaintext protocol to local VictoriaMetrics using `nc`: + +```console +echo "foo.bar:123|g|#foo:bar" | nc -N localhost 8125 +``` + +Explicit setting of timestamps is not supported for statsd protocol. Timestamp is set to the current time when VictoriaMetrics or vmagent receives it. + +An arbitrary number of lines delimited by `\n` (aka newline char) can be sent in one go. +After that the data may be read via [/api/v1/export](#how-to-export-data-in-json-line-format) endpoint: + +
+ +```console +curl -G 'http://localhost:8428/api/v1/export' -d 'match=foo.bar.baz' +``` + +
+ +The `/api/v1/export` endpoint should return the following response: + +```json +{"metric":{"__name__":"foo.bar.baz","tag1":"value1","tag2":"value2"},"values":[123],"timestamps":[1560277406000]} +``` + +Some examples of compatible statsd clients: +- [statsd-instrument](https://github.com/Shopify/statsd-instrument) +- [dogstatsd-ruby](https://github.com/DataDog/dogstatsd-ruby) +- [go-statsd-client](https://github.com/cactus/go-statsd-client) ## How to send data from Graphite-compatible agents such as [StatsD](https://github.com/etsy/statsd) Enable Graphite receiver in VictoriaMetrics by setting `-graphiteListenAddr` command line flag. For instance, @@ -1356,6 +1396,7 @@ Additionally, VictoriaMetrics can accept metrics via the following popular data * DataDog `submit metrics` API. See [these docs](#how-to-send-data-from-datadog-agent) for details. * InfluxDB line protocol. See [these docs](#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf) for details. * Graphite plaintext protocol. See [these docs](#how-to-send-data-from-graphite-compatible-agents-such-as-statsd) for details. +* Statsd plaintext protocol. See [these docs](#how-to-send-data-from-statsd-compatible-clients) for details. * OpenTelemetry http API. See [these docs](#sending-data-via-opentelemetry) for details. * OpenTSDB telnet put protocol. See [these docs](#sending-data-via-telnet-put-protocol) for details. * OpenTSDB http `/api/put` protocol. See [these docs](#sending-opentsdb-data-via-http-apiput-requests) for details. @@ -3162,6 +3203,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li An optional list of labels to drop from samples before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels Supports an array of values separated by comma or specified via multiple flags. Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces. + -streamAggr.ignoreFirstIntervals int + Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start -streamAggr.ignoreOldSamples Whether to ignore input samples with old timestamps outside the current aggregation interval. See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples -streamAggr.keepInput diff --git a/app/victoria-metrics/Makefile b/app/victoria-metrics/Makefile index f8fa3d089..e7f0f7f27 100644 --- a/app/victoria-metrics/Makefile +++ b/app/victoria-metrics/Makefile @@ -88,6 +88,9 @@ victoria-metrics-linux-ppc64le: victoria-metrics-linux-s390x: APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +victoria-metrics-linux-loong64: + APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + victoria-metrics-linux-386: APP_NAME=victoria-metrics CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmagent/Makefile b/app/vmagent/Makefile index 74c47f2df..e96a9c221 100644 --- a/app/vmagent/Makefile +++ b/app/vmagent/Makefile @@ -88,6 +88,9 @@ vmagent-linux-ppc64le: vmagent-linux-s390x: APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmagent-linux-loong64: + APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmagent-linux-386: APP_NAME=vmagent CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmagent/main.go b/app/vmagent/main.go index 0d8c910ed..e22fb4703 100644 --- a/app/vmagent/main.go +++ b/app/vmagent/main.go @@ -24,6 +24,7 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/prometheusimport" "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/promremotewrite" "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite" + "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/statsd" "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/vmimport" "github.com/VictoriaMetrics/VictoriaMetrics/lib/auth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/buildinfo" @@ -36,6 +37,7 @@ import ( influxserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/influx" opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb" opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp" + statsdserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/statsd" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape" @@ -61,6 +63,10 @@ var ( "See also -graphiteListenAddr.useProxyProtocol") graphiteUseProxyProtocol = flag.Bool("graphiteListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -graphiteListenAddr . "+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt") + statsdListenAddr = flag.String("statsdListenAddr", "", "TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. "+ + "See also -statsdListenAddr.useProxyProtocol") + statsdUseProxyProtocol = flag.Bool("statsdListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -statsdListenAddr . "+ + "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt") opentsdbListenAddr = flag.String("opentsdbListenAddr", "", "TCP and UDP address to listen for OpenTSDB metrics. "+ "Telnet put messages and HTTP /api/put messages are simultaneously served on TCP port. "+ "Usually :4242 must be set. Doesn't work if empty. See also -opentsdbListenAddr.useProxyProtocol") @@ -80,6 +86,7 @@ var ( var ( influxServer *influxserver.Server graphiteServer *graphiteserver.Server + statsdServer *statsdserver.Server opentsdbServer *opentsdbserver.Server opentsdbhttpServer *opentsdbhttpserver.Server ) @@ -137,6 +144,9 @@ func main() { if len(*graphiteListenAddr) > 0 { graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, *graphiteUseProxyProtocol, graphite.InsertHandler) } + if len(*statsdListenAddr) > 0 { + statsdServer = statsdserver.MustStart(*statsdListenAddr, *statsdUseProxyProtocol, statsd.InsertHandler) + } if len(*opentsdbListenAddr) > 0 { httpInsertHandler := getOpenTSDBHTTPInsertHandler() opentsdbServer = opentsdbserver.MustStart(*opentsdbListenAddr, *opentsdbUseProxyProtocol, opentsdb.InsertHandler, httpInsertHandler) @@ -172,6 +182,9 @@ func main() { if len(*graphiteListenAddr) > 0 { graphiteServer.MustStop() } + if len(*statsdListenAddr) > 0 { + statsdServer.MustStop() + } if len(*opentsdbListenAddr) > 0 { opentsdbServer.MustStop() } diff --git a/app/vmagent/remotewrite/pendingseries.go b/app/vmagent/remotewrite/pendingseries.go index efd760e6f..3204eb279 100644 --- a/app/vmagent/remotewrite/pendingseries.go +++ b/app/vmagent/remotewrite/pendingseries.go @@ -109,9 +109,10 @@ type writeRequest struct { wr prompbmarshal.WriteRequest - tss []prompbmarshal.TimeSeries - labels []prompbmarshal.Label - samples []prompbmarshal.Sample + tss []prompbmarshal.TimeSeries + labels []prompbmarshal.Label + samples []prompbmarshal.Sample + exemplars []prompbmarshal.Exemplar // buf holds labels data buf []byte @@ -129,6 +130,7 @@ func (wr *writeRequest) reset() { wr.labels = wr.labels[:0] wr.samples = wr.samples[:0] + wr.exemplars = wr.exemplars[:0] wr.buf = wr.buf[:0] } @@ -200,6 +202,7 @@ func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) { labelsDst := wr.labels labelsLen := len(wr.labels) samplesDst := wr.samples + exemplarsDst := wr.exemplars buf := wr.buf for i := range src.Labels { labelsDst = append(labelsDst, prompbmarshal.Label{}) @@ -216,8 +219,12 @@ func (wr *writeRequest) copyTimeSeries(dst, src *prompbmarshal.TimeSeries) { samplesDst = append(samplesDst, src.Samples...) dst.Samples = samplesDst[len(samplesDst)-len(src.Samples):] + exemplarsDst = append(exemplarsDst, src.Exemplars...) + dst.Exemplars = exemplarsDst[len(exemplarsDst)-len(src.Exemplars):] + wr.samples = samplesDst wr.labels = labelsDst + wr.exemplars = exemplarsDst wr.buf = buf } @@ -229,7 +236,6 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block // Nothing to push return true } - marshalConcurrencyCh <- struct{}{} bb := writeRequestBufPool.Get() @@ -266,6 +272,8 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block if len(wr.Timeseries) == 1 { // A single time series left. Recursively split its samples into smaller parts if possible. samples := wr.Timeseries[0].Samples + exemplars := wr.Timeseries[0].Exemplars + if len(samples) == 1 { logger.Warnf("dropping a sample for metric with too long labels exceeding -remoteWrite.maxBlockSize=%d bytes", maxUnpackedBlockSize.N) return true @@ -277,11 +285,16 @@ func tryPushWriteRequest(wr *prompbmarshal.WriteRequest, tryPushBlock func(block return false } wr.Timeseries[0].Samples = samples[n:] + // We do not want to send exemplars twice + wr.Timeseries[0].Exemplars = nil + if !tryPushWriteRequest(wr, tryPushBlock, isVMRemoteWrite) { wr.Timeseries[0].Samples = samples + wr.Timeseries[0].Exemplars = exemplars return false } wr.Timeseries[0].Samples = samples + wr.Timeseries[0].Exemplars = exemplars return true } timeseries := wr.Timeseries diff --git a/app/vmagent/remotewrite/pendingseries_test.go b/app/vmagent/remotewrite/pendingseries_test.go index 14b1bd451..487fdab49 100644 --- a/app/vmagent/remotewrite/pendingseries_test.go +++ b/app/vmagent/remotewrite/pendingseries_test.go @@ -10,8 +10,8 @@ import ( func TestPushWriteRequest(t *testing.T) { rowsCounts := []int{1, 10, 100, 1e3, 1e4} - expectedBlockLensProm := []int{216, 1848, 16424, 169882, 1757876} - expectedBlockLensVM := []int{138, 492, 3927, 34995, 288476} + expectedBlockLensProm := []int{248, 1952, 17433, 180381, 1861994} + expectedBlockLensVM := []int{170, 575, 4748, 44936, 367096} for i, rowsCount := range rowsCounts { expectedBlockLenProm := expectedBlockLensProm[i] expectedBlockLenVM := expectedBlockLensVM[i] @@ -59,6 +59,20 @@ func newTestWriteRequest(seriesCount, labelsCount int) *prompbmarshal.WriteReque Value: fmt.Sprintf("value_%d_%d", i, j), }) } + exemplar := prompbmarshal.Exemplar{ + Labels: []prompbmarshal.Label{ + { + Name: "trace_id", + Value: "123456", + }, + { + Name: "log_id", + Value: "987654", + }, + }, + Value: float64(i), + Timestamp: 1000 * int64(i), + } wr.Timeseries = append(wr.Timeseries, prompbmarshal.TimeSeries{ Labels: labels, Samples: []prompbmarshal.Sample{ @@ -67,6 +81,10 @@ func newTestWriteRequest(seriesCount, labelsCount int) *prompbmarshal.WriteReque Timestamp: 1000 * int64(i), }, }, + + Exemplars: []prompbmarshal.Exemplar{ + exemplar, + }, }) } return &wr diff --git a/app/vmagent/remotewrite/remotewrite.go b/app/vmagent/remotewrite/remotewrite.go index 41f41dc5f..13be88c5a 100644 --- a/app/vmagent/remotewrite/remotewrite.go +++ b/app/vmagent/remotewrite/remotewrite.go @@ -105,14 +105,16 @@ var ( "with -remoteWrite.streamAggr.config . See also -dedup.minScrapeInterval and https://docs.victoriametrics.com/stream-aggregation/#deduplication") streamAggrIgnoreOldSamples = flagutil.NewArrayBool("remoteWrite.streamAggr.ignoreOldSamples", "Whether to ignore input samples with old timestamps outside the current aggregation interval "+ "for the corresponding -remoteWrite.streamAggr.config . See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples") + streamAggrIgnoreFirstIntervals = flag.Int("remoteWrite.streamAggr.ignoreFirstIntervals", 0, "Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after vmagent restarts. It could be caused by receiving unordered delayed data from clients pushing data into the vmagent. "+ + "See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start") streamAggrDropInputLabels = flagutil.NewArrayString("streamAggr.dropInputLabels", "An optional list of labels to drop from samples "+ "before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels") - disableOnDiskQueue = flag.Bool("remoteWrite.disableOnDiskQueue", false, "Whether to disable storing pending data to -remoteWrite.tmpDataPath "+ - "when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence ."+ + disableOnDiskQueue = flagutil.NewArrayBool("remoteWrite.disableOnDiskQueue", "Whether to disable storing pending data to -remoteWrite.tmpDataPath "+ + "when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence ."+ "See also -remoteWrite.dropSamplesOnOverload") - dropSamplesOnOverload = flag.Bool("remoteWrite.dropSamplesOnOverload", false, "Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples "+ - "cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence") + dropSamplesOnOverload = flagutil.NewArrayBool("remoteWrite.dropSamplesOnOverload", "Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples "+ + "cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence") ) var ( @@ -133,6 +135,9 @@ var ( "see https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence"), StatusCode: http.StatusTooManyRequests, } + + // disableOnDiskQueueAll is set to true if all remoteWrite.urls were configured to disable persistent queue via disableOnDiskQueue + disableOnDiskQueueAll bool ) // MultitenancyEnabled returns true if -enableMultitenantHandlers or -remoteWrite.multitenantURL is specified. @@ -225,6 +230,15 @@ func Init() { if len(*remoteWriteURLs) > 0 { rwctxsDefault = newRemoteWriteCtxs(nil, *remoteWriteURLs) } + + disableOnDiskQueueAll = true + for _, v := range *disableOnDiskQueue { + if !v { + disableOnDiskQueueAll = false + break + } + } + dropDanglingQueues() // Start config reloader. @@ -257,6 +271,9 @@ func dropDanglingQueues() { // This is required for the case when the number of queues has been changed or URL have been changed. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4014 // + // In case if there were many persistent queues with identical *remoteWriteURLs + // the queue with the last index will be dropped. + // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/6140 existingQueues := make(map[string]struct{}, len(rwctxsDefault)) for _, rwctx := range rwctxsDefault { existingQueues[rwctx.fq.Dirname()] = struct{}{} @@ -435,10 +452,10 @@ func PushDropSamplesOnFailure(at *auth.Token, wr *prompbmarshal.WriteRequest) { // // The caller must return ErrQueueFullHTTPRetry to the client, which sends wr, if TryPush returns false. func TryPush(at *auth.Token, wr *prompbmarshal.WriteRequest) bool { - return tryPush(at, wr, *dropSamplesOnOverload) + return tryPush(at, wr, false) } -func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailure bool) bool { +func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, forceDropSamplesOnFailure bool) bool { tss := wr.Timeseries if at == nil && MultitenancyEnabled() { @@ -471,17 +488,18 @@ func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailur rowsCount := getRowsCount(tss) - if *disableOnDiskQueue { - // Quick check whether writes to configured remote storage systems are blocked. - // This allows saving CPU time spent on relabeling and block compression - // if some of remote storage systems cannot keep up with the data ingestion rate. + // Quick check whether writes to configured remote storage systems are blocked. + // This allows saving CPU time spent on relabeling and block compression + // if some of remote storage systems cannot keep up with the data ingestion rate. + // this shortcut is only applicable if all remote writes have disableOnDiskQueue = true + if disableOnDiskQueueAll { for _, rwctx := range rwctxs { if rwctx.fq.IsWriteBlocked() { - pushFailures.Inc() - if dropSamplesOnFailure { + rwctx.pushFailures.Inc() + if forceDropSamplesOnFailure || rwctx.dropSamplesOnOverload { // Just drop samples - samplesDropped.Add(rowsCount) - return true + rwctx.rowsDroppedOnPushFailure.Add(rowsCount) + continue } return false } @@ -534,27 +552,14 @@ func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailur } sortLabelsIfNeeded(tssBlock) tssBlock = limitSeriesCardinality(tssBlock) - if !tryPushBlockToRemoteStorages(rwctxs, tssBlock) { - if !*disableOnDiskQueue { - logger.Panicf("BUG: tryPushBlockToRemoteStorages must return true if -remoteWrite.disableOnDiskQueue isn't set") - } - pushFailures.Inc() - if dropSamplesOnFailure { - samplesDropped.Add(rowsCount) - return true - } + if !tryPushBlockToRemoteStorages(rwctxs, tssBlock, forceDropSamplesOnFailure) { return false } } return true } -var ( - samplesDropped = metrics.NewCounter(`vmagent_remotewrite_samples_dropped_total`) - pushFailures = metrics.NewCounter(`vmagent_remotewrite_push_failures_total`) -) - -func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries) bool { +func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, forceDropSamplesOnFailure bool) bool { if len(tssBlock) == 0 { // Nothing to push return true @@ -562,7 +567,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar if len(rwctxs) == 1 { // Fast path - just push data to the configured single remote storage - return rwctxs[0].TryPush(tssBlock) + return rwctxs[0].TryPush(tssBlock, forceDropSamplesOnFailure) } // We need to push tssBlock to multiple remote storages. @@ -573,7 +578,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar if replicas <= 0 { replicas = 1 } - return tryShardingBlockAmongRemoteStorages(rwctxs, tssBlock, replicas) + return tryShardingBlockAmongRemoteStorages(rwctxs, tssBlock, replicas, forceDropSamplesOnFailure) } // Replicate tssBlock samples among rwctxs. @@ -585,7 +590,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar for _, rwctx := range rwctxs { go func(rwctx *remoteWriteCtx) { defer wg.Done() - if !rwctx.TryPush(tssBlock) { + if !rwctx.TryPush(tssBlock, forceDropSamplesOnFailure) { anyPushFailed.Store(true) } }(rwctx) @@ -594,7 +599,7 @@ func tryPushBlockToRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmar return !anyPushFailed.Load() } -func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, replicas int) bool { +func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []prompbmarshal.TimeSeries, replicas int, forceDropSamplesOnFailure bool) bool { x := getTSSShards(len(rwctxs)) defer putTSSShards(x) @@ -648,7 +653,7 @@ func tryShardingBlockAmongRemoteStorages(rwctxs []*remoteWriteCtx, tssBlock []pr wg.Add(1) go func(rwctx *remoteWriteCtx, tss []prompbmarshal.TimeSeries) { defer wg.Done() - if !rwctx.TryPush(tss) { + if !rwctx.TryPush(tss, forceDropSamplesOnFailure) { anyPushFailed.Store(true) } }(rwctx, shard) @@ -781,14 +786,19 @@ type remoteWriteCtx struct { sas atomic.Pointer[streamaggr.Aggregators] deduplicator *streamaggr.Deduplicator - streamAggrKeepInput bool - streamAggrDropInput bool + streamAggrKeepInput bool + streamAggrDropInput bool + disableOnDiskQueue bool + dropSamplesOnOverload bool pss []*pendingSeries pssNextIdx atomic.Uint64 rowsPushedAfterRelabel *metrics.Counter rowsDroppedByRelabel *metrics.Counter + + pushFailures *metrics.Counter + rowsDroppedOnPushFailure *metrics.Counter } func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks int, sanitizedURL string) *remoteWriteCtx { @@ -804,7 +814,8 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in logger.Warnf("rounding the -remoteWrite.maxDiskUsagePerURL=%d to the minimum supported value: %d", maxPendingBytes, persistentqueue.DefaultChunkFileSize) maxPendingBytes = persistentqueue.DefaultChunkFileSize } - fq := persistentqueue.MustOpenFastQueue(queuePath, sanitizedURL, maxInmemoryBlocks, maxPendingBytes, *disableOnDiskQueue) + isPQDisabled := disableOnDiskQueue.GetOptionalArg(argIdx) + fq := persistentqueue.MustOpenFastQueue(queuePath, sanitizedURL, maxInmemoryBlocks, maxPendingBytes, isPQDisabled) _ = metrics.GetOrCreateGauge(fmt.Sprintf(`vmagent_remotewrite_pending_data_bytes{path=%q, url=%q}`, queuePath, sanitizedURL), func() float64 { return float64(fq.GetPendingBytes()) }) @@ -847,8 +858,14 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in c: c, pss: pss, + dropSamplesOnOverload: dropSamplesOnOverload.GetOptionalArg(argIdx), + disableOnDiskQueue: isPQDisabled, + rowsPushedAfterRelabel: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_rows_pushed_after_relabel_total{path=%q, url=%q}`, queuePath, sanitizedURL)), rowsDroppedByRelabel: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_relabel_metrics_dropped_total{path=%q, url=%q}`, queuePath, sanitizedURL)), + + pushFailures: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_push_failures_total{path=%q, url=%q}`, queuePath, sanitizedURL)), + rowsDroppedOnPushFailure: metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_samples_dropped_total{path=%q, url=%q}`, queuePath, sanitizedURL)), } // Initialize sas @@ -857,9 +874,10 @@ func newRemoteWriteCtx(argIdx int, remoteWriteURL *url.URL, maxInmemoryBlocks in ignoreOldSamples := streamAggrIgnoreOldSamples.GetOptionalArg(argIdx) if sasFile != "" { opts := &streamaggr.Options{ - DedupInterval: dedupInterval, - DropInputLabels: *streamAggrDropInputLabels, - IgnoreOldSamples: ignoreOldSamples, + DedupInterval: dedupInterval, + DropInputLabels: *streamAggrDropInputLabels, + IgnoreOldSamples: ignoreOldSamples, + IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals, } sas, err := streamaggr.LoadFromFile(sasFile, rwctx.pushInternalTrackDropped, opts) if err != nil { @@ -904,7 +922,11 @@ func (rwctx *remoteWriteCtx) MustStop() { rwctx.rowsDroppedByRelabel = nil } -func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool { +// TryPush sends tss series to the configured remote write endpoint +// +// TryPush can be called concurrently for multiple remoteWriteCtx, +// so it shouldn't modify tss entries. +func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries, forceDropSamplesOnFailure bool) bool { // Apply relabeling var rctx *relabelCtx var v *[]prompbmarshal.TimeSeries @@ -943,7 +965,6 @@ func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool { matchIdxsPool.Put(matchIdxs) } else if rwctx.deduplicator != nil { rwctx.deduplicator.Push(tss) - clear(tss) tss = tss[:0] } @@ -957,6 +978,14 @@ func (rwctx *remoteWriteCtx) TryPush(tss []prompbmarshal.TimeSeries) bool { putRelabelCtx(rctx) } + if !ok { + rwctx.pushFailures.Inc() + if forceDropSamplesOnFailure || rwctx.dropSamplesOnOverload { + rwctx.rowsDroppedOnPushFailure.Add(len(tss)) + return true + } + } + return ok } @@ -981,13 +1010,13 @@ func (rwctx *remoteWriteCtx) pushInternalTrackDropped(tss []prompbmarshal.TimeSe if rwctx.tryPushInternal(tss) { return } - if !*disableOnDiskQueue { + if !rwctx.disableOnDiskQueue { logger.Panicf("BUG: tryPushInternal must return true if -remoteWrite.disableOnDiskQueue isn't set") } - pushFailures.Inc() - if *dropSamplesOnOverload { + rwctx.pushFailures.Inc() + if dropSamplesOnOverload.GetOptionalArg(rwctx.idx) { rowsCount := getRowsCount(tss) - samplesDropped.Add(rowsCount) + rwctx.rowsDroppedOnPushFailure.Add(rowsCount) } } diff --git a/app/vmagent/remotewrite/remotewrite_test.go b/app/vmagent/remotewrite/remotewrite_test.go index 16c404ef2..6c384ba2e 100644 --- a/app/vmagent/remotewrite/remotewrite_test.go +++ b/app/vmagent/remotewrite/remotewrite_test.go @@ -3,9 +3,16 @@ package remotewrite import ( "fmt" "math" + "os" + "reflect" "testing" + "time" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/streamaggr" + "github.com/VictoriaMetrics/metrics" ) func TestGetLabelsHash_Distribution(t *testing.T) { @@ -46,3 +53,163 @@ func TestGetLabelsHash_Distribution(t *testing.T) { f(5) f(10) } + +func TestRemoteWriteContext_TryPush_ImmutableTimeseries(t *testing.T) { + f := func(streamAggrConfig, relabelConfig string, dedupInterval time.Duration, keepInput, dropInput bool, input string) { + t.Helper() + perURLRelabel, err := promrelabel.ParseRelabelConfigsData([]byte(relabelConfig)) + if err != nil { + t.Fatalf("cannot load relabel configs: %s", err) + } + rcs := &relabelConfigs{ + perURL: []*promrelabel.ParsedConfigs{ + perURLRelabel, + }, + } + allRelabelConfigs.Store(rcs) + + pss := make([]*pendingSeries, 1) + pss[0] = newPendingSeries(nil, true, 0, 100) + rwctx := &remoteWriteCtx{ + idx: 0, + streamAggrKeepInput: keepInput, + streamAggrDropInput: dropInput, + pss: pss, + rowsPushedAfterRelabel: metrics.GetOrCreateCounter(`foo`), + rowsDroppedByRelabel: metrics.GetOrCreateCounter(`bar`), + } + if dedupInterval > 0 { + rwctx.deduplicator = streamaggr.NewDeduplicator(nil, dedupInterval, nil) + } + + if len(streamAggrConfig) > 0 { + f := createFile(t, []byte(streamAggrConfig)) + sas, err := streamaggr.LoadFromFile(f.Name(), nil, nil) + if err != nil { + t.Fatalf("cannot load streamaggr configs: %s", err) + } + rwctx.sas.Store(sas) + } + + inputTss := mustParsePromMetrics(input) + expectedTss := make([]prompbmarshal.TimeSeries, len(inputTss)) + + // copy inputTss to make sure it is not mutated during TryPush call + copy(expectedTss, inputTss) + rwctx.TryPush(inputTss, false) + + if !reflect.DeepEqual(expectedTss, inputTss) { + t.Fatalf("unexpected samples;\ngot\n%v\nwant\n%v", inputTss, expectedTss) + } + } + + f(` +- interval: 1m + outputs: [sum_samples] +- interval: 2m + outputs: [count_series] +`, ` +- action: keep + source_labels: [env] + regex: "dev" +`, 0, false, false, ` +metric{env="dev"} 10 +metric{env="bar"} 20 +metric{env="dev"} 15 +metric{env="bar"} 25 +`) + f(``, ``, time.Hour, false, false, ` +metric{env="dev"} 10 +metric{env="foo"} 20 +metric{env="dev"} 15 +metric{env="foo"} 25 +`) + f(``, ` +- action: keep + source_labels: [env] + regex: "dev" +`, time.Hour, false, false, ` +metric{env="dev"} 10 +metric{env="bar"} 20 +metric{env="dev"} 15 +metric{env="bar"} 25 +`) + f(``, ` +- action: keep + source_labels: [env] + regex: "dev" +`, time.Hour, true, false, ` +metric{env="test"} 10 +metric{env="dev"} 20 +metric{env="foo"} 15 +metric{env="dev"} 25 +`) + f(``, ` +- action: keep + source_labels: [env] + regex: "dev" +`, time.Hour, false, true, ` +metric{env="foo"} 10 +metric{env="dev"} 20 +metric{env="foo"} 15 +metric{env="dev"} 25 +`) + f(``, ` +- action: keep + source_labels: [env] + regex: "dev" +`, time.Hour, true, true, ` +metric{env="dev"} 10 +metric{env="test"} 20 +metric{env="dev"} 15 +metric{env="bar"} 25 +`) +} + +func mustParsePromMetrics(s string) []prompbmarshal.TimeSeries { + var rows prometheus.Rows + errLogger := func(s string) { + panic(fmt.Errorf("unexpected error when parsing Prometheus metrics: %s", s)) + } + rows.UnmarshalWithErrLogger(s, errLogger) + var tss []prompbmarshal.TimeSeries + samples := make([]prompbmarshal.Sample, 0, len(rows.Rows)) + for _, row := range rows.Rows { + labels := make([]prompbmarshal.Label, 0, len(row.Tags)+1) + labels = append(labels, prompbmarshal.Label{ + Name: "__name__", + Value: row.Metric, + }) + for _, tag := range row.Tags { + labels = append(labels, prompbmarshal.Label{ + Name: tag.Key, + Value: tag.Value, + }) + } + samples = append(samples, prompbmarshal.Sample{ + Value: row.Value, + Timestamp: row.Timestamp, + }) + ts := prompbmarshal.TimeSeries{ + Labels: labels, + Samples: samples[len(samples)-1:], + } + tss = append(tss, ts) + } + return tss +} + +func createFile(t *testing.T, data []byte) *os.File { + t.Helper() + f, err := os.CreateTemp("", "") + if err != nil { + t.Fatal(err) + } + if err := os.WriteFile(f.Name(), data, 0644); err != nil { + t.Fatal(err) + } + if err := f.Sync(); err != nil { + t.Fatal(err) + } + return f +} diff --git a/app/vmagent/statsd/request_handler.go b/app/vmagent/statsd/request_handler.go new file mode 100644 index 000000000..9cf8fd667 --- /dev/null +++ b/app/vmagent/statsd/request_handler.go @@ -0,0 +1,68 @@ +package statsd + +import ( + "io" + + "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/common" + "github.com/VictoriaMetrics/VictoriaMetrics/app/vmagent/remotewrite" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/auth" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" + parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd/stream" + "github.com/VictoriaMetrics/metrics" +) + +var ( + rowsInserted = metrics.NewCounter(`vmagent_rows_inserted_total{type="statsd"}`) + rowsPerInsert = metrics.NewHistogram(`vmagent_rows_per_insert{type="statsd"}`) +) + +// InsertHandler processes remote write for statsd plaintext protocol. +// +// See https://github.com/statsd/statsd/blob/master/docs/metric_types.md +func InsertHandler(r io.Reader) error { + return stream.Parse(r, false, func(rows []parser.Row) error { + return insertRows(nil, rows) + }) +} + +func insertRows(at *auth.Token, rows []parser.Row) error { + ctx := common.GetPushCtx() + defer common.PutPushCtx(ctx) + + tssDst := ctx.WriteRequest.Timeseries[:0] + labels := ctx.Labels[:0] + samples := ctx.Samples[:0] + for i := range rows { + r := &rows[i] + labelsLen := len(labels) + labels = append(labels, prompbmarshal.Label{ + Name: "__name__", + Value: r.Metric, + }) + for j := range r.Tags { + tag := &r.Tags[j] + labels = append(labels, prompbmarshal.Label{ + Name: tag.Key, + Value: tag.Value, + }) + } + samples = append(samples, prompbmarshal.Sample{ + Value: r.Value, + Timestamp: r.Timestamp, + }) + tssDst = append(tssDst, prompbmarshal.TimeSeries{ + Labels: labels[labelsLen:], + Samples: samples[len(samples)-1:], + }) + } + ctx.WriteRequest.Timeseries = tssDst + ctx.Labels = labels + ctx.Samples = samples + if !remotewrite.TryPush(at, &ctx.WriteRequest) { + return remotewrite.ErrQueueFullHTTPRetry + } + rowsInserted.Add(len(rows)) + rowsPerInsert.Update(float64(len(rows))) + return nil +} diff --git a/app/vmalert/Makefile b/app/vmalert/Makefile index 0c57ffb29..610a8e137 100644 --- a/app/vmalert/Makefile +++ b/app/vmalert/Makefile @@ -119,6 +119,9 @@ vmalert-linux-ppc64le: vmalert-linux-s390x: APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmalert-linux-loong64: + APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmalert-linux-386: APP_NAME=vmalert CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmalert/rule/rule.go b/app/vmalert/rule/rule.go index 46e388c7f..e5e224881 100644 --- a/app/vmalert/rule/rule.go +++ b/app/vmalert/rule/rule.go @@ -32,7 +32,7 @@ type Rule interface { close() } -var errDuplicate = errors.New("result contains metrics with the same labelset after applying rule labels. See https://docs.victoriametrics.com/vmalert/#series-with-the-same-labelset for details") +var errDuplicate = errors.New("result contains metrics with the same labelset during evaluation. See https://docs.victoriametrics.com/vmalert/#series-with-the-same-labelset for details") type ruleState struct { sync.RWMutex diff --git a/app/vmalert/static/js/custom.js b/app/vmalert/static/js/custom.js index 25adebd2b..c79d342d2 100644 --- a/app/vmalert/static/js/custom.js +++ b/app/vmalert/static/js/custom.js @@ -13,6 +13,20 @@ function collapseAll() { $('.collapse').removeClass('show'); } +function showByID(id) { + if (!id) { + return + } + let parent = $("#" + id).parent(); + if (!parent) { + return + } + let target = $("#" + parent.attr("data-bs-target")); + if (target.length > 0) { + target.addClass('show'); + } +} + function toggleByID(id) { if (id) { let el = $("#" + id); @@ -61,7 +75,7 @@ function search() { function setParamURL(key, value) { let url = new URL(location.href) url.searchParams.set(key, value); - window.history.replaceState(null, null, `?${url.searchParams.toString()}`); + window.history.replaceState(null, null, `?${url.searchParams.toString()}${url.hash}`); } function getParamURL(key) { @@ -141,7 +155,7 @@ $(document).ready(function () { search() let hash = window.location.hash.substr(1); - toggleByID(hash); + showByID(hash); }); $(document).ready(function () { diff --git a/app/vmauth/Makefile b/app/vmauth/Makefile index 88fdbcd66..b3420a11c 100644 --- a/app/vmauth/Makefile +++ b/app/vmauth/Makefile @@ -87,6 +87,9 @@ vmauth-linux-ppc64le: vmauth-linux-s390x: APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmauth-linux-loong64: + APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmauth-linux-386: APP_NAME=vmauth CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmauth/auth_config_test.go b/app/vmauth/auth_config_test.go index b7f8eea4f..ca5aca2ea 100644 --- a/app/vmauth/auth_config_test.go +++ b/app/vmauth/auth_config_test.go @@ -685,6 +685,70 @@ func isSetBool(boolP *bool, expectedValue bool) bool { return *boolP == expectedValue } +func TestGetLeastLoadedBackendURL(t *testing.T) { + up := mustParseURLs([]string{ + "http://node1:343", + "http://node2:343", + "http://node3:343", + }) + up.loadBalancingPolicy = "least_loaded" + + fn := func(ns ...int) { + t.Helper() + bus := up.bus.Load() + pbus := *bus + for i, b := range pbus { + got := int(b.concurrentRequests.Load()) + exp := ns[i] + if got != exp { + t.Fatalf("expected %q to have %d concurrent requests; got %d instead", b.url, exp, got) + } + } + } + + up.getBackendURL() + fn(0, 1, 0) + up.getBackendURL() + fn(0, 1, 1) + up.getBackendURL() + fn(1, 1, 1) + + up.getBackendURL() + up.getBackendURL() + fn(1, 2, 2) + + bus := up.bus.Load() + pbus := *bus + pbus[0].concurrentRequests.Add(2) + pbus[2].concurrentRequests.Add(5) + fn(3, 2, 7) + + up.getBackendURL() + fn(3, 3, 7) + + up.getBackendURL() + fn(3, 4, 7) + + up.getBackendURL() + fn(4, 4, 7) + + up.getBackendURL() + fn(5, 4, 7) + + up.getBackendURL() + fn(5, 5, 7) + + up.getBackendURL() + fn(6, 5, 7) + + up.getBackendURL() + fn(6, 6, 7) + + up.getBackendURL() + up.getBackendURL() + fn(7, 7, 7) +} + func getRegexs(paths []string) []*Regex { var sps []*Regex for _, path := range paths { diff --git a/app/vmbackup/Makefile b/app/vmbackup/Makefile index 1e88ca68f..a36791e77 100644 --- a/app/vmbackup/Makefile +++ b/app/vmbackup/Makefile @@ -81,6 +81,9 @@ vmbackup-linux-ppc64le: vmbackup-linux-s390x: APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmbackup-linux-loong64: + APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmbackup-linux-386: APP_NAME=vmbackup CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmctl/Makefile b/app/vmctl/Makefile index 68155f9b7..589e40523 100644 --- a/app/vmctl/Makefile +++ b/app/vmctl/Makefile @@ -81,6 +81,9 @@ vmctl-linux-ppc64le: vmctl-linux-s390x: APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmctl-linux-loong64: + APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmctl-linux-386: APP_NAME=vmctl CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vminsert/common/streamaggr.go b/app/vminsert/common/streamaggr.go index 680956428..7a8ae0e11 100644 --- a/app/vminsert/common/streamaggr.go +++ b/app/vminsert/common/streamaggr.go @@ -32,6 +32,8 @@ var ( "See also -streamAggr.dropInputLabels and -dedup.minScrapeInterval and https://docs.victoriametrics.com/stream-aggregation/#deduplication") streamAggrDropInputLabels = flagutil.NewArrayString("streamAggr.dropInputLabels", "An optional list of labels to drop from samples "+ "before stream de-duplication and aggregation . See https://docs.victoriametrics.com/stream-aggregation/#dropping-unneeded-labels") + streamAggrIgnoreFirstIntervals = flag.Int("streamAggr.ignoreFirstIntervals", 0, "Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after restarts. It could be caused by receiving unordered delayed data from clients pushing data into the database. "+ + "See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start") streamAggrIgnoreOldSamples = flag.Bool("streamAggr.ignoreOldSamples", false, "Whether to ignore input samples with old timestamps outside the current aggregation interval. "+ "See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples") ) @@ -56,9 +58,10 @@ func CheckStreamAggrConfig() error { } pushNoop := func(_ []prompbmarshal.TimeSeries) {} opts := &streamaggr.Options{ - DedupInterval: *streamAggrDedupInterval, - DropInputLabels: *streamAggrDropInputLabels, - IgnoreOldSamples: *streamAggrIgnoreOldSamples, + DedupInterval: *streamAggrDedupInterval, + DropInputLabels: *streamAggrDropInputLabels, + IgnoreOldSamples: *streamAggrIgnoreOldSamples, + IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals, } sas, err := streamaggr.LoadFromFile(*streamAggrConfig, pushNoop, opts) if err != nil { @@ -84,9 +87,10 @@ func InitStreamAggr() { sighupCh := procutil.NewSighupChan() opts := &streamaggr.Options{ - DedupInterval: *streamAggrDedupInterval, - DropInputLabels: *streamAggrDropInputLabels, - IgnoreOldSamples: *streamAggrIgnoreOldSamples, + DedupInterval: *streamAggrDedupInterval, + DropInputLabels: *streamAggrDropInputLabels, + IgnoreOldSamples: *streamAggrIgnoreOldSamples, + IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals, } sas, err := streamaggr.LoadFromFile(*streamAggrConfig, pushAggregateSeries, opts) if err != nil { @@ -117,9 +121,10 @@ func reloadStreamAggrConfig() { saCfgReloads.Inc() opts := &streamaggr.Options{ - DedupInterval: *streamAggrDedupInterval, - DropInputLabels: *streamAggrDropInputLabels, - IgnoreOldSamples: *streamAggrIgnoreOldSamples, + DedupInterval: *streamAggrDedupInterval, + DropInputLabels: *streamAggrDropInputLabels, + IgnoreOldSamples: *streamAggrIgnoreOldSamples, + IgnoreFirstIntervals: *streamAggrIgnoreFirstIntervals, } sasNew, err := streamaggr.LoadFromFile(*streamAggrConfig, pushAggregateSeries, opts) if err != nil { diff --git a/app/vminsert/main.go b/app/vminsert/main.go index f7c383e11..3333e5c13 100644 --- a/app/vminsert/main.go +++ b/app/vminsert/main.go @@ -26,6 +26,7 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/prompush" "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/promremotewrite" "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/relabel" + "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/statsd" "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/vmimport" "github.com/VictoriaMetrics/VictoriaMetrics/lib/auth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" @@ -36,6 +37,7 @@ import ( influxserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/influx" opentsdbserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdb" opentsdbhttpserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/opentsdbhttp" + statsdserver "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver/statsd" "github.com/VictoriaMetrics/VictoriaMetrics/lib/procutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape" @@ -49,6 +51,10 @@ var ( "See also -graphiteListenAddr.useProxyProtocol") graphiteUseProxyProtocol = flag.Bool("graphiteListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -graphiteListenAddr . "+ "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt") + statsdListenAddr = flag.String("statsdListenAddr", "", "TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. "+ + "See also -statsdListenAddr.useProxyProtocol") + statsdUseProxyProtocol = flag.Bool("statsdListenAddr.useProxyProtocol", false, "Whether to use proxy protocol for connections accepted at -statsdListenAddr . "+ + "See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt") 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://:8428/write . "+ "See also -influxListenAddr.useProxyProtocol") @@ -72,6 +78,7 @@ var ( var ( graphiteServer *graphiteserver.Server + statsdServer *statsdserver.Server influxServer *influxserver.Server opentsdbServer *opentsdbserver.Server opentsdbhttpServer *opentsdbhttpserver.Server @@ -92,6 +99,9 @@ func Init() { if len(*graphiteListenAddr) > 0 { graphiteServer = graphiteserver.MustStart(*graphiteListenAddr, *graphiteUseProxyProtocol, graphite.InsertHandler) } + if len(*statsdListenAddr) > 0 { + statsdServer = statsdserver.MustStart(*statsdListenAddr, *statsdUseProxyProtocol, statsd.InsertHandler) + } if len(*influxListenAddr) > 0 { influxServer = influxserver.MustStart(*influxListenAddr, *influxUseProxyProtocol, influx.InsertHandlerForReader) } @@ -112,6 +122,9 @@ func Stop() { if len(*graphiteListenAddr) > 0 { graphiteServer.MustStop() } + if len(*statsdListenAddr) > 0 { + statsdServer.MustStop() + } if len(*influxListenAddr) > 0 { influxServer.MustStop() } diff --git a/app/vminsert/statsd/request_handler.go b/app/vminsert/statsd/request_handler.go new file mode 100644 index 000000000..b5901cfc3 --- /dev/null +++ b/app/vminsert/statsd/request_handler.go @@ -0,0 +1,54 @@ +package statsd + +import ( + "io" + + "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/common" + "github.com/VictoriaMetrics/VictoriaMetrics/app/vminsert/relabel" + parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd/stream" + "github.com/VictoriaMetrics/metrics" +) + +var ( + rowsInserted = metrics.NewCounter(`vm_rows_inserted_total{type="statsd"}`) + rowsPerInsert = metrics.NewHistogram(`vm_rows_per_insert{type="statsd"}`) +) + +// InsertHandler processes remote write for statsd protocol with tags. +// +// https://github.com/statsd/statsd/blob/master/docs/metric_types.md +func InsertHandler(r io.Reader) error { + return stream.Parse(r, false, insertRows) +} + +func insertRows(rows []parser.Row) error { + ctx := common.GetInsertCtx() + defer common.PutInsertCtx(ctx) + + ctx.Reset(len(rows)) + hasRelabeling := relabel.HasRelabeling() + for i := range rows { + r := &rows[i] + ctx.Labels = ctx.Labels[:0] + ctx.AddLabel("", r.Metric) + for j := range r.Tags { + tag := &r.Tags[j] + ctx.AddLabel(tag.Key, tag.Value) + } + if hasRelabeling { + ctx.ApplyRelabeling() + } + if len(ctx.Labels) == 0 { + // Skip metric without labels. + continue + } + ctx.SortLabelsIfNeeded() + if err := ctx.WriteDataPoint(nil, ctx.Labels, r.Timestamp, r.Value); err != nil { + return err + } + } + rowsInserted.Add(len(rows)) + rowsPerInsert.Update(float64(len(rows))) + return ctx.FlushBufs() +} diff --git a/app/vmrestore/Makefile b/app/vmrestore/Makefile index 824d50695..76d7e3bde 100644 --- a/app/vmrestore/Makefile +++ b/app/vmrestore/Makefile @@ -81,6 +81,9 @@ vmrestore-linux-ppc64le: vmrestore-linux-s390x: APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=s390x $(MAKE) app-local-goos-goarch +vmrestore-linux-loong64: + APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=loong64 $(MAKE) app-local-goos-goarch + vmrestore-linux-386: APP_NAME=vmrestore CGO_ENABLED=0 GOOS=linux GOARCH=386 $(MAKE) app-local-goos-goarch diff --git a/app/vmselect/promql/exec.go b/app/vmselect/promql/exec.go index c62170a75..c38de1293 100644 --- a/app/vmselect/promql/exec.go +++ b/app/vmselect/promql/exec.go @@ -13,6 +13,7 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/netstorage" "github.com/VictoriaMetrics/VictoriaMetrics/app/vmselect/querystats" "github.com/VictoriaMetrics/VictoriaMetrics/lib/decimal" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/querytracer" "github.com/VictoriaMetrics/VictoriaMetrics/lib/storage" "github.com/VictoriaMetrics/metrics" @@ -26,6 +27,12 @@ var ( `For example, foo{bar=~"a.b.c"} will be automatically converted to foo{bar=~"a\\.b\\.c"}, i.e. all the dots in regexp filters will be automatically escaped `+ `in order to match only dot char instead of matching any char. Dots in ".+", ".*" and ".{n}" regexps aren't escaped. `+ `This option is DEPRECATED in favor of {__graphite__="a.*.c"} syntax for selecting metrics matching the given Graphite metrics filter`) + disableImplicitConversion = flag.Bool("search.disableImplicitConversion", false, "Whether to return an error for queries that rely on implicit subquery conversions, "+ + "see https://docs.victoriametrics.com/metricsql/#subqueries for details. "+ + "See also -search.logImplicitConversion.") + logImplicitConversion = flag.Bool("search.logImplicitConversion", false, "Whether to log queries with implicit subquery conversions, "+ + "see https://docs.victoriametrics.com/metricsql/#subqueries for details. "+ + "Such conversion can be disabled using -search.disableImplicitConversion.") ) // UserReadableError is a type of error which supposed to be returned to the user without additional context. @@ -63,6 +70,16 @@ func Exec(qt *querytracer.Tracer, ec *EvalConfig, q string, isFirstPointOnly boo return nil, err } + if *disableImplicitConversion || *logImplicitConversion { + complete := isSubQueryComplete(e, false) + if !complete && *disableImplicitConversion { + return nil, fmt.Errorf("query contains subquery that requires implicit conversion and is rejected according to `-search.disableImplicitConversion=true` setting. See https://docs.victoriametrics.com/metricsql/#subqueries for details") + } + if !complete && *logImplicitConversion { + logger.Warnf("query=%q contains subquery that requires implicit conversion, see https://docs.victoriametrics.com/metricsql/#subqueries for details", e.AppendString(nil)) + } + } + qid := activeQueriesV.Add(ec, q) rv, err := evalExpr(qt, ec, e) activeQueriesV.Remove(qid) @@ -404,3 +421,55 @@ func (pc *parseCache) Put(q string, pcv *parseCacheValue) { pc.m[q] = pcv pc.mu.Unlock() } + +// isSubQueryComplete checks if expr contains incomplete subquery +func isSubQueryComplete(e metricsql.Expr, isSubExpr bool) bool { + switch exp := e.(type) { + case *metricsql.FuncExpr: + if isSubExpr { + return false + } + fe := e.(*metricsql.FuncExpr) + for _, arg := range exp.Args { + if getRollupFunc(fe.Name) != nil { + isSubExpr = true + } + if !isSubQueryComplete(arg, isSubExpr) { + return false + } + } + case *metricsql.RollupExpr: + if _, ok := exp.Expr.(*metricsql.MetricExpr); ok { + return true + } + // exp.Step is optional in subqueries + if exp.Window == nil { + return false + } + return isSubQueryComplete(exp.Expr, false) + case *metricsql.AggrFuncExpr: + if isSubExpr { + return false + } + for _, arg := range exp.Args { + if !isSubQueryComplete(arg, false) { + return false + } + } + case *metricsql.BinaryOpExpr: + if isSubExpr { + return false + } + if !isSubQueryComplete(exp.Left, false) { + return false + } + if !isSubQueryComplete(exp.Right, false) { + return false + } + case *metricsql.MetricExpr: + return true + default: + return true + } + return true +} diff --git a/app/vmselect/promql/exec_test.go b/app/vmselect/promql/exec_test.go index 0e687f8d4..9923160a2 100644 --- a/app/vmselect/promql/exec_test.go +++ b/app/vmselect/promql/exec_test.go @@ -9432,3 +9432,104 @@ func testAddLabels(t *testing.T, mn *storage.MetricName, labels ...string) { }) } } + +func TestIsSubQueryCompleteTrue(t *testing.T) { + f := func(q string) { + t.Helper() + e, err := metricsql.Parse(q) + if err != nil { + t.Fatal(err) + } + if !isSubQueryComplete(e, false) { + t.Fatalf("query should be complete: %s", e.AppendString(nil)) + } + } + + f("rate(http_total)") + f("sum(http_total)") + f("absent(http_total)") + f("rate(http_total[1m])") + f("avg_over_time(up[1m])") + f("sum(http_total[1m])") + f("sum(rate(http_total))") + f("sum(sum(http_total))") + f(`sum(sum_over_time(http_total[1m] )) by (instance)`) + f("sum(up{cluster='a'}[1m] or up{cluster='b'}[1m])") + f("(avg_over_time(alarm_test1[1m]) - avg_over_time(alarm_test1[1m] offset 5m)) > 0.1") + f("http_total[1m] offset 1m") + + // subquery + f("rate(http_total)[5m:1m]") + f("rate(sum(http_total)[5m:1m])") + f("rate(rate(http_total)[5m:1m])") + f("sum(rate(http_total[1m]))") + f("sum(rate(sum(http_total)[5m:1m]))") + f("rate(sum(rate(http_total))[5m:1m])") + f("rate(sum(sum(http_total))[5m:1m])") + f("rate(sum(rate(http_total))[5m:1m])") + f("rate(sum(sum(http_total))[5m:1m])") + f("avg_over_time(rate(http_total[5m])[5m:1m])") + f("delta(avg_over_time(up[1m])[5m:1m]) > 0.1") + f("avg_over_time(avg by (site) (metric)[2m:1m])") + + f("sum(http_total)[5m:1m] offset 1m") + f("round(sum(sum_over_time(http_total[1m])) by (instance)) [5m:1m] offset 1m") + + f("rate(sum(http_total)[5m:1m]) - rate(sum(http_total)[5m:1m])") + f("avg_over_time((rate(http_total)-rate(http_total))[5m:1m])") + + f("sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])") + f("sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])") + f("sum(sum_over_time((up{cluster='a'} or up{cluster='b'})[5m:1m])) by (instance)") + + // step (or resolution) is optional in subqueries + f("max_over_time(rate(my_counter_total[5m])[1h:])") + f("max_over_time(rate(my_counter_total[5m])[1h:1m])[5m:1m]") + f("max_over_time(rate(my_counter_total[5m])[1h:])[5m:]") + + f(` +WITH ( + cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}, + cpuIdle = rate(cpuSeconds{mode='idle'}[5m]) +) +max_over_time(cpuIdle[1h:])`) +} + +func TestIsSubQueryCompleteFalse(t *testing.T) { + f := func(q string) { + t.Helper() + e, err := metricsql.Parse(q) + if err != nil { + t.Fatal(err) + } + if isSubQueryComplete(e, false) { + t.Fatalf("expect to detect incomplete subquery: %s", e.AppendString(nil)) + } + } + + f("rate(sum(http_total))") + f("rate(rate(http_total))") + f("sum(rate(sum(http_total)))") + f("rate(sum(rate(http_total)))") + f("rate(sum(sum(http_total)))") + f("avg_over_time(rate(http_total[5m]))") + + // https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3974 + f("sum(http_total) offset 1m") + f(`round(sum(sum_over_time(http_total[1m])) by (instance)) offset 1m`) + + f("rate(sum(http_total)) - rate(sum(http_total))") + f("avg_over_time(rate(http_total)-rate(http_total))") + + // https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3996 + f("sum_over_time(up{cluster='a'} or up{cluster='b'})") + f("sum_over_time(up{cluster='a'}[1m] or up{cluster='b'}[1m])") + f("sum(sum_over_time(up{cluster='a'}[1m] or up{cluster='b'}[1m])) by (instance)") + + f(` +WITH ( + cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}, + cpuIdle = rate(cpuSeconds{mode='idle'}[5m]) +) +max_over_time(cpuIdle)`) +} diff --git a/app/vmselect/vmui/asset-manifest.json b/app/vmselect/vmui/asset-manifest.json index 6db50dda7..41b7ed7ae 100644 --- a/app/vmselect/vmui/asset-manifest.json +++ b/app/vmselect/vmui/asset-manifest.json @@ -1,13 +1,13 @@ { "files": { - "main.css": "./static/css/main.4ebf2874.css", - "main.js": "./static/js/main.202937c2.js", + "main.css": "./static/css/main.d0b400e3.css", + "main.js": "./static/js/main.59c17910.js", "static/js/685.bebe1265.chunk.js": "./static/js/685.bebe1265.chunk.js", - "static/media/MetricsQL.md": "./static/media/MetricsQL.da86c2db4f0b05e286b0.md", + "static/media/MetricsQL.md": "./static/media/MetricsQL.df7574389d8f8bbcf0c7.md", "index.html": "./index.html" }, "entrypoints": [ - "static/css/main.4ebf2874.css", - "static/js/main.202937c2.js" + "static/css/main.d0b400e3.css", + "static/js/main.59c17910.js" ] } \ No newline at end of file diff --git a/app/vmselect/vmui/index.html b/app/vmselect/vmui/index.html index 0697bf80b..7d9893af7 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/css/main.4ebf2874.css b/app/vmselect/vmui/static/css/main.4ebf2874.css deleted file mode 100644 index fa091d38e..000000000 --- a/app/vmselect/vmui/static/css/main.4ebf2874.css +++ /dev/null @@ -1 +0,0 @@ -.vm-tabs{gap:12px;height:100%;position:relative;-webkit-user-select:none;user-select:none}.vm-tabs,.vm-tabs-item{align-items:center;display:flex;justify-content:center}.vm-tabs-item{color:inherit;cursor:pointer;font-size:inherit;font-weight:inherit;opacity:.6;padding:12px 8px;text-decoration:none;text-transform:capitalize;transition:opacity .2s}.vm-tabs-item_active{opacity:1}.vm-tabs-item:hover{opacity:.8}.vm-tabs-item__icon{display:grid;margin-right:8px;width:16px}.vm-tabs-item__icon_single{margin-right:0}.vm-tabs__indicator{border-bottom:2px solid;position:absolute;transition:width .2s ease,left .3s cubic-bezier(.28,.84,.42,1)}.vm-alert{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-radius:8px;box-shadow:var(--box-shadow);color:var(--color-text);display:grid;font-size:14px;font-weight:400;gap:8px;grid-template-columns:20px 1fr;line-height:1.5;padding:12px;position:relative}.vm-alert_mobile{align-items:flex-start;border-radius:0}.vm-alert:after{border-radius:8px;content:"";height:100%;left:0;opacity:.1;position:absolute;top:0;width:100%;z-index:1}.vm-alert_mobile:after{border-radius:0}.vm-alert__content,.vm-alert__icon{position:relative;z-index:2}.vm-alert__icon{align-items:center;display:flex;justify-content:center}.vm-alert__content{filter:brightness(.6);white-space:pre-line}.vm-alert_success{color:var(--color-success)}.vm-alert_success:after{background-color:var(--color-success)}.vm-alert_error{color:var(--color-error)}.vm-alert_error:after{background-color:var(--color-error)}.vm-alert_info{color:var(--color-info)}.vm-alert_info:after{background-color:var(--color-info)}.vm-alert_warning{color:var(--color-warning)}.vm-alert_warning:after{background-color:var(--color-warning)}.vm-alert_dark:after{opacity:.1}.vm-alert_dark .vm-alert__content{filter:none}.vm-header{align-items:center;display:flex;flex-wrap:wrap;gap:0 16px;justify-content:flex-start;min-height:51px;padding:8px 12px;z-index:99}.vm-header_app{padding:8px 0}@media(max-width:1000px){.vm-header{gap:8px;padding:8px;position:-webkit-sticky;position:sticky;top:0}}.vm-header_sidebar{display:grid;grid-template-columns:40px auto 1fr}.vm-header_mobile{display:grid;grid-template-columns:33px 1fr 33px;justify-content:space-between}.vm-header_dark .vm-header-button,.vm-header_dark button,.vm-header_dark button:before{background-color:var(--color-background-block)}.vm-header-logo{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;margin-bottom:2px;overflow:hidden;position:relative;width:100%}.vm-header-logo,.vm-header-logo svg,.vm-header-logo_mobile{max-width:65px;min-width:65px}.vm-header-logo_mobile{margin:0 auto}.vm-header-logo_logs,.vm-header-logo_logs svg{max-width:75px;min-width:75px}.vm-header-nav{align-items:center;display:flex;gap:12px;justify-content:flex-start}.vm-header-nav_column{align-items:stretch;flex-direction:column;gap:8px}.vm-header-nav_column .vm-header-nav-item{padding:12px 0}.vm-header-nav_column .vm-header-nav-item_sub{justify-content:stretch}.vm-header-nav-item{cursor:pointer;font-size:14px;font-weight:400;opacity:1;padding:12px 8px;position:relative;text-transform:capitalize;transition:opacity .2s ease-in}.vm-header-nav-item_sub{grid-gap:4px;align-items:center;cursor:default;display:grid;gap:4px;grid-template-columns:auto 14px;justify-content:center}.vm-header-nav-item:hover{opacity:.7}.vm-header-nav-item_active{border-bottom:2px solid #110f0f33}.vm-header-nav-item svg{transform:rotate(0deg);transition:transform .2s ease-in}.vm-header-nav-item_open svg{transform:rotate(180deg)}.vm-header-nav-item-submenu{border-radius:4px;color:#fff;display:grid;opacity:1;padding:8px;transform-origin:top center;white-space:nowrap}.vm-header-nav-item-submenu-item{cursor:pointer}.vm-popper{background-color:var(--color-background-block);border-radius:4px;box-shadow:var(--box-shadow-popper);opacity:0;pointer-events:none;position:fixed;transition:opacity .1s ease-in-out;z-index:-99}.vm-popper_open{animation:vm-slider .15s cubic-bezier(.28,.84,.42,1.1);opacity:1;pointer-events:auto;transform-origin:top center;z-index:101}.vm-popper_mobile{animation:none;border-radius:0;bottom:0;left:0;overflow:auto;position:fixed;right:0;top:0;width:100%}.vm-popper-header{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-bottom:var(--border-divider);border-radius:4px 4px 0 0;color:var(--color-text);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px;min-height:51px;padding:8px 8px 8px 12px}.vm-popper-header__title{font-weight:700;-webkit-user-select:none;user-select:none}.vm-popper_dark{background-color:var(--color-background-tooltip);color:#fff}.vm-popper_dark .vm-popper-header{background-color:#0000;color:#fff}@keyframes vm-slider{0%{transform:scaleY(0)}to{transform:scaleY(1)}}.vm-modal{align-items:center;background:#110f0f8c;bottom:0;display:flex;justify-content:center;left:0;position:fixed;right:0;top:0;z-index:100}.vm-modal_mobile{align-items:flex-start;max-height:calc(var(--vh)*100);min-height:calc(var(--vh)*100);overflow:auto}.vm-modal_mobile .vm-modal-content{border-radius:0;grid-template-rows:70px -webkit-max-content;grid-template-rows:70px max-content;max-height:-webkit-max-content;max-height:max-content;min-height:100%;overflow:visible;width:100vw}.vm-modal_mobile .vm-modal-content-header{margin-bottom:12px;padding:8px 8px 8px 12px}.vm-modal_mobile .vm-modal-content-header__title{max-width:80vw}.vm-modal_mobile .vm-modal-content-body{align-items:flex-start;display:grid;min-height:100%;padding:0 12px 12px}.vm-modal-content{background:var(--color-background-block);border-radius:4px;box-shadow:0 0 24px #110f0f12;max-height:calc(var(--vh)*90);overflow:auto}.vm-modal-content-header{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-bottom:var(--border-divider);border-radius:4px 4px 0 0;color:var(--color-text);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px;min-height:51px;padding:12px;position:-webkit-sticky;position:sticky;top:0;z-index:3}.vm-modal-content-header__title{font-weight:700;max-width:50vw;overflow:hidden;text-overflow:ellipsis;-webkit-user-select:none;user-select:none;white-space:nowrap}.vm-modal-content-header__close{align-items:center;box-sizing:initial;color:#fff;cursor:pointer;display:flex;justify-content:center;padding:10px;width:24px}.vm-modal-content-body{padding:0 12px 12px}.vm-shortcuts{min-width:400px}@media(max-width:500px){.vm-shortcuts{min-width:100%}}.vm-shortcuts-section{border-bottom:var(--border-divider);margin-bottom:12px;padding-bottom:12px;position:relative}.vm-shortcuts-section__title{font-weight:700;margin-bottom:12px}.vm-shortcuts-section__read-more{position:absolute;right:0;top:-8px}.vm-shortcuts-section-list{grid-gap:12px;display:grid;gap:12px}@media(max-width:500px){.vm-shortcuts-section-list{gap:12px}}.vm-shortcuts-section-list-item{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:210px 1fr}@media(max-width:500px){.vm-shortcuts-section-list-item{grid-template-columns:1fr}}.vm-shortcuts-section-list-item__key{align-items:center;display:flex;gap:4px}.vm-shortcuts-section-list-item__key code,.vm-shortcuts-section-list-item__key svg{background-color:var(--color-background-body);background-repeat:repeat-x;border:var(--border-divider);border-radius:4px;color:var(--color-text);display:inline-block;font-size:12px;line-height:2;padding:2px 8px 0;text-align:center}.vm-shortcuts-section-list-item__key svg{color:var(--color-primary);padding:4px;width:24px}.vm-shortcuts-section-list-item__description{font-size:14px}.vm-tooltip{animation:vm-scale .15s cubic-bezier(.28,.84,.42,1);background-color:var(--color-background-tooltip);border-radius:4px;box-shadow:var(--box-shadow-popper);color:#fff;font-size:12px;line-height:150%;opacity:1;padding:3px 8px;pointer-events:auto;position:fixed;transition:opacity .1s ease-in-out;white-space:nowrap;z-index:101}@keyframes vm-scale{0%{transform:scale(0)}to{transform:scale(1)}}.fc-graph-tips{grid-gap:12px;display:grid;gap:12px;max-width:520px}.fc-graph-tips-item{grid-gap:8px;border-bottom:var(--border-divider);display:grid;gap:8px;line-height:1.3;padding-bottom:12px}.fc-graph-tips-item__action{color:var(--color-text-secondary);font-weight:700}.fc-graph-tips-item__description{display:inline-block;line-height:1.5}.fc-graph-tips-item__description code,.fc-graph-tips-item__description svg{align-items:center;background-color:var(--color-background-body);border:var(--border-divider);border-radius:4px;color:var(--color-text);display:inline-flex;font-size:12px;justify-content:center;margin:0 2px 2px;min-height:20px;min-width:20px;padding:0 4px}.fc-graph-tips-item svg{color:var(--color-primary);margin-top:-8px;padding:2px;transform:translateY(8px);width:18px}.vm-menu-burger{background:none;border:none;cursor:pointer;height:18px;outline:none;padding:0;position:relative;transform-style:preserve-3d;width:18px}.vm-menu-burger:after{background-color:#110f0f1a;border-radius:50%;content:"";height:calc(100% + 12px);left:-6px;position:absolute;top:-6px;transform:scale(0) translateZ(-2px);transition:transform .14s ease-in-out;width:calc(100% + 12px)}.vm-menu-burger:hover:after{transform:scale(1) translateZ(-2px)}.vm-menu-burger span{border-top:2px solid #fff;display:block;top:50%;transform:translateY(-50%);transition:transform .3s ease,border-color .3s ease}.vm-menu-burger span,.vm-menu-burger span:after,.vm-menu-burger span:before{border-radius:6px;height:2px;left:0;position:absolute;width:100%}.vm-menu-burger span:after,.vm-menu-burger span:before{animation-duration:.6s;animation-fill-mode:forwards;animation-timing-function:cubic-bezier(.645,.045,.355,1);background:#fff;content:"";top:0}.vm-menu-burger span:before{animation-name:topLineBurger}.vm-menu-burger span:after{animation-name:bottomLineBurger}.vm-menu-burger_opened span{border-color:#0000}.vm-menu-burger_opened span:before{animation-name:topLineCross}.vm-menu-burger_opened span:after{animation-name:bottomLineCross}@keyframes topLineCross{0%{transform:translateY(-7px)}50%{transform:translateY(0)}to{transform:translateY(-2px) translateX(30%) rotate(45deg);width:60%}}@keyframes bottomLineCross{0%{transform:translateY(3px)}50%{transform:translateY(0)}to{transform:translateY(-2px) translateX(30%) rotate(-45deg);width:60%}}@keyframes topLineBurger{0%{transform:translateY(0) rotate(45deg)}50%{transform:rotate(0deg)}to{transform:translateY(-7px) rotate(0deg)}}@keyframes bottomLineBurger{0%{transform:translateY(0) rotate(-45deg)}50%{transform:rotate(0deg)}to{transform:translateY(3px) rotate(0deg)}}.vm-header-sidebar{background-color:inherit;color:inherit;height:24px;width:24px}.vm-header-sidebar-button{align-items:center;display:flex;height:51px;justify-content:center;left:0;position:absolute;top:0;transition:left .35s cubic-bezier(.28,.84,.42,1);width:51px}.vm-header-sidebar-button_open{left:149px;position:fixed;z-index:102}.vm-header-sidebar-menu{grid-gap:12px;background-color:inherit;box-shadow:var(--box-shadow-popper);display:grid;gap:12px;grid-template-rows:1fr auto;height:100%;left:0;padding:12px;position:fixed;top:0;transform:translateX(-100%);transform-origin:left;transition:transform .3s cubic-bezier(.28,.84,.42,1);width:200px;z-index:101}.vm-header-sidebar-menu_open{transform:translateX(0)}.vm-header-sidebar-menu__logo{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;position:relative;width:65px}.vm-header-sidebar-menu-settings{grid-gap:8px;align-items:center;display:grid;gap:8px}.vm-header-controls{align-items:center;display:flex;flex-grow:1;gap:8px;justify-content:flex-end}.vm-header-controls_mobile{display:grid;grid-template-columns:1fr;padding:0}.vm-header-controls_mobile .vm-header-button{border:none}.vm-header-controls-modal{transform:scale(0)}.vm-header-controls-modal_open{transform:scale(1)}.vm-container{display:flex;flex-direction:column;min-height:calc(var(--vh)*100 - var(--scrollbar-height))}.vm-container-body{background-color:var(--color-background-body);flex-grow:1;min-height:100%;padding:12px}.vm-container-body_mobile{padding:8px 0 0}@media(max-width:768px){.vm-container-body{padding:8px 0 0}}.vm-container-body_app{background-color:#0000;padding:8px 0}.vm-footer{align-items:center;background:var(--color-background-body);border-top:var(--border-divider);color:var(--color-text-secondary);display:flex;flex-wrap:wrap;gap:12px;justify-content:center;padding:12px}@media(max-width:768px){.vm-footer{gap:12px;padding:12px}}.vm-footer__link,.vm-footer__website{grid-gap:6px;align-items:center;display:grid;gap:6px;grid-template-columns:12px auto;justify-content:center}.vm-footer__website{margin-right:12px}@media(max-width:768px){.vm-footer__website{margin-right:0}}.vm-footer__link{grid-template-columns:14px auto}.vm-footer__copyright{flex-grow:1;text-align:right}@media(max-width:768px){.vm-footer__copyright{font-size:12px;text-align:center;width:100%}}.vm-tenant-input{position:relative}.vm-tenant-input-list{border-radius:8px;max-height:300px;overflow:auto;overscroll-behavior:none}.vm-tenant-input-list_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-tenant-input-list_mobile .vm-tenant-input-list__search{padding:0 12px 8px}.vm-tenant-input-list__search{background-color:var(--color-background-block);padding:8px 12px;position:-webkit-sticky;position:sticky;top:0}.vm-text-field{display:grid;margin:6px 0;position:relative;width:100%}.vm-text-field_textarea:after{word-wrap:break-word;content:attr(data-replicated-value) " ";visibility:hidden;white-space:pre-wrap}.vm-text-field:after,.vm-text-field__input{background-color:#0000;border:var(--border-divider);box-sizing:border-box;font-family:monospace;font-size:14px;grid-area:1/1/2/2;line-height:18px;overflow:hidden;padding:8px 12px;width:100%}.vm-text-field__error,.vm-text-field__helper-text,.vm-text-field__label,.vm-text-field__warning{-webkit-line-clamp:1;line-clamp:1;-webkit-box-orient:vertical;background-color:var(--color-background-block);display:-webkit-box;font-size:12px;left:6px;line-height:14px;max-width:calc(100% - 12px);overflow:hidden;padding:0 3px;pointer-events:none;position:absolute;text-overflow:ellipsis;-webkit-user-select:none;user-select:none;z-index:2}.vm-text-field__label{color:var(--color-text-secondary);top:-8px}.vm-text-field__error,.vm-text-field__helper-text,.vm-text-field__warning{overflow-wrap:anywhere;pointer-events:auto;position:relative;top:-6px;-webkit-user-select:text;user-select:text;width:-webkit-fit-content;width:-moz-fit-content;width:fit-content}.vm-text-field__error_full,.vm-text-field__helper-text_full,.vm-text-field__warning_full{display:block;overflow:visible}.vm-text-field__error_overflowed,.vm-text-field__helper-text_overflowed,.vm-text-field__warning_overflowed{cursor:pointer}.vm-text-field__error{color:var(--color-error)}.vm-text-field__warning{color:var(--color-warning)}.vm-text-field__helper-text{color:var(--color-text-secondary)}.vm-text-field__input{background-color:#0000;border-radius:4px;color:var(--color-text);display:block;overflow:hidden;resize:none;transition:border .2s ease}.vm-text-field__input:focus,.vm-text-field__input:hover{border:1px solid var(--color-primary)}.vm-text-field__input_error,.vm-text-field__input_error:focus,.vm-text-field__input_error:hover{border-color:var(--color-error)}.vm-text-field__input_warning,.vm-text-field__input_warning:focus,.vm-text-field__input_warning:hover{border-color:var(--color-warning)}.vm-text-field__input_icon-start{padding-left:31px}.vm-text-field__input:disabled{background-color:inherit;color:inherit}.vm-text-field__input:disabled:hover{border-color:var(--color-text-disabled)}.vm-text-field__icon-end,.vm-text-field__icon-start{align-items:center;color:var(--color-text-secondary);display:flex;height:40px;justify-content:center;left:8px;max-width:15px;position:absolute;top:0}.vm-text-field__icon-end{left:auto;right:8px}.vm-text-field__controls-info{bottom:8px;color:var(--color-text-secondary);font-size:12px;opacity:.8;position:absolute;right:12px}.vm-step-control{display:inline-flex}.vm-step-control button{text-transform:none}.vm-step-control__value{display:inline;margin-left:3px}.vm-step-control-popper{grid-gap:8px;display:grid;font-size:14px;gap:8px;max-height:208px;max-width:300px;overflow:auto;padding:12px}.vm-step-control-popper_mobile{max-height:calc(var(--vh)*100 - 70px);max-width:100%;padding:0 12px 8px}.vm-step-control-popper_mobile .vm-step-control-popper-info{font-size:14px}.vm-step-control-popper-info{font-size:12px;line-height:1.8}.vm-step-control-popper-info a{margin:0 .4em}.vm-step-control-popper-info code{background-color:var(--color-hover-black);border-radius:6px;margin:0 .2em;padding:.2em .4em}.vm-time-duration{font-size:14px;max-height:227px;overflow:auto}.vm-time-duration_mobile{max-height:100%}.vm-time-selector{display:grid;grid-template-columns:repeat(2,230px);padding:12px 0}.vm-time-selector_mobile{grid-template-columns:1fr;max-height:calc(var(--vh)*100 - 70px);min-width:250px;overflow:auto;width:100%}.vm-time-selector_mobile .vm-time-selector-left{border-bottom:var(--border-divider);border-right:none;padding-bottom:12px}.vm-time-selector-left{border-right:var(--border-divider);display:flex;flex-direction:column;gap:8px;padding:0 12px}.vm-time-selector-left-inputs{align-items:flex-start;display:grid;flex-grow:1;justify-content:stretch}.vm-time-selector-left-timezone{align-items:center;display:flex;font-size:12px;gap:8px;justify-content:space-between;margin-bottom:8px}.vm-time-selector-left-timezone__utc{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;justify-content:center;padding:4px}.vm-time-selector-left__controls{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(2,1fr)}.vm-calendar{background-color:var(--color-background-block);border-radius:8px;display:grid;font-size:14px;grid-template-rows:auto 1fr auto;padding:12px;-webkit-user-select:none;user-select:none}.vm-calendar_mobile{padding:0 12px}.vm-calendar-header{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:center;min-height:36px;padding-bottom:12px}.vm-calendar-header-left{grid-gap:8px;align-items:center;cursor:pointer;display:grid;gap:8px;grid-template-columns:auto auto;justify-content:flex-start;transition:opacity .2s ease-in-out}.vm-calendar-header-left:hover{opacity:.8}.vm-calendar-header-left__date{color:var(--color-text);font-size:14px;font-weight:700}.vm-calendar-header-left__select-year{align-items:center;display:grid;height:14px;justify-content:center;width:14px}.vm-calendar-header-right{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:18px 18px;justify-content:center}.vm-calendar-header-right__next,.vm-calendar-header-right__prev{cursor:pointer;margin:-8px;padding:8px;transition:opacity .2s ease-in-out}.vm-calendar-header-right__next:hover,.vm-calendar-header-right__prev:hover{opacity:.8}.vm-calendar-header-right__prev{transform:rotate(90deg)}.vm-calendar-header-right__next{transform:rotate(-90deg)}.vm-calendar-body{grid-gap:2px;align-items:center;display:grid;gap:2px;grid-template-columns:repeat(7,32px);grid-template-rows:repeat(7,32px);justify-content:center}@media(max-width:500px){.vm-calendar-body{grid-template-columns:repeat(7,calc(14.28571vw - 5.14286px));grid-template-rows:repeat(7,calc(14.28571vw - 5.14286px))}}.vm-calendar-body-cell{align-items:center;border-radius:50%;display:flex;height:100%;justify-content:center;text-align:center}.vm-calendar-body-cell_weekday{color:var(--color-text-secondary)}.vm-calendar-body-cell_day{cursor:pointer;transition:color .2s ease,background-color .3s ease-in-out}.vm-calendar-body-cell_day:hover{background-color:var(--color-hover-black)}.vm-calendar-body-cell_day_empty{pointer-events:none}.vm-calendar-body-cell_day_active{color:#fff}.vm-calendar-body-cell_day_active,.vm-calendar-body-cell_day_active:hover{background-color:var(--color-primary)}.vm-calendar-body-cell_day_today{border:1px solid var(--color-primary)}.vm-calendar-years{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(3,1fr);max-height:400px;overflow:auto}.vm-calendar-years__year{align-items:center;border-radius:8px;cursor:pointer;display:flex;justify-content:center;padding:8px 16px;transition:color .2s ease,background-color .3s ease-in-out}.vm-calendar-years__year:hover{background-color:var(--color-hover-black)}.vm-calendar-years__year_selected{color:#fff}.vm-calendar-years__year_selected,.vm-calendar-years__year_selected:hover{background-color:var(--color-primary)}.vm-calendar-years__year_today{border:1px solid var(--color-primary)}.vm-calendar-footer{align-items:center;display:flex;justify-content:flex-end}.vm-date-time-input{grid-gap:8px 0;align-items:center;cursor:pointer;display:grid;gap:8px 0;grid-template-columns:1fr;justify-content:center;margin-bottom:12px;position:relative;transition:color .2s ease-in-out,border-bottom-color .3s ease}.vm-date-time-input:hover input{border-bottom-color:var(--color-primary)}.vm-date-time-input label{color:var(--color-text-secondary);font-size:12px;grid-column:1/3;-webkit-user-select:none;user-select:none;width:100%}.vm-date-time-input__icon{bottom:2px;position:absolute;right:0}.vm-date-time-input input{background:#0000;border:none;border-bottom:var(--border-divider);color:var(--color-text);padding:0 0 8px}.vm-date-time-input input:focus{border-bottom-color:var(--color-primary)}.vm-date-time-input_error input{border-color:var(--color-error)}.vm-date-time-input_error input:focus{border-bottom-color:var(--color-error)}.vm-date-time-input__error-text{bottom:-12px;color:var(--color-error);font-size:12px;left:0;position:absolute}.vm-button{align-items:center;border-radius:6px;color:#fff;cursor:pointer;display:flex;font-size:12px;font-weight:400;justify-content:center;line-height:15px;min-height:31px;padding:6px 14px;position:relative;text-transform:uppercase;transform-style:preserve-3d;-webkit-user-select:none;user-select:none;white-space:nowrap}.vm-button:hover:after{background-color:var(--color-hover-black)}.vm-button:after,.vm-button:before{border-radius:6px;content:"";height:100%;left:0;position:absolute;top:0;transition:background-color .2s ease;width:100%}.vm-button:before{transform:translateZ(-2px)}.vm-button:after{background-color:#0000;transform:translateZ(-1px)}.vm-button:active:after{transform:scale(.9)}.vm-button span{align-items:center;display:grid;justify-content:center;transform:translateZ(1px)}.vm-button span svg{width:15px}.vm-button__start-icon{margin-right:6px}.vm-button__end-icon{margin-left:6px}.vm-button_disabled{cursor:not-allowed;opacity:.3}.vm-button_icon{padding:6px 8px}.vm-button_icon .vm-button__end-icon,.vm-button_icon .vm-button__start-icon{margin:0}.vm-button_small{min-height:25px;padding:4px 8px}.vm-button_small span svg{width:13px}.vm-button_contained_primary{color:var(--color-primary-text)}.vm-button_contained_primary,.vm-button_contained_primary:before{background-color:var(--color-primary)}.vm-button_contained_primary:hover:after{background-color:#110f0f33}.vm-button_contained_secondary{color:var(--color-secondary-text)}.vm-button_contained_secondary:before{background-color:var(--color-secondary)}.vm-button_contained_secondary:hover:after{background-color:#110f0f33}.vm-button_contained_success{color:var(--color-success-text)}.vm-button_contained_success:before{background-color:var(--color-success)}.vm-button_contained_success:hover:after{background-color:#110f0f33}.vm-button_contained_error{color:var(--color-error-text)}.vm-button_contained_error:before{background-color:var(--color-error)}.vm-button_contained_gray{color:var(--color-text-secondary)}.vm-button_contained_gray:before{background-color:var(--color-text-secondary)}.vm-button_contained_warning{color:var(--color-warning)}.vm-button_contained_warning:before{background-color:var(--color-warning);opacity:.2}.vm-button_text_primary{color:var(--color-primary)}.vm-button_text_secondary{color:var(--color-secondary)}.vm-button_text_success{color:var(--color-success)}.vm-button_text_error{color:var(--color-error)}.vm-button_text_gray{color:var(--color-text-secondary)}.vm-button_text_white{color:#fff}.vm-button_text_warning{color:var(--color-warning)}.vm-button_outlined_primary{border:1px solid var(--color-primary);color:var(--color-primary)}.vm-button_outlined_error{border:1px solid var(--color-error);color:var(--color-error)}.vm-button_outlined_secondary{border:1px solid var(--color-secondary);color:var(--color-secondary)}.vm-button_outlined_success{border:1px solid var(--color-success);color:var(--color-success)}.vm-button_outlined_gray{border:1px solid var(--color-text-secondary);color:var(--color-text-secondary)}.vm-button_outlined_white{border:1px solid #fff;color:#fff}.vm-button_outlined_warning{border:1px solid var(--color-warning);color:var(--color-warning)}.vm-execution-controls-buttons{border-radius:7px;display:flex;justify-content:space-between;min-width:107px}.vm-execution-controls-buttons_mobile{flex-direction:column;gap:12px}.vm-execution-controls-buttons__arrow{align-items:center;display:flex;justify-content:center;transform:rotate(0);transition:transform .2s ease-in-out}.vm-execution-controls-buttons__arrow_open{transform:rotate(180deg)}.vm-execution-controls-list{font-size:14px;max-height:208px;overflow:auto;padding:8px 0;width:124px}.vm-execution-controls-list_mobile{max-height:calc(var(--vh)*100 - 70px);padding:0;width:100%}.vm-server-configurator{align-items:center;display:flex;flex-direction:column;gap:12px;padding-bottom:12px;width:600px}.vm-server-configurator_mobile{align-items:flex-start;grid-auto-rows:-webkit-min-content;grid-auto-rows:min-content;height:100%;width:100%}@media(max-width:768px){.vm-server-configurator{width:100%}}.vm-server-configurator__input{width:100%}.vm-server-configurator__input_flex{align-items:flex-start;display:flex;gap:12px}.vm-server-configurator__title{align-items:center;display:flex;font-size:14px;font-weight:700;grid-column:auto/span 2;justify-content:flex-start;margin-bottom:12px}.vm-server-configurator-url{align-items:flex-start;display:flex;gap:8px}.vm-server-configurator-url__button{margin-top:8px}.vm-server-configurator-footer{align-items:center;display:flex;gap:8px;justify-content:flex-end;width:100%}.vm-server-configurator_mobile .vm-server-configurator-footer{display:grid;grid-template-columns:1fr 1fr}.vm-limits-configurator-title__reset{align-items:center;display:flex;flex-grow:1;justify-content:flex-end}.vm-limits-configurator__inputs{grid-gap:12px;align-items:center;display:grid;flex-wrap:wrap;gap:12px;grid-template-columns:repeat(auto-fit,minmax(150px,1fr));justify-content:space-between}.vm-limits-configurator__inputs_mobile{gap:8px}.vm-accordion-header{align-items:center;cursor:pointer;display:grid;font-size:inherit;position:relative}.vm-accordion-header__arrow{align-items:center;display:flex;justify-content:center;position:absolute;right:14px;top:auto;transform:rotate(0);transition:transform .2s ease-in-out}.vm-accordion-header__arrow_open{transform:rotate(180deg)}.vm-accordion-header__arrow svg{height:auto;width:14px}.accordion-section{overflow:hidden}.vm-timezones-item{align-items:center;cursor:pointer;display:flex;gap:8px;justify-content:space-between}.vm-timezones-item_selected{border:var(--border-divider);border-radius:4px;padding:8px 12px}.vm-timezones-item__title{align-items:center;display:flex;gap:8px;text-transform:capitalize}.vm-timezones-item__title svg{color:var(--color-warning);width:14px}.vm-timezones-item__utc{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;justify-content:center;padding:4px}.vm-timezones-item__icon{align-items:center;display:inline-flex;justify-content:flex-end;margin:0 0 0 auto;transition:transform .2s ease-in}.vm-timezones-item__icon svg{width:14px}.vm-timezones-item__icon_open{transform:rotate(180deg)}.vm-timezones-list{background-color:var(--color-background-block);border-radius:8px;max-height:300px;overflow:auto}.vm-timezones-list_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-timezones-list_mobile .vm-timezones-list-header__search{padding:0 12px}.vm-timezones-list-header{background-color:var(--color-background-block);border-bottom:var(--border-divider);position:-webkit-sticky;position:sticky;top:0;z-index:2}.vm-timezones-list-header__search{padding:8px}.vm-timezones-list-group{border-bottom:var(--border-divider);padding:8px 0}.vm-timezones-list-group:last-child{border-bottom:none}.vm-timezones-list-group__title{color:var(--color-text-secondary);font-weight:700;padding:8px 12px}.vm-timezones-list-group-options{align-items:flex-start;display:grid}.vm-timezones-list-group-options__item{padding:8px 12px;transition:background-color .2s ease}.vm-timezones-list-group-options__item:hover{background-color:#110f0f1a}.vm-theme-control__toggle{display:inline-flex;min-width:300px;text-transform:capitalize}.vm-theme-control_mobile .vm-theme-control__toggle{display:flex;min-width:100%}.vm-toggles{grid-gap:3px;display:grid;gap:3px;position:relative;width:100%}.vm-toggles__label{color:var(--color-text-secondary);font-size:12px;line-height:1;padding:0 12px}.vm-toggles-group{overflow:hidden;width:100%}.vm-toggles-group,.vm-toggles-group-item{align-items:center;display:grid;justify-content:center;position:relative}.vm-toggles-group-item{border-bottom:var(--border-divider);border-right:var(--border-divider);border-top:var(--border-divider);color:var(--color-text-secondary);cursor:pointer;font-size:12px;font-weight:700;padding:8px;text-align:center;transition:color .15s ease-in;-webkit-user-select:none;user-select:none;z-index:2}.vm-toggles-group-item_first{border-left:var(--border-divider);border-radius:16px 0 0 16px}.vm-toggles-group-item:last-child{border-left:none;border-radius:0 16px 16px 0}.vm-toggles-group-item_icon{gap:4px;grid-template-columns:14px auto}.vm-toggles-group-item:hover{color:var(--color-primary)}.vm-toggles-group-item_active{border-color:#0000;color:var(--color-primary)}.vm-toggles-group-item_active:hover{background-color:#0000}.vm-toggles-group__highlight{background-color:rgba(var(--color-primary),.08);border:1px solid var(--color-primary);height:100%;position:absolute;top:0;transition:left .2s cubic-bezier(.28,.84,.42,1),border-radius .2s linear;z-index:1}.vm-autocomplete{max-height:300px;overflow:auto;overscroll-behavior:none;position:relative}.vm-autocomplete_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-autocomplete__no-options{color:var(--color-text-disabled);padding:12px;text-align:center}.vm-autocomplete__loader{grid-gap:8px;align-items:center;color:var(--color-text-secondary);display:grid;gap:8px;grid-template-columns:14px auto;justify-content:center;padding:12px;pointer-events:none;z-index:2}.vm-autocomplete__loader svg{animation:half-circle-spinner-animation 1s linear infinite,vm-fade .5s ease-in}.vm-autocomplete-info,.vm-autocomplete-message{background-color:var(--color-background-block);border-top:var(--border-divider);padding:12px}.vm-autocomplete-message{color:var(--color-warning);font-size:12px;position:relative}.vm-autocomplete-message:after{background:var(--color-warning);content:"";height:100%;left:0;opacity:.1;position:absolute;top:0;width:100%}.vm-autocomplete-info{max-width:500px;min-width:450px;overflow-wrap:anywhere}.vm-autocomplete-info__type{color:var(--color-text-secondary);margin-bottom:8px}.vm-autocomplete-info__description{line-height:130%}.vm-autocomplete-info__description p{margin:12px 0}.vm-autocomplete-info__description p:last-child{margin:0}.vm-query-editor{position:relative}.vm-additional-settings{align-items:center;display:inline-flex;flex-wrap:wrap;gap:12px;justify-content:flex-start}.vm-additional-settings__input{flex-basis:160px;margin-bottom:-6px}.vm-additional-settings_mobile{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:1fr;padding:0 12px;width:100%}.vm-switch{align-items:center;cursor:pointer;display:flex;font-size:12px;justify-content:flex-start;-webkit-user-select:none;user-select:none}.vm-switch_full-width{flex-direction:row-reverse;justify-content:space-between}.vm-switch_full-width .vm-switch__label{margin-left:0}.vm-switch_disabled{cursor:default;opacity:.6}.vm-switch_secondary_active .vm-switch-track{background-color:var(--color-secondary)}.vm-switch_primary_active .vm-switch-track{background-color:var(--color-primary)}.vm-switch_active .vm-switch-track__thumb{left:20px}.vm-switch:hover .vm-switch-track{opacity:.8}.vm-switch-track{align-items:center;background-color:#110f0f66;border-radius:17px;display:flex;height:17px;justify-content:flex-start;padding:3px;position:relative;transition:background-color .2s ease,opacity .3s ease-out;width:34px}.vm-switch-track__thumb{background-color:var(--color-background-block);border-radius:50%;left:3px;min-height:11px;min-width:11px;position:absolute;top:auto;transform-style:preserve-3d;transition:right .2s ease-out,left .2s ease-out}.vm-switch__label{color:var(--color-text-secondary);font-size:inherit;margin-left:8px;transition:color .2s ease;white-space:nowrap}.vm-switch_active .vm-switch__label{color:var(--color-text)}.vm-query-configurator{grid-gap:12px;display:grid;gap:12px}.vm-query-configurator-list{display:grid}.vm-query-configurator-list-row{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr auto auto auto}.vm-query-configurator-list-row_mobile{gap:4px}.vm-query-configurator-list-row_disabled{filter:grayscale(100%);opacity:.5}.vm-query-configurator-list-row__button{align-items:start;display:grid;min-height:36px;width:36px}.vm-query-configurator-settings{align-items:center;display:flex;flex-wrap:wrap;gap:12px;justify-content:space-between}.vm-query-configurator-settings__buttons{grid-gap:8px;display:grid;flex-grow:1;gap:8px;grid-template-columns:repeat(3,auto);justify-content:flex-end}.vm-query-history{max-width:80vw;min-width:500px}.vm-query-history_mobile{max-width:100vw;min-width:100vw}.vm-query-history__tabs{border-bottom:var(--border-divider);margin:-12px -12px 0;padding:0 8px}.vm-query-history__tabs_mobile{margin:-12px -12px 0}.vm-query-history-list{align-items:flex-start;display:grid}.vm-query-history-list__group-title{font-weight:700;margin:0 -12px;padding:12px 12px 8px}.vm-query-history-list__group-title_first{padding-top:12px}.vm-query-history-list__no-data{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;line-height:18px;padding:16px 12px;text-align:center;white-space:pre-line}.vm-query-history-item{grid-gap:8px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:8px;grid-template-columns:1fr auto;margin:0 -12px;padding:8px 12px}.vm-query-history-item__value{font-family:monospace;overflow-wrap:anywhere;white-space:pre-wrap}.vm-query-history-item__buttons{display:flex}.vm-query-history-footer{display:flex;justify-content:flex-end;padding-top:12px}.vm-spinner{align-items:center;animation:vm-fade 2s cubic-bezier(.28,.84,.42,1.1);background-color:#ffffff80;bottom:0;display:flex;flex-direction:column;justify-content:center;left:0;pointer-events:none;position:fixed;right:0;top:0;z-index:99}.vm-spinner_dark{background-color:#110f0f33}.vm-spinner__message{color:rgba(var(--color-text),.9);font-size:16px;line-height:1.3;margin-top:12px;text-align:center;white-space:pre-line}.half-circle-spinner,.half-circle-spinner *{box-sizing:border-box}.half-circle-spinner{border-radius:100%;height:60px;position:relative;width:60px}.half-circle-spinner .circle{border:6px solid #0000;border-radius:100%;content:"";height:100%;position:absolute;width:100%}.half-circle-spinner .circle.circle-1{animation:half-circle-spinner-animation 1s infinite;border-top-color:var(--color-primary)}.half-circle-spinner .circle.circle-2{animation:half-circle-spinner-animation 1s infinite alternate;border-bottom-color:var(--color-primary)}@keyframes half-circle-spinner-animation{0%{transform:rotate(0deg)}to{transform:rotate(1turn)}}@keyframes vm-fade{0%{opacity:0}to{opacity:1}}.vm-custom-panel{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:100%;height:100%}.vm-custom-panel_mobile{gap:8px}.vm-custom-panel__warning{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between}.vm-custom-panel__warning_mobile{grid-template-columns:1fr}.vm-custom-panel-body{position:relative}.vm-custom-panel-body-header{align-items:center;border-bottom:var(--border-divider);display:flex;font-size:12px;justify-content:space-between;margin:-12px -12px 12px;padding:0 12px;position:relative;z-index:1}.vm-custom-panel-body-header__tabs{display:flex;flex-grow:1;justify-content:flex-start}.vm-custom-panel-body-header__graph-controls{align-items:center;display:flex;gap:8px;margin:5px 10px}.vm-custom-panel-body_mobile .vm-custom-panel-body-header{margin:-12px -12px 12px;padding:0 12px}.vm-tracings-view{grid-gap:12px;display:grid;gap:12px}.vm-tracings-view-trace-header{align-items:center;border-bottom:var(--border-divider);display:flex;justify-content:space-between;padding:8px 8px 8px 12px}.vm-tracings-view-trace-header-title{flex-grow:1;font-size:16px;margin-right:8px}.vm-tracings-view-trace-header-title__query{font-weight:700}.vm-tracings-view-trace-header__expand-icon{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;transform:rotate(-90deg);transition:transform .2s ease-in-out;width:20px}.vm-tracings-view-trace-header__expand-icon_open{transform:rotate(0)}.vm-tracings-view-trace__nav{padding:12px 12px 12px 0}.vm-tracings-view-trace__nav_mobile{padding:8px 8px 8px 0}.vm-line-progress{grid-gap:8px;align-items:center;color:var(--color-text-secondary);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:center}.vm-line-progress-track{background-color:var(--color-hover-black);border-radius:4px;height:20px;width:100%}.vm-line-progress-track__thumb{background-color:#1a90ff;border-radius:4px;height:100%}.vm-nested-nav{border-radius:4px;margin-left:16px;position:relative}.vm-nested-nav_dark .vm-nested-nav-header,.vm-nested-nav_dark .vm-nested-nav-header:after,.vm-nested-nav_dark .vm-nested-nav-header:before{background-color:var(--color-background-body)}.vm-nested-nav_dark .vm-nested-nav-header:hover{box-shadow:0 0 0 1px #ffffff14}.vm-nested-nav_mobile{margin-left:8px}.vm-nested-nav_root>.vm-nested-nav-header:after,.vm-nested-nav_root>.vm-nested-nav-header:before{display:none}.vm-nested-nav-header{grid-gap:8px;background-color:#c9e3f666;border-radius:4px;cursor:pointer;display:grid;gap:8px;grid-template-columns:auto 1fr;margin-bottom:8px;padding:8px;position:relative;transition:box-shadow .2s ease-in-out;z-index:2}.vm-nested-nav-header:after{height:2px;top:calc(50% - 1px);width:12px}.vm-nested-nav-header:after,.vm-nested-nav-header:before{background-color:#c9e3f6;content:"";left:-12px;position:absolute}.vm-nested-nav-header:before{bottom:50%;height:calc(50% + 8px);width:2px}.vm-nested-nav-header:hover{box-shadow:0 0 0 1px #110f0f14}.vm-nested-nav-header__icon{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;transform:rotate(-90deg);transition:transform .2s ease-in-out;width:20px}.vm-nested-nav-header__icon_open{transform:rotate(0)}.vm-nested-nav-header__progress{grid-column:2}.vm-nested-nav-header__message{-webkit-line-clamp:3;-webkit-box-orient:vertical;line-clamp:3;display:-moz-box;display:-webkit-box;grid-column:2;line-height:130%;overflow:hidden;position:relative;text-overflow:ellipsis}.vm-nested-nav-header__message_show-full{display:block;overflow:visible}.vm-nested-nav-header__message_duration{color:var(--color-text-secondary)}.vm-nested-nav-header-bottom{align-items:center;display:grid;grid-column:2;grid-template-columns:1fr auto}.vm-nested-nav__childrens>.vm-nested-nav:not(:last-child):before{background-color:#c9e3f6;content:"";height:calc(100% + 32px);left:-12px;position:absolute;top:0;width:2px}.vm-nested-nav__childrens>.vm-nested-nav_dark:not(:last-child):before{background-color:var(--color-background-body)}.vm-nested-nav__childrens>.vm-nested-nav:last-child{margin-bottom:32px}.uplot,.uplot *,.uplot :after,.uplot :before{box-sizing:border-box}.uplot{font-family: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;line-height:1.5;width:-webkit-min-content;width:min-content}.u-title{font-size:18px;font-weight:700;text-align:center}.u-wrap{position:relative;-webkit-user-select:none;user-select:none}.u-over,.u-under{position:absolute}.u-under{overflow:hidden}.uplot canvas{display:block;height:100%;position:relative;width:100%}.u-axis{position:absolute}.u-legend{font-size:14px;margin:auto;text-align:center}.u-inline{display:block}.u-inline *{display:inline-block}.u-inline tr{margin-right:16px}.u-legend th{font-weight:600}.u-legend th>*{display:inline-block;vertical-align:middle}.u-legend .u-marker{background-clip:padding-box!important;height:1em;margin-right:4px;width:1em}.u-inline.u-live th:after{content:":";vertical-align:middle}.u-inline:not(.u-live) .u-value{display:none}.u-series>*{padding:4px}.u-series th{cursor:pointer}.u-legend .u-off>*{opacity:.3}.u-select{background:#00000012}.u-cursor-x,.u-cursor-y,.u-select{pointer-events:none;position:absolute}.u-cursor-x,.u-cursor-y{left:0;top:0;will-change:transform}.u-hz .u-cursor-x,.u-vt .u-cursor-y{border-right:1px dashed #607d8b;height:100%}.u-hz .u-cursor-y,.u-vt .u-cursor-x{border-bottom:1px dashed #607d8b;width:100%}.u-cursor-pt{background-clip:padding-box!important;border:0 solid;border-radius:50%;left:0;pointer-events:none;position:absolute;top:0;will-change:transform}.u-axis.u-off,.u-cursor-pt.u-off,.u-cursor-x.u-off,.u-cursor-y.u-off,.u-select.u-off{display:none}.vm-line-chart{pointer-events:auto}.vm-line-chart_panning{pointer-events:none}.vm-line-chart__u-plot{position:relative}.vm-chart-tooltip{grid-gap:12px;word-wrap:break-word;background:var(--color-background-tooltip);border-radius:8px;color:#fff;display:grid;font-family:monospace;font-size:12px;font-weight:400;gap:12px;line-height:150%;padding:12px;pointer-events:none;position:absolute;-webkit-user-select:text;user-select:text;width:370px;z-index:98}.vm-chart-tooltip_sticky{pointer-events:auto;z-index:99}.vm-chart-tooltip_moved{margin-left:-316.5px;margin-top:-24.5px;position:fixed}.vm-chart-tooltip-header{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr 25px 25px;justify-content:center;min-height:25px}.vm-chart-tooltip-header__title{grid-row:1}.vm-chart-tooltip-header__close{color:#fff;grid-column:3;grid-row:1}.vm-chart-tooltip-header__drag{color:#fff;cursor:move;grid-column:2;grid-row:1}.vm-chart-tooltip-header__date{grid-gap:2px;display:grid;gap:2px;grid-column:1}.vm-chart-tooltip-data{align-items:center;display:flex;gap:8px;justify-content:flex-start}.vm-chart-tooltip-data__marker{border:1px solid #ffffff80;height:14px;width:14px}.vm-chart-tooltip-data__value{font-size:14px;line-height:1}.vm-chart-tooltip-stats{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-start}.vm-chart-tooltip-stats-row{align-items:center;display:grid;justify-content:flex-start}.vm-chart-tooltip-stats-row:not(:last-child){padding-right:8px}.vm-chart-tooltip-stats-row__key{line-height:1;margin-right:4px}.vm-chart-tooltip-stats-row__value{font-weight:700}.vm-chart-tooltip__info{white-space:pre-wrap;word-break:break-all}.vm-legend-item{grid-gap:8px;align-items:start;background-color:var(--color-background-block);cursor:pointer;display:grid;font-size:12px;grid-template-columns:auto auto;justify-content:start;padding:8px;transition:.2s ease}.vm-legend-item:hover{background-color:#0000001a}.vm-legend-item_hide{opacity:.5;text-decoration:line-through}.vm-legend-item_static{cursor:default;grid-template-columns:1fr;margin:0;padding:0}.vm-legend-item_static:hover{background-color:var(--color-background-block)}.vm-legend-item__marker{border-radius:2px;box-sizing:border-box;height:14px;position:relative;transition:.2s ease;width:14px}.vm-legend-item-info{font-weight:400;word-break:break-all}.vm-legend-item-info__label{margin-right:2px}.vm-legend-item-info__free-fields{cursor:pointer;padding:2px}.vm-legend-item-info__free-fields:hover{text-decoration:underline}.vm-legend-item-stats{align-items:center;display:flex;gap:8px;grid-column:2}.vm-legend-item-stats-row{align-items:center;display:flex;justify-content:flex-start}.vm-legend-item-stats-row:not(:last-child){padding-right:12px}.vm-legend-item-stats-row__key{color:var(--color-text-secondary);line-height:1;margin-right:4px}.vm-legend{cursor:default;display:flex;flex-wrap:wrap;position:relative}.vm-legend-group{margin:0 12px 12px 0;min-width:23%;width:100%}.vm-legend-group-title{align-items:center;border-bottom:var(--border-divider);display:flex;margin-bottom:1px;padding:8px}.vm-legend-group-title__count{font-weight:700;margin-right:8px}.vm-legend-heatmap{align-items:center;display:inline-grid;gap:4px;grid-template-columns:auto auto;justify-content:space-between}.vm-legend-heatmap__wrapper{align-items:flex-start;display:flex;flex-wrap:wrap;gap:12px;justify-content:space-between}.vm-legend-heatmap__value{color:var(--color-text);font-size:12px}.vm-legend-heatmap__value:last-child{text-align:right}.vm-legend-heatmap-gradient{align-items:center;display:flex;grid-column:1/-1;height:12px;justify-content:center;position:relative;width:200px}.vm-legend-heatmap-gradient__value{align-items:center;border:2px solid var(--color-text);border-radius:50%;display:flex;height:16px;justify-content:center;position:absolute;top:-2px;transform:translateX(-8px);transition:left .1s ease;width:16px}.vm-legend-heatmap-gradient__value span{background-color:var(--color-background-block);box-shadow:var(--box-shadow);color:var(--color-text);font-size:12px;left:auto;padding:4px 8px;position:absolute;top:18px}.vm-legend-heatmap__labels{word-break:break-all}.vm-graph-view{width:100%}.vm-graph-view_full-width{width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-graph-view_full-width{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-graph-view_full-width_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-legend-anomaly{cursor:default;flex-wrap:wrap;gap:32px;position:relative}.vm-legend-anomaly,.vm-legend-anomaly-item{align-items:center;display:flex;justify-content:center}.vm-legend-anomaly-item{gap:8px}.vm-legend-anomaly-item svg{height:14px;width:30px}.vm-axes-limits{grid-gap:12px;align-items:center;display:grid;gap:12px;max-width:300px}.vm-axes-limits_mobile{gap:12px;max-width:100%;width:100%}.vm-axes-limits_mobile .vm-axes-limits-list__inputs{grid-template-columns:repeat(2,1fr)}.vm-axes-limits-list{grid-gap:12px;align-items:center;display:grid;gap:12px}.vm-axes-limits-list__inputs{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(2,120px)}.vm-graph-settings-popper{grid-gap:12px;display:grid;gap:12px;padding:0 0 12px}.vm-graph-settings-popper__body{grid-gap:16px;display:grid;gap:16px;padding:0 12px}.vm-json-view__copy{display:flex;justify-content:flex-end;position:-webkit-sticky;position:sticky;top:0;z-index:2}.vm-json-view__code{font-size:14px;line-height:1.4;transform:translateY(-32px);white-space:pre-wrap}.vm-table-view{max-width:100%;overflow:auto}.vm-table-view,.vm-table-view_mobile{margin-top:-12px}.vm-table-view table{margin-top:0}.vm-table-settings-popper{display:grid;min-width:250px}.vm-table-settings-popper_mobile .vm-table-settings-popper-list{gap:12px}.vm-table-settings-popper_mobile .vm-table-settings-popper-list:first-child{padding-top:0}.vm-table-settings-popper-list{grid-gap:12px;border-bottom:var(--border-divider);display:grid;gap:12px;max-height:350px;overflow:auto;padding:12px}.vm-table-settings-popper-list_first{padding-top:0}.vm-table-settings-popper-list-header{align-items:center;display:grid;grid-template-columns:1fr auto;justify-content:space-between;min-height:25px}.vm-table-settings-popper-list-header__title{font-weight:700}.vm-table-settings-popper-list__item{font-size:14px}.vm-checkbox{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;-webkit-user-select:none;user-select:none}.vm-checkbox_disabled{cursor:default;opacity:.6}.vm-checkbox_secondary_active .vm-checkbox-track{background-color:var(--color-secondary)}.vm-checkbox_secondary .vm-checkbox-track{border:1px solid var(--color-secondary)}.vm-checkbox_primary_active .vm-checkbox-track{background-color:var(--color-primary)}.vm-checkbox_primary .vm-checkbox-track{border:1px solid var(--color-primary)}.vm-checkbox_active .vm-checkbox-track__thumb{transform:scale(1)}.vm-checkbox:hover .vm-checkbox-track{opacity:.8}.vm-checkbox-track{align-items:center;background-color:#0000;border-radius:4px;display:flex;height:16px;justify-content:center;padding:2px;position:relative;transition:background-color .2s ease,opacity .3s ease-out;width:16px}.vm-checkbox-track__thumb{align-items:center;color:#fff;display:grid;height:12px;justify-content:center;transform:scale(0);transition:transform .1s ease-in-out;width:12px}.vm-checkbox__label{color:inherit;font-size:inherit;margin-left:8px;transition:color .2s ease;white-space:nowrap}.vm-download-report{grid-gap:16px;display:grid;gap:16px;min-width:400px;padding-top:4px}.vm-download-report-settings{grid-gap:12px;display:grid;gap:12px}.vm-download-report-settings textarea{min-height:100px}.vm-download-report__buttons{align-items:center;display:flex;gap:12px;justify-content:flex-end}.vm-download-report-helper{grid-gap:8px;display:grid;gap:8px;padding:12px}.vm-download-report-helper__description{line-height:1.3;max-width:400px;white-space:pre-line}.vm-download-report-helper__description p{margin-bottom:4px}.vm-download-report-helper__buttons{align-items:center;display:flex;gap:8px;justify-content:flex-end}.vm-predefined-panel-header{grid-gap:8px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:8px;grid-template-columns:auto 1fr auto;justify-content:flex-start;padding:8px 16px}.vm-predefined-panel-header__description{line-height:1.3;white-space:pre-wrap}.vm-predefined-panel-header__description ol,.vm-predefined-panel-header__description ul{list-style-position:inside}.vm-predefined-panel-header__description a{color:#c9e3f6;text-decoration:underline}.vm-predefined-panel-header__info{align-items:center;color:var(--color-primary);display:flex;justify-content:center;width:18px}.vm-predefined-panel-body{min-height:500px;padding:8px 16px}@media(max-width:500px){.vm-predefined-panel-body{padding:0}}.vm-predefined-dashboard{background-color:#0000}.vm-predefined-dashboard-header{align-items:center;border-radius:4px;box-shadow:var(--box-shadow);display:grid;font-weight:700;grid-template-columns:1fr auto;justify-content:space-between;line-height:1;overflow:hidden;padding:12px;position:relative;transform-style:preserve-3d;transition:box-shadow .2s ease-in-out}.vm-predefined-dashboard-header_open{border-radius:4px 4px 0 0;box-shadow:none}.vm-predefined-dashboard-header__title{font-size:14px}.vm-predefined-dashboard-header__count{font-size:12px;grid-column:2;margin-right:26px}.vm-predefined-dashboard-panels{grid-gap:12px;display:grid;gap:12px;grid-template-columns:repeat(12,1fr);padding:0}@media(max-width:1000px){.vm-predefined-dashboard-panels{grid-template-columns:1fr}}.vm-predefined-dashboard-panels-panel{border-radius:8px;overflow:hidden;position:relative}.vm-predefined-dashboard-panels-panel:hover .vm-predefined-dashboard-panels-panel__resizer{transform:scale(1)}.vm-predefined-dashboard-panels-panel__resizer{bottom:0;cursor:ew-resize;height:20px;position:absolute;right:0;transform:scale(0);transition:transform .2s ease-in-out;width:20px;z-index:1}.vm-predefined-dashboard-panels-panel__resizer:after{border-bottom:2px solid #110f0f33;border-right:2px solid #110f0f33;bottom:5px;content:"";height:5px;position:absolute;right:5px;width:5px}.vm-predefined-dashboard-panels-panel__alert{grid-column:span 12}.vm-predefined-panels{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:768px){.vm-predefined-panels{padding:12px 0}}@media(max-width:500px){.vm-predefined-panels{padding:8px 0}}.vm-predefined-panels-tabs{align-items:center;display:flex;flex-wrap:wrap;font-size:12px;gap:8px;justify-content:flex-start;overflow:hidden}@media(max-width:768px){.vm-predefined-panels-tabs{padding:0 12px}}.vm-predefined-panels-tabs__tab{background:var(--color-background-block);border:1px solid #110f0f33;border-radius:8px;color:var(--color-text-secondary);cursor:pointer;padding:8px 12px;text-align:center;text-transform:uppercase;transition:background .2s ease-in-out,color .15s ease-in}@media(max-width:500px){.vm-predefined-panels-tabs__tab{flex-grow:1}}.vm-predefined-panels-tabs__tab:hover{color:var(--color-primary)}.vm-predefined-panels-tabs__tab_active{border-color:var(--color-primary);color:var(--color-primary)}.vm-predefined-panels__dashboards{grid-gap:12px;display:grid;gap:12px}.vm-cardinality-configurator{grid-gap:8px;display:grid;gap:8px}.vm-cardinality-configurator-controls{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-start}.vm-cardinality-configurator-controls__query{flex-grow:10}.vm-cardinality-configurator-controls__item{flex-grow:2}.vm-cardinality-configurator-controls__item_limit{flex-grow:1}.vm-cardinality-configurator-controls__item svg{color:var(--color-text-disabled)}.vm-cardinality-configurator-bottom{align-items:center;display:flex;flex-wrap:wrap;gap:12px;justify-content:flex-end;width:100%}.vm-cardinality-configurator-bottom-helpful{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-end}.vm-cardinality-configurator-bottom-helpful a{color:var(--color-text-secondary)}.vm-cardinality-configurator-bottom__execute{align-items:center;display:flex;gap:8px}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom{justify-content:center}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom-helpful{flex-grow:1;justify-content:center}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom__execute,.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom__execute button:nth-child(3){width:100%}.vm-cardinality-totals{align-content:flex-start;display:inline-flex;flex-grow:1;flex-wrap:wrap;gap:12px;justify-content:flex-start}.vm-cardinality-totals_mobile{gap:12px;justify-content:center}.vm-cardinality-totals-card{grid-gap:8px 4px;align-items:center;display:grid;gap:8px 4px;grid-template-columns:auto 1fr;justify-content:center}.vm-cardinality-totals-card__info-icon{align-items:center;color:var(--color-primary);display:flex;justify-content:center;width:12px}.vm-cardinality-totals-card__title{align-items:center;color:var(--color-text);display:flex;gap:4px;grid-column:1/-1;justify-content:flex-start}.vm-cardinality-totals-card__tooltip{max-width:280px;padding:8px;white-space:normal}.vm-cardinality-totals-card__value{color:var(--color-primary);font-size:18px;font-weight:700;line-height:14px;text-align:center}.vm-metrics-content-header{margin:-12px -12px 0}.vm-metrics-content-header__title{align-items:center;display:flex;justify-content:flex-start}.vm-metrics-content-header__tip{max-width:300px;padding:8px;white-space:normal}.vm-metrics-content-header__tip p{margin-bottom:8px}.vm-metrics-content-header__tip-icon{align-items:center;color:var(--color-primary);display:flex;justify-content:center;margin-right:4px;width:12px}.vm-metrics-content_mobile .vm-metrics-content-header{margin:-12px -12px 0}.vm-metrics-content__table{overflow:auto;padding-top:12px;width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-metrics-content__table{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-metrics-content__table_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-metrics-content__table .vm-table-cell_header{white-space:nowrap}.vm-metrics-content_mobile .vm-metrics-content__table{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-metrics-content__chart{padding-top:12px}.vm-metrics-content-prom-data{align-items:center;display:flex;flex-direction:column;gap:8px;justify-content:center;margin-top:12px;text-align:center;width:100%}.vm-metrics-content-prom-data__icon{align-items:center;color:var(--color-primary);display:flex;height:30px;justify-content:center;margin-bottom:8px;width:30px}.vm-metrics-content-prom-data__title{font-size:16px;font-weight:700}.vm-metrics-content-prom-data__text{line-height:1.3;max-width:700px}.vm-simple-bar-chart{display:grid;grid-template-columns:auto 1fr;height:100%;overflow:hidden;padding-bottom:6px}.vm-simple-bar-chart-y-axis{display:grid;position:relative;transform:translateY(12px)}.vm-simple-bar-chart-y-axis__tick{align-items:center;display:flex;font-size:12px;justify-content:flex-end;line-height:2;padding-right:8px;position:relative;text-align:right;transform-style:preserve-3d;z-index:1}.vm-simple-bar-chart-y-axis__tick:after{border-bottom:var(--border-divider);content:"";height:0;left:100%;position:absolute;top:auto;transform:translateY(-1px) translateZ(-1);width:100vw}.vm-simple-bar-chart-data{align-items:flex-end;display:flex;gap:1%;justify-content:space-between;position:relative}.vm-simple-bar-chart-data-item{align-items:flex-start;background-color:#3b5;display:flex;flex-grow:1;height:calc(100% - 48px);justify-content:center;min-width:1px;transition:background-color .2s ease-in;width:100%}.vm-simple-bar-chart-data-item:hover{background-color:#51d071}.vm-simple-bar-chart-data-item:first-child{background-color:#f79420}.vm-simple-bar-chart-data-item:first-child:hover{background-color:#f9ac51}.vm-cardinality-panel{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-cardinality-panel_mobile,.vm-cardinality-panel_mobile .vm-cardinality-panel-tips{gap:8px}.vm-cardinality-panel-tips{align-content:flex-start;display:inline-flex;flex-grow:1;flex-wrap:wrap;gap:12px;justify-content:flex-start;width:100%}.vm-cardinality-panel-table__header th:first-child{width:60%}.vm-cardinality-panel-table__header th:not(:first-child){width:auto}.vm-cardinality-panel-table__progress{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:minmax(200px,1fr) 70px;justify-content:flex-start}.vm-cardinality-tip{background-color:var(--color-background-block);border-radius:8px;box-shadow:var(--box-shadow);color:var(--color-text-secondary);display:grid;flex-grow:1;grid-template-rows:auto 1fr;overflow:hidden;width:300px}.vm-cardinality-tip-header{align-items:center;border-bottom:var(--border-divider);display:flex;gap:4px;justify-content:center;padding:8px 12px;position:relative}.vm-cardinality-tip-header:after{background:var(--color-warning);content:"";height:100%;left:0;opacity:.1;pointer-events:none;position:absolute;top:0;width:100%}.vm-cardinality-tip-header__tip-icon{align-items:center;color:var(--color-warning);display:flex;justify-content:center;width:12px}.vm-cardinality-tip-header__title{color:var(--color-text);font-weight:700;text-align:center}.vm-cardinality-tip-header__tooltip{font-size:14px;line-height:130%;max-width:280px;padding:8px;white-space:normal}.vm-cardinality-tip__description{line-height:130%;padding:8px 12px}.vm-cardinality-tip__description p{margin-bottom:8px}.vm-cardinality-tip__description p:last-child{margin-bottom:0}.vm-cardinality-tip__description ol,.vm-cardinality-tip__description ul{list-style-position:inside}.vm-cardinality-tip__description ol li,.vm-cardinality-tip__description ul li{margin-bottom:4px}.vm-top-queries-panel-header,.vm-top-queries-panel-header_mobile{margin:-12px -12px 0}.vm-top-queries-panel__table{overflow:auto;padding-top:12px;width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-top-queries-panel__table{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-top-queries-panel__table_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-top-queries-panel__table .vm-table-cell_header{white-space:nowrap}.vm-top-queries{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-top-queries_mobile{gap:8px}.vm-top-queries-controls{grid-gap:8px;display:grid;gap:8px}.vm-top-queries-controls-fields{align-items:center;display:flex;flex-wrap:wrap;gap:12px}.vm-top-queries-controls-fields__item{flex-grow:1;min-width:200px}.vm-top-queries-controls-bottom{grid-gap:12px;align-items:flex-end;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:space-between}.vm-top-queries-controls-bottom_mobile{gap:8px;grid-template-columns:1fr}.vm-top-queries-controls-bottom__button{align-items:center;display:flex;justify-content:flex-end}.vm-top-queries-panels{grid-gap:12px;display:grid;gap:12px}.vm-top-queries-panels__table-actions{align-items:center;display:flex;gap:8px;height:100%;justify-content:flex-end;padding:0 8px}.vm-trace-page{display:flex;flex-direction:column;min-height:100%}@media(max-width:768px){.vm-trace-page{padding:12px 0}}.vm-trace-page-header{grid-gap:12px;align-items:start;display:grid;gap:12px;grid-template-columns:1fr auto;margin-bottom:12px}@media(max-width:768px){.vm-trace-page-header{grid-template-columns:1fr;padding:0 12px}}.vm-trace-page-header-errors{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:1fr;justify-content:stretch}@media(max-width:768px){.vm-trace-page-header-errors{grid-row:2}}.vm-trace-page-header-errors-item{align-items:center;display:grid;justify-content:stretch;position:relative}.vm-trace-page-header-errors-item_margin-bottom{margin-bottom:12px}.vm-trace-page-header-errors-item__filename{min-height:20px}.vm-trace-page-header-errors-item__close{position:absolute;right:8px;top:auto;z-index:2}.vm-trace-page-preview{align-items:center;display:flex;flex-direction:column;flex-grow:1;justify-content:center}.vm-trace-page-preview__text{font-size:14px;line-height:1.8;margin-bottom:12px;text-align:center;white-space:pre-line}.vm-trace-page__dropzone{align-items:center;box-shadow:inset var(--color-primary) 0 0 10px;display:flex;height:100%;justify-content:center;left:0;opacity:.5;pointer-events:none;position:fixed;top:0;width:100%;z-index:100}.vm-upload-json-buttons{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr 1fr;justify-content:center}.vm-explore-metrics{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:500px){.vm-explore-metrics{gap:8px}}.vm-explore-metrics-body{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:500px){.vm-explore-metrics-body{gap:8px}}.vm-explore-metrics-graph,.vm-explore-metrics-graph_mobile{padding:0 12px 12px}.vm-explore-metrics-graph__warning{align-items:center;display:grid;grid-template-columns:1fr auto;justify-content:space-between}.vm-explore-metrics-item-header{grid-gap:12px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:12px;grid-template-columns:auto 1fr auto auto;justify-content:flex-start;padding:12px}.vm-explore-metrics-item-header_mobile{grid-template-columns:1fr auto;padding:8px 12px}.vm-explore-metrics-item-header__index{color:var(--color-text-secondary);font-size:12px}.vm-explore-metrics-item-header__name{flex-grow:1;font-weight:700;line-height:130%;max-width:100%;overflow:hidden;text-overflow:ellipsis}.vm-explore-metrics-item-header-order{align-items:center;display:grid;grid-column:1;grid-template-columns:auto 20px auto;justify-content:flex-start;text-align:center}.vm-explore-metrics-item-header-order__up{transform:rotate(180deg)}.vm-explore-metrics-item-header__rate{grid-column:3}.vm-explore-metrics-item-header__close{align-items:center;display:grid;grid-column:4;grid-row:1}.vm-explore-metrics-item-header code{background-color:var(--color-hover-black);border-radius:6px;font-size:85%;padding:.2em .4em}.vm-explore-metrics-item-header-modal{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-explore-metrics-item-header-modal-order{align-items:center;display:flex;gap:12px;justify-content:space-between}.vm-explore-metrics-item-header-modal-order p{align-items:center;display:flex}.vm-explore-metrics-item-header-modal-order__index{margin-left:4px}.vm-explore-metrics-item-header-modal__rate{grid-gap:8px;display:grid;gap:8px}.vm-explore-metrics-item-header-modal__rate p{color:var(--color-text-secondary)}.vm-explore-metrics-item{position:relative}.vm-select-input{align-items:center;border:var(--border-divider);border-radius:4px;cursor:pointer;display:flex;min-height:40px;padding:8px 0 8px 12px;position:relative}.vm-select-input-content{align-items:center;display:flex;flex-grow:1;flex-wrap:wrap;gap:8px;justify-content:flex-start}.vm-select-input-content_mobile{flex-wrap:nowrap}.vm-select-input-content__counter{font-size:14px;line-height:14px}.vm-select-input-content__selected{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;font-size:14px;justify-content:center;line-height:14px;max-width:100%;padding:2px 2px 2px 6px}.vm-select-input-content__selected span{overflow:hidden;text-overflow:ellipsis;width:100%}.vm-select-input-content__selected svg{align-items:center;background-color:#0000;border-radius:4px;display:flex;justify-content:center;margin-left:10px;padding:4px;transition:background-color .2s ease-in-out;width:20px}.vm-select-input-content__selected svg:hover{background-color:#110f0f1a}.vm-select-input input{background-color:#0000;border:none;border-radius:4px;color:var(--color-text);display:inline-block;flex-grow:1;font-size:14px;height:18px;line-height:18px;min-width:100px;padding:0;position:relative;z-index:2}.vm-select-input input:placeholder-shown{width:auto}.vm-select-input__icon{align-items:center;border-right:var(--border-divider);color:var(--color-text-secondary);cursor:pointer;display:inline-flex;justify-content:flex-end;padding:0 8px;transition:transform .2s ease-in,opacity .2s ease-in}.vm-select-input__icon:last-child{border:none}.vm-select-input__icon svg{width:14px}.vm-select-input__icon_open{transform:rotate(180deg)}.vm-select-input__icon:hover{opacity:.7}.vm-select-options{grid-gap:8px;display:grid;font-size:14px;gap:8px;max-height:208px;max-width:300px;overflow:auto;padding:12px}.vm-select-options_mobile{max-height:calc(var(--vh)*100 - 70px);max-width:100%;padding:0 12px 8px}.vm-select_disabled *{cursor:not-allowed}.vm-select_disabled .vm-select-input-content input{color:var(--color-text-disabled)}.vm-explore-metrics-header{align-items:center;display:flex;flex-wrap:wrap;gap:12px 18px;justify-content:flex-start;max-width:calc(100vw - var(--scrollbar-width))}.vm-explore-metrics-header_mobile{align-items:stretch;flex-direction:column}.vm-explore-metrics-header__job{flex-grow:1;min-width:150px}.vm-explore-metrics-header__instance{flex-grow:2;min-width:150px}.vm-explore-metrics-header__size{grid-gap:12px;align-items:center;display:grid;flex-grow:1;gap:12px;grid-template-columns:1fr auto;min-width:150px}.vm-explore-metrics-header-description{grid-gap:8px;align-items:flex-start;display:grid;gap:8px;grid-template-columns:1fr auto}.vm-explore-metrics-header-description button{color:inherit;min-height:29px}.vm-explore-metrics-header-description code{margin:0 3px}.vm-explore-metrics-header-metrics{flex-grow:1;width:100%}.vm-explore-metrics-header__clear-icon{align-items:center;cursor:pointer;display:flex;justify-content:center;padding:2px}.vm-explore-metrics-header__clear-icon:hover{opacity:.7}.vm-preview-icons{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:repeat(auto-fill,100px);justify-content:center}.vm-preview-icons-item{grid-gap:8px;align-items:stretch;border:1px solid #0000;border-radius:4px;cursor:pointer;display:grid;gap:8px;grid-template-rows:1fr auto;height:100px;justify-content:center;padding:12px 8px;transition:box-shadow .2s ease-in-out}.vm-preview-icons-item:hover{box-shadow:0 1px 4px #00000029}.vm-preview-icons-item:active .vm-preview-icons-item__svg{transform:scale(.9)}.vm-preview-icons-item__name{font-size:12px;line-height:2;overflow:hidden;text-align:center;text-overflow:ellipsis;white-space:nowrap}.vm-preview-icons-item__svg{align-items:center;display:flex;height:100%;justify-content:center;transition:transform .1s ease-out}.vm-preview-icons-item__svg svg{height:24px;width:auto}.vm-with-template,.vm-with-template-body{grid-gap:12px;display:grid;gap:12px}.vm-with-template-body{align-items:flex-start;width:100%}.vm-with-template-body-top{align-items:center;display:flex;gap:8px;justify-content:flex-end}.vm-with-template-body__expr textarea{min-height:200px}.vm-with-template-body__result textarea{min-height:60px}.vm-with-template-body textarea{font-family:monospace;height:100%;overflow:auto;width:100%}.vm-with-template-tutorial{grid-gap:16px;display:grid;gap:16px}.vm-with-template-tutorial__title{font-size:16px;font-weight:700}.vm-with-template-tutorial-section{grid-gap:12px;display:grid;gap:12px}.vm-with-template-tutorial-section__text{font-size:14px;line-height:130%;max-width:720px}.vm-code-example{background-color:#110f0f0d;border-radius:4px;display:block;overflow:auto;padding:12px;position:relative;white-space:pre-wrap}.vm-code-example__copy{position:absolute;right:10px;top:10px}.vm-relabeling,.vm-relabeling-header{grid-gap:12px;display:grid;gap:12px}.vm-relabeling-header{align-items:flex-start;width:100%}.vm-relabeling-header-configs textarea{min-height:200px}.vm-relabeling-header__labels textarea{min-height:60px}.vm-relabeling-header textarea{font-family:monospace;height:100%;overflow:auto;width:100%}.vm-relabeling-header-bottom{align-items:center;display:flex;gap:12px;justify-content:flex-end}.vm-relabeling-header-bottom a{color:var(--color-text-secondary)}.vm-relabeling-steps,.vm-relabeling-steps-item{grid-gap:12px;display:grid;gap:12px}.vm-relabeling-steps-item{border-bottom:var(--border-divider);padding:0 12px 12px}.vm-relabeling-steps-item:last-child{border-bottom:none;padding-bottom:0}.vm-relabeling-steps-item__row{display:grid;grid-template-columns:100px 1fr}@media(max-width:500px){.vm-relabeling-steps-item__row{gap:4px;grid-template-columns:1fr}}.vm-relabeling-steps-item__row pre{white-space:pre-wrap}.vm-active-queries-header{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px}.vm-active-queries-header-controls{grid-gap:8px;display:grid;gap:8px;grid-column:2}.vm-active-queries-header__update-msg{color:var(--color-text-secondary);font-size:12px;white-space:nowrap}.vm-json-form{grid-gap:12px;display:grid;gap:12px;grid-template-rows:auto calc(var(--vh)*70 - 114px) auto;max-height:900px;max-width:1000px;overflow:hidden;width:70vw}.vm-json-form_mobile{grid-template-rows:auto calc(var(--vh)*100 - 236px) auto;min-height:100%;width:100%}.vm-json-form_one-field{grid-template-rows:calc(var(--vh)*70 - 114px) auto}.vm-json-form_one-field_mobile{grid-template-rows:calc(var(--vh)*100 - 184px) auto}.vm-json-form textarea{height:100%;max-height:900px;overflow:auto;width:100%}.vm-json-form-footer{align-items:center;display:flex;gap:8px;justify-content:space-between}@media(max-width:500px){.vm-json-form-footer{flex-direction:column}.vm-json-form-footer button{flex-grow:1}}.vm-json-form-footer__controls{align-items:center;display:flex;flex-grow:1;gap:8px;justify-content:flex-start}@media(max-width:500px){.vm-json-form-footer__controls{grid-template-columns:repeat(2,1fr);justify-content:center;width:100%}}.vm-json-form-footer__controls_right{display:grid;grid-template-columns:repeat(2,90px);justify-content:flex-end}@media(max-width:500px){.vm-json-form-footer__controls_right{grid-template-columns:repeat(2,1fr);justify-content:center;width:100%}}.vm-query-analyzer-view{grid-gap:12px;display:grid;gap:12px;position:relative}.vm-query-analyzer-view-header{align-items:center;border-bottom:var(--border-divider);display:flex;font-size:12px;justify-content:space-between;margin:-12px -12px 12px;padding:0 12px;position:relative;z-index:1}.vm-query-analyzer-view-header__left{align-items:center;display:flex;gap:8px}.vm-query-analyzer-view_mobile .vm-query-analyzer-view-header{margin:-12px -12px 12px;padding:0 12px}.vm-query-analyzer-info-header{display:flex;gap:12px}.vm-query-analyzer-info-header__period{align-items:center;border:var(--border-divider);border-radius:4px;display:flex;gap:8px;padding:6px 12px}.vm-query-analyzer-info-header__period svg{color:var(--color-primary);width:13px}.vm-query-analyzer-info{grid-gap:16px;display:grid;gap:16px;min-width:300px}.vm-query-analyzer-info-type{color:var(--color-text-secondary);font-style:italic;text-align:center}.vm-query-analyzer-info-item{border-bottom:var(--border-divider);display:grid;line-height:130%;padding-bottom:16px}.vm-query-analyzer-info-item__title{font-weight:700}.vm-query-analyzer-info-item__text{white-space:pre-wrap}#root,body,html{background-attachment:fixed;background-color:#fefeff;background-color:var(--color-background-body);background-repeat:no-repeat;color:#110f0f;color:var(--color-text);cursor:default;font-family:system-ui;font-size:14px;margin:0;min-height:100%}body{overflow:auto}*{-webkit-tap-highlight-color:rgba(0,0,0,0);cursor:inherit;font:inherit;touch-action:pan-x pan-y}code{font-family:monospace}b{font-weight:700}input,textarea{cursor:text}input::placeholder,textarea::placeholder{-webkit-user-select:none;user-select:none}input[type=number]::-webkit-inner-spin-button,input[type=number]::-webkit-outer-spin-button{-webkit-appearance:none;margin:0}.vm-snackbar{animation:vm-slide-snackbar .15s cubic-bezier(.28,.84,.42,1.1);bottom:12px;left:12px;position:fixed;z-index:999}.vm-snackbar-content{align-items:center;display:grid;grid-template-columns:1fr auto}.vm-snackbar-content__close{color:inherit;height:24px;opacity:.8;padding:4px;width:24px}.vm-snackbar_mobile{bottom:0;left:0;right:0}@keyframes vm-slide-snackbar{0%{transform:translateY(100%)}to{transform:translateY(0)}}svg{width:100%}*{scrollbar-color:#a09f9f #fff;scrollbar-color:var(--color-text-disabled) var(--color-background-block);scrollbar-width:thin}::-webkit-scrollbar{width:12px}::-webkit-scrollbar-track{background:#fff;background:var(--color-background-block)}::-webkit-scrollbar-thumb{background-color:#a09f9f;background-color:var(--color-text-disabled);border:3px solid #fff;border:3px solid var(--color-background-block);border-radius:20px}a,abbr,acronym,address,applet,article,aside,audio,big,body,canvas,caption,center,cite,code,del,details,dfn,div,em,embed,fieldset,figcaption,figure,footer,form,h1,h2,h3,h4,h5,h6,header,hgroup,html,iframe,img,ins,kbd,label,legend,li,mark,menu,nav,object,ol,output,p,pre,q,ruby,s,samp,section,small,span,strike,strong,sub,summary,sup,table,tbody,td,tfoot,th,thead,time,tr,tt,u,ul,var,video{border:0;margin:0;padding:0;vertical-align:initial}h1,h2,h3,h4,h5,h6{font-weight:400}article,aside,details,figcaption,figure,footer,header,hgroup,menu,nav,section{display:block}body{line-height:1}q:after,q:before{content:""}table{border-collapse:collapse;border-spacing:0}input::placeholder{opacity:1;transition:opacity .3s ease}input:focus::placeholder{opacity:0;transition:opacity .3s ease}*{box-sizing:border-box;outline:none}button{background:none;border:none;border-radius:0;padding:0}strong{letter-spacing:1px}input[type=file]{cursor:pointer;font-size:0;height:100%;left:0;opacity:0;position:absolute;top:0;width:100%}input[type=file]:disabled{cursor:not-allowed}a{color:inherit;text-decoration:inherit}input,textarea{-webkit-text-fill-color:inherit;appearance:none;-webkit-appearance:none}input:disabled,textarea:disabled{opacity:1!important}input:placeholder-shown,textarea:placeholder-shown{width:100%}input:-webkit-autofill,input:-webkit-autofill:active,input:-webkit-autofill:focus,input:-webkit-autofill:hover{-webkit-box-shadow:inset 0 0 0 0 #fff!important;width:100%;z-index:2}.vm-header-button{border:1px solid #110f0f33}.vm-list-item{background-color:#0000;cursor:pointer;padding:12px;transition:background-color .2s ease}.vm-list-item_mobile{padding:12px}.vm-list-item:hover,.vm-list-item_active{background-color:#0000000f;background-color:var(--color-hover-black)}.vm-list-item_multiselect{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:10px 1fr;justify-content:flex-start}.vm-list-item_multiselect svg{animation:vm-scale .15s cubic-bezier(.28,.84,.42,1)}.vm-list-item_multiselect span{grid-column:2}.vm-list-item_multiselect_selected{color:#3f51b5;color:var(--color-primary)}.vm-list-item_with-icon{grid-gap:4px;display:grid;gap:4px;grid-template-columns:14px 1fr}.vm-list-item_with-icon,.vm-mobile-option{align-items:center;justify-content:flex-start}.vm-mobile-option{display:flex;gap:8px;padding:6px 0;-webkit-user-select:none;user-select:none;width:100%}.vm-mobile-option__arrow,.vm-mobile-option__icon{align-items:center;display:flex;justify-content:center}.vm-mobile-option__icon{color:#3f51b5;color:var(--color-primary);height:22px;width:22px}.vm-mobile-option__arrow{color:#3f51b5;color:var(--color-primary);height:14px;transform:rotate(-90deg);width:14px}.vm-mobile-option-text{grid-gap:2px;align-items:center;display:grid;flex-grow:1;gap:2px}.vm-mobile-option-text__label{font-weight:700}.vm-mobile-option-text__value{color:#706f6f;color:var(--color-text-secondary);font-size:12px}.vm-block{background-color:#fff;background-color:var(--color-background-block);border-radius:8px;box-shadow:1px 2px 6px #00000014;box-shadow:var(--box-shadow);padding:12px}.vm-block_mobile{border-radius:0;padding:12px}.vm-block_empty-padding{padding:0}.vm-section-header{align-items:center;border-bottom:1px solid #00000026;border-bottom:var(--border-divider);border-radius:8px 8px 0 0;display:grid;grid-template-columns:1fr auto;justify-content:center;padding:0 12px}.vm-section-header__title{font-size:14px;font-weight:700}.vm-section-header__title_mobile{-webkit-line-clamp:2;line-clamp:2;-webkit-box-orient:vertical;display:-webkit-box;overflow:hidden;text-overflow:ellipsis}.vm-section-header__tabs{align-items:center;display:flex;font-size:12px;justify-content:flex-start}.vm-table{border-collapse:initial;border-spacing:0;margin-top:-12px;width:100%}.vm-table,.vm-table__row{background-color:#fff;background-color:var(--color-background-block)}.vm-table__row{transition:background-color .2s ease}.vm-table__row:hover:not(.vm-table__row_header){background-color:#0000000f;background-color:var(--color-hover-black)}.vm-table__row_header{position:relative;z-index:2}.vm-table__row_selected{background-color:#1a90ff0d}.vm-table-cell{border-bottom:1px solid #00000026;border-bottom:var(--border-divider);line-height:1.5;overflow-wrap:anywhere;padding:4px;vertical-align:top}.vm-table-cell__content{align-items:center;display:flex;justify-content:flex-start}.vm-table-cell_sort{cursor:pointer}.vm-table-cell_sort:hover{background-color:#0000000f;background-color:var(--color-hover-black)}.vm-table-cell_header{font-weight:700;overflow-wrap:normal;text-align:left}.vm-table-cell_gray{color:#110f0f;color:var(--color-text);opacity:.4}.vm-table-cell_right{text-align:right}.vm-table-cell_right .vm-table-cell__content{justify-content:flex-end}.vm-table-cell_no-wrap{white-space:nowrap}.vm-table-cell_no-padding{padding:0}.vm-table-cell_pre{white-space:pre-wrap}.vm-table-cell_logs-time{overflow-wrap:normal;white-space:pre}.vm-table-cell_logs{font-family:monospace;line-height:1.2}.vm-table__sort-icon{align-items:center;display:flex;justify-content:center;margin:0 8px;opacity:.4;transition:opacity .2s ease,transform .2s ease-in-out;width:15px}.vm-table__sort-icon_active{opacity:1}.vm-table__sort-icon_desc{transform:rotate(180deg)}.vm-link{cursor:pointer;transition:color .2s ease}.vm-link_colored{color:#3f51b5;color:var(--color-primary)}.vm-link_underlined{text-decoration:underline}.vm-link_with-icon{grid-gap:6px;align-items:center;display:grid;gap:6px;grid-template-columns:14px auto;justify-content:center}.vm-link:hover{color:#3f51b5;color:var(--color-primary);text-decoration:underline}.vm-dynamic-number{color:#a09f9f;color:var(--color-text-disabled);font-size:12px}.vm-dynamic-number_positive{color:#4caf50;color:var(--color-success)}.vm-dynamic-number_negative{color:#fd080e;color:var(--color-error)}.vm-dynamic-number_down:before{content:"↓"}.vm-dynamic-number_up:before{content:"↑"}:root{--color-primary:#3f51b5;--color-secondary:#e91e63;--color-error:#fd080e;--color-warning:#ff8308;--color-info:#03a9f4;--color-success:#4caf50;--color-primary-text:#fff;--color-secondary-text:#fff;--color-error-text:#fff;--color-warning-text:#fff;--color-info-text:#fff;--color-success-text:#fff;--color-background-body:#fefeff;--color-background-block:#fff;--color-background-tooltip:#505050e6;--color-text:#110f0f;--color-text-secondary:#706f6f;--color-text-disabled:#a09f9f;--box-shadow:#00000014 1px 2px 6px;--box-shadow-popper:#0000001a 0px 2px 8px 0px;--border-divider:1px solid #00000026;--color-hover-black:#0000000f} \ No newline at end of file diff --git a/app/vmselect/vmui/static/css/main.d0b400e3.css b/app/vmselect/vmui/static/css/main.d0b400e3.css new file mode 100644 index 000000000..e37c72423 --- /dev/null +++ b/app/vmselect/vmui/static/css/main.d0b400e3.css @@ -0,0 +1 @@ +.vm-tabs{gap:12px;height:100%;position:relative;-webkit-user-select:none;user-select:none}.vm-tabs,.vm-tabs-item{align-items:center;display:flex;justify-content:center}.vm-tabs-item{color:inherit;cursor:pointer;font-size:inherit;font-weight:inherit;opacity:.6;padding:12px 8px;text-decoration:none;text-transform:capitalize;transition:opacity .2s}.vm-tabs-item_active{opacity:1}.vm-tabs-item:hover{opacity:.8}.vm-tabs-item__icon{display:grid;margin-right:8px;width:16px}.vm-tabs-item__icon_single{margin-right:0}.vm-tabs__indicator{border-bottom:2px solid;position:absolute;transition:width .2s ease,left .3s cubic-bezier(.28,.84,.42,1)}.vm-alert{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-radius:8px;box-shadow:var(--box-shadow);color:var(--color-text);display:grid;font-size:14px;font-weight:400;gap:8px;grid-template-columns:20px 1fr;line-height:1.5;padding:12px;position:relative}.vm-alert_mobile{align-items:flex-start;border-radius:0}.vm-alert:after{border-radius:8px;content:"";height:100%;left:0;opacity:.1;position:absolute;top:0;width:100%;z-index:1}.vm-alert_mobile:after{border-radius:0}.vm-alert__content,.vm-alert__icon{position:relative;z-index:2}.vm-alert__icon{align-items:center;display:flex;justify-content:center}.vm-alert__content{filter:brightness(.6);white-space:pre-line}.vm-alert_success{color:var(--color-success)}.vm-alert_success:after{background-color:var(--color-success)}.vm-alert_error{color:var(--color-error)}.vm-alert_error:after{background-color:var(--color-error)}.vm-alert_info{color:var(--color-info)}.vm-alert_info:after{background-color:var(--color-info)}.vm-alert_warning{color:var(--color-warning)}.vm-alert_warning:after{background-color:var(--color-warning)}.vm-alert_dark:after{opacity:.1}.vm-alert_dark .vm-alert__content{filter:none}.vm-header{align-items:center;display:flex;flex-wrap:wrap;gap:0 16px;justify-content:flex-start;min-height:51px;padding:8px 12px;z-index:99}.vm-header_app{padding:8px 0}@media(max-width:1000px){.vm-header{gap:8px;padding:8px;position:-webkit-sticky;position:sticky;top:0}}.vm-header_sidebar{display:grid;grid-template-columns:40px auto 1fr}.vm-header_mobile{display:grid;grid-template-columns:33px 1fr 33px;justify-content:space-between}.vm-header_dark .vm-header-button,.vm-header_dark button,.vm-header_dark button:before{background-color:var(--color-background-block)}.vm-header-logo{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;margin-bottom:2px;overflow:hidden;position:relative;width:100%}.vm-header-logo,.vm-header-logo svg,.vm-header-logo_mobile{max-width:65px;min-width:65px}.vm-header-logo_mobile{margin:0 auto}.vm-header-logo_logs,.vm-header-logo_logs svg{max-width:75px;min-width:75px}.vm-header-nav{align-items:center;display:flex;gap:12px;justify-content:flex-start}.vm-header-nav_column{align-items:stretch;flex-direction:column;gap:8px}.vm-header-nav_column .vm-header-nav-item{padding:12px 0}.vm-header-nav_column .vm-header-nav-item_sub{justify-content:stretch}.vm-header-nav-item{cursor:pointer;font-size:14px;font-weight:400;opacity:1;padding:12px 8px;position:relative;text-transform:capitalize;transition:opacity .2s ease-in}.vm-header-nav-item_sub{grid-gap:4px;align-items:center;cursor:default;display:grid;gap:4px;grid-template-columns:auto 14px;justify-content:center}.vm-header-nav-item:hover{opacity:.7}.vm-header-nav-item_active{border-bottom:2px solid #110f0f33}.vm-header-nav-item svg{transform:rotate(0deg);transition:transform .2s ease-in}.vm-header-nav-item_open svg{transform:rotate(180deg)}.vm-header-nav-item-submenu{border-radius:4px;color:#fff;display:grid;opacity:1;padding:8px;transform-origin:top center;white-space:nowrap}.vm-header-nav-item-submenu-item{cursor:pointer}.vm-popper{background-color:var(--color-background-block);border-radius:4px;box-shadow:var(--box-shadow-popper);opacity:0;pointer-events:none;position:fixed;transition:opacity .1s ease-in-out;z-index:-99}.vm-popper_open{animation:vm-slider .15s cubic-bezier(.28,.84,.42,1.1);opacity:1;pointer-events:auto;transform-origin:top center;z-index:101}.vm-popper_mobile{animation:none;border-radius:0;bottom:0;left:0;overflow:auto;position:fixed;right:0;top:0;width:100%}.vm-popper-header{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-bottom:var(--border-divider);border-radius:4px 4px 0 0;color:var(--color-text);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px;min-height:51px;padding:8px 8px 8px 12px}.vm-popper-header__title{font-weight:700;-webkit-user-select:none;user-select:none}.vm-popper_dark{background-color:var(--color-background-tooltip);color:#fff}.vm-popper_dark .vm-popper-header{background-color:#0000;color:#fff}@keyframes vm-slider{0%{transform:scaleY(0)}to{transform:scaleY(1)}}.vm-modal{align-items:center;background:#110f0f8c;bottom:0;display:flex;justify-content:center;left:0;position:fixed;right:0;top:0;z-index:100}.vm-modal_mobile{align-items:flex-start;max-height:calc(var(--vh)*100);min-height:calc(var(--vh)*100);overflow:auto}.vm-modal_mobile .vm-modal-content{border-radius:0;grid-template-rows:70px -webkit-max-content;grid-template-rows:70px max-content;max-height:-webkit-max-content;max-height:max-content;min-height:100%;overflow:visible;width:100vw}.vm-modal_mobile .vm-modal-content-header{margin-bottom:12px;padding:8px 8px 8px 12px}.vm-modal_mobile .vm-modal-content-header__title{max-width:80vw}.vm-modal_mobile .vm-modal-content-body{align-items:flex-start;display:grid;min-height:100%;padding:0 12px 12px}.vm-modal-content{background:var(--color-background-block);border-radius:4px;box-shadow:0 0 24px #110f0f12;max-height:calc(var(--vh)*90);overflow:auto}.vm-modal-content-header{grid-gap:8px;align-items:center;background-color:var(--color-background-block);border-bottom:var(--border-divider);border-radius:4px 4px 0 0;color:var(--color-text);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px;min-height:51px;padding:12px;position:-webkit-sticky;position:sticky;top:0;z-index:3}.vm-modal-content-header__title{font-weight:700;max-width:50vw;overflow:hidden;text-overflow:ellipsis;-webkit-user-select:none;user-select:none;white-space:nowrap}.vm-modal-content-header__close{align-items:center;box-sizing:initial;color:#fff;cursor:pointer;display:flex;justify-content:center;padding:10px;width:24px}.vm-modal-content-body{padding:0 12px 12px}.vm-shortcuts{min-width:400px}@media(max-width:500px){.vm-shortcuts{min-width:100%}}.vm-shortcuts-section{border-bottom:var(--border-divider);margin-bottom:12px;padding-bottom:12px;position:relative}.vm-shortcuts-section__title{font-weight:700;margin-bottom:12px}.vm-shortcuts-section__read-more{position:absolute;right:0;top:-8px}.vm-shortcuts-section-list{grid-gap:12px;display:grid;gap:12px}@media(max-width:500px){.vm-shortcuts-section-list{gap:12px}}.vm-shortcuts-section-list-item{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:210px 1fr}@media(max-width:500px){.vm-shortcuts-section-list-item{grid-template-columns:1fr}}.vm-shortcuts-section-list-item__key{align-items:center;display:flex;gap:4px}.vm-shortcuts-section-list-item__key code,.vm-shortcuts-section-list-item__key svg{background-color:var(--color-background-body);background-repeat:repeat-x;border:var(--border-divider);border-radius:4px;color:var(--color-text);display:inline-block;font-size:12px;line-height:2;padding:2px 8px 0;text-align:center}.vm-shortcuts-section-list-item__key svg{color:var(--color-primary);padding:4px;width:24px}.vm-shortcuts-section-list-item__description{font-size:14px}.vm-tooltip{animation:vm-scale .15s cubic-bezier(.28,.84,.42,1);background-color:var(--color-background-tooltip);border-radius:4px;box-shadow:var(--box-shadow-popper);color:#fff;font-size:12px;line-height:150%;opacity:1;padding:3px 8px;pointer-events:auto;position:fixed;transition:opacity .1s ease-in-out;white-space:nowrap;z-index:101}@keyframes vm-scale{0%{transform:scale(0)}to{transform:scale(1)}}.fc-graph-tips{grid-gap:12px;display:grid;gap:12px;max-width:520px}.fc-graph-tips-item{grid-gap:8px;border-bottom:var(--border-divider);display:grid;gap:8px;line-height:1.3;padding-bottom:12px}.fc-graph-tips-item__action{color:var(--color-text-secondary);font-weight:700}.fc-graph-tips-item__description{display:inline-block;line-height:1.5}.fc-graph-tips-item__description code,.fc-graph-tips-item__description svg{align-items:center;background-color:var(--color-background-body);border:var(--border-divider);border-radius:4px;color:var(--color-text);display:inline-flex;font-size:12px;justify-content:center;margin:0 2px 2px;min-height:20px;min-width:20px;padding:0 4px}.fc-graph-tips-item svg{color:var(--color-primary);margin-top:-8px;padding:2px;transform:translateY(8px);width:18px}.vm-menu-burger{background:none;border:none;cursor:pointer;height:18px;outline:none;padding:0;position:relative;transform-style:preserve-3d;width:18px}.vm-menu-burger:after{background-color:#110f0f1a;border-radius:50%;content:"";height:calc(100% + 12px);left:-6px;position:absolute;top:-6px;transform:scale(0) translateZ(-2px);transition:transform .14s ease-in-out;width:calc(100% + 12px)}.vm-menu-burger:hover:after{transform:scale(1) translateZ(-2px)}.vm-menu-burger span{border-top:2px solid #fff;display:block;top:50%;transform:translateY(-50%);transition:transform .3s ease,border-color .3s ease}.vm-menu-burger span,.vm-menu-burger span:after,.vm-menu-burger span:before{border-radius:6px;height:2px;left:0;position:absolute;width:100%}.vm-menu-burger span:after,.vm-menu-burger span:before{animation-duration:.6s;animation-fill-mode:forwards;animation-timing-function:cubic-bezier(.645,.045,.355,1);background:#fff;content:"";top:0}.vm-menu-burger span:before{animation-name:topLineBurger}.vm-menu-burger span:after{animation-name:bottomLineBurger}.vm-menu-burger_opened span{border-color:#0000}.vm-menu-burger_opened span:before{animation-name:topLineCross}.vm-menu-burger_opened span:after{animation-name:bottomLineCross}@keyframes topLineCross{0%{transform:translateY(-7px)}50%{transform:translateY(0)}to{transform:translateY(-2px) translateX(30%) rotate(45deg);width:60%}}@keyframes bottomLineCross{0%{transform:translateY(3px)}50%{transform:translateY(0)}to{transform:translateY(-2px) translateX(30%) rotate(-45deg);width:60%}}@keyframes topLineBurger{0%{transform:translateY(0) rotate(45deg)}50%{transform:rotate(0deg)}to{transform:translateY(-7px) rotate(0deg)}}@keyframes bottomLineBurger{0%{transform:translateY(0) rotate(-45deg)}50%{transform:rotate(0deg)}to{transform:translateY(3px) rotate(0deg)}}.vm-header-sidebar{background-color:inherit;color:inherit;height:24px;width:24px}.vm-header-sidebar-button{align-items:center;display:flex;height:51px;justify-content:center;left:0;position:absolute;top:0;transition:left .35s cubic-bezier(.28,.84,.42,1);width:51px}.vm-header-sidebar-button_open{left:149px;position:fixed;z-index:102}.vm-header-sidebar-menu{grid-gap:12px;background-color:inherit;box-shadow:var(--box-shadow-popper);display:grid;gap:12px;grid-template-rows:1fr auto;height:100%;left:0;padding:12px;position:fixed;top:0;transform:translateX(-100%);transform-origin:left;transition:transform .3s cubic-bezier(.28,.84,.42,1);width:200px;z-index:101}.vm-header-sidebar-menu_open{transform:translateX(0)}.vm-header-sidebar-menu__logo{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;position:relative;width:65px}.vm-header-sidebar-menu-settings{grid-gap:8px;align-items:center;display:grid;gap:8px}.vm-header-controls{align-items:center;display:flex;flex-grow:1;gap:8px;justify-content:flex-end}.vm-header-controls_mobile{display:grid;grid-template-columns:1fr;padding:0}.vm-header-controls_mobile .vm-header-button{border:none}.vm-header-controls-modal{transform:scale(0)}.vm-header-controls-modal_open{transform:scale(1)}.vm-container{display:flex;flex-direction:column;min-height:calc(var(--vh)*100 - var(--scrollbar-height))}.vm-container-body{background-color:var(--color-background-body);flex-grow:1;min-height:100%;padding:12px}.vm-container-body_mobile{padding:8px 0 0}@media(max-width:768px){.vm-container-body{padding:8px 0 0}}.vm-container-body_app{background-color:#0000;padding:8px 0}.vm-footer{align-items:center;background:var(--color-background-body);border-top:var(--border-divider);color:var(--color-text-secondary);display:flex;flex-wrap:wrap;gap:12px;justify-content:center;padding:12px}@media(max-width:768px){.vm-footer{gap:12px;padding:12px}}.vm-footer__link,.vm-footer__website{grid-gap:6px;align-items:center;display:grid;gap:6px;grid-template-columns:12px auto;justify-content:center}.vm-footer__website{margin-right:12px}@media(max-width:768px){.vm-footer__website{margin-right:0}}.vm-footer__link{grid-template-columns:14px auto}.vm-footer__copyright{flex-grow:1;text-align:right}@media(max-width:768px){.vm-footer__copyright{font-size:12px;text-align:center;width:100%}}.vm-tenant-input{position:relative}.vm-tenant-input-list{border-radius:8px;max-height:300px;overflow:auto;overscroll-behavior:none}.vm-tenant-input-list_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-tenant-input-list_mobile .vm-tenant-input-list__search{padding:0 12px 8px}.vm-tenant-input-list__search{background-color:var(--color-background-block);padding:8px 12px;position:-webkit-sticky;position:sticky;top:0}.vm-text-field{display:grid;margin:6px 0;position:relative;width:100%}.vm-text-field_textarea:after{content:attr(data-replicated-value) " ";overflow-wrap:break-word;visibility:hidden;white-space:pre-wrap}.vm-text-field:after,.vm-text-field__input{background-color:#0000;border:var(--border-divider);box-sizing:border-box;font-family:monospace;font-size:14px;grid-area:1/1/2/2;line-height:18px;overflow:hidden;padding:8px 12px;width:100%}.vm-text-field__error,.vm-text-field__helper-text,.vm-text-field__label,.vm-text-field__warning{-webkit-line-clamp:1;line-clamp:1;-webkit-box-orient:vertical;background-color:var(--color-background-block);display:-webkit-box;font-size:12px;left:6px;line-height:14px;max-width:calc(100% - 12px);overflow:hidden;padding:0 3px;pointer-events:none;position:absolute;text-overflow:ellipsis;-webkit-user-select:none;user-select:none;z-index:2}.vm-text-field__label{color:var(--color-text-secondary);top:-8px}.vm-text-field__error,.vm-text-field__helper-text,.vm-text-field__warning{overflow-wrap:anywhere;pointer-events:auto;position:relative;top:-6px;-webkit-user-select:text;user-select:text;width:-webkit-fit-content;width:-moz-fit-content;width:fit-content}.vm-text-field__error_full,.vm-text-field__helper-text_full,.vm-text-field__warning_full{display:block;overflow:visible}.vm-text-field__error_overflowed,.vm-text-field__helper-text_overflowed,.vm-text-field__warning_overflowed{cursor:pointer}.vm-text-field__error{color:var(--color-error)}.vm-text-field__warning{color:var(--color-warning)}.vm-text-field__helper-text{color:var(--color-text-secondary)}.vm-text-field__input{background-color:#0000;border-radius:4px;color:var(--color-text);display:block;overflow:hidden;resize:none;transition:border .2s ease}.vm-text-field__input:focus,.vm-text-field__input:hover{border:1px solid var(--color-primary)}.vm-text-field__input_error,.vm-text-field__input_error:focus,.vm-text-field__input_error:hover{border-color:var(--color-error)}.vm-text-field__input_warning,.vm-text-field__input_warning:focus,.vm-text-field__input_warning:hover{border-color:var(--color-warning)}.vm-text-field__input_icon-start{padding-left:31px}.vm-text-field__input:disabled{background-color:inherit;color:inherit}.vm-text-field__input:disabled:hover{border-color:var(--color-text-disabled)}.vm-text-field__icon-end,.vm-text-field__icon-start{align-items:center;color:var(--color-text-secondary);display:flex;height:40px;justify-content:center;left:8px;max-width:15px;position:absolute;top:0}.vm-text-field__icon-end{left:auto;right:8px}.vm-text-field__controls-info{bottom:8px;color:var(--color-text-secondary);font-size:12px;opacity:.8;position:absolute;right:12px}.vm-step-control{display:inline-flex}.vm-step-control button{text-transform:none}.vm-step-control__value{display:inline;margin-left:3px}.vm-step-control-popper{grid-gap:8px;display:grid;font-size:14px;gap:8px;max-height:208px;max-width:300px;overflow:auto;padding:12px}.vm-step-control-popper_mobile{max-height:calc(var(--vh)*100 - 70px);max-width:100%;padding:0 12px 8px}.vm-step-control-popper_mobile .vm-step-control-popper-info{font-size:14px}.vm-step-control-popper-info{font-size:12px;line-height:1.8}.vm-step-control-popper-info a{margin:0 .4em}.vm-step-control-popper-info code{background-color:var(--color-hover-black);border-radius:6px;margin:0 .2em;padding:.2em .4em}.vm-time-duration{font-size:14px;max-height:227px;overflow:auto}.vm-time-duration_mobile{max-height:100%}.vm-time-selector{display:grid;grid-template-columns:repeat(2,230px);padding:12px 0}.vm-time-selector_mobile{grid-template-columns:1fr;max-height:calc(var(--vh)*100 - 70px);min-width:250px;overflow:auto;width:100%}.vm-time-selector_mobile .vm-time-selector-left{border-bottom:var(--border-divider);border-right:none;padding-bottom:12px}.vm-time-selector-left{border-right:var(--border-divider);display:flex;flex-direction:column;gap:8px;padding:0 12px}.vm-time-selector-left-inputs{align-items:flex-start;display:grid;flex-grow:1;justify-content:stretch}.vm-time-selector-left-timezone{align-items:center;display:flex;font-size:12px;gap:8px;justify-content:space-between;margin-bottom:8px}.vm-time-selector-left-timezone__utc{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;justify-content:center;padding:4px}.vm-time-selector-left__controls{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(2,1fr)}.vm-calendar{background-color:var(--color-background-block);border-radius:8px;display:grid;font-size:14px;grid-template-rows:auto 1fr auto;padding:12px;-webkit-user-select:none;user-select:none}.vm-calendar_mobile{padding:0 12px}.vm-calendar-header{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:center;min-height:36px;padding-bottom:12px}.vm-calendar-header-left{grid-gap:8px;align-items:center;cursor:pointer;display:grid;gap:8px;grid-template-columns:auto auto;justify-content:flex-start;transition:opacity .2s ease-in-out}.vm-calendar-header-left:hover{opacity:.8}.vm-calendar-header-left__date{color:var(--color-text);font-size:14px;font-weight:700}.vm-calendar-header-left__select-year{align-items:center;display:grid;height:14px;justify-content:center;width:14px}.vm-calendar-header-right{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:18px 18px;justify-content:center}.vm-calendar-header-right__next,.vm-calendar-header-right__prev{cursor:pointer;margin:-8px;padding:8px;transition:opacity .2s ease-in-out}.vm-calendar-header-right__next:hover,.vm-calendar-header-right__prev:hover{opacity:.8}.vm-calendar-header-right__prev{transform:rotate(90deg)}.vm-calendar-header-right__next{transform:rotate(-90deg)}.vm-calendar-body{grid-gap:2px;align-items:center;display:grid;gap:2px;grid-template-columns:repeat(7,32px);grid-template-rows:repeat(7,32px);justify-content:center}@media(max-width:500px){.vm-calendar-body{grid-template-columns:repeat(7,calc(14.28571vw - 5.14286px));grid-template-rows:repeat(7,calc(14.28571vw - 5.14286px))}}.vm-calendar-body-cell{align-items:center;border-radius:50%;display:flex;height:100%;justify-content:center;text-align:center}.vm-calendar-body-cell_weekday{color:var(--color-text-secondary)}.vm-calendar-body-cell_day{cursor:pointer;transition:color .2s ease,background-color .3s ease-in-out}.vm-calendar-body-cell_day:hover{background-color:var(--color-hover-black)}.vm-calendar-body-cell_day_empty{pointer-events:none}.vm-calendar-body-cell_day_active{color:#fff}.vm-calendar-body-cell_day_active,.vm-calendar-body-cell_day_active:hover{background-color:var(--color-primary)}.vm-calendar-body-cell_day_today{border:1px solid var(--color-primary)}.vm-calendar-years{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(3,1fr);max-height:400px;overflow:auto}.vm-calendar-years__year{align-items:center;border-radius:8px;cursor:pointer;display:flex;justify-content:center;padding:8px 16px;transition:color .2s ease,background-color .3s ease-in-out}.vm-calendar-years__year:hover{background-color:var(--color-hover-black)}.vm-calendar-years__year_selected{color:#fff}.vm-calendar-years__year_selected,.vm-calendar-years__year_selected:hover{background-color:var(--color-primary)}.vm-calendar-years__year_today{border:1px solid var(--color-primary)}.vm-calendar-footer{align-items:center;display:flex;justify-content:flex-end}.vm-date-time-input{grid-gap:8px 0;align-items:center;cursor:pointer;display:grid;gap:8px 0;grid-template-columns:1fr;justify-content:center;margin-bottom:12px;position:relative;transition:color .2s ease-in-out,border-bottom-color .3s ease}.vm-date-time-input:hover input{border-bottom-color:var(--color-primary)}.vm-date-time-input label{color:var(--color-text-secondary);font-size:12px;grid-column:1/3;-webkit-user-select:none;user-select:none;width:100%}.vm-date-time-input__icon{bottom:2px;position:absolute;right:0}.vm-date-time-input input{background:#0000;border:none;border-bottom:var(--border-divider);color:var(--color-text);padding:0 0 8px}.vm-date-time-input input:focus{border-bottom-color:var(--color-primary)}.vm-date-time-input_error input{border-color:var(--color-error)}.vm-date-time-input_error input:focus{border-bottom-color:var(--color-error)}.vm-date-time-input__error-text{bottom:-12px;color:var(--color-error);font-size:12px;left:0;position:absolute}.vm-button{align-items:center;border-radius:6px;color:#fff;cursor:pointer;display:flex;font-size:12px;font-weight:400;justify-content:center;line-height:15px;min-height:31px;padding:6px 14px;position:relative;text-transform:uppercase;transform-style:preserve-3d;-webkit-user-select:none;user-select:none;white-space:nowrap}.vm-button:hover:after{background-color:var(--color-hover-black)}.vm-button:after,.vm-button:before{border-radius:6px;content:"";height:100%;left:0;position:absolute;top:0;transition:background-color .2s ease;width:100%}.vm-button:before{transform:translateZ(-2px)}.vm-button:after{background-color:#0000;transform:translateZ(-1px)}.vm-button:active:after{transform:scale(.9)}.vm-button span{align-items:center;display:grid;justify-content:center;transform:translateZ(1px)}.vm-button span svg{width:15px}.vm-button__start-icon{margin-right:6px}.vm-button__end-icon{margin-left:6px}.vm-button_disabled{cursor:not-allowed;opacity:.3}.vm-button_icon{padding:6px 8px}.vm-button_icon .vm-button__end-icon,.vm-button_icon .vm-button__start-icon{margin:0}.vm-button_small{min-height:25px;padding:4px 8px}.vm-button_small span svg{width:13px}.vm-button_contained_primary{color:var(--color-primary-text)}.vm-button_contained_primary,.vm-button_contained_primary:before{background-color:var(--color-primary)}.vm-button_contained_primary:hover:after{background-color:#110f0f33}.vm-button_contained_secondary{color:var(--color-secondary-text)}.vm-button_contained_secondary:before{background-color:var(--color-secondary)}.vm-button_contained_secondary:hover:after{background-color:#110f0f33}.vm-button_contained_success{color:var(--color-success-text)}.vm-button_contained_success:before{background-color:var(--color-success)}.vm-button_contained_success:hover:after{background-color:#110f0f33}.vm-button_contained_error{color:var(--color-error-text)}.vm-button_contained_error:before{background-color:var(--color-error)}.vm-button_contained_gray{color:var(--color-text-secondary)}.vm-button_contained_gray:before{background-color:var(--color-text-secondary)}.vm-button_contained_warning{color:var(--color-warning)}.vm-button_contained_warning:before{background-color:var(--color-warning);opacity:.2}.vm-button_text_primary{color:var(--color-primary)}.vm-button_text_secondary{color:var(--color-secondary)}.vm-button_text_success{color:var(--color-success)}.vm-button_text_error{color:var(--color-error)}.vm-button_text_gray{color:var(--color-text-secondary)}.vm-button_text_white{color:#fff}.vm-button_text_warning{color:var(--color-warning)}.vm-button_outlined_primary{border:1px solid var(--color-primary);color:var(--color-primary)}.vm-button_outlined_error{border:1px solid var(--color-error);color:var(--color-error)}.vm-button_outlined_secondary{border:1px solid var(--color-secondary);color:var(--color-secondary)}.vm-button_outlined_success{border:1px solid var(--color-success);color:var(--color-success)}.vm-button_outlined_gray{border:1px solid var(--color-text-secondary);color:var(--color-text-secondary)}.vm-button_outlined_white{border:1px solid #fff;color:#fff}.vm-button_outlined_warning{border:1px solid var(--color-warning);color:var(--color-warning)}.vm-execution-controls-buttons{border-radius:7px;display:flex;justify-content:space-between;min-width:107px}.vm-execution-controls-buttons_mobile{flex-direction:column;gap:12px}.vm-execution-controls-buttons__arrow{align-items:center;display:flex;justify-content:center;transform:rotate(0);transition:transform .2s ease-in-out}.vm-execution-controls-buttons__arrow_open{transform:rotate(180deg)}.vm-execution-controls-list{font-size:14px;max-height:208px;overflow:auto;padding:8px 0;width:124px}.vm-execution-controls-list_mobile{max-height:calc(var(--vh)*100 - 70px);padding:0;width:100%}.vm-server-configurator{align-items:center;display:flex;flex-direction:column;gap:12px;padding-bottom:12px;width:600px}.vm-server-configurator_mobile{align-items:flex-start;grid-auto-rows:-webkit-min-content;grid-auto-rows:min-content;height:100%;width:100%}@media(max-width:768px){.vm-server-configurator{width:100%}}.vm-server-configurator__input{width:100%}.vm-server-configurator__input_flex{align-items:flex-start;display:flex;gap:12px}.vm-server-configurator__title{align-items:center;display:flex;font-size:14px;font-weight:700;grid-column:auto/span 2;justify-content:flex-start;margin-bottom:12px}.vm-server-configurator-url{align-items:flex-start;display:flex;gap:8px}.vm-server-configurator-url__button{margin-top:8px}.vm-server-configurator-footer{align-items:center;display:flex;gap:8px;justify-content:flex-end;width:100%}.vm-server-configurator_mobile .vm-server-configurator-footer{display:grid;grid-template-columns:1fr 1fr}.vm-limits-configurator-title__reset{align-items:center;display:flex;flex-grow:1;justify-content:flex-end}.vm-limits-configurator__inputs{grid-gap:12px;align-items:center;display:grid;flex-wrap:wrap;gap:12px;grid-template-columns:repeat(auto-fit,minmax(150px,1fr));justify-content:space-between}.vm-limits-configurator__inputs_mobile{gap:8px}.vm-accordion-header{align-items:center;cursor:pointer;display:grid;font-size:inherit;position:relative}.vm-accordion-header__arrow{align-items:center;display:flex;justify-content:center;position:absolute;right:14px;top:auto;transform:rotate(0);transition:transform .2s ease-in-out}.vm-accordion-header__arrow_open{transform:rotate(180deg)}.vm-accordion-header__arrow svg{height:auto;width:14px}.accordion-section{overflow:hidden}.vm-timezones-item{align-items:center;cursor:pointer;display:flex;gap:8px;justify-content:space-between}.vm-timezones-item_selected{border:var(--border-divider);border-radius:4px;padding:8px 12px}.vm-timezones-item__title{align-items:center;display:flex;gap:8px;text-transform:capitalize}.vm-timezones-item__title svg{color:var(--color-warning);width:14px}.vm-timezones-item__utc{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;justify-content:center;padding:4px}.vm-timezones-item__icon{align-items:center;display:inline-flex;justify-content:flex-end;margin:0 0 0 auto;transition:transform .2s ease-in}.vm-timezones-item__icon svg{width:14px}.vm-timezones-item__icon_open{transform:rotate(180deg)}.vm-timezones-list{background-color:var(--color-background-block);border-radius:8px;max-height:300px;overflow:auto}.vm-timezones-list_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-timezones-list_mobile .vm-timezones-list-header__search{padding:0 12px}.vm-timezones-list-header{background-color:var(--color-background-block);border-bottom:var(--border-divider);position:-webkit-sticky;position:sticky;top:0;z-index:2}.vm-timezones-list-header__search{padding:8px}.vm-timezones-list-group{border-bottom:var(--border-divider);padding:8px 0}.vm-timezones-list-group:last-child{border-bottom:none}.vm-timezones-list-group__title{color:var(--color-text-secondary);font-weight:700;padding:8px 12px}.vm-timezones-list-group-options{align-items:flex-start;display:grid}.vm-timezones-list-group-options__item{padding:8px 12px;transition:background-color .2s ease}.vm-timezones-list-group-options__item:hover{background-color:#110f0f1a}.vm-theme-control__toggle{display:inline-flex;min-width:300px;text-transform:capitalize}.vm-theme-control_mobile .vm-theme-control__toggle{display:flex;min-width:100%}.vm-toggles{grid-gap:3px;display:grid;gap:3px;position:relative;width:100%}.vm-toggles__label{color:var(--color-text-secondary);font-size:12px;line-height:1;padding:0 12px}.vm-toggles-group{overflow:hidden;width:100%}.vm-toggles-group,.vm-toggles-group-item{align-items:center;display:grid;justify-content:center;position:relative}.vm-toggles-group-item{border-bottom:var(--border-divider);border-right:var(--border-divider);border-top:var(--border-divider);color:var(--color-text-secondary);cursor:pointer;font-size:12px;font-weight:700;padding:8px;text-align:center;transition:color .15s ease-in;-webkit-user-select:none;user-select:none;z-index:2}.vm-toggles-group-item_first{border-left:var(--border-divider);border-radius:16px 0 0 16px}.vm-toggles-group-item:last-child{border-left:none;border-radius:0 16px 16px 0}.vm-toggles-group-item_icon{gap:4px;grid-template-columns:14px auto}.vm-toggles-group-item:hover{color:var(--color-primary)}.vm-toggles-group-item_active{border-color:#0000;color:var(--color-primary)}.vm-toggles-group-item_active:hover{background-color:#0000}.vm-toggles-group__highlight{background-color:rgba(var(--color-primary),.08);border:1px solid var(--color-primary);height:100%;position:absolute;top:0;transition:left .2s cubic-bezier(.28,.84,.42,1),border-radius .2s linear;z-index:1}.vm-autocomplete{max-height:300px;overflow:auto;overscroll-behavior:none;position:relative}.vm-autocomplete_mobile{max-height:calc(var(--vh)*100 - 70px)}.vm-autocomplete__no-options{color:var(--color-text-disabled);padding:12px;text-align:center}.vm-autocomplete__loader{grid-gap:8px;align-items:center;color:var(--color-text-secondary);display:grid;gap:8px;grid-template-columns:14px auto;justify-content:center;padding:12px;pointer-events:none;z-index:2}.vm-autocomplete__loader svg{animation:half-circle-spinner-animation 1s linear infinite,vm-fade .5s ease-in}.vm-autocomplete-info,.vm-autocomplete-message{background-color:var(--color-background-block);border-top:var(--border-divider);padding:12px}.vm-autocomplete-message{color:var(--color-warning);font-size:12px;position:relative}.vm-autocomplete-message:after{background:var(--color-warning);content:"";height:100%;left:0;opacity:.1;position:absolute;top:0;width:100%}.vm-autocomplete-info{max-width:500px;min-width:450px;overflow-wrap:anywhere}.vm-autocomplete-info__type{color:var(--color-text-secondary);margin-bottom:8px}.vm-autocomplete-info__description{line-height:130%}.vm-autocomplete-info__description p{margin:12px 0}.vm-autocomplete-info__description p:last-child{margin:0}.vm-query-editor{position:relative}.vm-query-editor .marker-detection{left:0;pointer-events:none;position:absolute;top:0;visibility:hidden;z-index:-9999}.vm-additional-settings{align-items:center;display:inline-flex;flex-wrap:wrap;gap:12px;justify-content:flex-start}.vm-additional-settings__input{flex-basis:160px;margin-bottom:-6px}.vm-additional-settings_mobile{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:1fr;padding:0 12px;width:100%}.vm-switch{align-items:center;cursor:pointer;display:flex;font-size:12px;justify-content:flex-start;-webkit-user-select:none;user-select:none}.vm-switch_full-width{flex-direction:row-reverse;justify-content:space-between}.vm-switch_full-width .vm-switch__label{margin-left:0}.vm-switch_disabled{cursor:default;opacity:.6}.vm-switch_secondary_active .vm-switch-track{background-color:var(--color-secondary)}.vm-switch_primary_active .vm-switch-track{background-color:var(--color-primary)}.vm-switch_active .vm-switch-track__thumb{left:20px}.vm-switch:hover .vm-switch-track{opacity:.8}.vm-switch-track{align-items:center;background-color:#110f0f66;border-radius:17px;display:flex;height:17px;justify-content:flex-start;padding:3px;position:relative;transition:background-color .2s ease,opacity .3s ease-out;width:34px}.vm-switch-track__thumb{background-color:var(--color-background-block);border-radius:50%;left:3px;min-height:11px;min-width:11px;position:absolute;top:auto;transform-style:preserve-3d;transition:right .2s ease-out,left .2s ease-out}.vm-switch__label{color:var(--color-text-secondary);font-size:inherit;margin-left:8px;transition:color .2s ease;white-space:nowrap}.vm-switch_active .vm-switch__label{color:var(--color-text)}.vm-query-configurator{grid-gap:12px;display:grid;gap:12px}.vm-query-configurator-list{display:grid}.vm-query-configurator-list-row{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr auto auto auto}.vm-query-configurator-list-row_mobile{gap:4px}.vm-query-configurator-list-row_disabled{filter:grayscale(100%);opacity:.5}.vm-query-configurator-list-row__button{align-items:start;display:grid;min-height:36px;width:36px}.vm-query-configurator-settings{align-items:center;display:flex;flex-wrap:wrap;gap:12px;justify-content:space-between}.vm-query-configurator-settings__buttons{grid-gap:8px;display:grid;flex-grow:1;gap:8px;grid-template-columns:repeat(3,auto);justify-content:flex-end}.vm-query-history{max-width:80vw;min-width:500px}.vm-query-history_mobile{max-width:100vw;min-width:100vw}.vm-query-history__tabs{border-bottom:var(--border-divider);margin:-12px -12px 0;padding:0 8px}.vm-query-history__tabs_mobile{margin:-12px -12px 0}.vm-query-history-list{align-items:flex-start;display:grid}.vm-query-history-list__group-title{font-weight:700;margin:0 -12px;padding:12px 12px 8px}.vm-query-history-list__group-title_first{padding-top:12px}.vm-query-history-list__no-data{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;line-height:18px;padding:16px 12px;text-align:center;white-space:pre-line}.vm-query-history-item{grid-gap:8px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:8px;grid-template-columns:1fr auto;margin:0 -12px;padding:8px 12px}.vm-query-history-item__value{font-family:monospace;overflow-wrap:anywhere;white-space:pre-wrap}.vm-query-history-item__buttons{display:flex}.vm-query-history-footer{display:flex;justify-content:flex-end;padding-top:12px}.vm-spinner{align-items:center;animation:vm-fade 2s cubic-bezier(.28,.84,.42,1.1);background-color:#ffffff80;bottom:0;display:flex;flex-direction:column;justify-content:center;left:0;pointer-events:none;position:fixed;right:0;top:0;z-index:99}.vm-spinner_dark{background-color:#110f0f33}.vm-spinner__message{color:rgba(var(--color-text),.9);font-size:16px;line-height:1.3;margin-top:12px;text-align:center;white-space:pre-line}.half-circle-spinner,.half-circle-spinner *{box-sizing:border-box}.half-circle-spinner{border-radius:100%;height:60px;position:relative;width:60px}.half-circle-spinner .circle{border:6px solid #0000;border-radius:100%;content:"";height:100%;position:absolute;width:100%}.half-circle-spinner .circle.circle-1{animation:half-circle-spinner-animation 1s infinite;border-top-color:var(--color-primary)}.half-circle-spinner .circle.circle-2{animation:half-circle-spinner-animation 1s infinite alternate;border-bottom-color:var(--color-primary)}@keyframes half-circle-spinner-animation{0%{transform:rotate(0deg)}to{transform:rotate(1turn)}}@keyframes vm-fade{0%{opacity:0}to{opacity:1}}.vm-custom-panel{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:100%;height:100%}.vm-custom-panel_mobile{gap:8px}.vm-custom-panel__warning{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:space-between}.vm-custom-panel__warning_mobile{grid-template-columns:1fr}.vm-custom-panel-body{position:relative}.vm-custom-panel-body-header{align-items:center;border-bottom:var(--border-divider);display:flex;font-size:12px;justify-content:space-between;margin:-12px -12px 12px;padding:0 12px;position:relative;z-index:1}.vm-custom-panel-body-header__tabs{display:flex;flex-grow:1;justify-content:flex-start}.vm-custom-panel-body-header__graph-controls{align-items:center;display:flex;gap:8px;margin:5px 10px}.vm-custom-panel-body_mobile .vm-custom-panel-body-header{margin:-12px -12px 12px;padding:0 12px}.vm-tracings-view{grid-gap:12px;display:grid;gap:12px}.vm-tracings-view-trace-header{align-items:center;border-bottom:var(--border-divider);display:flex;justify-content:space-between;padding:8px 8px 8px 12px}.vm-tracings-view-trace-header-title{flex-grow:1;font-size:16px;margin-right:8px}.vm-tracings-view-trace-header-title__query{font-weight:700}.vm-tracings-view-trace-header__expand-icon{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;transform:rotate(-90deg);transition:transform .2s ease-in-out;width:20px}.vm-tracings-view-trace-header__expand-icon_open{transform:rotate(0)}.vm-tracings-view-trace__nav{padding:12px 12px 12px 0}.vm-tracings-view-trace__nav_mobile{padding:8px 8px 8px 0}.vm-line-progress{grid-gap:8px;align-items:center;color:var(--color-text-secondary);display:grid;gap:8px;grid-template-columns:1fr auto;justify-content:center}.vm-line-progress-track{background-color:var(--color-hover-black);border-radius:4px;height:20px;width:100%}.vm-line-progress-track__thumb{background-color:#1a90ff;border-radius:4px;height:100%}.vm-nested-nav{border-radius:4px;margin-left:16px;position:relative}.vm-nested-nav_dark .vm-nested-nav-header,.vm-nested-nav_dark .vm-nested-nav-header:after,.vm-nested-nav_dark .vm-nested-nav-header:before{background-color:var(--color-background-body)}.vm-nested-nav_dark .vm-nested-nav-header:hover{box-shadow:0 0 0 1px #ffffff14}.vm-nested-nav_mobile{margin-left:8px}.vm-nested-nav_root>.vm-nested-nav-header:after,.vm-nested-nav_root>.vm-nested-nav-header:before{display:none}.vm-nested-nav-header{grid-gap:8px;background-color:#c9e3f666;border-radius:4px;cursor:pointer;display:grid;gap:8px;grid-template-columns:auto 1fr;margin-bottom:8px;padding:8px;position:relative;transition:box-shadow .2s ease-in-out;z-index:2}.vm-nested-nav-header:after{height:2px;top:calc(50% - 1px);width:12px}.vm-nested-nav-header:after,.vm-nested-nav-header:before{background-color:#c9e3f6;content:"";left:-12px;position:absolute}.vm-nested-nav-header:before{bottom:50%;height:calc(50% + 8px);width:2px}.vm-nested-nav-header:hover{box-shadow:0 0 0 1px #110f0f14}.vm-nested-nav-header__icon{align-items:center;color:var(--color-text-secondary);display:flex;justify-content:center;transform:rotate(-90deg);transition:transform .2s ease-in-out;width:20px}.vm-nested-nav-header__icon_open{transform:rotate(0)}.vm-nested-nav-header__progress{grid-column:2}.vm-nested-nav-header__message{-webkit-line-clamp:3;-webkit-box-orient:vertical;line-clamp:3;display:-moz-box;display:-webkit-box;grid-column:2;line-height:130%;overflow:hidden;position:relative;text-overflow:ellipsis}.vm-nested-nav-header__message_show-full{display:block;overflow:visible}.vm-nested-nav-header__message_duration{color:var(--color-text-secondary)}.vm-nested-nav-header-bottom{align-items:center;display:grid;grid-column:2;grid-template-columns:1fr auto}.vm-nested-nav__childrens>.vm-nested-nav:not(:last-child):before{background-color:#c9e3f6;content:"";height:calc(100% + 32px);left:-12px;position:absolute;top:0;width:2px}.vm-nested-nav__childrens>.vm-nested-nav_dark:not(:last-child):before{background-color:var(--color-background-body)}.vm-nested-nav__childrens>.vm-nested-nav:last-child{margin-bottom:32px}.uplot,.uplot *,.uplot :after,.uplot :before{box-sizing:border-box}.uplot{font-family: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;line-height:1.5;width:-webkit-min-content;width:min-content}.u-title{font-size:18px;font-weight:700;text-align:center}.u-wrap{position:relative;-webkit-user-select:none;user-select:none}.u-over,.u-under{position:absolute}.u-under{overflow:hidden}.uplot canvas{display:block;height:100%;position:relative;width:100%}.u-axis{position:absolute}.u-legend{font-size:14px;margin:auto;text-align:center}.u-inline{display:block}.u-inline *{display:inline-block}.u-inline tr{margin-right:16px}.u-legend th{font-weight:600}.u-legend th>*{display:inline-block;vertical-align:middle}.u-legend .u-marker{background-clip:padding-box!important;height:1em;margin-right:4px;width:1em}.u-inline.u-live th:after{content:":";vertical-align:middle}.u-inline:not(.u-live) .u-value{display:none}.u-series>*{padding:4px}.u-series th{cursor:pointer}.u-legend .u-off>*{opacity:.3}.u-select{background:#00000012}.u-cursor-x,.u-cursor-y,.u-select{pointer-events:none;position:absolute}.u-cursor-x,.u-cursor-y{left:0;top:0;will-change:transform}.u-hz .u-cursor-x,.u-vt .u-cursor-y{border-right:1px dashed #607d8b;height:100%}.u-hz .u-cursor-y,.u-vt .u-cursor-x{border-bottom:1px dashed #607d8b;width:100%}.u-cursor-pt{background-clip:padding-box!important;border:0 solid;border-radius:50%;left:0;pointer-events:none;position:absolute;top:0;will-change:transform}.u-axis.u-off,.u-cursor-pt.u-off,.u-cursor-x.u-off,.u-cursor-y.u-off,.u-select.u-off{display:none}.vm-line-chart{pointer-events:auto}.vm-line-chart_panning{pointer-events:none}.vm-line-chart__u-plot{position:relative}.vm-chart-tooltip{grid-gap:12px;word-wrap:break-word;background:var(--color-background-tooltip);border-radius:8px;color:#fff;display:grid;font-family:monospace;font-size:12px;font-weight:400;gap:12px;line-height:150%;padding:12px;pointer-events:none;position:absolute;-webkit-user-select:text;user-select:text;width:370px;z-index:98}.vm-chart-tooltip_sticky{pointer-events:auto;z-index:99}.vm-chart-tooltip_moved{margin-left:-316.5px;margin-top:-24.5px;position:fixed}.vm-chart-tooltip-header{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:1fr 25px 25px;justify-content:center;min-height:25px}.vm-chart-tooltip-header__title{grid-row:1}.vm-chart-tooltip-header__close{color:#fff;grid-column:3;grid-row:1}.vm-chart-tooltip-header__drag{color:#fff;cursor:move;grid-column:2;grid-row:1}.vm-chart-tooltip-header__date{grid-gap:2px;display:grid;gap:2px;grid-column:1}.vm-chart-tooltip-data{align-items:center;display:flex;gap:8px;justify-content:flex-start}.vm-chart-tooltip-data__marker{border:1px solid #ffffff80;height:14px;width:14px}.vm-chart-tooltip-data__value{font-size:14px;line-height:1}.vm-chart-tooltip-stats{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-start}.vm-chart-tooltip-stats-row{align-items:center;display:grid;justify-content:flex-start}.vm-chart-tooltip-stats-row:not(:last-child){padding-right:8px}.vm-chart-tooltip-stats-row__key{line-height:1;margin-right:4px}.vm-chart-tooltip-stats-row__value{font-weight:700}.vm-chart-tooltip__info{white-space:pre-wrap;word-break:break-all}.vm-legend-item{grid-gap:8px;align-items:start;background-color:var(--color-background-block);cursor:pointer;display:grid;font-size:12px;grid-template-columns:auto auto;justify-content:start;padding:8px;transition:.2s ease}.vm-legend-item:hover{background-color:#0000001a}.vm-legend-item_hide{opacity:.5;text-decoration:line-through}.vm-legend-item_static{cursor:default;grid-template-columns:1fr;margin:0;padding:0}.vm-legend-item_static:hover{background-color:var(--color-background-block)}.vm-legend-item__marker{border-radius:2px;box-sizing:border-box;height:14px;position:relative;transition:.2s ease;width:14px}.vm-legend-item-info{font-weight:400;word-break:break-all}.vm-legend-item-info__label{margin-right:2px}.vm-legend-item-info__free-fields{cursor:pointer;padding:2px}.vm-legend-item-info__free-fields:hover{text-decoration:underline}.vm-legend-item-stats{align-items:center;display:flex;gap:8px;grid-column:2}.vm-legend-item-stats-row{align-items:center;display:flex;justify-content:flex-start}.vm-legend-item-stats-row:not(:last-child){padding-right:12px}.vm-legend-item-stats-row__key{color:var(--color-text-secondary);line-height:1;margin-right:4px}.vm-legend{cursor:default;display:flex;flex-wrap:wrap;position:relative}.vm-legend-group{margin:0 12px 12px 0;min-width:23%;width:100%}.vm-legend-group-title{align-items:center;border-bottom:var(--border-divider);display:flex;margin-bottom:1px;padding:8px}.vm-legend-group-title__count{font-weight:700;margin-right:8px}.vm-legend-heatmap{align-items:center;display:inline-grid;gap:4px;grid-template-columns:auto auto;justify-content:space-between}.vm-legend-heatmap__wrapper{align-items:flex-start;display:flex;flex-wrap:wrap;gap:12px;justify-content:space-between}.vm-legend-heatmap__value{color:var(--color-text);font-size:12px}.vm-legend-heatmap__value:last-child{text-align:right}.vm-legend-heatmap-gradient{align-items:center;display:flex;grid-column:1/-1;height:12px;justify-content:center;position:relative;width:200px}.vm-legend-heatmap-gradient__value{align-items:center;border:2px solid var(--color-text);border-radius:50%;display:flex;height:16px;justify-content:center;position:absolute;top:-2px;transform:translateX(-8px);transition:left .1s ease;width:16px}.vm-legend-heatmap-gradient__value span{background-color:var(--color-background-block);box-shadow:var(--box-shadow);color:var(--color-text);font-size:12px;left:auto;padding:4px 8px;position:absolute;top:18px}.vm-legend-heatmap__labels{word-break:break-all}.vm-graph-view{width:100%}.vm-graph-view_full-width{width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-graph-view_full-width{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-graph-view_full-width_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-legend-anomaly{cursor:default;flex-wrap:wrap;gap:32px;position:relative}.vm-legend-anomaly,.vm-legend-anomaly-item{align-items:center;display:flex;justify-content:center}.vm-legend-anomaly-item{gap:8px}.vm-legend-anomaly-item svg{height:14px;width:30px}.vm-axes-limits{grid-gap:12px;align-items:center;display:grid;gap:12px;max-width:300px}.vm-axes-limits_mobile{gap:12px;max-width:100%;width:100%}.vm-axes-limits_mobile .vm-axes-limits-list__inputs{grid-template-columns:repeat(2,1fr)}.vm-axes-limits-list{grid-gap:12px;align-items:center;display:grid;gap:12px}.vm-axes-limits-list__inputs{grid-gap:8px;display:grid;gap:8px;grid-template-columns:repeat(2,120px)}.vm-graph-settings-popper{grid-gap:12px;display:grid;gap:12px;padding:0 0 12px}.vm-graph-settings-popper__body{grid-gap:16px;display:grid;gap:16px;padding:0 12px}.vm-json-view__copy{display:flex;justify-content:flex-end;position:-webkit-sticky;position:sticky;top:0;z-index:2}.vm-json-view__code{font-size:14px;line-height:1.4;transform:translateY(-32px);white-space:pre-wrap}.vm-table-view{max-width:100%;overflow:auto}.vm-table-view,.vm-table-view_mobile{margin-top:-12px}.vm-table-view table{margin-top:0}.vm-table-settings-popper{display:grid;min-width:250px}.vm-table-settings-popper_mobile .vm-table-settings-popper-list{gap:12px}.vm-table-settings-popper_mobile .vm-table-settings-popper-list:first-child{padding-top:0}.vm-table-settings-popper-list{grid-gap:12px;border-bottom:var(--border-divider);display:grid;gap:12px;max-height:350px;overflow:auto;padding:12px}.vm-table-settings-popper-list_first{padding-top:0}.vm-table-settings-popper-list-header{align-items:center;display:grid;grid-template-columns:1fr auto;justify-content:space-between;min-height:25px}.vm-table-settings-popper-list-header__title{font-weight:700}.vm-table-settings-popper-list__item{font-size:14px}.vm-checkbox{align-items:center;cursor:pointer;display:flex;justify-content:flex-start;-webkit-user-select:none;user-select:none}.vm-checkbox_disabled{cursor:default;opacity:.6}.vm-checkbox_secondary_active .vm-checkbox-track{background-color:var(--color-secondary)}.vm-checkbox_secondary .vm-checkbox-track{border:1px solid var(--color-secondary)}.vm-checkbox_primary_active .vm-checkbox-track{background-color:var(--color-primary)}.vm-checkbox_primary .vm-checkbox-track{border:1px solid var(--color-primary)}.vm-checkbox_active .vm-checkbox-track__thumb{transform:scale(1)}.vm-checkbox:hover .vm-checkbox-track{opacity:.8}.vm-checkbox-track{align-items:center;background-color:#0000;border-radius:4px;display:flex;height:16px;justify-content:center;padding:2px;position:relative;transition:background-color .2s ease,opacity .3s ease-out;width:16px}.vm-checkbox-track__thumb{align-items:center;color:#fff;display:grid;height:12px;justify-content:center;transform:scale(0);transition:transform .1s ease-in-out;width:12px}.vm-checkbox__label{color:inherit;font-size:inherit;margin-left:8px;transition:color .2s ease;white-space:nowrap}.vm-download-report{grid-gap:16px;display:grid;gap:16px;min-width:400px;padding-top:4px}.vm-download-report-settings{grid-gap:12px;display:grid;gap:12px}.vm-download-report-settings textarea{min-height:100px}.vm-download-report__buttons{align-items:center;display:flex;gap:12px;justify-content:flex-end}.vm-download-report-helper{grid-gap:8px;display:grid;gap:8px;padding:12px}.vm-download-report-helper__description{line-height:1.3;max-width:400px;white-space:pre-line}.vm-download-report-helper__description p{margin-bottom:4px}.vm-download-report-helper__buttons{align-items:center;display:flex;gap:8px;justify-content:flex-end}.vm-predefined-panel-header{grid-gap:8px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:8px;grid-template-columns:auto 1fr auto;justify-content:flex-start;padding:8px 16px}.vm-predefined-panel-header__description{line-height:1.3;white-space:pre-wrap}.vm-predefined-panel-header__description ol,.vm-predefined-panel-header__description ul{list-style-position:inside}.vm-predefined-panel-header__description a{color:#c9e3f6;text-decoration:underline}.vm-predefined-panel-header__info{align-items:center;color:var(--color-primary);display:flex;justify-content:center;width:18px}.vm-predefined-panel-body{min-height:500px;padding:8px 16px}@media(max-width:500px){.vm-predefined-panel-body{padding:0}}.vm-predefined-dashboard{background-color:#0000}.vm-predefined-dashboard-header{align-items:center;border-radius:4px;box-shadow:var(--box-shadow);display:grid;font-weight:700;grid-template-columns:1fr auto;justify-content:space-between;line-height:1;overflow:hidden;padding:12px;position:relative;transform-style:preserve-3d;transition:box-shadow .2s ease-in-out}.vm-predefined-dashboard-header_open{border-radius:4px 4px 0 0;box-shadow:none}.vm-predefined-dashboard-header__title{font-size:14px}.vm-predefined-dashboard-header__count{font-size:12px;grid-column:2;margin-right:26px}.vm-predefined-dashboard-panels{grid-gap:12px;display:grid;gap:12px;grid-template-columns:repeat(12,1fr);padding:0}@media(max-width:1000px){.vm-predefined-dashboard-panels{grid-template-columns:1fr}}.vm-predefined-dashboard-panels-panel{border-radius:8px;overflow:hidden;position:relative}.vm-predefined-dashboard-panels-panel:hover .vm-predefined-dashboard-panels-panel__resizer{transform:scale(1)}.vm-predefined-dashboard-panels-panel__resizer{bottom:0;cursor:ew-resize;height:20px;position:absolute;right:0;transform:scale(0);transition:transform .2s ease-in-out;width:20px;z-index:1}.vm-predefined-dashboard-panels-panel__resizer:after{border-bottom:2px solid #110f0f33;border-right:2px solid #110f0f33;bottom:5px;content:"";height:5px;position:absolute;right:5px;width:5px}.vm-predefined-dashboard-panels-panel__alert{grid-column:span 12}.vm-predefined-panels{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:768px){.vm-predefined-panels{padding:12px 0}}@media(max-width:500px){.vm-predefined-panels{padding:8px 0}}.vm-predefined-panels-tabs{align-items:center;display:flex;flex-wrap:wrap;font-size:12px;gap:8px;justify-content:flex-start;overflow:hidden}@media(max-width:768px){.vm-predefined-panels-tabs{padding:0 12px}}.vm-predefined-panels-tabs__tab{background:var(--color-background-block);border:1px solid #110f0f33;border-radius:8px;color:var(--color-text-secondary);cursor:pointer;padding:8px 12px;text-align:center;text-transform:uppercase;transition:background .2s ease-in-out,color .15s ease-in}@media(max-width:500px){.vm-predefined-panels-tabs__tab{flex-grow:1}}.vm-predefined-panels-tabs__tab:hover{color:var(--color-primary)}.vm-predefined-panels-tabs__tab_active{border-color:var(--color-primary);color:var(--color-primary)}.vm-predefined-panels__dashboards{grid-gap:12px;display:grid;gap:12px}.vm-cardinality-configurator{grid-gap:8px;display:grid;gap:8px}.vm-cardinality-configurator-controls{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-start}.vm-cardinality-configurator-controls__query{flex-grow:10}.vm-cardinality-configurator-controls__item{flex-grow:2}.vm-cardinality-configurator-controls__item_limit{flex-grow:1}.vm-cardinality-configurator-controls__item svg{color:var(--color-text-disabled)}.vm-cardinality-configurator-bottom{align-items:center;display:flex;flex-wrap:wrap;gap:12px;justify-content:flex-end;width:100%}.vm-cardinality-configurator-bottom-helpful{align-items:center;display:flex;flex-wrap:wrap;gap:8px 12px;justify-content:flex-end}.vm-cardinality-configurator-bottom-helpful a{color:var(--color-text-secondary)}.vm-cardinality-configurator-bottom__execute{align-items:center;display:flex;gap:8px}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom{justify-content:center}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom-helpful{flex-grow:1;justify-content:center}.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom__execute,.vm-cardinality-configurator_mobile .vm-cardinality-configurator-bottom__execute button:nth-child(3){width:100%}.vm-cardinality-totals{align-content:flex-start;display:inline-flex;flex-grow:1;flex-wrap:wrap;gap:12px;justify-content:flex-start}.vm-cardinality-totals_mobile{gap:12px;justify-content:center}.vm-cardinality-totals-card{grid-gap:8px 4px;align-items:center;display:grid;gap:8px 4px;grid-template-columns:auto 1fr;justify-content:center}.vm-cardinality-totals-card__info-icon{align-items:center;color:var(--color-primary);display:flex;justify-content:center;width:12px}.vm-cardinality-totals-card__title{align-items:center;color:var(--color-text);display:flex;gap:4px;grid-column:1/-1;justify-content:flex-start}.vm-cardinality-totals-card__tooltip{max-width:280px;padding:8px;white-space:normal}.vm-cardinality-totals-card__value{color:var(--color-primary);font-size:18px;font-weight:700;line-height:14px;text-align:center}.vm-metrics-content-header{margin:-12px -12px 0}.vm-metrics-content-header__title{align-items:center;display:flex;justify-content:flex-start}.vm-metrics-content-header__tip{max-width:300px;padding:8px;white-space:normal}.vm-metrics-content-header__tip p{margin-bottom:8px}.vm-metrics-content-header__tip-icon{align-items:center;color:var(--color-primary);display:flex;justify-content:center;margin-right:4px;width:12px}.vm-metrics-content_mobile .vm-metrics-content-header{margin:-12px -12px 0}.vm-metrics-content__table{overflow:auto;padding-top:12px;width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-metrics-content__table{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-metrics-content__table_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-metrics-content__table .vm-table-cell_header{white-space:nowrap}.vm-metrics-content_mobile .vm-metrics-content__table{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-metrics-content__chart{padding-top:12px}.vm-metrics-content-prom-data{align-items:center;display:flex;flex-direction:column;gap:8px;justify-content:center;margin-top:12px;text-align:center;width:100%}.vm-metrics-content-prom-data__icon{align-items:center;color:var(--color-primary);display:flex;height:30px;justify-content:center;margin-bottom:8px;width:30px}.vm-metrics-content-prom-data__title{font-size:16px;font-weight:700}.vm-metrics-content-prom-data__text{line-height:1.3;max-width:700px}.vm-simple-bar-chart{display:grid;grid-template-columns:auto 1fr;height:100%;overflow:hidden;padding-bottom:6px}.vm-simple-bar-chart-y-axis{display:grid;position:relative;transform:translateY(12px)}.vm-simple-bar-chart-y-axis__tick{align-items:center;display:flex;font-size:12px;justify-content:flex-end;line-height:2;padding-right:8px;position:relative;text-align:right;transform-style:preserve-3d;z-index:1}.vm-simple-bar-chart-y-axis__tick:after{border-bottom:var(--border-divider);content:"";height:0;left:100%;position:absolute;top:auto;transform:translateY(-1px) translateZ(-1);width:100vw}.vm-simple-bar-chart-data{align-items:flex-end;display:flex;gap:1%;justify-content:space-between;position:relative}.vm-simple-bar-chart-data-item{align-items:flex-start;background-color:#3b5;display:flex;flex-grow:1;height:calc(100% - 48px);justify-content:center;min-width:1px;transition:background-color .2s ease-in;width:100%}.vm-simple-bar-chart-data-item:hover{background-color:#51d071}.vm-simple-bar-chart-data-item:first-child{background-color:#f79420}.vm-simple-bar-chart-data-item:first-child:hover{background-color:#f9ac51}.vm-cardinality-panel{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-cardinality-panel_mobile,.vm-cardinality-panel_mobile .vm-cardinality-panel-tips{gap:8px}.vm-cardinality-panel-tips{align-content:flex-start;display:inline-flex;flex-grow:1;flex-wrap:wrap;gap:12px;justify-content:flex-start;width:100%}.vm-cardinality-panel-table__header th:first-child{width:60%}.vm-cardinality-panel-table__header th:not(:first-child){width:auto}.vm-cardinality-panel-table__progress{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:minmax(200px,1fr) 70px;justify-content:flex-start}.vm-cardinality-tip{background-color:var(--color-background-block);border-radius:8px;box-shadow:var(--box-shadow);color:var(--color-text-secondary);display:grid;flex-grow:1;grid-template-rows:auto 1fr;overflow:hidden;width:300px}.vm-cardinality-tip-header{align-items:center;border-bottom:var(--border-divider);display:flex;gap:4px;justify-content:center;padding:8px 12px;position:relative}.vm-cardinality-tip-header:after{background:var(--color-warning);content:"";height:100%;left:0;opacity:.1;pointer-events:none;position:absolute;top:0;width:100%}.vm-cardinality-tip-header__tip-icon{align-items:center;color:var(--color-warning);display:flex;justify-content:center;width:12px}.vm-cardinality-tip-header__title{color:var(--color-text);font-weight:700;text-align:center}.vm-cardinality-tip-header__tooltip{font-size:14px;line-height:130%;max-width:280px;padding:8px;white-space:normal}.vm-cardinality-tip__description{line-height:130%;padding:8px 12px}.vm-cardinality-tip__description p{margin-bottom:8px}.vm-cardinality-tip__description p:last-child{margin-bottom:0}.vm-cardinality-tip__description ol,.vm-cardinality-tip__description ul{list-style-position:inside}.vm-cardinality-tip__description ol li,.vm-cardinality-tip__description ul li{margin-bottom:4px}.vm-top-queries-panel-header,.vm-top-queries-panel-header_mobile{margin:-12px -12px 0}.vm-top-queries-panel__table{overflow:auto;padding-top:12px;width:calc(100vw - 48px - var(--scrollbar-width))}@media(max-width:768px){.vm-top-queries-panel__table{width:calc(100vw - 24px - var(--scrollbar-width))}}.vm-top-queries-panel__table_mobile{width:calc(100vw - 24px - var(--scrollbar-width))}.vm-top-queries-panel__table .vm-table-cell_header{white-space:nowrap}.vm-top-queries{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-top-queries_mobile{gap:8px}.vm-top-queries-controls{grid-gap:8px;display:grid;gap:8px}.vm-top-queries-controls-fields{align-items:center;display:flex;flex-wrap:wrap;gap:12px}.vm-top-queries-controls-fields__item{flex-grow:1;min-width:200px}.vm-top-queries-controls-bottom{grid-gap:12px;align-items:flex-end;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:space-between}.vm-top-queries-controls-bottom_mobile{gap:8px;grid-template-columns:1fr}.vm-top-queries-controls-bottom__button{align-items:center;display:flex;justify-content:flex-end}.vm-top-queries-panels{grid-gap:12px;display:grid;gap:12px}.vm-top-queries-panels__table-actions{align-items:center;display:flex;gap:8px;height:100%;justify-content:flex-end;padding:0 8px}.vm-trace-page{display:flex;flex-direction:column;min-height:100%}@media(max-width:768px){.vm-trace-page{padding:12px 0}}.vm-trace-page-header{grid-gap:12px;align-items:start;display:grid;gap:12px;grid-template-columns:1fr auto;margin-bottom:12px}@media(max-width:768px){.vm-trace-page-header{grid-template-columns:1fr;padding:0 12px}}.vm-trace-page-header-errors{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:1fr;justify-content:stretch}@media(max-width:768px){.vm-trace-page-header-errors{grid-row:2}}.vm-trace-page-header-errors-item{align-items:center;display:grid;justify-content:stretch;position:relative}.vm-trace-page-header-errors-item_margin-bottom{margin-bottom:12px}.vm-trace-page-header-errors-item__filename{min-height:20px}.vm-trace-page-header-errors-item__close{position:absolute;right:8px;top:auto;z-index:2}.vm-trace-page-preview{align-items:center;display:flex;flex-direction:column;flex-grow:1;justify-content:center}.vm-trace-page-preview__text{font-size:14px;line-height:1.8;margin-bottom:12px;text-align:center;white-space:pre-line}.vm-trace-page__dropzone{align-items:center;box-shadow:inset var(--color-primary) 0 0 10px;display:flex;height:100%;justify-content:center;left:0;opacity:.5;pointer-events:none;position:fixed;top:0;width:100%;z-index:100}.vm-upload-json-buttons{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr 1fr;justify-content:center}.vm-explore-metrics{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:500px){.vm-explore-metrics{gap:8px}}.vm-explore-metrics-body{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}@media(max-width:500px){.vm-explore-metrics-body{gap:8px}}.vm-explore-metrics-graph,.vm-explore-metrics-graph_mobile{padding:0 12px 12px}.vm-explore-metrics-graph__warning{align-items:center;display:grid;grid-template-columns:1fr auto;justify-content:space-between}.vm-explore-metrics-item-header{grid-gap:12px;align-items:center;border-bottom:var(--border-divider);display:grid;gap:12px;grid-template-columns:auto 1fr auto auto;justify-content:flex-start;padding:12px}.vm-explore-metrics-item-header_mobile{grid-template-columns:1fr auto;padding:8px 12px}.vm-explore-metrics-item-header__index{color:var(--color-text-secondary);font-size:12px}.vm-explore-metrics-item-header__name{flex-grow:1;font-weight:700;line-height:130%;max-width:100%;overflow:hidden;text-overflow:ellipsis}.vm-explore-metrics-item-header-order{align-items:center;display:grid;grid-column:1;grid-template-columns:auto 20px auto;justify-content:flex-start;text-align:center}.vm-explore-metrics-item-header-order__up{transform:rotate(180deg)}.vm-explore-metrics-item-header__rate{grid-column:3}.vm-explore-metrics-item-header__close{align-items:center;display:grid;grid-column:4;grid-row:1}.vm-explore-metrics-item-header code{background-color:var(--color-hover-black);border-radius:6px;font-size:85%;padding:.2em .4em}.vm-explore-metrics-item-header-modal{grid-gap:12px;align-items:flex-start;display:grid;gap:12px}.vm-explore-metrics-item-header-modal-order{align-items:center;display:flex;gap:12px;justify-content:space-between}.vm-explore-metrics-item-header-modal-order p{align-items:center;display:flex}.vm-explore-metrics-item-header-modal-order__index{margin-left:4px}.vm-explore-metrics-item-header-modal__rate{grid-gap:8px;display:grid;gap:8px}.vm-explore-metrics-item-header-modal__rate p{color:var(--color-text-secondary)}.vm-explore-metrics-item{position:relative}.vm-select-input{align-items:center;border:var(--border-divider);border-radius:4px;cursor:pointer;display:flex;min-height:40px;padding:8px 0 8px 12px;position:relative}.vm-select-input-content{align-items:center;display:flex;flex-grow:1;flex-wrap:wrap;gap:8px;justify-content:flex-start}.vm-select-input-content_mobile{flex-wrap:nowrap}.vm-select-input-content__counter{font-size:14px;line-height:14px}.vm-select-input-content__selected{align-items:center;background-color:var(--color-hover-black);border-radius:4px;display:inline-flex;font-size:14px;justify-content:center;line-height:14px;max-width:100%;padding:2px 2px 2px 6px}.vm-select-input-content__selected span{overflow:hidden;text-overflow:ellipsis;width:100%}.vm-select-input-content__selected svg{align-items:center;background-color:#0000;border-radius:4px;display:flex;justify-content:center;margin-left:10px;padding:4px;transition:background-color .2s ease-in-out;width:20px}.vm-select-input-content__selected svg:hover{background-color:#110f0f1a}.vm-select-input input{background-color:#0000;border:none;border-radius:4px;color:var(--color-text);display:inline-block;flex-grow:1;font-size:14px;height:18px;line-height:18px;min-width:100px;padding:0;position:relative;z-index:2}.vm-select-input input:placeholder-shown{width:auto}.vm-select-input__icon{align-items:center;border-right:var(--border-divider);color:var(--color-text-secondary);cursor:pointer;display:inline-flex;justify-content:flex-end;padding:0 8px;transition:transform .2s ease-in,opacity .2s ease-in}.vm-select-input__icon:last-child{border:none}.vm-select-input__icon svg{width:14px}.vm-select-input__icon_open{transform:rotate(180deg)}.vm-select-input__icon:hover{opacity:.7}.vm-select-options{grid-gap:8px;display:grid;font-size:14px;gap:8px;max-height:208px;max-width:300px;overflow:auto;padding:12px}.vm-select-options_mobile{max-height:calc(var(--vh)*100 - 70px);max-width:100%;padding:0 12px 8px}.vm-select_disabled *{cursor:not-allowed}.vm-select_disabled .vm-select-input-content input{color:var(--color-text-disabled)}.vm-explore-metrics-header{align-items:center;display:flex;flex-wrap:wrap;gap:12px 18px;justify-content:flex-start;max-width:calc(100vw - var(--scrollbar-width))}.vm-explore-metrics-header_mobile{align-items:stretch;flex-direction:column}.vm-explore-metrics-header__job{flex-grow:1;min-width:150px}.vm-explore-metrics-header__instance{flex-grow:2;min-width:150px}.vm-explore-metrics-header__size{grid-gap:12px;align-items:center;display:grid;flex-grow:1;gap:12px;grid-template-columns:1fr auto;min-width:150px}.vm-explore-metrics-header-description{grid-gap:8px;align-items:flex-start;display:grid;gap:8px;grid-template-columns:1fr auto}.vm-explore-metrics-header-description button{color:inherit;min-height:29px}.vm-explore-metrics-header-description code{margin:0 3px}.vm-explore-metrics-header-metrics{flex-grow:1;width:100%}.vm-explore-metrics-header__clear-icon{align-items:center;cursor:pointer;display:flex;justify-content:center;padding:2px}.vm-explore-metrics-header__clear-icon:hover{opacity:.7}.vm-preview-icons{grid-gap:12px;align-items:flex-start;display:grid;gap:12px;grid-template-columns:repeat(auto-fill,100px);justify-content:center}.vm-preview-icons-item{grid-gap:8px;align-items:stretch;border:1px solid #0000;border-radius:4px;cursor:pointer;display:grid;gap:8px;grid-template-rows:1fr auto;height:100px;justify-content:center;padding:12px 8px;transition:box-shadow .2s ease-in-out}.vm-preview-icons-item:hover{box-shadow:0 1px 4px #00000029}.vm-preview-icons-item:active .vm-preview-icons-item__svg{transform:scale(.9)}.vm-preview-icons-item__name{font-size:12px;line-height:2;overflow:hidden;text-align:center;text-overflow:ellipsis;white-space:nowrap}.vm-preview-icons-item__svg{align-items:center;display:flex;height:100%;justify-content:center;transition:transform .1s ease-out}.vm-preview-icons-item__svg svg{height:24px;width:auto}.vm-with-template,.vm-with-template-body{grid-gap:12px;display:grid;gap:12px}.vm-with-template-body{align-items:flex-start;width:100%}.vm-with-template-body-top{align-items:center;display:flex;gap:8px;justify-content:flex-end}.vm-with-template-body__expr textarea{min-height:200px}.vm-with-template-body__result textarea{min-height:60px}.vm-with-template-body textarea{font-family:monospace;height:100%;overflow:auto;width:100%}.vm-with-template-tutorial{grid-gap:16px;display:grid;gap:16px}.vm-with-template-tutorial__title{font-size:16px;font-weight:700}.vm-with-template-tutorial-section{grid-gap:12px;display:grid;gap:12px}.vm-with-template-tutorial-section__text{font-size:14px;line-height:130%;max-width:720px}.vm-code-example{background-color:#110f0f0d;border-radius:4px;display:block;overflow:auto;padding:12px;position:relative;white-space:pre-wrap}.vm-code-example__copy{position:absolute;right:10px;top:10px}.vm-relabeling,.vm-relabeling-header{grid-gap:12px;display:grid;gap:12px}.vm-relabeling-header{align-items:flex-start;width:100%}.vm-relabeling-header-configs textarea{min-height:200px}.vm-relabeling-header__labels textarea{min-height:60px}.vm-relabeling-header textarea{font-family:monospace;height:100%;overflow:auto;width:100%}.vm-relabeling-header-bottom{align-items:center;display:flex;gap:12px;justify-content:flex-end}.vm-relabeling-header-bottom a{color:var(--color-text-secondary)}.vm-relabeling-steps,.vm-relabeling-steps-item{grid-gap:12px;display:grid;gap:12px}.vm-relabeling-steps-item{border-bottom:var(--border-divider);padding:0 12px 12px}.vm-relabeling-steps-item:last-child{border-bottom:none;padding-bottom:0}.vm-relabeling-steps-item__row{display:grid;grid-template-columns:100px 1fr}@media(max-width:500px){.vm-relabeling-steps-item__row{gap:4px;grid-template-columns:1fr}}.vm-relabeling-steps-item__row pre{white-space:pre-wrap}.vm-active-queries-header{grid-gap:12px;align-items:center;display:grid;gap:12px;grid-template-columns:1fr auto;justify-content:space-between;margin-bottom:12px}.vm-active-queries-header-controls{grid-gap:8px;display:grid;gap:8px;grid-column:2}.vm-active-queries-header__update-msg{color:var(--color-text-secondary);font-size:12px;white-space:nowrap}.vm-json-form{grid-gap:12px;display:grid;gap:12px;grid-template-rows:auto calc(var(--vh)*70 - 114px) auto;max-height:900px;max-width:1000px;overflow:hidden;width:70vw}.vm-json-form_mobile{grid-template-rows:auto calc(var(--vh)*100 - 236px) auto;min-height:100%;width:100%}.vm-json-form_one-field{grid-template-rows:calc(var(--vh)*70 - 114px) auto}.vm-json-form_one-field_mobile{grid-template-rows:calc(var(--vh)*100 - 184px) auto}.vm-json-form textarea{height:100%;max-height:900px;overflow:auto;width:100%}.vm-json-form-footer{align-items:center;display:flex;gap:8px;justify-content:space-between}@media(max-width:500px){.vm-json-form-footer{flex-direction:column}.vm-json-form-footer button{flex-grow:1}}.vm-json-form-footer__controls{align-items:center;display:flex;flex-grow:1;gap:8px;justify-content:flex-start}@media(max-width:500px){.vm-json-form-footer__controls{grid-template-columns:repeat(2,1fr);justify-content:center;width:100%}}.vm-json-form-footer__controls_right{display:grid;grid-template-columns:repeat(2,90px);justify-content:flex-end}@media(max-width:500px){.vm-json-form-footer__controls_right{grid-template-columns:repeat(2,1fr);justify-content:center;width:100%}}.vm-query-analyzer-view{grid-gap:12px;display:grid;gap:12px;position:relative}.vm-query-analyzer-view-header{align-items:center;border-bottom:var(--border-divider);display:flex;font-size:12px;justify-content:space-between;margin:-12px -12px 12px;padding:0 12px;position:relative;z-index:1}.vm-query-analyzer-view-header__left{align-items:center;display:flex;gap:8px}.vm-query-analyzer-view_mobile .vm-query-analyzer-view-header{margin:-12px -12px 12px;padding:0 12px}.vm-query-analyzer-info-header{display:flex;gap:12px}.vm-query-analyzer-info-header__period{align-items:center;border:var(--border-divider);border-radius:4px;display:flex;gap:8px;padding:6px 12px}.vm-query-analyzer-info-header__period svg{color:var(--color-primary);width:13px}.vm-query-analyzer-info{grid-gap:16px;display:grid;gap:16px;min-width:300px}.vm-query-analyzer-info-type{color:var(--color-text-secondary);font-style:italic;text-align:center}.vm-query-analyzer-info-item{border-bottom:var(--border-divider);display:grid;line-height:130%;padding-bottom:16px}.vm-query-analyzer-info-item__title{font-weight:700}.vm-query-analyzer-info-item__text{white-space:pre-wrap}#root,body,html{background-attachment:fixed;background-color:#fefeff;background-color:var(--color-background-body);background-repeat:no-repeat;color:#110f0f;color:var(--color-text);cursor:default;font-family:system-ui;font-size:14px;margin:0;min-height:100%}body{overflow:auto}*{-webkit-tap-highlight-color:rgba(0,0,0,0);cursor:inherit;font:inherit;touch-action:pan-x pan-y}code{font-family:monospace}b{font-weight:700}input,textarea{cursor:text}input::placeholder,textarea::placeholder{-webkit-user-select:none;user-select:none}input[type=number]::-webkit-inner-spin-button,input[type=number]::-webkit-outer-spin-button{-webkit-appearance:none;margin:0}.vm-snackbar{animation:vm-slide-snackbar .15s cubic-bezier(.28,.84,.42,1.1);bottom:12px;left:12px;position:fixed;z-index:999}.vm-snackbar-content{align-items:center;display:grid;grid-template-columns:1fr auto}.vm-snackbar-content__close{color:inherit;height:24px;opacity:.8;padding:4px;width:24px}.vm-snackbar_mobile{bottom:0;left:0;right:0}@keyframes vm-slide-snackbar{0%{transform:translateY(100%)}to{transform:translateY(0)}}svg{width:100%}*{scrollbar-color:#a09f9f #fff;scrollbar-color:var(--color-text-disabled) var(--color-background-block);scrollbar-width:thin}::-webkit-scrollbar{width:12px}::-webkit-scrollbar-track{background:#fff;background:var(--color-background-block)}::-webkit-scrollbar-thumb{background-color:#a09f9f;background-color:var(--color-text-disabled);border:3px solid #fff;border:3px solid var(--color-background-block);border-radius:20px}a,abbr,acronym,address,applet,article,aside,audio,big,body,canvas,caption,center,cite,code,del,details,dfn,div,em,embed,fieldset,figcaption,figure,footer,form,h1,h2,h3,h4,h5,h6,header,hgroup,html,iframe,img,ins,kbd,label,legend,li,mark,menu,nav,object,ol,output,p,pre,q,ruby,s,samp,section,small,span,strike,strong,sub,summary,sup,table,tbody,td,tfoot,th,thead,time,tr,tt,u,ul,var,video{border:0;margin:0;padding:0;vertical-align:initial}h1,h2,h3,h4,h5,h6{font-weight:400}article,aside,details,figcaption,figure,footer,header,hgroup,menu,nav,section{display:block}body{line-height:1}q:after,q:before{content:""}table{border-collapse:collapse;border-spacing:0}input::placeholder{opacity:1;transition:opacity .3s ease}input:focus::placeholder{opacity:0;transition:opacity .3s ease}*{box-sizing:border-box;outline:none}button{background:none;border:none;border-radius:0;padding:0}strong{letter-spacing:1px}input[type=file]{cursor:pointer;font-size:0;height:100%;left:0;opacity:0;position:absolute;top:0;width:100%}input[type=file]:disabled{cursor:not-allowed}a{color:inherit;text-decoration:inherit}input,textarea{-webkit-text-fill-color:inherit;appearance:none;-webkit-appearance:none}input:disabled,textarea:disabled{opacity:1!important}input:placeholder-shown,textarea:placeholder-shown{width:100%}input:-webkit-autofill,input:-webkit-autofill:active,input:-webkit-autofill:focus,input:-webkit-autofill:hover{-webkit-box-shadow:inset 0 0 0 0 #fff!important;width:100%;z-index:2}.vm-header-button{border:1px solid #110f0f33}.vm-list-item{background-color:#0000;cursor:pointer;padding:12px;transition:background-color .2s ease}.vm-list-item_mobile{padding:12px}.vm-list-item:hover,.vm-list-item_active{background-color:#0000000f;background-color:var(--color-hover-black)}.vm-list-item_multiselect{grid-gap:8px;align-items:center;display:grid;gap:8px;grid-template-columns:10px 1fr;justify-content:flex-start}.vm-list-item_multiselect svg{animation:vm-scale .15s cubic-bezier(.28,.84,.42,1)}.vm-list-item_multiselect span{grid-column:2}.vm-list-item_multiselect_selected{color:#3f51b5;color:var(--color-primary)}.vm-list-item_with-icon{grid-gap:4px;display:grid;gap:4px;grid-template-columns:14px 1fr}.vm-list-item_with-icon,.vm-mobile-option{align-items:center;justify-content:flex-start}.vm-mobile-option{display:flex;gap:8px;padding:6px 0;-webkit-user-select:none;user-select:none;width:100%}.vm-mobile-option__arrow,.vm-mobile-option__icon{align-items:center;display:flex;justify-content:center}.vm-mobile-option__icon{color:#3f51b5;color:var(--color-primary);height:22px;width:22px}.vm-mobile-option__arrow{color:#3f51b5;color:var(--color-primary);height:14px;transform:rotate(-90deg);width:14px}.vm-mobile-option-text{grid-gap:2px;align-items:center;display:grid;flex-grow:1;gap:2px}.vm-mobile-option-text__label{font-weight:700}.vm-mobile-option-text__value{color:#706f6f;color:var(--color-text-secondary);font-size:12px}.vm-block{background-color:#fff;background-color:var(--color-background-block);border-radius:8px;box-shadow:1px 2px 6px #00000014;box-shadow:var(--box-shadow);padding:12px}.vm-block_mobile{border-radius:0;padding:12px}.vm-block_empty-padding{padding:0}.vm-section-header{align-items:center;border-bottom:1px solid #00000026;border-bottom:var(--border-divider);border-radius:8px 8px 0 0;display:grid;grid-template-columns:1fr auto;justify-content:center;padding:0 12px}.vm-section-header__title{font-size:14px;font-weight:700}.vm-section-header__title_mobile{-webkit-line-clamp:2;line-clamp:2;-webkit-box-orient:vertical;display:-webkit-box;overflow:hidden;text-overflow:ellipsis}.vm-section-header__tabs{align-items:center;display:flex;font-size:12px;justify-content:flex-start}.vm-table{border-collapse:initial;border-spacing:0;margin-top:-12px;width:100%}.vm-table,.vm-table__row{background-color:#fff;background-color:var(--color-background-block)}.vm-table__row{transition:background-color .2s ease}.vm-table__row:hover:not(.vm-table__row_header){background-color:#0000000f;background-color:var(--color-hover-black)}.vm-table__row_header{position:relative;z-index:2}.vm-table__row_selected{background-color:#1a90ff0d}.vm-table-cell{border-bottom:1px solid #00000026;border-bottom:var(--border-divider);line-height:1.5;overflow-wrap:anywhere;padding:4px;vertical-align:top}.vm-table-cell__content{align-items:center;display:flex;justify-content:flex-start}.vm-table-cell_sort{cursor:pointer}.vm-table-cell_sort:hover{background-color:#0000000f;background-color:var(--color-hover-black)}.vm-table-cell_header{font-weight:700;overflow-wrap:normal;text-align:left}.vm-table-cell_gray{color:#110f0f;color:var(--color-text);opacity:.4}.vm-table-cell_right{text-align:right}.vm-table-cell_right .vm-table-cell__content{justify-content:flex-end}.vm-table-cell_no-wrap{white-space:nowrap}.vm-table-cell_no-padding{padding:0}.vm-table-cell_pre{white-space:pre-wrap}.vm-table-cell_logs-time{overflow-wrap:normal;white-space:pre}.vm-table-cell_logs{font-family:monospace;line-height:1.2}.vm-table__sort-icon{align-items:center;display:flex;justify-content:center;margin:0 8px;opacity:.4;transition:opacity .2s ease,transform .2s ease-in-out;width:15px}.vm-table__sort-icon_active{opacity:1}.vm-table__sort-icon_desc{transform:rotate(180deg)}.vm-link{cursor:pointer;transition:color .2s ease}.vm-link_colored{color:#3f51b5;color:var(--color-primary)}.vm-link_underlined{text-decoration:underline}.vm-link_with-icon{grid-gap:6px;align-items:center;display:grid;gap:6px;grid-template-columns:14px auto;justify-content:center}.vm-link:hover{color:#3f51b5;color:var(--color-primary);text-decoration:underline}.vm-dynamic-number{color:#a09f9f;color:var(--color-text-disabled);font-size:12px}.vm-dynamic-number_positive{color:#4caf50;color:var(--color-success)}.vm-dynamic-number_negative{color:#fd080e;color:var(--color-error)}.vm-dynamic-number_down:before{content:"↓"}.vm-dynamic-number_up:before{content:"↑"}:root{--color-primary:#3f51b5;--color-secondary:#e91e63;--color-error:#fd080e;--color-warning:#ff8308;--color-info:#03a9f4;--color-success:#4caf50;--color-primary-text:#fff;--color-secondary-text:#fff;--color-error-text:#fff;--color-warning-text:#fff;--color-info-text:#fff;--color-success-text:#fff;--color-background-body:#fefeff;--color-background-block:#fff;--color-background-tooltip:#505050e6;--color-text:#110f0f;--color-text-secondary:#706f6f;--color-text-disabled:#a09f9f;--box-shadow:#00000014 1px 2px 6px;--box-shadow-popper:#0000001a 0px 2px 8px 0px;--border-divider:1px solid #00000026;--color-hover-black:#0000000f} \ No newline at end of file diff --git a/app/vmselect/vmui/static/js/main.202937c2.js b/app/vmselect/vmui/static/js/main.202937c2.js deleted file mode 100644 index 67855af6e..000000000 --- a/app/vmselect/vmui/static/js/main.202937c2.js +++ /dev/null @@ -1,2 +0,0 @@ -/*! For license information please see main.202937c2.js.LICENSE.txt */ -(()=>{var e={61:(e,t,n)=>{"use strict";var r=n(375),a=n(629),i=a(r("String.prototype.indexOf"));e.exports=function(e,t){var n=r(e,!!t);return"function"===typeof n&&i(e,".prototype.")>-1?a(n):n}},629:(e,t,n)=>{"use strict";var r=n(989),a=n(375),i=n(259),o=n(277),l=a("%Function.prototype.apply%"),s=a("%Function.prototype.call%"),c=a("%Reflect.apply%",!0)||r.call(s,l),u=n(709),d=a("%Math.max%");e.exports=function(e){if("function"!==typeof e)throw new o("a function is required");var t=c(r,s,arguments);return i(t,1+d(0,e.length-(arguments.length-1)),!0)};var h=function(){return c(r,l,arguments)};u?u(e.exports,"apply",{value:h}):e.exports.apply=h},159:function(e){e.exports=function(){"use strict";var e=1e3,t=6e4,n=36e5,r="millisecond",a="second",i="minute",o="hour",l="day",s="week",c="month",u="quarter",d="year",h="date",m="Invalid Date",p=/^(\d{4})[-/]?(\d{1,2})?[-/]?(\d{0,2})[Tt\s]*(\d{1,2})?:?(\d{1,2})?:?(\d{1,2})?[.:]?(\d+)?$/,f=/\[([^\]]+)]|Y{1,4}|M{1,4}|D{1,2}|d{1,4}|H{1,2}|h{1,2}|a|A|m{1,2}|s{1,2}|Z{1,2}|SSS/g,v={name:"en",weekdays:"Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),months:"January_February_March_April_May_June_July_August_September_October_November_December".split("_"),ordinal:function(e){var t=["th","st","nd","rd"],n=e%100;return"["+e+(t[(n-20)%10]||t[n]||t[0])+"]"}},g=function(e,t,n){var r=String(e);return!r||r.length>=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),a=n%60;return(t<=0?"+":"-")+g(r,2,"0")+":"+g(a,2,"0")},m:function e(t,n){if(t.date()1)return e(o[0])}else{var l=t.name;b[l]=t,a=l}return!r&&a&&(_=a),a||!r&&_},S=function(e,t){if(k(e))return e.clone();var n="object"==typeof t?t:{};return n.date=e,n.args=arguments,new E(n)},C=y;C.l=x,C.i=k,C.w=function(e,t){return S(e,{locale:t.$L,utc:t.$u,x:t.$x,$offset:t.$offset})};var E=function(){function v(e){this.$L=x(e.locale,null,!0),this.parse(e),this.$x=this.$x||e.x||{},this[w]=!0}var g=v.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(C.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 a=r[2]-1||0,i=(r[7]||"0").substring(0,3);return n?new Date(Date.UTC(r[1],a,r[3]||1,r[4]||0,r[5]||0,r[6]||0,i)):new Date(r[1],a,r[3]||1,r[4]||0,r[5]||0,r[6]||0,i)}}return new Date(t)}(e),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 C},g.isValid=function(){return!(this.$d.toString()===m)},g.isSame=function(e,t){var n=S(e);return this.startOf(t)<=n&&n<=this.endOf(t)},g.isAfter=function(e,t){return S(e)=0&&(i[d]=parseInt(u,10))}var h=i[3],m=24===h?0:h,p=i[0]+"-"+i[1]+"-"+i[2]+" "+m+":"+i[4]+":"+i[5]+":000",f=+t;return(a.utc(p).valueOf()-(f-=f%1e3))/6e4},s=r.prototype;s.tz=function(e,t){void 0===e&&(e=i);var n=this.utcOffset(),r=this.toDate(),o=r.toLocaleString("en-US",{timeZone:e}),l=Math.round((r-new Date(o))/1e3/60),s=a(o,{locale:this.$L}).$set("millisecond",this.$ms).utcOffset(15*-Math.round(r.getTimezoneOffset()/15)-l,!0);if(t){var c=s.utcOffset();s=s.add(n-c,"minute")}return s.$x.$timezone=e,s},s.offsetName=function(e){var t=this.$x.$timezone||a.tz.guess(),n=o(this.valueOf(),t,{timeZoneName:e}).find((function(e){return"timezonename"===e.type.toLowerCase()}));return n&&n.value};var c=s.startOf;s.startOf=function(e,t){if(!this.$x||!this.$x.$timezone)return c.call(this,e,t);var n=a(this.format("YYYY-MM-DD HH:mm:ss:SSS"),{locale:this.$L});return c.call(n,e,t).tz(this.$x.$timezone,!0)},a.tz=function(e,t,n){var r=n&&t,o=n||t||i,s=l(+a(),o);if("string"!=typeof e)return a(e).tz(o);var c=function(e,t,n){var r=e-60*t*1e3,a=l(r,n);if(t===a)return[r,t];var i=l(r-=60*(a-t)*1e3,n);return a===i?[r,a]:[e-60*Math.min(a,i)*1e3,Math.max(a,i)]}(a.utc(e,r).valueOf(),s,o),u=c[0],d=c[1],h=a(u).utcOffset(d);return h.$x.$timezone=o,h},a.tz.guess=function(){return Intl.DateTimeFormat().resolvedOptions().timeZone},a.tz.setDefault=function(e){i=e}}}()},220:function(e){e.exports=function(){"use strict";var e="minute",t=/[+-]\d\d(?::?\d\d)?/g,n=/([+-]|\d\d)/g;return function(r,a,i){var o=a.prototype;i.utc=function(e){return new a({date:e,utc:!0,args:arguments})},o.utc=function(t){var n=i(this.toDate(),{locale:this.$L,utc:!0});return t?n.add(this.utcOffset(),e):n},o.local=function(){return i(this.toDate(),{locale:this.$L,utc:!1})};var l=o.parse;o.parse=function(e){e.utc&&(this.$u=!0),this.$utils().u(e.$offset)||(this.$offset=e.$offset),l.call(this,e)};var s=o.init;o.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 s.call(this)};var c=o.utcOffset;o.utcOffset=function(r,a){var i=this.$utils().u;if(i(r))return this.$u?0:i(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 a=(""+r[0]).match(n)||["-",0,0],i=a[0],o=60*+a[1]+ +a[2];return 0===o?0:"+"===i?o:-o}(r),null===r))return this;var o=Math.abs(r)<=16?60*r:r,l=this;if(a)return l.$offset=o,l.$u=0===r,l;if(0!==r){var s=this.$u?this.toDate().getTimezoneOffset():-1*this.utcOffset();(l=this.local().add(o+s,e)).$offset=o,l.$x.$localOffset=s}else l=this.utc();return l};var u=o.format;o.format=function(e){var t=e||(this.$u?"YYYY-MM-DDTHH:mm:ss[Z]":"");return u.call(this,t)},o.valueOf=function(){var e=this.$utils().u(this.$offset)?0:this.$offset+(this.$x.$localOffset||this.$d.getTimezoneOffset());return this.$d.valueOf()-6e4*e},o.isUTC=function(){return!!this.$u},o.toISOString=function(){return this.toDate().toISOString()},o.toString=function(){return this.toDate().toUTCString()};var d=o.toDate;o.toDate=function(e){return"s"===e&&this.$offset?i(this.format("YYYY-MM-DD HH:mm:ss:SSS")).toDate():d.call(this)};var h=o.diff;o.diff=function(e,t,n){if(e&&this.$u===e.$u)return h.call(this,e,t,n);var r=this.local(),a=i(e).local();return h.call(r,a,t,n)}}}()},411:(e,t,n)=>{"use strict";var r=n(709),a=n(430),i=n(277),o=n(553);e.exports=function(e,t,n){if(!e||"object"!==typeof e&&"function"!==typeof e)throw new i("`obj` must be an object or a function`");if("string"!==typeof t&&"symbol"!==typeof t)throw new i("`property` must be a string or a symbol`");if(arguments.length>3&&"boolean"!==typeof arguments[3]&&null!==arguments[3])throw new i("`nonEnumerable`, if provided, must be a boolean or null");if(arguments.length>4&&"boolean"!==typeof arguments[4]&&null!==arguments[4])throw new i("`nonWritable`, if provided, must be a boolean or null");if(arguments.length>5&&"boolean"!==typeof arguments[5]&&null!==arguments[5])throw new i("`nonConfigurable`, if provided, must be a boolean or null");if(arguments.length>6&&"boolean"!==typeof arguments[6])throw new i("`loose`, if provided, must be a boolean");var l=arguments.length>3?arguments[3]:null,s=arguments.length>4?arguments[4]:null,c=arguments.length>5?arguments[5]:null,u=arguments.length>6&&arguments[6],d=!!o&&o(e,t);if(r)r(e,t,{configurable:null===c&&d?d.configurable:!c,enumerable:null===l&&d?d.enumerable:!l,value:n,writable:null===s&&d?d.writable:!s});else{if(!u&&(l||s||c))throw new a("This environment does not support defining a property as non-configurable, non-writable, or non-enumerable.");e[t]=n}}},709:(e,t,n)=>{"use strict";var r=n(375)("%Object.defineProperty%",!0)||!1;if(r)try{r({},"a",{value:1})}catch(a){r=!1}e.exports=r},123:e=>{"use strict";e.exports=EvalError},953:e=>{"use strict";e.exports=Error},780:e=>{"use strict";e.exports=RangeError},768:e=>{"use strict";e.exports=ReferenceError},430:e=>{"use strict";e.exports=SyntaxError},277:e=>{"use strict";e.exports=TypeError},619:e=>{"use strict";e.exports=URIError},307:e=>{"use strict";var t=Object.prototype.toString,n=Math.max,r=function(e,t){for(var n=[],r=0;r{"use strict";var r=n(307);e.exports=Function.prototype.bind||r},375:(e,t,n)=>{"use strict";var r,a=n(953),i=n(123),o=n(780),l=n(768),s=n(430),c=n(277),u=n(619),d=Function,h=function(e){try{return d('"use strict"; return ('+e+").constructor;")()}catch(t){}},m=Object.getOwnPropertyDescriptor;if(m)try{m({},"")}catch(R){m=null}var p=function(){throw new c},f=m?function(){try{return p}catch(e){try{return m(arguments,"callee").get}catch(t){return p}}}():p,v=n(757)(),g=n(442)(),y=Object.getPrototypeOf||(g?function(e){return e.__proto__}:null),_={},b="undefined"!==typeof Uint8Array&&y?y(Uint8Array):r,w={__proto__:null,"%AggregateError%":"undefined"===typeof AggregateError?r:AggregateError,"%Array%":Array,"%ArrayBuffer%":"undefined"===typeof ArrayBuffer?r:ArrayBuffer,"%ArrayIteratorPrototype%":v&&y?y([][Symbol.iterator]()):r,"%AsyncFromSyncIteratorPrototype%":r,"%AsyncFunction%":_,"%AsyncGenerator%":_,"%AsyncGeneratorFunction%":_,"%AsyncIteratorPrototype%":_,"%Atomics%":"undefined"===typeof Atomics?r:Atomics,"%BigInt%":"undefined"===typeof BigInt?r:BigInt,"%BigInt64Array%":"undefined"===typeof BigInt64Array?r:BigInt64Array,"%BigUint64Array%":"undefined"===typeof BigUint64Array?r:BigUint64Array,"%Boolean%":Boolean,"%DataView%":"undefined"===typeof DataView?r:DataView,"%Date%":Date,"%decodeURI%":decodeURI,"%decodeURIComponent%":decodeURIComponent,"%encodeURI%":encodeURI,"%encodeURIComponent%":encodeURIComponent,"%Error%":a,"%eval%":eval,"%EvalError%":i,"%Float32Array%":"undefined"===typeof Float32Array?r:Float32Array,"%Float64Array%":"undefined"===typeof Float64Array?r:Float64Array,"%FinalizationRegistry%":"undefined"===typeof FinalizationRegistry?r:FinalizationRegistry,"%Function%":d,"%GeneratorFunction%":_,"%Int8Array%":"undefined"===typeof Int8Array?r:Int8Array,"%Int16Array%":"undefined"===typeof Int16Array?r:Int16Array,"%Int32Array%":"undefined"===typeof Int32Array?r:Int32Array,"%isFinite%":isFinite,"%isNaN%":isNaN,"%IteratorPrototype%":v&&y?y(y([][Symbol.iterator]())):r,"%JSON%":"object"===typeof JSON?JSON:r,"%Map%":"undefined"===typeof Map?r:Map,"%MapIteratorPrototype%":"undefined"!==typeof Map&&v&&y?y((new Map)[Symbol.iterator]()):r,"%Math%":Math,"%Number%":Number,"%Object%":Object,"%parseFloat%":parseFloat,"%parseInt%":parseInt,"%Promise%":"undefined"===typeof Promise?r:Promise,"%Proxy%":"undefined"===typeof Proxy?r:Proxy,"%RangeError%":o,"%ReferenceError%":l,"%Reflect%":"undefined"===typeof Reflect?r:Reflect,"%RegExp%":RegExp,"%Set%":"undefined"===typeof Set?r:Set,"%SetIteratorPrototype%":"undefined"!==typeof Set&&v&&y?y((new Set)[Symbol.iterator]()):r,"%SharedArrayBuffer%":"undefined"===typeof SharedArrayBuffer?r:SharedArrayBuffer,"%String%":String,"%StringIteratorPrototype%":v&&y?y(""[Symbol.iterator]()):r,"%Symbol%":v?Symbol:r,"%SyntaxError%":s,"%ThrowTypeError%":f,"%TypedArray%":b,"%TypeError%":c,"%Uint8Array%":"undefined"===typeof Uint8Array?r:Uint8Array,"%Uint8ClampedArray%":"undefined"===typeof Uint8ClampedArray?r:Uint8ClampedArray,"%Uint16Array%":"undefined"===typeof Uint16Array?r:Uint16Array,"%Uint32Array%":"undefined"===typeof Uint32Array?r:Uint32Array,"%URIError%":u,"%WeakMap%":"undefined"===typeof WeakMap?r:WeakMap,"%WeakRef%":"undefined"===typeof WeakRef?r:WeakRef,"%WeakSet%":"undefined"===typeof WeakSet?r:WeakSet};if(y)try{null.error}catch(R){var k=y(y(R));w["%Error.prototype%"]=k}var x=function e(t){var n;if("%AsyncFunction%"===t)n=h("async function () {}");else if("%GeneratorFunction%"===t)n=h("function* () {}");else if("%AsyncGeneratorFunction%"===t)n=h("async function* () {}");else if("%AsyncGenerator%"===t){var r=e("%AsyncGeneratorFunction%");r&&(n=r.prototype)}else if("%AsyncIteratorPrototype%"===t){var a=e("%AsyncGenerator%");a&&y&&(n=y(a.prototype))}return w[t]=n,n},S={__proto__:null,"%ArrayBufferPrototype%":["ArrayBuffer","prototype"],"%ArrayPrototype%":["Array","prototype"],"%ArrayProto_entries%":["Array","prototype","entries"],"%ArrayProto_forEach%":["Array","prototype","forEach"],"%ArrayProto_keys%":["Array","prototype","keys"],"%ArrayProto_values%":["Array","prototype","values"],"%AsyncFunctionPrototype%":["AsyncFunction","prototype"],"%AsyncGenerator%":["AsyncGeneratorFunction","prototype"],"%AsyncGeneratorPrototype%":["AsyncGeneratorFunction","prototype","prototype"],"%BooleanPrototype%":["Boolean","prototype"],"%DataViewPrototype%":["DataView","prototype"],"%DatePrototype%":["Date","prototype"],"%ErrorPrototype%":["Error","prototype"],"%EvalErrorPrototype%":["EvalError","prototype"],"%Float32ArrayPrototype%":["Float32Array","prototype"],"%Float64ArrayPrototype%":["Float64Array","prototype"],"%FunctionPrototype%":["Function","prototype"],"%Generator%":["GeneratorFunction","prototype"],"%GeneratorPrototype%":["GeneratorFunction","prototype","prototype"],"%Int8ArrayPrototype%":["Int8Array","prototype"],"%Int16ArrayPrototype%":["Int16Array","prototype"],"%Int32ArrayPrototype%":["Int32Array","prototype"],"%JSONParse%":["JSON","parse"],"%JSONStringify%":["JSON","stringify"],"%MapPrototype%":["Map","prototype"],"%NumberPrototype%":["Number","prototype"],"%ObjectPrototype%":["Object","prototype"],"%ObjProto_toString%":["Object","prototype","toString"],"%ObjProto_valueOf%":["Object","prototype","valueOf"],"%PromisePrototype%":["Promise","prototype"],"%PromiseProto_then%":["Promise","prototype","then"],"%Promise_all%":["Promise","all"],"%Promise_reject%":["Promise","reject"],"%Promise_resolve%":["Promise","resolve"],"%RangeErrorPrototype%":["RangeError","prototype"],"%ReferenceErrorPrototype%":["ReferenceError","prototype"],"%RegExpPrototype%":["RegExp","prototype"],"%SetPrototype%":["Set","prototype"],"%SharedArrayBufferPrototype%":["SharedArrayBuffer","prototype"],"%StringPrototype%":["String","prototype"],"%SymbolPrototype%":["Symbol","prototype"],"%SyntaxErrorPrototype%":["SyntaxError","prototype"],"%TypedArrayPrototype%":["TypedArray","prototype"],"%TypeErrorPrototype%":["TypeError","prototype"],"%Uint8ArrayPrototype%":["Uint8Array","prototype"],"%Uint8ClampedArrayPrototype%":["Uint8ClampedArray","prototype"],"%Uint16ArrayPrototype%":["Uint16Array","prototype"],"%Uint32ArrayPrototype%":["Uint32Array","prototype"],"%URIErrorPrototype%":["URIError","prototype"],"%WeakMapPrototype%":["WeakMap","prototype"],"%WeakSetPrototype%":["WeakSet","prototype"]},C=n(989),E=n(155),N=C.call(Function.call,Array.prototype.concat),A=C.call(Function.apply,Array.prototype.splice),M=C.call(Function.call,String.prototype.replace),T=C.call(Function.call,String.prototype.slice),L=C.call(Function.call,RegExp.prototype.exec),O=/[^%.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|%$))/g,P=/\\(\\)?/g,I=function(e,t){var n,r=e;if(E(S,r)&&(r="%"+(n=S[r])[0]+"%"),E(w,r)){var a=w[r];if(a===_&&(a=x(r)),"undefined"===typeof a&&!t)throw new c("intrinsic "+e+" exists, but is not available. Please file an issue!");return{alias:n,name:r,value:a}}throw new s("intrinsic "+e+" does not exist!")};e.exports=function(e,t){if("string"!==typeof e||0===e.length)throw new c("intrinsic name must be a non-empty string");if(arguments.length>1&&"boolean"!==typeof t)throw new c('"allowMissing" argument must be a boolean');if(null===L(/^%?[^%]*%?$/,e))throw new s("`%` may not be present anywhere but at the beginning and end of the intrinsic name");var n=function(e){var t=T(e,0,1),n=T(e,-1);if("%"===t&&"%"!==n)throw new s("invalid intrinsic syntax, expected closing `%`");if("%"===n&&"%"!==t)throw new s("invalid intrinsic syntax, expected opening `%`");var r=[];return M(e,O,(function(e,t,n,a){r[r.length]=n?M(a,P,"$1"):t||e})),r}(e),r=n.length>0?n[0]:"",a=I("%"+r+"%",t),i=a.name,o=a.value,l=!1,u=a.alias;u&&(r=u[0],A(n,N([0,1],u)));for(var d=1,h=!0;d=n.length){var g=m(o,p);o=(h=!!g)&&"get"in g&&!("originalValue"in g.get)?g.get:o[p]}else h=E(o,p),o=o[p];h&&!l&&(w[i]=o)}}return o}},553:(e,t,n)=>{"use strict";var r=n(375)("%Object.getOwnPropertyDescriptor%",!0);if(r)try{r([],"length")}catch(a){r=null}e.exports=r},734:(e,t,n)=>{"use strict";var r=n(709),a=function(){return!!r};a.hasArrayLengthDefineBug=function(){if(!r)return null;try{return 1!==r([],"length",{value:1}).length}catch(e){return!0}},e.exports=a},442:e=>{"use strict";var t={foo:{}},n=Object;e.exports=function(){return{__proto__:t}.foo===t.foo&&!({__proto__:null}instanceof n)}},757:(e,t,n)=>{"use strict";var r="undefined"!==typeof Symbol&&Symbol,a=n(175);e.exports=function(){return"function"===typeof r&&("function"===typeof Symbol&&("symbol"===typeof r("foo")&&("symbol"===typeof Symbol("bar")&&a())))}},175: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 a=Object.getOwnPropertyDescriptor(e,t);if(42!==a.value||!0!==a.enumerable)return!1}return!0}},155:(e,t,n)=>{"use strict";var r=Function.prototype.call,a=Object.prototype.hasOwnProperty,i=n(989);e.exports=i.call(r,a)},267:(e,t,n)=>{var r=NaN,a="[object Symbol]",i=/^\s+|\s+$/g,o=/^[-+]0x[0-9a-f]+$/i,l=/^0b[01]+$/i,s=/^0o[0-7]+$/i,c=parseInt,u="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,d="object"==typeof self&&self&&self.Object===Object&&self,h=u||d||Function("return this")(),m=Object.prototype.toString,p=Math.max,f=Math.min,v=function(){return h.Date.now()};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)&&m.call(e)==a}(e))return r;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=l.test(e);return n||s.test(e)?c(e.slice(2),n?2:8):o.test(e)?r:+e}e.exports=function(e,t,n){var r,a,i,o,l,s,c=0,u=!1,d=!1,h=!0;if("function"!=typeof e)throw new TypeError("Expected a function");function m(t){var n=r,i=a;return r=a=void 0,c=t,o=e.apply(i,n)}function _(e){var n=e-s;return void 0===s||n>=t||n<0||d&&e-c>=i}function b(){var e=v();if(_(e))return w(e);l=setTimeout(b,function(e){var n=t-(e-s);return d?f(n,i-(e-c)):n}(e))}function w(e){return l=void 0,h&&r?m(e):(r=a=void 0,o)}function k(){var e=v(),n=_(e);if(r=arguments,a=this,s=e,n){if(void 0===l)return function(e){return c=e,l=setTimeout(b,t),u?m(e):o}(s);if(d)return l=setTimeout(b,t),m(s)}return void 0===l&&(l=setTimeout(b,t)),o}return t=y(t)||0,g(n)&&(u=!!n.leading,i=(d="maxWait"in n)?p(y(n.maxWait)||0,t):i,h="trailing"in n?!!n.trailing:h),k.cancel=function(){void 0!==l&&clearTimeout(l),c=0,r=s=a=l=void 0},k.flush=function(){return void 0===l?o:w(v())},k}},424:(e,t,n)=>{var r="__lodash_hash_undefined__",a=1/0,i="[object Function]",o="[object GeneratorFunction]",l="[object Symbol]",s=/\.|\[(?:[^[\]]*|(["'])(?:(?!\1)[^\\]|\\.)*?\1)\]/,c=/^\w*$/,u=/^\./,d=/[^.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|$))/g,h=/\\(\\)?/g,m=/^\[object .+?Constructor\]$/,p="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,f="object"==typeof self&&self&&self.Object===Object&&self,v=p||f||Function("return this")();var g=Array.prototype,y=Function.prototype,_=Object.prototype,b=v["__core-js_shared__"],w=function(){var e=/[^.]+$/.exec(b&&b.keys&&b.keys.IE_PROTO||"");return e?"Symbol(src)_1."+e:""}(),k=y.toString,x=_.hasOwnProperty,S=_.toString,C=RegExp("^"+k.call(x).replace(/[\\^$.*+?()[\]{}|]/g,"\\$&").replace(/hasOwnProperty|(function).*?(?=\\\()| for .+?(?=\\\])/g,"$1.*?")+"$"),E=v.Symbol,N=g.splice,A=j(v,"Map"),M=j(Object,"create"),T=E?E.prototype:void 0,L=T?T.toString:void 0;function O(e){var t=-1,n=e?e.length:0;for(this.clear();++t-1},P.prototype.set=function(e,t){var n=this.__data__,r=R(n,e);return r<0?n.push([e,t]):n[r][1]=t,this},I.prototype.clear=function(){this.__data__={hash:new O,map:new(A||P),string:new O}},I.prototype.delete=function(e){return F(this,e).delete(e)},I.prototype.get=function(e){return F(this,e).get(e)},I.prototype.has=function(e){return F(this,e).has(e)},I.prototype.set=function(e,t){return F(this,e).set(e,t),this};var $=V((function(e){var t;e=null==(t=e)?"":function(e){if("string"==typeof e)return e;if(q(e))return L?L.call(e):"";var t=e+"";return"0"==t&&1/e==-a?"-0":t}(t);var n=[];return u.test(e)&&n.push(""),e.replace(d,(function(e,t,r,a){n.push(r?a.replace(h,"$1"):t||e)})),n}));function H(e){if("string"==typeof e||q(e))return e;var t=e+"";return"0"==t&&1/e==-a?"-0":t}function V(e,t){if("function"!=typeof e||t&&"function"!=typeof t)throw new TypeError("Expected a function");var n=function(){var r=arguments,a=t?t.apply(this,r):r[0],i=n.cache;if(i.has(a))return i.get(a);var o=e.apply(this,r);return n.cache=i.set(a,o),o};return n.cache=new(V.Cache||I),n}V.Cache=I;var U=Array.isArray;function B(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function q(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&S.call(e)==l}e.exports=function(e,t,n){var r=null==e?void 0:D(e,t);return void 0===r?n:r}},141:(e,t,n)=>{var r="function"===typeof Map&&Map.prototype,a=Object.getOwnPropertyDescriptor&&r?Object.getOwnPropertyDescriptor(Map.prototype,"size"):null,i=r&&a&&"function"===typeof a.get?a.get:null,o=r&&Map.prototype.forEach,l="function"===typeof Set&&Set.prototype,s=Object.getOwnPropertyDescriptor&&l?Object.getOwnPropertyDescriptor(Set.prototype,"size"):null,c=l&&s&&"function"===typeof s.get?s.get:null,u=l&&Set.prototype.forEach,d="function"===typeof WeakMap&&WeakMap.prototype?WeakMap.prototype.has:null,h="function"===typeof WeakSet&&WeakSet.prototype?WeakSet.prototype.has:null,m="function"===typeof WeakRef&&WeakRef.prototype?WeakRef.prototype.deref:null,p=Boolean.prototype.valueOf,f=Object.prototype.toString,v=Function.prototype.toString,g=String.prototype.match,y=String.prototype.slice,_=String.prototype.replace,b=String.prototype.toUpperCase,w=String.prototype.toLowerCase,k=RegExp.prototype.test,x=Array.prototype.concat,S=Array.prototype.join,C=Array.prototype.slice,E=Math.floor,N="function"===typeof BigInt?BigInt.prototype.valueOf:null,A=Object.getOwnPropertySymbols,M="function"===typeof Symbol&&"symbol"===typeof Symbol.iterator?Symbol.prototype.toString:null,T="function"===typeof Symbol&&"object"===typeof Symbol.iterator,L="function"===typeof Symbol&&Symbol.toStringTag&&(typeof Symbol.toStringTag===T||"symbol")?Symbol.toStringTag:null,O=Object.prototype.propertyIsEnumerable,P=("function"===typeof Reflect?Reflect.getPrototypeOf:Object.getPrototypeOf)||([].__proto__===Array.prototype?function(e){return e.__proto__}:null);function I(e,t){if(e===1/0||e===-1/0||e!==e||e&&e>-1e3&&e<1e3||k.call(/e/,t))return t;var n=/[0-9](?=(?:[0-9]{3})+(?![0-9]))/g;if("number"===typeof e){var r=e<0?-E(-e):E(e);if(r!==e){var a=String(r),i=y.call(t,a.length+1);return _.call(a,n,"$&_")+"."+_.call(_.call(i,/([0-9]{3})/g,"$&_"),/_$/,"")}}return _.call(t,n,"$&_")}var R=n(634),D=R.custom,z=V(D)?D:null;function F(e,t,n){var r="double"===(n.quoteStyle||t)?'"':"'";return r+e+r}function j(e){return _.call(String(e),/"/g,""")}function $(e){return"[object Array]"===q(e)&&(!L||!("object"===typeof e&&L in e))}function H(e){return"[object RegExp]"===q(e)&&(!L||!("object"===typeof e&&L in e))}function V(e){if(T)return e&&"object"===typeof e&&e instanceof Symbol;if("symbol"===typeof e)return!0;if(!e||"object"!==typeof e||!M)return!1;try{return M.call(e),!0}catch(t){}return!1}e.exports=function e(t,r,a,l){var s=r||{};if(B(s,"quoteStyle")&&"single"!==s.quoteStyle&&"double"!==s.quoteStyle)throw new TypeError('option "quoteStyle" must be "single" or "double"');if(B(s,"maxStringLength")&&("number"===typeof s.maxStringLength?s.maxStringLength<0&&s.maxStringLength!==1/0:null!==s.maxStringLength))throw new TypeError('option "maxStringLength", if provided, must be a positive integer, Infinity, or `null`');var f=!B(s,"customInspect")||s.customInspect;if("boolean"!==typeof f&&"symbol"!==f)throw new TypeError("option \"customInspect\", if provided, must be `true`, `false`, or `'symbol'`");if(B(s,"indent")&&null!==s.indent&&"\t"!==s.indent&&!(parseInt(s.indent,10)===s.indent&&s.indent>0))throw new TypeError('option "indent" must be "\\t", an integer > 0, or `null`');if(B(s,"numericSeparator")&&"boolean"!==typeof s.numericSeparator)throw new TypeError('option "numericSeparator", if provided, must be `true` or `false`');var b=s.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,s);if("number"===typeof t){if(0===t)return 1/0/t>0?"0":"-0";var k=String(t);return b?I(t,k):k}if("bigint"===typeof t){var E=String(t)+"n";return b?I(t,E):E}var A="undefined"===typeof s.depth?5:s.depth;if("undefined"===typeof a&&(a=0),a>=A&&A>0&&"object"===typeof t)return $(t)?"[Array]":"[Object]";var D=function(e,t){var n;if("\t"===e.indent)n="\t";else{if(!("number"===typeof e.indent&&e.indent>0))return null;n=S.call(Array(e.indent+1)," ")}return{base:n,prev:S.call(Array(t+1),n)}}(s,a);if("undefined"===typeof l)l=[];else if(Y(l,t)>=0)return"[Circular]";function U(t,n,r){if(n&&(l=C.call(l)).push(n),r){var i={depth:s.depth};return B(s,"quoteStyle")&&(i.quoteStyle=s.quoteStyle),e(t,i,a+1,l)}return e(t,s,a+1,l)}if("function"===typeof t&&!H(t)){var K=function(e){if(e.name)return e.name;var t=g.call(v.call(e),/^function\s*([\w$]+)/);if(t)return t[1];return null}(t),ee=X(t,U);return"[Function"+(K?": "+K:" (anonymous)")+"]"+(ee.length>0?" { "+S.call(ee,", ")+" }":"")}if(V(t)){var te=T?_.call(String(t),/^(Symbol\(.*\))_[^)]*$/,"$1"):M.call(t);return"object"!==typeof t||T?te:Q(te)}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 ne="<"+w.call(String(t.nodeName)),re=t.attributes||[],ae=0;ae"}if($(t)){if(0===t.length)return"[]";var ie=X(t,U);return D&&!function(e){for(var t=0;t=0)return!1;return!0}(ie)?"["+J(ie,D)+"]":"[ "+S.call(ie,", ")+" ]"}if(function(e){return"[object Error]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t)){var oe=X(t,U);return"cause"in Error.prototype||!("cause"in t)||O.call(t,"cause")?0===oe.length?"["+String(t)+"]":"{ ["+String(t)+"] "+S.call(oe,", ")+" }":"{ ["+String(t)+"] "+S.call(x.call("[cause]: "+U(t.cause),oe),", ")+" }"}if("object"===typeof t&&f){if(z&&"function"===typeof t[z]&&R)return R(t,{depth:A-a});if("symbol"!==f&&"function"===typeof t.inspect)return t.inspect()}if(function(e){if(!i||!e||"object"!==typeof e)return!1;try{i.call(e);try{c.call(e)}catch(ne){return!0}return e instanceof Map}catch(t){}return!1}(t)){var le=[];return o&&o.call(t,(function(e,n){le.push(U(n,t,!0)+" => "+U(e,t))})),G("Map",i.call(t),le,D)}if(function(e){if(!c||!e||"object"!==typeof e)return!1;try{c.call(e);try{i.call(e)}catch(t){return!0}return e instanceof Set}catch(n){}return!1}(t)){var se=[];return u&&u.call(t,(function(e){se.push(U(e,t))})),G("Set",c.call(t),se,D)}if(function(e){if(!d||!e||"object"!==typeof e)return!1;try{d.call(e,d);try{h.call(e,h)}catch(ne){return!0}return e instanceof WeakMap}catch(t){}return!1}(t))return Z("WeakMap");if(function(e){if(!h||!e||"object"!==typeof e)return!1;try{h.call(e,h);try{d.call(e,d)}catch(ne){return!0}return e instanceof WeakSet}catch(t){}return!1}(t))return Z("WeakSet");if(function(e){if(!m||!e||"object"!==typeof e)return!1;try{return m.call(e),!0}catch(t){}return!1}(t))return Z("WeakRef");if(function(e){return"[object Number]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(U(Number(t)));if(function(e){if(!e||"object"!==typeof e||!N)return!1;try{return N.call(e),!0}catch(t){}return!1}(t))return Q(U(N.call(t)));if(function(e){return"[object Boolean]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(p.call(t));if(function(e){return"[object String]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(U(String(t)));if("undefined"!==typeof window&&t===window)return"{ [object Window] }";if(t===n.g)return"{ [object globalThis] }";if(!function(e){return"[object Date]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t)&&!H(t)){var ce=X(t,U),ue=P?P(t)===Object.prototype:t instanceof Object||t.constructor===Object,de=t instanceof Object?"":"null prototype",he=!ue&&L&&Object(t)===t&&L in t?y.call(q(t),8,-1):de?"Object":"",me=(ue||"function"!==typeof t.constructor?"":t.constructor.name?t.constructor.name+" ":"")+(he||de?"["+S.call(x.call([],he||[],de||[]),": ")+"] ":"");return 0===ce.length?me+"{}":D?me+"{"+J(ce,D)+"}":me+"{ "+S.call(ce,", ")+" }"}return String(t)};var U=Object.prototype.hasOwnProperty||function(e){return e in this};function B(e,t){return U.call(e,t)}function q(e){return f.call(e)}function Y(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 F(_.call(_.call(e,/(['\\])/g,"\\$1"),/[\x00-\x1f]/g,K),"single",t)}function K(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 Q(e){return"Object("+e+")"}function Z(e){return e+" { ? }"}function G(e,t,n,r){return e+" ("+t+") {"+(r?J(n,r):S.call(n,", "))+"}"}function J(e,t){if(0===e.length)return"";var n="\n"+t.prev+t.base;return n+S.call(e,","+n)+"\n"+t.prev}function X(e,t){var n=$(e),r=[];if(n){r.length=e.length;for(var a=0;a{"use strict";n.r(t),n.d(t,{Children:()=>Y,Component:()=>l.uA,Fragment:()=>l.FK,PureComponent:()=>$,StrictMode:()=>Oe,Suspense:()=>G,SuspenseList:()=>ee,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:()=>ke,cloneElement:()=>Ne,createContext:()=>l.q6,createElement:()=>l.n,createFactory:()=>Se,createPortal:()=>ae,createRef:()=>l._3,default:()=>$e,findDOMNode:()=>Me,flushSync:()=>Le,forwardRef:()=>B,hydrate:()=>he,isElement:()=>ze,isFragment:()=>Ee,isValidElement:()=>Ce,lazy:()=>X,memo:()=>H,render:()=>de,startTransition:()=>Pe,unmountComponentAtNode:()=>Ae,unstable_batchedUpdates:()=>Te,useCallback:()=>E,useContext:()=>N,useDebugValue:()=>A,useDeferredValue:()=>Ie,useEffect:()=>w,useErrorBoundary:()=>M,useId:()=>T,useImperativeHandle:()=>S,useInsertionEffect:()=>De,useLayoutEffect:()=>k,useMemo:()=>C,useReducer:()=>b,useRef:()=>x,useState:()=>_,useSyncExternalStore:()=>Fe,useTransition:()=>Re,version:()=>xe});var r,a,i,o,l=n(746),s=0,c=[],u=[],d=l.fF,h=d.__b,m=d.__r,p=d.diffed,f=d.__c,v=d.unmount,g=d.__;function y(e,t){d.__h&&d.__h(a,e,s||t),s=0;var n=a.__H||(a.__H={__:[],__h:[]});return e>=n.__.length&&n.__.push({__V:u}),n.__[e]}function _(e){return s=1,b(z,e)}function b(e,t,n){var i=y(r++,2);if(i.t=e,!i.__c&&(i.__=[n?n(t):z(void 0,t),function(e){var t=i.__N?i.__N[0]:i.__[0],n=i.t(t,e);t!==n&&(i.__N=[n,i.__[1]],i.__c.setState({}))}],i.__c=a,!a.u)){var o=function(e,t,n){if(!i.__c.__H)return!0;var r=i.__c.__H.__.filter((function(e){return!!e.__c}));if(r.every((function(e){return!e.__N})))return!l||l.call(this,e,t,n);var a=!1;return r.forEach((function(e){if(e.__N){var t=e.__[0];e.__=e.__N,e.__N=void 0,t!==e.__[0]&&(a=!0)}})),!(!a&&i.__c.props===e)&&(!l||l.call(this,e,t,n))};a.u=!0;var l=a.shouldComponentUpdate,s=a.componentWillUpdate;a.componentWillUpdate=function(e,t,n){if(this.__e){var r=l;l=void 0,o(e,t,n),l=r}s&&s.call(this,e,t,n)},a.shouldComponentUpdate=o}return i.__N||i.__}function w(e,t){var n=y(r++,3);!d.__s&&D(n.__H,t)&&(n.__=e,n.i=t,a.__H.__h.push(n))}function k(e,t){var n=y(r++,4);!d.__s&&D(n.__H,t)&&(n.__=e,n.i=t,a.__h.push(n))}function x(e){return s=5,C((function(){return{current:e}}),[])}function S(e,t,n){s=6,k((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 C(e,t){var n=y(r++,7);return D(n.__H,t)?(n.__V=e(),n.i=t,n.__h=e,n.__V):n.__}function E(e,t){return s=8,C((function(){return e}),t)}function N(e){var t=a.context[e.__c],n=y(r++,9);return n.c=e,t?(null==n.__&&(n.__=!0,t.sub(a)),t.props.value):e.__}function A(e,t){d.useDebugValue&&d.useDebugValue(t?t(e):e)}function M(e){var t=y(r++,10),n=_();return t.__=e,a.componentDidCatch||(a.componentDidCatch=function(e,r){t.__&&t.__(e,r),n[1](e)}),[n[0],function(){n[1](void 0)}]}function T(){var e=y(r++,11);if(!e.__){for(var t=a.__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 L(){for(var e;e=c.shift();)if(e.__P&&e.__H)try{e.__H.__h.forEach(I),e.__H.__h.forEach(R),e.__H.__h=[]}catch(r){e.__H.__h=[],d.__e(r,e.__v)}}d.__b=function(e){a=null,h&&h(e)},d.__=function(e,t){e&&t.__k&&t.__k.__m&&(e.__m=t.__k.__m),g&&g(e,t)},d.__r=function(e){m&&m(e),r=0;var t=(a=e.__c).__H;t&&(i===a?(t.__h=[],a.__h=[],t.__.forEach((function(e){e.__N&&(e.__=e.__N),e.__V=u,e.__N=e.i=void 0}))):(t.__h.forEach(I),t.__h.forEach(R),t.__h=[],r=0)),i=a},d.diffed=function(e){p&&p(e);var t=e.__c;t&&t.__H&&(t.__H.__h.length&&(1!==c.push(t)&&o===d.requestAnimationFrame||((o=d.requestAnimationFrame)||P)(L)),t.__H.__.forEach((function(e){e.i&&(e.__H=e.i),e.__V!==u&&(e.__=e.__V),e.i=void 0,e.__V=u}))),i=a=null},d.__c=function(e,t){t.some((function(e){try{e.__h.forEach(I),e.__h=e.__h.filter((function(e){return!e.__||R(e)}))}catch(a){t.some((function(e){e.__h&&(e.__h=[])})),t=[],d.__e(a,e.__v)}})),f&&f(e,t)},d.unmount=function(e){v&&v(e);var t,n=e.__c;n&&n.__H&&(n.__H.__.forEach((function(e){try{I(e)}catch(e){t=e}})),n.__H=void 0,t&&d.__e(t,n.__v))};var O="function"==typeof requestAnimationFrame;function P(e){var t,n=function(){clearTimeout(r),O&&cancelAnimationFrame(t),setTimeout(e)},r=setTimeout(n,100);O&&(t=requestAnimationFrame(n))}function I(e){var t=a,n=e.__c;"function"==typeof n&&(e.__c=void 0,n()),a=t}function R(e){var t=a;e.__c=e.__(),a=t}function D(e,t){return!e||e.length!==t.length||t.some((function(t,n){return t!==e[n]}))}function z(e,t){return"function"==typeof t?t(e):t}function F(e,t){for(var n in t)e[n]=t[n];return e}function j(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 $(e,t){this.props=e,this.context=t}function H(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:j(this.props,e)}function r(t){return this.shouldComponentUpdate=n,(0,l.n)(e,t)}return r.displayName="Memo("+(e.displayName||e.name)+")",r.prototype.isReactComponent=!0,r.__f=!0,r}($.prototype=new l.uA).isPureReactComponent=!0,$.prototype.shouldComponentUpdate=function(e,t){return j(this.props,e)||j(this.state,t)};var V=l.fF.__b;l.fF.__b=function(e){e.type&&e.type.__f&&e.ref&&(e.props.ref=e.ref,e.ref=null),V&&V(e)};var U="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.forward_ref")||3911;function B(e){function t(t){var n=F({},t);return delete n.ref,e(n,t.ref||null)}return t.$$typeof=U,t.render=t,t.prototype.isReactComponent=t.__f=!0,t.displayName="ForwardRef("+(e.displayName||e.name)+")",t}var q=function(e,t){return null==e?null:(0,l.v2)((0,l.v2)(e).map(t))},Y={map:q,forEach:q,count:function(e){return e?(0,l.v2)(e).length:0},only:function(e){var t=(0,l.v2)(e);if(1!==t.length)throw"Children.only";return t[0]},toArray:l.v2},W=l.fF.__e;l.fF.__e=function(e,t,n,r){if(e.then)for(var a,i=t;i=i.__;)if((a=i.__c)&&a.__c)return null==t.__e&&(t.__e=n.__e,t.__k=n.__k),a.__c(e,t);W(e,t,n,r)};var K=l.fF.unmount;function Q(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=F({},e)).__c&&(e.__c.__P===n&&(e.__c.__P=t),e.__c=null),e.__k=e.__k&&e.__k.map((function(e){return Q(e,t,n)}))),e}function Z(e,t,n){return e&&n&&(e.__v=null,e.__k=e.__k&&e.__k.map((function(e){return Z(e,t,n)})),e.__c&&e.__c.__P===t&&(e.__e&&n.appendChild(e.__e),e.__c.__e=!0,e.__c.__P=n)),e}function G(){this.__u=0,this.t=null,this.__b=null}function J(e){var t=e.__.__c;return t&&t.__a&&t.__a(e)}function X(e){var t,n,r;function a(a){if(t||(t=e()).then((function(e){n=e.default||e}),(function(e){r=e})),r)throw r;if(!n)throw t;return(0,l.n)(n,a)}return a.displayName="Lazy",a.__f=!0,a}function ee(){this.u=null,this.o=null}l.fF.unmount=function(e){var t=e.__c;t&&t.__R&&t.__R(),t&&32&e.__u&&(e.type=null),K&&K(e)},(G.prototype=new l.uA).__c=function(e,t){var n=t.__c,r=this;null==r.t&&(r.t=[]),r.t.push(n);var a=J(r.__v),i=!1,o=function(){i||(i=!0,n.__R=null,a?a(l):l())};n.__R=o;var l=function(){if(! --r.__u){if(r.state.__a){var e=r.state.__a;r.__v.__k[0]=Z(e,e.__c.__P,e.__c.__O)}var t;for(r.setState({__a:r.__b=null});t=r.t.pop();)t.forceUpdate()}};r.__u++||32&t.__u||r.setState({__a:r.__b=r.__v.__k[0]}),e.then(o,o)},G.prototype.componentWillUnmount=function(){this.t=[]},G.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]=Q(this.__b,n,r.__O=r.__P)}this.__b=null}var a=t.__a&&(0,l.n)(l.FK,null,e.fallback);return a&&(a.__u&=-33),[(0,l.n)(l.FK,null,t.__a?null:e.children),a]};var te=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)}}),(0,l.XX)((0,l.n)(ne,{context:t.context},e.__v),t.l)}function ae(e,t){var n=(0,l.n)(re,{__v:e,i:t});return n.containerInfo=t,n}(ee.prototype=new l.uA).__a=function(e){var t=this,n=J(t.__v),r=t.o.get(e);return r[0]++,function(a){var i=function(){t.props.revealOrder?(r.push(a),te(t,e,r)):a()};n?n(i):i()}},ee.prototype.render=function(e){this.u=null,this.o=new Map;var t=(0,l.v2)(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},ee.prototype.componentDidUpdate=ee.prototype.componentDidMount=function(){var e=this;this.o.forEach((function(t,n){te(e,n,t)}))};var ie="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.element")||60103,oe=/^(?:accent|alignment|arabic|baseline|cap|clip(?!PathU)|color|dominant|fill|flood|font|glyph(?!R)|horiz|image(!S)|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]/,le=/^on(Ani|Tra|Tou|BeforeInp|Compo)/,se=/[A-Z0-9]/g,ce="undefined"!=typeof document,ue=function(e){return("undefined"!=typeof Symbol&&"symbol"==typeof Symbol()?/fil|che|rad/:/fil|che|ra/).test(e)};function de(e,t,n){return null==t.__k&&(t.textContent=""),(0,l.XX)(e,t),"function"==typeof n&&n(),e?e.__c:null}function he(e,t,n){return(0,l.Qv)(e,t),"function"==typeof n&&n(),e?e.__c:null}l.uA.prototype.isReactComponent={},["componentWillMount","componentWillReceiveProps","componentWillUpdate"].forEach((function(e){Object.defineProperty(l.uA.prototype,e,{configurable:!0,get:function(){return this["UNSAFE_"+e]},set:function(t){Object.defineProperty(this,e,{configurable:!0,writable:!0,value:t})}})}));var me=l.fF.event;function pe(){}function fe(){return this.cancelBubble}function ve(){return this.defaultPrevented}l.fF.event=function(e){return me&&(e=me(e)),e.persist=pe,e.isPropagationStopped=fe,e.isDefaultPrevented=ve,e.nativeEvent=e};var ge,ye={enumerable:!1,configurable:!0,get:function(){return this.class}},_e=l.fF.vnode;l.fF.vnode=function(e){"string"==typeof e.type&&function(e){var t=e.props,n=e.type,r={};for(var a in t){var i=t[a];if(!("value"===a&&"defaultValue"in t&&null==i||ce&&"children"===a&&"noscript"===n||"class"===a||"className"===a)){var o=a.toLowerCase();"defaultValue"===a&&"value"in t&&null==t.value?a="value":"download"===a&&!0===i?i="":"translate"===o&&"no"===i?i=!1:"ondoubleclick"===o?a="ondblclick":"onchange"!==o||"input"!==n&&"textarea"!==n||ue(t.type)?"onfocus"===o?a="onfocusin":"onblur"===o?a="onfocusout":le.test(a)?a=o:-1===n.indexOf("-")&&oe.test(a)?a=a.replace(se,"-$&").toLowerCase():null===i&&(i=void 0):o=a="oninput","oninput"===o&&r[a=o]&&(a="oninputCapture"),r[a]=i}}"select"==n&&r.multiple&&Array.isArray(r.value)&&(r.value=(0,l.v2)(t.children).forEach((function(e){e.props.selected=-1!=r.value.indexOf(e.props.value)}))),"select"==n&&null!=r.defaultValue&&(r.value=(0,l.v2)(t.children).forEach((function(e){e.props.selected=r.multiple?-1!=r.defaultValue.indexOf(e.props.value):r.defaultValue==e.props.value}))),t.class&&!t.className?(r.class=t.class,Object.defineProperty(r,"className",ye)):(t.className&&!t.class||t.class&&t.className)&&(r.class=r.className=t.className),e.props=r}(e),e.$$typeof=ie,_e&&_e(e)};var be=l.fF.__r;l.fF.__r=function(e){be&&be(e),ge=e.__c};var we=l.fF.diffed;l.fF.diffed=function(e){we&&we(e);var t=e.props,n=e.__e;null!=n&&"textarea"===e.type&&"value"in t&&t.value!==n.value&&(n.value=null==t.value?"":t.value),ge=null};var ke={ReactCurrentDispatcher:{current:{readContext:function(e){return ge.__n[e.__c].props.value}}}},xe="17.0.2";function Se(e){return l.n.bind(null,e)}function Ce(e){return!!e&&e.$$typeof===ie}function Ee(e){return Ce(e)&&e.type===l.FK}function Ne(e){return Ce(e)?l.Ob.apply(null,arguments):e}function Ae(e){return!!e.__k&&((0,l.XX)(null,e),!0)}function Me(e){return e&&(e.base||1===e.nodeType&&e)||null}var Te=function(e,t){return e(t)},Le=function(e,t){return e(t)},Oe=l.FK;function Pe(e){e()}function Ie(e){return e}function Re(){return[!1,Pe]}var De=k,ze=Ce;function Fe(e,t){var n=t(),r=_({h:{__:n,v:t}}),a=r[0].h,i=r[1];return k((function(){a.__=n,a.v=t,je(a)&&i({h:a})}),[e,n,t]),w((function(){return je(a)&&i({h:a}),e((function(){je(a)&&i({h:a})}))}),[e]),n}function je(e){var t,n,r=e.v,a=e.__;try{var i=r();return!((t=a)===(n=i)&&(0!==t||1/t==1/n)||t!=t&&n!=n)}catch(e){return!0}}var $e={useState:_,useId:T,useReducer:b,useEffect:w,useLayoutEffect:k,useInsertionEffect:De,useTransition:Re,useDeferredValue:Ie,useSyncExternalStore:Fe,startTransition:Pe,useRef:x,useImperativeHandle:S,useMemo:C,useCallback:E,useContext:N,useDebugValue:A,version:"17.0.2",Children:Y,render:de,hydrate:he,unmountComponentAtNode:Ae,createPortal:ae,createElement:l.n,createContext:l.q6,createFactory:Se,cloneElement:Ne,createRef:l._3,Fragment:l.FK,isValidElement:Ce,isElement:ze,isFragment:Ee,findDOMNode:Me,Component:l.uA,PureComponent:$,memo:H,forwardRef:B,flushSync:Le,unstable_batchedUpdates:Te,StrictMode:Oe,Suspense:G,SuspenseList:ee,lazy:X,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:ke}},746:(e,t,n)=>{"use strict";n.d(t,{FK:()=>b,Ob:()=>B,Qv:()=>U,XX:()=>V,_3:()=>_,fF:()=>a,n:()=>g,q6:()=>q,uA:()=>w,v2:()=>T});var r,a,i,o,l,s,c,u,d={},h=[],m=/acit|ex(?:s|g|n|p|$)|rph|grid|ows|mnc|ntw|ine[ch]|zoo|^ord|itera/i,p=Array.isArray;function f(e,t){for(var n in t)e[n]=t[n];return e}function v(e){var t=e.parentNode;t&&t.removeChild(e)}function g(e,t,n){var a,i,o,l={};for(o in t)"key"==o?a=t[o]:"ref"==o?i=t[o]:l[o]=t[o];if(arguments.length>2&&(l.children=arguments.length>3?r.call(arguments,2):n),"function"==typeof e&&null!=e.defaultProps)for(o in e.defaultProps)void 0===l[o]&&(l[o]=e.defaultProps[o]);return y(e,l,a,i,null)}function y(e,t,n,r,o){var l={type:e,props:t,key:n,ref:r,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,constructor:void 0,__v:null==o?++i:o,__i:-1,__u:0};return null==o&&null!=a.vnode&&a.vnode(l),l}function _(){return{current:null}}function b(e){return e.children}function w(e,t){this.props=e,this.context=t}function k(e,t){if(null==t)return e.__?k(e.__,e.__i+1):null;for(var n;tn?(z(r,t,i),i.length=r.length=0,t=void 0,o.sort(c)):t&&a.__c&&a.__c(t,h));t&&z(r,t,i),E.__r=0}function N(e,t,n,r,a,i,o,l,s,c,u){var m,p,f,v,g,y=r&&r.__k||h,_=t.length;for(n.__d=s,A(n,t,y),s=n.__d,m=0;m<_;m++)null!=(f=n.__k[m])&&"boolean"!=typeof f&&"function"!=typeof f&&(p=-1===f.__i?d:y[f.__i]||d,f.__i=m,D(e,f,p,a,i,o,l,s,c,u),v=f.__e,f.ref&&p.ref!=f.ref&&(p.ref&&j(p.ref,null,f),u.push(f.ref,f.__c||v,f)),null==g&&null!=v&&(g=v),65536&f.__u||p.__k===f.__k?s=M(f,s,e):"function"==typeof f.type&&void 0!==f.__d?s=f.__d:v&&(s=v.nextSibling),f.__d=void 0,f.__u&=-196609);n.__d=s,n.__e=g}function A(e,t,n){var r,a,i,o,l,s=t.length,c=n.length,u=c,d=0;for(e.__k=[],r=0;r0?y(a.type,a.props,a.key,a.ref?a.ref:null,a.__v):a)?(a.__=e,a.__b=e.__b+1,l=L(a,n,o=r+d,u),a.__i=l,i=null,-1!==l&&(u--,(i=n[l])&&(i.__u|=131072)),null==i||null===i.__v?(-1==l&&d--,"function"!=typeof a.type&&(a.__u|=65536)):l!==o&&(l===o+1?d++:l>o?u>s-o?d+=l-o:d--:d=l(null!=s&&0==(131072&s.__u)?1:0))for(;o>=0||l=0){if((s=t[o])&&0==(131072&s.__u)&&a==s.key&&i===s.type)return o;o--}if(l2&&(s.children=arguments.length>3?r.call(arguments,2):n),y(e.type,s,a||e.key,i||e.ref,null)}function q(e,t){var n={__c:t="__cC"+u++,__: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((function(e){e.__e=!0,C(e)}))},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=h.slice,a={__e:function(e,t,n,r){for(var a,i,o;t=t.__;)if((a=t.__c)&&!a.__)try{if((i=a.constructor)&&null!=i.getDerivedStateFromError&&(a.setState(i.getDerivedStateFromError(e)),o=a.__d),null!=a.componentDidCatch&&(a.componentDidCatch(e,r||{}),o=a.__d),o)return a.__E=a}catch(t){e=t}throw e}},i=0,w.prototype.setState=function(e,t){var n;n=null!=this.__s&&this.__s!==this.state?this.__s:this.__s=f({},this.state),"function"==typeof e&&(e=e(f({},n),this.props)),e&&f(n,e),null!=e&&this.__v&&(t&&this._sb.push(t),C(this))},w.prototype.forceUpdate=function(e){this.__v&&(this.__e=!0,e&&this.__h.push(e),C(this))},w.prototype.render=b,o=[],s="function"==typeof Promise?Promise.prototype.then.bind(Promise.resolve()):setTimeout,c=function(e,t){return e.__v.__b-t.__v.__b},E.__r=0,u=0},640:e=>{"use strict";var t=String.prototype.replace,n=/%20/g,r="RFC1738",a="RFC3986";e.exports={default:a,formatters:{RFC1738:function(e){return t.call(e,n,"+")},RFC3986:function(e){return String(e)}},RFC1738:r,RFC3986:a}},215:(e,t,n)=>{"use strict";var r=n(518),a=n(968),i=n(640);e.exports={formats:i,parse:a,stringify:r}},968:(e,t,n)=>{"use strict";var r=n(570),a=Object.prototype.hasOwnProperty,i=Array.isArray,o={allowDots:!1,allowPrototypes:!1,allowSparse:!1,arrayLimit:20,charset:"utf-8",charsetSentinel:!1,comma:!1,decoder:r.decode,delimiter:"&",depth:5,ignoreQueryPrefix:!1,interpretNumericEntities:!1,parameterLimit:1e3,parseArrays:!0,plainObjects:!1,strictNullHandling:!1},l=function(e){return e.replace(/&#(\d+);/g,(function(e,t){return String.fromCharCode(parseInt(t,10))}))},s=function(e,t){return e&&"string"===typeof e&&t.comma&&e.indexOf(",")>-1?e.split(","):e},c=function(e,t,n,r){if(e){var i=n.allowDots?e.replace(/\.([^.[]+)/g,"[$1]"):e,o=/(\[[^[\]]*])/g,l=n.depth>0&&/(\[[^[\]]*])/.exec(i),c=l?i.slice(0,l.index):i,u=[];if(c){if(!n.plainObjects&&a.call(Object.prototype,c)&&!n.allowPrototypes)return;u.push(c)}for(var d=0;n.depth>0&&null!==(l=o.exec(i))&&d=0;--i){var o,l=e[i];if("[]"===l&&n.parseArrays)o=[].concat(a);else{o=n.plainObjects?Object.create(null):{};var c="["===l.charAt(0)&&"]"===l.charAt(l.length-1)?l.slice(1,-1):l,u=parseInt(c,10);n.parseArrays||""!==c?!isNaN(u)&&l!==c&&String(u)===c&&u>=0&&n.parseArrays&&u<=n.arrayLimit?(o=[])[u]=a:"__proto__"!==c&&(o[c]=a):o={0:a}}a=o}return a}(u,t,n,r)}};e.exports=function(e,t){var n=function(e){if(!e)return o;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?o.charset:e.charset;return{allowDots:"undefined"===typeof e.allowDots?o.allowDots:!!e.allowDots,allowPrototypes:"boolean"===typeof e.allowPrototypes?e.allowPrototypes:o.allowPrototypes,allowSparse:"boolean"===typeof e.allowSparse?e.allowSparse:o.allowSparse,arrayLimit:"number"===typeof e.arrayLimit?e.arrayLimit:o.arrayLimit,charset:t,charsetSentinel:"boolean"===typeof e.charsetSentinel?e.charsetSentinel:o.charsetSentinel,comma:"boolean"===typeof e.comma?e.comma:o.comma,decoder:"function"===typeof e.decoder?e.decoder:o.decoder,delimiter:"string"===typeof e.delimiter||r.isRegExp(e.delimiter)?e.delimiter:o.delimiter,depth:"number"===typeof e.depth||!1===e.depth?+e.depth:o.depth,ignoreQueryPrefix:!0===e.ignoreQueryPrefix,interpretNumericEntities:"boolean"===typeof e.interpretNumericEntities?e.interpretNumericEntities:o.interpretNumericEntities,parameterLimit:"number"===typeof e.parameterLimit?e.parameterLimit:o.parameterLimit,parseArrays:!1!==e.parseArrays,plainObjects:"boolean"===typeof e.plainObjects?e.plainObjects:o.plainObjects,strictNullHandling:"boolean"===typeof e.strictNullHandling?e.strictNullHandling:o.strictNullHandling}}(t);if(""===e||null===e||"undefined"===typeof e)return n.plainObjects?Object.create(null):{};for(var u="string"===typeof e?function(e,t){var n,c={__proto__:null},u=t.ignoreQueryPrefix?e.replace(/^\?/,""):e,d=t.parameterLimit===1/0?void 0:t.parameterLimit,h=u.split(t.delimiter,d),m=-1,p=t.charset;if(t.charsetSentinel)for(n=0;n-1&&(v=i(v)?[v]:v),a.call(c,f)?c[f]=r.combine(c[f],v):c[f]=v}return c}(e,n):e,d=n.plainObjects?Object.create(null):{},h=Object.keys(u),m=0;m{"use strict";var r=n(670),a=n(570),i=n(640),o=Object.prototype.hasOwnProperty,l={brackets:function(e){return e+"[]"},comma:"comma",indices:function(e,t){return e+"["+t+"]"},repeat:function(e){return e}},s=Array.isArray,c=Array.prototype.push,u=function(e,t){c.apply(e,s(t)?t:[t])},d=Date.prototype.toISOString,h=i.default,m={addQueryPrefix:!1,allowDots:!1,charset:"utf-8",charsetSentinel:!1,delimiter:"&",encode:!0,encoder:a.encode,encodeValuesOnly:!1,format:h,formatter:i.formatters[h],indices:!1,serializeDate:function(e){return d.call(e)},skipNulls:!1,strictNullHandling:!1},p={},f=function e(t,n,i,o,l,c,d,h,f,v,g,y,_,b,w,k){for(var x,S=t,C=k,E=0,N=!1;void 0!==(C=C.get(p))&&!N;){var A=C.get(t);if(E+=1,"undefined"!==typeof A){if(A===E)throw new RangeError("Cyclic object value");N=!0}"undefined"===typeof C.get(p)&&(E=0)}if("function"===typeof h?S=h(n,S):S instanceof Date?S=g(S):"comma"===i&&s(S)&&(S=a.maybeMap(S,(function(e){return e instanceof Date?g(e):e}))),null===S){if(l)return d&&!b?d(n,m.encoder,w,"key",y):n;S=""}if("string"===typeof(x=S)||"number"===typeof x||"boolean"===typeof x||"symbol"===typeof x||"bigint"===typeof x||a.isBuffer(S))return d?[_(b?n:d(n,m.encoder,w,"key",y))+"="+_(d(S,m.encoder,w,"value",y))]:[_(n)+"="+_(String(S))];var M,T=[];if("undefined"===typeof S)return T;if("comma"===i&&s(S))b&&d&&(S=a.maybeMap(S,d)),M=[{value:S.length>0?S.join(",")||null:void 0}];else if(s(h))M=h;else{var L=Object.keys(S);M=f?L.sort(f):L}for(var O=o&&s(S)&&1===S.length?n+"[]":n,P=0;P0?w+b:""}},570:(e,t,n)=>{"use strict";var r=n(640),a=Object.prototype.hasOwnProperty,i=Array.isArray,o=function(){for(var e=[],t=0;t<256;++t)e.push("%"+((t<16?"0":"")+t.toString(16)).toUpperCase());return e}(),l=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(i(n)){for(var r=[],a=0;a=48&&u<=57||u>=65&&u<=90||u>=97&&u<=122||i===r.RFC1738&&(40===u||41===u)?s+=l.charAt(c):u<128?s+=o[u]:u<2048?s+=o[192|u>>6]+o[128|63&u]:u<55296||u>=57344?s+=o[224|u>>12]+o[128|u>>6&63]+o[128|63&u]:(c+=1,u=65536+((1023&u)<<10|1023&l.charCodeAt(c)),s+=o[240|u>>18]+o[128|u>>12&63]+o[128|u>>6&63]+o[128|63&u])}return s},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(i(e)){for(var n=[],r=0;r{e.exports=n(204)},204:(e,t,n)=>{"use strict";var r=function(e){return e&&"object"==typeof e&&"default"in e?e.default:e}(n(609)),a=n(609);function i(){return(i=Object.assign||function(e){for(var t=1;tr.length&&h(e,t.length-1);)t=t.slice(0,t.length-1);return t.length}for(var a=r.length,i=t.length;i>=r.length;i--){var o=t[i];if(!h(e,i)&&m(e,i,o)){a=i+1;break}}return a}function v(e,t){return f(e,t)===e.mask.length}function g(e,t){var n=e.maskChar,r=e.mask,a=e.prefix;if(!n){for((t=y(e,"",t,0)).lengtht.length&&(t+=a.slice(t.length,r)),l.every((function(n){for(;u=n,h(e,c=r)&&u!==a[c];){if(r>=t.length&&(t+=a[r]),l=n,i&&h(e,r)&&l===i)return!0;if(++r>=a.length)return!1}var l,c,u;return!m(e,r,n)&&n!==i||(ra.start?d=(u=function(e,t,n,r){var a=e.mask,i=e.maskChar,o=n.split(""),l=r;return o.every((function(t){for(;o=t,h(e,n=r)&&o!==a[n];)if(++r>=a.length)return!1;var n,o;return(m(e,r,t)||t===i)&&r++,r=i.length?p=i.length:p=o.length&&p{"use strict";var r=n(375),a=n(411),i=n(734)(),o=n(553),l=n(277),s=r("%Math.floor%");e.exports=function(e,t){if("function"!==typeof e)throw new l("`fn` is not a function");if("number"!==typeof t||t<0||t>4294967295||s(t)!==t)throw new l("`length` must be a positive 32-bit integer");var n=arguments.length>2&&!!arguments[2],r=!0,c=!0;if("length"in e&&o){var u=o(e,"length");u&&!u.configurable&&(r=!1),u&&!u.writable&&(c=!1)}return(r||c||!n)&&(i?a(e,"length",t,!0,!0):a(e,"length",t)),e}},670:(e,t,n)=>{"use strict";var r=n(375),a=n(61),i=n(141),o=n(277),l=r("%WeakMap%",!0),s=r("%Map%",!0),c=a("WeakMap.prototype.get",!0),u=a("WeakMap.prototype.set",!0),d=a("WeakMap.prototype.has",!0),h=a("Map.prototype.get",!0),m=a("Map.prototype.set",!0),p=a("Map.prototype.has",!0),f=function(e,t){for(var n,r=e;null!==(n=r.next);r=n)if(n.key===t)return r.next=n.next,n.next=e.next,e.next=n,n};e.exports=function(){var e,t,n,r={assert:function(e){if(!r.has(e))throw new o("Side channel does not contain "+i(e))},get:function(r){if(l&&r&&("object"===typeof r||"function"===typeof r)){if(e)return c(e,r)}else if(s){if(t)return h(t,r)}else if(n)return function(e,t){var n=f(e,t);return n&&n.value}(n,r)},has:function(r){if(l&&r&&("object"===typeof r||"function"===typeof r)){if(e)return d(e,r)}else if(s){if(t)return p(t,r)}else if(n)return function(e,t){return!!f(e,t)}(n,r);return!1},set:function(r,a){l&&r&&("object"===typeof r||"function"===typeof r)?(e||(e=new l),u(e,r,a)):s?(t||(t=new s),m(t,r,a)):(n||(n={key:{},next:null}),function(e,t,n){var r=f(e,t);r?r.value=n:e.next={key:t,next:e.next,value:n}}(n,r,a))}};return r}},634:()=>{},738:(e,t)=>{var n;!function(){"use strict";var r={}.hasOwnProperty;function a(){for(var e="",t=0;t{var t=e&&e.__esModule?()=>e.default:()=>e;return n.d(t,{a:t}),t},n.d=(e,t)=>{for(var r in t)n.o(t,r)&&!n.o(e,r)&&Object.defineProperty(e,r,{enumerable:!0,get:t[r]})},n.f={},n.e=e=>Promise.all(Object.keys(n.f).reduce(((t,r)=>(n.f[r](e,t),t)),[])),n.u=e=>"static/js/"+e+".bebe1265.chunk.js",n.miniCssF=e=>{},n.g=function(){if("object"===typeof globalThis)return globalThis;try{return this||new Function("return this")()}catch(e){if("object"===typeof window)return window}}(),n.o=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),(()=>{var e={},t="vmui:";n.l=(r,a,i,o)=>{if(e[r])e[r].push(a);else{var l,s;if(void 0!==i)for(var c=document.getElementsByTagName("script"),u=0;u{l.onerror=l.onload=null,clearTimeout(m);var a=e[r];if(delete e[r],l.parentNode&&l.parentNode.removeChild(l),a&&a.forEach((e=>e(n))),t)return t(n)},m=setTimeout(h.bind(null,void 0,{type:"timeout",target:l}),12e4);l.onerror=h.bind(null,l.onerror),l.onload=h.bind(null,l.onload),s&&document.head.appendChild(l)}}})(),n.r=e=>{"undefined"!==typeof Symbol&&Symbol.toStringTag&&Object.defineProperty(e,Symbol.toStringTag,{value:"Module"}),Object.defineProperty(e,"__esModule",{value:!0})},n.p="./",(()=>{var e={792:0};n.f.j=(t,r)=>{var a=n.o(e,t)?e[t]:void 0;if(0!==a)if(a)r.push(a[2]);else{var i=new Promise(((n,r)=>a=e[t]=[n,r]));r.push(a[2]=i);var o=n.p+n.u(t),l=new Error;n.l(o,(r=>{if(n.o(e,t)&&(0!==(a=e[t])&&(e[t]=void 0),a)){var i=r&&("load"===r.type?"missing":r.type),o=r&&r.target&&r.target.src;l.message="Loading chunk "+t+" failed.\n("+i+": "+o+")",l.name="ChunkLoadError",l.type=i,l.request=o,a[1](l)}}),"chunk-"+t,t)}};var t=(t,r)=>{var a,i,o=r[0],l=r[1],s=r[2],c=0;if(o.some((t=>0!==e[t]))){for(a in l)n.o(l,a)&&(n.m[a]=l[a]);if(s)s(n)}for(t&&t(r);c{"use strict";var e={};n.r(e),n.d(e,{AlarmIcon:()=>Bn,ArrowDownIcon:()=>$n,ArrowDropDownIcon:()=>Hn,CalendarIcon:()=>Un,ChartIcon:()=>Kn,ClockIcon:()=>Vn,CloseIcon:()=>Pn,CodeIcon:()=>Zn,CopyIcon:()=>ar,DeleteIcon:()=>Gn,DoneIcon:()=>er,DownloadIcon:()=>wr,DragIcon:()=>ir,ErrorIcon:()=>zn,FunctionIcon:()=>yr,InfoIcon:()=>Rn,IssueIcon:()=>sr,KeyboardIcon:()=>qn,LabelIcon:()=>_r,ListIcon:()=>pr,LogoAnomalyIcon:()=>Tn,LogoIcon:()=>An,LogoLogsIcon:()=>Mn,LogoShortIcon:()=>Ln,MetricIcon:()=>gr,MinusIcon:()=>Xn,MoreIcon:()=>dr,PlayCircleOutlineIcon:()=>Wn,PlayIcon:()=>Yn,PlusIcon:()=>Jn,Prettify:()=>rr,QuestionIcon:()=>cr,RefreshIcon:()=>jn,RestartIcon:()=>In,SettingsIcon:()=>On,StarBorderIcon:()=>fr,StarIcon:()=>vr,StorageIcon:()=>ur,SuccessIcon:()=>Fn,TableIcon:()=>Qn,TimelineIcon:()=>or,TipIcon:()=>mr,TuneIcon:()=>hr,ValueIcon:()=>br,VisibilityIcon:()=>tr,VisibilityOffIcon:()=>nr,WarningIcon:()=>Dn,WikiIcon:()=>lr});var t,r=n(609),a=n(159),i=n.n(a),o=n(7),l=n.n(o),s=n(648),c=n.n(s),u=n(220),d=n.n(u);function h(){return h=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0&&(t.hash=e.substr(n),e=e.substr(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substr(r),e=e.substr(0,r)),e&&(t.pathname=e)}return t}function b(e,n,r,a){void 0===a&&(a={});let{window:i=document.defaultView,v5Compat:o=!1}=a,l=i.history,s=t.Pop,c=null,u=d();function d(){return(l.state||{idx:null}).idx}function f(){s=t.Pop;let e=d(),n=null==e?null:e-u;u=e,c&&c({action:s,location:b.location,delta:n})}function _(e){let t="null"!==i.location.origin?i.location.origin:i.location.href,n="string"===typeof e?e:y(e);return n=n.replace(/ $/,"%20"),p(t,"No window.location.(origin|href) available to create URL for href: "+n),new URL(n,t)}null==u&&(u=0,l.replaceState(h({},l.state,{idx:u}),""));let b={get action(){return s},get location(){return e(i,l)},listen(e){if(c)throw new Error("A history only accepts one active listener");return i.addEventListener(m,f),c=e,()=>{i.removeEventListener(m,f),c=null}},createHref:e=>n(i,e),createURL:_,encodeLocation(e){let t=_(e);return{pathname:t.pathname,search:t.search,hash:t.hash}},push:function(e,n){s=t.Push;let a=g(b.location,e,n);r&&r(a,e),u=d()+1;let h=v(a,u),m=b.createHref(a);try{l.pushState(h,"",m)}catch(p){if(p instanceof DOMException&&"DataCloneError"===p.name)throw p;i.location.assign(m)}o&&c&&c({action:s,location:b.location,delta:1})},replace:function(e,n){s=t.Replace;let a=g(b.location,e,n);r&&r(a,e),u=d();let i=v(a,u),h=b.createHref(a);l.replaceState(i,"",h),o&&c&&c({action:s,location:b.location,delta:0})},go:e=>l.go(e)};return b}var w;!function(e){e.data="data",e.deferred="deferred",e.redirect="redirect",e.error="error"}(w||(w={}));new Set(["lazy","caseSensitive","path","id","index","children"]);function k(e,t,n){void 0===n&&(n="/");let r=D(("string"===typeof t?_(t):t).pathname||"/",n);if(null==r)return null;let a=x(e);!function(e){e.sort(((e,t)=>e.score!==t.score?t.score-e.score:function(e,t){let n=e.length===t.length&&e.slice(0,-1).every(((e,n)=>e===t[n]));return n?e[e.length-1]-t[t.length-1]:0}(e.routesMeta.map((e=>e.childrenIndex)),t.routesMeta.map((e=>e.childrenIndex)))))}(a);let i=null;for(let o=0;null==i&&o{let o={relativePath:void 0===i?e.path||"":i,caseSensitive:!0===e.caseSensitive,childrenIndex:a,route:e};o.relativePath.startsWith("/")&&(p(o.relativePath.startsWith(r),'Absolute route path "'+o.relativePath+'" nested under path "'+r+'" is not valid. An absolute child route path must start with the combined path of all its parent routes.'),o.relativePath=o.relativePath.slice(r.length));let l=H([r,o.relativePath]),s=n.concat(o);e.children&&e.children.length>0&&(p(!0!==e.index,'Index routes must not have child routes. Please remove all child routes from route path "'+l+'".'),x(e.children,t,s,l)),(null!=e.path||e.index)&&t.push({path:l,score:O(l,e.index),routesMeta:s})};return e.forEach(((e,t)=>{var n;if(""!==e.path&&null!=(n=e.path)&&n.includes("?"))for(let r of S(e.path))a(e,t,r);else a(e,t)})),t}function S(e){let t=e.split("/");if(0===t.length)return[];let[n,...r]=t,a=n.endsWith("?"),i=n.replace(/\?$/,"");if(0===r.length)return a?[i,""]:[i];let o=S(r.join("/")),l=[];return l.push(...o.map((e=>""===e?i:[i,e].join("/")))),a&&l.push(...o),l.map((t=>e.startsWith("/")&&""===t?"/":t))}const C=/^:[\w-]+$/,E=3,N=2,A=1,M=10,T=-2,L=e=>"*"===e;function O(e,t){let n=e.split("/"),r=n.length;return n.some(L)&&(r+=T),t&&(r+=N),n.filter((e=>!L(e))).reduce(((e,t)=>e+(C.test(t)?E:""===t?A:M)),r)}function P(e,t){let{routesMeta:n}=e,r={},a="/",i=[];for(let o=0;o(r.push({paramName:t,isOptional:null!=n}),n?"/?([^\\/]+)?":"/([^\\/]+)")));e.endsWith("*")?(r.push({paramName:"*"}),a+="*"===e||"/*"===e?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?a+="\\/*$":""!==e&&"/"!==e&&(a+="(?:(?=\\/|$))");let i=new RegExp(a,t?void 0:"i");return[i,r]}(e.path,e.caseSensitive,e.end),a=t.match(n);if(!a)return null;let i=a[0],o=i.replace(/(.)\/+$/,"$1"),l=a.slice(1);return{params:r.reduce(((e,t,n)=>{let{paramName:r,isOptional:a}=t;if("*"===r){let e=l[n]||"";o=i.slice(0,i.length-e.length).replace(/(.)\/+$/,"$1")}const s=l[n];return e[r]=a&&!s?void 0:(s||"").replace(/%2F/g,"/"),e}),{}),pathname:i,pathnameBase:o,pattern:e}}function R(e){try{return e.split("/").map((e=>decodeURIComponent(e).replace(/\//g,"%2F"))).join("/")}catch(t){return f(!1,'The URL path "'+e+'" could not be decoded because it is is a malformed URL segment. This is probably due to a bad percent encoding ('+t+")."),e}}function D(e,t){if("/"===t)return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&"/"!==r?null:e.slice(n)||"/"}function z(e,t,n,r){return"Cannot include a '"+e+"' character in a manually specified `to."+t+"` field ["+JSON.stringify(r)+"]. Please separate it out to the `to."+n+'` field. Alternatively you may provide the full path as a string in and the router will parse it for you.'}function F(e){return e.filter(((e,t)=>0===t||e.route.path&&e.route.path.length>0))}function j(e,t){let n=F(e);return t?n.map(((t,n)=>n===e.length-1?t.pathname:t.pathnameBase)):n.map((e=>e.pathnameBase))}function $(e,t,n,r){let a;void 0===r&&(r=!1),"string"===typeof e?a=_(e):(a=h({},e),p(!a.pathname||!a.pathname.includes("?"),z("?","pathname","search",a)),p(!a.pathname||!a.pathname.includes("#"),z("#","pathname","hash",a)),p(!a.search||!a.search.includes("#"),z("#","search","hash",a)));let i,o=""===e||""===a.pathname,l=o?"/":a.pathname;if(null==l)i=n;else{let e=t.length-1;if(!r&&l.startsWith("..")){let t=l.split("/");for(;".."===t[0];)t.shift(),e-=1;a.pathname=t.join("/")}i=e>=0?t[e]:"/"}let s=function(e,t){void 0===t&&(t="/");let{pathname:n,search:r="",hash:a=""}="string"===typeof e?_(e):e,i=n?n.startsWith("/")?n:function(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach((e=>{".."===e?n.length>1&&n.pop():"."!==e&&n.push(e)})),n.length>1?n.join("/"):"/"}(n,t):t;return{pathname:i,search:U(r),hash:B(a)}}(a,i),c=l&&"/"!==l&&l.endsWith("/"),u=(o||"."===l)&&n.endsWith("/");return s.pathname.endsWith("/")||!c&&!u||(s.pathname+="/"),s}const H=e=>e.join("/").replace(/\/\/+/g,"/"),V=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),U=e=>e&&"?"!==e?e.startsWith("?")?e:"?"+e:"",B=e=>e&&"#"!==e?e.startsWith("#")?e:"#"+e:"";Error;function q(e){return null!=e&&"number"===typeof e.status&&"string"===typeof e.statusText&&"boolean"===typeof e.internal&&"data"in e}const Y=["post","put","patch","delete"],W=(new Set(Y),["get",...Y]);new Set(W),new Set([301,302,303,307,308]),new Set([307,308]);Symbol("deferred");function K(){return K=Object.assign?Object.assign.bind():function(e){for(var t=1;t{n.current=!0}));let a=r.useCallback((function(r,a){void 0===a&&(a={}),n.current&&("number"===typeof r?e.navigate(r):e.navigate(r,K({fromRouteId:t},a)))}),[e,t]);return a}():function(){te()||p(!1);let e=r.useContext(Q),{basename:t,future:n,navigator:a}=r.useContext(G),{matches:i}=r.useContext(X),{pathname:o}=ne(),l=JSON.stringify(j(i,n.v7_relativeSplatPath)),s=r.useRef(!1);re((()=>{s.current=!0}));let c=r.useCallback((function(n,r){if(void 0===r&&(r={}),!s.current)return;if("number"===typeof n)return void a.go(n);let i=$(n,JSON.parse(l),o,"path"===r.relative);null==e&&"/"!==t&&(i.pathname="/"===i.pathname?t:H([t,i.pathname])),(r.replace?a.replace:a.push)(i,r.state,r)}),[t,a,l,o,e]);return c}()}const ie=r.createContext(null);function oe(e,t){let{relative:n}=void 0===t?{}:t,{future:a}=r.useContext(G),{matches:i}=r.useContext(X),{pathname:o}=ne(),l=JSON.stringify(j(i,a.v7_relativeSplatPath));return r.useMemo((()=>$(e,JSON.parse(l),o,"path"===n)),[e,l,o,n])}function le(e,n,a,i){te()||p(!1);let{navigator:o}=r.useContext(G),{matches:l}=r.useContext(X),s=l[l.length-1],c=s?s.params:{},u=(s&&s.pathname,s?s.pathnameBase:"/");s&&s.route;let d,h=ne();if(n){var m;let e="string"===typeof n?_(n):n;"/"===u||(null==(m=e.pathname)?void 0:m.startsWith(u))||p(!1),d=e}else d=h;let f=d.pathname||"/",v=f;if("/"!==u){let e=u.replace(/^\//,"").split("/");v="/"+f.replace(/^\//,"").split("/").slice(e.length).join("/")}let g=k(e,{pathname:v});let y=he(g&&g.map((e=>Object.assign({},e,{params:Object.assign({},c,e.params),pathname:H([u,o.encodeLocation?o.encodeLocation(e.pathname).pathname:e.pathname]),pathnameBase:"/"===e.pathnameBase?u:H([u,o.encodeLocation?o.encodeLocation(e.pathnameBase).pathname:e.pathnameBase])}))),l,a,i);return n&&y?r.createElement(J.Provider,{value:{location:K({pathname:"/",search:"",hash:"",state:null,key:"default"},d),navigationType:t.Pop}},y):y}function se(){let e=function(){var e;let t=r.useContext(ee),n=ve(pe.UseRouteError),a=ge(pe.UseRouteError);if(void 0!==t)return t;return null==(e=n.errors)?void 0:e[a]}(),t=q(e)?e.status+" "+e.statusText:e instanceof Error?e.message:JSON.stringify(e),n=e instanceof Error?e.stack:null,a="rgba(200,200,200, 0.5)",i={padding:"0.5rem",backgroundColor:a};return r.createElement(r.Fragment,null,r.createElement("h2",null,"Unexpected Application Error!"),r.createElement("h3",{style:{fontStyle:"italic"}},t),n?r.createElement("pre",{style:i},n):null,null)}const ce=r.createElement(se,null);class ue extends r.Component{constructor(e){super(e),this.state={location:e.location,revalidation:e.revalidation,error:e.error}}static getDerivedStateFromError(e){return{error:e}}static getDerivedStateFromProps(e,t){return t.location!==e.location||"idle"!==t.revalidation&&"idle"===e.revalidation?{error:e.error,location:e.location,revalidation:e.revalidation}:{error:void 0!==e.error?e.error:t.error,location:t.location,revalidation:e.revalidation||t.revalidation}}componentDidCatch(e,t){console.error("React Router caught the following error during render",e,t)}render(){return void 0!==this.state.error?r.createElement(X.Provider,{value:this.props.routeContext},r.createElement(ee.Provider,{value:this.state.error,children:this.props.component})):this.props.children}}function de(e){let{routeContext:t,match:n,children:a}=e,i=r.useContext(Q);return i&&i.static&&i.staticContext&&(n.route.errorElement||n.route.ErrorBoundary)&&(i.staticContext._deepestRenderedBoundaryId=n.route.id),r.createElement(X.Provider,{value:t},a)}function he(e,t,n,a){var i;if(void 0===t&&(t=[]),void 0===n&&(n=null),void 0===a&&(a=null),null==e){var o;if(null==(o=n)||!o.errors)return null;e=n.matches}let l=e,s=null==(i=n)?void 0:i.errors;if(null!=s){let e=l.findIndex((e=>e.route.id&&(null==s?void 0:s[e.route.id])));e>=0||p(!1),l=l.slice(0,Math.min(l.length,e+1))}let c=!1,u=-1;if(n&&a&&a.v7_partialHydration)for(let r=0;r=0?l.slice(0,u+1):[l[0]];break}}}return l.reduceRight(((e,a,i)=>{let o,d=!1,h=null,m=null;var p;n&&(o=s&&a.route.id?s[a.route.id]:void 0,h=a.route.errorElement||ce,c&&(u<0&&0===i?(p="route-fallback",!1||ye[p]||(ye[p]=!0),d=!0,m=null):u===i&&(d=!0,m=a.route.hydrateFallbackElement||null)));let f=t.concat(l.slice(0,i+1)),v=()=>{let t;return t=o?h:d?m:a.route.Component?r.createElement(a.route.Component,null):a.route.element?a.route.element:e,r.createElement(de,{match:a,routeContext:{outlet:e,matches:f,isDataRoute:null!=n},children:t})};return n&&(a.route.ErrorBoundary||a.route.errorElement||0===i)?r.createElement(ue,{location:n.location,revalidation:n.revalidation,component:h,error:o,children:v(),routeContext:{outlet:null,matches:f,isDataRoute:!0}}):v()}),null)}var me=function(e){return e.UseBlocker="useBlocker",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e}(me||{}),pe=function(e){return e.UseBlocker="useBlocker",e.UseLoaderData="useLoaderData",e.UseActionData="useActionData",e.UseRouteError="useRouteError",e.UseNavigation="useNavigation",e.UseRouteLoaderData="useRouteLoaderData",e.UseMatches="useMatches",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e.UseRouteId="useRouteId",e}(pe||{});function fe(e){let t=r.useContext(Q);return t||p(!1),t}function ve(e){let t=r.useContext(Z);return t||p(!1),t}function ge(e){let t=function(e){let t=r.useContext(X);return t||p(!1),t}(),n=t.matches[t.matches.length-1];return n.route.id||p(!1),n.route.id}const ye={};r.startTransition;function _e(e){return function(e){let t=r.useContext(X).outlet;return t?r.createElement(ie.Provider,{value:e},t):t}(e.context)}function be(e){p(!1)}function we(e){let{basename:n="/",children:a=null,location:i,navigationType:o=t.Pop,navigator:l,static:s=!1,future:c}=e;te()&&p(!1);let u=n.replace(/^\/*/,"/"),d=r.useMemo((()=>({basename:u,navigator:l,static:s,future:K({v7_relativeSplatPath:!1},c)})),[u,c,l,s]);"string"===typeof i&&(i=_(i));let{pathname:h="/",search:m="",hash:f="",state:v=null,key:g="default"}=i,y=r.useMemo((()=>{let e=D(h,u);return null==e?null:{location:{pathname:e,search:m,hash:f,state:v,key:g},navigationType:o}}),[u,h,m,f,v,g,o]);return null==y?null:r.createElement(G.Provider,{value:d},r.createElement(J.Provider,{children:a,value:y}))}function ke(e){let{children:t,location:n}=e;return le(xe(t),n)}new Promise((()=>{}));r.Component;function xe(e,t){void 0===t&&(t=[]);let n=[];return r.Children.forEach(e,((e,a)=>{if(!r.isValidElement(e))return;let i=[...t,a];if(e.type===r.Fragment)return void n.push.apply(n,xe(e.props.children,i));e.type!==be&&p(!1),e.props.index&&e.props.children&&p(!1);let o={id:e.props.id||i.join("-"),caseSensitive:e.props.caseSensitive,element:e.props.element,Component:e.props.Component,index:e.props.index,path:e.props.path,loader:e.props.loader,action:e.props.action,errorElement:e.props.errorElement,ErrorBoundary:e.props.ErrorBoundary,hasErrorBoundary:null!=e.props.ErrorBoundary||null!=e.props.errorElement,shouldRevalidate:e.props.shouldRevalidate,handle:e.props.handle,lazy:e.props.lazy};e.props.children&&(o.children=xe(e.props.children,i)),n.push(o)})),n}function Se(){return Se=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0||(a[n]=e[n]);return a}function Ee(e){return void 0===e&&(e=""),new URLSearchParams("string"===typeof e||Array.isArray(e)||e instanceof URLSearchParams?e:Object.keys(e).reduce(((t,n)=>{let r=e[n];return t.concat(Array.isArray(r)?r.map((e=>[n,e])):[[n,r]])}),[]))}new Set(["application/x-www-form-urlencoded","multipart/form-data","text/plain"]);const Ne=["onClick","relative","reloadDocument","replace","state","target","to","preventScrollReset","unstable_viewTransition"],Ae=["aria-current","caseSensitive","className","end","style","to","unstable_viewTransition","children"];try{window.__reactRouterVersion="6"}catch(ap){}const Me=r.createContext({isTransitioning:!1});new Map;const Te=r.startTransition;r.flushSync,r.useId;function Le(e){let{basename:t,children:n,future:a,window:i}=e,o=r.useRef();null==o.current&&(o.current=function(e){return void 0===e&&(e={}),b((function(e,t){let{pathname:n="/",search:r="",hash:a=""}=_(e.location.hash.substr(1));return n.startsWith("/")||n.startsWith(".")||(n="/"+n),g("",{pathname:n,search:r,hash:a},t.state&&t.state.usr||null,t.state&&t.state.key||"default")}),(function(e,t){let n=e.document.querySelector("base"),r="";if(n&&n.getAttribute("href")){let t=e.location.href,n=t.indexOf("#");r=-1===n?t:t.slice(0,n)}return r+"#"+("string"===typeof t?t:y(t))}),(function(e,t){f("/"===e.pathname.charAt(0),"relative pathnames are not supported in hash history.push("+JSON.stringify(t)+")")}),e)}({window:i,v5Compat:!0}));let l=o.current,[s,c]=r.useState({action:l.action,location:l.location}),{v7_startTransition:u}=a||{},d=r.useCallback((e=>{u&&Te?Te((()=>c(e))):c(e)}),[c,u]);return r.useLayoutEffect((()=>l.listen(d)),[l,d]),r.createElement(we,{basename:t,children:n,location:s.location,navigationType:s.action,navigator:l,future:a})}const Oe="undefined"!==typeof window&&"undefined"!==typeof window.document&&"undefined"!==typeof window.document.createElement,Pe=/^(?:[a-z][a-z0-9+.-]*:|\/\/)/i,Ie=r.forwardRef((function(e,t){let n,{onClick:a,relative:i,reloadDocument:o,replace:l,state:s,target:c,to:u,preventScrollReset:d,unstable_viewTransition:h}=e,m=Ce(e,Ne),{basename:f}=r.useContext(G),v=!1;if("string"===typeof u&&Pe.test(u)&&(n=u,Oe))try{let e=new URL(window.location.href),t=u.startsWith("//")?new URL(e.protocol+u):new URL(u),n=D(t.pathname,f);t.origin===e.origin&&null!=n?u=n+t.search+t.hash:v=!0}catch(ap){}let g=function(e,t){let{relative:n}=void 0===t?{}:t;te()||p(!1);let{basename:a,navigator:i}=r.useContext(G),{hash:o,pathname:l,search:s}=oe(e,{relative:n}),c=l;return"/"!==a&&(c="/"===l?a:H([a,l])),i.createHref({pathname:c,search:s,hash:o})}(u,{relative:i}),_=function(e,t){let{target:n,replace:a,state:i,preventScrollReset:o,relative:l,unstable_viewTransition:s}=void 0===t?{}:t,c=ae(),u=ne(),d=oe(e,{relative:l});return r.useCallback((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,n)){t.preventDefault();let n=void 0!==a?a:y(u)===y(d);c(e,{replace:n,state:i,preventScrollReset:o,relative:l,unstable_viewTransition:s})}}),[u,c,d,a,i,n,e,o,l,s])}(u,{replace:l,state:s,target:c,preventScrollReset:d,relative:i,unstable_viewTransition:h});return r.createElement("a",Se({},m,{href:n||g,onClick:v||o?a:function(e){a&&a(e),e.defaultPrevented||_(e)},ref:t,target:c}))}));const Re=r.forwardRef((function(e,t){let{"aria-current":n="page",caseSensitive:a=!1,className:i="",end:o=!1,style:l,to:s,unstable_viewTransition:c,children:u}=e,d=Ce(e,Ae),h=oe(s,{relative:d.relative}),m=ne(),f=r.useContext(Z),{navigator:v,basename:g}=r.useContext(G),y=null!=f&&function(e,t){void 0===t&&(t={});let n=r.useContext(Me);null==n&&p(!1);let{basename:a}=Fe(De.useViewTransitionState),i=oe(e,{relative:t.relative});if(!n.isTransitioning)return!1;let o=D(n.currentLocation.pathname,a)||n.currentLocation.pathname,l=D(n.nextLocation.pathname,a)||n.nextLocation.pathname;return null!=I(i.pathname,l)||null!=I(i.pathname,o)}(h)&&!0===c,_=v.encodeLocation?v.encodeLocation(h).pathname:h.pathname,b=m.pathname,w=f&&f.navigation&&f.navigation.location?f.navigation.location.pathname:null;a||(b=b.toLowerCase(),w=w?w.toLowerCase():null,_=_.toLowerCase()),w&&g&&(w=D(w,g)||w);const k="/"!==_&&_.endsWith("/")?_.length-1:_.length;let x,S=b===_||!o&&b.startsWith(_)&&"/"===b.charAt(k),C=null!=w&&(w===_||!o&&w.startsWith(_)&&"/"===w.charAt(_.length)),E={isActive:S,isPending:C,isTransitioning:y},N=S?n:void 0;x="function"===typeof i?i(E):[i,S?"active":null,C?"pending":null,y?"transitioning":null].filter(Boolean).join(" ");let A="function"===typeof l?l(E):l;return r.createElement(Ie,Se({},d,{"aria-current":N,className:x,ref:t,style:A,to:s,unstable_viewTransition:c}),"function"===typeof u?u(E):u)}));var De,ze;function Fe(e){let t=r.useContext(Q);return t||p(!1),t}function je(e){let t=r.useRef(Ee(e)),n=r.useRef(!1),a=ne(),i=r.useMemo((()=>function(e,t){let n=Ee(e);return t&&t.forEach(((e,r)=>{n.has(r)||t.getAll(r).forEach((e=>{n.append(r,e)}))})),n}(a.search,n.current?null:t.current)),[a.search]),o=ae(),l=r.useCallback(((e,t)=>{const r=Ee("function"===typeof e?e(i):e);n.current=!0,o("?"+r,t)}),[o,i]);return[i,l]}(function(e){e.UseScrollRestoration="useScrollRestoration",e.UseSubmit="useSubmit",e.UseSubmitFetcher="useSubmitFetcher",e.UseFetcher="useFetcher",e.useViewTransitionState="useViewTransitionState"})(De||(De={})),function(e){e.UseFetcher="useFetcher",e.UseFetchers="useFetchers",e.UseScrollRestoration="useScrollRestoration"}(ze||(ze={}));let $e=function(e){return e.logs="logs",e.anomaly="anomaly",e}({});const He={home:"/",metrics:"/metrics",dashboards:"/dashboards",cardinality:"/cardinality",topQueries:"/top-queries",trace:"/trace",withTemplate:"/expand-with-exprs",relabel:"/relabeling",logs:"/logs",activeQueries:"/active-queries",queryAnalyzer:"/query-analyzer",icons:"/icons",anomaly:"/anomaly",query:"/query"},{REACT_APP_TYPE:Ve}={},Ue=Ve===$e.logs,Be={header:{tenant:!0,stepControl:!Ue,timeSelector:!Ue,executionControls:!Ue}},qe={[He.home]:{title:"Query",...Be},[He.metrics]:{title:"Explore Prometheus metrics",header:{tenant:!0,stepControl:!0,timeSelector:!0}},[He.cardinality]:{title:"Explore cardinality",header:{tenant:!0,cardinalityDatePicker:!0}},[He.topQueries]:{title:"Top queries",header:{tenant:!0}},[He.trace]:{title:"Trace analyzer",header:{}},[He.queryAnalyzer]:{title:"Query analyzer",header:{}},[He.dashboards]:{title:"Dashboards",...Be},[He.withTemplate]:{title:"WITH templates",header:{}},[He.relabel]:{title:"Metric relabel debug",header:{}},[He.logs]:{title:"Logs Explorer",header:{}},[He.activeQueries]:{title:"Active Queries",header:{}},[He.icons]:{title:"Icons",header:{}},[He.anomaly]:{title:"Anomaly exploration",...Be},[He.query]:{title:"Query",...Be}},Ye=He,We=()=>{var e;const t=(null===(e=document.getElementById("root"))||void 0===e?void 0:e.dataset.params)||"{}";try{return JSON.parse(t)}catch(ap){return console.error(ap),{}}},Ke=()=>!!Object.keys(We()).length,Qe=/(\/select\/)(\d+|\d.+)(\/)(.+)/,Ze=(e,t)=>e.replace(Qe,"$1".concat(t,"/$4")),Ge=e=>{var t;return(null===(t=e.match(Qe))||void 0===t?void 0:t[2])||""},Je=(e,t)=>{t?window.localStorage.setItem(e,JSON.stringify({value:t})):et([e]),window.dispatchEvent(new Event("storage"))},Xe=e=>{const t=window.localStorage.getItem(e);if(null!==t)try{var n;return null===(n=JSON.parse(t))||void 0===n?void 0:n.value}catch(ap){return t}},et=e=>e.forEach((e=>window.localStorage.removeItem(e))),{REACT_APP_TYPE:tt}={};var nt=n(215),rt=n.n(nt),at=n(424),it=n.n(at);const ot={table:100,chart:20,code:1e3},lt=[{id:"small",isDefault:!0,height:()=>.2*window.innerHeight},{id:"medium",height:()=>.4*window.innerHeight},{id:"large",height:()=>.8*window.innerHeight}],st=["min","median","max"],ct=(e,t)=>{const n=window.location.hash.split("?")[1],r=rt().parse(n,{ignoreQueryPrefix:!0});return it()(r,e,t||"")};let ut=function(e){return e.yhat="yhat",e.yhatUpper="yhat_upper",e.yhatLower="yhat_lower",e.anomaly="vmui_anomalies_points",e.training="vmui_training_data",e.actual="actual",e.anomalyScore="anomaly_score",e}({}),dt=function(e){return e.table="table",e.chart="chart",e.code="code",e}({}),ht=function(e){return e.emptyServer="Please enter Server URL",e.validServer="Please provide a valid Server URL",e.validQuery="Please enter a valid Query and execute it",e.traceNotFound="Not found the tracing information",e.emptyTitle="Please enter title",e.positiveNumber="Please enter positive number",e.validStep="Please enter a valid step",e.unknownType="Unknown server response format: must have 'errorType'",e}({}),mt=function(e){return e.system="system",e.light="light",e.dark="dark",e}({}),pt=function(e){return e.empty="empty",e.metricsql="metricsql",e.label="label",e.labelValue="labelValue",e}({});const ft=e=>getComputedStyle(document.documentElement).getPropertyValue("--".concat(e)),vt=(e,t)=>{document.documentElement.style.setProperty("--".concat(e),t)},gt=()=>window.matchMedia("(prefers-color-scheme: dark)").matches,yt=e=>{let t;try{t=new URL(e)}catch(n){return!1}return"http:"===t.protocol||"https:"===t.protocol},_t=e=>e.replace(/\/$/,""),bt=ct("g0.tenantID",""),wt={serverUrl:_t((e=>{const{serverURL:t}=We(),n=Xe("SERVER_URL"),r=window.location.href.replace(/\/(select\/)?(vmui)\/.*/,""),a=window.location.href.replace(/(?:graph|vmui)\/.*/,""),i=window.location.href.replace(/\/(?:prometheus\/)?(?:graph|vmui)\/.*/,"/prometheus"),o=t||n||i;switch(tt){case $e.logs:return r;case $e.anomaly:return t||n||a;default:return e?Ze(o,e):o}})(bt)),tenantId:bt,theme:Xe("THEME")||mt.system,isDarkTheme:null};function kt(e,t){switch(t.type){case"SET_SERVER":return{...e,serverUrl:_t(t.payload)};case"SET_TENANT_ID":return{...e,tenantId:t.payload};case"SET_THEME":return Je("THEME",t.payload),{...e,theme:t.payload};case"SET_DARK_THEME":return{...e,isDarkTheme:(n=e.theme,n===mt.system&>()||n===mt.dark)};default:throw new Error}var n}var xt=n(746);var St=0;Array.isArray;function Ct(e,t,n,r,a,i){var o,l,s={};for(l in t)"ref"==l?o=t[l]:s[l]=t[l];var c={type:e,props:s,key:n,ref:o,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,constructor:void 0,__v:--St,__i:-1,__u:0,__source:a,__self:i};if("function"==typeof e&&(o=e.defaultProps))for(l in o)void 0===s[l]&&(s[l]=o[l]);return xt.fF.vnode&&xt.fF.vnode(c),c}const Et=(0,r.createContext)({}),Nt=()=>(0,r.useContext)(Et).state,At=()=>(0,r.useContext)(Et).dispatch,Mt=Object.entries(wt).reduce(((e,t)=>{let[n,r]=t;return{...e,[n]:ct(n)||r}}),{}),Tt="YYYY-MM-DD",Lt="YYYY-MM-DD HH:mm:ss",Ot="YYYY-MM-DD HH:mm:ss:SSS (Z)",Pt="YYYY-MM-DD[T]HH:mm:ss",It="YYYY-MM-DD_HHmmss",Rt=window.innerWidth/4,Dt=window.innerWidth/40,zt=1,Ft=1578e8,jt=Intl.supportedValuesOf,$t=jt?jt("timeZone"):["Africa/Abidjan","Africa/Accra","Africa/Addis_Ababa","Africa/Algiers","Africa/Asmera","Africa/Bamako","Africa/Bangui","Africa/Banjul","Africa/Bissau","Africa/Blantyre","Africa/Brazzaville","Africa/Bujumbura","Africa/Cairo","Africa/Casablanca","Africa/Ceuta","Africa/Conakry","Africa/Dakar","Africa/Dar_es_Salaam","Africa/Djibouti","Africa/Douala","Africa/El_Aaiun","Africa/Freetown","Africa/Gaborone","Africa/Harare","Africa/Johannesburg","Africa/Juba","Africa/Kampala","Africa/Khartoum","Africa/Kigali","Africa/Kinshasa","Africa/Lagos","Africa/Libreville","Africa/Lome","Africa/Luanda","Africa/Lubumbashi","Africa/Lusaka","Africa/Malabo","Africa/Maputo","Africa/Maseru","Africa/Mbabane","Africa/Mogadishu","Africa/Monrovia","Africa/Nairobi","Africa/Ndjamena","Africa/Niamey","Africa/Nouakchott","Africa/Ouagadougou","Africa/Porto-Novo","Africa/Sao_Tome","Africa/Tripoli","Africa/Tunis","Africa/Windhoek","America/Adak","America/Anchorage","America/Anguilla","America/Antigua","America/Araguaina","America/Argentina/La_Rioja","America/Argentina/Rio_Gallegos","America/Argentina/Salta","America/Argentina/San_Juan","America/Argentina/San_Luis","America/Argentina/Tucuman","America/Argentina/Ushuaia","America/Aruba","America/Asuncion","America/Bahia","America/Bahia_Banderas","America/Barbados","America/Belem","America/Belize","America/Blanc-Sablon","America/Boa_Vista","America/Bogota","America/Boise","America/Buenos_Aires","America/Cambridge_Bay","America/Campo_Grande","America/Cancun","America/Caracas","America/Catamarca","America/Cayenne","America/Cayman","America/Chicago","America/Chihuahua","America/Coral_Harbour","America/Cordoba","America/Costa_Rica","America/Creston","America/Cuiaba","America/Curacao","America/Danmarkshavn","America/Dawson","America/Dawson_Creek","America/Denver","America/Detroit","America/Dominica","America/Edmonton","America/Eirunepe","America/El_Salvador","America/Fort_Nelson","America/Fortaleza","America/Glace_Bay","America/Godthab","America/Goose_Bay","America/Grand_Turk","America/Grenada","America/Guadeloupe","America/Guatemala","America/Guayaquil","America/Guyana","America/Halifax","America/Havana","America/Hermosillo","America/Indiana/Knox","America/Indiana/Marengo","America/Indiana/Petersburg","America/Indiana/Tell_City","America/Indiana/Vevay","America/Indiana/Vincennes","America/Indiana/Winamac","America/Indianapolis","America/Inuvik","America/Iqaluit","America/Jamaica","America/Jujuy","America/Juneau","America/Kentucky/Monticello","America/Kralendijk","America/La_Paz","America/Lima","America/Los_Angeles","America/Louisville","America/Lower_Princes","America/Maceio","America/Managua","America/Manaus","America/Marigot","America/Martinique","America/Matamoros","America/Mazatlan","America/Mendoza","America/Menominee","America/Merida","America/Metlakatla","America/Mexico_City","America/Miquelon","America/Moncton","America/Monterrey","America/Montevideo","America/Montreal","America/Montserrat","America/Nassau","America/New_York","America/Nipigon","America/Nome","America/Noronha","America/North_Dakota/Beulah","America/North_Dakota/Center","America/North_Dakota/New_Salem","America/Ojinaga","America/Panama","America/Pangnirtung","America/Paramaribo","America/Phoenix","America/Port-au-Prince","America/Port_of_Spain","America/Porto_Velho","America/Puerto_Rico","America/Punta_Arenas","America/Rainy_River","America/Rankin_Inlet","America/Recife","America/Regina","America/Resolute","America/Rio_Branco","America/Santa_Isabel","America/Santarem","America/Santiago","America/Santo_Domingo","America/Sao_Paulo","America/Scoresbysund","America/Sitka","America/St_Barthelemy","America/St_Johns","America/St_Kitts","America/St_Lucia","America/St_Thomas","America/St_Vincent","America/Swift_Current","America/Tegucigalpa","America/Thule","America/Thunder_Bay","America/Tijuana","America/Toronto","America/Tortola","America/Vancouver","America/Whitehorse","America/Winnipeg","America/Yakutat","America/Yellowknife","Antarctica/Casey","Antarctica/Davis","Antarctica/DumontDUrville","Antarctica/Macquarie","Antarctica/Mawson","Antarctica/McMurdo","Antarctica/Palmer","Antarctica/Rothera","Antarctica/Syowa","Antarctica/Troll","Antarctica/Vostok","Arctic/Longyearbyen","Asia/Aden","Asia/Almaty","Asia/Amman","Asia/Anadyr","Asia/Aqtau","Asia/Aqtobe","Asia/Ashgabat","Asia/Atyrau","Asia/Baghdad","Asia/Bahrain","Asia/Baku","Asia/Bangkok","Asia/Barnaul","Asia/Beirut","Asia/Bishkek","Asia/Brunei","Asia/Calcutta","Asia/Chita","Asia/Choibalsan","Asia/Colombo","Asia/Damascus","Asia/Dhaka","Asia/Dili","Asia/Dubai","Asia/Dushanbe","Asia/Famagusta","Asia/Gaza","Asia/Hebron","Asia/Hong_Kong","Asia/Hovd","Asia/Irkutsk","Asia/Jakarta","Asia/Jayapura","Asia/Jerusalem","Asia/Kabul","Asia/Kamchatka","Asia/Karachi","Asia/Katmandu","Asia/Khandyga","Asia/Krasnoyarsk","Asia/Kuala_Lumpur","Asia/Kuching","Asia/Kuwait","Asia/Macau","Asia/Magadan","Asia/Makassar","Asia/Manila","Asia/Muscat","Asia/Nicosia","Asia/Novokuznetsk","Asia/Novosibirsk","Asia/Omsk","Asia/Oral","Asia/Phnom_Penh","Asia/Pontianak","Asia/Pyongyang","Asia/Qatar","Asia/Qostanay","Asia/Qyzylorda","Asia/Rangoon","Asia/Riyadh","Asia/Saigon","Asia/Sakhalin","Asia/Samarkand","Asia/Seoul","Asia/Shanghai","Asia/Singapore","Asia/Srednekolymsk","Asia/Taipei","Asia/Tashkent","Asia/Tbilisi","Asia/Tehran","Asia/Thimphu","Asia/Tokyo","Asia/Tomsk","Asia/Ulaanbaatar","Asia/Urumqi","Asia/Ust-Nera","Asia/Vientiane","Asia/Vladivostok","Asia/Yakutsk","Asia/Yekaterinburg","Asia/Yerevan","Atlantic/Azores","Atlantic/Bermuda","Atlantic/Canary","Atlantic/Cape_Verde","Atlantic/Faeroe","Atlantic/Madeira","Atlantic/Reykjavik","Atlantic/South_Georgia","Atlantic/St_Helena","Atlantic/Stanley","Australia/Adelaide","Australia/Brisbane","Australia/Broken_Hill","Australia/Currie","Australia/Darwin","Australia/Eucla","Australia/Hobart","Australia/Lindeman","Australia/Lord_Howe","Australia/Melbourne","Australia/Perth","Australia/Sydney","Europe/Amsterdam","Europe/Andorra","Europe/Astrakhan","Europe/Athens","Europe/Belgrade","Europe/Berlin","Europe/Bratislava","Europe/Brussels","Europe/Bucharest","Europe/Budapest","Europe/Busingen","Europe/Chisinau","Europe/Copenhagen","Europe/Dublin","Europe/Gibraltar","Europe/Guernsey","Europe/Helsinki","Europe/Isle_of_Man","Europe/Istanbul","Europe/Jersey","Europe/Kaliningrad","Europe/Kiev","Europe/Kirov","Europe/Lisbon","Europe/Ljubljana","Europe/London","Europe/Luxembourg","Europe/Madrid","Europe/Malta","Europe/Mariehamn","Europe/Minsk","Europe/Monaco","Europe/Moscow","Europe/Oslo","Europe/Paris","Europe/Podgorica","Europe/Prague","Europe/Riga","Europe/Rome","Europe/Samara","Europe/San_Marino","Europe/Sarajevo","Europe/Saratov","Europe/Simferopol","Europe/Skopje","Europe/Sofia","Europe/Stockholm","Europe/Tallinn","Europe/Tirane","Europe/Ulyanovsk","Europe/Uzhgorod","Europe/Vaduz","Europe/Vatican","Europe/Vienna","Europe/Vilnius","Europe/Volgograd","Europe/Warsaw","Europe/Zagreb","Europe/Zaporozhye","Europe/Zurich","Indian/Antananarivo","Indian/Chagos","Indian/Christmas","Indian/Cocos","Indian/Comoro","Indian/Kerguelen","Indian/Mahe","Indian/Maldives","Indian/Mauritius","Indian/Mayotte","Indian/Reunion","Pacific/Apia","Pacific/Auckland","Pacific/Bougainville","Pacific/Chatham","Pacific/Easter","Pacific/Efate","Pacific/Enderbury","Pacific/Fakaofo","Pacific/Fiji","Pacific/Funafuti","Pacific/Galapagos","Pacific/Gambier","Pacific/Guadalcanal","Pacific/Guam","Pacific/Honolulu","Pacific/Johnston","Pacific/Kiritimati","Pacific/Kosrae","Pacific/Kwajalein","Pacific/Majuro","Pacific/Marquesas","Pacific/Midway","Pacific/Nauru","Pacific/Niue","Pacific/Norfolk","Pacific/Noumea","Pacific/Pago_Pago","Pacific/Palau","Pacific/Pitcairn","Pacific/Ponape","Pacific/Port_Moresby","Pacific/Rarotonga","Pacific/Saipan","Pacific/Tahiti","Pacific/Tarawa","Pacific/Tongatapu","Pacific/Truk","Pacific/Wake","Pacific/Wallis"],Ht=[{long:"years",short:"y",possible:"year"},{long:"weeks",short:"w",possible:"week"},{long:"days",short:"d",possible:"day"},{long:"hours",short:"h",possible:"hour"},{long:"minutes",short:"m",possible:"min"},{long:"seconds",short:"s",possible:"sec"},{long:"milliseconds",short:"ms",possible:"millisecond"}],Vt=Ht.map((e=>e.short)),Ut=e=>Math.round(1e3*e)/1e3,Bt=e=>Jt(i().duration(e,"seconds").asMilliseconds()),qt=e=>{let t=Ut(e);const n=Math.round(e);e>=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);return Bt(t||.001).replace(/\s/g,"")},Yt=e=>{const t=e.match(/\d+/g),n=e.match(/[a-zA-Z]+/g);if(n&&t&&Vt.includes(n[0]))return{[n[0]]:t[0]}},Wt=e=>{const t=Ht.map((e=>e.short)).join("|"),n=new RegExp("\\d+(\\.\\d+)?[".concat(t,"]+"),"g"),r=(e.match(n)||[]).reduce(((e,t)=>{const n=Yt(t);return n?{...e,...n}:{...e}}),{});return i().duration(r).asSeconds()},Kt=(e,t)=>qt(e/(t?Dt:Rt)),Qt=(e,t)=>{const n=(t||i()().toDate()).valueOf()/1e3,r=Wt(e);return{start:n-r,end:n,step:Kt(r),date:Zt(t||i()().toDate())}},Zt=e=>i().tz(e).utc().format(Pt),Gt=e=>i().tz(e).format(Pt),Jt=e=>{const t=Math.floor(e%1e3),n=Math.floor(e/1e3%60),r=Math.floor(e/1e3/60%60),a=Math.floor(e/1e3/3600%24),i=Math.floor(e/864e5),o=["d","h","m","s","ms"],l=[i,a,r,n,t].map(((e,t)=>e?"".concat(e).concat(o[t]):""));return l.filter((e=>e)).join("")},Xt=e=>{const t=i()(1e3*e);return t.isValid()?t.toDate():new Date},en=[{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:()=>i()().tz().subtract(1,"day").endOf("day").toDate()},{title:"Today",duration:"1d",until:()=>i()().tz().endOf("day").toDate()}].map((e=>({id:e.title.replace(/\s/g,"_").toLocaleLowerCase(),until:e.until?e.until:()=>i()().tz().toDate(),...e}))),tn=e=>{var t;let{relativeTimeId:n,defaultDuration:r,defaultEndInput:a}=e;const i=null===(t=en.find((e=>e.isDefault)))||void 0===t?void 0:t.id,o=n||ct("g0.relative_time",i),l=en.find((e=>e.id===o));return{relativeTimeId:l?o:"none",duration:l?l.duration:r,endInput:l?l.until():a}},nn=e=>{const t=i()().tz(e);return"UTC".concat(t.format("Z"))},rn=function(){let e=arguments.length>0&&void 0!==arguments[0]?arguments[0]:"";const t=new RegExp(e,"i");return $t.reduce(((n,r)=>{const a=(r.match(/^(.*?)\//)||[])[1]||"unknown",i=nn(r),o=i.replace(/UTC|0/,""),l=r.replace(/[/_]/g," "),s={region:r,utc:i,search:"".concat(r," ").concat(i," ").concat(l," ").concat(o)},c=!e||e&&t.test(s.search);return c&&n[a]?n[a].push(s):c&&(n[a]=[s]),n}),{})},an=e=>{i().tz.setDefault(e)},on=()=>{const e=i().tz.guess(),t=(e=>{try{return i()().tz(e),!0}catch(ap){return!1}})(e);return{isValid:t,title:t?"Browser Time (".concat(e,")"):"Browser timezone (UTC)",region:t?e:"UTC"}},ln=Xe("TIMEZONE")||on().region;an(ln);const sn=ct("g0.range_input"),{duration:cn,endInput:un,relativeTimeId:dn}=tn({defaultDuration:sn||"1h",defaultEndInput:(hn=ct("g0.end_input",i()().utc().format(Pt)),i()(hn).utcOffset(0,!0).toDate()),relativeTimeId:sn?ct("g0.relative_time","none"):void 0});var hn;const mn={duration:cn,period:Qt(cn,un),relativeTime:dn,timezone:ln};function pn(e,t){switch(t.type){case"SET_DURATION":return{...e,duration:t.payload,period:Qt(t.payload,Xt(e.period.end)),relativeTime:"none"};case"SET_RELATIVE_TIME":return{...e,duration:t.payload.duration,period:Qt(t.payload.duration,t.payload.until),relativeTime:t.payload.id};case"SET_PERIOD":const n=(e=>{const t=e.to.valueOf()-e.from.valueOf();return Jt(t)})(t.payload);return{...e,duration:n,period:Qt(n,t.payload.to),relativeTime:"none"};case"RUN_QUERY":const{duration:r,endInput:a}=tn({relativeTimeId:e.relativeTime,defaultDuration:e.duration,defaultEndInput:Xt(e.period.end)});return{...e,period:Qt(r,a)};case"RUN_QUERY_TO_NOW":return{...e,period:Qt(e.duration)};case"SET_TIMEZONE":return an(t.payload),Je("TIMEZONE",t.payload),e.defaultTimezone&&Je("DISABLED_DEFAULT_TIMEZONE",t.payload!==e.defaultTimezone),{...e,timezone:t.payload};case"SET_DEFAULT_TIMEZONE":return{...e,defaultTimezone:t.payload};default:throw new Error}}const fn=(0,r.createContext)({}),vn=()=>(0,r.useContext)(fn).state,gn=()=>(0,r.useContext)(fn).dispatch,yn=e=>{const t=Xe(e);return t?JSON.parse(t):[]},_n=50,bn=1e3,wn=1e3;const kn=(()=>{var e;const 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(((e,t)=>ct("g".concat(t,".expr"),"")))})(),xn={query:kn,queryHistory:kn.map((e=>({index:0,values:[e]}))),autocomplete:Xe("AUTOCOMPLETE")||!1,autocompleteQuick:!1,autocompleteCache:new class{constructor(){this.maxSize=void 0,this.map=void 0,this.maxSize=wn,this.map=new Map}get(e){for(const[t,n]of this.map){const r=JSON.parse(t),a=r.start===e.start&&r.end===e.end,i=r.type===e.type,o=e.value&&r.value&&e.value.includes(r.value),l=r.match===e.match||o,s=n.length=this.maxSize){const e=this.map.keys().next().value;this.map.delete(e)}this.map.set(JSON.stringify(e),t)}},metricsQLFunctions:[]};function Sn(e,t){switch(t.type){case"SET_QUERY":return{...e,query:t.payload.map((e=>e))};case"SET_QUERY_HISTORY":return(e=>{const t=e.map((e=>e.values[e.index])),n=yn("QUERY_HISTORY");n[0]||(n[0]=[]);const r=n[0];t.forEach((e=>{!r.includes(e)&&e&&r.unshift(e),r.length>100&&r.shift()})),Je("QUERY_HISTORY",JSON.stringify(n))})(t.payload),{...e,queryHistory:t.payload};case"SET_QUERY_HISTORY_BY_INDEX":return e.queryHistory.splice(t.payload.queryNumber,1,t.payload.value),{...e,queryHistory:e.queryHistory};case"TOGGLE_AUTOCOMPLETE":return Je("AUTOCOMPLETE",!e.autocomplete),{...e,autocomplete:!e.autocomplete};case"SET_AUTOCOMPLETE_QUICK":return{...e,autocompleteQuick:t.payload};case"SET_AUTOCOMPLETE_CACHE":return e.autocompleteCache.put(t.payload.key,t.payload.value),{...e};case"SET_METRICSQL_FUNCTIONS":return{...e,metricsQLFunctions:t.payload};default:throw new Error}}const Cn=(0,r.createContext)({}),En=()=>(0,r.useContext)(Cn).state,Nn=()=>(0,r.useContext)(Cn).dispatch,An=()=>Ct("svg",{viewBox:"0 0 74 24",fill:"currentColor",children:Ct("path",{d:"M6.12 10.48c.36.28.8.43 1.26.43h.05c.48 0 .96-.19 1.25-.44 1.5-1.28 5.88-5.29 5.88-5.29C15.73 4.1 12.46 3.01 7.43 3h-.06C2.33 3-.93 4.1.24 5.18c0 0 4.37 4 5.88 5.3Zm2.56 2.16c-.36.28-.8.44-1.26.45h-.04c-.46 0-.9-.17-1.26-.45-1.04-.88-4.74-4.22-6.12-5.5v1.94c0 .21.08.5.22.63l.07.06c1.05.96 4.55 4.16 5.83 5.25.36.28.8.43 1.26.44h.04c.49-.02.96-.2 1.26-.44 1.3-1.11 4.94-4.45 5.88-5.31.15-.14.23-.42.23-.63V7.15a454.94 454.94 0 0 1-6.11 5.5Zm-1.26 4.99c.46 0 .9-.16 1.26-.44a454.4 454.4 0 0 0 6.1-5.5v1.94c0 .2-.07.48-.22.62-.94.87-4.57 4.2-5.88 5.3-.3.26-.77.44-1.26.45h-.04c-.46 0-.9-.16-1.26-.44-1.2-1.02-4.38-3.92-5.62-5.06l-.28-.25c-.14-.14-.22-.42-.22-.62v-1.94c1.38 1.26 5.08 4.6 6.12 5.5.36.28.8.43 1.26.44h.04ZM35 5l-5.84 14.46h-2.43L20.89 5h2.16a.9.9 0 0 1 .9.61l3.41 8.82a18.8 18.8 0 0 1 .62 2.02 19.44 19.44 0 0 1 .57-2.02l3.39-8.82c.05-.15.16-.3.31-.42a.9.9 0 0 1 .58-.19H35Zm17.18 0v14.46H49.8v-9.34c0-.37.02-.78.06-1.21l-4.37 8.21c-.21.4-.53.59-.95.59h-.38c-.43 0-.75-.2-.95-.59L38.8 8.88a22.96 22.96 0 0 1 .07 1.24v9.34H36.5V5h2.03l.3.01c.1 0 .17.02.24.05.07.03.13.07.19.13a1 1 0 0 1 .17.24l4.33 8.03a16.97 16.97 0 0 1 .6 1.36 14.34 14.34 0 0 1 .6-1.38l4.28-8.01c.05-.1.1-.18.17-.24.06-.06.12-.1.19-.13a.9.9 0 0 1 .24-.05l.3-.01h2.04Zm8.88 13.73a4.5 4.5 0 0 0 1.82-.35 3.96 3.96 0 0 0 2.22-2.47c.2-.57.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.12 1.51-.37 2.19a4.88 4.88 0 0 1-2.76 2.95c-.66.29-1.4.43-2.23.43-.82 0-1.57-.14-2.24-.43a5.01 5.01 0 0 1-2.75-2.95 6.37 6.37 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.1 1.28.3 1.85a3.98 3.98 0 0 0 2.21 2.47c.53.24 1.14.36 1.82.36Zm10.38.73h-1.03V5.31h1.03v14.15Z"})}),Mn=()=>Ct("svg",{viewBox:"0 0 85 38",fill:"currentColor",children:[Ct("path",{d:"M11.12 10.48c.36.28.8.43 1.26.43h.05c.48 0 .96-.19 1.25-.44 1.5-1.28 5.88-5.29 5.88-5.29 1.17-1.09-2.1-2.17-7.13-2.18h-.06c-5.04 0-8.3 1.1-7.13 2.18 0 0 4.37 4 5.88 5.3Zm2.56 2.16c-.36.28-.8.44-1.26.45h-.04c-.46 0-.9-.17-1.26-.45-1.04-.88-4.74-4.22-6.12-5.5v1.94c0 .21.08.5.22.63l.07.06c1.05.96 4.55 4.16 5.83 5.25.36.28.8.43 1.26.44h.04c.49-.02.96-.2 1.26-.44 1.3-1.11 4.94-4.45 5.88-5.31.15-.14.23-.42.23-.63V7.15a455.13 455.13 0 0 1-6.11 5.5Zm-1.26 4.99c.46 0 .9-.16 1.26-.44 2.05-1.82 4.09-3.65 6.1-5.5v1.94c0 .2-.07.48-.22.62-.94.87-4.57 4.2-5.88 5.3-.3.26-.77.44-1.26.45h-.04c-.46 0-.9-.16-1.26-.44-1.2-1.02-4.38-3.92-5.62-5.06l-.28-.25c-.14-.14-.22-.42-.22-.62v-1.94c1.38 1.26 5.08 4.6 6.12 5.5.36.28.8.43 1.26.44h.04ZM40 5l-5.84 14.46h-2.43L25.89 5h2.16a.9.9 0 0 1 .9.61l3.41 8.82a18.8 18.8 0 0 1 .62 2.02 19.44 19.44 0 0 1 .57-2.02l3.39-8.82c.05-.15.16-.3.31-.42a.9.9 0 0 1 .58-.19H40Zm17.18 0v14.46H54.8v-9.34c0-.37.02-.78.06-1.21l-4.37 8.21c-.21.4-.53.59-.95.59h-.38c-.43 0-.75-.2-.95-.59L43.8 8.88a22.96 22.96 0 0 1 .07 1.24v9.34H41.5V5h2.03l.3.01c.1 0 .17.02.24.05.07.03.13.07.19.13a1 1 0 0 1 .17.24l4.33 8.03a16.97 16.97 0 0 1 .6 1.36 14.34 14.34 0 0 1 .6-1.38l4.28-8.01c.05-.1.1-.18.17-.24.06-.06.12-.1.19-.13a.9.9 0 0 1 .24-.05l.3-.01h2.04Zm8.88 13.73a4.5 4.5 0 0 0 1.82-.35 3.96 3.96 0 0 0 2.22-2.47c.2-.57.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.12 1.51-.37 2.19a4.88 4.88 0 0 1-2.76 2.95c-.66.29-1.4.43-2.23.43-.82 0-1.57-.14-2.24-.43a5.01 5.01 0 0 1-2.75-2.95 6.37 6.37 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.1 1.28.3 1.85a3.98 3.98 0 0 0 2.21 2.47c.53.24 1.14.36 1.82.36Zm10.38.73h-1.03V5.31h1.03v14.15ZM1.73 36v-5.17l-.67-.07a.6.6 0 0 1-.21-.1.23.23 0 0 1-.08-.18v-.44h.96v-.59c0-.34.05-.65.14-.92a1.79 1.79 0 0 1 1.08-1.11 2.45 2.45 0 0 1 1.62-.02l-.03.53c0 .1-.06.15-.16.16H4c-.18 0-.35.03-.5.08a.95.95 0 0 0-.39.23c-.1.11-.19.25-.25.43-.05.18-.08.4-.08.65v.56h1.75v.78H2.8V36H1.73Zm6.17-6.17c.45 0 .85.07 1.2.22a2.57 2.57 0 0 1 1.5 1.62c.13.38.2.81.2 1.29s-.07.91-.2 1.3a2.57 2.57 0 0 1-1.49 1.61c-.36.14-.76.21-1.2.21-.45 0-.86-.07-1.22-.21a2.57 2.57 0 0 1-1.5-1.62c-.12-.38-.19-.81-.19-1.3 0-.47.07-.9.2-1.28a2.57 2.57 0 0 1 1.5-1.62c.35-.15.76-.22 1.2-.22Zm0 5.42c.6 0 1.05-.2 1.35-.6.3-.4.44-.97.44-1.69s-.15-1.28-.44-1.69c-.3-.4-.75-.6-1.35-.6-.3 0-.57.05-.8.15-.22.1-.4.26-.56.45-.15.2-.26.44-.33.73-.08.28-.11.6-.11.96 0 .72.15 1.29.44 1.69.3.4.76.6 1.36.6Zm5.26-4.11c.2-.42.43-.74.71-.97.28-.24.62-.36 1.03-.36.13 0 .25.02.36.05.12.02.23.07.32.13l-.08.8c-.02.1-.08.15-.18.15l-.24-.04a1.7 1.7 0 0 0-.88.05c-.15.05-.29.14-.4.25-.12.1-.23.24-.32.4-.1.17-.18.35-.26.56V36h-1.07v-6.08h.61c.12 0 .2.02.24.07.05.04.08.12.1.23l.06.92Zm13.73-3.82L23.39 36h-1.46l-3.5-8.68h1.29a.54.54 0 0 1 .54.37l2.04 5.3a11.31 11.31 0 0 1 .37 1.21 11.65 11.65 0 0 1 .35-1.22l2.03-5.29c.03-.1.1-.18.19-.25.1-.08.21-.12.35-.12h1.3Zm2.2 2.52V36H27.6v-6.16h1.49Zm.2-1.79c0 .13-.02.25-.08.36a1 1 0 0 1-.51.5.96.96 0 0 1-.73 0 1.02 1.02 0 0 1-.5-.5.96.96 0 0 1 0-.73.93.93 0 0 1 .86-.58.9.9 0 0 1 .37.08c.12.05.22.11.3.2a.94.94 0 0 1 .3.67Zm5.72 3.1a.68.68 0 0 1-.13.13c-.04.03-.1.05-.18.05a.42.42 0 0 1-.22-.07 3.95 3.95 0 0 0-.62-.31c-.14-.05-.3-.07-.51-.07-.26 0-.5.04-.69.14-.2.1-.36.23-.49.4-.13.18-.22.4-.29.64-.06.25-.1.53-.1.85 0 .33.04.62.1.88.08.25.18.47.32.64.13.18.29.3.48.4.18.09.4.13.63.13a1.6 1.6 0 0 0 .94-.27l.26-.2a.4.4 0 0 1 .25-.09.3.3 0 0 1 .27.14l.43.54a2.76 2.76 0 0 1-1.77.96c-.22.03-.43.05-.65.05a2.57 2.57 0 0 1-1.96-.83c-.25-.28-.45-.6-.6-1-.14-.4-.21-.85-.21-1.35 0-.45.06-.87.2-1.25a2.61 2.61 0 0 1 1.51-1.67c.37-.16.8-.24 1.28-.24.46 0 .86.07 1.2.22.35.15.66.36.94.64l-.4.54Zm3.43 4.95c-.54 0-.95-.15-1.24-.45-.28-.3-.42-.73-.42-1.26v-3.44h-.63a.29.29 0 0 1-.2-.07c-.06-.06-.09-.13-.09-.24v-.59l.99-.16.31-1.68a.33.33 0 0 1 .12-.18.34.34 0 0 1 .21-.07h.77v1.94h1.64v1.05h-1.64v3.34c0 .2.05.34.14.45.1.1.22.16.39.16a.73.73 0 0 0 .39-.1l.12-.07a.2.2 0 0 1 .11-.03c.05 0 .08.01.11.03l.09.1.44.72c-.21.18-.46.32-.74.4-.28.1-.57.15-.87.15Zm5.09-6.35c.46 0 .87.07 1.24.22a2.7 2.7 0 0 1 1.58 1.63c.14.39.22.83.22 1.31 0 .49-.08.93-.22 1.32-.14.4-.35.73-.62 1-.26.28-.58.49-.96.64-.37.15-.78.22-1.24.22a3.4 3.4 0 0 1-1.25-.22 2.71 2.71 0 0 1-1.59-1.64 3.8 3.8 0 0 1-.21-1.32c0-.48.07-.92.21-1.31a2.75 2.75 0 0 1 1.58-1.63c.38-.15.8-.22 1.26-.22Zm0 5.2c.51 0 .89-.17 1.13-.52.25-.34.38-.84.38-1.5a2.6 2.6 0 0 0-.38-1.53c-.24-.34-.62-.52-1.13-.52-.52 0-.9.18-1.16.53-.25.35-.37.85-.37 1.51s.12 1.17.37 1.51c.25.35.64.52 1.16.52Zm5.56-4.04c.2-.37.42-.65.69-.86.26-.21.57-.32.94-.32.28 0 .5.06.68.19l-.1 1.1a.3.3 0 0 1-.09.16.24.24 0 0 1-.15.04 1.8 1.8 0 0 1-.27-.03 2.01 2.01 0 0 0-.34-.03c-.16 0-.3.03-.44.08a1.1 1.1 0 0 0-.34.2c-.1.1-.2.2-.27.33-.08.13-.15.27-.22.44V36H47.7v-6.16h.87c.15 0 .26.03.31.09.06.05.1.15.13.29l.09.7Zm4.62-1.07V36h-1.49v-6.16h1.49Zm.2-1.79c0 .13-.02.25-.07.36a1 1 0 0 1-.51.5.96.96 0 0 1-.74 0 1.02 1.02 0 0 1-.5-.5.96.96 0 0 1 0-.73.93.93 0 0 1 .86-.58.9.9 0 0 1 .38.08c.11.05.21.11.3.2a.94.94 0 0 1 .28.67Zm4.56 5.32a7.8 7.8 0 0 0-1.08.12c-.29.05-.52.12-.7.2a.92.92 0 0 0-.38.3.64.64 0 0 0-.11.36c0 .26.07.45.23.56.15.11.35.17.6.17.3 0 .57-.06.79-.17.22-.1.44-.28.65-.5v-1.04Zm-3.4-2.67c.71-.65 1.57-.97 2.56-.97.36 0 .68.06.97.18a1.99 1.99 0 0 1 1.16 1.24c.1.3.16.61.16.96V36h-.67a.7.7 0 0 1-.33-.06c-.07-.04-.13-.13-.18-.26l-.13-.44c-.16.14-.3.26-.46.37a2.8 2.8 0 0 1-.97.43 2.77 2.77 0 0 1-1.32-.05 1.62 1.62 0 0 1-.57-.31 1.41 1.41 0 0 1-.38-.53 1.85 1.85 0 0 1-.05-1.18c.05-.16.14-.3.25-.45.12-.14.28-.27.46-.4a3 3 0 0 1 .7-.32 9.19 9.19 0 0 1 2.2-.33v-.36c0-.41-.09-.71-.26-.91-.18-.2-.43-.3-.76-.3a1.84 1.84 0 0 0-1.02.28l-.33.18c-.1.06-.2.09-.32.09-.1 0-.2-.03-.27-.08a.72.72 0 0 1-.17-.2l-.26-.47Zm11.49 4.32V36h-4.88v-8.6h1.16v7.62h3.72Zm3.16-5.2c.44 0 .84.08 1.2.23a2.57 2.57 0 0 1 1.49 1.62c.13.38.2.81.2 1.29s-.07.91-.2 1.3a2.57 2.57 0 0 1-1.49 1.61c-.36.14-.76.21-1.2.21-.45 0-.85-.07-1.21-.21a2.57 2.57 0 0 1-1.5-1.62c-.13-.38-.2-.81-.2-1.3 0-.47.07-.9.2-1.28.14-.39.33-.72.59-1 .25-.26.55-.47.9-.62.37-.15.77-.22 1.22-.22Zm0 5.43c.6 0 1.05-.2 1.34-.6.3-.4.45-.97.45-1.69s-.15-1.28-.45-1.69c-.3-.4-.74-.6-1.34-.6-.3 0-.57.05-.8.15-.22.1-.4.26-.56.45-.15.2-.26.44-.34.73-.07.28-.1.6-.1.96 0 .72.14 1.29.44 1.69.3.4.75.6 1.36.6Zm6.33-2.22c.22 0 .4-.03.57-.09.16-.06.3-.14.41-.25.12-.11.2-.24.26-.39.05-.15.08-.31.08-.5 0-.37-.11-.66-.34-.88-.23-.22-.55-.33-.98-.33-.43 0-.76.1-.99.33-.22.22-.34.51-.34.89 0 .18.03.34.09.5a1.1 1.1 0 0 0 .67.63c.16.06.35.09.57.09Zm1.93 3.3a.51.51 0 0 0-.13-.36.84.84 0 0 0-.34-.22 8.57 8.57 0 0 0-1.73-.2 7.5 7.5 0 0 1-.62-.05c-.23.1-.41.23-.56.4a.8.8 0 0 0-.1.92c.07.12.18.22.32.3.14.1.32.16.54.21a3.5 3.5 0 0 0 1.55 0c.23-.05.42-.12.57-.22.16-.1.29-.21.37-.34a.8.8 0 0 0 .13-.44Zm1.08-6.17v.4c0 .13-.08.21-.25.25l-.69.09c.14.26.2.56.2.88a1.86 1.86 0 0 1-1.36 1.82 3.07 3.07 0 0 1-1.72.04c-.12.08-.22.16-.29.25a.44.44 0 0 0-.1.27c0 .15.06.26.17.33.12.08.28.13.47.16a5 5 0 0 0 .66.06 16.56 16.56 0 0 1 1.5.13c.26.05.48.12.67.22.19.1.34.24.46.41.12.18.18.4.18.69 0 .26-.07.5-.2.75s-.31.46-.56.65c-.24.2-.54.34-.9.46a4.57 4.57 0 0 1-2.36.04c-.33-.09-.6-.2-.82-.36a1.56 1.56 0 0 1-.5-.51c-.1-.2-.16-.4-.16-.6 0-.3.1-.56.28-.77.19-.2.45-.37.77-.5a1.15 1.15 0 0 1-.43-.32.88.88 0 0 1-.15-.54c0-.09.01-.18.04-.27.04-.1.08-.2.15-.28a1.55 1.55 0 0 1 .58-.5c-.3-.16-.53-.39-.7-.66-.17-.28-.25-.6-.25-.97 0-.3.05-.57.16-.8.12-.25.28-.46.48-.63.2-.17.45-.3.73-.4a3 3 0 0 1 2.3.21h1.64Zm4.65.76a.24.24 0 0 1-.23.14.42.42 0 0 1-.2-.07 3.59 3.59 0 0 0-.67-.3 1.8 1.8 0 0 0-1.03 0c-.14.05-.27.11-.37.2a.87.87 0 0 0-.23.27.75.75 0 0 0-.08.35c0 .15.04.28.13.39.1.1.21.19.36.27.15.07.32.14.5.2a13.63 13.63 0 0 1 1.16.4c.2.08.36.18.5.3a1.33 1.33 0 0 1 .5 1.07 2 2 0 0 1-.15.78c-.1.24-.25.44-.45.62-.2.17-.43.3-.72.4a3.1 3.1 0 0 1-2.14-.05 2.97 2.97 0 0 1-.87-.53l.25-.41c.04-.05.07-.1.12-.12a.3.3 0 0 1 .17-.04.4.4 0 0 1 .22.08l.3.19a1.91 1.91 0 0 0 1.03.27c.2 0 .38-.03.54-.08.16-.06.29-.13.4-.22a.96.96 0 0 0 .3-.7c0-.17-.05-.31-.14-.42-.09-.11-.2-.2-.36-.28a2.6 2.6 0 0 0-.5-.2l-.59-.19c-.2-.06-.39-.14-.58-.22a2.14 2.14 0 0 1-.5-.3 1.45 1.45 0 0 1-.36-.46c-.1-.19-.14-.41-.14-.67a1.6 1.6 0 0 1 .57-1.23c.18-.16.4-.3.68-.39.26-.1.57-.14.91-.14a2.84 2.84 0 0 1 1.9.7l-.23.4Z"}),Ct("defs",{children:Ct("path",{d:"M0 0h85v38H0z"})})]}),Tn=()=>Ct("svg",{viewBox:"0 0 85 38",fill:"currentColor",children:Ct("path",{d:"M11.118 10.476c.36.28.801.433 1.257.436h.052c.48-.007.961-.192 1.25-.444 1.509-1.279 5.88-5.287 5.88-5.287 1.168-1.087-2.093-2.174-7.13-2.181h-.06c-5.036.007-8.298 1.094-7.13 2.181 0 0 4.372 4.008 5.88 5.295zm2.559 2.166c-.359.283-.801.439-1.258.444h-.044a2.071 2.071 0 0 1-1.257-.444C10.082 11.755 6.384 8.42 5 7.148v1.93c0 .215.081.496.222.629l.07.064c1.045.955 4.546 4.154 5.825 5.245.358.283.8.438 1.257.444h.044c.489-.015.962-.2 1.258-.444 1.309-1.11 4.948-4.444 5.887-5.31.148-.132.222-.413.222-.628v-1.93a455.127 455.127 0 0 1-6.11 5.494zm-1.258 4.984a2.071 2.071 0 0 0 1.258-.436c2.053-1.815 4.09-3.65 6.11-5.502v1.938c0 .207-.075.488-.223.621-.94.873-4.578 4.2-5.887 5.31-.296.25-.77.436-1.258.443h-.044a2.071 2.071 0 0 1-1.257-.436c-1.204-1.027-4.376-3.928-5.616-5.062l-.28-.255c-.14-.133-.221-.414-.221-.621v-1.938c1.383 1.265 5.081 4.607 6.117 5.495.358.282.8.438 1.257.443h.044zM40 5l-5.84 14.46h-2.43L25.89 5h2.16c.233 0 .423.057.57.17.146.113.256.26.33.44l3.41 8.82c.113.287.22.603.32.95.106.34.206.697.3 1.07.08-.373.166-.73.26-1.07a8.84 8.84 0 0 1 .31-.95l3.39-8.82a.959.959 0 0 1 .31-.42.906.906 0 0 1 .58-.19H40zm17.176 0v14.46h-2.37v-9.34c0-.373.02-.777.06-1.21l-4.37 8.21c-.206.393-.523.59-.95.59h-.38c-.426 0-.743-.197-.95-.59l-4.42-8.24c.02.22.037.437.05.65.014.213.02.41.02.59v9.34h-2.37V5h2.03c.12 0 .224.003.31.01a.778.778 0 0 1 .23.05c.074.027.137.07.19.13.06.06.117.14.17.24l4.33 8.03c.114.213.217.433.31.66.1.227.197.46.29.7.094-.247.19-.483.29-.71.1-.233.207-.457.32-.67l4.27-8.01c.054-.1.11-.18.17-.24a.57.57 0 0 1 .19-.13.903.903 0 0 1 .24-.05c.087-.007.19-.01.31-.01h2.03zm8.887 13.73c.68 0 1.286-.117 1.82-.35.54-.24.996-.57 1.37-.99a4.28 4.28 0 0 0 .85-1.48c.2-.573.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.124 1.51-.37 2.19a5.248 5.248 0 0 1-1.07 1.77c-.46.5-1.024.893-1.69 1.18-.66.287-1.404.43-2.23.43-.827 0-1.574-.143-2.24-.43a5.012 5.012 0 0 1-1.69-1.18 5.33 5.33 0 0 1-1.06-1.77 6.373 6.373 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.096 1.277.29 1.85.2.567.483 1.06.85 1.48.373.42.826.75 1.36.99.54.24 1.15.36 1.83.36zm10.38.73h-1.03V5.31h1.03v14.15zM4.242 35v-5.166l-.672-.078a.595.595 0 0 1-.21-.09.23.23 0 0 1-.078-.186v-.438h.96v-.588c0-.348.048-.656.144-.924.1-.272.24-.5.42-.684a1.79 1.79 0 0 1 .66-.426c.256-.096.544-.144.864-.144.272 0 .522.04.75.12l-.024.534c-.008.096-.062.148-.162.156a4.947 4.947 0 0 1-.39.012c-.184 0-.352.024-.504.072a.949.949 0 0 0-.384.234c-.108.108-.192.25-.252.426a2.184 2.184 0 0 0-.084.654v.558h1.752v.774H5.316V35H4.242zM10.416 28.826a3.1 3.1 0 0 1 1.2.222c.356.148.66.358.912.63s.444.602.576.99c.136.384.204.814.204 1.29 0 .48-.068.912-.204 1.296a2.735 2.735 0 0 1-.576.984 2.572 2.572 0 0 1-.912.63 3.175 3.175 0 0 1-1.2.216c-.448 0-.852-.072-1.212-.216a2.572 2.572 0 0 1-.912-.63 2.805 2.805 0 0 1-.582-.984 3.972 3.972 0 0 1-.198-1.296c0-.476.066-.906.198-1.29.136-.388.33-.718.582-.99.252-.272.556-.482.912-.63.36-.148.764-.222 1.212-.222zm0 5.424c.6 0 1.048-.2 1.344-.6.296-.404.444-.966.444-1.686 0-.724-.148-1.288-.444-1.692-.296-.404-.744-.606-1.344-.606-.304 0-.57.052-.798.156a1.507 1.507 0 0 0-.564.45c-.148.196-.26.438-.336.726a3.941 3.941 0 0 0-.108.966c0 .72.148 1.282.444 1.686.3.4.754.6 1.362.6zM15.677 30.14c.192-.416.428-.74.708-.972.28-.236.622-.354 1.026-.354.128 0 .25.014.366.042.12.028.226.072.318.132l-.078.798c-.024.1-.084.15-.18.15-.056 0-.138-.012-.246-.036a1.694 1.694 0 0 0-.366-.036c-.192 0-.364.028-.516.084-.148.056-.282.14-.402.252a1.782 1.782 0 0 0-.318.408c-.092.16-.176.344-.252.552V35h-1.074v-6.078h.612c.116 0 .196.022.24.066.044.044.074.12.09.228l.072.924zM26.761 28.922 24.283 35h-.96l-2.478-6.078h.87a.33.33 0 0 1 .33.222l1.542 3.912c.048.148.09.292.126.432.036.14.07.28.102.42.032-.14.066-.28.102-.42.036-.14.08-.284.132-.432l1.56-3.912a.33.33 0 0 1 .12-.156.311.311 0 0 1 .198-.066h.834zM27.74 35v-6.078h.643c.152 0 .246.074.282.222l.078.624c.224-.276.476-.502.756-.678.28-.176.604-.264.972-.264.408 0 .738.114.99.342.256.228.44.536.552.924.088-.22.2-.41.336-.57a1.987 1.987 0 0 1 1.014-.624c.196-.048.394-.072.594-.072.32 0 .604.052.852.156.252.1.464.248.636.444.176.196.31.438.402.726.092.284.138.61.138.978V35H34.91v-3.87c0-.476-.104-.836-.312-1.08-.208-.248-.508-.372-.9-.372-.176 0-.344.032-.504.096-.156.06-.294.15-.414.27-.12.12-.216.272-.288.456-.068.18-.102.39-.102.63V35h-1.074v-3.87c0-.488-.098-.852-.294-1.092-.196-.24-.482-.36-.858-.36-.264 0-.508.072-.732.216a2.38 2.38 0 0 0-.618.576V35H27.74zM40.746 32.372c-.428.02-.788.058-1.08.114-.292.052-.526.12-.702.204a.923.923 0 0 0-.378.294.639.639 0 0 0-.114.366c0 .26.076.446.228.558.156.112.358.168.606.168.304 0 .566-.054.786-.162.224-.112.442-.28.654-.504v-1.038zm-3.396-2.67c.708-.648 1.56-.972 2.556-.972.36 0 .682.06.966.18.284.116.524.28.72.492.196.208.344.458.444.75.104.292.156.612.156.96V35h-.672a.708.708 0 0 1-.324-.06c-.076-.044-.136-.13-.18-.258l-.132-.444c-.156.14-.308.264-.456.372a2.804 2.804 0 0 1-.462.264c-.16.072-.332.126-.516.162-.18.04-.38.06-.6.06-.26 0-.5-.034-.72-.102a1.618 1.618 0 0 1-.57-.318 1.414 1.414 0 0 1-.372-.522 1.852 1.852 0 0 1-.132-.726 1.419 1.419 0 0 1 .33-.906c.12-.14.274-.272.462-.396s.418-.232.69-.324c.276-.092.596-.166.96-.222.364-.06.78-.096 1.248-.108v-.36c0-.412-.088-.716-.264-.912-.176-.2-.43-.3-.762-.3-.24 0-.44.028-.6.084-.156.056-.294.12-.414.192l-.33.186a.631.631 0 0 1-.324.084.439.439 0 0 1-.264-.078.716.716 0 0 1-.174-.192l-.264-.474zM44.974 29.6c.124-.124.254-.238.39-.342a2.395 2.395 0 0 1 .936-.444c.176-.044.368-.066.576-.066.336 0 .634.058.894.174.26.112.476.272.648.48.176.204.308.45.396.738.092.284.138.598.138.942V35H47.47v-3.918c0-.376-.086-.666-.258-.87-.172-.208-.434-.312-.786-.312-.256 0-.496.058-.72.174a2.58 2.58 0 0 0-.636.474V35h-1.482v-6.156h.906c.192 0 .318.09.378.27l.102.486zM53.085 28.748c.456 0 .87.074 1.242.222a2.692 2.692 0 0 1 1.578 1.626c.144.392.216.83.216 1.314 0 .488-.072.928-.216 1.32-.144.392-.35.726-.618 1.002a2.653 2.653 0 0 1-.96.636 3.333 3.333 0 0 1-1.242.222c-.46 0-.878-.074-1.254-.222a2.712 2.712 0 0 1-.966-.636 2.922 2.922 0 0 1-.618-1.002 3.807 3.807 0 0 1-.216-1.32c0-.484.072-.922.216-1.314.148-.392.354-.724.618-.996.268-.272.59-.482.966-.63a3.397 3.397 0 0 1 1.254-.222zm0 5.202c.512 0 .89-.172 1.134-.516.248-.344.372-.848.372-1.512s-.124-1.17-.372-1.518c-.244-.348-.622-.522-1.134-.522-.52 0-.906.176-1.158.528-.248.348-.372.852-.372 1.512s.124 1.164.372 1.512c.252.344.638.516 1.158.516zM57.252 35v-6.156h.906c.192 0 .318.09.378.27l.096.456c.108-.12.22-.23.336-.33a2.017 2.017 0 0 1 1.32-.492c.388 0 .706.106.954.318.252.208.44.486.564.834a1.93 1.93 0 0 1 .834-.882c.172-.092.354-.16.546-.204.196-.044.392-.066.588-.066.34 0 .642.052.906.156.264.104.486.256.666.456.18.2.316.444.408.732.096.288.144.618.144.99V35h-1.482v-3.918c0-.392-.086-.686-.258-.882-.172-.2-.424-.3-.756-.3-.152 0-.294.026-.426.078a1.026 1.026 0 0 0-.342.228 1.019 1.019 0 0 0-.228.366 1.435 1.435 0 0 0-.084.51V35h-1.488v-3.918c0-.412-.084-.712-.252-.9-.164-.188-.406-.282-.726-.282-.216 0-.418.054-.606.162a1.979 1.979 0 0 0-.516.432V35h-1.482zM70.558 32.372c-.428.02-.788.058-1.08.114-.292.052-.526.12-.702.204a.923.923 0 0 0-.378.294.639.639 0 0 0-.114.366c0 .26.076.446.228.558.156.112.358.168.606.168.304 0 .566-.054.786-.162.224-.112.442-.28.654-.504v-1.038zm-3.396-2.67c.708-.648 1.56-.972 2.556-.972.36 0 .682.06.966.18.284.116.524.28.72.492.196.208.344.458.444.75.104.292.156.612.156.96V35h-.672a.708.708 0 0 1-.324-.06c-.076-.044-.136-.13-.18-.258l-.132-.444c-.156.14-.308.264-.456.372a2.804 2.804 0 0 1-.462.264c-.16.072-.332.126-.516.162-.18.04-.38.06-.6.06-.26 0-.5-.034-.72-.102a1.618 1.618 0 0 1-.57-.318 1.414 1.414 0 0 1-.372-.522 1.852 1.852 0 0 1-.132-.726 1.419 1.419 0 0 1 .33-.906c.12-.14.274-.272.462-.396s.418-.232.69-.324c.276-.092.596-.166.96-.222.364-.06.78-.096 1.248-.108v-.36c0-.412-.088-.716-.264-.912-.176-.2-.43-.3-.762-.3-.24 0-.44.028-.6.084-.156.056-.294.12-.414.192l-.33.186a.631.631 0 0 1-.324.084.439.439 0 0 1-.264-.078.716.716 0 0 1-.174-.192l-.264-.474zM74.9 26.084V35h-1.482v-8.916H74.9zM81.969 28.844l-3.354 7.848a.538.538 0 0 1-.174.234c-.068.056-.174.084-.318.084h-1.104l1.152-2.472-2.49-5.694h1.302c.116 0 .206.028.27.084.068.056.118.12.15.192l1.308 3.192c.044.108.08.216.108.324.032.108.062.218.09.33a32.3 32.3 0 0 1 .108-.33c.036-.112.076-.222.12-.33l1.236-3.186a.437.437 0 0 1 .408-.276h1.188z"})}),Ln=()=>Ct("svg",{viewBox:"0 0 15 17",fill:"currentColor",children:Ct("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"})}),On=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Pn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),In=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Rn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Dn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M1 21h22L12 2 1 21zm12-3h-2v-2h2v2zm0-4h-2v-4h2v4z"})}),zn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Fn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),jn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),$n=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M7.41 8.59 12 13.17l4.59-4.58L18 10l-6 6-6-6 1.41-1.41z"})}),Hn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"m7 10 5 5 5-5z"})}),Vn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Ct("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"}),Ct("path",{d:"M12.5 7H11v6l5.25 3.15.75-1.23-4.5-2.67z"})]}),Un=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Bn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),qn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Yn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M8 5v14l11-7z"})}),Wn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Kn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Qn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Zn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Gn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M6 19c0 1.1.9 2 2 2h8c1.1 0 2-.9 2-2V7H6v12zM19 4h-3.5l-1-1h-5l-1 1H5v2h14V4z"})}),Jn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 13h-6v6h-2v-6H5v-2h6V5h2v6h6v2z"})}),Xn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 13H5v-2h14v2z"})}),er=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),tr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),nr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),rr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 9l1.25-2.75L23 5l-2.75-1.25L19 1l-1.25 2.75L15 5l2.75 1.25L19 9zm-7.5.5L9 4 6.5 9.5 1 12l5.5 2.5L9 20l2.5-5.5L17 12l-5.5-2.5zM19 15l-1.25 2.75L15 19l2.75 1.25L19 23l1.25-2.75L23 19l-2.75-1.25L19 15z"})}),ar=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),ir=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M20 9H4v2h16V9zM4 15h16v-2H4v2z"})}),or=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),lr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Ct("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"}),Ct("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"}),Ct("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"})]}),sr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),cr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),ur=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),dr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),hr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M3 17v2h6v-2H3zM3 5v2h10V5H3zm10 16v-2h8v-2h-8v-2h-2v6h2zM7 9v2H3v2h4v2h2V9H7zm14 4v-2H11v2h10zm-6-4h2V7h4V5h-4V3h-2v6z"})}),mr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M7 20h4c0 1.1-.9 2-2 2s-2-.9-2-2zm-2-1h8v-2H5v2zm11.5-9.5c0 3.82-2.66 5.86-3.77 6.5H5.27c-1.11-.64-3.77-2.68-3.77-6.5C1.5 5.36 4.86 2 9 2s7.5 3.36 7.5 7.5zm4.87-2.13L20 8l1.37.63L22 10l.63-1.37L24 8l-1.37-.63L22 6l-.63 1.37zM19 6l.94-2.06L22 3l-2.06-.94L19 0l-.94 2.06L16 3l2.06.94L19 6z"})}),pr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M3 14h4v-4H3v4zm0 5h4v-4H3v4zM3 9h4V5H3v4zm5 5h13v-4H8v4zm0 5h13v-4H8v4zM8 5v4h13V5H8z"})}),fr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"m22 9.24-7.19-.62L12 2 9.19 8.63 2 9.24l5.46 4.73L5.82 21 12 17.27 18.18 21l-1.63-7.03L22 9.24zM12 15.4l-3.76 2.27 1-4.28-3.32-2.88 4.38-.38L12 6.1l1.71 4.04 4.38.38-3.32 2.88 1 4.28L12 15.4z"})}),vr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M12 17.27 18.18 21l-1.64-7.03L22 9.24l-7.19-.61L12 2 9.19 8.63 2 9.24l5.46 4.73L5.82 21z"})}),gr=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-error"),children:Ct("path",{d:"M13.5095 4L8.50952 1H7.50952L2.50952 4L2.01953 4.85999V10.86L2.50952 11.71L7.50952 14.71H8.50952L13.5095 11.71L13.9995 10.86V4.85999L13.5095 4ZM7.50952 13.5601L3.00952 10.86V5.69995L7.50952 8.15002V13.5601ZM3.26953 4.69995L8.00952 1.85999L12.7495 4.69995L8.00952 7.29004L3.26953 4.69995ZM13.0095 10.86L8.50952 13.5601V8.15002L13.0095 5.69995V10.86Z"})}),yr=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-primary"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M2 5H4V4H1.5L1 4.5V12.5L1.5 13H4V12H2V5ZM14.5 4H12V5H14V12H12V13H14.5L15 12.5V4.5L14.5 4ZM11.76 6.56995L12 7V9.51001L11.7 9.95996L7.19995 11.96H6.73999L4.23999 10.46L4 10.03V7.53003L4.30005 7.06995L8.80005 5.06995H9.26001L11.76 6.56995ZM5 9.70996L6.5 10.61V9.28003L5 8.38V9.70996ZM5.57996 7.56006L7.03003 8.43005L10.42 6.93005L8.96997 6.06006L5.57996 7.56006ZM7.53003 10.73L11.03 9.17004V7.77002L7.53003 9.31995V10.73Z"})}),_r=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-warning"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M14 2H8L7 3V6H8V3H14V8H10V9H14L15 8V3L14 2ZM9 6H13V7H9.41L9 6.59V6ZM7 7H2L1 8V13L2 14H8L9 13V8L8 7H7ZM8 13H2V8H8V9V13ZM3 9H7V10H3V9ZM3 11H7V12H3V11ZM9 4H13V5H9V4Z"})}),br=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-primary"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M7 3L8 2H14L15 3V8L14 9H10V8H14V3H8V6H7V3ZM9 9V8L8 7H7H2L1 8V13L2 14H8L9 13V9ZM8 8V9V13H2V8H7H8ZM9.41421 7L9 6.58579V6H13V7H9.41421ZM9 4H13V5H9V4ZM7 10H3V11H7V10Z"})}),wr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 9h-4V3H9v6H5l7 7 7-7zM5 18v2h14v-2H5z"})});var kr=n(738),xr=n.n(kr);const Sr=e=>{let{to:t,isNavLink:n,children:r,...a}=e;return n?Ct(Re,{to:t,...a,children:r}):Ct("div",{...a,children:r})},Cr=e=>{let{activeItem:t,item:n,color:r=ft("color-primary"),activeNavRef:a,onChange:i,isNavLink:o}=e;return Ct(Sr,{className:xr()({"vm-tabs-item":!0,"vm-tabs-item_active":t===n.value,[n.className||""]:n.className}),isNavLink:o,to:n.value,style:{color:r},onClick:(l=n.value,()=>{i&&i(l)}),ref:t===n.value?a:void 0,children:[n.icon&&Ct("div",{className:xr()({"vm-tabs-item__icon":!0,"vm-tabs-item__icon_single":!n.label}),children:n.icon}),n.label]});var l},Er="undefined"!==typeof window?r.useLayoutEffect:r.useEffect;const Nr=function(e,t,n,a){const i=(0,r.useRef)(t);Er((()=>{i.current=t}),[t]),(0,r.useEffect)((()=>{var t;const r=null!==(t=null===n||void 0===n?void 0:n.current)&&void 0!==t?t:window;if(!r||!r.addEventListener)return;const o=e=>i.current(e);return r.addEventListener(e,o,a),()=>{r.removeEventListener(e,o,a)}}),[e,n,a])},Ar=()=>{const[e,t]=(0,r.useState)({width:0,height:0}),n=()=>{t({width:window.innerWidth,height:window.innerHeight})};return Nr("resize",n),Er(n,[]),e},Mr=e=>{let{activeItem:t,items:n,color:a=ft("color-primary"),onChange:i,indicatorPlacement:o="bottom",isNavLink:l}=e;const s=Ar(),c=(0,r.useRef)(null),[u,d]=(0,r.useState)({left:0,width:0,bottom:0});return(0,r.useEffect)((()=>{var e;if((null===(e=c.current)||void 0===e?void 0:e.base)instanceof HTMLElement){const{offsetLeft:e,offsetWidth:t,offsetHeight:n}=c.current.base;d({left:e,width:t,bottom:"top"===o?n-2:0})}}),[s,t,c,n]),Ct("div",{className:"vm-tabs",children:[n.map((e=>Ct(Cr,{activeItem:t,item:e,onChange:i,color:a,activeNavRef:c,isNavLink:l},e.value))),Ct("div",{className:"vm-tabs__indicator",style:{...u,borderColor:a}})]})},Tr=[{value:dt.chart,icon:Ct(Kn,{}),label:"Graph",prometheusCode:0},{value:dt.code,icon:Ct(Zn,{}),label:"JSON",prometheusCode:3},{value:dt.table,icon:Ct(Qn,{}),label:"Table",prometheusCode:1}],Lr=()=>{const{displayType:e}=Fr(),t=jr();return Ct(Mr,{activeItem:e,items:Tr,onChange:n=>{var r;t({type:"SET_DISPLAY_TYPE",payload:null!==(r=n)&&void 0!==r?r:e})}})},Or=ct("g0.tab",0),Pr=Tr.find((e=>e.prometheusCode===+Or||e.value===Or)),Ir=Xe("SERIES_LIMITS"),Rr={displayType:(null===Pr||void 0===Pr?void 0:Pr.value)||dt.chart,nocache:!1,isTracingEnabled:!1,seriesLimits:Ir?JSON.parse(Ir):ot,tableCompact:Xe("TABLE_COMPACT")||!1};function Dr(e,t){switch(t.type){case"SET_DISPLAY_TYPE":return{...e,displayType:t.payload};case"SET_SERIES_LIMITS":return Je("SERIES_LIMITS",JSON.stringify(t.payload)),{...e,seriesLimits:t.payload};case"TOGGLE_QUERY_TRACING":return{...e,isTracingEnabled:!e.isTracingEnabled};case"TOGGLE_NO_CACHE":return{...e,nocache:!e.nocache};case"TOGGLE_TABLE_COMPACT":return Je("TABLE_COMPACT",!e.tableCompact),{...e,tableCompact:!e.tableCompact};default:throw new Error}}const zr=(0,r.createContext)({}),Fr=()=>(0,r.useContext)(zr).state,jr=()=>(0,r.useContext)(zr).dispatch,$r={customStep:ct("g0.step_input",""),yaxis:{limits:{enable:!1,range:{1:[0,0]}}},isHistogram:!1,spanGaps:!1};function Hr(e,t){switch(t.type){case"TOGGLE_ENABLE_YAXIS_LIMITS":return{...e,yaxis:{...e.yaxis,limits:{...e.yaxis.limits,enable:!e.yaxis.limits.enable}}};case"SET_CUSTOM_STEP":return{...e,customStep:t.payload};case"SET_YAXIS_LIMITS":return{...e,yaxis:{...e.yaxis,limits:{...e.yaxis.limits,range:t.payload}}};case"SET_IS_HISTOGRAM":return{...e,isHistogram:t.payload};case"SET_SPAN_GAPS":return{...e,spanGaps:t.payload};default:throw new Error}}const Vr=(0,r.createContext)({}),Ur=()=>(0,r.useContext)(Vr).state,Br=()=>(0,r.useContext)(Vr).dispatch,qr={windows:"Windows",mac:"Mac OS",linux:"Linux"},Yr=()=>(Object.values(qr).find((e=>navigator.userAgent.indexOf(e)>=0))||"unknown")===qr.mac;function Wr(){const e=Ar(),t=()=>{const e=["Android","webOS","iPhone","iPad","iPod","BlackBerry","Windows Phone"].map((e=>navigator.userAgent.match(new RegExp(e,"i")))).some((e=>e)),t=window.innerWidth<500;return e||t},[n,a]=(0,r.useState)(t());return(0,r.useEffect)((()=>{a(t())}),[e]),{isMobile:n}}const Kr={success:Ct(Fn,{}),error:Ct(zn,{}),warning:Ct(Dn,{}),info:Ct(Rn,{})},Qr=e=>{let{variant:t,children:n}=e;const{isDarkTheme:r}=Nt(),{isMobile:a}=Wr();return Ct("div",{className:xr()({"vm-alert":!0,["vm-alert_".concat(t)]:t,"vm-alert_dark":r,"vm-alert_mobile":a}),children:[Ct("div",{className:"vm-alert__icon",children:Kr[t||"info"]}),Ct("div",{className:"vm-alert__content",children:n})]})},Zr=(0,r.createContext)({showInfoMessage:()=>{}}),Gr={dashboardsSettings:[],dashboardsLoading:!1,dashboardsError:""};function Jr(e,t){switch(t.type){case"SET_DASHBOARDS_SETTINGS":return{...e,dashboardsSettings:t.payload};case"SET_DASHBOARDS_LOADING":return{...e,dashboardsLoading:t.payload};case"SET_DASHBOARDS_ERROR":return{...e,dashboardsError:t.payload};default:throw new Error}}const Xr=(0,r.createContext)({}),ea=()=>(0,r.useContext)(Xr).state,ta=function(){for(var e=arguments.length,t=new Array(e),n=0;nn=>{let{children:r}=n;return Ct(e,{children:Ct(t,{children:r})})}),(e=>{let{children:t}=e;return Ct(xt.FK,{children:t})}))}(...[e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(kt,Mt),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Et.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(pn,mn),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(fn.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Sn,xn),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Cn.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Dr,Rr),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(zr.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Hr,$r),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Vr.Provider,{value:i,children:t})},e=>{let{children:t}=e;const{isMobile:n}=Wr(),[a,i]=(0,r.useState)({}),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(void 0);(0,r.useEffect)((()=>{if(!s)return;i({message:s.text,variant:s.type,key:Date.now()}),l(!0);const e=setTimeout(u,4e3);return()=>clearTimeout(e)}),[s]);const u=()=>{c(void 0),l(!1)};return Ct(Zr.Provider,{value:{showInfoMessage:c},children:[o&&Ct("div",{className:xr()({"vm-snackbar":!0,"vm-snackbar_mobile":n}),children:Ct(Qr,{variant:a.variant,children:Ct("div",{className:"vm-snackbar-content",children:[Ct("span",{children:a.message}),Ct("div",{className:"vm-snackbar-content__close",onClick:u,children:Ct(Pn,{})})]})})}),t]})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Jr,Gr),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Xr.Provider,{value:i,children:t})}]),na=e=>{let{activeMenu:t,label:n,value:r,color:a}=e;return Ct(Re,{className:xr()({"vm-header-nav-item":!0,"vm-header-nav-item_active":t===r}),style:{color:a},to:r,children:n})},ra=(e,t,n)=>{const a=(0,r.useCallback)((r=>{const a=null===e||void 0===e?void 0:e.current,i=r.target,o=(null===n||void 0===n?void 0:n.current)&&n.current.contains(i);!a||a.contains((null===r||void 0===r?void 0:r.target)||null)||o||t(r)}),[e,t]);Nr("mousedown",a),Nr("touchstart",a)},aa=e=>{let{variant:t="contained",color:n="primary",size:r="medium",ariaLabel:a,children:i,endIcon:o,startIcon:l,fullWidth:s=!1,className:c,disabled:u,onClick:d,onMouseDown:h}=e;return Ct("button",{className:xr()({"vm-button":!0,["vm-button_".concat(t,"_").concat(n)]:!0,["vm-button_".concat(r)]:r,"vm-button_icon":(l||o)&&!i,"vm-button_full-width":s,"vm-button_with-icon":l||o,"vm-button_disabled":u,[c||""]:c}),disabled:u,"aria-label":a,onClick:d,onMouseDown:h,children:Ct(xt.FK,{children:[l&&Ct("span",{className:"vm-button__start-icon",children:l}),i&&Ct("span",{children:i}),o&&Ct("span",{className:"vm-button__end-icon",children:o})]})})},ia=e=>{let{children:t,buttonRef:n,placement:a="bottom-left",open:i=!1,onClose:o,offset:l={top:6,left:0},clickOutside:s=!0,fullWidth:c,title:u,disabledFullScreen:d,variant:h}=e;const{isMobile:m}=Wr(),p=ae(),f=ne(),[v,g]=(0,r.useState)({width:0,height:0}),[y,_]=(0,r.useState)(!1),b=(0,r.useRef)(null);(0,r.useEffect)((()=>(_(i),!i&&o&&o(),i&&m&&!d&&(document.body.style.overflow="hidden"),()=>{document.body.style.overflow="auto"})),[i]),(0,r.useEffect)((()=>{var e,t;g({width:(null===b||void 0===b||null===(e=b.current)||void 0===e?void 0:e.clientWidth)||0,height:(null===b||void 0===b||null===(t=b.current)||void 0===t?void 0:t.clientHeight)||0}),_(!1)}),[b]);const w=(0,r.useMemo)((()=>{const e=n.current;if(!e||!y)return{};const t=e.getBoundingClientRect(),r={top:0,left:0,width:"auto"},i="bottom-right"===a||"top-right"===a,o=null===a||void 0===a?void 0:a.includes("top"),s=(null===l||void 0===l?void 0:l.top)||0,u=(null===l||void 0===l?void 0:l.left)||0;r.left=r.left=t.left+u,r.top=t.height+t.top+s,i&&(r.left=t.right-v.width),o&&(r.top=t.top-v.height-s);const{innerWidth:d,innerHeight:h}=window,m=r.top+v.height+20>h,p=r.top-20<0,f=r.left+v.width+20>d,g=r.left-20<0;return m&&(r.top=t.top-v.height-s),p&&(r.top=t.height+t.top+s),f&&(r.left=t.right-v.width-u),g&&(r.left=t.left+u),c&&(r.width="".concat(t.width,"px")),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[n,a,y,t,c]),k=()=>{_(!1),o()};(0,r.useEffect)((()=>{if(!b.current||!y||m&&!d)return;const{right:e,width:t}=b.current.getBoundingClientRect();if(e>window.innerWidth){const e=window.innerWidth-20-t;b.current.style.left=e{y&&m&&!d&&(p(f,{replace:!0}),o())}),[y,m,d,f,o]);return Nr("scroll",k),Nr("popstate",x),ra(b,(()=>{s&&k()}),n),Ct(xt.FK,{children:(y||!v.width)&&r.default.createPortal(Ct("div",{className:xr()({"vm-popper":!0,["vm-popper_".concat(h)]:h,"vm-popper_mobile":m&&!d,"vm-popper_open":(m||Object.keys(w).length)&&y}),ref:b,style:m&&!d?{}:w,children:[(u||m&&!d)&&Ct("div",{className:"vm-popper-header",children:[Ct("p",{className:"vm-popper-header__title",children:u}),Ct(aa,{variant:"text",color:"dark"===h?"white":"primary",size:"small",onClick:e=>{e.stopPropagation(),o()},ariaLabel:"close",children:Ct(Pn,{})})]}),t]}),document.body)})},oa=e=>{const[t,n]=(0,r.useState)(!!e),a=(0,r.useCallback)((()=>n(!0)),[]),i=(0,r.useCallback)((()=>n(!1)),[]),o=(0,r.useCallback)((()=>n((e=>!e))),[]);return{value:t,setValue:n,setTrue:a,setFalse:i,toggle:o}},la=e=>{let{activeMenu:t,label:n,color:a,background:i,submenu:o,direction:l}=e;const{pathname:s}=ne(),[c,u]=(0,r.useState)(null),d=(0,r.useRef)(null),{value:h,setFalse:m,setTrue:p}=oa(!1),f=()=>{c&&clearTimeout(c);const e=setTimeout(m,300);u(e)};return(0,r.useEffect)((()=>{m()}),[s]),"column"===l?Ct(xt.FK,{children:o.map((e=>Ct(na,{activeMenu:t,value:e.value||"",label:e.label||""},e.value)))}):Ct("div",{className:xr()({"vm-header-nav-item":!0,"vm-header-nav-item_sub":!0,"vm-header-nav-item_open":h,"vm-header-nav-item_active":o.find((e=>e.value===t))}),style:{color:a},onMouseEnter:()=>{p(),c&&clearTimeout(c)},onMouseLeave:f,ref:d,children:[n,Ct(Hn,{}),Ct(ia,{open:h,placement:"bottom-left",offset:{top:12,left:0},onClose:m,buttonRef:d,children:Ct("div",{className:"vm-header-nav-item-submenu",style:{background:i},onMouseLeave:f,onMouseEnter:()=>{c&&clearTimeout(c)},children:o.map((e=>Ct(na,{activeMenu:t,value:e.value||"",label:e.label||"",color:a},e.value)))})})]})},sa={label:"Explore",submenu:[{label:qe[Ye.metrics].title,value:Ye.metrics},{label:qe[Ye.cardinality].title,value:Ye.cardinality},{label:qe[Ye.topQueries].title,value:Ye.topQueries},{label:qe[Ye.activeQueries].title,value:Ye.activeQueries}]},ca={label:"Tools",submenu:[{label:qe[Ye.trace].title,value:Ye.trace},{label:qe[Ye.queryAnalyzer].title,value:Ye.queryAnalyzer},{label:qe[Ye.withTemplate].title,value:Ye.withTemplate},{label:qe[Ye.relabel].title,value:Ye.relabel}]},ua=[{label:qe[Ye.logs].title,value:Ye.home}],da=[{label:qe[Ye.anomaly].title,value:Ye.home}],ha=[{label:qe[Ye.home].title,value:Ye.home},sa,ca],ma=e=>{let{color:t,background:n,direction:a}=e;const i=Ke(),{dashboardsSettings:o}=ea(),{pathname:l}=ne(),[s,c]=(0,r.useState)(l),u=(0,r.useMemo)((()=>{switch({NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){case $e.logs:return ua;case $e.anomaly:return da;default:return[...ha,{label:qe[Ye.dashboards].title,value:Ye.dashboards,hide:i||!o.length}].filter((e=>!e.hide))}}),[i,o]);return(0,r.useEffect)((()=>{c(l)}),[l]),Ct("nav",{className:xr()({"vm-header-nav":!0,["vm-header-nav_".concat(a)]:a}),children:u.map((e=>e.submenu?Ct(la,{activeMenu:s,label:e.label||"",submenu:e.submenu,color:t,background:n,direction:a},e.label):Ct(na,{activeMenu:s,value:e.value||"",label:e.label||"",color:t},e.value)))})},pa=e=>{let{title:t,children:n,onClose:a,className:i,isOpen:o=!0}=e;const{isMobile:l}=Wr(),s=ae(),c=ne(),u=(0,r.useCallback)((e=>{o&&"Escape"===e.key&&a()}),[o]),d=e=>{e.stopPropagation()},h=(0,r.useCallback)((()=>{o&&(s(c,{replace:!0}),a())}),[o,c,a]);return(0,r.useEffect)((()=>{if(o)return document.body.style.overflow="hidden",()=>{document.body.style.overflow="auto"}}),[o]),Nr("popstate",h),Nr("keyup",u),r.default.createPortal(Ct("div",{className:xr()({"vm-modal":!0,"vm-modal_mobile":l,["".concat(i)]:i}),onMouseDown:a,children:Ct("div",{className:"vm-modal-content",children:[Ct("div",{className:"vm-modal-content-header",onMouseDown:d,children:[t&&Ct("div",{className:"vm-modal-content-header__title",children:t}),Ct("div",{className:"vm-modal-header__close",children:Ct(aa,{variant:"text",size:"small",onClick:a,ariaLabel:"close",children:Ct(Pn,{})})})]}),Ct("div",{className:"vm-modal-content-body",onMouseDown:d,children:n})]})}),document.body)},fa=e=>{let{children:t,title:n,open:a,placement:i="bottom-center",offset:o={top:6,left:0}}=e;const{isMobile:l}=Wr(),[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)({width:0,height:0}),h=(0,r.useRef)(null),m=(0,r.useRef)(null),p=()=>c(!1);(0,r.useEffect)((()=>{if(m.current&&s)return d({width:m.current.clientWidth,height:m.current.clientHeight}),window.addEventListener("scroll",p),()=>{window.removeEventListener("scroll",p)}}),[s,n]);const f=(0,r.useMemo)((()=>{var e;const t=null===h||void 0===h||null===(e=h.current)||void 0===e?void 0:e.base;if(!t||!s)return{};const n=t.getBoundingClientRect(),r={top:0,left:0},a="bottom-right"===i||"top-right"===i,l="bottom-left"===i||"top-left"===i,c=null===i||void 0===i?void 0:i.includes("top"),d=(null===o||void 0===o?void 0:o.top)||0,m=(null===o||void 0===o?void 0:o.left)||0;r.left=n.left-(u.width-n.width)/2+m,r.top=n.height+n.top+d,a&&(r.left=n.right-u.width),l&&(r.left=n.left+m),c&&(r.top=n.top-u.height-d);const{innerWidth:p,innerHeight:f}=window,v=r.top+u.height+20>f,g=r.top-20<0,y=r.left+u.width+20>p,_=r.left-20<0;return v&&(r.top=n.top-u.height-d),g&&(r.top=n.height+n.top+d),y&&(r.left=n.right-u.width-m),_&&(r.left=n.left+m),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[h,i,s,u]),v=()=>{"boolean"!==typeof a&&c(!0)},g=()=>{c(!1)};return(0,r.useEffect)((()=>{"boolean"===typeof a&&c(a)}),[a]),(0,r.useEffect)((()=>{var e;const t=null===h||void 0===h||null===(e=h.current)||void 0===e?void 0:e.base;if(t)return t.addEventListener("mouseenter",v),t.addEventListener("mouseleave",g),()=>{t.removeEventListener("mouseenter",v),t.removeEventListener("mouseleave",g)}}),[h]),Ct(xt.FK,{children:[Ct(r.Fragment,{ref:h,children:t}),!l&&s&&r.default.createPortal(Ct("div",{className:"vm-tooltip",ref:m,style:f,children:n}),document.body)]})},va=Ct("code",{children:Yr()?"Cmd":"Ctrl"}),ga=[{title:"Zoom in",description:Ct(xt.FK,{children:["To zoom in, hold down the ",va," + ",Ct("code",{children:"scroll up"}),", or press the ",Ct("code",{children:"+"}),". Also, you can zoom in on a range on the graph by holding down your mouse button and selecting the range."]})},{title:"Zoom out",description:Ct(xt.FK,{children:["To zoom out, hold down the ",va," + ",Ct("code",{children:"scroll down"}),", or press the ",Ct("code",{children:"-"}),"."]})},{title:"Move horizontal axis",description:Ct(xt.FK,{children:["To move the graph, hold down the ",va," + ",Ct("code",{children:"drag"})," the graph to the right or left."]})},{title:"Fixing a tooltip",description:Ct(xt.FK,{children:["To fix the tooltip, ",Ct("code",{children:"click"})," mouse when it's open. Then, you can drag the fixed tooltip by ",Ct("code",{children:"clicking"})," and ",Ct("code",{children:"dragging"})," on the ",Ct(ir,{})," icon."]})},{title:"Set a custom range for the vertical axis",description:Ct(xt.FK,{children:["To set a custom range for the vertical axis, click on the ",Ct(On,{})," icon located in the upper right corner of the graph, activate the toggle, and set the values."]})}],ya=[{title:"Show/hide a legend item",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on a legend item to isolate it on the graph.",va," + ",Ct("code",{children:"click"})," on a legend item to remove it from the graph. To revert to the previous state, click again."]})},{title:"Copy label key-value pairs",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on a label key-value pair to save it to the clipboard."]})},{title:"Collapse/Expand the legend group",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on the group name (e.g. ",Ct("b",{children:'Query 1: {__name__!=""}'}),") to collapse or expand the legend."]})}],_a=ga.concat(ya),ba=()=>{const{value:e,setFalse:t,setTrue:n}=oa(!1);return Ct(xt.FK,{children:[Ct(fa,{title:"Show tips on working with the graph",children:Ct(aa,{variant:"text",color:"gray",startIcon:Ct(mr,{}),onClick:n,ariaLabel:"open the tips"})}),e&&Ct(pa,{title:"Tips on working with the graph and the legend",onClose:t,children:Ct("div",{className:"fc-graph-tips",children:_a.map((e=>{let{title:t,description:n}=e;return Ct("div",{className:"fc-graph-tips-item",children:[Ct("h4",{className:"fc-graph-tips-item__action",children:t}),Ct("p",{className:"fc-graph-tips-item__description",children:n})]},t)}))})})]})},wa=Ct("code",{children:Yr()?"Cmd":"Ctrl"}),ka=Ct(xt.FK,{children:[Ct("code",{children:Yr()?"Option":"Ctrl"})," + ",Ct("code",{children:"Space"})]}),xa=[{title:"Query",list:[{keys:Ct("code",{children:"Enter"}),description:"Run"},{keys:Ct(xt.FK,{children:[Ct("code",{children:"Shift"})," + ",Ct("code",{children:"Enter"})]}),description:"Multi-line queries"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"Arrow Up"})]}),description:"Previous command from the Query history"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"Arrow Down"})]}),description:"Next command from the Query history"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"click"})," by ",Ct(tr,{})]}),description:"Toggle multiple queries"},{keys:ka,description:"Show quick autocomplete tips"}]},{title:"Graph",readMore:Ct(ba,{}),list:[{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"scroll Up"})," or ",Ct("code",{children:"+"})]}),description:"Zoom in"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"scroll Down"})," or ",Ct("code",{children:"-"})]}),description:"Zoom out"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"drag"})]}),description:"Move the graph left/right"},{keys:Ct(xt.FK,{children:Ct("code",{children:"click"})}),description:"Select the series in the legend"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"click"})]}),description:"Toggle multiple series in the legend"}]}],Sa="Shortcut keys",Ca=Yr(),Ea=Ca?"Cmd + /":"F1",Na=e=>{let{showTitle:t}=e;const n=Ke(),{value:a,setTrue:i,setFalse:o}=oa(!1),l=(0,r.useCallback)((e=>{const t=Ca&&"/"===e.key&&e.metaKey,n=!Ca&&"F1"===e.key&&!e.metaKey;(t||n)&&i()}),[i]);return Nr("keydown",l),Ct(xt.FK,{children:[Ct(fa,{open:!0!==t&&void 0,title:"".concat(Sa," (").concat(Ea,")"),placement:"bottom-center",children:Ct(aa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(qn,{}),onClick:i,ariaLabel:Sa,children:t&&Sa})}),a&&Ct(pa,{title:"Shortcut keys",onClose:o,children:Ct("div",{className:"vm-shortcuts",children:xa.map((e=>Ct("div",{className:"vm-shortcuts-section",children:[e.readMore&&Ct("div",{className:"vm-shortcuts-section__read-more",children:e.readMore}),Ct("h3",{className:"vm-shortcuts-section__title",children:e.title}),Ct("div",{className:"vm-shortcuts-section-list",children:e.list.map(((t,n)=>Ct("div",{className:"vm-shortcuts-section-list-item",children:[Ct("div",{className:"vm-shortcuts-section-list-item__key",children:t.keys}),Ct("p",{className:"vm-shortcuts-section-list-item__description",children:t.description})]},"".concat(e.title,"_").concat(n))))})]},e.title)))})})]})},Aa=e=>{let{open:t}=e;return Ct("button",{className:xr()({"vm-menu-burger":!0,"vm-menu-burger_opened":t}),"aria-label":"menu",children:Ct("span",{})})},{REACT_APP_TYPE:Ma}={},Ta=Ma===$e.logs,La=e=>{let{background:t,color:n}=e;const{pathname:a}=ne(),{isMobile:i}=Wr(),o=(0,r.useRef)(null),{value:l,toggle:s,setFalse:c}=oa(!1);return(0,r.useEffect)(c,[a]),ra(o,c),Ct("div",{className:"vm-header-sidebar",ref:o,children:[Ct("div",{className:xr()({"vm-header-sidebar-button":!0,"vm-header-sidebar-button_open":l}),onClick:s,children:Ct(Aa,{open:l})}),Ct("div",{className:xr()({"vm-header-sidebar-menu":!0,"vm-header-sidebar-menu_open":l}),children:[Ct("div",{children:Ct(ma,{color:n,background:t,direction:"column"})}),Ct("div",{className:"vm-header-sidebar-menu-settings",children:!i&&!Ta&&Ct(Na,{showTitle:!0})})]})]})},Oa=e=>{let{controlsComponent:t,isMobile:n,...a}=e;const i=Ke(),{pathname:o}=ne(),{accountIds:l}=(()=>{const{useTenantID:e}=We(),t=Ke(),{serverUrl:n}=Nt(),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(),[s,c]=(0,r.useState)([]),u=(0,r.useMemo)((()=>"".concat(n.replace(/^(.+)(\/select.+)/,"$1"),"/admin/tenants")),[n]),d=(0,r.useMemo)((()=>!!Ge(n)),[n]),h=t?!e:!d;return(0,r.useEffect)((()=>{h||(async()=>{i(!0);try{const e=await fetch(u),t=await e.json(),n=t.data||[];c(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?l(void 0):l("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(ap){ap instanceof Error&&l("".concat(ap.name,": ").concat(ap.message))}i(!1)})().catch(console.error)}),[u]),{accountIds:s,isLoading:a,error:o}})(),{value:s,toggle:c,setFalse:u}=oa(!1),d=Ct(t,{...a,isMobile:n,accountIds:l,headerSetup:(0,r.useMemo)((()=>(qe[o]||{}).header||{}),[o])});return n?Ct(xt.FK,{children:[Ct("div",{children:Ct(aa,{className:xr()({"vm-header-button":!i}),startIcon:Ct(dr,{}),onClick:c,ariaLabel:"controls"})}),Ct(pa,{title:"Controls",onClose:u,isOpen:s,className:xr()({"vm-header-controls-modal":!0,"vm-header-controls-modal_open":s}),children:d})]}):d},{REACT_APP_TYPE:Pa}={},Ia=Pa===$e.logs||Pa===$e.anomaly,Ra=()=>{switch(Pa){case $e.logs:return Ct(Mn,{});case $e.anomaly:return Ct(Tn,{});default:return Ct(An,{})}},Da=e=>{let{controlsComponent:t}=e;const{isMobile:n}=Wr(),a=Ar(),i=(0,r.useMemo)((()=>window.innerWidth<1e3),[a]),{isDarkTheme:o}=Nt(),l=Ke(),s=(0,r.useMemo)((()=>ft(o?"color-background-block":"color-primary")),[o]),{background:c,color:u}=(0,r.useMemo)((()=>{const{headerStyles:{background:e=(l?"#FFF":s),color:t=(l?s:"#FFF")}={}}=We();return{background:e,color:t}}),[s]),d=ae(),h=()=>{d({pathname:Ye.home}),window.location.reload()};return Ct("header",{className:xr()({"vm-header":!0,"vm-header_app":l,"vm-header_dark":o,"vm-header_sidebar":i,"vm-header_mobile":n}),style:{background:c,color:u},children:[i?Ct(La,{background:c,color:u}):Ct(xt.FK,{children:[!l&&Ct("div",{className:xr()({"vm-header-logo":!0,"vm-header-logo_logs":Ia}),onClick:h,style:{color:u},children:Ct(Ra,{})}),Ct(ma,{color:u,background:c})]}),i&&Ct("div",{className:xr()({"vm-header-logo":!0,"vm-header-logo_mobile":!0,"vm-header-logo_logs":Ia}),onClick:h,style:{color:u},children:Ct(Ra,{})}),Ct(Oa,{controlsComponent:t,displaySidebar:i,isMobile:n})]})},za=(0,r.memo)((()=>{const e="2019-".concat((new Date).getFullYear());return Ct("footer",{className:"vm-footer",children:[Ct("a",{className:"vm-link vm-footer__website",target:"_blank",href:"https://victoriametrics.com/",rel:"me noreferrer",children:[Ct(Ln,{}),"victoriametrics.com"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/MetricsQL.html",rel:"help noreferrer",children:[Ct(Zn,{}),"MetricsQL"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/#vmui",rel:"help noreferrer",children:[Ct(lr,{}),"Documentation"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://github.com/VictoriaMetrics/VictoriaMetrics/issues/new/choose",rel:"noreferrer",children:[Ct(sr,{}),"Create an issue"]}),Ct("div",{className:"vm-footer__copyright",children:["\xa9 ",e," VictoriaMetrics"]})]})})),Fa=()=>{const e=Ke(),{serverUrl:t}=Nt(),n=(0,r.useContext)(Xr).dispatch,[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(""),[s,c]=(0,r.useState)([]),u=async()=>{try{const e=window.__VMUI_PREDEFINED_DASHBOARDS__;if(null===e||void 0===e||!e.length)return[];const t=await Promise.all(e.map((async e=>(async e=>{const t=await fetch("./dashboards/".concat(e));return await t.json()})(e))));c((e=>[...t,...e]))}catch(ap){ap instanceof Error&&l("".concat(ap.name,": ").concat(ap.message))}};return(0,r.useEffect)((()=>{e||(c([]),(async()=>{if(t&&!{NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){l(""),i(!0);try{const e=await fetch("".concat(t,"/vmui/custom-dashboards")),n=await e.json();if(e.ok){const{dashboardsSettings:e}=n;e&&e.length>0?c((t=>[...t,...e])):await u(),i(!1)}else await u(),l(n.error),i(!1)}catch(ap){i(!1),ap instanceof Error&&l("".concat(ap.name,": ").concat(ap.message)),await u()}}})())}),[t]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_SETTINGS",payload:s})}),[s]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_LOADING",payload:a})}),[a]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_ERROR",payload:o})}),[o]),{dashboardsSettings:s,isLoading:a,error:o}},ja=e=>{let{error:t,warning:n,info:a}=e;const i=(0,r.useRef)(null),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(!1),u=(0,r.useMemo)((()=>t?"ERROR: ":n?"WARNING: ":""),[t,n]),d="".concat(u).concat(t||n||a),h=()=>{const e=i.current;if(e){const{offsetWidth:t,scrollWidth:n,offsetHeight:r,scrollHeight:a}=e;l(t+1{c(!1),h()}),[i,d]),Nr("resize",h),t||n||a?Ct("span",{className:xr()({"vm-text-field__error":!0,"vm-text-field__warning":n&&!t,"vm-text-field__helper-text":!n&&!t,"vm-text-field__error_overflowed":o,"vm-text-field__error_full":s}),"data-show":!!d,ref:i,onClick:()=>{o&&(c(!0),l(!1))},children:d}):null},$a=e=>{let{label:t,value:n,type:a="text",error:i="",warning:o="",helperText:l="",placeholder:s,endIcon:c,startIcon:u,disabled:d=!1,autofocus:h=!1,inputmode:m="text",onChange:p,onEnter:f,onKeyDown:v,onFocus:g,onBlur:y,onChangeCaret:_}=e;const{isDarkTheme:b}=Nt(),{isMobile:w}=Wr(),k=(0,r.useRef)(null),x=(0,r.useRef)(null),S=(0,r.useMemo)((()=>"textarea"===a?x:k),[a]),C=xr()({"vm-text-field__input":!0,"vm-text-field__input_error":i,"vm-text-field__input_warning":!i&&o,"vm-text-field__input_icon-start":u,"vm-text-field__input_disabled":d,"vm-text-field__input_textarea":"textarea"===a}),E=e=>{const{selectionStart:t,selectionEnd:n}=e;_&&_([t||0,n||0])},N=e=>{E(e.currentTarget)},A=e=>{v&&v(e);const{key:t,ctrlKey:n,metaKey:r}=e,i="Enter"===t;("textarea"!==a?i:i&&(r||n))&&f&&(e.preventDefault(),f())},M=e=>{E(e.currentTarget)},T=e=>{d||(p&&p(e.currentTarget.value),E(e.currentTarget))};(0,r.useEffect)((()=>{var e;h&&!w&&(null===S||void 0===S||null===(e=S.current)||void 0===e?void 0:e.focus)&&S.current.focus()}),[S,h]);const L=()=>{g&&g()},O=()=>{y&&y()};return Ct("label",{className:xr()({"vm-text-field":!0,"vm-text-field_textarea":"textarea"===a,"vm-text-field_dark":b}),"data-replicated-value":n,children:[u&&Ct("div",{className:"vm-text-field__icon-start",children:u}),c&&Ct("div",{className:"vm-text-field__icon-end",children:c}),"textarea"===a?Ct("textarea",{className:C,disabled:d,ref:x,value:n,rows:1,inputMode:m,placeholder:s,autoCapitalize:"none",onInput:T,onKeyDown:A,onKeyUp:M,onFocus:L,onBlur:O,onMouseUp:N}):Ct("input",{className:C,disabled:d,ref:k,value:n,type:a,placeholder:s,inputMode:m,autoCapitalize:"none",onInput:T,onKeyDown:A,onKeyUp:M,onFocus:L,onBlur:O,onMouseUp:N}),t&&Ct("span",{className:"vm-text-field__label",children:t}),Ct(ja,{error:i,warning:o,info:l})]})},Ha=e=>{let{accountIds:t}=e;const n=Ke(),{isMobile:a}=Wr(),{tenantId:i,serverUrl:o}=Nt(),l=At(),s=gn(),[c,u]=(0,r.useState)(""),d=(0,r.useRef)(null),{value:h,toggle:m,setFalse:p}=oa(!1),f=(0,r.useMemo)((()=>{if(!c)return t;try{const e=new RegExp(c,"i");return t.filter((t=>e.test(t))).sort(((t,n)=>{var r,a;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(a=n.match(e))||void 0===a?void 0:a.index)||0)}))}catch(ap){return[]}}),[c,t]),v=(0,r.useMemo)((()=>t.length>1),[t]),g=e=>()=>{const t=e;if(l({type:"SET_TENANT_ID",payload:t}),o){const e=Ze(o,t);if(e===o)return;l({type:"SET_SERVER",payload:e}),s({type:"RUN_QUERY"})}p()};return(0,r.useEffect)((()=>{const e=Ge(o);i&&i!==e?g(i)():g(e)()}),[o]),v?Ct("div",{className:"vm-tenant-input",children:[Ct(fa,{title:"Define Tenant ID if you need request to another storage",children:Ct("div",{ref:d,children:a?Ct("div",{className:"vm-mobile-option",onClick:m,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(ur,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Tenant ID"}),Ct("span",{className:"vm-mobile-option-text__value",children:i})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(aa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",fullWidth:!0,startIcon:Ct(ur,{}),endIcon:Ct("div",{className:xr()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":h}),children:Ct($n,{})}),onClick:m,children:i})})}),Ct(ia,{open:h,placement:"bottom-right",onClose:p,buttonRef:d,title:a?"Define Tenant ID":void 0,children:Ct("div",{className:xr()({"vm-list vm-tenant-input-list":!0,"vm-list vm-tenant-input-list_mobile":a}),children:[Ct("div",{className:"vm-tenant-input-list__search",children:Ct($a,{autofocus:!0,label:"Search",value:c,onChange:u,type:"search"})}),f.map((e=>Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":a,"vm-list-item_active":e===i}),onClick:g(e),children:e},e)))]})})]}):null};const Va=function(e){const t=(0,r.useRef)();return(0,r.useEffect)((()=>{t.current=e}),[e]),t.current},Ua=()=>{const e=Ke(),{isMobile:t}=Wr(),{customStep:n,isHistogram:a}=Ur(),{period:{step:i,end:o,start:l}}=vn(),s=Br(),c=Va(o-l),u=(0,r.useMemo)((()=>Kt(o-l,a)),[i,a]),[d,h]=(0,r.useState)(n||u),[m,p]=(0,r.useState)(""),{value:f,toggle:v,setFalse:g}=oa(!1),y=(0,r.useRef)(null),_=e=>{const t=e||d||u||"1s",n=(t.match(/[a-zA-Z]+/g)||[]).length?t:"".concat(t,"s");s({type:"SET_CUSTOM_STEP",payload:n}),h(n),p("")},b=()=>{_(),g()},w=e=>{const t=e.match(/[-+]?([0-9]*\.[0-9]+|[0-9]+)/g)||[],n=e.match(/[a-zA-Z]+/g)||[],r=t.length&&t.every((e=>parseFloat(e)>0)),a=n.every((e=>Ht.find((t=>t.short===e)))),i=r&&a;h(e),p(i?"":ht.validStep)};return(0,r.useEffect)((()=>{n&&_(n)}),[n]),(0,r.useEffect)((()=>{!n&&u&&_(u)}),[u]),(0,r.useEffect)((()=>{o-l!==c&&c&&u&&_(u)}),[o,l,c,u]),(0,r.useEffect)((()=>{i!==n&&i!==u||_(u)}),[a]),Ct("div",{className:"vm-step-control",ref:y,children:[t?Ct("div",{className:"vm-mobile-option",onClick:v,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(or,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Step"}),Ct("span",{className:"vm-mobile-option-text__value",children:d})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(fa,{title:"Query resolution step width",children:Ct(aa,{className:e?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(or,{}),onClick:v,children:Ct("p",{children:["STEP",Ct("p",{className:"vm-step-control__value",children:d})]})})}),Ct(ia,{open:f,placement:"bottom-right",onClose:b,buttonRef:y,title:t?"Query resolution step width":void 0,children:Ct("div",{className:xr()({"vm-step-control-popper":!0,"vm-step-control-popper_mobile":t}),children:[Ct($a,{autofocus:!0,label:"Step value",value:d,error:m,onChange:w,onEnter:()=>{_(),b()},onFocus:()=>{document.activeElement instanceof HTMLInputElement&&document.activeElement.select()},onBlur:_,endIcon:Ct(fa,{title:"Set default step value: ".concat(u),children:Ct(aa,{size:"small",variant:"text",color:"primary",startIcon:Ct(In,{}),onClick:()=>{const e=u||"1s";w(e),_(e)},ariaLabel:"reset step"})})}),Ct("div",{className:"vm-step-control-popper-info",children:[Ct("code",{children:"step"})," - the ",Ct("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 ",Ct("code",{children:"query"})," is executed at",Ct("code",{children:"start"}),", ",Ct("code",{children:"start+step"}),", ",Ct("code",{children:"start+2*step"}),", \u2026, ",Ct("code",{children:"end"})," timestamps.",Ct("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"})]})]})})]})},Ba=e=>{let{relativeTime:t,setDuration:n}=e;const{isMobile:r}=Wr();return Ct("div",{className:xr()({"vm-time-duration":!0,"vm-time-duration_mobile":r}),children:en.map((e=>{let{id:a,duration:i,until:o,title:l}=e;return Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":a===t}),onClick:(s={duration:i,until:o(),id:a},()=>{n(s)}),children:l||i},a);var s}))})},qa=e=>{let{viewDate:t,showArrowNav:n,onChangeViewDate:r,toggleDisplayYears:a}=e;return Ct("div",{className:"vm-calendar-header",children:[Ct("div",{className:"vm-calendar-header-left",onClick:a,children:[Ct("span",{className:"vm-calendar-header-left__date",children:t.format("MMMM YYYY")}),Ct("div",{className:"vm-calendar-header-left__select-year",children:Ct(Hn,{})})]}),n&&Ct("div",{className:"vm-calendar-header-right",children:[Ct("div",{className:"vm-calendar-header-right__prev",onClick:()=>{r(t.subtract(1,"month"))},children:Ct($n,{})}),Ct("div",{className:"vm-calendar-header-right__next",onClick:()=>{r(t.add(1,"month"))},children:Ct($n,{})})]})]})},Ya=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],Wa=e=>{let{viewDate:t,selectDate:n,onChangeSelectDate:a}=e;const o="YYYY-MM-DD",l=i()().tz().startOf("day"),s=(0,r.useMemo)((()=>{const e=new Array(42).fill(null),n=t.startOf("month"),r=t.endOf("month").diff(n,"day")+1,a=new Array(r).fill(n).map(((e,t)=>e.add(t,"day"))),i=n.day();return e.splice(i,r,...a),e}),[t]),c=e=>()=>{e&&a(e)};return Ct("div",{className:"vm-calendar-body",children:[Ya.map((e=>Ct("div",{className:"vm-calendar-body-cell vm-calendar-body-cell_weekday",children:e[0]},e))),s.map(((e,t)=>Ct("div",{className:xr()({"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.format(o))===n.format(o),"vm-calendar-body-cell_day_today":(e&&e.format(o))===l.format(o)}),onClick:c(e),children:e&&e.format("D")},e?e.format(o):t)))]})},Ka=e=>{let{viewDate:t,onChangeViewDate:n}=e;const a=i()().format("YYYY"),o=(0,r.useMemo)((()=>t.format("YYYY")),[t]),l=(0,r.useMemo)((()=>{const e=i()().subtract(9,"year");return new Array(18).fill(e).map(((e,t)=>e.add(t,"year")))}),[t]);(0,r.useEffect)((()=>{const e=document.getElementById("vm-calendar-year-".concat(o));e&&e.scrollIntoView({block:"center"})}),[]);return Ct("div",{className:"vm-calendar-years",children:l.map((e=>{return Ct("div",{className:xr()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("YYYY")===o,"vm-calendar-years__year_today":e.format("YYYY")===a}),id:"vm-calendar-year-".concat(e.format("YYYY")),onClick:(t=e,()=>{n(t)}),children:e.format("YYYY")},e.format("YYYY"));var t}))})},Qa=e=>{let{viewDate:t,selectDate:n,onChangeViewDate:a}=e;const o=i()().format("MM"),l=(0,r.useMemo)((()=>n.format("MM")),[n]),s=(0,r.useMemo)((()=>new Array(12).fill("").map(((e,n)=>i()(t).month(n)))),[t]);(0,r.useEffect)((()=>{const e=document.getElementById("vm-calendar-year-".concat(l));e&&e.scrollIntoView({block:"center"})}),[]);const c=e=>()=>{a(e)};return Ct("div",{className:"vm-calendar-years",children:s.map((e=>Ct("div",{className:xr()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("MM")===l,"vm-calendar-years__year_today":e.format("MM")===o}),id:"vm-calendar-year-".concat(e.format("MM")),onClick:c(e),children:e.format("MMMM")},e.format("MM"))))})};var Za=function(e){return e[e.days=0]="days",e[e.months=1]="months",e[e.years=2]="years",e}(Za||{});const Ga=e=>{let{date:t,format:n=Lt,onChange:a}=e;const[o,l]=(0,r.useState)(Za.days),[s,c]=(0,r.useState)(i().tz(t)),[u,d]=(0,r.useState)(i().tz(t)),h=i()().startOf("day").tz(),m=h.format()===s.format(),{isMobile:p}=Wr(),f=e=>{c(e),l((e=>e===Za.years?Za.months:Za.days))};return(0,r.useEffect)((()=>{u.format()!==i().tz(t).format()&&a(u.format(n))}),[u]),(0,r.useEffect)((()=>{const e=i().tz(t);c(e),d(e)}),[t]),Ct("div",{className:xr()({"vm-calendar":!0,"vm-calendar_mobile":p}),children:[Ct(qa,{viewDate:s,onChangeViewDate:f,toggleDisplayYears:()=>{l((e=>e===Za.years?Za.days:Za.years))},showArrowNav:o===Za.days}),o===Za.days&&Ct(Wa,{viewDate:s,selectDate:u,onChangeSelectDate:e=>{d(e)}}),o===Za.years&&Ct(Ka,{viewDate:s,onChangeViewDate:f}),o===Za.months&&Ct(Qa,{selectDate:u,viewDate:s,onChangeViewDate:f}),!m&&o===Za.days&&Ct("div",{className:"vm-calendar-footer",children:Ct(aa,{variant:"text",size:"small",onClick:()=>{c(h)},children:"show today"})})]})},Ja=(0,r.forwardRef)(((e,t)=>{let{date:n,targetRef:a,format:o=Lt,onChange:l,label:s}=e;const c=(0,r.useMemo)((()=>i()(n).isValid()?i().tz(n):i()().tz()),[n]),{isMobile:u}=Wr(),{value:d,toggle:h,setFalse:m}=oa(!1);return Nr("click",h,a),Nr("keyup",(e=>{"Escape"!==e.key&&"Enter"!==e.key||m()})),Ct(xt.FK,{children:Ct(ia,{open:d,buttonRef:a,placement:"bottom-right",onClose:m,title:u?s:void 0,children:Ct("div",{ref:t,children:Ct(Ga,{date:c,format:o,onChange:e=>{l(e),m()}})})})})})),Xa=Ja;var ei=n(494),ti=n.n(ei);const ni=e=>i()(e).isValid()?i().tz(e).format(Lt):e,ri=e=>{let{value:t="",label:n,pickerLabel:a,pickerRef:o,onChange:l,onEnter:s}=e;const c=(0,r.useRef)(null),[u,d]=(0,r.useState)(null),[h,m]=(0,r.useState)(ni(t)),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)(!1),y=i()(h).isValid()?"":"Invalid date format";return(0,r.useEffect)((()=>{const e=ni(t);e!==h&&m(e),v&&(s(),g(!1))}),[t]),(0,r.useEffect)((()=>{p&&u&&(u.focus(),u.setSelectionRange(11,11),f(!1))}),[p]),Ct("div",{className:xr()({"vm-date-time-input":!0,"vm-date-time-input_error":y}),children:[Ct("label",{children:n}),Ct(ti(),{tabIndex:1,inputRef:d,mask:"9999-99-99 99:99:99",placeholder:"YYYY-MM-DD HH:mm:ss",value:h,autoCapitalize:"none",inputMode:"numeric",maskChar:null,onChange:e=>{m(e.currentTarget.value)},onBlur:()=>{l(h)},onKeyUp:e=>{"Enter"===e.key&&(l(h),g(!0))}}),y&&Ct("span",{className:"vm-date-time-input__error-text",children:y}),Ct("div",{className:"vm-date-time-input__icon",ref:c,children:Ct(aa,{variant:"text",color:"gray",size:"small",startIcon:Ct(Un,{}),ariaLabel:"calendar"})}),Ct(Xa,{label:a,ref:o,date:h,onChange:e=>{m(e),f(!0)},targetRef:c})]})},ai=()=>{const{isMobile:e}=Wr(),{isDarkTheme:t}=Nt(),n=(0,r.useRef)(null),a=Ar(),o=(0,r.useMemo)((()=>a.width>1120),[a]),[l,s]=(0,r.useState)(),[c,u]=(0,r.useState)(),{period:{end:d,start:h},relativeTime:m,timezone:p,duration:f}=vn(),v=gn(),g=Ke(),y=Va(p),{value:_,toggle:b,setFalse:w}=oa(!1),k=(0,r.useMemo)((()=>({region:p,utc:nn(p)})),[p]);(0,r.useEffect)((()=>{s(Gt(Xt(d)))}),[p,d]),(0,r.useEffect)((()=>{u(Gt(Xt(h)))}),[p,h]);const x=e=>{let{duration:t,until:n,id:r}=e;v({type:"SET_RELATIVE_TIME",payload:{duration:t,until:n,id:r}}),w()},S=(0,r.useMemo)((()=>({start:i().tz(Xt(h)).format(Lt),end:i().tz(Xt(d)).format(Lt)})),[h,d,p]),C=(0,r.useMemo)((()=>m&&"none"!==m?m.replace(/_/g," "):"".concat(S.start," - ").concat(S.end)),[m,S]),E=(0,r.useRef)(null),N=(0,r.useRef)(null),A=(0,r.useRef)(null),M=()=>{c&&l&&v({type:"SET_PERIOD",payload:{from:i().tz(c).toDate(),to:i().tz(l).toDate()}}),w()};return(0,r.useEffect)((()=>{const e=tn({relativeTimeId:m,defaultDuration:f,defaultEndInput:Xt(d)});y&&p!==y&&x({id:e.relativeTimeId,duration:e.duration,until:e.endInput})}),[p,y]),ra(n,(t=>{var n,r;if(e)return;const a=t.target,i=(null===E||void 0===E?void 0:E.current)&&(null===E||void 0===E||null===(n=E.current)||void 0===n?void 0:n.contains(a)),o=(null===N||void 0===N?void 0:N.current)&&(null===N||void 0===N||null===(r=N.current)||void 0===r?void 0:r.contains(a));i||o||w()})),Ct(xt.FK,{children:[Ct("div",{ref:A,children:e?Ct("div",{className:"vm-mobile-option",onClick:b,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(Vn,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Time range"}),Ct("span",{className:"vm-mobile-option-text__value",children:C})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(fa,{title:o?"Time range controls":C,children:Ct(aa,{className:g?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(Vn,{}),onClick:b,ariaLabel:"time range controls",children:o&&Ct("span",{children:C})})})}),Ct(ia,{open:_,buttonRef:A,placement:"bottom-right",onClose:w,clickOutside:!1,title:e?"Time range controls":"",children:Ct("div",{className:xr()({"vm-time-selector":!0,"vm-time-selector_mobile":e}),ref:n,children:[Ct("div",{className:"vm-time-selector-left",children:[Ct("div",{className:xr()({"vm-time-selector-left-inputs":!0,"vm-time-selector-left-inputs_dark":t}),children:[Ct(ri,{value:c,label:"From:",pickerLabel:"Date From",pickerRef:E,onChange:u,onEnter:M}),Ct(ri,{value:l,label:"To:",pickerLabel:"Date To",pickerRef:N,onChange:s,onEnter:M})]}),Ct("div",{className:"vm-time-selector-left-timezone",children:[Ct("div",{className:"vm-time-selector-left-timezone__title",children:k.region}),Ct("div",{className:"vm-time-selector-left-timezone__utc",children:k.utc})]}),Ct(aa,{variant:"text",startIcon:Ct(Bn,{}),onClick:()=>v({type:"RUN_QUERY_TO_NOW"}),children:"switch to now"}),Ct("div",{className:"vm-time-selector-left__controls",children:[Ct(aa,{color:"error",variant:"outlined",onClick:()=>{s(Gt(Xt(d))),u(Gt(Xt(h))),w()},children:"Cancel"}),Ct(aa,{color:"primary",onClick:M,children:"Apply"})]})]}),Ct(Ba,{relativeTime:m||"",setDuration:x})]})})]})},ii=()=>{const e=ae(),[t,n]=je();return{setSearchParamsFromKeys:(0,r.useCallback)((r=>{const a=!!Array.from(t.values()).length;let i=!1;Object.entries(r).forEach((e=>{let[n,r]=e;t.get(n)!=="".concat(r)&&(t.set(n,"".concat(r)),i=!0)})),i&&(a?n(t):e("?".concat(t.toString()),{replace:!0}))}),[t,e])}},oi=()=>{const{isMobile:e}=Wr(),t=Ke(),n=(0,r.useRef)(null),[a]=je(),{setSearchParamsFromKeys:o}=ii(),l=a.get("date")||i()().tz().format(Tt),s=(0,r.useMemo)((()=>i().tz(l).format(Tt)),[l]),c=e=>{o({date:e})};return(0,r.useEffect)((()=>{c(l)}),[]),Ct("div",{children:[Ct("div",{ref:n,children:e?Ct("div",{className:"vm-mobile-option",children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(Un,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Date control"}),Ct("span",{className:"vm-mobile-option-text__value",children:s})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(fa,{title:"Date control",children:Ct(aa,{className:t?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(Un,{}),children:s})})}),Ct(Xa,{label:"Date control",date:l||"",format:Tt,onChange:c,targetRef:n})]})},li=[{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"}],si=()=>{const{isMobile:e}=Wr(),t=gn(),n=Ke(),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(li[0]),{value:s,toggle:c,setFalse:u}=oa(!1),d=(0,r.useRef)(null);(0,r.useEffect)((()=>{const e=o.seconds;let n;return a?n=setInterval((()=>{t({type:"RUN_QUERY"})}),1e3*e):l(li[0]),()=>{n&&clearInterval(n)}}),[o,a]);const h=e=>()=>{(e=>{(a&&!e.seconds||!a&&e.seconds)&&i((e=>!e)),l(e),u()})(e)};return Ct(xt.FK,{children:[Ct("div",{className:"vm-execution-controls",children:Ct("div",{className:xr()({"vm-execution-controls-buttons":!0,"vm-execution-controls-buttons_mobile":e,"vm-header-button":!n}),children:[!e&&Ct(fa,{title:"Refresh dashboard",children:Ct(aa,{variant:"contained",color:"primary",onClick:()=>{t({type:"RUN_QUERY"})},startIcon:Ct(jn,{}),ariaLabel:"refresh dashboard"})}),e?Ct("div",{className:"vm-mobile-option",onClick:c,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(In,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Auto-refresh"}),Ct("span",{className:"vm-mobile-option-text__value",children:o.title})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(fa,{title:"Auto-refresh control",children:Ct("div",{ref:d,children:Ct(aa,{variant:"contained",color:"primary",fullWidth:!0,endIcon:Ct("div",{className:xr()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":s}),children:Ct($n,{})}),onClick:c,children:o.title})})})]})}),Ct(ia,{open:s,placement:"bottom-right",onClose:u,buttonRef:d,title:e?"Auto-refresh duration":void 0,children:Ct("div",{className:xr()({"vm-execution-controls-list":!0,"vm-execution-controls-list_mobile":e}),children:li.map((t=>Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":e,"vm-list-item_active":t.seconds===o.seconds}),onClick:h(t),children:t.title},t.seconds)))})})]})},ci="Enable to save the modified server URL to local storage, preventing reset upon page refresh.",ui="Disable to stop saving the server URL to local storage, reverting to the default URL on page refresh.",di=e=>{let{serverUrl:t,stateServerUrl:n,onChange:a,onEnter:i}=e;const{value:o,toggle:l}=oa(!!Xe("SERVER_URL")),[s,c]=(0,r.useState)("");return(0,r.useEffect)((()=>{n||c(ht.emptyServer),yt(n)||c(ht.validServer)}),[n]),(0,r.useEffect)((()=>{o?Je("SERVER_URL",t):et(["SERVER_URL"])}),[o]),(0,r.useEffect)((()=>{o&&Je("SERVER_URL",t)}),[t]),Ct("div",{children:[Ct("div",{className:"vm-server-configurator__title",children:"Server URL"}),Ct("div",{className:"vm-server-configurator-url",children:[Ct($a,{autofocus:!0,value:t,error:s,onChange:e=>{a(e||""),c("")},onEnter:i,inputmode:"url"}),Ct(fa,{title:o?ui:ci,children:Ct(aa,{className:"vm-server-configurator-url__button",variant:"text",color:o?"primary":"gray",onClick:l,startIcon:Ct(ur,{})})})]})]})},hi=[{label:"Graph",type:dt.chart},{label:"JSON",type:dt.code},{label:"Table",type:dt.table}],mi=e=>{let{limits:t,onChange:n,onEnter:a}=e;const{isMobile:i}=Wr(),[o,l]=(0,r.useState)({table:"",chart:"",code:""}),s=e=>r=>{((e,r)=>{const a=e||"";l((e=>({...e,[r]:+a<0?ht.positiveNumber:""}))),n({...t,[r]:a||1/0})})(r,e)};return Ct("div",{className:"vm-limits-configurator",children:[Ct("div",{className:"vm-server-configurator__title",children:["Series limits by tabs",Ct(fa,{title:"Set to 0 to disable the limit",children:Ct(aa,{variant:"text",color:"primary",size:"small",startIcon:Ct(Rn,{})})}),Ct("div",{className:"vm-limits-configurator-title__reset",children:Ct(aa,{variant:"text",color:"primary",size:"small",startIcon:Ct(In,{}),onClick:()=>{n(ot)},children:"Reset limits"})})]}),Ct("div",{className:xr()({"vm-limits-configurator__inputs":!0,"vm-limits-configurator__inputs_mobile":i}),children:hi.map((e=>Ct("div",{children:Ct($a,{label:e.label,value:t[e.type],error:o[e.type],onChange:s(e.type),onEnter:a,type:"number"})},e.type)))})]})},pi=e=>{let{defaultExpanded:t=!1,onChange:n,title:a,children:i}=e;const[o,l]=(0,r.useState)(t);return(0,r.useEffect)((()=>{n&&n(o)}),[o]),Ct(xt.FK,{children:[Ct("header",{className:"vm-accordion-header ".concat(o&&"vm-accordion-header_open"),onClick:()=>{l((e=>!e))},children:[a,Ct("div",{className:"vm-accordion-header__arrow ".concat(o&&"vm-accordion-header__arrow_open"),children:Ct($n,{})})]}),o&&Ct("section",{className:"vm-accordion-section",children:i},"content")]})},fi=()=>Ct(fa,{title:"Browser timezone is not recognized, supported, or could not be determined.",children:Ct(Dn,{})}),vi=on(),gi=e=>{let{timezoneState:t,defaultTimezone:n,onChange:a}=e;const{isMobile:i}=Wr(),o=rn(),[l,s]=(0,r.useState)(""),c=(0,r.useRef)(null),{value:u,toggle:d,setFalse:h}=oa(!1),m=(0,r.useMemo)((()=>[{title:"Default time (".concat(n,")"),region:n,utc:n?nn(n):"UTC"},{title:vi.title,region:vi.region,utc:nn(vi.region),isInvalid:!vi.isValid},{title:"UTC (Coordinated Universal Time)",region:"UTC",utc:"UTC"}].filter((e=>e.region))),[n]),p=(0,r.useMemo)((()=>{if(!l)return o;try{return rn(l)}catch(ap){return{}}}),[l,o]),f=(0,r.useMemo)((()=>Object.keys(p)),[p]),v=(0,r.useMemo)((()=>({region:t,utc:nn(t)})),[t]),g=e=>()=>{(e=>{a(e.region),s(""),h()})(e)};return Ct("div",{className:"vm-timezones",children:[Ct("div",{className:"vm-server-configurator__title",children:"Time zone"}),Ct("div",{className:"vm-timezones-item vm-timezones-item_selected",onClick:d,ref:c,children:[Ct("div",{className:"vm-timezones-item__title",children:v.region}),Ct("div",{className:"vm-timezones-item__utc",children:v.utc}),Ct("div",{className:xr()({"vm-timezones-item__icon":!0,"vm-timezones-item__icon_open":u}),children:Ct(Hn,{})})]}),Ct(ia,{open:u,buttonRef:c,placement:"bottom-left",onClose:h,fullWidth:!0,title:i?"Time zone":void 0,children:Ct("div",{className:xr()({"vm-timezones-list":!0,"vm-timezones-list_mobile":i}),children:[Ct("div",{className:"vm-timezones-list-header",children:[Ct("div",{className:"vm-timezones-list-header__search",children:Ct($a,{autofocus:!0,label:"Search",value:l,onChange:e=>{s(e)}})}),m.map(((e,t)=>e&&Ct("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:g(e),children:[Ct("div",{className:"vm-timezones-item__title",children:[e.title,e.isInvalid&&Ct(fi,{})]}),Ct("div",{className:"vm-timezones-item__utc",children:e.utc})]},"".concat(t,"_").concat(e.region))))]}),f.map((e=>Ct("div",{className:"vm-timezones-list-group",children:Ct(pi,{defaultExpanded:!0,title:Ct("div",{className:"vm-timezones-list-group__title",children:e}),children:Ct("div",{className:"vm-timezones-list-group-options",children:p[e]&&p[e].map((e=>Ct("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:g(e),children:[Ct("div",{className:"vm-timezones-item__title",children:e.region}),Ct("div",{className:"vm-timezones-item__utc",children:e.utc})]},e.search)))})})},e)))]})})]})},yi=e=>{let{options:t,value:n,label:a,onChange:i}=e;const o=(0,r.useRef)(null),[l,s]=(0,r.useState)({width:"0px",left:"0px",borderRadius:"0px"}),c=e=>()=>{i(e)};return(0,r.useEffect)((()=>{if(!o.current)return void s({width:"0px",left:"0px",borderRadius:"0px"});const e=t.findIndex((e=>e.value===n)),{width:r}=o.current.getBoundingClientRect();let a=r,i=e*a,l="0";0===e&&(l="16px 0 0 16px"),e===t.length-1&&(l="10px",i-=1,l="0 16px 16px 0"),0!==e&&e!==t.length-1&&(a+=1,i-=1),s({width:"".concat(a,"px"),left:"".concat(i,"px"),borderRadius:l})}),[o,n,t]),Ct("div",{className:"vm-toggles",children:[a&&Ct("label",{className:"vm-toggles__label",children:a}),Ct("div",{className:"vm-toggles-group",style:{gridTemplateColumns:"repeat(".concat(t.length,", 1fr)")},children:[l.borderRadius&&Ct("div",{className:"vm-toggles-group__highlight",style:l}),t.map(((e,t)=>Ct("div",{className:xr()({"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)))]})]})},_i=Object.values(mt).map((e=>({title:e,value:e}))),bi=e=>{let{theme:t,onChange:n}=e;const{isMobile:r}=Wr();return Ct("div",{className:xr()({"vm-theme-control":!0,"vm-theme-control_mobile":r}),children:[Ct("div",{className:"vm-server-configurator__title",children:"Theme preferences"}),Ct("div",{className:"vm-theme-control__toggle",children:Ct(yi,{options:_i,value:t,onChange:e=>{n(e)}})},"".concat(r))]})},wi="Settings",{REACT_APP_TYPE:ki}={},xi=ki===$e.logs,Si=()=>{const{isMobile:e}=Wr(),t=Ke(),{serverUrl:n,theme:a}=Nt(),{timezone:i,defaultTimezone:o}=vn(),{seriesLimits:l}=Fr(),s=At(),c=gn(),u=jr(),[d,h]=(0,r.useState)(n),[m,p]=(0,r.useState)(l),[f,v]=(0,r.useState)(i),{value:g,setTrue:y,setFalse:_}=oa(!1),b=()=>{_(),h(n),p(l),v(i)},w=()=>{const e=Ge(d);""!==e&&s({type:"SET_TENANT_ID",payload:e}),s({type:"SET_SERVER",payload:d}),c({type:"SET_TIMEZONE",payload:f}),u({type:"SET_SERIES_LIMITS",payload:m}),_()};(0,r.useEffect)((()=>{n!==d&&h(n)}),[n]),(0,r.useEffect)((()=>{v(i)}),[i]);const k=[{show:!t&&!xi,component:Ct(di,{stateServerUrl:n,serverUrl:d,onChange:h,onEnter:w})},{show:!xi,component:Ct(mi,{limits:m,onChange:p,onEnter:w})},{show:!0,component:Ct(gi,{timezoneState:f,defaultTimezone:o,onChange:v})},{show:!t,component:Ct(bi,{theme:a,onChange:e=>{s({type:"SET_THEME",payload:e})}})}].filter((e=>e.show));return Ct(xt.FK,{children:[e?Ct("div",{className:"vm-mobile-option",onClick:y,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(On,{})}),Ct("div",{className:"vm-mobile-option-text",children:Ct("span",{className:"vm-mobile-option-text__label",children:wi})}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct($n,{})})]}):Ct(fa,{title:wi,children:Ct(aa,{className:xr()({"vm-header-button":!t}),variant:"contained",color:"primary",startIcon:Ct(On,{}),onClick:y,ariaLabel:"settings"})}),g&&Ct(pa,{title:wi,onClose:b,children:Ct("div",{className:xr()({"vm-server-configurator":!0,"vm-server-configurator_mobile":e}),children:[k.map(((e,t)=>Ct("div",{className:"vm-server-configurator__input",children:e.component},t))),Ct("div",{className:"vm-server-configurator-footer",children:[Ct(aa,{color:"error",variant:"outlined",onClick:b,children:"Cancel"}),Ct(aa,{color:"primary",variant:"contained",onClick:w,children:"Apply"})]})]})})]})},Ci=e=>{let{displaySidebar:t,isMobile:n,headerSetup:r,accountIds:a}=e;return Ct("div",{className:xr()({"vm-header-controls":!0,"vm-header-controls_mobile":n}),children:[(null===r||void 0===r?void 0:r.tenant)&&Ct(Ha,{accountIds:a||[]}),(null===r||void 0===r?void 0:r.stepControl)&&Ct(Ua,{}),(null===r||void 0===r?void 0:r.timeSelector)&&Ct(ai,{}),(null===r||void 0===r?void 0:r.cardinalityDatePicker)&&Ct(oi,{}),(null===r||void 0===r?void 0:r.executionControls)&&Ct(si,{}),Ct(Si,{}),!t&&Ct(Na,{})]})},Ei=Boolean(Xe("DISABLED_DEFAULT_TIMEZONE")),Ni=()=>{const{serverUrl:e}=Nt(),t=gn(),[n,a]=(0,r.useState)(!1),[o,l]=(0,r.useState)(""),s=async()=>{if(e&&!{NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){l(""),a(!0);try{const n=await fetch("".concat(e,"/vmui/timezone")),r=await n.json();n.ok?((e=>{const n="local"===e.toLowerCase()?on().region:e;try{if(i()().tz(n).isValid(),t({type:"SET_DEFAULT_TIMEZONE",payload:n}),Ei)return;t({type:"SET_TIMEZONE",payload:n})}catch(ap){ap instanceof Error&&l("".concat(ap.name,": ").concat(ap.message))}})(r.timezone),a(!1)):(l(r.error),a(!1))}catch(ap){a(!1),ap instanceof Error&&l("".concat(ap.name,": ").concat(ap.message))}}};return(0,r.useEffect)((()=>{s()}),[e]),{isLoading:n,error:o}},Ai=()=>{const e=Ke(),{isMobile:t}=Wr(),{pathname:n}=ne(),[a,i]=je();Fa(),Ni();return(0,r.useEffect)((()=>{var e;const t="vmui",r=null===(e=qe[n])||void 0===e?void 0:e.title;document.title=r?"".concat(r," - ").concat(t):t}),[n]),(0,r.useEffect)((()=>{const{search:e,href:t}=window.location;if(e){const t=rt().parse(e,{ignoreQueryPrefix:!0});Object.entries(t).forEach((e=>{let[t,n]=e;return a.set(t,n)})),i(a),window.location.search=""}const n=t.replace(/\/\?#\//,"/#/");n!==t&&window.location.replace(n)}),[]),Ct("section",{className:"vm-container",children:[Ct(Da,{controlsComponent:Ci}),Ct("div",{className:xr()({"vm-container-body":!0,"vm-container-body_mobile":t,"vm-container-body_app":e}),children:Ct(_e,{})}),!e&&Ct(za,{})]})};var Mi=function(e){return e[e.mouse=0]="mouse",e[e.keyboard=1]="keyboard",e}(Mi||{});const Ti=e=>{var t;let{value:n,options:a,anchor:i,disabled:o,minLength:l=2,fullWidth:s,selected:c,noOptionsText:u,label:d,disabledFullScreen:h,offset:m,maxDisplayResults:p,loading:f,onSelect:v,onOpenAutocomplete:g,onFoundOptions:y,onChangeWrapperRef:_}=e;const{isMobile:b}=Wr(),w=(0,r.useRef)(null),[k,x]=(0,r.useState)({index:-1}),[S,C]=(0,r.useState)(""),[E,N]=(0,r.useState)(0),{value:A,setValue:M,setFalse:T}=oa(!1),L=(0,r.useMemo)((()=>{if(!A)return[];try{const e=new RegExp(String(n.trim()),"i"),t=a.filter((t=>e.test(t.value))).sort(((t,r)=>{var a,i;return t.value.toLowerCase()===n.trim().toLowerCase()?-1:r.value.toLowerCase()===n.trim().toLowerCase()?1:((null===(a=t.value.match(e))||void 0===a?void 0:a.index)||0)-((null===(i=r.value.match(e))||void 0===i?void 0:i.index)||0)}));return N(t.length),C(t.length>Number(null===p||void 0===p?void 0:p.limit)&&(null===p||void 0===p?void 0:p.message)||""),null!==p&&void 0!==p&&p.limit?t.slice(0,p.limit):t}catch(ap){return[]}}),[A,a,n]),O=(0,r.useMemo)((()=>{var e;return 1===L.length&&(null===(e=L[0])||void 0===e?void 0:e.value)===n}),[L]),P=(0,r.useMemo)((()=>u&&!L.length),[u,L]),I=()=>{x({index:-1})},R=(0,r.useCallback)((e=>{const{key:t,ctrlKey:n,metaKey:r,shiftKey:a}=e,i=n||r||a,o=L.length;if("ArrowUp"===t&&!i&&o&&(e.preventDefault(),x((e=>{let{index:t}=e;return{index:t<=0?0:t-1,type:Mi.keyboard}}))),"ArrowDown"===t&&!i&&o){e.preventDefault();const t=L.length-1;x((e=>{let{index:n}=e;return{index:n>=t?t:n+1,type:Mi.keyboard}}))}if("Enter"===t){const e=L[k.index];e&&v(e.value),c||T()}"Escape"===t&&T()}),[k,L,T,v,c]);return(0,r.useEffect)((()=>{M(n.length>=l)}),[n,a]),Nr("keydown",R),(0,r.useEffect)((()=>{if(!w.current||k.type===Mi.mouse)return;const e=w.current.childNodes[k.index];null!==e&&void 0!==e&&e.scrollIntoView&&e.scrollIntoView({block:"center"})}),[k,L]),(0,r.useEffect)((()=>{x({index:-1})}),[L]),(0,r.useEffect)((()=>{g&&g(A)}),[A]),(0,r.useEffect)((()=>{y&&y(O?[]:L)}),[L,O]),(0,r.useEffect)((()=>{_&&_(w)}),[w]),Ct(ia,{open:A,buttonRef:i,placement:"bottom-left",onClose:T,fullWidth:s,title:b?d:void 0,disabledFullScreen:h,offset:m,children:[Ct("div",{className:xr()({"vm-autocomplete":!0,"vm-autocomplete_mobile":b&&!h}),ref:w,children:[f&&Ct("div",{className:"vm-autocomplete__loader",children:[Ct(jn,{}),Ct("span",{children:"Loading..."})]}),P&&Ct("div",{className:"vm-autocomplete__no-options",children:u}),!O&&L.map(((e,t)=>{return Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":b,"vm-list-item_active":t===k.index,"vm-list-item_multiselect":c,"vm-list-item_multiselect_selected":null===c||void 0===c?void 0:c.includes(e.value),"vm-list-item_with-icon":e.icon}),id:"$autocomplete$".concat(e.value),onClick:(r=e.value,()=>{o||(v(r),c||T())}),onMouseEnter:(n=t,()=>{x({index:n,type:Mi.mouse})}),onMouseLeave:I,children:[(null===c||void 0===c?void 0:c.includes(e.value))&&Ct(er,{}),Ct(xt.FK,{children:e.icon}),Ct("span",{children:e.value})]},"".concat(t).concat(e.value));var n,r}))]}),S&&Ct("div",{className:"vm-autocomplete-message",children:["Shown ",null===p||void 0===p?void 0:p.limit," results out of ",E,". ",S]}),(null===(t=L[k.index])||void 0===t?void 0:t.description)&&Ct("div",{className:"vm-autocomplete-info",children:[Ct("div",{className:"vm-autocomplete-info__type",children:L[k.index].type}),Ct("div",{className:"vm-autocomplete-info__description",dangerouslySetInnerHTML:{__html:L[k.index].description||""}})]})]})};var Li=n(267),Oi=n.n(Li);const Pi=e=>e.replace(/[/\-\\^$*+?.()|[\]{}]/g,"\\$&"),Ii=e=>JSON.stringify(e).slice(1,-1);var Ri=function(e){return e.metric="metric",e.label="label",e.labelValue="labelValue",e}(Ri||{});const Di={[Ri.metric]:Ct(gr,{}),[Ri.label]:Ct(_r,{}),[Ri.labelValue]:Ct(br,{})},zi=e=>{let t=e.length,n=-1/0;for(;t--;){const r=e[t];Number.isFinite(r)&&r>n&&(n=r)}return Number.isFinite(n)?n:null},Fi=e=>{let t=e.length,n=1/0;for(;t--;){const r=e[t];Number.isFinite(r)&&r{let t=e.length;const n=[];for(;t--;){const r=e[t];Number.isFinite(r)&&n.push(r)}return n.sort(),n[n.length>>1]},$i=e=>{let t=e.length;for(;t--;){const n=e[t];if(Number.isFinite(n))return n}},Hi=(e,t,n)=>{if(void 0===e||null===e)return"";n=n||0,t=t||0;const r=Math.abs(n-t);if(isNaN(r)||0==r)return Math.abs(e)>=1e3?e.toLocaleString("en-US"):e.toString();let a=3+Math.floor(1+Math.log10(Math.max(Math.abs(t),Math.abs(n)))-Math.log10(r));return(isNaN(a)||a>20)&&(a=20),e.toLocaleString("en-US",{minimumSignificantDigits:1,maximumSignificantDigits:a})},Vi=(e,t)=>{const 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);const r=n.offsetWidth;return n.remove(),r},Ui=e=>{const t=(null===e||void 0===e?void 0:e.metric)||{},n=Object.keys(t).filter((e=>"__name__"!=e)).map((e=>"".concat(e,"=").concat(JSON.stringify(t[e]))));let r=t.__name__||"";return n.length>0&&(r+="{"+n.join(",")+"}"),r},Bi=[[31536e3,"{YYYY}",null,null,null,null,null,null,1],[2419200,"{MMM}","\n{YYYY}",null,null,null,null,null,1],[86400,"{MM}-{DD}","\n{YYYY}",null,null,null,null,null,1],[3600,"{HH}:{mm}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD}",null,null,null,1],[60,"{HH}:{mm}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD}",null,null,null,1],[1,"{HH}:{mm}:{ss}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD} {HH}:{mm}",null,null,null,1],[.001,":{ss}.{fff}","\n{YYYY}-{MM}-{DD} {HH}:{mm}",null,"\n{MM}-{DD} {HH}:{mm}",null,"\n{HH}:{mm}",null,1]],qi=(e,t)=>Array.from(new Set(e.map((e=>e.scale)))).map((e=>{const n="10px Arial",r=ft("color-text"),a={scale:e,show:!0,size:Wi,stroke:r,font:n,values:(e,n)=>function(e,t){let n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"";const r=t[0],a=t[t.length-1];return n?t.map((e=>"".concat(Hi(e,r,a)," ").concat(n))):t.map((e=>Hi(e,r,a)))}(e,n,t)};return e?Number(e)%2?a:{...a,side:1}:{space:80,values:Bi,stroke:r,font:n}})),Yi=(e,t)=>{if(null==e||null==t)return[-1,1];const n=.02*(Math.abs(t-e)||Math.abs(e)||1);return[e-n,t+n]},Wi=(e,t,n,r)=>{var a;const i=e.axes[n];if(r>1)return i._size||60;let o=6+((null===i||void 0===i||null===(a=i.ticks)||void 0===a?void 0:a.size)||0)+(i.gap||0);const l=(null!==t&&void 0!==t?t:[]).reduce(((e,t)=>(null===t||void 0===t?void 0:t.length)>e.length?t:e),"");return""!=l&&(o+=Vi(l,"10px Arial")),Math.ceil(o)},Ki="u-off",Qi="u-label",Zi="width",Gi="height",Ji="top",Xi="bottom",eo="left",to="right",no="#000",ro=no+"0",ao="mousemove",io="mousedown",oo="mouseup",lo="mouseenter",so="mouseleave",co="dblclick",uo="change",ho="dppxchange",mo="--",po="undefined"!=typeof window,fo=po?document:null,vo=po?window:null,go=po?navigator:null;let yo,_o;function bo(e,t){if(null!=t){let n=e.classList;!n.contains(t)&&n.add(t)}}function wo(e,t){let n=e.classList;n.contains(t)&&n.remove(t)}function ko(e,t,n){e.style[t]=n+"px"}function xo(e,t,n,r){let a=fo.createElement(e);return null!=t&&bo(a,t),null!=n&&n.insertBefore(a,r),a}function So(e,t){return xo("div",e,t)}const Co=new WeakMap;function Eo(e,t,n,r,a){let i="translate("+t+"px,"+n+"px)";i!=Co.get(e)&&(e.style.transform=i,Co.set(e,i),t<0||n<0||t>r||n>a?bo(e,Ki):wo(e,Ki))}const No=new WeakMap;function Ao(e,t,n){let r=t+n;r!=No.get(e)&&(No.set(e,r),e.style.background=t,e.style.borderColor=n)}const Mo=new WeakMap;function To(e,t,n,r){let a=t+""+n;a!=Mo.get(e)&&(Mo.set(e,a),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)}const Lo={passive:!0},Oo={...Lo,capture:!0};function Po(e,t,n,r){t.addEventListener(e,n,r?Oo:Lo)}function Io(e,t,n,r){t.removeEventListener(e,n,r?Oo:Lo)}function Ro(e,t,n,r){let a;n=n||0;let i=(r=r||t.length-1)<=2147483647;for(;r-n>1;)a=i?n+r>>1:Go((n+r)/2),t[a]=t&&a<=n;a+=r)if(null!=e[a])return a;return-1}function zo(e,t,n,r){let a=rl(e),i=rl(t);e==t&&(-1==a?(e*=n,t/=n):(e/=n,t*=n));let o=10==n?al:il,l=1==i?Xo:Go,s=(1==a?Go:Xo)(o(Zo(e))),c=l(o(Zo(t))),u=nl(n,s),d=nl(n,c);return 10==n&&(s<0&&(u=bl(u,-s)),c<0&&(d=bl(d,-c))),r||2==n?(e=u*a,t=d*i):(e=_l(e,u),t=yl(t,d)),[e,t]}function Fo(e,t,n,r){let a=zo(e,t,n,r);return 0==e&&(a[0]=0),0==t&&(a[1]=0),a}po&&function e(){let t=devicePixelRatio;yo!=t&&(yo=t,_o&&Io(uo,_o,e),_o=matchMedia("(min-resolution: ".concat(yo-.001,"dppx) and (max-resolution: ").concat(yo+.001,"dppx)")),Po(uo,_o,e),vo.dispatchEvent(new CustomEvent(ho)))}();const jo=.1,$o={mode:3,pad:jo},Ho={pad:0,soft:null,mode:0},Vo={min:Ho,max:Ho};function Uo(e,t,n,r){return Tl(n)?qo(e,t,n):(Ho.pad=n,Ho.soft=r?0:null,Ho.mode=r?3:0,qo(e,t,Vo))}function Bo(e,t){return null==e?t:e}function qo(e,t,n){let r=n.min,a=n.max,i=Bo(r.pad,0),o=Bo(a.pad,0),l=Bo(r.hard,-ll),s=Bo(a.hard,ll),c=Bo(r.soft,ll),u=Bo(a.soft,-ll),d=Bo(r.mode,0),h=Bo(a.mode,0),m=t-e,p=al(m),f=tl(Zo(e),Zo(t)),v=al(f),g=Zo(v-p);(m<1e-9||g>10)&&(m=0,0!=e&&0!=t||(m=1e-9,2==d&&c!=ll&&(i=0),2==h&&u!=-ll&&(o=0)));let y=m||f||1e3,_=al(y),b=nl(10,Go(_)),w=bl(_l(e-y*(0==m?0==e?.1:1:i),b/10),9),k=e>=c&&(1==d||3==d&&w<=c||2==d&&w>=c)?c:ll,x=tl(l,w=k?k:el(k,w)),S=bl(yl(t+y*(0==m?0==t?.1:1:o),b/10),9),C=t<=u&&(1==h||3==h&&S>=u||2==h&&S<=u)?u:-ll,E=el(s,S>C&&t<=C?C:tl(C,S));return x==E&&0==x&&(E=100),[x,E]}const Yo=new Intl.NumberFormat(po?go.language:"en-US"),Wo=e=>Yo.format(e),Ko=Math,Qo=Ko.PI,Zo=Ko.abs,Go=Ko.floor,Jo=Ko.round,Xo=Ko.ceil,el=Ko.min,tl=Ko.max,nl=Ko.pow,rl=Ko.sign,al=Ko.log10,il=Ko.log2,ol=function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return Ko.asinh(e/t)},ll=1/0;function sl(e){return 1+(0|al((e^e>>31)-(e>>31)))}function cl(e,t,n){return el(tl(e,t),n)}function ul(e){return"function"==typeof e?e:()=>e}const dl=e=>e,hl=(e,t)=>t,ml=e=>null,pl=e=>!0,fl=(e,t)=>e==t,vl=e=>bl(e,14);function gl(e,t){return vl(bl(vl(e/t))*t)}function yl(e,t){return vl(Xo(vl(e/t))*t)}function _l(e,t){return vl(Go(vl(e/t))*t)}function bl(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0;if(Al(e))return e;let n=10**t,r=e*n*(1+Number.EPSILON);return Jo(r)/n}const wl=new Map;function kl(e){return((""+e).split(".")[1]||"").length}function xl(e,t,n,r){let a=[],i=r.map(kl);for(let o=t;o=0&&o>=0?0:t)+(o>=i[e]?0:i[e]),c=bl(l,s);a.push(c),wl.set(c,s)}}return a}const Sl={},Cl=[],El=[null,null],Nl=Array.isArray,Al=Number.isInteger;function Ml(e){return"string"==typeof e}function Tl(e){let t=!1;if(null!=e){let n=e.constructor;t=null==n||n==Object}return t}function Ll(e){return null!=e&&"object"==typeof e}const Ol=Object.getPrototypeOf(Uint8Array);function Pl(e){let t,n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:Tl;if(Nl(e)){let r=e.find((e=>null!=e));if(Nl(r)||n(r)){t=Array(e.length);for(let r=0;ri){for(r=o-1;r>=0&&null==e[r];)e[r--]=null;for(r=o+1;rPromise.resolve().then(e):queueMicrotask;const zl=["January","February","March","April","May","June","July","August","September","October","November","December"],Fl=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"];function jl(e){return e.slice(0,3)}const $l=Fl.map(jl),Hl=zl.map(jl),Vl={MMMM:zl,MMM:Hl,WWWW:Fl,WWW:$l};function Ul(e){return(e<10?"0":"")+e}const Bl={YYYY:e=>e.getFullYear(),YY:e=>(e.getFullYear()+"").slice(2),MMMM:(e,t)=>t.MMMM[e.getMonth()],MMM:(e,t)=>t.MMM[e.getMonth()],MM:e=>Ul(e.getMonth()+1),M:e=>e.getMonth()+1,DD:e=>Ul(e.getDate()),D:e=>e.getDate(),WWWW:(e,t)=>t.WWWW[e.getDay()],WWW:(e,t)=>t.WWW[e.getDay()],HH:e=>Ul(e.getHours()),H:e=>e.getHours(),h:e=>{let t=e.getHours();return 0==t?12:t>12?t-12:t},AA:e=>e.getHours()>=12?"PM":"AM",aa:e=>e.getHours()>=12?"pm":"am",a:e=>e.getHours()>=12?"p":"a",mm:e=>Ul(e.getMinutes()),m:e=>e.getMinutes(),ss:e=>Ul(e.getSeconds()),s:e=>e.getSeconds(),fff:e=>{return((t=e.getMilliseconds())<10?"00":t<100?"0":"")+t;var t}};function ql(e,t){t=t||Vl;let n,r=[],a=/\{([a-z]+)\}|[^{]+/gi;for(;n=a.exec(e);)r.push("{"==n[0][0]?Bl[n[1]]:n[0]);return e=>{let n="";for(let a=0;ae%1==0,Kl=[1,2,2.5,5],Ql=xl(10,-16,0,Kl),Zl=xl(10,0,16,Kl),Gl=Zl.filter(Wl),Jl=Ql.concat(Zl),Xl="{YYYY}",es="\n"+Xl,ts="{M}/{D}",ns="\n"+ts,rs=ns+"/{YY}",as="{aa}",is="{h}:{mm}"+as,os="\n"+is,ls=":{ss}",ss=null;function cs(e){let t=1e3*e,n=60*t,r=60*n,a=24*r,i=30*a,o=365*a;return[(1==e?xl(10,0,3,Kl).filter(Wl):xl(10,-3,0,Kl)).concat([t,5*t,10*t,15*t,30*t,n,5*n,10*n,15*n,30*n,r,2*r,3*r,4*r,6*r,8*r,12*r,a,2*a,3*a,4*a,5*a,6*a,7*a,8*a,9*a,10*a,15*a,i,2*i,3*i,4*i,6*i,o,2*o,5*o,10*o,25*o,50*o,100*o]),[[o,Xl,ss,ss,ss,ss,ss,ss,1],[28*a,"{MMM}",es,ss,ss,ss,ss,ss,1],[a,ts,es,ss,ss,ss,ss,ss,1],[r,"{h}"+as,rs,ss,ns,ss,ss,ss,1],[n,is,rs,ss,ns,ss,ss,ss,1],[t,ls,rs+" "+is,ss,ns+" "+is,ss,os,ss,1],[e,ls+".{fff}",rs+" "+is,ss,ns+" "+is,ss,os,ss,1]],function(t){return(l,s,c,u,d,h)=>{let m=[],p=d>=o,f=d>=i&&d=a?a:d,o=_+(Go(c)-Go(g))+yl(g-_,i);m.push(o);let p=t(o),f=p.getHours()+p.getMinutes()/n+p.getSeconds()/r,v=d/r,y=h/l.axes[s]._space;for(;o=bl(o+d,1==e?0:3),!(o>u);)if(v>1){let e=Go(bl(f+v,6))%24,n=t(o).getHours()-e;n>1&&(n=-1),o-=n*r,f=(f+v)%24,bl((o-m[m.length-1])/d,3)*y>=.7&&m.push(o)}else m.push(o)}return m}}]}const[us,ds,hs]=cs(1),[ms,ps,fs]=cs(.001);function vs(e,t){return e.map((e=>e.map(((n,r)=>0==r||8==r||null==n?n:t(1==r||0==e[8]?n:e[1]+n)))))}function gs(e,t){return(n,r,a,i,o)=>{let l,s,c,u,d,h,m=t.find((e=>o>=e[0]))||t[t.length-1];return r.map((t=>{let n=e(t),r=n.getFullYear(),a=n.getMonth(),i=n.getDate(),o=n.getHours(),p=n.getMinutes(),f=n.getSeconds(),v=r!=l&&m[2]||a!=s&&m[3]||i!=c&&m[4]||o!=u&&m[5]||p!=d&&m[6]||f!=h&&m[7]||m[1];return l=r,s=a,c=i,u=o,d=p,h=f,v(n)}))}}function ys(e,t,n){return new Date(e,t,n)}function _s(e,t){return t(e)}xl(2,-53,53,[1]);function bs(e,t){return(n,r,a,i)=>null==i?mo:t(e(r))}const ws={show:!0,live:!0,isolate:!1,mount:()=>{},markers:{show:!0,width:2,stroke:function(e,t){let 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:[]};const ks=[0,0];function xs(e,t,n){let r=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];return e=>{0==e.button&&(!r||e.target==t)&&n(e)}}function Ss(e,t,n){let r=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];return e=>{(!r||e.target==t)&&n(e)}}const Cs={show:!0,x:!0,y:!0,lock:!1,move:function(e,t,n){return ks[0]=t,ks[1]=n,ks},points:{show:function(e,t){let n=e.cursor.points,r=So(),a=n.size(e,t);ko(r,Zi,a),ko(r,Gi,a);let i=a/-2;ko(r,"marginLeft",i),ko(r,"marginTop",i);let o=n.width(e,t,a);return o&&ko(r,"borderWidth",o),r},size:function(e,t){return e.series[t].points.size},width:0,stroke:function(e,t){let n=e.series[t].points;return n._stroke||n._fill},fill:function(e,t){let n=e.series[t].points;return n._fill||n._stroke}},bind:{mousedown:xs,mouseup:xs,click:xs,dblclick:xs,mousemove:Ss,mouseleave:Ss,mouseenter:Ss},drag:{setScale:!0,x:!0,y:!1,dist:0,uni:null,click:(e,t)=>{t.stopPropagation(),t.stopImmediatePropagation()},_x:!1,_y:!1},focus:{dist:(e,t,n,r,a)=>r-a,prox:-1,bias:0},hover:{skip:[void 0],prox:null,bias:0},left:-10,top:-10,idx:null,dataIdx:null,idxs:null,event:null},Es={show:!0,stroke:"rgba(0,0,0,0.07)",width:2},Ns=Il({},Es,{filter:hl}),As=Il({},Ns,{size:10}),Ms=Il({},Es,{show:!1}),Ts='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"',Ls="bold "+Ts,Os={show:!0,scale:"x",stroke:no,space:50,gap:5,size:50,labelGap:0,labelSize:30,labelFont:Ls,side:2,grid:Ns,ticks:As,border:Ms,font:Ts,lineGap:1.5,rotate:0},Ps={show:!0,scale:"x",auto:!1,sorted:1,min:ll,max:-ll,idxs:[]};function Is(e,t,n,r,a){return t.map((e=>null==e?"":Wo(e)))}function Rs(e,t,n,r,a,i,o){let l=[],s=wl.get(a)||0;for(let c=n=o?n:bl(yl(n,a),s);c<=r;c=bl(c+a,s))l.push(Object.is(c,-0)?0:c);return l}function Ds(e,t,n,r,a,i,o){const l=[],s=e.scales[e.axes[t].scale].log,c=Go((10==s?al:il)(n));a=nl(s,c),10==s&&c<0&&(a=bl(a,-c));let u=n;do{l.push(u),u+=a,10==s&&(u=bl(u,wl.get(a))),u>=a*s&&(a=u)}while(u<=r);return l}function zs(e,t,n,r,a,i,o){let l=e.scales[e.axes[t].scale].asinh,s=r>l?Ds(e,t,tl(l,n),r,a):[l],c=r>=0&&n<=0?[0]:[];return(n<-l?Ds(e,t,tl(l,-r),-n,a):[l]).reverse().map((e=>-e)).concat(c,s)}const Fs=/./,js=/[12357]/,$s=/[125]/,Hs=/1/,Vs=(e,t,n,r)=>e.map(((e,a)=>4==t&&0==e||a%r==0&&n.test(e.toExponential()[e<0?1:0])?e:null));function Us(e,t,n,r,a){let i=e.axes[n],o=i.scale,l=e.scales[o],s=e.valToPos,c=i._space,u=s(10,o),d=s(9,o)-u>=c?Fs:s(7,o)-u>=c?js:s(5,o)-u>=c?$s:Hs;if(d==Hs){let e=Zo(s(1,o)-u);if(ea,Qs={show:!0,auto:!0,sorted:0,gaps:Ks,alpha:1,facets:[Il({},Ws,{scale:"x"}),Il({},Ws,{scale:"y"})]},Zs={scale:"y",auto:!0,sorted:0,show:!0,spanGaps:!1,gaps:Ks,alpha:1,points:{show:function(e,t){let{scale:n,idxs:r}=e.series[0],a=e._data[0],i=e.valToPos(a[r[0]],n,!0),o=e.valToPos(a[r[1]],n,!0),l=Zo(o-i)/(e.series[t].points.space*yo);return r[1]-r[0]<=l},filter:null},values:null,min:ll,max:-ll,idxs:[],path:null,clip:null};function Gs(e,t,n,r,a){return n/10}const Js={time:!0,auto:!0,distr:1,log:10,asinh:1,min:null,max:null,dir:1,ori:0},Xs=Il({},Js,{time:!1,ori:1}),ec={};function tc(e,t){let n=ec[e];return n||(n={key:e,plots:[],sub(e){n.plots.push(e)},unsub(e){n.plots=n.plots.filter((t=>t!=e))},pub(e,t,r,a,i,o,l){for(let s=0;s{let f=e.pxRound;const v=l.dir*(0==l.ori?1:-1),g=0==l.ori?mc:pc;let y,_;1==v?(y=n,_=r):(y=r,_=n);let b=f(c(t[y],l,m,d)),w=f(u(o[y],s,p,h)),k=f(c(t[_],l,m,d)),x=f(u(1==i?s.max:s.min,s,p,h)),S=new Path2D(a);return g(S,k,x),g(S,b,x),g(S,b,w),S}))}function lc(e,t,n,r,a,i){let o=null;if(e.length>0){o=new Path2D;const l=0==t?fc:vc;let s=n;for(let t=0;tn[0]){let e=n[0]-s;e>0&&l(o,s,r,e,r+i),s=n[1]}}let c=n+a-s,u=10;c>0&&l(o,s,r-u/2,c,r+i+u)}return o}function sc(e,t,n,r,a,i,o){let l=[],s=e.length;for(let c=1==a?n:r;c>=n&&c<=r;c+=a){if(null===t[c]){let u=c,d=c;if(1==a)for(;++c<=r&&null===t[c];)d=c;else for(;--c>=n&&null===t[c];)d=c;let h=i(e[u]),m=d==u?h:i(e[d]),p=u-a;h=o<=0&&p>=0&&p=0&&f>=0&&f=h&&l.push([h,m])}}return l}function cc(e){return 0==e?dl:1==e?Jo:t=>gl(t,e)}function uc(e){let t=0==e?dc:hc,n=0==e?(e,t,n,r,a,i)=>{e.arcTo(t,n,r,a,i)}:(e,t,n,r,a,i)=>{e.arcTo(n,t,a,r,i)},r=0==e?(e,t,n,r,a)=>{e.rect(t,n,r,a)}:(e,t,n,r,a)=>{e.rect(n,t,a,r)};return function(e,a,i,o,l){let s=arguments.length>5&&void 0!==arguments[5]?arguments[5]:0,c=arguments.length>6&&void 0!==arguments[6]?arguments[6]:0;0==s&&0==c?r(e,a,i,o,l):(s=el(s,o/2,l/2),c=el(c,o/2,l/2),t(e,a+s,i),n(e,a+o,i,a+o,i+l,s),n(e,a+o,i+l,a,i+l,c),n(e,a,i+l,a,i,c),n(e,a,i,a+o,i,s),e.closePath())}}const dc=(e,t,n)=>{e.moveTo(t,n)},hc=(e,t,n)=>{e.moveTo(n,t)},mc=(e,t,n)=>{e.lineTo(t,n)},pc=(e,t,n)=>{e.lineTo(n,t)},fc=uc(0),vc=uc(1),gc=(e,t,n,r,a,i)=>{e.arc(t,n,r,a,i)},yc=(e,t,n,r,a,i)=>{e.arc(n,t,r,a,i)},_c=(e,t,n,r,a,i,o)=>{e.bezierCurveTo(t,n,r,a,i,o)},bc=(e,t,n,r,a,i,o)=>{e.bezierCurveTo(n,t,a,r,o,i)};function wc(e){return(e,t,n,r,a)=>rc(e,t,((t,i,o,l,s,c,u,d,h,m,p)=>{let f,v,{pxRound:g,points:y}=t;0==l.ori?(f=dc,v=gc):(f=hc,v=yc);const _=bl(y.width*yo,3);let b=(y.size-y.width)/2*yo,w=bl(2*b,3),k=new Path2D,x=new Path2D,{left:S,top:C,width:E,height:N}=e.bbox;fc(x,S-w,C-w,E+2*w,N+2*w);const A=e=>{if(null!=o[e]){let t=g(c(i[e],l,m,d)),n=g(u(o[e],s,p,h));f(k,t+b,n),v(k,t,n,b,0,2*Qo)}};if(a)a.forEach(A);else for(let e=n;e<=r;e++)A(e);return{stroke:_>0?k:null,fill:k,clip:x,flags:2|nc}}))}function kc(e){return(t,n,r,a,i,o)=>{r!=a&&(i!=r&&o!=r&&e(t,n,r),i!=a&&o!=a&&e(t,n,a),e(t,n,o))}}const xc=kc(mc),Sc=kc(pc);function Cc(e){const t=Bo(null===e||void 0===e?void 0:e.alignGaps,0);return(e,n,r,a)=>rc(e,n,((i,o,l,s,c,u,d,h,m,p,f)=>{let v,g,y=i.pxRound,_=e=>y(u(e,s,p,h)),b=e=>y(d(e,c,f,m));0==s.ori?(v=mc,g=xc):(v=pc,g=Sc);const w=s.dir*(0==s.ori?1:-1),k={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:nc},x=k.stroke;let S,C,E,N=ll,A=-ll,M=_(o[1==w?r:a]),T=Do(l,r,a,1*w),L=Do(l,r,a,-1*w),O=_(o[T]),P=_(o[L]),I=!1;for(let e=1==w?r:a;e>=r&&e<=a;e+=w){let t=_(o[e]),n=l[e];t==M?null!=n?(C=b(n),N==ll&&(v(x,t,C),S=C),N=el(C,N),A=tl(C,A)):null===n&&(I=!0):(N!=ll&&(g(x,M,N,A,S,C),E=M),null!=n?(C=b(n),v(x,t,C),N=A=S=C):(N=ll,A=-ll,null===n&&(I=!0)),M=t)}N!=ll&&N!=A&&E!=M&&g(x,M,N,A,S,C);let[R,D]=ac(e,n);if(null!=i.fill||0!=R){let t=k.fill=new Path2D(x),r=b(i.fillTo(e,n,i.min,i.max,R));v(t,P,r),v(t,O,r)}if(!i.spanGaps){let c=[];I&&c.push(...sc(o,l,r,a,w,_,t)),k.gaps=c=i.gaps(e,n,r,a,c),k.clip=lc(c,s.ori,h,m,p,f)}return 0!=D&&(k.band=2==D?[oc(e,n,r,a,x,-1),oc(e,n,r,a,x,1)]:oc(e,n,r,a,x,D)),k}))}function Ec(e,t,n,r,a,i){let o=arguments.length>6&&void 0!==arguments[6]?arguments[6]:ll;if(e.length>1){let l=null;for(let s=0,c=1/0;s0!==r[e]>0?n[e]=0:(n[e]=3*(s[e-1]+s[e])/((2*s[e]+s[e-1])/r[e-1]+(s[e]+2*s[e-1])/r[e]),isFinite(n[e])||(n[e]=0));n[o-1]=r[o-2];for(let c=0;c{Bc.pxRatio=yo})));const Tc=Cc(),Lc=wc();function Oc(e,t,n,r){return(r?[e[0],e[1]].concat(e.slice(2)):[e[0]].concat(e.slice(1))).map(((e,r)=>Pc(e,r,t,n)))}function Pc(e,t,n,r){return Il({},0==t?n:r,e)}function Ic(e,t,n){return null==t?El:[t,n]}const Rc=Ic;function Dc(e,t,n){return null==t?El:Uo(t,n,jo,!0)}function zc(e,t,n,r){return null==t?El:zo(t,n,e.scales[r].log,!1)}const Fc=zc;function jc(e,t,n,r){return null==t?El:Fo(t,n,e.scales[r].log,!1)}const $c=jc;function Hc(e,t,n,r,a){let i=tl(sl(e),sl(t)),o=t-e,l=Ro(a/r*o,n);do{let e=n[l],t=r*e/o;if(t>=a&&i+(e<5?wl.get(e):0)<=17)return[e,t]}while(++l(t=Jo((n=+r)*yo))+"px")),t,n]}function Uc(e){e.show&&[e.font,e.labelFont].forEach((e=>{let t=bl(e[2]*yo,1);e[0]=e[0].replace(/[0-9.]+px/,t+"px"),e[1]=t}))}function Bc(e,t,n){const r={mode:Bo(e.mode,1)},a=r.mode;function i(e,t){return((3==t.distr?al(e>0?e:t.clamp(r,e,t.min,t.max,t.key)):4==t.distr?ol(e,t.asinh):e)-t._min)/(t._max-t._min)}function o(e,t,n,r){let a=i(e,t);return r+n*(-1==t.dir?1-a:a)}function l(e,t,n,r){let a=i(e,t);return r+n*(-1==t.dir?a:1-a)}function s(e,t,n,r){return 0==t.ori?o(e,t,n,r):l(e,t,n,r)}r.valToPosH=o,r.valToPosV=l;let c=!1;r.status=0;const u=r.root=So("uplot");if(null!=e.id&&(u.id=e.id),bo(u,e.class),e.title){So("u-title",u).textContent=e.title}const d=xo("canvas"),h=r.ctx=d.getContext("2d"),m=So("u-wrap",u);Po("click",m,(e=>{if(e.target===f){(Mt!=Ct||Tt!=Et)&&jt.click(r,e)}}),!0);const p=r.under=So("u-under",m);m.appendChild(d);const f=r.over=So("u-over",m),v=+Bo((e=Pl(e)).pxAlign,1),g=cc(v);(e.plugins||[]).forEach((t=>{t.opts&&(e=t.opts(r,e)||e)}));const y=e.ms||.001,_=r.series=1==a?Oc(e.series||[],Ps,Zs,!1):(b=e.series||[null],w=Qs,b.map(((e,t)=>0==t?null:Il({},w,e))));var b,w;const k=r.axes=Oc(e.axes||[],Os,Ys,!0),x=r.scales={},S=r.bands=e.bands||[];S.forEach((e=>{e.fill=ul(e.fill||null),e.dir=Bo(e.dir,-1)}));const C=2==a?_[1].facets[0].scale:_[0].scale,E={axes:function(){for(let e=0;ett[e])):y,b=2==m.distr?tt[y[1]]-tt[y[0]]:u,w=t.ticks,S=t.border,C=w.show?Jo(w.size*yo):0,E=t._rotate*-Qo/180,N=g(t._pos*yo),A=N+(C+v)*c;a=0==o?A:0,n=1==o?A:0,ot(t.font[0],l,1==t.align?eo:2==t.align?to:E>0?eo:E<0?to:0==o?"center":3==i?to:eo,E||1==o?"middle":2==i?Ji:Xi);let M=t.font[1]*t.lineGap,T=y.map((e=>g(s(e,m,p,f)))),L=t._values;for(let e=0;e0&&(_.forEach(((e,n)=>{if(n>0&&e.show&&(ct(n,!1),ct(n,!0),null==e._paths)){et!=e.alpha&&(h.globalAlpha=et=e.alpha);let i=2==a?[0,t[n][0].length-1]:function(e){let t=cl(Ve-1,0,He-1),n=cl(Ue+1,0,He-1);for(;null==e[t]&&t>0;)t--;for(;null==e[n]&&n{if(t>0&&e.show){et!=e.alpha&&(h.globalAlpha=et=e.alpha),null!=e._paths&&ut(t,!1);{let n=null!=e._paths?e._paths.gaps:null,a=e.points.show(r,t,Ve,Ue,n),i=e.points.filter(r,t,a,n);(a||i)&&(e.points._paths=e.points.paths(r,t,Ve,Ue,i),ut(t,!0))}1!=et&&(h.globalAlpha=et=1),kn("drawSeries",t)}})))}},N=(e.drawOrder||["axes","series"]).map((e=>E[e]));function A(t){let n=x[t];if(null==n){let r=(e.scales||Sl)[t]||Sl;if(null!=r.from)A(r.from),x[t]=Il({},x[r.from],r,{key:t});else{n=x[t]=Il({},t==C?Js:Xs,r),n.key=t;let e=n.time,i=n.range,o=Nl(i);if((t!=C||2==a&&!e)&&(!o||null!=i[0]&&null!=i[1]||(i={min:null==i[0]?$o:{mode:1,hard:i[0],soft:i[0]},max:null==i[1]?$o:{mode:1,hard:i[1],soft:i[1]}},o=!1),!o&&Tl(i))){let e=i;i=(t,n,r)=>null==n?El:Uo(n,r,e)}n.range=ul(i||(e?Rc:t==C?3==n.distr?Fc:4==n.distr?$c:Ic:3==n.distr?zc:4==n.distr?jc:Dc)),n.auto=ul(!o&&n.auto),n.clamp=ul(n.clamp||Gs),n._min=n._max=null}}}A("x"),A("y"),1==a&&_.forEach((e=>{A(e.scale)})),k.forEach((e=>{A(e.scale)}));for(let Mn in e.scales)A(Mn);const M=x[C],T=M.distr;let L,O;0==M.ori?(bo(u,"u-hz"),L=o,O=l):(bo(u,"u-vt"),L=l,O=o);const P={};for(let Mn in x){let e=x[Mn];null==e.min&&null==e.max||(P[Mn]={min:e.min,max:e.max},e.min=e.max=null)}const I=e.tzDate||(e=>new Date(Jo(e/y))),R=e.fmtDate||ql,D=1==y?hs(I):fs(I),z=gs(I,vs(1==y?ds:ps,R)),F=bs(I,_s("{YYYY}-{MM}-{DD} {h}:{mm}{aa}",R)),j=[],$=r.legend=Il({},ws,e.legend),H=$.show,V=$.markers;let U,B,q;$.idxs=j,V.width=ul(V.width),V.dash=ul(V.dash),V.stroke=ul(V.stroke),V.fill=ul(V.fill);let Y,W=[],K=[],Q=!1,Z={};if($.live){const e=_[1]?_[1].values:null;Q=null!=e,Y=Q?e(r,1,0):{_:0};for(let t in Y)Z[t]=mo}if(H)if(U=xo("table","u-legend",u),q=xo("tbody",null,U),$.mount(r,U),Q){B=xo("thead",null,U,q);let e=xo("tr",null,B);for(var G in xo("th",null,e),Y)xo("th",Qi,e).textContent=G}else bo(U,"u-inline"),$.live&&bo(U,"u-live");const J={show:!0},X={show:!1};const ee=new Map;function te(e,t,n){let a=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];const i=ee.get(t)||{},o=Ee.bind[e](r,t,n,a);o&&(Po(e,t,i[e]=o),ee.set(t,i))}function ne(e,t,n){const r=ee.get(t)||{};for(let a in r)null!=e&&a!=e||(Io(a,t,r[a]),delete r[a]);null==e&&ee.delete(t)}let re=0,ae=0,ie=0,oe=0,le=0,se=0,ce=le,ue=se,de=ie,he=oe,me=0,pe=0,fe=0,ve=0;r.bbox={};let ge=!1,ye=!1,_e=!1,be=!1,we=!1,ke=!1;function xe(e,t,n){(n||e!=r.width||t!=r.height)&&Se(e,t),yt(!1),_e=!0,ye=!0,Rt()}function Se(e,t){r.width=re=ie=e,r.height=ae=oe=t,le=se=0,function(){let e=!1,t=!1,n=!1,r=!1;k.forEach(((a,i)=>{if(a.show&&a._show){let{side:i,_size:o}=a,l=i%2,s=o+(null!=a.label?a.labelSize:0);s>0&&(l?(ie-=s,3==i?(le+=s,r=!0):n=!0):(oe-=s,0==i?(se+=s,e=!0):t=!0))}})),ze[0]=e,ze[1]=n,ze[2]=t,ze[3]=r,ie-=$e[1]+$e[3],le+=$e[3],oe-=$e[2]+$e[0],se+=$e[0]}(),function(){let e=le+ie,t=se+oe,n=le,r=se;function a(a,i){switch(a){case 1:return e+=i,e-i;case 2:return t+=i,t-i;case 3:return n-=i,n+i;case 0:return r-=i,r+i}}k.forEach(((e,t)=>{if(e.show&&e._show){let t=e.side;e._pos=a(t,e._size),null!=e.label&&(e._lpos=a(t,e.labelSize))}}))}();let n=r.bbox;me=n.left=gl(le*yo,.5),pe=n.top=gl(se*yo,.5),fe=n.width=gl(ie*yo,.5),ve=n.height=gl(oe*yo,.5)}const Ce=3;r.setSize=function(e){let{width:t,height:n}=e;xe(t,n)};const Ee=r.cursor=Il({},Cs,{drag:{y:2==a}},e.cursor);if(null==Ee.dataIdx){var Ne,Ae;let e=Ee.hover,n=e.skip=new Set(null!==(Ne=e.skip)&&void 0!==Ne?Ne:[]);n.add(void 0);let r=e.prox=ul(e.prox),a=null!==(Ae=e.bias)&&void 0!==Ae?Ae:e.bias=0;Ee.dataIdx=(e,i,o,l)=>{var s;if(0==i)return o;let c=o,u=null!==(s=r(e,i,o,l))&&void 0!==s?s:ll,d=u>=0&&u0;)n.has(f[e])||(t=e);if(0==a||1==a)for(e=o;null==r&&e++u&&(c=null)}return c}}const Me=e=>{Ee.event=e};Ee.idxs=j,Ee._lock=!1;let Te=Ee.points;Te.show=ul(Te.show),Te.size=ul(Te.size),Te.stroke=ul(Te.stroke),Te.width=ul(Te.width),Te.fill=ul(Te.fill);const Le=r.focus=Il({},e.focus||{alpha:.3},Ee.focus),Oe=Le.prox>=0;let Pe=[null],Ie=[null],Re=[null];function De(e,t){if(1==a||t>0){let t=1==a&&x[e.scale].time,n=e.value;e.value=t?Ml(n)?bs(I,_s(n,R)):n||F:n||qs,e.label=e.label||(t?"Time":"Value")}if(t>0){e.width=null==e.width?1:e.width,e.paths=e.paths||Tc||ml,e.fillTo=ul(e.fillTo||ic),e.pxAlign=+Bo(e.pxAlign,v),e.pxRound=cc(e.pxAlign),e.stroke=ul(e.stroke||null),e.fill=ul(e.fill||null),e._stroke=e._fill=e._paths=e._focus=null;let t=bl((3+2*(tl(1,e.width)||1))*1,3),n=e.points=Il({},{size:t,width:tl(1,.2*t),stroke:e.stroke,space:2*t,paths:Lc,_stroke:null,_fill:null},e.points);n.show=ul(n.show),n.filter=ul(n.filter),n.fill=ul(n.fill),n.stroke=ul(n.stroke),n.paths=ul(n.paths),n.pxAlign=e.pxAlign}if(H){let n=function(e,t){if(0==t&&(Q||!$.live||2==a))return El;let n=[],i=xo("tr","u-series",q,q.childNodes[t]);bo(i,e.class),e.show||bo(i,Ki);let o=xo("th",null,i);if(V.show){let e=So("u-marker",o);if(t>0){let n=V.width(r,t);n&&(e.style.border=n+"px "+V.dash(r,t)+" "+V.stroke(r,t)),e.style.background=V.fill(r,t)}}let l=So(Qi,o);for(var s in l.textContent=e.label,t>0&&(V.show||(l.style.color=e.width>0?V.stroke(r,t):V.fill(r,t)),te("click",o,(t=>{if(Ee._lock)return;Me(t);let n=_.indexOf(e);if((t.ctrlKey||t.metaKey)!=$.isolate){let e=_.some(((e,t)=>t>0&&t!=n&&e.show));_.forEach(((t,r)=>{r>0&&Yt(r,e?r==n?J:X:J,!0,Sn.setSeries)}))}else Yt(n,{show:!e.show},!0,Sn.setSeries)}),!1),Oe&&te(lo,o,(t=>{Ee._lock||(Me(t),Yt(_.indexOf(e),Zt,!0,Sn.setSeries))}),!1)),Y){let e=xo("td","u-value",i);e.textContent="--",n.push(e)}return[i,n]}(e,t);W.splice(t,0,n[0]),K.splice(t,0,n[1]),$.values.push(null)}if(Ee.show){j.splice(t,0,null);let n=function(e,t){if(t>0){let n=Ee.points.show(r,t);if(n)return bo(n,"u-cursor-pt"),bo(n,e.class),Eo(n,-10,-10,ie,oe),f.insertBefore(n,Pe[t]),n}}(e,t);null!=n&&(Pe.splice(t,0,n),Ie.splice(t,0,0),Re.splice(t,0,0))}kn("addSeries",t)}r.addSeries=function(e,t){t=null==t?_.length:t,e=1==a?Pc(e,t,Ps,Zs):Pc(e,t,null,Qs),_.splice(t,0,e),De(_[t],t)},r.delSeries=function(e){if(_.splice(e,1),H){$.values.splice(e,1),K.splice(e,1);let t=W.splice(e,1)[0];ne(null,t.firstChild),t.remove()}Ee.show&&(j.splice(e,1),Pe.length>1&&(Pe.splice(e,1)[0].remove(),Ie.splice(e,1),Re.splice(e,1))),kn("delSeries",e)};const ze=[!1,!1,!1,!1];function Fe(e,t,n,r){let[a,i,o,l]=n,s=t%2,c=0;return 0==s&&(l||i)&&(c=0==t&&!a||2==t&&!o?Jo(Os.size/3):0),1==s&&(a||o)&&(c=1==t&&!i||3==t&&!l?Jo(Ys.size/2):0),c}const je=r.padding=(e.padding||[Fe,Fe,Fe,Fe]).map((e=>ul(Bo(e,Fe)))),$e=r._padding=je.map(((e,t)=>e(r,t,ze,0)));let He,Ve=null,Ue=null;const Be=1==a?_[0].idxs:null;let qe,Ye,We,Ke,Qe,Ze,Ge,Je,Xe,et,tt=null,nt=!1;function rt(e,n){if(t=null==e?[]:e,r.data=r._data=t,2==a){He=0;for(let e=1;e<_.length;e++)He+=t[e][0].length}else{0==t.length&&(r.data=r._data=t=[[]]),tt=t[0],He=tt.length;let e=t;if(2==T){e=t.slice();let n=e[0]=Array(He);for(let e=0;e=0,ke=!0,Rt()}}function at(){let e,n;nt=!0,1==a&&(He>0?(Ve=Be[0]=0,Ue=Be[1]=He-1,e=t[0][Ve],n=t[0][Ue],2==T?(e=Ve,n=Ue):e==n&&(3==T?[e,n]=zo(e,e,M.log,!1):4==T?[e,n]=Fo(e,e,M.log,!1):M.time?n=e+Jo(86400/y):[e,n]=Uo(e,n,jo,!0))):(Ve=Be[0]=e=null,Ue=Be[1]=n=null)),qt(C,e,n)}function it(e,t,n,r,a,i){var o,l,s,c,u;null!==(o=e)&&void 0!==o||(e=ro),null!==(l=n)&&void 0!==l||(n=Cl),null!==(s=r)&&void 0!==s||(r="butt"),null!==(c=a)&&void 0!==c||(a=ro),null!==(u=i)&&void 0!==u||(i="round"),e!=qe&&(h.strokeStyle=qe=e),a!=Ye&&(h.fillStyle=Ye=a),t!=We&&(h.lineWidth=We=t),i!=Qe&&(h.lineJoin=Qe=i),r!=Ze&&(h.lineCap=Ze=r),n!=Ke&&h.setLineDash(Ke=n)}function ot(e,t,n,r){t!=Ye&&(h.fillStyle=Ye=t),e!=Ge&&(h.font=Ge=e),n!=Je&&(h.textAlign=Je=n),r!=Xe&&(h.textBaseline=Xe=r)}function lt(e,t,n,a){let i=arguments.length>4&&void 0!==arguments[4]?arguments[4]:0;if(a.length>0&&e.auto(r,nt)&&(null==t||null==t.min)){let t=Bo(Ve,0),r=Bo(Ue,a.length-1),o=null==n.min?3==e.distr?function(e,t,n){let r=ll,a=-ll;for(let i=t;i<=n;i++){let t=e[i];null!=t&&t>0&&(ta&&(a=t))}return[r,a]}(a,t,r):function(e,t,n,r){let a=ll,i=-ll;if(1==r)a=e[t],i=e[n];else if(-1==r)a=e[n],i=e[t];else for(let o=t;o<=n;o++){let t=e[o];null!=t&&(ti&&(i=t))}return[a,i]}(a,t,r,i):[n.min,n.max];e.min=el(e.min,n.min=o[0]),e.max=tl(e.max,n.max=o[1])}}r.setData=rt;const st={min:null,max:null};function ct(e,t){let n=t?_[e].points:_[e];n._stroke=n.stroke(r,e),n._fill=n.fill(r,e)}function ut(e,n){let a=n?_[e].points:_[e],{stroke:i,fill:o,clip:l,flags:s,_stroke:c=a._stroke,_fill:u=a._fill,_width:d=a.width}=a._paths;d=bl(d*yo,3);let m=null,p=d%2/2;n&&null==u&&(u=d>0?"#fff":c);let f=1==a.pxAlign&&p>0;if(f&&h.translate(p,p),!n){let e=me-d/2,t=pe-d/2,n=fe+d,r=ve+d;m=new Path2D,m.rect(e,t,n,r)}n?ht(c,d,a.dash,a.cap,u,i,o,s,l):function(e,n,a,i,o,l,s,c,u,d,h){let m=!1;0!=u&&S.forEach(((p,f)=>{if(p.series[0]==e){let e,v=_[p.series[1]],g=t[p.series[1]],y=(v._paths||Sl).band;Nl(y)&&(y=1==p.dir?y[0]:y[1]);let b=null;v.show&&y&&function(e,t,n){for(t=Bo(t,0),n=Bo(n,e.length-1);t<=n;){if(null!=e[t])return!0;t++}return!1}(g,Ve,Ue)?(b=p.fill(r,f)||l,e=v._paths.clip):y=null,ht(n,a,i,o,b,s,c,u,d,h,e,y),m=!0}})),m||ht(n,a,i,o,l,s,c,u,d,h)}(e,c,d,a.dash,a.cap,u,i,o,s,m,l),f&&h.translate(-p,-p)}const dt=2|nc;function ht(e,t,n,r,a,i,o,l,s,c,u,d){it(e,t,n,r,a),(s||c||d)&&(h.save(),s&&h.clip(s),c&&h.clip(c)),d?(l&dt)==dt?(h.clip(d),u&&h.clip(u),pt(a,o),mt(e,i,t)):2&l?(pt(a,o),h.clip(d),mt(e,i,t)):l&nc&&(h.save(),h.clip(d),u&&h.clip(u),pt(a,o),h.restore(),mt(e,i,t)):(pt(a,o),mt(e,i,t)),(s||c||d)&&h.restore()}function mt(e,t,n){n>0&&(t instanceof Map?t.forEach(((e,t)=>{h.strokeStyle=qe=t,h.stroke(e)})):null!=t&&e&&h.stroke(t))}function pt(e,t){t instanceof Map?t.forEach(((e,t)=>{h.fillStyle=Ye=t,h.fill(e)})):null!=t&&e&&h.fill(t)}function ft(e,t,n,r,a,i,o,l,s,c){let u=o%2/2;1==v&&h.translate(u,u),it(l,o,s,c,l),h.beginPath();let d,m,p,f,g=a+(0==r||3==r?-i:i);0==n?(m=a,f=g):(d=a,p=g);for(let v=0;v{if(!n.show)return;let i=x[n.scale];if(null==i.min)return void(n._show&&(t=!1,n._show=!1,yt(!1)));n._show||(t=!1,n._show=!0,yt(!1));let o=n.side,l=o%2,{min:s,max:c}=i,[u,d]=function(e,t,n,a){let i,o=k[e];if(a<=0)i=[0,0];else{let l=o._space=o.space(r,e,t,n,a);i=Hc(t,n,o._incrs=o.incrs(r,e,t,n,a,l),a,l)}return o._found=i}(a,s,c,0==l?ie:oe);if(0==d)return;let h=2==i.distr,m=n._splits=n.splits(r,a,s,c,u,d,h),p=2==i.distr?m.map((e=>tt[e])):m,f=2==i.distr?tt[m[1]]-tt[m[0]]:u,v=n._values=n.values(r,n.filter(r,p,a,d,f),a,d,f);n._rotate=2==o?n.rotate(r,v,a,d):0;let g=n._size;n._size=Xo(n.size(r,v,a,e)),null!=g&&n._size!=g&&(t=!1)})),t}function gt(e){let t=!0;return je.forEach(((n,a)=>{let i=n(r,a,ze,e);i!=$e[a]&&(t=!1),$e[a]=i})),t}function yt(e){_.forEach(((t,n)=>{n>0&&(t._paths=null,e&&(1==a?(t.min=null,t.max=null):t.facets.forEach((e=>{e.min=null,e.max=null}))))}))}let _t,bt,wt,kt,xt,St,Ct,Et,Nt,At,Mt,Tt,Lt=!1,Ot=!1,Pt=[];function It(){Ot=!1;for(let e=0;e0){_.forEach(((n,i)=>{if(1==a){let a=n.scale,o=P[a];if(null==o)return;let l=e[a];if(0==i){let e=l.range(r,l.min,l.max,a);l.min=e[0],l.max=e[1],Ve=Ro(l.min,t[0]),Ue=Ro(l.max,t[0]),Ue-Ve>1&&(t[0][Ve]l.max&&Ue--),n.min=tt[Ve],n.max=tt[Ue]}else n.show&&n.auto&<(l,o,n,t[i],n.sorted);n.idxs[0]=Ve,n.idxs[1]=Ue}else if(i>0&&n.show&&n.auto){let[r,a]=n.facets,o=r.scale,l=a.scale,[s,c]=t[i],u=e[o],d=e[l];null!=u&<(u,P[o],r,s,r.sorted),null!=d&<(d,P[l],a,c,a.sorted),n.min=a.min,n.max=a.max}}));for(let t in e){let n=e[t],a=P[t];if(null==n.from&&(null==a||null==a.min)){let e=n.range(r,n.min==ll?null:n.min,n.max==-ll?null:n.max,t);n.min=e[0],n.max=e[1]}}}for(let t in e){let n=e[t];if(null!=n.from){let a=e[n.from];if(null==a.min)n.min=n.max=null;else{let e=n.range(r,a.min,a.max,t);n.min=e[0],n.max=e[1]}}}let n={},i=!1;for(let t in e){let r=e[t],a=x[t];if(a.min!=r.min||a.max!=r.max){a.min=r.min,a.max=r.max;let e=a.distr;a._min=3==e?al(a.min):4==e?ol(a.min,a.asinh):a.min,a._max=3==e?al(a.max):4==e?ol(a.max,a.asinh):a.max,n[t]=i=!0}}if(i){_.forEach(((e,t)=>{2==a?t>0&&n.y&&(e._paths=null):n[e.scale]&&(e._paths=null)}));for(let e in n)_e=!0,kn("setScale",e);Ee.show&&Ee.left>=0&&(be=ke=!0)}for(let t in P)P[t]=null}(),ge=!1),_e&&(!function(){let e=!1,t=0;for(;!e;){t++;let n=vt(t),a=gt(t);e=t==Ce||n&&a,e||(Se(r.width,r.height),ye=!0)}}(),_e=!1),ye){if(ko(p,eo,le),ko(p,Ji,se),ko(p,Zi,ie),ko(p,Gi,oe),ko(f,eo,le),ko(f,Ji,se),ko(f,Zi,ie),ko(f,Gi,oe),ko(m,Zi,re),ko(m,Gi,ae),d.width=Jo(re*yo),d.height=Jo(ae*yo),k.forEach((e=>{let{_el:t,_show:n,_size:r,_pos:a,side:i}=e;if(null!=t)if(n){let e=i%2==1;ko(t,e?"left":"top",a-(3===i||0===i?r:0)),ko(t,e?"width":"height",r),ko(t,e?"top":"left",e?se:le),ko(t,e?"height":"width",e?oe:ie),wo(t,Ki)}else bo(t,Ki)})),qe=Ye=We=Qe=Ze=Ge=Je=Xe=Ke=null,et=1,ln(!0),le!=ce||se!=ue||ie!=de||oe!=he){yt(!1);let e=ie/de,t=oe/he;if(Ee.show&&!be&&Ee.left>=0){Ee.left*=e,Ee.top*=t,wt&&Eo(wt,Jo(Ee.left),0,ie,oe),kt&&Eo(kt,0,Jo(Ee.top),ie,oe);for(let n=1;n=0&&Vt.width>0){Vt.left*=e,Vt.width*=e,Vt.top*=t,Vt.height*=t;for(let e in un)ko(Ut,e,Vt[e])}ce=le,ue=se,de=ie,he=oe}kn("setSize"),ye=!1}re>0&&ae>0&&(h.clearRect(0,0,d.width,d.height),kn("drawClear"),N.forEach((e=>e())),kn("draw")),Vt.show&&we&&(Bt(Vt),we=!1),Ee.show&&be&&(an(null,!0,!1),be=!1),$.show&&$.live&&ke&&(nn(),ke=!1),c||(c=!0,r.status=1,kn("ready")),nt=!1,Lt=!1}function zt(e,n){let a=x[e];if(null==a.from){if(0==He){let t=a.range(r,n.min,n.max,e);n.min=t[0],n.max=t[1]}if(n.min>n.max){let e=n.min;n.min=n.max,n.max=e}if(He>1&&null!=n.min&&null!=n.max&&n.max-n.min<1e-16)return;e==C&&2==a.distr&&He>0&&(n.min=Ro(n.min,t[0]),n.max=Ro(n.max,t[0]),n.min==n.max&&n.max++),P[e]=n,ge=!0,Rt()}}r.batch=function(e){let t=arguments.length>1&&void 0!==arguments[1]&&arguments[1];Lt=!0,Ot=t,e(r),Dt(),t&&Pt.length>0&&queueMicrotask(It)},r.redraw=(e,t)=>{_e=t||!1,!1!==e?qt(C,M.min,M.max):Rt()},r.setScale=zt;let Ft=!1;const jt=Ee.drag;let $t=jt.x,Ht=jt.y;Ee.show&&(Ee.x&&(_t=So("u-cursor-x",f)),Ee.y&&(bt=So("u-cursor-y",f)),0==M.ori?(wt=_t,kt=bt):(wt=bt,kt=_t),Mt=Ee.left,Tt=Ee.top);const Vt=r.select=Il({show:!0,over:!0,left:0,width:0,top:0,height:0},e.select),Ut=Vt.show?So("u-select",Vt.over?f:p):null;function Bt(e,t){if(Vt.show){for(let t in e)Vt[t]=e[t],t in un&&ko(Ut,t,e[t]);!1!==t&&kn("setSelect")}}function qt(e,t,n){zt(e,{min:t,max:n})}function Yt(e,t,n,i){null!=t.focus&&function(e){if(e!=Qt){let t=null==e,n=1!=Le.alpha;_.forEach(((r,i)=>{if(1==a||i>0){let a=t||0==i||i==e;r._focus=t?null:a,n&&function(e,t){_[e].alpha=t,Ee.show&&Pe[e]&&(Pe[e].style.opacity=t);H&&W[e]&&(W[e].style.opacity=t)}(i,a?1:Le.alpha)}})),Qt=e,n&&Rt()}}(e),null!=t.show&&_.forEach(((n,r)=>{r>0&&(e==r||null==e)&&(n.show=t.show,function(e,t){let n=_[e],r=H?W[e]:null;n.show?r&&wo(r,Ki):(r&&bo(r,Ki),Pe.length>1&&Eo(Pe[e],-10,-10,ie,oe))}(r,t.show),2==a?(qt(n.facets[0].scale,null,null),qt(n.facets[1].scale,null,null)):qt(n.scale,null,null),Rt())})),!1!==n&&kn("setSeries",e,t),i&&Nn("setSeries",r,e,t)}let Wt,Kt,Qt;r.setSelect=Bt,r.setSeries=Yt,r.addBand=function(e,t){e.fill=ul(e.fill||null),e.dir=Bo(e.dir,-1),t=null==t?S.length:t,S.splice(t,0,e)},r.setBand=function(e,t){Il(S[e],t)},r.delBand=function(e){null==e?S.length=0:S.splice(e,1)};const Zt={focus:!0};function Gt(e,t,n){let r=x[t];n&&(e=e/yo-(1==r.ori?se:le));let a=ie;1==r.ori&&(a=oe,e=a-e),-1==r.dir&&(e=a-e);let i=r._min,o=i+(r._max-i)*(e/a),l=r.distr;return 3==l?nl(10,o):4==l?function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return Ko.sinh(e)*t}(o,r.asinh):o}function Jt(e,t){ko(Ut,eo,Vt.left=e),ko(Ut,Zi,Vt.width=t)}function Xt(e,t){ko(Ut,Ji,Vt.top=e),ko(Ut,Gi,Vt.height=t)}H&&Oe&&te(so,U,(e=>{Ee._lock||(Me(e),null!=Qt&&Yt(null,Zt,!0,Sn.setSeries))})),r.valToIdx=e=>Ro(e,t[0]),r.posToIdx=function(e,n){return Ro(Gt(e,C,n),t[0],Ve,Ue)},r.posToVal=Gt,r.valToPos=(e,t,n)=>0==x[t].ori?o(e,x[t],n?fe:ie,n?me:0):l(e,x[t],n?ve:oe,n?pe:0),r.setCursor=(e,t,n)=>{Mt=e.left,Tt=e.top,an(null,t,n)};let en=0==M.ori?Jt:Xt,tn=1==M.ori?Jt:Xt;function nn(e,t){null!=e&&(e.idxs?e.idxs.forEach(((e,t)=>{j[t]=e})):void 0!==e.idx&&j.fill(e.idx),$.idx=j[0]);for(let n=0;n<_.length;n++)(n>0||1==a&&!Q)&&rn(n,j[n]);H&&$.live&&function(){if(H&&$.live)for(let e=2==a?1:0;e<_.length;e++){if(0==e&&Q)continue;let t=$.values[e],n=0;for(let r in t)K[e][n++].firstChild.nodeValue=t[r]}}(),ke=!1,!1!==t&&kn("setLegend")}function rn(e,n){var a;let i,o=_[e],l=0==e&&2==T?tt:t[e];Q?i=null!==(a=o.values(r,e,n))&&void 0!==a?a:Z:(i=o.value(r,null==n?null:l[n],e,n),i=null==i?Z:{_:i}),$.values[e]=i}function an(e,n,i){let o;Nt=Mt,At=Tt,[Mt,Tt]=Ee.move(r,Mt,Tt),Ee.left=Mt,Ee.top=Tt,Ee.show&&(wt&&Eo(wt,Jo(Mt),0,ie,oe),kt&&Eo(kt,0,Jo(Tt),ie,oe));let l=Ve>Ue;Wt=ll;let s=0==M.ori?ie:oe,c=1==M.ori?ie:oe;if(Mt<0||0==He||l){o=Ee.idx=null;for(let e=0;e<_.length;e++)e>0&&Pe.length>1&&Eo(Pe[e],-10,-10,ie,oe);Oe&&Yt(null,Zt,!0,null==e&&Sn.setSeries),$.live&&(j.fill(o),ke=!0)}else{let e,n,i;1==a&&(e=0==M.ori?Mt:Tt,n=Gt(e,C),o=Ee.idx=Ro(n,t[0],Ve,Ue),i=L(t[0][o],M,s,0));for(let l=2==a?1:0;l<_.length;l++){let e=_[l],u=j[l],d=null==u?null:1==a?t[l][u]:t[l][1][u],h=Ee.dataIdx(r,l,o,n),m=null==h?null:1==a?t[l][h]:t[l][1][h];ke=ke||m!=d||h!=u,j[l]=h;let p=h==o?i:L(1==a?t[0][h]:t[l][0][h],M,s,0);if(l>0&&e.show){let t,n,i=null==m?-10:O(m,1==a?x[e.scale]:x[e.facets[1].scale],c,0);if(Oe&&null!=m){let t=1==M.ori?Mt:Tt,n=Zo(Le.dist(r,l,h,i,t));if(n=0?1:-1;i==(m>=0?1:-1)&&(1==i?1==r?m>=a:m<=a:1==r?m<=a:m>=a)&&(Wt=n,Kt=l)}else Wt=n,Kt=l}}if(0==M.ori?(t=p,n=i):(t=i,n=p),ke&&Pe.length>1){Ao(Pe[l],Ee.points.fill(r,l),Ee.points.stroke(r,l));let e,a,i,o,s=!0,c=Ee.points.bbox;if(null!=c){s=!1;let t=c(r,l);i=t.left,o=t.top,e=t.width,a=t.height}else i=t,o=n,e=a=Ee.points.size(r,l);To(Pe[l],e,a,s),Ie[l]=i,Re[l]=o,Eo(Pe[l],yl(i,1),yl(o,1),ie,oe)}}}}if(Vt.show&&Ft)if(null!=e){let[t,n]=Sn.scales,[r,a]=Sn.match,[i,o]=e.cursor.sync.scales,l=e.cursor.drag;if($t=l._x,Ht=l._y,$t||Ht){let l,u,d,h,m,{left:p,top:f,width:v,height:g}=e.select,y=e.scales[t].ori,_=e.posToVal,b=null!=t&&r(t,i),w=null!=n&&a(n,o);b&&$t?(0==y?(l=p,u=v):(l=f,u=g),d=x[t],h=L(_(l,i),d,s,0),m=L(_(l+u,i),d,s,0),en(el(h,m),Zo(m-h))):en(0,s),w&&Ht?(1==y?(l=p,u=v):(l=f,u=g),d=x[n],h=O(_(l,o),d,c,0),m=O(_(l+u,o),d,c,0),tn(el(h,m),Zo(m-h))):tn(0,c)}else dn()}else{let e=Zo(Nt-xt),t=Zo(At-St);if(1==M.ori){let n=e;e=t,t=n}$t=jt.x&&e>=jt.dist,Ht=jt.y&&t>=jt.dist;let n,r,a=jt.uni;null!=a?$t&&Ht&&($t=e>=a,Ht=t>=a,$t||Ht||(t>e?Ht=!0:$t=!0)):jt.x&&jt.y&&($t||Ht)&&($t=Ht=!0),$t&&(0==M.ori?(n=Ct,r=Mt):(n=Et,r=Tt),en(el(n,r),Zo(r-n)),Ht||tn(0,c)),Ht&&(1==M.ori?(n=Ct,r=Mt):(n=Et,r=Tt),tn(el(n,r),Zo(r-n)),$t||en(0,s)),$t||Ht||(en(0,0),tn(0,0))}if(jt._x=$t,jt._y=Ht,null==e){if(i){if(null!=Cn){let[e,t]=Sn.scales;Sn.values[0]=null!=e?Gt(0==M.ori?Mt:Tt,e):null,Sn.values[1]=null!=t?Gt(1==M.ori?Mt:Tt,t):null}Nn(ao,r,Mt,Tt,ie,oe,o)}if(Oe){let e=i&&Sn.setSeries,t=Le.prox;null==Qt?Wt<=t&&Yt(Kt,Zt,!0,e):Wt>t?Yt(null,Zt,!0,e):Kt!=Qt&&Yt(Kt,Zt,!0,e)}}ke&&($.idx=o,nn()),!1!==n&&kn("setCursor")}r.setLegend=nn;let on=null;function ln(){arguments.length>0&&void 0!==arguments[0]&&arguments[0]?on=null:(on=f.getBoundingClientRect(),kn("syncRect",on))}function sn(e,t,n,r,a,i,o){Ee._lock||Ft&&null!=e&&0==e.movementX&&0==e.movementY||(cn(e,t,n,r,a,i,o,!1,null!=e),null!=e?an(null,!0,!0):an(t,!0,!1))}function cn(e,t,n,a,i,o,l,c,u){if(null==on&&ln(!1),Me(e),null!=e)n=e.clientX-on.left,a=e.clientY-on.top;else{if(n<0||a<0)return Mt=-10,void(Tt=-10);let[e,r]=Sn.scales,l=t.cursor.sync,[c,u]=l.values,[d,h]=l.scales,[m,p]=Sn.match,f=t.axes[0].side%2==1,v=0==M.ori?ie:oe,g=1==M.ori?ie:oe,y=f?o:i,_=f?i:o,b=f?a:n,w=f?n:a;if(n=null!=d?m(e,d)?s(c,x[e],v,0):-10:v*(b/y),a=null!=h?p(r,h)?s(u,x[r],g,0):-10:g*(w/_),1==M.ori){let e=n;n=a,a=e}}u&&((n<=1||n>=ie-1)&&(n=gl(n,ie)),(a<=1||a>=oe-1)&&(a=gl(a,oe))),c?(xt=n,St=a,[Ct,Et]=Ee.move(r,n,a)):(Mt=n,Tt=a)}Object.defineProperty(r,"rect",{get:()=>(null==on&&ln(!1),on)});const un={width:0,height:0,left:0,top:0};function dn(){Bt(un,!1)}let hn,mn,pn,fn;function vn(e,t,n,a,i,o,l){Ft=!0,$t=Ht=jt._x=jt._y=!1,cn(e,t,n,a,i,o,0,!0,!1),null!=e&&(te(oo,fo,gn,!1),Nn(io,r,Ct,Et,ie,oe,null));let{left:s,top:c,width:u,height:d}=Vt;hn=s,mn=c,pn=u,fn=d,dn()}function gn(e,t,n,a,i,o,l){Ft=jt._x=jt._y=!1,cn(e,t,n,a,i,o,0,!1,!0);let{left:s,top:c,width:u,height:d}=Vt,h=u>0||d>0,m=hn!=s||mn!=c||pn!=u||fn!=d;if(h&&m&&Bt(Vt),jt.setScale&&h&&m){let e=s,t=u,n=c,r=d;if(1==M.ori&&(e=c,t=d,n=s,r=u),$t&&qt(C,Gt(e,C),Gt(e+t,C)),Ht)for(let a in x){let e=x[a];a!=C&&null==e.from&&e.min!=ll&&qt(a,Gt(n+r,a),Gt(n,a))}dn()}else Ee.lock&&(Ee._lock=!Ee._lock,Ee._lock||an(null,!0,!1));null!=e&&(ne(oo,fo),Nn(oo,r,Mt,Tt,ie,oe,null))}function yn(e,t,n,a,i,o,l){Ee._lock||(Me(e),at(),dn(),null!=e&&Nn(co,r,Mt,Tt,ie,oe,null))}function _n(){k.forEach(Uc),xe(r.width,r.height,!0)}Po(ho,vo,_n);const bn={};bn.mousedown=vn,bn.mousemove=sn,bn.mouseup=gn,bn.dblclick=yn,bn.setSeries=(e,t,n,a)=>{-1!=(n=(0,Sn.match[2])(r,t,n))&&Yt(n,a,!0,!1)},Ee.show&&(te(io,f,vn),te(ao,f,sn),te(lo,f,(e=>{Me(e),ln(!1)})),te(so,f,(function(e,t,n,r,a,i,o){if(Ee._lock)return;Me(e);let l=Ft;if(Ft){let e,t,n=!0,r=!0,a=10;0==M.ori?(e=$t,t=Ht):(e=Ht,t=$t),e&&t&&(n=Mt<=a||Mt>=ie-a,r=Tt<=a||Tt>=oe-a),e&&n&&(Mt=Mt{e.call(null,r,t,n)}))}(e.plugins||[]).forEach((e=>{for(let t in e.hooks)wn[t]=(wn[t]||[]).concat(e.hooks[t])}));const xn=(e,t,n)=>n,Sn=Il({key:null,setSeries:!1,filters:{pub:pl,sub:pl},scales:[C,_[1]?_[1].scale:null],match:[fl,fl,xn],values:[null,null]},Ee.sync);2==Sn.match.length&&Sn.match.push(xn),Ee.sync=Sn;const Cn=Sn.key,En=tc(Cn);function Nn(e,t,n,r,a,i,o){Sn.filters.pub(e,t,n,r,a,i,o)&&En.pub(e,t,n,r,a,i,o)}function An(){kn("init",e,t),rt(t||e.data,!1),P[C]?zt(C,P[C]):at(),we=Vt.show&&(Vt.width>0||Vt.height>0),be=ke=!0,xe(e.width,e.height)}return En.sub(r),r.pub=function(e,t,n,r,a,i,o){Sn.filters.sub(e,t,n,r,a,i,o)&&bn[e](null,t,n,r,a,i,o)},r.destroy=function(){var e;En.unsub(r),Ac.delete(r),ee.clear(),Io(ho,vo,_n),u.remove(),null===(e=U)||void 0===e||e.remove(),kn("destroy")},_.forEach(De),k.forEach((function(e,t){if(e._show=e.show,e.show){let n=e.side%2,a=x[e.scale];null==a&&(e.scale=n?_[1].scale:C,a=x[e.scale]);let i=a.time;e.size=ul(e.size),e.space=ul(e.space),e.rotate=ul(e.rotate),Nl(e.incrs)&&e.incrs.forEach((e=>{!wl.has(e)&&wl.set(e,kl(e))})),e.incrs=ul(e.incrs||(2==a.distr?Gl:i?1==y?us:ms:Jl)),e.splits=ul(e.splits||(i&&1==a.distr?D:3==a.distr?Ds:4==a.distr?zs:Rs)),e.stroke=ul(e.stroke),e.grid.stroke=ul(e.grid.stroke),e.ticks.stroke=ul(e.ticks.stroke),e.border.stroke=ul(e.border.stroke);let o=e.values;e.values=Nl(o)&&!Nl(o[0])?ul(o):i?Nl(o)?gs(I,vs(o,R)):Ml(o)?function(e,t){let n=ql(t);return(t,r,a,i,o)=>r.map((t=>n(e(t))))}(I,o):o||z:o||Is,e.filter=ul(e.filter||(a.distr>=3&&10==a.log?Us:3==a.distr&&2==a.log?Bs:hl)),e.font=Vc(e.font),e.labelFont=Vc(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&&(ze[t]=!0,e._el=So("u-axis",m))}})),n?n instanceof HTMLElement?(n.appendChild(u),An()):n(r,An):An(),r}Bc.assign=Il,Bc.fmtNum=Wo,Bc.rangeNum=Uo,Bc.rangeLog=zo,Bc.rangeAsinh=Fo,Bc.orient=rc,Bc.pxRatio=yo,Bc.join=function(e,t){if(function(e){let t=e[0][0],n=t.length;for(let r=1;r1&&void 0!==arguments[1]?arguments[1]:100;const n=e.length;if(n<=1)return!0;let r=0,a=n-1;for(;r<=a&&null==e[r];)r++;for(;a>=r&&null==e[a];)a--;if(a<=r)return!0;const i=tl(1,Go((a-r+1)/t));for(let o=e[r],l=r+i;l<=a;l+=i){const t=e[l];if(null!=t){if(t<=o)return!1;o=t}}return!0}(t[0])||(t=function(e){let t=e[0],n=t.length,r=Array(n);for(let i=0;it[e]-t[n]));let a=[];for(let i=0;ie-t))],a=r[0].length,i=new Map;for(let o=0;orc(e,i,((s,c,u,d,h,m,p,f,v,g,y)=>{let _=s.pxRound,{left:b,width:w}=e.bbox,k=e=>_(m(e,d,g,f)),x=e=>_(p(e,h,y,v)),S=0==d.ori?mc:pc;const C={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:nc},E=C.stroke,N=d.dir*(0==d.ori?1:-1);o=Do(u,o,l,1),l=Do(u,o,l,-1);let A=x(u[1==N?o:l]),M=k(c[1==N?o:l]),T=M,L=M;a&&-1==t&&(L=b,S(E,L,A)),S(E,M,A);for(let e=1==N?o:l;e>=o&&e<=l;e+=N){let n=u[e];if(null==n)continue;let r=k(c[e]),a=x(n);1==t?S(E,r,A):S(E,T,a),S(E,r,a),A=a,T=r}let O=T;a&&1==t&&(O=b+w,S(E,O,A));let[P,I]=ac(e,i);if(null!=s.fill||0!=P){let t=C.fill=new Path2D(E),n=x(s.fillTo(e,i,s.min,s.max,P));S(t,O,n),S(t,L,n)}if(!s.spanGaps){let a=[];a.push(...sc(c,u,o,l,N,k,r));let h=s.width*yo/2,m=n||1==t?h:-h,p=n||-1==t?-h:h;a.forEach((e=>{e[0]+=m,e[1]+=p})),C.gaps=a=s.gaps(e,i,o,l,a),C.clip=lc(a,d.ori,f,v,g,y)}return 0!=I&&(C.band=2==I?[oc(e,i,o,l,E,-1),oc(e,i,o,l,E,1)]:oc(e,i,o,l,E,I)),C}))},e.bars=function(e){const t=Bo((e=e||Sl).size,[.6,ll,1]),n=e.align||0,r=e.gap||0;let a=e.radius;a=null==a?[0,0]:"number"==typeof a?[a,0]:a;const i=ul(a),o=1-t[0],l=Bo(t[1],ll),s=Bo(t[2],1),c=Bo(e.disp,Sl),u=Bo(e.each,(e=>{})),{fill:d,stroke:h}=c;return(e,t,a,m)=>rc(e,t,((p,f,v,g,y,_,b,w,k,x,S)=>{let C,E,N=p.pxRound,A=n,M=r*yo,T=l*yo,L=s*yo;0==g.ori?[C,E]=i(e,t):[E,C]=i(e,t);const O=g.dir*(0==g.ori?1:-1);let P,I,R,D=0==g.ori?fc:vc,z=0==g.ori?u:(e,t,n,r,a,i,o)=>{u(e,t,n,a,r,o,i)},F=Bo(e.bands,Cl).find((e=>e.series[0]==t)),j=null!=F?F.dir:0,$=p.fillTo(e,t,p.min,p.max,j),H=N(b($,y,S,k)),V=x,U=N(p.width*yo),B=!1,q=null,Y=null,W=null,K=null;null==d||0!=U&&null==h||(B=!0,q=d.values(e,t,a,m),Y=new Map,new Set(q).forEach((e=>{null!=e&&Y.set(e,new Path2D)})),U>0&&(W=h.values(e,t,a,m),K=new Map,new Set(W).forEach((e=>{null!=e&&K.set(e,new Path2D)}))));let{x0:Q,size:Z}=c;if(null!=Q&&null!=Z){A=1,f=Q.values(e,t,a,m),2==Q.unit&&(f=f.map((t=>e.posToVal(w+t*x,g.key,!0))));let n=Z.values(e,t,a,m);I=2==Z.unit?n[0]*x:_(n[0],g,x,w)-_(0,g,x,w),V=Ec(f,v,_,g,x,w,V),R=V-I+M}else V=Ec(f,v,_,g,x,w,V),R=V*o+M,I=V-R;R<1&&(R=0),U>=I/2&&(U=0),R<5&&(N=dl);let G=R>0;I=N(cl(V-R-(G?U:0),L,T)),P=(0==A?I/2:A==O?0:I)-A*O*((0==A?M/2:0)+(G?U/2:0));const J={stroke:null,fill:null,clip:null,band:null,gaps:null,flags:0},X=B?null:new Path2D;let ee=null;if(null!=F)ee=e.data[F.series[1]];else{let{y0:n,y1:r}=c;null!=n&&null!=r&&(v=r.values(e,t,a,m),ee=n.values(e,t,a,m))}let te=C*I,ne=E*I;for(let n=1==O?a:m;n>=a&&n<=m;n+=O){let r=v[n];if(null==r)continue;if(null!=ee){var re;let e=null!==(re=ee[n])&&void 0!==re?re:0;if(r-e==0)continue;H=b(e,y,S,k)}let a=_(2!=g.distr||null!=c?f[n]:n,g,x,w),i=b(Bo(r,$),y,S,k),o=N(a-P),l=N(tl(i,H)),s=N(el(i,H)),u=l-s;if(null!=r){let a=r<0?ne:te,i=r<0?te:ne;B?(U>0&&null!=W[n]&&D(K.get(W[n]),o,s+Go(U/2),I,tl(0,u-U),a,i),null!=q[n]&&D(Y.get(q[n]),o,s+Go(U/2),I,tl(0,u-U),a,i)):D(X,o,s+Go(U/2),I,tl(0,u-U),a,i),z(e,t,n,o-U/2,s,I+U,u)}}if(U>0)J.stroke=B?K:X;else if(!B){var ae;J._fill=0==p.width?p._fill:null!==(ae=p._stroke)&&void 0!==ae?ae:p._fill,J.width=0}return J.fill=B?Y:X,J}))},e.spline=function(e){return function(e,t){const n=Bo(null===t||void 0===t?void 0:t.alignGaps,0);return(t,r,a,i)=>rc(t,r,((o,l,s,c,u,d,h,m,p,f,v)=>{let g,y,_,b=o.pxRound,w=e=>b(d(e,c,f,m)),k=e=>b(h(e,u,v,p));0==c.ori?(g=dc,_=mc,y=_c):(g=hc,_=pc,y=bc);const x=c.dir*(0==c.ori?1:-1);a=Do(s,a,i,1),i=Do(s,a,i,-1);let S=w(l[1==x?a:i]),C=S,E=[],N=[];for(let e=1==x?a:i;e>=a&&e<=i;e+=x)if(null!=s[e]){let t=w(l[e]);E.push(C=t),N.push(k(s[e]))}const A={stroke:e(E,N,g,_,y,b),fill:null,clip:null,band:null,gaps:null,flags:nc},M=A.stroke;let[T,L]=ac(t,r);if(null!=o.fill||0!=T){let e=A.fill=new Path2D(M),n=k(o.fillTo(t,r,o.min,o.max,T));_(e,C,n),_(e,S,n)}if(!o.spanGaps){let e=[];e.push(...sc(l,s,a,i,x,w,n)),A.gaps=e=o.gaps(t,r,a,i,e),A.clip=lc(e,c.ori,m,p,f,v)}return 0!=L&&(A.band=2==L?[oc(t,r,a,i,M,-1),oc(t,r,a,i,M,1)]:oc(t,r,a,i,M,L)),A}))}(Nc,e)}}const qc=["#e54040","#32a9dc","#2ee329","#7126a1","#e38f0f","#3d811a","#ffea00","#2d2d2d","#da42a6","#a44e0c"],Yc=e=>{if(7!=e.length)return"0, 0, 0";const t=parseInt(e.slice(1,3),16),n=parseInt(e.slice(3,5),16),r=parseInt(e.slice(5,7),16);return"".concat(t,", ").concat(n,", ").concat(r)},Wc={[ut.yhatUpper]:"#7126a1",[ut.yhatLower]:"#7126a1",[ut.yhat]:"#da42a6",[ut.anomaly]:"#da4242",[ut.anomalyScore]:"#7126a1",[ut.actual]:"#203ea9",[ut.training]:"rgba(".concat(Yc("#203ea9"),", 0.2)")},Kc=e=>{const t=16777215;let n=1,r=0,a=1;if(e.length>0)for(let o=0;or&&(r=e[o].charCodeAt(0)),a=parseInt(String(t/r)),n=(n+e[o].charCodeAt(0)*a*49979693)%t;let i=(n*e.length%t).toString(16);return i=i.padEnd(6,i),"#".concat(i)},Qc=function(e,t){let n=!(arguments.length>2&&void 0!==arguments[2])||arguments[2];const{__name__:r,...a}=e.metric,i=t||"".concat(n?"[Query ".concat(e.group,"] "):"").concat(r||"");return 0==Object.keys(a).length?i||"value":"".concat(i,"{").concat(Object.entries(a).map((e=>"".concat(e[0],"=").concat(JSON.stringify(e[1])))).join(", "),"}")},Zc=e=>{switch(e){case"NaN":return NaN;case"Inf":case"+Inf":return 1/0;case"-Inf":return-1/0;default:return parseFloat(e)}},Gc=e=>{if(e.length<2)return!1;const t=["le","vmrange"],n=Object.keys(e[0].metric).filter((e=>!t.includes(e))),r=e.every((r=>{const a=Object.keys(r.metric).filter((e=>!t.includes(e)));return n.length===a.length&&a.every((t=>r.metric[t]===e[0].metric[t]))}));return r&&e.every((e=>t.some((t=>t in e.metric))))},Jc=((e,t,n)=>{const r=[];for(let a=0;aMath.round(e))).join(", "))}return r.map((e=>"rgb(".concat(e,")")))})([246,226,219],[127,39,4],16),Xc=()=>(e,t)=>{const n=Math.round(devicePixelRatio);Bc.orient(e,t,((r,a,i,o,l,s,c,u,d,h,m,p,f,v)=>{const[g,y,_]=e.data[t],b=g.length,w=((e,t)=>{const n=e.data[t][2],r=Jc;let a=1/0,i=-1/0;for(let c=0;c0&&(a=Math.min(a,n[c]),i=Math.max(i,n[c]));const o=i-a,l=r.length,s=Array(n.length);for(let c=0;cnew Path2D)),S=b-y.lastIndexOf(y[0]),C=b/S,E=y[1]-y[0],N=g[S]-g[0],A=s(N,o,h,u)-s(0,o,h,u)-n,M=c(E,l,m,d)-c(0,l,m,d)+n,T=y.slice(0,S).map((e=>Math.round(c(e,l,m,d)-M/2))),L=Array.from({length:C},((e,t)=>Math.round(s(g[t*S],o,h,u)-A)));for(let e=0;e0&&g[e]>=(o.min||-1/0)&&g[e]<=(o.max||1/0)&&y[e]>=(l.min||-1/0)&&y[e]<=(l.max||1/0)){const t=L[~~(e/S)],n=T[e%S];v(x[w[e]],t,n,A,M)}e.ctx.save(),e.ctx.rect(e.bbox.left,e.bbox.top,e.bbox.width,e.bbox.height),e.ctx.clip(),x.forEach(((t,n)=>{e.ctx.fillStyle=k[n],e.ctx.fill(t)})),e.ctx.restore()}))},eu=e=>{const t=(e.metric.vmrange||e.metric.le||"").split("...");return Zc(t[t.length-1])},tu=(e,t)=>eu(e)-eu(t),nu=(e,t)=>{if(!t)return e;const n=(e=>{var t;if(!e.every((e=>e.metric.le)))return e;const n=e.sort(((e,t)=>parseFloat(e.metric.le)-parseFloat(t.metric.le))),r=(null===(t=e[0])||void 0===t?void 0:t.group)||1;let a={metric:{le:""},values:[],group:r};const i=[];for(const l of n){const e=[a.metric.le,l.metric.le].filter((e=>e)).join("..."),t=[];for(const[n,r]of l.values){var o;const e=+r-+((null===(o=a.values.find((e=>e[0]===n)))||void 0===o?void 0:o[1])||0);t.push([n,"".concat(e)])}i.push({metric:{vmrange:e},values:t,group:r}),a=l}return i})(e.sort(tu)),r={};n.forEach((e=>e.values.forEach((e=>{let[t,n]=e;r[t]=(r[t]||0)+ +n}))));return n.map((e=>{const t=e.values.map((e=>{let[t,n]=e;const a=r[t];return[t,"".concat(Math.round(+n/a*100))]}));return{...e,values:t}})).filter((e=>!e.values.every((e=>"0"===e[1]))))},ru=e=>{const t=["__name__","for"];return Object.entries(e).filter((e=>{let[n]=e;return!t.includes(n)})).map((e=>{let[t,n]=e;return"".concat(t,": ").concat(n)})).join(",")},au=(e,t,n,r)=>{const a={},i=r?0:Math.min(e.length,qc.length);for(let o=0;o{const l=r?(e=>{const t=(null===e||void 0===e?void 0:e.__name__)||"",n=new RegExp("(".concat(Object.values(ut).join("|"),")$")),r=t.match(n),a=r&&r[0];return{value:/(?:^|[^a-zA-Z0-9_])y(?:$|[^a-zA-Z0-9_])/.test(t)?ut.actual:a,group:ru(e)}})(e[o].metric):null,s=r?(null===l||void 0===l?void 0:l.group)||"":Qc(i,n[i.group-1]);return{label:s,dash:cu(l),width:uu(l),stroke:hu({metricInfo:l,label:s,isAnomalyUI:r,colorState:a}),points:du(l),spanGaps:!1,forecast:null===l||void 0===l?void 0:l.value,forecastGroup:null===l||void 0===l?void 0:l.group,freeFormFields:i.metric,show:!lu(s,t),scale:"1",...iu(i)}}},iu=e=>{const t=e.values.map((e=>Zc(e[1]))),{min:n,max:r,median:a,last:i}={min:Fi(t),max:zi(t),median:ji(t),last:$i(t)};return{median:a,statsFormatted:{min:Hi(n,n,r),max:Hi(r,n,r),median:Hi(a,n,r),last:Hi(i,n,r)}}},ou=(e,t)=>({group:t,label:e.label||"",color:e.stroke,checked:e.show||!1,freeFormFields:e.freeFormFields,statsFormatted:e.statsFormatted,median:e.median}),lu=(e,t)=>t.includes("".concat(e)),su=e=>{for(let t=e.series.length-1;t>=0;t--)e.delSeries(t)},cu=e=>{const t=(null===e||void 0===e?void 0:e.value)===ut.yhatLower,n=(null===e||void 0===e?void 0:e.value)===ut.yhatUpper,r=(null===e||void 0===e?void 0:e.value)===ut.yhat;return t||n?[10,5]:r?[10,2]:[]},uu=e=>{const t=(null===e||void 0===e?void 0:e.value)===ut.yhatLower,n=(null===e||void 0===e?void 0:e.value)===ut.yhatUpper,r=(null===e||void 0===e?void 0:e.value)===ut.yhat,a=(null===e||void 0===e?void 0:e.value)===ut.anomaly;return n||t?.7:r?1:a?0:1.4},du=e=>(null===e||void 0===e?void 0:e.value)===ut.anomaly?{size:8,width:4,space:0}:{size:4.2,width:1.4},hu=e=>{let{metricInfo:t,label:n,isAnomalyUI:r,colorState:a}=e;const i=a[n]||Kc(n),o=(null===t||void 0===t?void 0:t.value)===ut.anomaly;return r&&o?Wc[ut.anomaly]:!r||o||null!==t&&void 0!==t&&t.value?null!==t&&void 0!==t&&t.value?null!==t&&void 0!==t&&t.value?Wc[null===t||void 0===t?void 0:t.value]:i:a[n]||Kc(n):Wc[ut.actual]},mu=e=>{let{width:t=400,height:n=500}=e;return{width:t,height:n,series:[],tzDate:e=>i()(Gt(Xt(e))).local().toDate(),legend:{show:!1},cursor:{drag:{x:!0,y:!1},focus:{prox:30},points:{size:5.6,width:1.4},bind:{click:()=>null,dblclick:()=>null}}}},pu=e=>{su(e),(e=>{Object.keys(e.hooks).forEach((t=>{e.hooks[t]=[]}))})(e),e.setData([])},fu=e=>{let{min:t,max:n}=e;return[t,n]},vu=function(e){let 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,a=arguments.length>4?arguments[4]:void 0;return a.limits.enable?a.limits.range[r]:Yi(t,n)},gu=(e,t)=>{const n={x:{range:()=>fu(t)}},r=Object.keys(e.limits.range);return(r.length?r:["1"]).forEach((t=>{n[t]={range:function(n){return vu(n,arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,t,e)}}})),n},yu=e=>t=>{const n=t.posToVal(t.select.left,"x"),r=t.posToVal(t.select.left+t.select.width,"x");e({min:n,max:r})};function _u(e){const t=Yc(Wc[e]);return"rgba(".concat(t,", 0.05)")}function bu(e,t){!function(e,t){if(t.has(e))throw new TypeError("Cannot initialize the same private elements twice on an object")}(e,t),t.add(e)}function wu(e,t,n){if(!t.has(e))throw new TypeError("attempted to get private field on non-instance");return n}function ku(e){return ku="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},ku(e)}function xu(e){var t=function(e,t){if("object"!=ku(e)||!e)return e;var n=e[Symbol.toPrimitive];if(void 0!==n){var r=n.call(e,t||"default");if("object"!=ku(r))return r;throw new TypeError("@@toPrimitive must return a primitive value.")}return("string"===t?String:Number)(e)}(e,"string");return"symbol"==ku(t)?t:String(t)}function Su(e,t,n){return(t=xu(t))in e?Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}):e[t]=n,e}function Cu(){return{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}}let Eu={async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1};function Nu(e){Eu=e}const Au=/[&<>"']/,Mu=new RegExp(Au.source,"g"),Tu=/[<>"']|&(?!(#\d{1,7}|#[Xx][a-fA-F0-9]{1,6}|\w+);)/,Lu=new RegExp(Tu.source,"g"),Ou={"&":"&","<":"<",">":">",'"':""","'":"'"},Pu=e=>Ou[e];function Iu(e,t){if(t){if(Au.test(e))return e.replace(Mu,Pu)}else if(Tu.test(e))return e.replace(Lu,Pu);return e}const Ru=/&(#(?:\d+)|(?:#x[0-9A-Fa-f]+)|(?:\w+));?/gi;function Du(e){return e.replace(Ru,((e,t)=>"colon"===(t=t.toLowerCase())?":":"#"===t.charAt(0)?"x"===t.charAt(1)?String.fromCharCode(parseInt(t.substring(2),16)):String.fromCharCode(+t.substring(1)):""))}const zu=/(^|[^\[])\^/g;function Fu(e,t){e="string"===typeof e?e:e.source,t=t||"";const n={replace:(t,r)=>(r=(r=r.source||r).replace(zu,"$1"),e=e.replace(t,r),n),getRegex:()=>new RegExp(e,t)};return n}const ju=/[^\w:]/g,$u=/^$|^[a-z][a-z0-9+.-]*:|^[?#]/i;function Hu(e,t,n){if(e){let e;try{e=decodeURIComponent(Du(n)).replace(ju,"").toLowerCase()}catch(ap){return null}if(0===e.indexOf("javascript:")||0===e.indexOf("vbscript:")||0===e.indexOf("data:"))return null}t&&!$u.test(n)&&(n=function(e,t){Vu[" "+e]||(Uu.test(e)?Vu[" "+e]=e+"/":Vu[" "+e]=Ku(e,"/",!0));e=Vu[" "+e];const n=-1===e.indexOf(":");return"//"===t.substring(0,2)?n?t:e.replace(Bu,"$1")+t:"/"===t.charAt(0)?n?t:e.replace(qu,"$1")+t:e+t}(t,n));try{n=encodeURI(n).replace(/%25/g,"%")}catch(ap){return null}return n}const Vu={},Uu=/^[^:]+:\/*[^/]*$/,Bu=/^([^:]+:)[\s\S]*$/,qu=/^([^:]+:\/*[^/]*)[\s\S]*$/;const Yu={exec:function(){}};function Wu(e,t){const n=e.replace(/\|/g,((e,t,n)=>{let r=!1,a=t;for(;--a>=0&&"\\"===n[a];)r=!r;return r?"|":" |"})).split(/ \|/);let 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.length0)return{type:"space",raw:t[0]}}code(e){const t=this.rules.block.code.exec(e);if(t){const e=t[0].replace(/^ {1,4}/gm,"");return{type:"code",raw:t[0],codeBlockStyle:"indented",text:this.options.pedantic?e:Ku(e,"\n")}}}fences(e){const t=this.rules.block.fences.exec(e);if(t){const e=t[0],n=function(e,t){const n=e.match(/^(\s+)(?:```)/);if(null===n)return t;const r=n[1];return t.split("\n").map((e=>{const t=e.match(/^\s+/);if(null===t)return e;const[n]=t;return n.length>=r.length?e.slice(r.length):e})).join("\n")}(e,t[3]||"");return{type:"code",raw:e,lang:t[2]?t[2].trim().replace(this.rules.inline._escapes,"$1"):t[2],text:n}}}heading(e){const t=this.rules.block.heading.exec(e);if(t){let e=t[2].trim();if(/#$/.test(e)){const t=Ku(e,"#");this.options.pedantic?e=t.trim():t&&!/ $/.test(t)||(e=t.trim())}return{type:"heading",raw:t[0],depth:t[1].length,text:e,tokens:this.lexer.inline(e)}}}hr(e){const t=this.rules.block.hr.exec(e);if(t)return{type:"hr",raw:t[0]}}blockquote(e){const t=this.rules.block.blockquote.exec(e);if(t){const e=t[0].replace(/^ *>[ \t]?/gm,""),n=this.lexer.state.top;this.lexer.state.top=!0;const r=this.lexer.blockTokens(e);return this.lexer.state.top=n,{type:"blockquote",raw:t[0],tokens:r,text:e}}}list(e){let t=this.rules.block.list.exec(e);if(t){let n,r,a,i,o,l,s,c,u,d,h,m,p=t[1].trim();const f=p.length>1,v={type:"list",raw:"",ordered:f,start:f?+p.slice(0,-1):"",loose:!1,items:[]};p=f?"\\d{1,9}\\".concat(p.slice(-1)):"\\".concat(p),this.options.pedantic&&(p=f?p:"[*+-]");const g=new RegExp("^( {0,3}".concat(p,")((?:[\t ][^\\n]*)?(?:\\n|$))"));for(;e&&(m=!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+/,(e=>" ".repeat(3*e.length))),u=e.split("\n",1)[0],this.options.pedantic?(i=2,h=c.trimLeft()):(i=t[2].search(/[^ ]/),i=i>4?1:i,h=c.slice(i),i+=t[1].length),l=!1,!c&&/^ *$/.test(u)&&(n+=u+"\n",e=e.substring(u.length+1),m=!0),!m){const t=new RegExp("^ {0,".concat(Math.min(3,i-1),"}(?:[*+-]|\\d{1,9}[.)])((?:[ \t][^\\n]*)?(?:\\n|$))")),r=new RegExp("^ {0,".concat(Math.min(3,i-1),"}((?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$)")),a=new RegExp("^ {0,".concat(Math.min(3,i-1),"}(?:```|~~~)")),o=new RegExp("^ {0,".concat(Math.min(3,i-1),"}#"));for(;e&&(d=e.split("\n",1)[0],u=d,this.options.pedantic&&(u=u.replace(/^ {1,4}(?=( {4})*[^ ])/g," ")),!a.test(u))&&!o.test(u)&&!t.test(u)&&!r.test(e);){if(u.search(/[^ ]/)>=i||!u.trim())h+="\n"+u.slice(i);else{if(l)break;if(c.search(/[^ ]/)>=4)break;if(a.test(c))break;if(o.test(c))break;if(r.test(c))break;h+="\n"+u}l||u.trim()||(l=!0),n+=d+"\n",e=e.substring(d.length+1),c=u.slice(i)}}v.loose||(s?v.loose=!0:/\n *\n *$/.test(n)&&(s=!0)),this.options.gfm&&(r=/^\[[ xX]\] /.exec(h),r&&(a="[ ] "!==r[0],h=h.replace(/^\[[ xX]\] +/,""))),v.items.push({type:"list_item",raw:n,task:!!r,checked:a,loose:!1,text:h}),v.raw+=n}v.items[v.items.length-1].raw=n.trimRight(),v.items[v.items.length-1].text=h.trimRight(),v.raw=v.raw.trimRight();const y=v.items.length;for(o=0;o"space"===e.type)),t=e.length>0&&e.some((e=>/\n.*\n/.test(e.raw)));v.loose=t}if(v.loose)for(o=0;o$/,"$1").replace(this.rules.inline._escapes,"$1"):"",r=t[3]?t[3].substring(1,t[3].length-1).replace(this.rules.inline._escapes,"$1"):t[3];return{type:"def",tag:e,raw:t[0],href:n,title:r}}}table(e){const t=this.rules.block.table.exec(e);if(t){const e={type:"table",header:Wu(t[1]).map((e=>({text:e}))),align:t[2].replace(/^ *|\| *$/g,"").split(/ *\| */),rows:t[3]&&t[3].trim()?t[3].replace(/\n[ \t]*$/,"").split("\n"):[]};if(e.header.length===e.align.length){e.raw=t[0];let n,r,a,i,o=e.align.length;for(n=0;n({text:e})));for(o=e.header.length,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,block:!1,text:this.options.sanitize?this.options.sanitizer?this.options.sanitizer(t[0]):Iu(t[0]):t[0]}}link(e){const t=this.rules.inline.link.exec(e);if(t){const e=t[2].trim();if(!this.options.pedantic&&/^$/.test(e))return;const t=Ku(e.slice(0,-1),"\\");if((e.length-t.length)%2===0)return}else{const e=function(e,t){if(-1===e.indexOf(t[1]))return-1;const n=e.length;let r=0,a=0;for(;a-1){const n=(0===t[0].indexOf("!")?5:4)+t[1].length+e;t[2]=t[2].substring(0,e),t[0]=t[0].substring(0,n).trim(),t[3]=""}}let n=t[2],r="";if(this.options.pedantic){const e=/^([^'"]*[^\s])\s+(['"])(.*)\2/.exec(n);e&&(n=e[1],r=e[3])}else r=t[3]?t[3].slice(1,-1):"";return n=n.trim(),/^$/.test(e)?n.slice(1):n.slice(1,-1)),Qu(t,{href:n?n.replace(this.rules.inline._escapes,"$1"):n,title:r?r.replace(this.rules.inline._escapes,"$1"):r},t[0],this.lexer)}}reflink(e,t){let n;if((n=this.rules.inline.reflink.exec(e))||(n=this.rules.inline.nolink.exec(e))){let e=(n[2]||n[1]).replace(/\s+/g," ");if(e=t[e.toLowerCase()],!e){const e=n[0].charAt(0);return{type:"text",raw:e,text:e}}return Qu(n,e,n[0],this.lexer)}}emStrong(e,t){let n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"",r=this.rules.inline.emStrong.lDelim.exec(e);if(!r)return;if(r[3]&&n.match(/[\p{L}\p{N}]/u))return;if(!(r[1]||r[2]||"")||!n||this.rules.inline.punctuation.exec(n)){const n=r[0].length-1;let a,i,o=n,l=0;const s="*"===r[0][0]?this.rules.inline.emStrong.rDelimAst:this.rules.inline.emStrong.rDelimUnd;for(s.lastIndex=0,t=t.slice(-1*e.length+n);null!=(r=s.exec(t));){if(a=r[1]||r[2]||r[3]||r[4]||r[5]||r[6],!a)continue;if(i=a.length,r[3]||r[4]){o+=i;continue}if((r[5]||r[6])&&n%3&&!((n+i)%3)){l+=i;continue}if(o-=i,o>0)continue;i=Math.min(i,i+o+l);const t=e.slice(0,n+r.index+i+1);if(Math.min(n,i)%2){const e=t.slice(1,-1);return{type:"em",raw:t,text:e,tokens:this.lexer.inlineTokens(e)}}const s=t.slice(2,-2);return{type:"strong",raw:t,text:s,tokens:this.lexer.inlineTokens(s)}}}}codespan(e){const t=this.rules.inline.code.exec(e);if(t){let e=t[2].replace(/\n/g," ");const n=/[^ ]/.test(e),r=/^ /.test(e)&&/ $/.test(e);return n&&r&&(e=e.substring(1,e.length-1)),e=Iu(e,!0),{type:"codespan",raw:t[0],text:e}}}br(e){const t=this.rules.inline.br.exec(e);if(t)return{type:"br",raw:t[0]}}del(e){const t=this.rules.inline.del.exec(e);if(t)return{type:"del",raw:t[0],text:t[2],tokens:this.lexer.inlineTokens(t[2])}}autolink(e,t){const n=this.rules.inline.autolink.exec(e);if(n){let e,r;return"@"===n[2]?(e=Iu(this.options.mangle?t(n[1]):n[1]),r="mailto:"+e):(e=Iu(n[1]),r=e),{type:"link",raw:n[0],text:e,href:r,tokens:[{type:"text",raw:e,text:e}]}}}url(e,t){let n;if(n=this.rules.inline.url.exec(e)){let e,r;if("@"===n[2])e=Iu(this.options.mangle?t(n[0]):n[0]),r="mailto:"+e;else{let t;do{t=n[0],n[0]=this.rules.inline._backpedal.exec(n[0])[0]}while(t!==n[0]);e=Iu(n[0]),r="www."===n[1]?"http://"+n[0]:n[0]}return{type:"link",raw:n[0],text:e,href:r,tokens:[{type:"text",raw:e,text:e}]}}}inlineText(e,t){const n=this.rules.inline.text.exec(e);if(n){let e;return e=this.lexer.state.inRawBlock?this.options.sanitize?this.options.sanitizer?this.options.sanitizer(n[0]):Iu(n[0]):n[0]:Iu(this.options.smartypants?t(n[0]):n[0]),{type:"text",raw:n[0],text:e}}}}const Gu={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]*?(?:[^\\n]*\\n+|$)|comment[^\\n]*(\\n+|$)|<\\?[\\s\\S]*?(?:\\?>\\n*|$)|\\n*|$)|\\n*|$)|)[\\s\\S]*?(?:(?:\\n *)+\\n|$)|<(?!script|pre|style|textarea)([a-z][\\w-]*)(?:attribute)*? */?>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$)|(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$))",def:/^ {0,3}\[(label)\]: *(?:\n *)?([^<\s][^\s]*|<.*?>)(?:(?: +(?:\n *)?| *\n *)(title))? *(?:\n+|$)/,table:Yu,lheading:/^((?:(?!^bull ).|\n(?!\n|bull ))+?)\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?'|\([^()]*\))/};Gu.def=Fu(Gu.def).replace("label",Gu._label).replace("title",Gu._title).getRegex(),Gu.bullet=/(?:[*+-]|\d{1,9}[.)])/,Gu.listItemStart=Fu(/^( *)(bull) */).replace("bull",Gu.bullet).getRegex(),Gu.list=Fu(Gu.list).replace(/bull/g,Gu.bullet).replace("hr","\\n+(?=\\1?(?:(?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$))").replace("def","\\n+(?="+Gu.def.source+")").getRegex(),Gu._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",Gu._comment=/|$)/,Gu.html=Fu(Gu.html,"i").replace("comment",Gu._comment).replace("tag",Gu._tag).replace("attribute",/ +[a-zA-Z:_][\w.:-]*(?: *= *"[^"\n]*"| *= *'[^'\n]*'| *= *[^\s"'=<>`]+)?/).getRegex(),Gu.lheading=Fu(Gu.lheading).replace(/bull/g,Gu.bullet).getRegex(),Gu.paragraph=Fu(Gu._paragraph).replace("hr",Gu.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",")|<(?:script|pre|style|textarea|!--)").replace("tag",Gu._tag).getRegex(),Gu.blockquote=Fu(Gu.blockquote).replace("paragraph",Gu.paragraph).getRegex(),Gu.normal={...Gu},Gu.gfm={...Gu.normal,table:"^ *([^\\n ].*\\|.*)\\n {0,3}(?:\\| *)?(:?-+:? *(?:\\| *:?-+:? *)*)(?:\\| *)?(?:\\n((?:(?! *\\n|hr|heading|blockquote|code|fences|list|html).*(?:\\n|$))*)\\n*|$)"},Gu.gfm.table=Fu(Gu.gfm.table).replace("hr",Gu.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",")|<(?:script|pre|style|textarea|!--)").replace("tag",Gu._tag).getRegex(),Gu.gfm.paragraph=Fu(Gu._paragraph).replace("hr",Gu.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("table",Gu.gfm.table).replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html",")|<(?:script|pre|style|textarea|!--)").replace("tag",Gu._tag).getRegex(),Gu.pedantic={...Gu.normal,html:Fu("^ *(?:comment *(?:\\n|\\s*$)|<(tag)[\\s\\S]+? *(?:\\n{2,}|\\s*$)|\\s]*)*?/?> *(?:\\n{2,}|\\s*$))").replace("comment",Gu._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:/^ *\[([^\]]+)\]: *]+)>?(?: +(["(][^\n]+[")]))? *(?:\n+|$)/,heading:/^(#{1,6})(.*)(?:\n+|$)/,fences:Yu,lheading:/^(.+?)\n {0,3}(=+|-+) *(?:\n+|$)/,paragraph:Fu(Gu.normal._paragraph).replace("hr",Gu.hr).replace("heading"," *#{1,6} *[^\n]").replace("lheading",Gu.lheading).replace("blockquote"," {0,3}>").replace("|fences","").replace("|list","").replace("|html","").getRegex()};const Ju={escape:/^\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/,autolink:/^<(scheme:[^\s\x00-\x1f<>]*|email)>/,url:Yu,tag:"^comment|^|^<[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:Yu,text:/^(`+|[^`])(?:(?= {2,}\n)|[\s\S]*?(?:(?=[\\.5&&(n="x"+n.toString(16)),r+="&#"+n+";";return r}Ju._punctuation="\\p{P}$+<=>`^|~",Ju.punctuation=Fu(Ju.punctuation,"u").replace(/punctuation/g,Ju._punctuation).getRegex(),Ju.blockSkip=/\[[^[\]]*?\]\([^\(\)]*?\)|`[^`]*?`|<[^<>]*?>/g,Ju.anyPunctuation=/\\[punct]/g,Ju._escapes=/\\([punct])/g,Ju._comment=Fu(Gu._comment).replace("(?:--\x3e|$)","--\x3e").getRegex(),Ju.emStrong.lDelim=Fu(Ju.emStrong.lDelim,"u").replace(/punct/g,Ju._punctuation).getRegex(),Ju.emStrong.rDelimAst=Fu(Ju.emStrong.rDelimAst,"gu").replace(/punct/g,Ju._punctuation).getRegex(),Ju.emStrong.rDelimUnd=Fu(Ju.emStrong.rDelimUnd,"gu").replace(/punct/g,Ju._punctuation).getRegex(),Ju.anyPunctuation=Fu(Ju.anyPunctuation,"gu").replace(/punct/g,Ju._punctuation).getRegex(),Ju._escapes=Fu(Ju._escapes,"gu").replace(/punct/g,Ju._punctuation).getRegex(),Ju._scheme=/[a-zA-Z][a-zA-Z0-9+.-]{1,31}/,Ju._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])?)+(?![-_])/,Ju.autolink=Fu(Ju.autolink).replace("scheme",Ju._scheme).replace("email",Ju._email).getRegex(),Ju._attribute=/\s+[a-zA-Z:_][\w.:-]*(?:\s*=\s*"[^"]*"|\s*=\s*'[^']*'|\s*=\s*[^\s"'=<>`]+)?/,Ju.tag=Fu(Ju.tag).replace("comment",Ju._comment).replace("attribute",Ju._attribute).getRegex(),Ju._label=/(?:\[(?:\\.|[^\[\]\\])*\]|\\.|`[^`]*`|[^\[\]\\`])*?/,Ju._href=/<(?:\\.|[^\n<>\\])+>|[^\s\x00-\x1f]*/,Ju._title=/"(?:\\"?|[^"\\])*"|'(?:\\'?|[^'\\])*'|\((?:\\\)?|[^)\\])*\)/,Ju.link=Fu(Ju.link).replace("label",Ju._label).replace("href",Ju._href).replace("title",Ju._title).getRegex(),Ju.reflink=Fu(Ju.reflink).replace("label",Ju._label).replace("ref",Gu._label).getRegex(),Ju.nolink=Fu(Ju.nolink).replace("ref",Gu._label).getRegex(),Ju.reflinkSearch=Fu(Ju.reflinkSearch,"g").replace("reflink",Ju.reflink).replace("nolink",Ju.nolink).getRegex(),Ju.normal={...Ju},Ju.pedantic={...Ju.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:Fu(/^!?\[(label)\]\((.*?)\)/).replace("label",Ju._label).getRegex(),reflink:Fu(/^!?\[(label)\]\s*\[([^\]]*)\]/).replace("label",Ju._label).getRegex()},Ju.gfm={...Ju.normal,escape:Fu(Ju.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]:[];for(e=this.options.pedantic?e.replace(/\t/g," ").replace(/^ +$/gm,""):e.replace(/^( *)(\t+)/gm,((e,t,n)=>t+" ".repeat(n.length)));e;)if(!(this.options.extensions&&this.options.extensions.block&&this.options.extensions.block.some((n=>!!(t=n.call({lexer:this},e,i))&&(e=e.substring(t.raw.length),i.push(t),!0)))))if(t=this.tokenizer.space(e))e=e.substring(t.raw.length),1===t.raw.length&&i.length>0?i[i.length-1].raw+="\n":i.push(t);else if(t=this.tokenizer.code(e))e=e.substring(t.raw.length),n=i[i.length-1],!n||"paragraph"!==n.type&&"text"!==n.type?i.push(t):(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue[this.inlineQueue.length-1].src=n.text);else if(t=this.tokenizer.fences(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.heading(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.hr(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.blockquote(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.list(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.html(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.def(e))e=e.substring(t.raw.length),n=i[i.length-1],!n||"paragraph"!==n.type&&"text"!==n.type?this.tokens.links[t.tag]||(this.tokens.links[t.tag]={href:t.href,title:t.title}):(n.raw+="\n"+t.raw,n.text+="\n"+t.raw,this.inlineQueue[this.inlineQueue.length-1].src=n.text);else if(t=this.tokenizer.table(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.lheading(e))e=e.substring(t.raw.length),i.push(t);else{if(r=e,this.options.extensions&&this.options.extensions.startBlock){let t=1/0;const n=e.slice(1);let a;this.options.extensions.startBlock.forEach((function(e){a=e.call({lexer:this},n),"number"===typeof a&&a>=0&&(t=Math.min(t,a))})),t<1/0&&t>=0&&(r=e.substring(0,t+1))}if(this.state.top&&(t=this.tokenizer.paragraph(r)))n=i[i.length-1],a&&"paragraph"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue.pop(),this.inlineQueue[this.inlineQueue.length-1].src=n.text):i.push(t),a=r.length!==e.length,e=e.substring(t.raw.length);else if(t=this.tokenizer.text(e))e=e.substring(t.raw.length),n=i[i.length-1],n&&"text"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue.pop(),this.inlineQueue[this.inlineQueue.length-1].src=n.text):i.push(t);else if(e){const t="Infinite loop on byte: "+e.charCodeAt(0);if(this.options.silent){console.error(t);break}throw new Error(t)}}return this.state.top=!0,i}inline(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[];return this.inlineQueue.push({src:e,tokens:t}),t}inlineTokens(e){let t,n,r,a,i,o,l=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[],s=e;if(this.tokens.links){const e=Object.keys(this.tokens.links);if(e.length>0)for(;null!=(a=this.tokenizer.rules.inline.reflinkSearch.exec(s));)e.includes(a[0].slice(a[0].lastIndexOf("[")+1,-1))&&(s=s.slice(0,a.index)+"["+"a".repeat(a[0].length-2)+"]"+s.slice(this.tokenizer.rules.inline.reflinkSearch.lastIndex))}for(;null!=(a=this.tokenizer.rules.inline.blockSkip.exec(s));)s=s.slice(0,a.index)+"["+"a".repeat(a[0].length-2)+"]"+s.slice(this.tokenizer.rules.inline.blockSkip.lastIndex);for(;null!=(a=this.tokenizer.rules.inline.anyPunctuation.exec(s));)s=s.slice(0,a.index)+"++"+s.slice(this.tokenizer.rules.inline.anyPunctuation.lastIndex);for(;e;)if(i||(o=""),i=!1,!(this.options.extensions&&this.options.extensions.inline&&this.options.extensions.inline.some((n=>!!(t=n.call({lexer:this},e,l))&&(e=e.substring(t.raw.length),l.push(t),!0)))))if(t=this.tokenizer.escape(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.tag(e))e=e.substring(t.raw.length),n=l[l.length-1],n&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(t=this.tokenizer.link(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.reflink(e,this.tokens.links))e=e.substring(t.raw.length),n=l[l.length-1],n&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(t=this.tokenizer.emStrong(e,s,o))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.codespan(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.br(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.del(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.autolink(e,ed))e=e.substring(t.raw.length),l.push(t);else if(this.state.inLink||!(t=this.tokenizer.url(e,ed))){if(r=e,this.options.extensions&&this.options.extensions.startInline){let t=1/0;const n=e.slice(1);let a;this.options.extensions.startInline.forEach((function(e){a=e.call({lexer:this},n),"number"===typeof a&&a>=0&&(t=Math.min(t,a))})),t<1/0&&t>=0&&(r=e.substring(0,t+1))}if(t=this.tokenizer.inlineText(r,Xu))e=e.substring(t.raw.length),"_"!==t.raw.slice(-1)&&(o=t.raw.slice(-1)),i=!0,n=l[l.length-1],n&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(e){const t="Infinite loop on byte: "+e.charCodeAt(0);if(this.options.silent){console.error(t);break}throw new Error(t)}}else e=e.substring(t.raw.length),l.push(t);return l}}class nd{constructor(e){this.options=e||Eu}code(e,t,n){const r=(t||"").match(/\S*/)[0];if(this.options.highlight){const t=this.options.highlight(e,r);null!=t&&t!==e&&(n=!0,e=t)}return e=e.replace(/\n$/,"")+"\n",r?'
'+(n?e:Iu(e,!0))+"
\n":"
"+(n?e:Iu(e,!0))+"
\n"}blockquote(e){return"
\n".concat(e,"
\n")}html(e,t){return e}heading(e,t,n,r){if(this.options.headerIds){const a=this.options.headerPrefix+r.slug(n);return"').concat(e,"\n")}return"").concat(e,"\n")}hr(){return this.options.xhtml?"
\n":"
\n"}list(e,t,n){const r=t?"ol":"ul";return"<"+r+(t&&1!==n?' start="'+n+'"':"")+">\n"+e+"\n"}listitem(e){return"
  • ".concat(e,"
  • \n")}checkbox(e){return" "}paragraph(e){return"

    ".concat(e,"

    \n")}table(e,t){return t&&(t="".concat(t,"")),"\n\n"+e+"\n"+t+"
    \n"}tablerow(e){return"\n".concat(e,"\n")}tablecell(e,t){const n=t.header?"th":"td";return(t.align?"<".concat(n,' align="').concat(t.align,'">'):"<".concat(n,">"))+e+"\n")}strong(e){return"".concat(e,"")}em(e){return"".concat(e,"")}codespan(e){return"".concat(e,"")}br(){return this.options.xhtml?"
    ":"
    "}del(e){return"".concat(e,"")}link(e,t,n){if(null===(e=Hu(this.options.sanitize,this.options.baseUrl,e)))return n;let r='",r}image(e,t,n){if(null===(e=Hu(this.options.sanitize,this.options.baseUrl,e)))return n;let r='').concat(n,'":">",r}text(e){return e}}class rd{strong(e){return e}em(e){return e}codespan(e){return e}del(e){return e}html(e){return e}text(e){return e}link(e,t,n){return""+n}image(e,t,n){return""+n}br(){return""}}class ad{constructor(){this.seen={}}serialize(e){return e.toLowerCase().trim().replace(/<[!\/a-z].*?>/gi,"").replace(/[\u2000-\u206F\u2E00-\u2E7F\\'!"#$%&()*+,./:;<=>?@[\]^`{|}~]/g,"").replace(/\s/g,"-")}getNextSafeSlug(e,t){let n=e,r=0;if(this.seen.hasOwnProperty(n)){r=this.seen[e];do{r++,n=e+"-"+r}while(this.seen.hasOwnProperty(n))}return t||(this.seen[e]=r,this.seen[n]=0),n}slug(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:{};const n=this.serialize(e);return this.getNextSafeSlug(n,t.dryrun)}}class id{constructor(e){this.options=e||Eu,this.options.renderer=this.options.renderer||new nd,this.renderer=this.options.renderer,this.renderer.options=this.options,this.textRenderer=new rd,this.slugger=new ad}static parse(e,t){return new id(t).parse(e)}static parseInline(e,t){return new id(t).parseInline(e)}parse(e){let t,n,r,a,i,o,l,s,c,u,d,h,m,p,f,v,g,y,_,b=!(arguments.length>1&&void 0!==arguments[1])||arguments[1],w="";const k=e.length;for(t=0;t0&&"paragraph"===f.tokens[0].type?(f.tokens[0].text=y+" "+f.tokens[0].text,f.tokens[0].tokens&&f.tokens[0].tokens.length>0&&"text"===f.tokens[0].tokens[0].type&&(f.tokens[0].tokens[0].text=y+" "+f.tokens[0].tokens[0].text)):f.tokens.unshift({type:"text",text:y}):p+=y),p+=this.parse(f.tokens,m),c+=this.renderer.listitem(p,g,v);w+=this.renderer.list(c,d,h);continue;case"html":w+=this.renderer.html(u.text,u.block);continue;case"paragraph":w+=this.renderer.paragraph(this.parseInline(u.tokens));continue;case"text":for(c=u.tokens?this.parseInline(u.tokens):u.text;t+1{"function"===typeof r&&(a=r,r=null);const i={...r};r={...this.defaults,...i};const o=wu(this,sd,ud).call(this,r.silent,r.async,a);if("undefined"===typeof n||null===n)return o(new Error("marked(): input parameter is undefined or null"));if("string"!==typeof n)return o(new Error("marked(): input parameter is of type "+Object.prototype.toString.call(n)+", string expected"));if(function(e,t){e&&!e.silent&&(t&&console.warn("marked(): callback is deprecated since version 5.0.0, should not be used and will be removed in the future. Read more here: https://marked.js.org/using_pro#async"),(e.sanitize||e.sanitizer)&&console.warn("marked(): sanitize and sanitizer parameters are deprecated since version 0.7.0, should not be used and will be removed in the future. Read more here: https://marked.js.org/#/USING_ADVANCED.md#options"),(e.highlight||"language-"!==e.langPrefix)&&console.warn("marked(): highlight and langPrefix parameters are deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-highlight."),e.mangle&&console.warn("marked(): mangle parameter is enabled by default, but is deprecated since version 5.0.0, and will be removed in the future. To clear this warning, install https://www.npmjs.com/package/marked-mangle, or disable by setting `{mangle: false}`."),e.baseUrl&&console.warn("marked(): baseUrl parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-base-url."),e.smartypants&&console.warn("marked(): smartypants parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-smartypants."),e.xhtml&&console.warn("marked(): xhtml parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-xhtml."),(e.headerIds||e.headerPrefix)&&console.warn("marked(): headerIds and headerPrefix parameters enabled by default, but are deprecated since version 5.0.0, and will be removed in the future. To clear this warning, install https://www.npmjs.com/package/marked-gfm-heading-id, or disable by setting `{headerIds: false}`."))}(r,a),r.hooks&&(r.hooks.options=r),a){const i=r.highlight;let l;try{r.hooks&&(n=r.hooks.preprocess(n)),l=e(n,r)}catch(ap){return o(ap)}const s=e=>{let n;if(!e)try{r.walkTokens&&this.walkTokens(l,r.walkTokens),n=t(l,r),r.hooks&&(n=r.hooks.postprocess(n))}catch(ap){e=ap}return r.highlight=i,e?o(e):a(null,n)};if(!i||i.length<3)return s();if(delete r.highlight,!l.length)return s();let c=0;return this.walkTokens(l,(e=>{"code"===e.type&&(c++,setTimeout((()=>{i(e.text,e.lang,((t,n)=>{if(t)return s(t);null!=n&&n!==e.text&&(e.text=n,e.escaped=!0),c--,0===c&&s()}))}),0))})),void(0===c&&s())}if(r.async)return Promise.resolve(r.hooks?r.hooks.preprocess(n):n).then((t=>e(t,r))).then((e=>r.walkTokens?Promise.all(this.walkTokens(e,r.walkTokens)).then((()=>e)):e)).then((e=>t(e,r))).then((e=>r.hooks?r.hooks.postprocess(e):e)).catch(o);try{r.hooks&&(n=r.hooks.preprocess(n));const a=e(n,r);r.walkTokens&&this.walkTokens(a,r.walkTokens);let i=t(a,r);return r.hooks&&(i=r.hooks.postprocess(i)),i}catch(ap){return o(ap)}}}function ud(e,t,n){return r=>{if(r.message+="\nPlease report this to https://github.com/markedjs/marked.",e){const e="

    An error occurred:

    "+Iu(r.message+"",!0)+"
    ";return t?Promise.resolve(e):n?void n(null,e):e}if(t)return Promise.reject(r);if(!n)throw r;n(r)}}const dd=new class{constructor(){bu(this,sd),bu(this,ld),Su(this,"defaults",{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}),Su(this,"options",this.setOptions),Su(this,"parse",wu(this,ld,cd).call(this,td.lex,id.parse)),Su(this,"parseInline",wu(this,ld,cd).call(this,td.lexInline,id.parseInline)),Su(this,"Parser",id),Su(this,"parser",id.parse),Su(this,"Renderer",nd),Su(this,"TextRenderer",rd),Su(this,"Lexer",td),Su(this,"lexer",td.lex),Su(this,"Tokenizer",Zu),Su(this,"Slugger",ad),Su(this,"Hooks",od),this.use(...arguments)}walkTokens(e,t){let n=[];for(const r of e)switch(n=n.concat(t.call(this,r)),r.type){case"table":for(const e of r.header)n=n.concat(this.walkTokens(e.tokens,t));for(const e of r.rows)for(const r of e)n=n.concat(this.walkTokens(r.tokens,t));break;case"list":n=n.concat(this.walkTokens(r.items,t));break;default:this.defaults.extensions&&this.defaults.extensions.childTokens&&this.defaults.extensions.childTokens[r.type]?this.defaults.extensions.childTokens[r.type].forEach((e=>{n=n.concat(this.walkTokens(r[e],t))})):r.tokens&&(n=n.concat(this.walkTokens(r.tokens,t)))}return n}use(){const e=this.defaults.extensions||{renderers:{},childTokens:{}};for(var t=arguments.length,n=new Array(t),r=0;r{const n={...t};if(n.async=this.defaults.async||n.async||!1,t.extensions&&(t.extensions.forEach((t=>{if(!t.name)throw new Error("extension name required");if(t.renderer){const n=e.renderers[t.name];e.renderers[t.name]=n?function(){for(var e=arguments.length,r=new Array(e),a=0;a{if(this.defaults.async)return Promise.resolve(t.hooks[n].call(e,a)).then((t=>r.call(e,t)));const i=t.hooks[n].call(e,a);return r.call(e,i)}:e[n]=function(){for(var a=arguments.length,i=new Array(a),o=0;o{const t='$1 target="_blank" class="'.concat("vm-link vm-link_colored",'" $2').concat("https://docs.victoriametrics.com/MetricsQL.html","#");return e.replace(/({var n;const r="h3"===e.tagName.toLowerCase();return t=r?null!==(n=e.textContent)&&void 0!==n?n:"":t,r?null:((e,t)=>{var n;const r=null!==(n=t.textContent)&&void 0!==n?n:"",a=(e=>{const t=[];let n=e.nextElementSibling;for(;n&&"p"===n.tagName.toLowerCase();)n&&t.push(n),n=n.nextElementSibling;return t})(t).map((e=>{var t;return null!==(t=e.outerHTML)&&void 0!==t?t:""})).join("\n");return{type:e,value:r,description:pd(a),icon:Ct(yr,{})}})(t,e)})).filter(Boolean)},vd=()=>{const{metricsQLFunctions:e}=En(),t=Nn();return(0,r.useEffect)((()=>{e.length||(async()=>{try{const e=await fetch(md),n=(e=>{const t=document.createElement("div");t.innerHTML=hd(e);const n=t.querySelectorAll("".concat("h3",", ").concat("h4"));return fd(n)})(await e.text());t({type:"SET_METRICSQL_FUNCTIONS",payload:n})}catch(ap){console.error("Error fetching or processing the MetricsQL.md file:",ap)}})()}),[]),e},gd=e=>{let{value:t,anchorEl:n,caretPosition:a,onSelect:o,onFoundOptions:l}=e;const[s,c]=(0,r.useState)(0),u=vd(),d=(0,r.useMemo)((()=>{const e=t.split("}");return e[e.length-1]}),[t]),h=(0,r.useMemo)((()=>{const e=d.match(/\b[^{}(),\s]+(?={|$)/g);return e?e[0]:""}),[d]),m=(0,r.useMemo)((()=>{const e=d.match(/[a-z_:-][\w\-.:/]*\b(?=\s*(=|!=|=~|!~))/g);return e?e[e.length-1]:""}),[d]),p=(0,r.useMemo)((()=>{if(!t||t.endsWith("}")||(e=>{const t=e.split(/\s+/),n=t.length,r=t[n-1],a=t[n-2],i=!r&&(e=>{const t=e.match(/"/g);return!!t&&t.length%2!==0})(e),o=(!r||t.length>1)&&!/([(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/.test(a);return i||o})(t))return pt.empty;const e=/\{[^}]*$/;switch(!0){case new RegExp("(".concat(Pi(h),")?{?.+").concat(Pi(m),'(=|!=|=~|!~)"?([^"]*)$'),"g").test(t):return pt.labelValue;case e.test(t):return pt.label;default:return pt.metricsql}}),[t,h,m]),f=(0,r.useMemo)((()=>{const e=t.match(/([\w_\-.:/]+(?![},]))$/);return e?e[0]:""}),[t]),{metrics:v,labels:g,labelValues:y,loading:_}=(e=>{let{valueByContext:t,metric:n,label:a,context:o}=e;const{serverUrl:l}=Nt(),{period:{start:s,end:c}}=vn(),{autocompleteCache:u}=En(),d=Nn(),[h,m]=(0,r.useState)(!1),[p,f]=(0,r.useState)(t),v=Oi()(f,500);(0,r.useEffect)((()=>(v(t),v.cancel)),[t,v]);const[g,y]=(0,r.useState)([]),[_,b]=(0,r.useState)([]),[w,k]=(0,r.useState)([]),x=(0,r.useRef)(new AbortController),S=(0,r.useCallback)((e=>{const t=i()(1e3*s).startOf("day").valueOf()/1e3,n=i()(1e3*c).endOf("day").valueOf()/1e3;return new URLSearchParams({...e||{},limit:"".concat(bn),start:"".concat(t),end:"".concat(n)})}),[s,c]),C=(e,t)=>e.map((e=>({value:e,type:"".concat(t),icon:Di[t]}))),E=async e=>{let{value:t,urlSuffix:n,setter:r,type:a,params:i}=e;if(!t&&a===Ri.metric)return;x.current.abort(),x.current=new AbortController;const{signal:o}=x.current,s={type:a,value:t,start:(null===i||void 0===i?void 0:i.get("start"))||"",end:(null===i||void 0===i?void 0:i.get("end"))||"",match:(null===i||void 0===i?void 0:i.get("match[]"))||""};m(!0);try{const e=u.get(s);if(e)return void r(C(e,a));const t=await fetch("".concat(l,"/api/v1/").concat(n,"?").concat(i),{signal:o});if(t.ok){const{data:e}=await t.json();r(C(e,a)),d({type:"SET_AUTOCOMPLETE_CACHE",payload:{key:s,value:e}})}}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&(d({type:"SET_AUTOCOMPLETE_CACHE",payload:{key:s,value:[]}}),console.error(ap))}finally{m(!1)}};return(0,r.useEffect)((()=>{const e=o!==pt.metricsql&&o!==pt.empty;if(!l||!n||e)return;y([]);const t=Ii(Pi(n));return E({value:p,urlSuffix:"label/__name__/values",setter:y,type:Ri.metric,params:S({"match[]":'{__name__=~".*'.concat(t,'.*"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n]),(0,r.useEffect)((()=>{if(!l||!n||o!==pt.label)return;b([]);const e=Ii(n);return E({value:p,urlSuffix:"labels",setter:b,type:Ri.label,params:S({"match[]":'{__name__="'.concat(e,'"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n]),(0,r.useEffect)((()=>{if(!l||!n||!a||o!==pt.labelValue)return;k([]);const e=Ii(n),t=Ii(Pi(p));return E({value:p,urlSuffix:"label/".concat(a,"/values"),setter:k,type:Ri.labelValue,params:S({"match[]":'{__name__="'.concat(e,'", ').concat(a,'=~".*').concat(t,'.*"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n,a]),{metrics:g,labels:_,labelValues:w,loading:h}})({valueByContext:f,metric:h,label:m,context:p}),b=(0,r.useMemo)((()=>{switch(p){case pt.metricsql:return[...v,...u];case pt.label:return g;case pt.labelValue:return y;default:return[]}}),[p,v,g,y]);return(0,r.useEffect)((()=>{if(!n.current)return void c(0);const e=window.getComputedStyle(n.current),r="".concat(e.getPropertyValue("font-size")),a="".concat(e.getPropertyValue("font-family")),i=Vi(t,"".concat(r," ").concat(a));c(i)}),[n,a]),Ct(xt.FK,{children:Ct(Ti,{loading:_,disabledFullScreen:!0,value:f,options:b,anchor:n,minLength:0,offset:{top:0,left:s},onSelect:e=>{const n=t.lastIndexOf(f,a[0]),r=n+f.length,i=t.substring(0,n),l=t.substring(r);if(p===pt.labelValue){const t='"',n=/(?:=|!=|=~|!~)$/.test(i);e="".concat(n?t:"").concat(e)}const s="".concat(i).concat(e).concat(l);o(s)},onFoundOptions:l,maxDisplayResults:{limit:_n,message:"Please, specify the query more precisely."}})})},yd="No match! \nThis query hasn't selected any time series from database.\nEither the requested metrics are missing in the database,\nor there is a typo in series selector.",_d="The shown results are marked as PARTIAL.\nThe result is marked as partial if one or more vmstorage nodes failed to respond to the query.",bd=e=>{let{value:t,onChange:n,onEnter:a,onArrowUp:i,onArrowDown:o,autocomplete:l,error:s,stats:c,label:u,disabled:d=!1}=e;const{autocompleteQuick:h}=En(),{isMobile:m}=Wr(),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)([0,0]),y=(0,r.useRef)(null),_=[{show:"0"===(null===c||void 0===c?void 0:c.seriesFetched)&&!c.resultLength,text:yd},{show:null===c||void 0===c?void 0:c.isPartial,text:_d}].filter((e=>e.show)).map((e=>e.text)).join("");c&&(u="".concat(u," (").concat(c.executionTimeMsec||0,"ms)"));return(0,r.useEffect)((()=>{f(l)}),[h]),Ct("div",{className:"vm-query-editor",ref:y,children:[Ct($a,{value:t,label:u,type:"textarea",autofocus:!m,error:s,warning:_,onKeyDown:e=>{const{key:t,ctrlKey:n,metaKey:r,shiftKey:l}=e,s=(e.target.value||"").split("\n").length>1,c=n||r,u="ArrowDown"===t,d="Enter"===t;"ArrowUp"===t&&c&&(e.preventDefault(),i()),u&&c&&(e.preventDefault(),o()),d&&p&&e.preventDefault(),!d||l||s&&!c||p||(e.preventDefault(),a())},onChange:n,onChangeCaret:e=>{g(e)},disabled:d,inputmode:"search"}),l&&Ct(gd,{value:t,anchorEl:y,caretPosition:v,onSelect:e=>{n(e)},onFoundOptions:e=>{f(!!e.length)}})]})},wd=e=>{let{value:t=!1,disabled:n=!1,label:r,color:a="secondary",fullWidth:i,onChange:o}=e;return Ct("div",{className:xr()({"vm-switch":!0,"vm-switch_full-width":i,"vm-switch_disabled":n,"vm-switch_active":t,["vm-switch_".concat(a,"_active")]:t,["vm-switch_".concat(a)]:a}),onClick:()=>{n||o(!t)},children:[Ct("div",{className:"vm-switch-track",children:Ct("div",{className:"vm-switch-track__thumb"})}),r&&Ct("span",{className:"vm-switch__label",children:r})]})},kd=e=>{let{isMobile:t,hideButtons:n}=e;const{autocomplete:r}=En(),a=Nn(),{nocache:i,isTracingEnabled:o}=Fr(),l=jr();return Nr("keydown",(e=>{const{code:t,ctrlKey:n,altKey:r}=e;"Space"===t&&(n||r)&&(e.preventDefault(),a({type:"SET_AUTOCOMPLETE_QUICK",payload:!0}))})),Ct("div",{className:xr()({"vm-additional-settings":!0,"vm-additional-settings_mobile":t}),children:[!(null!==n&&void 0!==n&&n.autocomplete)&&Ct(fa,{title:Ct(xt.FK,{children:["Quick tip: ",ka]}),children:Ct(wd,{label:"Autocomplete",value:r,onChange:()=>{a({type:"TOGGLE_AUTOCOMPLETE"})},fullWidth:t})}),Ct(wd,{label:"Disable cache",value:i,onChange:()=>{l({type:"TOGGLE_NO_CACHE"})},fullWidth:t}),!(null!==n&&void 0!==n&&n.traceQuery)&&Ct(wd,{label:"Trace query",value:o,onChange:()=>{l({type:"TOGGLE_QUERY_TRACING"})},fullWidth:t})]})},xd=e=>{const{isMobile:t}=Wr(),n=(0,r.useRef)(null),{value:a,toggle:i,setFalse:o}=oa(!1);return t?Ct(xt.FK,{children:[Ct("div",{ref:n,children:Ct(aa,{variant:"outlined",startIcon:Ct(hr,{}),onClick:i,ariaLabel:"additional the query settings"})}),Ct(ia,{open:a,buttonRef:n,placement:"bottom-left",onClose:o,title:"Query settings",children:Ct(kd,{isMobile:t,...e})})]}):Ct(kd,{...e})},Sd=(e,t)=>e.length===t.length&&e.every(((e,n)=>e===t[n]));const Cd=()=>{const{showInfoMessage:e}=(0,r.useContext)(Zr);return async(t,n)=>{var r;if(null===(r=navigator)||void 0===r||!r.clipboard)return e({text:"Clipboard not supported",type:"error"}),console.warn("Clipboard not supported"),!1;try{return await navigator.clipboard.writeText(t),n&&e({text:n,type:"success"}),!0}catch(a){return a instanceof Error&&e({text:"".concat(a.name,": ").concat(a.message),type:"error"}),console.warn("Copy failed",a),!1}}},Ed=e=>{let{query:t,favorites:n,onRun:a,onToggleFavorite:i}=e;const o=Cd(),l=(0,r.useMemo)((()=>n.includes(t)),[t,n]);return Ct("div",{className:"vm-query-history-item",children:[Ct("span",{className:"vm-query-history-item__value",children:t}),Ct("div",{className:"vm-query-history-item__buttons",children:[Ct(fa,{title:"Execute query",children:Ct(aa,{size:"small",variant:"text",onClick:()=>{a(t)},startIcon:Ct(Wn,{})})}),Ct(fa,{title:"Copy query",children:Ct(aa,{size:"small",variant:"text",onClick:async()=>{await o(t,"Query has been copied")},startIcon:Ct(ar,{})})}),Ct(fa,{title:l?"Remove Favorite":"Add to Favorites",children:Ct(aa,{size:"small",variant:"text",color:l?"warning":"primary",onClick:()=>{i(t,l)},startIcon:Ct(l?vr:fr,{})})})]})]})},Nd="saved",Ad="favorite",Md=[{label:"Session history",value:"session"},{label:"Saved history",value:Nd},{label:"Favorite queries",value:Ad}],Td=e=>{let{handleSelectQuery:t}=e;const{queryHistory:n}=En(),{isMobile:a}=Wr(),{value:i,setTrue:o,setFalse:l}=oa(!1),[s,c]=(0,r.useState)(Md[0].value),[u,d]=(0,r.useState)(yn("QUERY_HISTORY")),[h,m]=(0,r.useState)(yn("QUERY_FAVORITES")),p=(0,r.useMemo)((()=>n.map((e=>e.values.filter((e=>e)).reverse()))),[n]),f=(0,r.useMemo)((()=>{switch(s){case Ad:return h;case Nd:return u;default:return p}}),[s,h,u,p]),v=null===f||void 0===f?void 0:f.every((e=>!e.length)),g=(0,r.useMemo)((()=>s===Ad?"Favorites queries are empty.\nTo see your favorites, mark a query as a favorite.":"Query history is empty.\nTo see the history, please make a query."),[s]),y=e=>n=>{t(n,e),l()},_=(e,t)=>{m((n=>{const r=n[0]||[];return t?[r.filter((t=>t!==e))]:t||r.includes(e)?n:[[...r,e]]}))};return(0,r.useEffect)((()=>{const e=h[0]||[],t=yn("QUERY_FAVORITES")[0]||[];Sd(e,t)||Je("QUERY_FAVORITES",JSON.stringify(h))}),[h]),Nr("storage",(()=>{d(yn("QUERY_HISTORY")),m(yn("QUERY_FAVORITES"))})),Ct(xt.FK,{children:[Ct(fa,{title:"Show history",children:Ct(aa,{color:"primary",variant:"text",onClick:o,startIcon:Ct(Vn,{}),ariaLabel:"Show history"})}),i&&Ct(pa,{title:"Query history",onClose:l,children:Ct("div",{className:xr()({"vm-query-history":!0,"vm-query-history_mobile":a}),children:[Ct("div",{className:xr()({"vm-query-history__tabs":!0,"vm-section-header__tabs":!0,"vm-query-history__tabs_mobile":a}),children:Ct(Mr,{activeItem:s,items:Md,onChange:c})}),Ct("div",{className:"vm-query-history-list",children:[v&&Ct("div",{className:"vm-query-history-list__no-data",children:g}),f.map(((e,t)=>Ct("div",{children:[f.length>1&&Ct("div",{className:xr()({"vm-query-history-list__group-title":!0,"vm-query-history-list__group-title_first":0===t}),children:["Query ",t+1]}),e.map(((e,n)=>Ct(Ed,{query:e,favorites:h.flat(),onRun:y(t),onToggleFavorite:_},n)))]},t))),s===Nd&&!v&&Ct("div",{className:"vm-query-history-footer",children:Ct(aa,{color:"error",variant:"outlined",size:"small",startIcon:Ct(Gn,{}),onClick:()=>{Je("QUERY_HISTORY","")},children:"clear history"})})]})]})})]})},Ld=e=>{let{queryErrors:t,setQueryErrors:n,setHideError:a,stats:i,onHideQuery:o,onRunQuery:l,hideButtons:s}=e;const{isMobile:c}=Wr(),{query:u,queryHistory:d,autocomplete:h,autocompleteQuick:m}=En(),p=Nn(),f=gn(),[v,g]=(0,r.useState)(u||[]),[y,_]=(0,r.useState)([]),[b,w]=(0,r.useState)(!1),k=Va(v),x=(()=>{const{serverUrl:e}=Nt();return async t=>{try{const n=encodeURIComponent(t),r="".concat(e,"/prettify-query?query=").concat(n),a=await fetch(r);if(200!=a.status)return{query:t,error:"Error requesting /prettify-query, status: "+a.status};const i=await a.json();return"success"!=i.status?{query:t,error:String(i.msg)}:{query:String(i.query),error:""}}catch(ap){return console.error(ap),ap instanceof Error&&"AbortError"!==ap.name?{query:t,error:"".concat(ap.name,": ").concat(ap.message)}:{query:t,error:String(ap)}}}})(),S=()=>{p({type:"SET_QUERY_HISTORY",payload:v.map(((e,t)=>{const n=d[t]||{values:[]},r=e===n.values[n.values.length-1],a=!r&&e?[...n.values,e]:n.values;return a.length>25&&a.shift(),{index:n.values.length-Number(r),values:a}}))}),p({type:"SET_QUERY",payload:v}),f({type:"RUN_QUERY"}),l()},C=(e,t)=>{g((n=>n.map(((n,r)=>r===t?e:n))))},E=(e,t)=>()=>{((e,t)=>{const{index:n,values:r}=d[t],a=n+e;a<0||a>=r.length||(C(r[a]||"",t),p({type:"SET_QUERY_HISTORY_BY_INDEX",payload:{value:{values:r,index:a},queryNumber:t}}))})(e,t)},N=e=>t=>{C(t,e),p({type:"SET_AUTOCOMPLETE_QUICK",payload:!1})},A=e=>()=>{var t;t=e,g((e=>e.filter(((e,n)=>n!==t)))),_((t=>t.includes(e)?t.filter((t=>t!==e)):t.map((t=>t>e?t-1:t))))},M=e=>t=>{((e,t)=>{const{ctrlKey:n,metaKey:r}=e;if(n||r){const e=v.map(((e,t)=>t)).filter((e=>e!==t));_((t=>Sd(e,t)?[]:e))}else _((e=>e.includes(t)?e.filter((e=>e!==t)):[...e,t]))})(t,e)};return(0,r.useEffect)((()=>{k&&v.length{o&&o(y)}),[y]),(0,r.useEffect)((()=>{b&&(S(),w(!1))}),[v,b]),Ct("div",{className:xr()({"vm-query-configurator":!0,"vm-block":!0,"vm-block_mobile":c}),children:[Ct("div",{className:"vm-query-configurator-list",children:v.map(((e,r)=>Ct("div",{className:xr()({"vm-query-configurator-list-row":!0,"vm-query-configurator-list-row_disabled":y.includes(r),"vm-query-configurator-list-row_mobile":c}),children:[Ct(bd,{value:v[r],autocomplete:!(null!==s&&void 0!==s&&s.autocomplete)&&(h||m),error:t[r],stats:i[r],onArrowUp:E(-1,r),onArrowDown:E(1,r),onEnter:S,onChange:N(r),label:"Query ".concat(v.length>1?r+1:""),disabled:y.includes(r)}),o&&Ct(fa,{title:y.includes(r)?"Enable query":"Disable query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"gray",startIcon:y.includes(r)?Ct(nr,{}):Ct(tr,{}),onClick:M(r),ariaLabel:"visibility query"})})}),!(null!==s&&void 0!==s&&s.prettify)&&Ct(fa,{title:"Prettify query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"gray",startIcon:Ct(rr,{}),onClick:async()=>await(async e=>{const t=await x(v[e]);a(!1),C(t.query,e),n((n=>(n[e]=t.error,[...n])))})(r),className:"prettify",ariaLabel:"prettify the query"})})}),v.length>1&&Ct(fa,{title:"Remove Query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"error",startIcon:Ct(Gn,{}),onClick:A(r),ariaLabel:"remove query"})})})]},r)))}),Ct("div",{className:"vm-query-configurator-settings",children:[Ct(xd,{hideButtons:s}),Ct("div",{className:"vm-query-configurator-settings__buttons",children:[Ct(Td,{handleSelectQuery:(e,t)=>{C(e,t),w(!0)}}),!(null!==s&&void 0!==s&&s.addQuery)&&v.length<4&&Ct(aa,{variant:"outlined",onClick:()=>{g((e=>[...e,""]))},startIcon:Ct(Jn,{}),children:"Add Query"}),Ct(aa,{variant:"contained",onClick:S,startIcon:Ct(Yn,{}),children:c?"Execute":"Execute Query"})]})]})]})};let Od=0;class Pd{constructor(e,t){this.tracing=void 0,this.query=void 0,this.tracingChildren=void 0,this.originalTracing=void 0,this.id=void 0,this.tracing=e,this.originalTracing=JSON.parse(JSON.stringify(e)),this.query=t,this.id=Od++;const n=e.children||[];this.tracingChildren=n.map((e=>new Pd(e,t)))}get queryValue(){return this.query}get idValue(){return this.id}get children(){return this.tracingChildren}get message(){return this.tracing.message}get duration(){return this.tracing.duration_msec}get JSON(){return JSON.stringify(this.tracing,null,2)}get originalJSON(){return JSON.stringify(this.originalTracing,null,2)}setTracing(e){this.tracing=e;const t=e.children||[];this.tracingChildren=t.map((e=>new Pd(e,this.query)))}setQuery(e){this.query=e}resetTracing(){this.tracing=this.originalTracing}}const Id=$e.anomaly==={NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE,Rd=e=>{let{predefinedQuery:t,visible:n,display:a,customStep:i,hideQuery:o,showAllSeries:l}=e;const{query:s}=En(),{period:c}=vn(),{displayType:u,nocache:d,isTracingEnabled:h,seriesLimits:m}=Fr(),{serverUrl:p}=Nt(),{isHistogram:f}=Ur(),[v,g]=(0,r.useState)(!1),[y,_]=(0,r.useState)(),[b,w]=(0,r.useState)(),[k,x]=(0,r.useState)(),[S,C]=(0,r.useState)(),[E,N]=(0,r.useState)([]),[A,M]=(0,r.useState)([]),[T,L]=(0,r.useState)(),[O,P]=(0,r.useState)([]),[I,R]=(0,r.useState)(!1),D=(0,r.useMemo)((()=>{const{end:e,start:t}=c;return Kt(e-t,f)}),[c,f]),z=(0,r.useCallback)(Oi()((async e=>{let{fetchUrl:t,fetchQueue:n,displayType:r,query:a,stateSeriesLimits:i,showAllSeries:o,hideQuery:l}=e;const s=new AbortController;P([...n,s]);try{const e=r===dt.chart,n=o?1/0:+i[r]||1/0;let c=n;const u=[],d=[];let h=1,m=0,p=!1;for await(const r of t){if(null===l||void 0===l?void 0:l.includes(h-1)){N((e=>[...e,""])),M((e=>[...e,{}])),h++;continue}const t=await fetch(r,{signal:s.signal}),i=await t.json();if(t.ok){if(M((e=>[...e,{...null===i||void 0===i?void 0:i.stats,isPartial:null===i||void 0===i?void 0:i.isPartial,resultLength:i.data.result.length}])),N((e=>[...e,""])),i.trace){const e=new Pd(i.trace,a[h-1]);d.push(e)}p=!Id&&e&&Gc(i.data.result),c=p?1/0:n;const t=c-u.length;i.data.result.slice(0,t).forEach((e=>{e.group=h,u.push(e)})),m+=i.data.result.length}else{u.push({metric:{},values:[],group:h});const e=i.errorType||ht.unknownType,t=[e,(null===i||void 0===i?void 0:i.error)||(null===i||void 0===i?void 0:i.message)||"see console for more details"].join(",\r\n");N((e=>[...e,"".concat(t)])),console.error("Fetch query error: ".concat(e),i)}h++}const f="Showing ".concat(u.length," series out of ").concat(m," series due to performance reasons. Please narrow down the query, so it returns less series");L(m>c?f:""),e?_(u):w(u),x(d),R((e=>m?p:e))}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&C("".concat(ap.name,": ").concat(ap.message))}g(!1)}),300),[]),F=(0,r.useMemo)((()=>{C(""),N([]),M([]);const e=null!==t&&void 0!==t?t:s,n=(a||u)===dt.chart;if(c)if(p)if(e.every((e=>!e.trim())))N(e.map((()=>ht.validQuery)));else{if(yt(p)){const t={...c};return t.step=Id?"".concat(1e3*Wt(i),"ms"):i,e.map((e=>n?((e,t,n,r,a)=>"".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(a?"&trace=1":""))(p,e,t,d,h):((e,t,n,r,a)=>"".concat(e,"/api/v1/query?query=").concat(encodeURIComponent(t),"&time=").concat(n.end,"&step=").concat(n.step).concat(r?"&nocache=1":"").concat(a?"&trace=1":""))(p,e,t,d,h)))}C(ht.validServer)}else C(ht.emptyServer)}),[p,c,u,i,o]),[j,$]=(0,r.useState)([]);return(0,r.useEffect)((()=>{const e=F===j&&!!t;if(!n||null===F||void 0===F||!F.length||e)return;g(!0);z({fetchUrl:F,fetchQueue:O,displayType:a||u,query:null!==t&&void 0!==t?t:s,stateSeriesLimits:m,showAllSeries:l,hideQuery:o}),$(F)}),[F,n,m,l]),(0,r.useEffect)((()=>{const e=O.slice(0,-1);e.length&&(e.map((e=>e.abort())),P(O.filter((e=>!e.signal.aborted))))}),[O]),(0,r.useEffect)((()=>{D===i&&_([])}),[I]),{fetchUrl:F,isLoading:v,graphData:y,liveData:b,error:S,queryErrors:E,setQueryErrors:N,queryStats:A,warning:T,traces:k,isHistogram:I}},Dd=e=>{let{containerStyles:t={},message:n}=e;const{isDarkTheme:r}=Nt();return Ct("div",{className:xr()({"vm-spinner":!0,"vm-spinner_dark":r}),style:t&&{},children:[Ct("div",{className:"half-circle-spinner",children:[Ct("div",{className:"circle circle-1"}),Ct("div",{className:"circle circle-2"})]}),n&&Ct("div",{className:"vm-spinner__message",children:n})]})};function zd(e){return function(e,t){return Object.fromEntries(Object.entries(e).filter(t))}(e,(e=>!!e[1]||"number"===typeof e[1]))}const Fd=e=>{let{text:t,href:n,children:r,colored:a=!0,underlined:i=!1,withIcon:o=!1}=e;return Ct("a",{href:n,className:xr()({"vm-link":!0,"vm-link_colored":a,"vm-link_underlined":i,"vm-link_with-icon":o}),target:"_blank",rel:"noreferrer",children:t||r})},jd=Ct(Fd,{text:"last_over_time",href:"https://docs.victoriametrics.com/MetricsQL.html#last_over_time",underlined:!0}),$d=Ct(Fd,{text:"instant query",href:"https://docs.victoriametrics.com/keyConcepts.html#instant-query",underlined:!0}),Hd=()=>Ct("div",{children:[Ct("p",{children:["This tab shows ",$d," results for the last 5 minutes ending at the selected time range."]}),Ct("p",{children:["Please wrap the query into ",jd," if you need results over arbitrary lookbehind interval."]})]}),Vd=e=>{let{value:t}=e;return Ct("div",{className:"vm-line-progress",children:[Ct("div",{className:"vm-line-progress-track",children:Ct("div",{className:"vm-line-progress-track__thumb",style:{width:"".concat(t,"%")}})}),Ct("span",{children:[t.toFixed(2),"%"]})]})},Ud=e=>{let{isRoot:t,trace:n,totalMsec:a,isExpandedAll:i}=e;const{isDarkTheme:o}=Nt(),{isMobile:l}=Wr(),[s,c]=(0,r.useState)({}),u=(0,r.useRef)(null),[d,h]=(0,r.useState)(!1),[m,p]=(0,r.useState)(!1),f=Bt(n.duration/1e3)||"".concat(n.duration,"ms");(0,r.useEffect)((()=>{if(!u.current)return;const e=u.current,t=u.current.children[0],{height:n}=t.getBoundingClientRect();h(n>e.clientHeight)}),[n]);const v=n.children&&!!n.children.length,g=n.duration/a*100,y=e=>{var t;const n=[e.idValue];return null===e||void 0===e||null===(t=e.children)||void 0===t||t.forEach((e=>{n.push(...y(e))})),n};return(0,r.useEffect)((()=>{if(!i)return void c([]);const e=y(n),t={};e.forEach((e=>{t[e]=!0})),c(t)}),[i]),Ct("div",{className:xr()({"vm-nested-nav":!0,"vm-nested-nav_root":t,"vm-nested-nav_dark":o,"vm-nested-nav_mobile":l}),children:[Ct("div",{className:xr()({"vm-nested-nav-header":!0,"vm-nested-nav-header_open":s[n.idValue]}),onClick:(_=n.idValue,()=>{v&&c((e=>({...e,[_]:!e[_]})))}),children:[v&&Ct("div",{className:xr()({"vm-nested-nav-header__icon":!0,"vm-nested-nav-header__icon_open":s[n.idValue]}),children:Ct($n,{})}),Ct("div",{className:"vm-nested-nav-header__progress",children:Ct(Vd,{value:g})}),Ct("div",{className:xr()({"vm-nested-nav-header__message":!0,"vm-nested-nav-header__message_show-full":m}),ref:u,children:[Ct("span",{className:"vm-nested-nav-header__message_duration",children:f}),":\xa0",Ct("span",{children:n.message})]}),Ct("div",{className:"vm-nested-nav-header-bottom",children:(d||m)&&Ct(aa,{variant:"text",size:"small",onClick:e=>{e.stopPropagation(),p((e=>!e))},children:m?"Hide":"Show full query"})})]}),s[n.idValue]&&Ct("div",{className:"vm-nested-nav__childrens",children:v&&n.children.map((e=>Ct(Ud,{trace:e,totalMsec:a,isExpandedAll:i},e.duration)))})]});var _},Bd=Ud,qd=e=>{let{editable:t=!1,defaultTile:n="JSON",displayTitle:a=!0,defaultJson:i="",resetValue:o="",onClose:l,onUpload:s}=e;const c=Cd(),{isMobile:u}=Wr(),[d,h]=(0,r.useState)(i),[m,p]=(0,r.useState)(n),[f,v]=(0,r.useState)(""),[g,y]=(0,r.useState)(""),_=(0,r.useMemo)((()=>{try{const e=JSON.parse(d),t=e.trace||e;return t.duration_msec?(new Pd(t,""),""):ht.traceNotFound}catch(ap){return ap instanceof Error?ap.message:"Unknown error"}}),[d]),b=()=>{y(_);m.trim()||v(ht.emptyTitle),_||f||(s(d,m),l())};return Ct("div",{className:xr()({"vm-json-form":!0,"vm-json-form_one-field":!a,"vm-json-form_one-field_mobile":!a&&u,"vm-json-form_mobile":u}),children:[a&&Ct($a,{value:m,label:"Title",error:f,onEnter:b,onChange:e=>{p(e)}}),Ct($a,{value:d,label:"JSON",type:"textarea",error:g,autofocus:!0,onChange:e=>{y(""),h(e)},onEnter:b,disabled:!t}),Ct("div",{className:"vm-json-form-footer",children:[Ct("div",{className:"vm-json-form-footer__controls",children:[Ct(aa,{variant:"outlined",startIcon:Ct(ar,{}),onClick:async()=>{await c(d,"Formatted JSON has been copied")},children:"Copy JSON"}),o&&Ct(aa,{variant:"text",startIcon:Ct(In,{}),onClick:()=>{h(o)},children:"Reset JSON"})]}),Ct("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Ct(aa,{variant:"outlined",color:"error",onClick:l,children:"Cancel"}),Ct(aa,{variant:"contained",onClick:b,children:"apply"})]})]})]})},Yd=e=>{let{traces:t,jsonEditor:n=!1,onDeleteClick:a}=e;const{isMobile:i}=Wr(),[o,l]=(0,r.useState)(null),[s,c]=(0,r.useState)([]),u=()=>{l(null)};if(!t.length)return Ct(Qr,{variant:"info",children:"Please re-run the query to see results of the tracing"});const d=e=>()=>{a(e)},h=e=>()=>{l(e)},m=e=>()=>{const t=new Blob([e.originalJSON],{type:"application/json"}),n=URL.createObjectURL(t),r=document.createElement("a");r.href=n,r.download="vmui_trace_".concat(e.queryValue,".json"),document.body.appendChild(r),r.click(),document.body.removeChild(r),URL.revokeObjectURL(n)};return Ct(xt.FK,{children:[Ct("div",{className:"vm-tracings-view",children:t.map((e=>{return Ct("div",{className:"vm-tracings-view-trace vm-block vm-block_empty-padding",children:[Ct("div",{className:"vm-tracings-view-trace-header",children:[Ct("h3",{className:"vm-tracings-view-trace-header-title",children:["Trace for ",Ct("b",{className:"vm-tracings-view-trace-header-title__query",children:e.queryValue})]}),Ct(fa,{title:s.includes(e.idValue)?"Collapse All":"Expand All",children:Ct(aa,{variant:"text",startIcon:Ct("div",{className:xr()({"vm-tracings-view-trace-header__expand-icon":!0,"vm-tracings-view-trace-header__expand-icon_open":s.includes(e.idValue)}),children:Ct($n,{})}),onClick:(t=e,()=>{c((e=>e.includes(t.idValue)?e.filter((e=>e!==t.idValue)):[...e,t.idValue]))}),ariaLabel:s.includes(e.idValue)?"Collapse All":"Expand All"})}),Ct(fa,{title:"Save Trace to JSON",children:Ct(aa,{variant:"text",startIcon:Ct(wr,{}),onClick:m(e),ariaLabel:"Save trace to JSON"})}),Ct(fa,{title:"Open JSON",children:Ct(aa,{variant:"text",startIcon:Ct(Zn,{}),onClick:h(e),ariaLabel:"open JSON"})}),Ct(fa,{title:"Remove trace",children:Ct(aa,{variant:"text",color:"error",startIcon:Ct(Gn,{}),onClick:d(e),ariaLabel:"remove trace"})})]}),Ct("nav",{className:xr()({"vm-tracings-view-trace__nav":!0,"vm-tracings-view-trace__nav_mobile":i}),children:Ct(Bd,{isRoot:!0,trace:e,totalMsec:e.duration,isExpandedAll:s.includes(e.idValue)})})]},e.idValue);var t}))}),o&&Ct(pa,{title:o.queryValue,onClose:u,children:Ct(qd,{editable:n,displayTitle:n,defaultTile:o.queryValue,defaultJson:o.JSON,resetValue:o.originalJSON,onClose:u,onUpload:(e,t)=>{if(n&&o)try{o.setTracing(JSON.parse(e)),o.setQuery(t),l(null)}catch(ap){console.error(ap)}}})})]})},Wd=e=>{let{traces:t,displayType:n}=e;const{isTracingEnabled:a}=Fr(),[i,o]=(0,r.useState)([]);return(0,r.useEffect)((()=>{t&&o([...i,...t])}),[t]),(0,r.useEffect)((()=>{o([])}),[n]),Ct(xt.FK,{children:a&&Ct("div",{className:"vm-custom-panel__trace",children:Ct(Yd,{traces:i,onDeleteClick:e=>{const t=i.filter((t=>t.idValue!==e.idValue));o([...t])}})})})},Kd=e=>{let{warning:t,query:n,onChange:a}=e;const{isMobile:i}=Wr(),{value:o,setTrue:l,setFalse:s}=oa(!1);return(0,r.useEffect)(s,[n]),(0,r.useEffect)((()=>{a(o)}),[o]),Ct(Qr,{variant:"warning",children:Ct("div",{className:xr()({"vm-custom-panel__warning":!0,"vm-custom-panel__warning_mobile":i}),children:[Ct("p",{children:t}),Ct(aa,{color:"warning",variant:"outlined",onClick:l,children:"Show all"})]})})},Qd=e=>(e=>e instanceof MouseEvent)(e)?e.clientX:e.touches[0].clientX,Zd=e=>{let{dragSpeed:t=.85,setPanning:n,setPlotScale:a}=e;const i=(0,r.useRef)({leftStart:0,xUnitsPerPx:0,scXMin:0,scXMax:0}),o=e=>{e.preventDefault();const n=Qd(e),{leftStart:r,xUnitsPerPx:o,scXMin:l,scXMax:s}=i.current,c=o*((n-r)*t);a({min:l-c,max:s-c})},l=()=>{n(!1),document.removeEventListener("mousemove",o),document.removeEventListener("mouseup",l),document.removeEventListener("touchmove",o),document.removeEventListener("touchend",l)};return e=>{let{e:t,u:r}=e;t.preventDefault(),n(!0),i.current={leftStart:Qd(t),xUnitsPerPx:r.posToVal(1,"x")-r.posToVal(0,"x"),scXMin:r.scales.x.min||0,scXMax:r.scales.x.max||0},document.addEventListener("mousemove",o),document.addEventListener("mouseup",l),document.addEventListener("touchmove",o),document.addEventListener("touchend",l)}},Gd=e=>{const[t,n]=(0,r.useState)(!1),a=Zd({dragSpeed:.9,setPanning:n,setPlotScale:e});return{onReadyChart:t=>{const n=e=>{const n=e instanceof MouseEvent&&(e=>{const{ctrlKey:t,metaKey:n,button:r}=e;return 0===r&&(t||n)})(e),r=window.TouchEvent&&e instanceof TouchEvent&&e.touches.length>1;(n||r)&&a({u:t,e:e})};t.over.addEventListener("mousedown",n),t.over.addEventListener("touchstart",n),t.over.addEventListener("wheel",(n=>{if(!n.ctrlKey&&!n.metaKey)return;n.preventDefault();const{width:r}=t.over.getBoundingClientRect(),a=t.cursor.left&&t.cursor.left>0?t.cursor.left:0,i=t.posToVal(a,"x"),o=(t.scales.x.max||0)-(t.scales.x.min||0),l=n.deltaY<0?.9*o:o/.9,s=i-a/r*l,c=s+l;t.batch((()=>e({min:s,max:c})))}))},isPanning:t}},Jd=e=>{const t=e[0].clientX-e[1].clientX,n=e[0].clientY-e[1].clientY;return Math.sqrt(t*t+n*n)},Xd=e=>{let{uPlotInst:t,xRange:n,setPlotScale:a}=e;const[i,o]=(0,r.useState)(0),l=(0,r.useCallback)((e=>{const{target:r,ctrlKey:i,metaKey:o,key:l}=e,s=r instanceof HTMLInputElement||r instanceof HTMLTextAreaElement;if(!t||s)return;const c="+"===l||"="===l;if(("-"===l||c)&&!(i||o)){e.preventDefault();const t=(n.max-n.min)/10*(c?1:-1);a({min:n.min+t,max:n.max-t})}}),[t,n]),s=(0,r.useCallback)((e=>{if(!t||2!==e.touches.length)return;e.preventDefault();const r=Jd(e.touches),o=i-r,l=t.scales.x.max||n.max,s=t.scales.x.min||n.min,c=(l-s)/50*(o>0?-1:1);t.batch((()=>a({min:s+c,max:l-c})))}),[t,i,n]);return Nr("keydown",l),Nr("touchmove",s),Nr("touchstart",(e=>{2===e.touches.length&&(e.preventDefault(),o(Jd(e.touches)))})),null},eh=e=>{let{period:t,setPeriod:n}=e;const[a,o]=(0,r.useState)({min:t.start,max:t.end});return(0,r.useEffect)((()=>{o({min:t.start,max:t.end})}),[t]),{xRange:a,setPlotScale:e=>{let{min:t,max:r}=e;const a=1e3*(r-t);aFt||n({from:i()(1e3*t).toDate(),to:i()(1e3*r).toDate()})}}},th=e=>{let{u:t,metrics:n,series:a,unit:o,isAnomalyView:l}=e;const[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)({seriesIdx:-1,dataIdx:-1}),[h,m]=(0,r.useState)([]),p=(0,r.useCallback)((()=>{const{seriesIdx:e,dataIdx:r}=u,s=n[e-1],c=a[e],d=new Set(n.map((e=>e.group))),h=(null===s||void 0===s?void 0:s.group)||0,m=it()(t,["data",e,r],0),p=it()(t,["scales","1","min"],0),f=it()(t,["scales","1","max"],1),v=it()(t,["data",0,r],0),g={top:t?t.valToPos(m||0,(null===c||void 0===c?void 0:c.scale)||"1"):0,left:t?t.valToPos(v,"x"):0};return{unit:o,point:g,u:t,id:"".concat(e,"_").concat(r),title:d.size>1&&!l?"Query ".concat(h):"",dates:[v?i()(1e3*v).tz().format(Ot):"-"],value:Hi(m,p,f),info:Ui(s),statsFormatted:null===c||void 0===c?void 0:c.statsFormatted,marker:"".concat(null===c||void 0===c?void 0:c.stroke)}}),[t,u,n,a,o,l]),f=(0,r.useCallback)((()=>{if(!s)return;const e=p();h.find((t=>t.id===e.id))||m((t=>[...t,e]))}),[p,h,s]);return(0,r.useEffect)((()=>{c(-1!==u.dataIdx&&-1!==u.seriesIdx)}),[u]),Nr("click",f),{showTooltip:s,stickyTooltips:h,handleUnStick:e=>{m((t=>t.filter((t=>t.id!==e))))},getTooltipProps:p,seriesFocus:(e,t)=>{const n=null!==t&&void 0!==t?t:-1;d((e=>({...e,seriesIdx:n})))},setCursor:e=>{var t;const n=null!==(t=e.cursor.idx)&&void 0!==t?t:-1;d((e=>({...e,dataIdx:n})))},resetTooltips:()=>{m([]),d({seriesIdx:-1,dataIdx:-1})}}},nh=e=>{let{u:t,id:n,title:a,dates:i,value:o,point:l,unit:s="",info:c,statsFormatted:u,isSticky:d,marker:h,onClose:m}=e;const p=(0,r.useRef)(null),[f,v]=(0,r.useState)({top:-999,left:-999}),[g,y]=(0,r.useState)(!1),[_,b]=(0,r.useState)(!1),w=(0,r.useCallback)((e=>{if(!g)return;const{clientX:t,clientY:n}=e;v({top:n,left:t})}),[g]);return(0,r.useEffect)((()=>{if(!p.current||!t)return;const{top:e,left:n}=l,r=parseFloat(t.over.style.left),a=parseFloat(t.over.style.top),{width:i,height:o}=t.over.getBoundingClientRect(),{width:s,height:c}=p.current.getBoundingClientRect(),u={top:e+a+10-(e+c>=o?c+20:0),left:n+r+10-(n+s>=i?s+20:0)};u.left<0&&(u.left=20),u.top<0&&(u.top=20),v(u)}),[t,o,l,p]),Nr("mousemove",w),Nr("mouseup",(()=>{y(!1)})),t?r.default.createPortal(Ct("div",{className:xr()({"vm-chart-tooltip":!0,"vm-chart-tooltip_sticky":d,"vm-chart-tooltip_moved":_}),ref:p,style:f,children:[Ct("div",{className:"vm-chart-tooltip-header",children:[a&&Ct("div",{className:"vm-chart-tooltip-header__title",children:a}),Ct("div",{className:"vm-chart-tooltip-header__date",children:i.map(((e,t)=>Ct("span",{children:e},t)))}),d&&Ct(xt.FK,{children:[Ct(aa,{className:"vm-chart-tooltip-header__drag",variant:"text",size:"small",startIcon:Ct(ir,{}),onMouseDown:e=>{b(!0),y(!0);const{clientX:t,clientY:n}=e;v({top:n,left:t})},ariaLabel:"drag the tooltip"}),Ct(aa,{className:"vm-chart-tooltip-header__close",variant:"text",size:"small",startIcon:Ct(Pn,{}),onClick:()=>{m&&m(n)},ariaLabel:"close the tooltip"})]})]}),Ct("div",{className:"vm-chart-tooltip-data",children:[h&&Ct("span",{className:"vm-chart-tooltip-data__marker",style:{background:h}}),Ct("p",{className:"vm-chart-tooltip-data__value",children:[Ct("b",{children:o}),s]})]}),u&&Ct("table",{className:"vm-chart-tooltip-stats",children:st.map(((e,t)=>Ct("div",{className:"vm-chart-tooltip-stats-row",children:[Ct("span",{className:"vm-chart-tooltip-stats-row__key",children:[e,":"]}),Ct("span",{className:"vm-chart-tooltip-stats-row__value",children:u[e]})]},t)))}),c&&Ct("p",{className:"vm-chart-tooltip__info",children:c})]}),t.root):null},rh=e=>{let{showTooltip:t,tooltipProps:n,stickyTooltips:a,handleUnStick:i}=e;return Ct(xt.FK,{children:[t&&n&&Ct(nh,{...n}),a.map((e=>(0,r.createElement)(nh,{...e,isSticky:!0,key:e.id,onClose:i})))]})},ah=e=>{let{data:t,series:n,metrics:a=[],period:i,yaxis:o,unit:l,setPeriod:s,layoutSize:c,height:u,isAnomalyView:d,spanGaps:h=!1}=e;const{isDarkTheme:m}=Nt(),p=(0,r.useRef)(null),[f,v]=(0,r.useState)(),{xRange:g,setPlotScale:y}=eh({period:i,setPeriod:s}),{onReadyChart:_,isPanning:b}=Gd(y);Xd({uPlotInst:f,xRange:g,setPlotScale:y});const{showTooltip:w,stickyTooltips:k,handleUnStick:x,getTooltipProps:S,seriesFocus:C,setCursor:E,resetTooltips:N}=th({u:f,metrics:a,series:n,unit:l,isAnomalyView:d}),A={...mu({width:c.width,height:u}),series:n,axes:qi([{},{scale:"1"}],l),scales:gu(o,g),hooks:{ready:[_],setSeries:[C],setCursor:[E],setSelect:[yu(y)],destroy:[pu]},bands:[]};return(0,r.useEffect)((()=>{if(N(),!p.current)return;f&&f.destroy();const e=new Bc(A,t,p.current);return v(e),e.destroy}),[p,m]),(0,r.useEffect)((()=>{f&&(f.setData(t),f.redraw())}),[t]),(0,r.useEffect)((()=>{f&&(su(f),function(e,t){let n=arguments.length>2&&void 0!==arguments[2]&&arguments[2];t.forEach((t=>{t.label&&(t.spanGaps=n),e.addSeries(t)}))}(f,n,h),((e,t)=>{if(e.delBand(),t.length<2)return;const n=t.map(((e,t)=>({...e,index:t}))),r=n.filter((e=>e.forecast===ut.yhatUpper)),a=n.filter((e=>e.forecast===ut.yhatLower)),i=r.map((e=>{const t=a.find((t=>t.forecastGroup===e.forecastGroup));return t?{series:[e.index,t.index],fill:_u(ut.yhatUpper)}:null})).filter((e=>null!==e));i.length&&i.forEach((t=>{e.addBand(t)}))})(f,n),f.redraw())}),[n,h]),(0,r.useEffect)((()=>{f&&(Object.keys(o.limits.range).forEach((e=>{f.scales[e]&&(f.scales[e].range=function(t){return vu(t,arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,e,o)})})),f.redraw())}),[o]),(0,r.useEffect)((()=>{f&&(f.scales.x.range=()=>fu(g),f.redraw())}),[g]),(0,r.useEffect)((()=>{f&&(f.setSize({width:c.width||400,height:u||500}),f.redraw())}),[u,c]),Ct("div",{className:xr()({"vm-line-chart":!0,"vm-line-chart_panning":b}),style:{minWidth:"".concat(c.width||400,"px"),minHeight:"".concat(u||500,"px")},children:[Ct("div",{className:"vm-line-chart__u-plot",ref:p}),Ct(rh,{showTooltip:w,tooltipProps:S(),stickyTooltips:k,handleUnStick:x})]})},ih=e=>{let{legend:t,onChange:n,isHeatmap:a,isAnomalyView:i}=e;const o=Cd(),l=(0,r.useMemo)((()=>{const e=(e=>{const t=Object.keys(e.freeFormFields).filter((e=>"__name__"!==e));return t.map((t=>{const n="".concat(t,"=").concat(JSON.stringify(e.freeFormFields[t]));return{id:"".concat(e.label,".").concat(n),freeField:n,key:t}}))})(t);return a?e.filter((e=>"vmrange"!==e.key)):e}),[t,a]),s=t.statsFormatted,c=Object.values(s).some((e=>e)),u=e=>t=>{t.stopPropagation(),(async e=>{await o(e,"".concat(e," has been copied"))})(e)};return Ct("div",{className:xr()({"vm-legend-item":!0,"vm-legend-row":!0,"vm-legend-item_hide":!t.checked&&!a,"vm-legend-item_static":a}),onClick:(e=>t=>{n&&n(e,t.ctrlKey||t.metaKey)})(t),children:[!i&&!a&&Ct("div",{className:"vm-legend-item__marker",style:{backgroundColor:t.color}}),Ct("div",{className:"vm-legend-item-info",children:Ct("span",{className:"vm-legend-item-info__label",children:[t.freeFormFields.__name__,!!l.length&&Ct(xt.FK,{children:"{"}),l.map(((e,t)=>Ct("span",{className:"vm-legend-item-info__free-fields",onClick:u(e.freeField),title:"copy to clipboard",children:[e.freeField,t+1Ct("div",{className:"vm-legend-item-stats-row",children:[Ct("span",{className:"vm-legend-item-stats-row__key",children:[e,":"]}),Ct("span",{className:"vm-legend-item-stats-row__value",children:s[e]})]},t)))})]})},oh=e=>{let{labels:t,query:n,isAnomalyView:a,onChange:i}=e;const o=(0,r.useMemo)((()=>Array.from(new Set(t.map((e=>e.group))))),[t]),l=o.length>1;return Ct(xt.FK,{children:Ct("div",{className:"vm-legend",children:o.map((e=>Ct("div",{className:"vm-legend-group",children:Ct(pi,{defaultExpanded:!0,title:Ct("div",{className:"vm-legend-group-title",children:[l&&Ct("span",{className:"vm-legend-group-title__count",children:["Query ",e,": "]}),Ct("span",{className:"vm-legend-group-title__query",children:n[e-1]})]}),children:Ct("div",{children:t.filter((t=>t.group===e)).sort(((e,t)=>(t.median||0)-(e.median||0))).map((e=>Ct(ih,{legend:e,isAnomalyView:a,onChange:i},e.label)))})})},e)))})})},lh=e=>{var t;let{min:n,max:a,legendValue:i,series:o}=e;const[l,s]=(0,r.useState)(0),[c,u]=(0,r.useState)(""),[d,h]=(0,r.useState)(""),[m,p]=(0,r.useState)(""),f=(0,r.useMemo)((()=>parseFloat(String((null===i||void 0===i?void 0:i.value)||0).replace("%",""))),[i]);return(0,r.useEffect)((()=>{s(f?(f-n)/(a-n)*100:0),u(f?"".concat(f,"%"):""),h("".concat(n,"%")),p("".concat(a,"%"))}),[f,n,a]),Ct("div",{className:"vm-legend-heatmap__wrapper",children:[Ct("div",{className:"vm-legend-heatmap",children:[Ct("div",{className:"vm-legend-heatmap-gradient",style:{background:"linear-gradient(to right, ".concat(Jc.join(", "),")")},children:!!f&&Ct("div",{className:"vm-legend-heatmap-gradient__value",style:{left:"".concat(l,"%")},children:Ct("span",{children:c})})}),Ct("div",{className:"vm-legend-heatmap__value",children:d}),Ct("div",{className:"vm-legend-heatmap__value",children:m})]}),o[1]&&Ct(ih,{legend:o[1],isHeatmap:!0},null===(t=o[1])||void 0===t?void 0:t.label)]})},sh=e=>{let{u:t,metrics:n,unit:a}=e;const[o,l]=(0,r.useState)({left:0,top:0}),[s,c]=(0,r.useState)([]),u=(0,r.useCallback)((()=>{var e;const{left:r,top:l}=o,s=it()(t,["data",1,0],[])||[],c=t?t.posToVal(r,"x"):0,u=t?t.posToVal(l,"y"):0,d=s.findIndex(((e,t)=>c>=e&&ce[0]===h))||[],v=s[d],g=i()(1e3*v).tz().format(Ot),y=i()(1e3*p).tz().format(Ot),_=(null===m||void 0===m||null===(e=m.metric)||void 0===e?void 0:e.vmrange)||"";return{unit:a,point:o,u:t,id:"".concat(_,"_").concat(g),dates:[g,y],value:"".concat(f,"%"),info:_,show:+f>0}}),[t,o,n,a]),d=(0,r.useCallback)((()=>{const e=u();e.show&&(s.find((t=>t.id===e.id))||c((t=>[...t,e])))}),[u,s]);return Nr("click",d),{stickyTooltips:s,handleUnStick:e=>{c((t=>t.filter((t=>t.id!==e))))},getTooltipProps:u,setCursor:e=>{const t=e.cursor.left||0,n=e.cursor.top||0;l({left:t,top:n})},resetTooltips:()=>{c([]),l({left:0,top:0})}}},ch=e=>{let{data:t,metrics:n=[],period:a,unit:i,setPeriod:o,layoutSize:l,height:s,onChangeLegend:c}=e;const{isDarkTheme:u}=Nt(),d=(0,r.useRef)(null),[h,m]=(0,r.useState)(),{xRange:p,setPlotScale:f}=eh({period:a,setPeriod:o}),{onReadyChart:v,isPanning:g}=Gd(f);Xd({uPlotInst:h,xRange:p,setPlotScale:f});const{stickyTooltips:y,handleUnStick:_,getTooltipProps:b,setCursor:w,resetTooltips:k}=sh({u:h,metrics:n,unit:i}),x=(0,r.useMemo)((()=>b()),[b]),S={...mu({width:l.width,height:s}),mode:2,series:[{},{paths:Xc(),facets:[{scale:"x",auto:!0,sorted:1},{scale:"y",auto:!0}]}],axes:(()=>{const e=qi([{}],i);return[...e,{scale:"y",stroke:e[0].stroke,font:e[0].font,size:Wi,splits:n.map(((e,t)=>t)),values:n.map((e=>e.metric.vmrange))}]})(),scales:{x:{time:!0},y:{log:2,time:!1,range:(e,t,n)=>[t-1,n+1]}},hooks:{ready:[v],setCursor:[w],setSelect:[yu(f)],destroy:[pu]}};return(0,r.useEffect)((()=>{k();const e=null===t[0]&&Array.isArray(t[1]);if(!d.current||!e)return;const n=new Bc(S,t,d.current);return m(n),n.destroy}),[d,t,u]),(0,r.useEffect)((()=>{h&&(h.setSize({width:l.width||400,height:s||500}),h.redraw())}),[s,l]),(0,r.useEffect)((()=>{c(x)}),[x]),Ct("div",{className:xr()({"vm-line-chart":!0,"vm-line-chart_panning":g}),style:{minWidth:"".concat(l.width||400,"px"),minHeight:"".concat(s||500,"px")},children:[Ct("div",{className:"vm-line-chart__u-plot",ref:d}),Ct(rh,{showTooltip:!!x.show,tooltipProps:x,stickyTooltips:y,handleUnStick:_})]})},uh=()=>{const[e,t]=(0,r.useState)(null),[n,a]=(0,r.useState)({width:0,height:0}),i=(0,r.useCallback)((()=>{a({width:(null===e||void 0===e?void 0:e.offsetWidth)||0,height:(null===e||void 0===e?void 0:e.offsetHeight)||0})}),[null===e||void 0===e?void 0:e.offsetHeight,null===e||void 0===e?void 0:e.offsetWidth]);return Nr("resize",i),Er(i,[null===e||void 0===e?void 0:e.offsetHeight,null===e||void 0===e?void 0:e.offsetWidth]),[t,n]},dh={[ut.yhat]:"yhat",[ut.yhatLower]:"yhat_upper - yhat_lower",[ut.yhatUpper]:"yhat_upper - yhat_lower",[ut.anomaly]:"anomalies",[ut.training]:"training data",[ut.actual]:"y"},hh=e=>{let{series:t}=e;const n=(0,r.useMemo)((()=>{const e=t.reduce(((e,t)=>{const n=Object.prototype.hasOwnProperty.call(t,"forecast"),r=t.forecast!==ut.yhatUpper,a=!e.find((e=>e.forecast===t.forecast));return n&&a&&r&&e.push(t),e}),[]),n={...e[0],forecast:ut.training,color:Wc[ut.training]};return e.splice(1,0,n),e.map((e=>({...e,color:"string"===typeof e.stroke?e.stroke:Wc[e.forecast||ut.actual]})))}),[t]);return Ct(xt.FK,{children:Ct("div",{className:"vm-legend-anomaly",children:n.filter((e=>e.forecast!==ut.training)).map(((e,t)=>{var n;return Ct("div",{className:"vm-legend-anomaly-item",children:[Ct("svg",{children:e.forecast===ut.anomaly?Ct("circle",{cx:"15",cy:"7",r:"4",fill:e.color,stroke:e.color,strokeWidth:"1.4"}):Ct("line",{x1:"0",y1:"7",x2:"30",y2:"7",stroke:e.color,strokeWidth:e.width||1,strokeDasharray:null===(n=e.dash)||void 0===n?void 0:n.join(",")})}),Ct("div",{className:"vm-legend-anomaly-item__title",children:dh[e.forecast||ut.actual]})]},"".concat(t,"_").concat(e.forecast))}))})})},mh=e=>{let{data:t=[],period:n,customStep:a,query:i,yaxis:o,unit:l,showLegend:s=!0,setYaxisLimits:c,setPeriod:u,alias:d=[],fullWidth:h=!0,height:m,isHistogram:p,isAnomalyView:f,spanGaps:v}=e;const{isMobile:g}=Wr(),{timezone:y}=vn(),_=(0,r.useMemo)((()=>a||n.step||"1s"),[n.step,a]),b=(0,r.useMemo)((()=>nu(t,p)),[p,t]),[w,k]=(0,r.useState)([[]]),[x,S]=(0,r.useState)([]),[C,E]=(0,r.useState)([]),[N,A]=(0,r.useState)([]),[M,T]=(0,r.useState)(null),L=(0,r.useMemo)((()=>au(b,N,d,f)),[b,N,d,f]),O=e=>{const t=((e,t)=>{const n={},r=Object.values(e).flat(),a=Fi(r)||0,i=zi(r)||1;return n[1]=t?Yi(a,i):[a,i],n})(e,!p);c(t)},P=e=>{if(!f)return e;const t=function(e,t){const n=e.reduce(((e,n)=>{const r=t.map((e=>"".concat(e,": ").concat(n[e]||"-"))).join("|");return(e[r]=e[r]||[]).push(n),e}),{});return Object.entries(n).map((e=>{let[t,n]=e;return{keys:t.split("|"),values:n}}))}(e,["group","label"]);return t.map((e=>{const t=e.values[0];return{...t,freeFormFields:{...t.freeFormFields,__name__:""}}}))};(0,r.useEffect)((()=>{const e=[],t={},r=[],a=[{}];null===b||void 0===b||b.forEach(((n,i)=>{const o=L(n,i);a.push(o),r.push(ou(o,n.group));const l=t[n.group]||[];for(const t of n.values)e.push(t[0]),l.push(Zc(t[1]));t[n.group]=l}));const i=((e,t,n)=>{const r=Wt(t)||1,a=Array.from(new Set(e)).sort(((e,t)=>e-t));let i=n.start;const o=Ut(n.end+r);let l=0;const s=[];for(;i<=o;){for(;l=a.length||a[l]>i)&&s.push(i)}for(;s.length<2;)s.push(i),i=Ut(i+r);return s})(e,_,n),o=b.map((e=>{const t=[],n=e.values,r=n.length;let a=0;for(const u of i){for(;anull!==e)),l=Math.abs((e=>{let t=e[0],n=1;for(let r=1;r1e10*c&&!f?t.map((()=>l)):t}));o.unshift(i),O(t);const l=p?(e=>{const t=e.slice(1,e.length),n=[],r=[];t.forEach(((e,n)=>{e.forEach(((e,a)=>{const i=a*t.length+n;r[i]=e}))})),e[0].forEach((e=>{const r=new Array(t.length).fill(e);n.push(...r)}));const a=new Array(n.length).fill(0).map(((e,n)=>n%t.length));return[null,[n,a,r]]})(o):o;k(l),S(a);const s=P(r);E(s),f&&A(s.map((e=>e.label||"")).slice(1))}),[b,y,p]),(0,r.useEffect)((()=>{const e=[],t=[{}];null===b||void 0===b||b.forEach(((n,r)=>{const a=L(n,r);t.push(a),e.push(ou(a,n.group))})),S(t),E(P(e))}),[N]);const[I,R]=uh();return Ct("div",{className:xr()({"vm-graph-view":!0,"vm-graph-view_full-width":h,"vm-graph-view_full-width_mobile":h&&g}),ref:I,children:[!p&&Ct(ah,{data:w,series:x,metrics:b,period:n,yaxis:o,unit:l,setPeriod:u,layoutSize:R,height:m,isAnomalyView:f,spanGaps:v}),p&&Ct(ch,{data:w,metrics:b,period:n,unit:l,setPeriod:u,layoutSize:R,height:m,onChangeLegend:T}),f&&s&&Ct(hh,{series:x}),!p&&s&&Ct(oh,{labels:C,query:i,isAnomalyView:f,onChange:(e,t)=>{A((e=>{let{hideSeries:t,legend:n,metaKey:r,series:a,isAnomalyView:i}=e;const{label:o}=n,l=lu(o,t),s=a.map((e=>e.label||""));return i?s.filter((e=>e!==o)):r?l?t.filter((e=>e!==o)):[...t,o]:t.length?l?[...s.filter((e=>e!==o))]:[]:[...s.filter((e=>e!==o))]})({hideSeries:N,legend:e,metaKey:t,series:x,isAnomalyView:f}))}}),p&&s&&Ct(lh,{series:x,min:o.limits.range[1][0]||0,max:o.limits.range[1][1]||0,legendValue:M})]})},ph=e=>{let{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a}=e;const{isMobile:i}=Wr(),o=(0,r.useMemo)((()=>Object.keys(t.limits.range)),[t.limits.range]),l=(0,r.useCallback)(Oi()(((e,r,a)=>{const i=t.limits.range;i[r][a]=+e,i[r][0]===i[r][1]||i[r][0]>i[r][1]||n(i)}),500),[t.limits.range]),s=(e,t)=>n=>{l(n,e,t)};return Ct("div",{className:xr()({"vm-axes-limits":!0,"vm-axes-limits_mobile":i}),children:[Ct(wd,{value:t.limits.enable,onChange:a,label:"Fix the limits for y-axis",fullWidth:i}),Ct("div",{className:"vm-axes-limits-list",children:o.map((e=>Ct("div",{className:"vm-axes-limits-list__inputs",children:[Ct($a,{label:"Min ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][0],onChange:s(e,0)}),Ct($a,{label:"Max ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][1],onChange:s(e,1)})]},e)))})]})},fh=e=>{let{spanGaps:t,onChange:n}=e;const{isMobile:r}=Wr();return Ct("div",{children:Ct(wd,{value:t,onChange:n,label:"Connect null values",fullWidth:r})})},vh="Graph settings",gh=e=>{let{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a,spanGaps:i}=e;const o=(0,r.useRef)(null),l=(0,r.useRef)(null),{value:s,toggle:c,setFalse:u}=oa(!1);return Ct("div",{className:"vm-graph-settings",children:[Ct(fa,{title:vh,children:Ct("div",{ref:l,children:Ct(aa,{variant:"text",startIcon:Ct(On,{}),onClick:c,ariaLabel:"settings"})})}),Ct(ia,{open:s,buttonRef:l,placement:"bottom-right",onClose:u,title:vh,children:Ct("div",{className:"vm-graph-settings-popper",ref:o,children:Ct("div",{className:"vm-graph-settings-popper__body",children:[Ct(ph,{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a}),Ct(fh,{spanGaps:i.value,onChange:i.onChange})]})})})]})},yh=e=>{let{isHistogram:t,graphData:n,controlsRef:a,isAnomalyView:i}=e;const{isMobile:o}=Wr(),{customStep:l,yaxis:s,spanGaps:c}=Ur(),{period:u}=vn(),{query:d}=En(),h=gn(),m=Br(),p=e=>{m({type:"SET_YAXIS_LIMITS",payload:e})},f=Ct("div",{className:"vm-custom-panel-body-header__graph-controls",children:[Ct(ba,{}),Ct(gh,{yaxis:s,setYaxisLimits:p,toggleEnableLimits:()=>{m({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})},spanGaps:{value:c,onChange:e=>{m({type:"SET_SPAN_GAPS",payload:e})}}})]});return Ct(xt.FK,{children:[a.current&&(0,r.createPortal)(f,a.current),Ct(mh,{data:n,period:u,customStep:l,query:d,yaxis:s,setYaxisLimits:p,setPeriod:e=>{let{from:t,to:n}=e;h({type:"SET_PERIOD",payload:{from:t,to:n}})},height:o?.5*window.innerHeight:500,isHistogram:t,isAnomalyView:i,spanGaps:c})]})},_h=e=>{let{data:t}=e;const n=Cd(),a=(0,r.useMemo)((()=>JSON.stringify(t,null,2)),[t]);return Ct("div",{className:"vm-json-view",children:[Ct("div",{className:"vm-json-view__copy",children:Ct(aa,{variant:"outlined",onClick:async()=>{await n(a,"Formatted JSON has been copied")},children:"Copy JSON"})}),Ct("pre",{className:"vm-json-view__code",children:Ct("code",{children:a})})]})},bh=e=>{const t={};return e.forEach((e=>Object.entries(e.metric).forEach((e=>t[e[0]]?t[e[0]].options.add(e[1]):t[e[0]]={options:new Set([e[1]])})))),Object.entries(t).map((e=>({key:e[0],variations:e[1].options.size}))).sort(((e,t)=>e.variations-t.variations))},wh=(e,t)=>(0,r.useMemo)((()=>{const n=bh(e);return t?n.filter((e=>t.includes(e.key))):n}),[e,t]),kh=e=>{let{data:t,displayColumns:n}=e;const a=Cd(),{isMobile:i}=Wr(),{tableCompact:o}=Fr(),l=(0,r.useRef)(null),[s,c]=(0,r.useState)(""),[u,d]=(0,r.useState)("asc"),h=o?wh([{group:0,metric:{Data:"Data"}}],["Data"]):wh(t,n),m=e=>{const{__name__:t,...n}=e;return t||Object.keys(n).length?"".concat(t," ").concat(JSON.stringify(n)):""},p=new Set(null===t||void 0===t?void 0:t.map((e=>e.group))).size>1,f=(0,r.useMemo)((()=>{const e=null===t||void 0===t?void 0:t.map((e=>({metadata:h.map((t=>o?Qc(e,"",p):e.metric[t.key]||"-")),value:e.value?e.value[1]:"-",values:e.values?e.values.map((e=>{let[t,n]=e;return"".concat(n," @").concat(t)})):[],copyValue:m(e.metric)}))),n="Value"===s,r=h.findIndex((e=>e.key===s));return n||-1!==r?e.sort(((e,t)=>{const a=n?Number(e.value):e.metadata[r],i=n?Number(t.value):t.metadata[r];return("asc"===u?ai)?-1:1})):e}),[h,t,s,u,o]),v=(0,r.useMemo)((()=>f.some((e=>e.copyValue))),[f]),g=e=>()=>{(e=>{d((t=>"asc"===t&&s===e?"desc":"asc")),c(e)})(e)};return f.length?Ct("div",{className:xr()({"vm-table-view":!0,"vm-table-view_mobile":i}),children:Ct("table",{className:"vm-table",ref:l,children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[h.map(((e,t)=>Ct("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:g(e.key),children:Ct("div",{className:"vm-table-cell__content",children:[e.key,Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":s===e.key,"vm-table__sort-icon_desc":"desc"===u&&s===e.key}),children:Ct(Hn,{})})]})},t))),Ct("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_right vm-table-cell_sort",onClick:g("Value"),children:Ct("div",{className:"vm-table-cell__content",children:[Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":"Value"===s,"vm-table__sort-icon_desc":"desc"===u}),children:Ct(Hn,{})}),"Value"]})}),v&&Ct("td",{className:"vm-table-cell vm-table-cell_header"})]})}),Ct("tbody",{className:"vm-table-body",children:f.map(((e,t)=>{return Ct("tr",{className:"vm-table__row",children:[e.metadata.map(((e,n)=>Ct("td",{className:xr()({"vm-table-cell vm-table-cell_no-wrap":!0,"vm-table-cell_gray":f[t-1]&&f[t-1].metadata[n]===e}),children:e},n))),Ct("td",{className:"vm-table-cell vm-table-cell_right vm-table-cell_no-wrap",children:e.values.length?e.values.map((e=>Ct("p",{children:e},e))):e.value}),v&&Ct("td",{className:"vm-table-cell vm-table-cell_right",children:e.copyValue&&Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:"Copy row",children:Ct(aa,{variant:"text",color:"gray",size:"small",startIcon:Ct(ar,{}),onClick:(n=e.copyValue,async()=>{await a(n,"Row has been copied")}),ariaLabel:"copy row"})})})})]},t);var n}))})]})}):Ct(Qr,{variant:"warning",children:"No data to show"})},xh=e=>{let{checked:t=!1,disabled:n=!1,label:r,color:a="secondary",onChange:i}=e;return Ct("div",{className:xr()({"vm-checkbox":!0,"vm-checkbox_disabled":n,"vm-checkbox_active":t,["vm-checkbox_".concat(a,"_active")]:t,["vm-checkbox_".concat(a)]:a}),onClick:()=>{n||i(!t)},children:[Ct("div",{className:"vm-checkbox-track",children:Ct("div",{className:"vm-checkbox-track__thumb",children:Ct(er,{})})}),r&&Ct("span",{className:"vm-checkbox__label",children:r})]})},Sh="Table settings",Ch=e=>{let{columns:t,defaultColumns:n=[],tableCompact:a,onChangeColumns:i,toggleTableCompact:o}=e;const{isMobile:l}=Wr(),s=(0,r.useRef)(null),{value:c,toggle:u,setFalse:d}=oa(!1),h=(0,r.useMemo)((()=>!t.length),[t]),m=e=>()=>{(e=>{i(n.includes(e)?n.filter((t=>t!==e)):[...n,e])})(e)};return(0,r.useEffect)((()=>{Sd(t,n)||i(t)}),[t]),Ct("div",{className:"vm-table-settings",children:[Ct(fa,{title:Sh,children:Ct("div",{ref:s,children:Ct(aa,{variant:"text",startIcon:Ct(On,{}),onClick:u,disabled:h,ariaLabel:"table settings"})})}),Ct(ia,{open:c,onClose:d,placement:"bottom-right",buttonRef:s,title:Sh,children:Ct("div",{className:xr()({"vm-table-settings-popper":!0,"vm-table-settings-popper_mobile":l}),children:[Ct("div",{className:"vm-table-settings-popper-list vm-table-settings-popper-list_first",children:Ct(wd,{label:"Compact view",value:a,onChange:o})}),Ct("div",{className:"vm-table-settings-popper-list",children:[Ct("div",{className:"vm-table-settings-popper-list-header",children:[Ct("h3",{className:"vm-table-settings-popper-list-header__title",children:"Display columns"}),Ct(fa,{title:"Reset to default",children:Ct(aa,{color:"primary",variant:"text",size:"small",onClick:()=>{d(),i(t)},startIcon:Ct(In,{}),ariaLabel:"reset columns"})})]}),t.map((e=>Ct("div",{className:"vm-table-settings-popper-list__item",children:Ct(xh,{checked:n.includes(e),onChange:m(e),label:e,disabled:a})},e)))]})]})})]})},Eh=e=>{let{liveData:t,controlsRef:n}=e;const{tableCompact:a}=Fr(),i=jr(),[o,l]=(0,r.useState)(),s=(0,r.useMemo)((()=>bh(t||[]).map((e=>e.key))),[t]),c=Ct(Ch,{columns:s,defaultColumns:o,onChangeColumns:l,tableCompact:a,toggleTableCompact:()=>{i({type:"TOGGLE_TABLE_COMPACT"})}});return Ct(xt.FK,{children:[n.current&&(0,r.createPortal)(c,n.current),Ct(kh,{data:t,displayColumns:o})]})},Nh=e=>{let{graphData:t,liveData:n,isHistogram:r,displayType:a,controlsRef:i}=e;return a===dt.code&&n?Ct(_h,{data:n}):a===dt.table&&n?Ct(Eh,{liveData:n,controlsRef:i}):a===dt.chart&&t?Ct(yh,{graphData:t,isHistogram:r,controlsRef:i}):null},Ah=[Ct(xt.FK,{children:[Ct("p",{children:"Filename - specify the name for your report file."}),Ct("p",{children:["Default format: ",Ct("code",{children:["vmui_report_$",It,".json"]}),"."]}),Ct("p",{children:"This name will be used when saving your report on your device."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Comment (optional) - add a comment to your report."}),Ct("p",{children:"This can be any additional information that will be useful when reviewing the report later."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Query trace - enable this option to include a query trace in your report."}),Ct("p",{children:"This will assist in analyzing and diagnosing the query processing."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Generate Report - click this button to generate and save your report. "}),Ct("p",{children:["After creation, the report can be downloaded and examined on the ",Ct(Ie,{to:Ye.queryAnalyzer,target:"_blank",rel:"noreferrer",className:"vm-link vm-link_underlined",children:qe[Ye.queryAnalyzer].title})," page."]})]})],Mh=()=>"vmui_report_".concat(i()().utc().format(It)),Th=e=>{let{fetchUrl:t}=e;const{query:n}=En(),[a,i]=(0,r.useState)(Mh()),[o,l]=(0,r.useState)(""),[s,c]=(0,r.useState)(!0),[u,d]=(0,r.useState)(),[h,m]=(0,r.useState)(!1),p=(0,r.useRef)(null),f=(0,r.useRef)(null),v=(0,r.useRef)(null),g=(0,r.useRef)(null),y=[p,f,v,g],[_,b]=(0,r.useState)(0),{value:w,toggle:k,setFalse:x}=oa(!1),{value:S,toggle:C,setFalse:E}=oa(!1),N=(0,r.useMemo)((()=>{if(t)return t.map(((e,t)=>{const n=new URL(e);return s?n.searchParams.set("trace","1"):n.searchParams.delete("trace"),{id:t,url:n}}))}),[t,s]),A=(0,r.useCallback)((e=>{const t=JSON.stringify(e,null,2),n=new Blob([t],{type:"application/json"}),r=URL.createObjectURL(n),i=document.createElement("a");i.href=r,i.download="".concat(a||Mh(),".json"),document.body.appendChild(i),i.click(),document.body.removeChild(i),URL.revokeObjectURL(r),x()}),[a]),M=(0,r.useCallback)((async()=>{if(N){d(""),m(!0);try{const e=[];for await(const{url:t,id:n}of N){const r=await fetch(t),a=await r.json();if(r.ok)a.vmui={id:n,comment:o,params:rt().parse(new URL(t).search.replace(/^\?/,""))},e.push(a);else{const e=a.errorType?"".concat(a.errorType,"\r\n"):"";d("".concat(e).concat((null===a||void 0===a?void 0:a.error)||(null===a||void 0===a?void 0:a.message)||"unknown error"))}}e.length&&A(e)}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&d("".concat(ap.name,": ").concat(ap.message))}finally{m(!1)}}else d(ht.validQuery)}),[N,o,A,n]),T=e=>()=>{b((t=>t+e))};return(0,r.useEffect)((()=>{d(""),i(Mh()),l("")}),[w]),(0,r.useEffect)((()=>{b(0)}),[S]),Ct(xt.FK,{children:[Ct(fa,{title:"Export query",children:Ct(aa,{variant:"text",startIcon:Ct(wr,{}),onClick:k,ariaLabel:"export query"})}),w&&Ct(pa,{title:"Export query",onClose:x,isOpen:w,children:Ct("div",{className:"vm-download-report",children:[Ct("div",{className:"vm-download-report-settings",children:[Ct("div",{ref:p,children:Ct($a,{label:"Filename",value:a,onChange:i})}),Ct("div",{ref:f,children:Ct($a,{type:"textarea",label:"Comment",value:o,onChange:l})}),Ct("div",{ref:v,children:Ct(xh,{checked:s,onChange:c,label:"Include query trace"})})]}),u&&Ct(Qr,{variant:"error",children:u}),Ct("div",{className:"vm-download-report__buttons",children:[Ct(aa,{variant:"text",onClick:C,children:"Help"}),Ct("div",{ref:g,children:Ct(aa,{onClick:M,disabled:h,children:h?"Loading data...":"Generate Report"})})]}),Ct(ia,{open:S,buttonRef:y[_],placement:"top-left",variant:"dark",onClose:E,children:Ct("div",{className:"vm-download-report-helper",children:[Ct("div",{className:"vm-download-report-helper__description",children:Ah[_]}),Ct("div",{className:"vm-download-report-helper__buttons",children:[0!==_&&Ct(aa,{onClick:T(-1),size:"small",color:"white",children:"Prev"}),Ct(aa,{onClick:_===y.length-1?E:T(1),size:"small",color:"white",variant:"text",children:_===y.length-1?"Close":"Next"})]})]})})]})})]})},Lh=()=>{(()=>{const{tenantId:e}=Nt(),{displayType:t}=Fr(),{query:n}=En(),{duration:a,relativeTime:i,period:{date:o,step:l}}=vn(),{customStep:s}=Ur(),[,c]=je(),u=()=>{const r={};n.forEach(((n,c)=>{var u;const d="g".concat(c);r["".concat(d,".expr")]=n,r["".concat(d,".range_input")]=a,r["".concat(d,".end_input")]=o,r["".concat(d,".tab")]=(null===(u=Tr.find((e=>e.value===t)))||void 0===u?void 0:u.prometheusCode)||0,r["".concat(d,".relative_time")]=i,r["".concat(d,".tenantID")]=e,l!==s&&s&&(r["".concat(d,".step_input")]=s)})),c(zd(r))};(0,r.useEffect)(u,[e,t,n,a,i,o,l,s]),(0,r.useEffect)(u,[])})();const{isMobile:e}=Wr(),{displayType:t}=Fr(),{query:n}=En(),{customStep:a}=Ur(),i=Br(),[o,l]=(0,r.useState)([]),[s,c]=(0,r.useState)(!n[0]),[u,d]=(0,r.useState)(!1),h=(0,r.useRef)(null),{fetchUrl:m,isLoading:p,liveData:f,graphData:v,error:g,queryErrors:y,setQueryErrors:_,queryStats:b,warning:w,traces:k,isHistogram:x}=Rd({visible:!0,customStep:a,hideQuery:o,showAllSeries:u}),S=!(null!==f&&void 0!==f&&f.length)&&t!==dt.chart,C=!s&&g;return Nr("popstate",(()=>window.location.reload())),(0,r.useEffect)((()=>{i({type:"SET_IS_HISTOGRAM",payload:x})}),[v]),Ct("div",{className:xr()({"vm-custom-panel":!0,"vm-custom-panel_mobile":e}),children:[Ct(Ld,{queryErrors:s?[]:y,setQueryErrors:_,setHideError:c,stats:b,onHideQuery:e=>{l(e)},onRunQuery:()=>{c(!1)}}),Ct(Wd,{traces:k,displayType:t}),p&&Ct(Dd,{}),C&&Ct(Qr,{variant:"error",children:g}),S&&Ct(Qr,{variant:"info",children:Ct(Hd,{})}),w&&Ct(Kd,{warning:w,query:n,onChange:d}),Ct("div",{className:xr()({"vm-custom-panel-body":!0,"vm-custom-panel-body_mobile":e,"vm-block":!0,"vm-block_mobile":e}),children:[Ct("div",{className:"vm-custom-panel-body-header",ref:h,children:[Ct("div",{className:"vm-custom-panel-body-header__tabs",children:Ct(Lr,{})}),(v||f)&&Ct(Th,{fetchUrl:m})]}),Ct(Nh,{graphData:v,liveData:f,isHistogram:x,displayType:t,controlsRef:h})]})]})},Oh=e=>{let{title:t,description:n,unit:a,expr:i,showLegend:o,filename:l,alias:s}=e;const{isMobile:c}=Wr(),{period:u}=vn(),{customStep:d}=Ur(),h=gn(),m=(0,r.useRef)(null),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)(!1),[y,_]=(0,r.useState)({limits:{enable:!1,range:{1:[0,0]}}}),b=(0,r.useMemo)((()=>Array.isArray(i)&&i.every((e=>e))),[i]),{isLoading:w,graphData:k,error:x,warning:S}=Rd({predefinedQuery:b?i:[],display:dt.chart,visible:p,customStep:d}),C=e=>{const t={...y};t.limits.range=e,_(t)};if((0,r.useEffect)((()=>{const e=new IntersectionObserver((e=>{e.forEach((e=>f(e.isIntersecting)))}),{threshold:.1});return m.current&&e.observe(m.current),()=>{m.current&&e.unobserve(m.current)}}),[m]),!b)return Ct(Qr,{variant:"error",children:[Ct("code",{children:'"expr"'})," not found. Check the configuration file ",Ct("b",{children:l}),"."]});const E=()=>Ct("div",{className:"vm-predefined-panel-header__description vm-default-styles",children:[n&&Ct(xt.FK,{children:[Ct("div",{children:[Ct("span",{children:"Description:"}),Ct("div",{dangerouslySetInnerHTML:{__html:hd.parse(n)}})]}),Ct("hr",{})]}),Ct("div",{children:[Ct("span",{children:"Queries:"}),Ct("div",{children:i.map(((e,t)=>Ct("div",{children:e},"".concat(t,"_").concat(e))))})]})]});return Ct("div",{className:"vm-predefined-panel",ref:m,children:[Ct("div",{className:"vm-predefined-panel-header",children:[Ct(fa,{title:Ct(E,{}),children:Ct("div",{className:"vm-predefined-panel-header__info",children:Ct(Rn,{})})}),Ct("h3",{className:"vm-predefined-panel-header__title",children:t||""}),Ct(gh,{yaxis:y,setYaxisLimits:C,toggleEnableLimits:()=>{const e={...y};e.limits.enable=!e.limits.enable,_(e)},spanGaps:{value:v,onChange:g}})]}),Ct("div",{className:"vm-predefined-panel-body",children:[w&&Ct(Dd,{}),x&&Ct(Qr,{variant:"error",children:x}),S&&Ct(Qr,{variant:"warning",children:S}),k&&Ct(mh,{data:k,period:u,customStep:d,query:i,yaxis:y,unit:a,alias:s,showLegend:o,setYaxisLimits:C,setPeriod:e=>{let{from:t,to:n}=e;h({type:"SET_PERIOD",payload:{from:t,to:n}})},fullWidth:!1,height:c?.5*window.innerHeight:500,spanGaps:v})]})]})},Ph=e=>{let{index:t,title:n,panels:a,filename:i}=e;const o=Ar(),l=(0,r.useMemo)((()=>o.width/12),[o]),[s,c]=(0,r.useState)(!t),[u,d]=(0,r.useState)([]);(0,r.useEffect)((()=>{d(a&&a.map((e=>e.width||12)))}),[a]);const[h,m]=(0,r.useState)({start:0,target:0,enable:!1}),p=(0,r.useCallback)((e=>{if(!h.enable)return;const{start:t}=h,n=Math.ceil((t-e.clientX)/l);if(Math.abs(n)>=12)return;const r=u.map(((e,t)=>e-(t===h.target?n:0)));d(r)}),[h,l]),f=(0,r.useCallback)((()=>{m({...h,enable:!1})}),[h]),v=e=>t=>{((e,t)=>{m({start:e.clientX,target:t,enable:!0})})(t,e)};Nr("mousemove",p),Nr("mouseup",f);return Ct("div",{className:"vm-predefined-dashboard",children:Ct(pi,{defaultExpanded:s,onChange:e=>c(e),title:Ct((()=>Ct("div",{className:xr()({"vm-predefined-dashboard-header":!0,"vm-predefined-dashboard-header_open":s}),children:[(n||i)&&Ct("span",{className:"vm-predefined-dashboard-header__title",children:n||"".concat(t+1,". ").concat(i)}),a&&Ct("span",{className:"vm-predefined-dashboard-header__count",children:["(",a.length," panels)"]})]})),{}),children:Ct("div",{className:"vm-predefined-dashboard-panels",children:Array.isArray(a)&&a.length?a.map(((e,t)=>Ct("div",{className:"vm-predefined-dashboard-panels-panel vm-block vm-block_empty-padding",style:{gridColumn:"span ".concat(u[t])},children:[Ct(Oh,{title:e.title,description:e.description,unit:e.unit,expr:e.expr,alias:e.alias,filename:i,showLegend:e.showLegend}),Ct("button",{className:"vm-predefined-dashboard-panels-panel__resizer",onMouseDown:v(t),"aria-label":"resize the panel"})]},t))):Ct("div",{className:"vm-predefined-dashboard-panels-panel__alert",children:Ct(Qr,{variant:"error",children:[Ct("code",{children:'"panels"'})," not found. Check the configuration file ",Ct("b",{children:i}),"."]})})})})})},Ih=()=>{(()=>{const{duration:e,relativeTime:t,period:{date:n}}=vn(),{customStep:a}=Ur(),{setSearchParamsFromKeys:i}=ii(),o=()=>{const r=zd({"g0.range_input":e,"g0.end_input":n,"g0.step_input":a,"g0.relative_time":t});i(r)};(0,r.useEffect)(o,[e,t,n,a]),(0,r.useEffect)(o,[])})();const{isMobile:e}=Wr(),{dashboardsSettings:t,dashboardsLoading:n,dashboardsError:a}=ea(),[i,o]=(0,r.useState)(0),l=(0,r.useMemo)((()=>t.map(((e,t)=>({label:e.title||"",value:t})))),[t]),s=(0,r.useMemo)((()=>t[i]||{}),[t,i]),c=(0,r.useMemo)((()=>null===s||void 0===s?void 0:s.rows),[s]),u=(0,r.useMemo)((()=>s.title||s.filename||""),[s]),d=(0,r.useMemo)((()=>Array.isArray(c)&&!!c.length),[c]),h=e=>()=>{(e=>{o(e)})(e)};return Ct("div",{className:"vm-predefined-panels",children:[n&&Ct(Dd,{}),!t.length&&a&&Ct(Qr,{variant:"error",children:a}),!t.length&&Ct(Qr,{variant:"info",children:"Dashboards not found"}),l.length>1&&Ct("div",{className:xr()({"vm-predefined-panels-tabs":!0,"vm-predefined-panels-tabs_mobile":e}),children:l.map((e=>Ct("div",{className:xr()({"vm-predefined-panels-tabs__tab":!0,"vm-predefined-panels-tabs__tab_active":e.value==i}),onClick:h(e.value),children:e.label},e.value)))}),Ct("div",{className:"vm-predefined-panels__dashboards",children:[d&&c.map(((e,t)=>Ct(Ph,{index:t,filename:u,title:e.title,panels:e.panels},"".concat(i,"_").concat(t)))),!!t.length&&!d&&Ct(Qr,{variant:"error",children:[Ct("code",{children:'"rows"'})," not found. Check the configuration file ",Ct("b",{children:u}),"."]})]})]})},Rh=(e,t)=>{const 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)};class Dh{constructor(){this.tsdbStatus=void 0,this.tabsNames=void 0,this.isPrometheus=void 0,this.tsdbStatus=this.defaultTSDBStatus,this.tabsNames=["table","graph"],this.isPrometheus=!1,this.getDefaultState=this.getDefaultState.bind(this)}set tsdbStatusData(e){this.isPrometheus=!(null===e||void 0===e||!e.headStats),this.tsdbStatus=e}get tsdbStatusData(){return this.tsdbStatus}get defaultTSDBStatus(){return{totalSeries:0,totalSeriesPrev:0,totalSeriesByAll:0,totalLabelValuePairs:0,seriesCountByMetricName:[],seriesCountByLabelName:[],seriesCountByFocusLabelValue:[],seriesCountByLabelValuePair:[],labelValueCountByLabelName:[]}}get isPrometheusData(){return this.isPrometheus}keys(e,t){const n=e&&/__name__=".+"/.test(e),r=e&&/{.+=".+"}/g.test(e),a=e&&/__name__=".+", .+!=""/g.test(e);let i=[];return i=t||a?i.concat("seriesCountByFocusLabelValue"):n?i.concat("labelValueCountByLabelName"):r?i.concat("seriesCountByMetricName","seriesCountByLabelName"):i.concat("seriesCountByMetricName","seriesCountByLabelName","seriesCountByLabelValuePair","labelValueCountByLabelName"),i}getDefaultState(e,t){return this.keys(e,t).reduce(((e,t)=>({...e,tabs:{...e.tabs,[t]:this.tabsNames},containerRefs:{...e.containerRefs,[t]:(0,r.useRef)(null)}})),{tabs:{},containerRefs:{}})}sectionsTitles(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"}}get sectionsTips(){return{seriesCountByMetricName:"\n

    \n This table returns a list of metrics with the highest cardinality.\n The cardinality of a metric is the number of time series associated with that metric,\n where each time series is defined as a unique combination of key-value label pairs.\n

    \n

    \n When looking to reduce the number of active series in your data source,\n you can start by inspecting individual metrics with high cardinality\n (i.e. that have lots of active time series associated with them),\n since that single metric contributes a large fraction of the series that make up your total series count.\n

    ",seriesCountByLabelName:"\n

    \n This table returns a list of the labels with the highest number of series.\n

    \n

    \n Use this table to identify labels that are storing dimensions with high cardinality\n (many different label values).\n

    \n

    \n It is recommended to choose labels such that they have a finite set of values,\n since every unique combination of key-value label pairs creates a new time series\n and therefore can dramatically increase the number of time series in your system.\n

    ",seriesCountByFocusLabelValue:"\n

    \n This table returns a list of unique label values per selected label.\n

    \n

    \n Use this table to identify label values that are storing per each selected series.\n

    ",labelValueCountByLabelName:"\n

    \n This table returns a list of labels with the highest number of the unique values.\n

    \n ",seriesCountByLabelValuePair:"\n

    \n This table returns a list of the label values pairs with the highest number of series.\n

    \n

    \n Use this table to identify unique label values pairs. This helps to identify same labels \n is applied to count timeseries in your system, since every unique combination of key-value label pairs \n creates a new time series and therefore can dramatically increase the number of time series in your system\n

    "}}get tablesHeaders(){return{seriesCountByMetricName:zh,seriesCountByLabelName:Fh,seriesCountByFocusLabelValue:jh,seriesCountByLabelValuePair:$h,labelValueCountByLabelName:Hh}}totalSeries(e){return"labelValueCountByLabelName"===e?-1:arguments.length>1&&void 0!==arguments[1]&&arguments[1]?this.tsdbStatus.totalSeriesPrev:this.tsdbStatus.totalSeries}}const zh=[{id:"name",label:"Metric name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of a metric to the total number of series"},{id:"action",label:""}],Fh=[{id:"name",label:"Label name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of the label to the total number of series"},{id:"action",label:""}],jh=[{id:"name",label:"Label value"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total"},{disablePadding:!1,id:"action",label:"",numeric:!1}],$h=[{id:"name",label:"Label=value pair"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of the label value pair to the total number of series"},{id:"action",label:""}],Hh=[{id:"name",label:"Label name"},{id:"value",label:"Number of unique values"},{id:"action",label:""}],Vh=()=>{const e=new Dh,[t]=je(),n=t.get("match"),a=t.get("focusLabel"),o=+(t.get("topN")||10),l=t.get("date")||i()().tz().format(Tt),s=Va(l),{serverUrl:c}=Nt(),[u,d]=(0,r.useState)(!1),[h,m]=(0,r.useState)(),[p,f]=(0,r.useState)(e.defaultTSDBStatus),[v,g]=(0,r.useState)(!1),y=async e=>{const t=await fetch(e);if(t.ok)return await t.json();throw new Error("Request failed with status ".concat(t.status))},_=async t=>{if(!c)return;m(""),d(!0),f(e.defaultTSDBStatus);const r={...t,date:t.date,topN:0,match:"",focusLabel:""},a={...t,date:i()(t.date).subtract(1,"day").tz().format(Tt)},o=[Rh(c,t),Rh(c,a)];s!==l&&o.push(Rh(c,r));try{var u,h,v,g,_,b,w,k,x,S;const[e,t,r={}]=await Promise.all(o.map(y)),a={...t.data},{data:i}=r,l={...e.data,totalSeries:(null===(u=e.data)||void 0===u?void 0:u.totalSeries)||(null===(h=e.data)||void 0===h||null===(v=h.headStats)||void 0===v?void 0:v.numSeries)||0,totalLabelValuePairs:(null===(g=e.data)||void 0===g?void 0:g.totalLabelValuePairs)||(null===(_=e.data)||void 0===_||null===(b=_.headStats)||void 0===b?void 0:b.numLabelValuePairs)||0,seriesCountByLabelName:(null===(w=e.data)||void 0===w?void 0:w.seriesCountByLabelName)||[],seriesCountByFocusLabelValue:(null===(k=e.data)||void 0===k?void 0:k.seriesCountByFocusLabelValue)||[],totalSeriesByAll:(null===i||void 0===i?void 0:i.totalSeries)||(null===i||void 0===i||null===(x=i.headStats)||void 0===x?void 0:x.numSeries)||p.totalSeriesByAll||0,totalSeriesPrev:(null===a||void 0===a?void 0:a.totalSeries)||(null===a||void 0===a||null===(S=a.headStats)||void 0===S?void 0:S.numSeries)||0},s=null===n||void 0===n?void 0:n.replace(/[{}"]/g,"");l.seriesCountByLabelValuePair=l.seriesCountByLabelValuePair.filter((e=>e.name!==s)),((e,t)=>{Object.keys(e).forEach((n=>{const r=n,a=e[r],i=t[r];Array.isArray(a)&&Array.isArray(i)&&a.forEach((e=>{var t;const n=null===(t=i.find((t=>t.name===e.name)))||void 0===t?void 0:t.value;e.diff=n?e.value-n:0,e.valuePrev=n||0}))}))})(l,a),f(l),d(!1)}catch(ap){d(!1),ap instanceof Error&&m("".concat(ap.name,": ").concat(ap.message))}};return(0,r.useEffect)((()=>{_({topN:o,match:n,date:l,focusLabel:a})}),[c,n,a,o,l]),(0,r.useEffect)((()=>{h&&(f(e.defaultTSDBStatus),d(!1))}),[h]),(0,r.useEffect)((()=>{const e=Ge(c);g(!!e)}),[c]),e.tsdbStatusData=p,{isLoading:u,appConfigurator:e,error:h,isCluster:v}},Uh={seriesCountByMetricName:e=>{let{query:t}=e;return Bh("__name__",t)},seriesCountByLabelName:e=>{let{query:t}=e;return"{".concat(t,'!=""}')},seriesCountByFocusLabelValue:e=>{let{query:t,focusLabel:n}=e;return Bh(n,t)},seriesCountByLabelValuePair:e=>{let{query:t}=e;const n=t.split("="),r=n[0],a=n.slice(1).join("=");return Bh(r,a)},labelValueCountByLabelName:e=>{let{query:t,match:n}=e;return""===n?"{".concat(t,'!=""}'):"".concat(n.replace("}",""),", ").concat(t,'!=""}')}},Bh=(e,t)=>e?"{"+e+"="+JSON.stringify(t)+"}":"",qh=e=>{var t;let{totalSeries:n=0,totalSeriesPrev:r=0,totalSeriesAll:a=0,seriesCountByMetricName:i=[],isPrometheus:o}=e;const{isMobile:l}=Wr(),[s]=je(),c=s.get("match"),u=s.get("focusLabel"),d=/__name__/.test(c||""),h=(null===(t=i[0])||void 0===t?void 0:t.value)/a*100,m=n-r,p=Math.abs(m)/r*100,f=[{title:"Total series",value:n.toLocaleString("en-US"),dynamic:n&&r&&!o?"".concat(p.toFixed(2),"%"):"",display:!u,info:'The total number of active time series. \n A time series is uniquely identified by its name plus a set of its labels. \n For example, temperature{city="NY",country="US"} and temperature{city="SF",country="US"} \n are two distinct series, since they differ by the city label.'},{title:"Percentage from total",value:isNaN(h)?"-":"".concat(h.toFixed(2),"%"),display:d,info:"The share of these series in the total number of time series."}].filter((e=>e.display));return f.length?Ct("div",{className:xr()({"vm-cardinality-totals":!0,"vm-cardinality-totals_mobile":l}),children:f.map((e=>{let{title:t,value:n,info:a,dynamic:i}=e;return Ct("div",{className:"vm-cardinality-totals-card",children:[Ct("h4",{className:"vm-cardinality-totals-card__title",children:[t,a&&Ct(fa,{title:Ct("p",{className:"vm-cardinality-totals-card__tooltip",children:a}),children:Ct("div",{className:"vm-cardinality-totals-card__info-icon",children:Ct(Rn,{})})})]}),Ct("span",{className:"vm-cardinality-totals-card__value",children:n}),!!i&&Ct(fa,{title:"in relation to the previous day: ".concat(r.toLocaleString("en-US")),children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive vm-dynamic-number_down":m<0,"vm-dynamic-number_negative vm-dynamic-number_up":m>0}),children:i})})]},t)}))}):null},Yh=(e,t)=>{const[n]=je(),a=n.get(t)?n.get(t):e,[i,o]=(0,r.useState)(a);return(0,r.useEffect)((()=>{a!==i&&o(a)}),[a]),[i,o]},Wh=e=>{let{isPrometheus:t,isCluster:n,...a}=e;const{isMobile:i}=Wr(),[o]=je(),{setSearchParamsFromKeys:l}=ii(),s=o.get("tips")||"",[c,u]=Yh("","match"),[d,h]=Yh("","focusLabel"),[m,p]=Yh(10,"topN"),f=(0,r.useMemo)((()=>m<0?"Number must be bigger than zero":""),[m]),v=()=>{l({match:c,topN:m,focusLabel:d})};return(0,r.useEffect)((()=>{const e=o.get("match"),t=+(o.get("topN")||10),n=o.get("focusLabel");e!==c&&u(e||""),t!==m&&p(t),n!==d&&h(n||"")}),[o]),Ct("div",{className:xr()({"vm-cardinality-configurator":!0,"vm-cardinality-configurator_mobile":i,"vm-block":!0,"vm-block_mobile":i}),children:[Ct("div",{className:"vm-cardinality-configurator-controls",children:[Ct("div",{className:"vm-cardinality-configurator-controls__query",children:Ct($a,{label:"Time series selector",type:"string",value:c,onChange:u,onEnter:v})}),Ct("div",{className:"vm-cardinality-configurator-controls__item",children:Ct($a,{label:"Focus label",type:"text",value:d||"",onChange:h,onEnter:v,endIcon:Ct(fa,{title:Ct("div",{children:Ct("p",{children:"To identify values with the highest number of series for the selected label."})}),children:Ct(cr,{})})})}),Ct("div",{className:"vm-cardinality-configurator-controls__item vm-cardinality-configurator-controls__item_limit",children:Ct($a,{label:"Limit entries",type:"number",value:t?10:m,error:f,disabled:t,helperText:t?"not available for Prometheus":"",onChange:e=>{const t=+e;p(isNaN(t)?0:t)},onEnter:v})})]}),Ct("div",{className:"vm-cardinality-configurator-bottom",children:[Ct(qh,{isPrometheus:t,isCluster:n,...a}),n&&Ct("div",{className:"vm-cardinality-configurator-bottom-helpful",children:Ct(Fd,{href:"https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#cardinality-explorer-statistic-inaccuracy",withIcon:!0,children:[Ct(lr,{}),"Statistic inaccuracy explanation"]})}),Ct("div",{className:"vm-cardinality-configurator-bottom-helpful",children:Ct(Fd,{href:"https://docs.victoriametrics.com/#cardinality-explorer",withIcon:!0,children:[Ct(lr,{}),"Documentation"]})}),Ct("div",{className:"vm-cardinality-configurator-bottom__execute",children:[Ct(fa,{title:s?"Hide tips":"Show tips",children:Ct(aa,{variant:"text",color:s?"warning":"gray",startIcon:Ct(mr,{}),onClick:()=>{const e=o.get("tips")||"";l({tips:e?"":"true"})},ariaLabel:"visibility tips"})}),Ct(aa,{variant:"text",startIcon:Ct(In,{}),onClick:()=>{l({match:"",focusLabel:""})},children:"Reset"}),Ct(aa,{startIcon:Ct(Yn,{}),onClick:v,children:"Execute Query"})]})]})]})};function Kh(e){const{order:t,orderBy:n,onRequestSort:r,headerCells:a}=e;return Ct("thead",{className:"vm-table-header vm-cardinality-panel-table__header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:a.map((e=>{return Ct("th",{className:xr()({"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:(a=e.id,e=>{r(e,a)}),children:Ct("div",{className:"vm-table-cell__content",children:[e.info?Ct(fa,{title:e.info,children:[Ct("div",{className:"vm-metrics-content-header__tip-icon",children:Ct(Rn,{})}),e.label]}):Ct(xt.FK,{children:e.label}),"action"!==e.id&&"percentage"!==e.id&&Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":n===e.id,"vm-table__sort-icon_desc":"desc"===t&&n===e.id}),children:Ct(Hn,{})})]})},e.id);var a}))})})}const Qh=["date","timestamp","time"];function Zh(e,t,n){const r=e[n],a=t[n],o=Qh.includes("".concat(n))?i()("".concat(r)).unix():r,l=Qh.includes("".concat(n))?i()("".concat(a)).unix():a;return lo?1:0}function Gh(e,t){return"desc"===e?(e,n)=>Zh(e,n,t):(e,n)=>-Zh(e,n,t)}function Jh(e,t){const n=e.map(((e,t)=>[e,t]));return n.sort(((e,n)=>{const r=t(e[0],n[0]);return 0!==r?r:e[1]-n[1]})),n.map((e=>e[0]))}const Xh=e=>{let{rows:t,headerCells:n,defaultSortColumn:a,tableCells:i}=e;const[o,l]=(0,r.useState)("desc"),[s,c]=(0,r.useState)(a),u=Jh(t,Gh(o,s));return Ct("table",{className:"vm-table vm-cardinality-panel-table",children:[Ct(Kh,{order:o,orderBy:s,onRequestSort:(e,t)=>{l(s===t&&"asc"===o?"desc":"asc"),c(t)},rowCount:t.length,headerCells:n}),Ct("tbody",{className:"vm-table-header",children:u.map((e=>Ct("tr",{className:"vm-table__row",children:i(e)},e.name)))})]})},em=e=>{let{row:t,totalSeries:n,totalSeriesPrev:r,onActionClick:a}=e;const i=n>0?t.value/n*100:-1,o=r>0?t.valuePrev/r*100:-1,l=[i,o].some((e=>-1===e)),s=i-o,c=l?"":"".concat(s.toFixed(2),"%"),u=()=>{a(t.name)};return Ct(xt.FK,{children:[Ct("td",{className:"vm-table-cell",children:Ct("span",{className:"vm-link vm-link_colored",onClick:u,children:t.name})},t.name),Ct("td",{className:"vm-table-cell",children:[t.value,!!t.diff&&Ct(fa,{title:"in relation to the previous day: ".concat(t.valuePrev),children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive":t.diff<0,"vm-dynamic-number_negative":t.diff>0}),children:["\xa0",t.diff>0?"+":"",t.diff]})})]},t.value),i>0&&Ct("td",{className:"vm-table-cell",children:Ct("div",{className:"vm-cardinality-panel-table__progress",children:[Ct(Vd,{value:i}),c&&Ct(fa,{title:"in relation to the previous day",children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive vm-dynamic-number_down":s<0,"vm-dynamic-number_negative vm-dynamic-number_up":s>0}),children:c})})]})},t.progressValue),Ct("td",{className:"vm-table-cell vm-table-cell_right",children:Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:"Filter by ".concat(t.name),children:Ct(aa,{variant:"text",size:"small",onClick:u,children:Ct(Wn,{})})})})},"action")]})},tm=e=>{let{data:t}=e;const[n,a]=(0,r.useState)([]),[i,o]=(0,r.useState)([0,0]);return(0,r.useEffect)((()=>{const e=t.sort(((e,t)=>t.value-e.value)),n=(e=>{const t=e.map((e=>e.value)),n=Math.ceil(t[0]||1),r=n/9;return new Array(11).fill(n+r).map(((e,t)=>Math.round(e-r*t)))})(e);o(n),a(e.map((e=>({...e,percentage:e.value/n[0]*100}))))}),[t]),Ct("div",{className:"vm-simple-bar-chart",children:[Ct("div",{className:"vm-simple-bar-chart-y-axis",children:i.map((e=>Ct("div",{className:"vm-simple-bar-chart-y-axis__tick",children:e},e)))}),Ct("div",{className:"vm-simple-bar-chart-data",children:n.map((e=>{let{name:t,value:n,percentage:r}=e;return Ct(fa,{title:"".concat(t,": ").concat(n),placement:"top-center",children:Ct("div",{className:"vm-simple-bar-chart-data-item",style:{maxHeight:"".concat(r||0,"%")}})},"".concat(t,"_").concat(n))}))})]})},nm=e=>{let{rows:t,tabs:n=[],chartContainer:a,totalSeries:i,totalSeriesPrev:o,onActionClick:l,sectionTitle:s,tip:c,tableHeaderCells:u,isPrometheus:d}=e;const{isMobile:h}=Wr(),[m,p]=(0,r.useState)("table"),f=d&&!t.length,v=(0,r.useMemo)((()=>n.map(((e,t)=>({value:e,label:e,icon:Ct(0===t?Qn:Kn,{})})))),[n]);return Ct("div",{className:xr()({"vm-metrics-content":!0,"vm-metrics-content_mobile":h,"vm-block":!0,"vm-block_mobile":h}),children:[Ct("div",{className:"vm-metrics-content-header vm-section-header",children:[Ct("h5",{className:xr()({"vm-metrics-content-header__title":!0,"vm-section-header__title":!0,"vm-section-header__title_mobile":h}),children:[!h&&c&&Ct(fa,{title:Ct("p",{dangerouslySetInnerHTML:{__html:c},className:"vm-metrics-content-header__tip"}),children:Ct("div",{className:"vm-metrics-content-header__tip-icon",children:Ct(Rn,{})})}),s]}),Ct("div",{className:"vm-section-header__tabs",children:Ct(Mr,{activeItem:m,items:v,onChange:p})})]}),f&&Ct("div",{className:"vm-metrics-content-prom-data",children:[Ct("div",{className:"vm-metrics-content-prom-data__icon",children:Ct(Rn,{})}),Ct("h3",{className:"vm-metrics-content-prom-data__title",children:"Prometheus Data Limitation"}),Ct("p",{className:"vm-metrics-content-prom-data__text",children:["Due to missing data from your Prometheus source, some tables may appear empty.",Ct("br",{}),"This does not indicate an issue with your system or our tool."]})]}),!f&&"table"===m&&Ct("div",{ref:a,className:xr()({"vm-metrics-content__table":!0,"vm-metrics-content__table_mobile":h}),children:Ct(Xh,{rows:t,headerCells:u,defaultSortColumn:"value",tableCells:e=>Ct(em,{row:e,totalSeries:i,totalSeriesPrev:o,onActionClick:l})})}),!f&&"graph"===m&&Ct("div",{className:"vm-metrics-content__chart",children:Ct(tm,{data:t.map((e=>{let{name:t,value:n}=e;return{name:t,value:n}}))})})]})},rm=e=>{let{title:t,children:n}=e;return Ct("div",{className:"vm-cardinality-tip",children:[Ct("div",{className:"vm-cardinality-tip-header",children:[Ct("div",{className:"vm-cardinality-tip-header__tip-icon",children:Ct(mr,{})}),Ct("h4",{className:"vm-cardinality-tip-header__title",children:t||"Tips"})]}),Ct("p",{className:"vm-cardinality-tip__description",children:n})]})},am=()=>Ct(rm,{title:"Metrics with a high number of series",children:Ct("ul",{children:[Ct("li",{children:["Identify and eliminate labels with frequently changed values to reduce their\xa0",Ct(Fd,{href:"https://docs.victoriametrics.com/FAQ.html#what-is-high-cardinality",children:"cardinality"}),"\xa0and\xa0",Ct(Fd,{href:"https://docs.victoriametrics.com/FAQ.html#what-is-high-churn-rate",children:"high churn rate"})]}),Ct("li",{children:["Find unused time series and\xa0",Ct(Fd,{href:"https://docs.victoriametrics.com/relabeling.html",children:"drop entire metrics"})]}),Ct("li",{children:["Aggregate time series before they got ingested into the database via\xa0",Ct(Fd,{href:"https://docs.victoriametrics.com/stream-aggregation.html",children:"streaming aggregation"})]})]})}),im=()=>Ct(rm,{title:"Labels with a high number of unique values",children:Ct("ul",{children:[Ct("li",{children:"Decrease the number of unique label values to reduce cardinality"}),Ct("li",{children:["Drop the label entirely via\xa0",Ct(Fd,{href:"https://docs.victoriametrics.com/relabeling.html",children:"relabeling"})]}),Ct("li",{children:"For volatile label values (such as URL path, user session, etc.) consider printing them to the log file instead of adding to time series"})]})}),om=()=>Ct(rm,{title:"Dashboard of a single metric",children:[Ct("p",{children:"This dashboard helps to understand the cardinality of a single metric."}),Ct("p",{children:"Each time series is a unique combination of key-value label pairs. Therefore a label key with many values can create a lot of time series for a particular metric. If you\u2019re trying to decrease the cardinality of a metric, start by looking at the labels with the highest number of values."}),Ct("p",{children:"Use the series selector at the top of the page to apply additional filters."})]}),lm=()=>Ct(rm,{title:"Dashboard of a label",children:[Ct("p",{children:"This dashboard helps you understand the count of time series per label."}),Ct("p",{children:"Use the selector at the top of the page to pick a label name you\u2019d like to inspect. For the selected label name, you\u2019ll see the label values that have the highest number of series associated with them. So if you\u2019ve chosen `instance` as your label name, you may see that `657` time series have value \u201chost-1\u201d attached to them and `580` time series have value `host-2` attached to them."}),Ct("p",{children:"This can be helpful in allowing you to determine where the bulk of your time series are coming from. If the label \u201cinstance=host-1\u201d was applied to 657 series and the label \u201cinstance=host-2\u201d was only applied to 580 series, you\u2019d know, for example, that host-01 was responsible for sending the majority of the time series."})]}),sm=()=>{const{isMobile:e}=Wr(),[t]=je(),{setSearchParamsFromKeys:n}=ii(),r=t.get("tips")||"",a=t.get("match")||"",i=t.get("focusLabel")||"",{isLoading:o,appConfigurator:l,error:s,isCluster:c}=Vh(),{tsdbStatusData:u,getDefaultState:d,tablesHeaders:h,sectionsTips:m}=l,p=d(a,i);return Ct("div",{className:xr()({"vm-cardinality-panel":!0,"vm-cardinality-panel_mobile":e}),children:[o&&Ct(Dd,{message:"Please wait while cardinality stats is calculated. \n This may take some time if the db contains big number of time series."}),Ct(Wh,{isPrometheus:l.isPrometheusData,totalSeries:u.totalSeries,totalSeriesPrev:u.totalSeriesPrev,totalSeriesAll:u.totalSeriesByAll,totalLabelValuePairs:u.totalLabelValuePairs,seriesCountByMetricName:u.seriesCountByMetricName,isCluster:c}),r&&Ct("div",{className:"vm-cardinality-panel-tips",children:[!a&&!i&&Ct(am,{}),a&&!i&&Ct(om,{}),!a&&!i&&Ct(im,{}),i&&Ct(lm,{})]}),s&&Ct(Qr,{variant:"error",children:s}),l.keys(a,i).map((e=>{return Ct(nm,{sectionTitle:l.sectionsTitles(i)[e],tip:m[e],rows:u[e],onActionClick:(t=e,e=>{const r={match:Uh[t]({query:e,focusLabel:i,match:a})};"labelValueCountByLabelName"!==t&&"seriesCountByLabelName"!=t||(r.focusLabel=e),"seriesCountByFocusLabelValue"==t&&(r.focusLabel=""),n(r)}),tabs:p.tabs[e],chartContainer:p.containerRefs[e],totalSeriesPrev:l.totalSeries(e,!0),totalSeries:l.totalSeries(e),tableHeaderCells:h[e],isPrometheus:l.isPrometheusData},e);var t}))]})},cm=e=>(["topByAvgDuration","topByCount","topBySumDuration"].forEach((t=>{const n=e[t];Array.isArray(n)&&n.forEach((e=>{const t=Jt(1e3*e.timeRangeSeconds);e.url=((e,t)=>{var n;const{query:r,timeRangeSeconds:a}=e,i=["g0.expr=".concat(encodeURIComponent(r))],o=null===(n=en.find((e=>e.duration===t)))||void 0===n?void 0:n.id;return o&&i.push("g0.relative_time=".concat(o)),a&&i.push("g0.range_input=".concat(t)),"".concat(Ye.home,"?").concat(i.join("&"))})(e,t),e.timeRange=t}))})),e),um=e=>{let{topN:t,maxLifetime:n}=e;const{serverUrl:a}=Nt(),{setSearchParamsFromKeys:i}=ii(),[o,l]=(0,r.useState)(null),[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)(),h=(0,r.useMemo)((()=>((e,t,n)=>"".concat(e,"/api/v1/status/top_queries?topN=").concat(t||"","&maxLifetime=").concat(n||""))(a,t,n)),[a,t,n]);return{data:o,error:u,loading:s,fetch:async()=>{c(!0),i({topN:t,maxLifetime:n});try{const e=await fetch(h),t=await e.json();l(e.ok?cm(t):null),d(String(t.error||""))}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&d("".concat(ap.name,": ").concat(ap.message))}c(!1)}}},dm=e=>{let{rows:t,columns:n,defaultOrderBy:a}=e;const i=Cd(),[o,l]=(0,r.useState)(a||"count"),[s,c]=(0,r.useState)("desc"),u=(0,r.useMemo)((()=>Jh(t,Gh(s,o))),[t,o,s]),d=e=>()=>{var t;t=e,c((e=>"asc"===e&&o===t?"desc":"asc")),l(t)},h=e=>{let{query:t}=e;return async()=>{await i(t,"Query has been copied")}};return Ct("table",{className:"vm-table",children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[n.map((e=>Ct("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:d(e.sortBy||e.key),children:Ct("div",{className:"vm-table-cell__content",children:[e.title||e.key,Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":o===e.key,"vm-table__sort-icon_desc":"desc"===s&&o===e.key}),children:Ct(Hn,{})})]})},e.key))),Ct("th",{className:"vm-table-cell vm-table-cell_header"})," "]})}),Ct("tbody",{className:"vm-table-body",children:u.map(((e,t)=>Ct("tr",{className:"vm-table__row",children:[n.map((t=>Ct("td",{className:"vm-table-cell",children:e[t.key]||"-"},t.key))),Ct("td",{className:"vm-table-cell vm-table-cell_no-padding",children:Ct("div",{className:"vm-top-queries-panels__table-actions",children:[e.url&&Ct(fa,{title:"Execute query",children:Ct(Ie,{to:e.url,target:"_blank",rel:"noreferrer","aria-disabled":!0,children:Ct(aa,{variant:"text",size:"small",startIcon:Ct(Wn,{}),ariaLabel:"execute query"})})}),Ct(fa,{title:"Copy query",children:Ct(aa,{variant:"text",size:"small",startIcon:Ct(ar,{}),onClick:h(e),ariaLabel:"copy query"})})]})})]},t)))})]})},hm=["table","JSON"].map(((e,t)=>({value:String(t),label:e,icon:Ct(0===t?Qn:Zn,{})}))),mm=e=>{let{rows:t,title:n,columns:a,defaultOrderBy:i}=e;const{isMobile:o}=Wr(),[l,s]=(0,r.useState)(0);return Ct("div",{className:xr()({"vm-top-queries-panel":!0,"vm-block":!0,"vm-block_mobile":o}),children:[Ct("div",{className:xr()({"vm-top-queries-panel-header":!0,"vm-section-header":!0,"vm-top-queries-panel-header_mobile":o}),children:[Ct("h5",{className:xr()({"vm-section-header__title":!0,"vm-section-header__title_mobile":o}),children:n}),Ct("div",{className:"vm-section-header__tabs",children:Ct(Mr,{activeItem:String(l),items:hm,onChange:e=>{s(+e)}})})]}),Ct("div",{className:xr()({"vm-top-queries-panel__table":!0,"vm-top-queries-panel__table_mobile":o}),children:[0===l&&Ct(dm,{rows:t,columns:a,defaultOrderBy:i}),1===l&&Ct(_h,{data:t})]})]})},pm=()=>{const{isMobile:e}=Wr(),[t,n]=Yh(10,"topN"),[a,o]=Yh("10m","maxLifetime"),{data:l,error:s,loading:c,fetch:u}=um({topN:t,maxLifetime:a}),d=(0,r.useMemo)((()=>{const e=a.trim().split(" ").reduce(((e,t)=>{const n=Yt(t);return n?{...e,...n}:{...e}}),{});return!!i().duration(e).asMilliseconds()}),[a]),h=(0,r.useMemo)((()=>!!t&&t<1),[t]),m=(0,r.useMemo)((()=>h?"Number must be bigger than zero":""),[h]),p=(0,r.useMemo)((()=>d?"":"Invalid duration value"),[d]),f=e=>{if(!l)return e;const t=l[e];return"number"===typeof t?Hi(t,t,t):t||e},v=e=>{"Enter"===e.key&&u()};return(0,r.useEffect)((()=>{l&&(t||n(+l.topN),a||o(l.maxLifetime))}),[l]),(0,r.useEffect)((()=>(u(),window.addEventListener("popstate",u),()=>{window.removeEventListener("popstate",u)})),[]),Ct("div",{className:xr()({"vm-top-queries":!0,"vm-top-queries_mobile":e}),children:[c&&Ct(Dd,{containerStyles:{height:"500px"}}),Ct("div",{className:xr()({"vm-top-queries-controls":!0,"vm-block":!0,"vm-block_mobile":e}),children:[Ct("div",{className:"vm-top-queries-controls-fields",children:[Ct("div",{className:"vm-top-queries-controls-fields__item",children:Ct($a,{label:"Max lifetime",value:a,error:p,helperText:"For example ".concat("30ms, 15s, 3d4h, 1y2w"),onChange:e=>{o(e)},onKeyDown:v})}),Ct("div",{className:"vm-top-queries-controls-fields__item",children:Ct($a,{label:"Number of returned queries",type:"number",value:t||"",error:m,onChange:e=>{n(+e)},onKeyDown:v})})]}),Ct("div",{className:xr()({"vm-top-queries-controls-bottom":!0,"vm-top-queries-controls-bottom_mobile":e}),children:[Ct("div",{className:"vm-top-queries-controls-bottom__info",children:["VictoriaMetrics tracks the last\xa0",Ct(fa,{title:"search.queryStats.lastQueriesCount",children:Ct("b",{children:f("search.queryStats.lastQueriesCount")})}),"\xa0queries with durations at least\xa0",Ct(fa,{title:"search.queryStats.minQueryDuration",children:Ct("b",{children:f("search.queryStats.minQueryDuration")})})]}),Ct("div",{className:"vm-top-queries-controls-bottom__button",children:Ct(aa,{startIcon:Ct(Yn,{}),onClick:u,children:"Execute"})})]})]}),s&&Ct(Qr,{variant:"error",children:s}),l&&Ct(xt.FK,{children:Ct("div",{className:"vm-top-queries-panels",children:[Ct(mm,{rows:l.topBySumDuration,title:"Queries with most summary time to execute",columns:[{key:"query"},{key:"sumDurationSeconds",title:"sum duration, sec"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}],defaultOrderBy:"sumDurationSeconds"}),Ct(mm,{rows:l.topByAvgDuration,title:"Most heavy queries",columns:[{key:"query"},{key:"avgDurationSeconds",title:"avg duration, sec"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}],defaultOrderBy:"avgDurationSeconds"}),Ct(mm,{rows:l.topByCount,title:"Most frequently executed queries",columns:[{key:"query"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}]})]})})]})},fm={"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)"},vm={"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(80,80,80,0.9)","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)"},gm=()=>{const[e,t]=(0,r.useState)(gt()),n=e=>{t(e.matches)};return(0,r.useEffect)((()=>{const e=window.matchMedia("(prefers-color-scheme: dark)");return e.addEventListener("change",n),()=>e.removeEventListener("change",n)}),[]),e},ym=["primary","secondary","error","warning","info","success"],_m=e=>{let{onLoaded:t}=e;const n=Ke(),{palette:a={}}=We(),{theme:i}=Nt(),o=gm(),l=At(),s=Ar(),[c,u]=(0,r.useState)({[mt.dark]:fm,[mt.light]:vm,[mt.system]:gt()?fm:vm}),d=()=>{const{innerWidth:e,innerHeight:t}=window,{clientWidth:n,clientHeight:r}=document.documentElement;vt("scrollbar-width","".concat(e-n,"px")),vt("scrollbar-height","".concat(t-r,"px")),vt("vh","".concat(.01*t,"px"))},h=()=>{ym.forEach(((e,n)=>{const r=(e=>{let 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"})(ft("color-".concat(e)));vt("".concat(e,"-text"),r),n===ym.length-1&&(l({type:"SET_DARK_THEME"}),t(!0))}))},m=()=>{const e=Xe("THEME")||mt.system,t=c[e];Object.entries(t).forEach((e=>{let[t,n]=e;vt(t,n)})),h(),n&&(ym.forEach((e=>{const t=a[e];t&&vt("color-".concat(e),t)})),h())};return(0,r.useEffect)((()=>{d(),m()}),[c]),(0,r.useEffect)(d,[s]),(0,r.useEffect)((()=>{const e=gt()?fm:vm;c[mt.system]!==e?u((t=>({...t,[mt.system]:e}))):m()}),[i,o]),(0,r.useEffect)((()=>{n&&l({type:"SET_THEME",payload:mt.light})}),[]),null},bm=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)(!1),i=(0,r.useRef)(document.body),o=e=>{e.preventDefault(),e.stopPropagation(),"dragenter"===e.type||"dragover"===e.type?a(!0):"dragleave"===e.type&&a(!1)};return Nr("dragenter",o,i),Nr("dragleave",o,i),Nr("dragover",o,i),Nr("drop",(e=>{var n;e.preventDefault(),e.stopPropagation(),a(!1),null!==e&&void 0!==e&&null!==(n=e.dataTransfer)&&void 0!==n&&n.files&&e.dataTransfer.files[0]&&(e=>{const n=Array.from(e||[]);t(n)})(e.dataTransfer.files)}),i),Nr("paste",(e=>{var n;const r=null===(n=e.clipboardData)||void 0===n?void 0:n.items;if(!r)return;const a=Array.from(r).filter((e=>"application/json"===e.type)).map((e=>e.getAsFile())).filter((e=>null!==e));t(a)}),i),{files:e,dragging:n}},wm=e=>{let{onOpenModal:t,onChange:n}=e;return Ct("div",{className:"vm-upload-json-buttons",children:[Ct(aa,{variant:"outlined",onClick:t,children:"Paste JSON"}),Ct(aa,{children:["Upload Files",Ct("input",{id:"json",type:"file",accept:"application/json",multiple:!0,title:" ",onChange:n})]})]})},km=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)([]),i=(0,r.useMemo)((()=>!!e.length),[e]),{value:o,setTrue:l,setFalse:s}=oa(!1),c=function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:"";a((n=>[{filename:t,text:": ".concat(e.message)},...n]))},u=(e,n)=>{try{const r=JSON.parse(e),a=r.trace||r;if(!a.duration_msec)return void c(new Error(ht.traceNotFound),n);const i=new Pd(a,n);t((e=>[i,...e]))}catch(ap){ap instanceof Error&&c(ap,n)}},d=e=>{e.map((e=>{const t=new FileReader,n=(null===e||void 0===e?void 0:e.name)||"";t.onload=e=>{var t;const r=String(null===(t=e.target)||void 0===t?void 0:t.result);u(r,n)},t.readAsText(e)}))},h=e=>{a([]);const t=Array.from(e.target.files||[]);d(t),e.target.value=""},m=e=>()=>{(e=>{a((t=>t.filter(((t,n)=>n!==e))))})(e)},{files:p,dragging:f}=bm();return(0,r.useEffect)((()=>{d(p)}),[p]),Ct("div",{className:"vm-trace-page",children:[Ct("div",{className:"vm-trace-page-header",children:[Ct("div",{className:"vm-trace-page-header-errors",children:n.map(((e,t)=>Ct("div",{className:"vm-trace-page-header-errors-item",children:[Ct(Qr,{variant:"error",children:[Ct("b",{className:"vm-trace-page-header-errors-item__filename",children:e.filename}),Ct("span",{children:e.text})]}),Ct(aa,{className:"vm-trace-page-header-errors-item__close",startIcon:Ct(Pn,{}),variant:"text",color:"error",onClick:m(t)})]},"".concat(e,"_").concat(t))))}),Ct("div",{children:i&&Ct(wm,{onOpenModal:l,onChange:h})})]}),i&&Ct("div",{children:Ct(Yd,{jsonEditor:!0,traces:e,onDeleteClick:n=>{const r=e.filter((e=>e.idValue!==n.idValue));t([...r])}})}),!i&&Ct("div",{className:"vm-trace-page-preview",children:[Ct("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",Ct("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.","\n","Attach files by dragging & dropping, selecting or pasting them."]}),Ct(wm,{onOpenModal:l,onChange:h})]}),o&&Ct(pa,{title:"Paste JSON",onClose:s,children:Ct(qd,{editable:!0,displayTitle:!0,defaultTile:"JSON ".concat(e.length+1),onClose:s,onUpload:u})}),f&&Ct("div",{className:"vm-trace-page__dropzone"})]})},xm=e=>{const{serverUrl:t}=Nt(),{period:n}=vn(),[a,i]=(0,r.useState)([]),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(),u=(0,r.useMemo)((()=>((e,t,n)=>{const 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(0,r.useEffect)((()=>{if(!e)return;(async()=>{l(!0);try{const e=await fetch(u),t=await e.json(),n=t.data||[];i(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?c(void 0):c("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(ap){ap instanceof Error&&c("".concat(ap.name,": ").concat(ap.message))}l(!1)})().catch(console.error)}),[u]),{instances:a,isLoading:o,error:s}},Sm=(e,t)=>{const{serverUrl:n}=Nt(),{period:a}=vn(),[i,o]=(0,r.useState)([]),[l,s]=(0,r.useState)(!1),[c,u]=(0,r.useState)(),d=(0,r.useMemo)((()=>((e,t,n,r)=>{const a=Object.entries({job:n,instance:r}).filter((e=>e[1])).map((e=>{let[t,n]=e;return"".concat(t,"=").concat(JSON.stringify(n))})).join(","),i="{".concat(a,"}");return"".concat(e,"/api/v1/label/__name__/values?match[]=").concat(encodeURIComponent(i),"&start=").concat(t.start,"&end=").concat(t.end)})(n,a,e,t)),[n,a,e,t]);return(0,r.useEffect)((()=>{if(!e)return;(async()=>{s(!0);try{const e=await fetch(d),t=await e.json(),n=t.data||[];o(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?u(void 0):u("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(ap){ap instanceof Error&&u("".concat(ap.name,": ").concat(ap.message))}s(!1)})().catch(console.error)}),[d]),{names:i,isLoading:l,error:c}},Cm=e=>{let{name:t,job:n,instance:a,rateEnabled:i,isBucket:o,height:l}=e;const{isMobile:s}=Wr(),{customStep:c,yaxis:u}=Ur(),{period:d}=vn(),h=Br(),m=gn(),p=Kt(d.end-d.start),f=Wt(c),v=Jt(10*f*1e3),[g,y]=(0,r.useState)(!1),[_,b]=(0,r.useState)(!1),w=g&&c===p?v:c,k=(0,r.useMemo)((()=>{const e=Object.entries({job:n,instance:a}).filter((e=>e[1])).map((e=>{let[t,n]=e;return"".concat(t,"=").concat(JSON.stringify(n))}));e.push("__name__=".concat(JSON.stringify(t))),"node_cpu_seconds_total"==t&&e.push('mode!="idle"');const r="{".concat(e.join(","),"}");if(o)return"sum(rate(".concat(r,")) by (vmrange, le)");const l=i?"rollup_rate(".concat(r,")"):"rollup(".concat(r,")");return"\nwith (q = ".concat(l,') (\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,a,i,o]),{isLoading:x,graphData:S,error:C,queryErrors:E,warning:N,isHistogram:A}=Rd({predefinedQuery:[k],visible:!0,customStep:w,showAllSeries:_});return(0,r.useEffect)((()=>{y(A)}),[A]),Ct("div",{className:xr()({"vm-explore-metrics-graph":!0,"vm-explore-metrics-graph_mobile":s}),children:[x&&Ct(Dd,{}),C&&Ct(Qr,{variant:"error",children:C}),E[0]&&Ct(Qr,{variant:"error",children:E[0]}),N&&Ct(Kd,{warning:N,query:[k],onChange:b}),S&&d&&Ct(mh,{data:S,period:d,customStep:w,query:[k],yaxis:u,setYaxisLimits:e=>{h({type:"SET_YAXIS_LIMITS",payload:e})},setPeriod:e=>{let{from:t,to:n}=e;m({type:"SET_PERIOD",payload:{from:t,to:n}})},showLegend:!1,height:l,isHistogram:A})]})},Em=e=>{let{name:t,index:n,length:r,isBucket:a,rateEnabled:i,onChangeRate:o,onRemoveItem:l,onChangeOrder:s}=e;const{isMobile:c}=Wr(),{value:u,setTrue:d,setFalse:h}=oa(!1),m=()=>{l(t)},p=()=>{s(t,n,n+1)},f=()=>{s(t,n,n-1)};return Ct("div",c?{className:"vm-explore-metrics-item-header vm-explore-metrics-item-header_mobile",children:[Ct("div",{className:"vm-explore-metrics-item-header__name",children:t}),Ct(aa,{variant:"text",size:"small",startIcon:Ct(dr,{}),onClick:d,ariaLabel:"open panel settings"}),u&&Ct(pa,{title:t,onClose:h,children:Ct("div",{className:"vm-explore-metrics-item-header-modal",children:[Ct("div",{className:"vm-explore-metrics-item-header-modal-order",children:[Ct(aa,{startIcon:Ct(Xn,{}),variant:"outlined",onClick:f,disabled:0===n,ariaLabel:"move graph up"}),Ct("p",{children:["position:",Ct("span",{className:"vm-explore-metrics-item-header-modal-order__index",children:["#",n+1]})]}),Ct(aa,{endIcon:Ct(Jn,{}),variant:"outlined",onClick:p,disabled:n===r-1,ariaLabel:"move graph down"})]}),!a&&Ct("div",{className:"vm-explore-metrics-item-header-modal__rate",children:[Ct(wd,{label:Ct("span",{children:["enable ",Ct("code",{children:"rate()"})]}),value:i,onChange:o,fullWidth:!0}),Ct("p",{children:"calculates the average per-second speed of metrics change"})]}),Ct(aa,{startIcon:Ct(Pn,{}),color:"error",variant:"outlined",onClick:m,fullWidth:!0,children:"Remove graph"})]})})]}:{className:"vm-explore-metrics-item-header",children:[Ct("div",{className:"vm-explore-metrics-item-header-order",children:[Ct(fa,{title:"move graph up",children:Ct(aa,{className:"vm-explore-metrics-item-header-order__up",startIcon:Ct($n,{}),variant:"text",color:"gray",size:"small",onClick:f,ariaLabel:"move graph up"})}),Ct("div",{className:"vm-explore-metrics-item-header__index",children:["#",n+1]}),Ct(fa,{title:"move graph down",children:Ct(aa,{className:"vm-explore-metrics-item-header-order__down",startIcon:Ct($n,{}),variant:"text",color:"gray",size:"small",onClick:p,ariaLabel:"move graph down"})})]}),Ct("div",{className:"vm-explore-metrics-item-header__name",children:t}),!a&&Ct("div",{className:"vm-explore-metrics-item-header__rate",children:Ct(fa,{title:"calculates the average per-second speed of metric's change",children:Ct(wd,{label:Ct("span",{children:["enable ",Ct("code",{children:"rate()"})]}),value:i,onChange:o})})}),Ct("div",{className:"vm-explore-metrics-item-header__close",children:Ct(fa,{title:"close graph",children:Ct(aa,{startIcon:Ct(Pn,{}),variant:"text",color:"gray",size:"small",onClick:m,ariaLabel:"close graph"})})})]})},Nm=e=>{let{name:t,job:n,instance:a,index:i,length:o,size:l,onRemoveItem:s,onChangeOrder:c}=e;const u=(0,r.useMemo)((()=>/_sum?|_total?|_count?/.test(t)),[t]),d=(0,r.useMemo)((()=>/_bucket?/.test(t)),[t]),[h,m]=(0,r.useState)(u),p=Ar(),f=(0,r.useMemo)(l.height,[l,p]);return(0,r.useEffect)((()=>{m(u)}),[n]),Ct("div",{className:"vm-explore-metrics-item vm-block vm-block_empty-padding",children:[Ct(Em,{name:t,index:i,length:o,isBucket:d,rateEnabled:h,size:l.id,onChangeRate:m,onRemoveItem:s,onChangeOrder:c}),Ct(Cm,{name:t,job:n,instance:a,rateEnabled:h,isBucket:d,height:f},"".concat(t,"_").concat(n,"_").concat(a,"_").concat(h))]})},Am=e=>{let{values:t,onRemoveItem:n}=e;const{isMobile:r}=Wr();return r?Ct("span",{className:"vm-select-input-content__counter",children:["selected ",t.length]}):Ct(xt.FK,{children:t.map((e=>{return Ct("div",{className:"vm-select-input-content__selected",children:[Ct("span",{children:e}),Ct("div",{onClick:(t=e,e=>{n(t),e.stopPropagation()}),children:Ct(Pn,{})})]},e);var t}))})},Mm=e=>{let{value:t,list:n,label:a,placeholder:i,noOptionsText:o,clearable:l=!1,searchable:s=!1,autofocus:c,disabled:u,onChange:d}=e;const{isDarkTheme:h}=Nt(),{isMobile:m}=Wr(),[p,f]=(0,r.useState)(""),v=(0,r.useRef)(null),[g,y]=(0,r.useState)(null),[_,b]=(0,r.useState)(!1),w=(0,r.useRef)(null),k=Array.isArray(t),x=Array.isArray(t)?t:void 0,S=m&&k&&!(null===x||void 0===x||!x.length),C=(0,r.useMemo)((()=>_?p:Array.isArray(t)?"":t),[t,p,_,k]),E=(0,r.useMemo)((()=>_?p||"(.+)":""),[p,_]),N=()=>{w.current&&w.current.blur()},A=()=>{b(!1),N()},M=e=>{f(""),d(e),k||A(),k&&w.current&&w.current.focus()};return(0,r.useEffect)((()=>{f(""),_&&w.current&&w.current.focus(),_||N()}),[_,w]),(0,r.useEffect)((()=>{c&&w.current&&!m&&w.current.focus()}),[c,w]),Nr("keyup",(e=>{w.current!==e.target&&b(!1)})),ra(v,A,g),Ct("div",{className:xr()({"vm-select":!0,"vm-select_dark":h,"vm-select_disabled":u}),children:[Ct("div",{className:"vm-select-input",onClick:e=>{e.target instanceof HTMLInputElement||u||b((e=>!e))},ref:v,children:[Ct("div",{className:"vm-select-input-content",children:[!(null===x||void 0===x||!x.length)&&Ct(Am,{values:x,onRemoveItem:M}),!S&&Ct("input",{value:C,type:"text",placeholder:i,onInput:e=>{f(e.target.value)},onFocus:()=>{u||b(!0)},onBlur:()=>{n.includes(p)&&d(p)},ref:w,readOnly:m||!s})]}),a&&Ct("span",{className:"vm-text-field__label",children:a}),l&&t&&Ct("div",{className:"vm-select-input__icon",onClick:(e=>t=>{M(e),t.stopPropagation()})(""),children:Ct(Pn,{})}),Ct("div",{className:xr()({"vm-select-input__icon":!0,"vm-select-input__icon_open":_}),children:Ct(Hn,{})})]}),Ct(Ti,{label:a,value:E,options:n.map((e=>({value:e}))),anchor:v,selected:x,minLength:1,fullWidth:!0,noOptionsText:o,onSelect:M,onOpenAutocomplete:b,onChangeWrapperRef:y})]})},Tm=lt.map((e=>e.id)),Lm=e=>{let{jobs:t,instances:n,names:a,job:i,instance:o,size:l,selectedMetrics:s,onChangeJob:c,onChangeInstance:u,onToggleMetric:d,onChangeSize:h}=e;const m=(0,r.useMemo)((()=>i?"":"No instances. Please select job"),[i]),p=(0,r.useMemo)((()=>i?"":"No metric names. Please select job"),[i]),{isMobile:f}=Wr(),{value:v,toggle:g,setFalse:y}=oa("false"!==Xe("EXPLORE_METRICS_TIPS"));return(0,r.useEffect)((()=>{Je("EXPLORE_METRICS_TIPS","".concat(v))}),[v]),Ct(xt.FK,{children:[Ct("div",{className:xr()({"vm-explore-metrics-header":!0,"vm-explore-metrics-header_mobile":f,"vm-block":!0,"vm-block_mobile":f}),children:[Ct("div",{className:"vm-explore-metrics-header__job",children:Ct(Mm,{value:i,list:t,label:"Job",placeholder:"Please select job",onChange:c,autofocus:!i&&!!t.length&&!f,searchable:!0})}),Ct("div",{className:"vm-explore-metrics-header__instance",children:Ct(Mm,{value:o,list:n,label:"Instance",placeholder:"Please select instance",onChange:u,noOptionsText:m,clearable:!0,searchable:!0})}),Ct("div",{className:"vm-explore-metrics-header__size",children:[Ct(Mm,{label:"Size graphs",value:l,list:Tm,onChange:h}),Ct(fa,{title:"".concat(v?"Hide":"Show"," tip"),children:Ct(aa,{variant:"text",color:v?"warning":"gray",startIcon:Ct(mr,{}),onClick:g,ariaLabel:"visibility tips"})})]}),Ct("div",{className:"vm-explore-metrics-header-metrics",children:Ct(Mm,{label:"Metrics",value:s,list:a,placeholder:"Search metric name",onChange:d,noOptionsText:p,clearable:!0,searchable:!0})})]}),v&&Ct(Qr,{variant:"warning",children:Ct("div",{className:"vm-explore-metrics-header-description",children:[Ct("p",{children:["Please note: this page is solely designed for exploring Prometheus metrics. Prometheus metrics always contain ",Ct("code",{children:"job"})," and ",Ct("code",{children:"instance"})," labels (see ",Ct("a",{className:"vm-link vm-link_colored",href:"https://prometheus.io/docs/concepts/jobs_instances/",children:"these docs"}),"), and this page relies on them as filters. ",Ct("br",{}),"Please use this page for Prometheus metrics only, in accordance with their naming conventions."]}),Ct(aa,{variant:"text",size:"small",startIcon:Ct(Pn,{}),onClick:y,ariaLabel:"close tips"})]})})]})},Om=ct("job",""),Pm=ct("instance",""),Im=ct("metrics",""),Rm=ct("size",""),Dm=lt.find((e=>Rm?e.id===Rm:e.isDefault))||lt[0],zm=()=>{const[e,t]=(0,r.useState)(Om),[n,a]=(0,r.useState)(Pm),[i,o]=(0,r.useState)(Im?Im.split("&"):[]),[l,s]=(0,r.useState)(Dm);(e=>{let{job:t,instance:n,metrics:a,size:i}=e;const{duration:o,relativeTime:l,period:{date:s}}=vn(),{customStep:c}=Ur(),{setSearchParamsFromKeys:u}=ii(),d=()=>{const e=zd({"g0.range_input":o,"g0.end_input":s,"g0.step_input":c,"g0.relative_time":l,size:i,job:t,instance:n,metrics:a});u(e)};(0,r.useEffect)(d,[o,l,s,c,t,n,a,i]),(0,r.useEffect)(d,[])})({job:e,instance:n,metrics:i.join("&"),size:l.id});const{jobs:c,isLoading:u,error:d}=(()=>{const{serverUrl:e}=Nt(),{period:t}=vn(),[n,a]=(0,r.useState)([]),[i,o]=(0,r.useState)(!1),[l,s]=(0,r.useState)(),c=(0,r.useMemo)((()=>((e,t)=>"".concat(e,"/api/v1/label/job/values?start=").concat(t.start,"&end=").concat(t.end))(e,t)),[e,t]);return(0,r.useEffect)((()=>{(async()=>{o(!0);try{const e=await fetch(c),t=await e.json(),n=t.data||[];a(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?s(void 0):s("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(ap){ap instanceof Error&&s("".concat(ap.name,": ").concat(ap.message))}o(!1)})().catch(console.error)}),[c]),{jobs:n,isLoading:i,error:l}})(),{instances:h,isLoading:m,error:p}=xm(e),{names:f,isLoading:v,error:g}=Sm(e,n),y=(0,r.useMemo)((()=>u||m||v),[u,m,v]),_=(0,r.useMemo)((()=>d||p||g),[d,p,g]),b=e=>{o(e?t=>t.includes(e)?t.filter((t=>t!==e)):[...t,e]:[])},w=(e,t,n)=>{const r=n>i.length-1;n<0||r||o((e=>{const r=[...e],[a]=r.splice(t,1);return r.splice(n,0,a),r}))};return(0,r.useEffect)((()=>{n&&h.length&&!h.includes(n)&&a("")}),[h,n]),Ct("div",{className:"vm-explore-metrics",children:[Ct(Lm,{jobs:c,instances:h,names:f,job:e,size:l.id,instance:n,selectedMetrics:i,onChangeJob:t,onChangeSize:e=>{const t=lt.find((t=>t.id===e));t&&s(t)},onChangeInstance:a,onToggleMetric:b}),y&&Ct(Dd,{}),_&&Ct(Qr,{variant:"error",children:_}),!e&&Ct(Qr,{variant:"info",children:"Please select job to see list of metric names."}),e&&!i.length&&Ct(Qr,{variant:"info",children:"Please select metric names to see the graphs."}),Ct("div",{className:"vm-explore-metrics-body",children:i.map(((t,r)=>Ct(Nm,{name:t,job:e,instance:n,index:r,length:i.length,size:l,onRemoveItem:b,onChangeOrder:w},t)))})]})},Fm=()=>{const t=Cd();return Ct("div",{className:"vm-preview-icons",children:Object.entries(e).map((e=>{let[n,r]=e;return Ct("div",{className:"vm-preview-icons-item",onClick:(a=n,async()=>{await t("<".concat(a,"/>"),"<".concat(a,"/> has been copied"))}),children:[Ct("div",{className:"vm-preview-icons-item__svg",children:r()}),Ct("div",{className:"vm-preview-icons-item__name",children:"<".concat(n,"/>")})]},n);var a}))})};var jm=function(e){return e.copy="Copy",e.copied="Copied",e}(jm||{});const $m=e=>{let{code:t}=e;const[n,a]=(0,r.useState)(jm.copy);return(0,r.useEffect)((()=>{let e=null;return n===jm.copied&&(e=setTimeout((()=>a(jm.copy)),1e3)),()=>{e&&clearTimeout(e)}}),[n]),Ct("code",{className:"vm-code-example",children:[t,Ct("div",{className:"vm-code-example__copy",children:Ct(fa,{title:n,children:Ct(aa,{size:"small",variant:"text",onClick:()=>{navigator.clipboard.writeText(t),a(jm.copied)},startIcon:Ct(ar,{}),ariaLabel:"close"})})})]})},Hm=()=>Ct("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/MetricsQL.html",target:"_blank",rel:"help noreferrer",children:"MetricsQL"}),Vm=()=>Ct("a",{className:"vm-link vm-link_colored",href:"https://grafana.com/grafana/dashboards/1860-node-exporter-full/",target:"_blank",rel:"help noreferrer",children:"Node Exporter Full"}),Um=()=>Ct("section",{className:"vm-with-template-tutorial",children:[Ct("h2",{className:"vm-with-template-tutorial__title",children:["Tutorial for WITH expressions in ",Ct(Hm,{})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Let's look at the following real query from ",Ct(Vm,{})," dashboard:"]}),Ct($m,{code:'(\n (\n node_memory_MemTotal_bytes{instance=~"$node:$port", job=~"$job"}\n -\n node_memory_MemFree_bytes{instance=~"$node:$port", job=~"$job"}\n )\n /\n node_memory_MemTotal_bytes{instance=~"$node:$port", job=~"$job"}\n) * 100'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"It is clear the query calculates the percentage of used memory for the given $node, $port and $job. Isn't it? :)"})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"What's wrong with this query? Copy-pasted label filters for distinct timeseries which makes it easy to mistype these filters during modification. Let's simplify the query with WITH expressions:"}),Ct($m,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"}\n)\n(\n node_memory_MemTotal_bytes{commonFilters}\n -\n node_memory_MemFree_bytes{commonFilters}\n)\n /\nnode_memory_MemTotal_bytes{commonFilters} * 100'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Now label filters are located in a single place instead of three distinct places. The query mentions node_memory_MemTotal_bytes metric twice and ","{commonFilters}"," three times. WITH expressions may improve this:"]}),Ct($m,{code:'WITH (\n my_resource_utilization(free, limit, filters) = (limit{filters} - free{filters}) / limit{filters} * 100\n)\nmy_resource_utilization(\n node_memory_MemFree_bytes,\n node_memory_MemTotal_bytes,\n {instance=~"$node:$port",job=~"$job"},\n)'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Now the template function my_resource_utilization() may be used for monitoring arbitrary resources - memory, CPU, network, storage, you name it."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Let's take another nice query from ",Ct(Vm,{})," dashboard:"]}),Ct($m,{code:'(\n (\n (\n count(\n count(node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}) by (cpu)\n )\n )\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',instance=~"$node:$port",job=~"$job"}[5m]))\n )\n )\n *\n 100\n)\n /\ncount(\n count(node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}) by (cpu)\n)'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Do you understand what does this mess do? Is it manageable? :) WITH expressions are happy to help in a few iterations."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"1. Extract common filters used in multiple places into a commonFilters variable:"}),Ct($m,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"}\n)\n(\n (\n (\n count(\n count(node_cpu_seconds_total{commonFilters}) by (cpu)\n )\n )\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n )\n )\n *\n 100\n)\n /\ncount(\n count(node_cpu_seconds_total{commonFilters}) by (cpu)\n)'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:'2. Extract "count(count(...) by (cpu))" into cpuCount variable:'}),Ct($m,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(node_cpu_seconds_total{commonFilters}) by (cpu))\n)\n(\n (\n cpuCount\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n )\n )\n *\n 100\n) / cpuCount'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"3. Extract rate(...) part into cpuIdle variable, since it is clear now that this part calculates the number of idle CPUs:"}),Ct($m,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(node_cpu_seconds_total{commonFilters}) by (cpu)),\n cpuIdle = sum(rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n)\n((cpuCount - cpuIdle) * 100) / cpuCount'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["4. Put node_cpu_seconds_total","{commonFilters}"," into its own varialbe with the name cpuSeconds:"]}),Ct($m,{code:'WITH (\n cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(cpuSeconds) by (cpu)),\n cpuIdle = sum(rate(cpuSeconds{mode=\'idle\'}[5m]))\n)\n((cpuCount - cpuIdle) * 100) / cpuCount'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Now the query became more clear comparing to the initial query."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"WITH expressions may be nested and may be put anywhere. Try expanding the following query:"}),Ct($m,{code:"WITH (\n f(a, b) = WITH (\n f1(x) = b-x,\n f2(x) = x+x\n ) f1(a)*f2(b)\n) f(foo, with(x=bar) x)"})]})]}),Bm=()=>{const{serverUrl:e}=Nt(),[t,n]=je(),[a,i]=(0,r.useState)(""),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)();return{data:a,error:s,loading:o,expand:async r=>{t.set("expr",r),n(t);const a=((e,t)=>"".concat(e,"/expand-with-exprs?query=").concat(encodeURIComponent(t),"&format=json"))(e,r);l(!0);try{const e=await fetch(a),t=await e.json();i((null===t||void 0===t?void 0:t.expr)||""),c(String(t.error||""))}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&c("".concat(ap.name,": ").concat(ap.message))}l(!1)}}},qm=()=>{const[e]=je(),{data:t,loading:n,error:a,expand:i}=Bm(),[o,l]=(0,r.useState)(e.get("expr")||""),s=()=>{i(o)};return(0,r.useEffect)((()=>{o&&i(o)}),[]),Ct("section",{className:"vm-with-template",children:[n&&Ct(Dd,{}),Ct("div",{className:"vm-with-template-body vm-block",children:[Ct("div",{className:"vm-with-template-body__expr",children:Ct($a,{type:"textarea",label:"MetricsQL query with optional WITH expressions",value:o,error:a,autofocus:!0,onEnter:s,onChange:e=>{l(e)}})}),Ct("div",{className:"vm-with-template-body__result",children:Ct($a,{type:"textarea",label:"MetricsQL query after expanding WITH expressions and applying other optimizations",value:t,disabled:!0})}),Ct("div",{className:"vm-with-template-body-top",children:Ct(aa,{variant:"contained",onClick:s,startIcon:Ct(Yn,{}),children:"Expand"})})]}),Ct("div",{className:"vm-block",children:Ct(Um,{})})]})},Ym=()=>{const{serverUrl:e}=Nt(),[t,n]=(0,r.useState)(null),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)();return{data:t,error:o,loading:a,fetchData:async(t,r)=>{const a=((e,t,n)=>{const r=["format=json","relabel_configs=".concat(encodeURIComponent(t)),"metric=".concat(encodeURIComponent(n))];return"".concat(e,"/metric-relabel-debug?").concat(r.join("&"))})(e,t,r);i(!0);try{const e=await fetch(a),t=await e.json();n(t.error?null:t),l(String(t.error||""))}catch(ap){ap instanceof Error&&"AbortError"!==ap.name&&l("".concat(ap.name,": ").concat(ap.message))}i(!1)}}},Wm={config:'- if: \'{bar_label=~"b.*"}\'\n source_labels: [foo_label, bar_label]\n separator: "_"\n target_label: foobar\n- action: labeldrop\n regex: "foo_.*"\n- target_label: job\n replacement: "my-application-2"',labels:'{__name__="my_metric", bar_label="bar", foo_label="foo", job="my-application", instance="192.168.0.1"}'},Km=()=>{const[e,t]=je(),{data:n,loading:a,error:i,fetchData:o}=Ym(),[l,s]=Yh("","config"),[c,u]=Yh("","labels"),d=(0,r.useCallback)((()=>{o(l,c),e.set("config",l),e.set("labels",c),t(e)}),[l,c]);return(0,r.useEffect)((()=>{const t=e.get("config")||"",n=e.get("labels")||"";(n||t)&&(o(t,n),s(t),u(n))}),[]),Ct("section",{className:"vm-relabeling",children:[a&&Ct(Dd,{}),Ct("div",{className:"vm-relabeling-header vm-block",children:[Ct("div",{className:"vm-relabeling-header-configs",children:Ct($a,{type:"textarea",label:"Relabel configs",value:l,autofocus:!0,onChange:e=>{s(e||"")},onEnter:d})}),Ct("div",{className:"vm-relabeling-header__labels",children:Ct($a,{type:"textarea",label:"Labels",value:c,onChange:e=>{u(e||"")},onEnter:d})}),Ct("div",{className:"vm-relabeling-header-bottom",children:[Ct("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/relabeling.html",rel:"help noreferrer",children:[Ct(Rn,{}),"Relabeling cookbook"]}),Ct("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/vmagent.html#relabeling",rel:"help noreferrer",children:[Ct(lr,{}),"Documentation"]}),Ct(aa,{variant:"text",onClick:()=>{const{config:n,labels:r}=Wm;s(n),u(r),o(n,r),e.set("config",n),e.set("labels",r),t(e)},children:"Try example"}),Ct(aa,{variant:"contained",onClick:d,startIcon:Ct(Yn,{}),children:"Submit"})]})]}),i&&Ct(Qr,{variant:"error",children:i}),n&&Ct("div",{className:"vm-relabeling-steps vm-block",children:[n.originalLabels&&Ct("div",{className:"vm-relabeling-steps-item",children:Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Original labels:"}),Ct("code",{dangerouslySetInnerHTML:{__html:n.originalLabels}})]})}),n.steps.map(((e,t)=>Ct("div",{className:"vm-relabeling-steps-item",children:[Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Step:"}),t+1]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Relabeling Rule:"}),Ct("code",{children:Ct("pre",{children:e.rule})})]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Input Labels:"}),Ct("code",{children:Ct("pre",{dangerouslySetInnerHTML:{__html:e.inLabels}})})]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Output labels:"}),Ct("code",{children:Ct("pre",{dangerouslySetInnerHTML:{__html:e.outLabels}})})]})]},t))),n.resultingLabels&&Ct("div",{className:"vm-relabeling-steps-item",children:Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Resulting labels:"}),Ct("code",{dangerouslySetInnerHTML:{__html:n.resultingLabels}})]})})]})]})},Qm=e=>{let{rows:t,columns:n,defaultOrderBy:a,copyToClipboard:i,paginationOffset:o}=e;const[l,s]=(0,r.useState)(a),[c,u]=(0,r.useState)("desc"),[d,h]=(0,r.useState)(null),m=(0,r.useMemo)((()=>{const{startIndex:e,endIndex:n}=o;return Jh(t,Gh(c,l)).slice(e,n)}),[t,l,c,o]),p=(e,t)=>async()=>{if(d!==t)try{await navigator.clipboard.writeText(String(e)),h(t)}catch(ap){console.error(ap)}};return(0,r.useEffect)((()=>{if(null===d)return;const e=setTimeout((()=>h(null)),2e3);return()=>clearTimeout(e)}),[d]),Ct("table",{className:"vm-table",children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[n.map((e=>{return Ct("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:(t=e.key,()=>{u((e=>"asc"===e&&l===t?"desc":"asc")),s(t)}),children:Ct("div",{className:"vm-table-cell__content",children:[Ct("div",{children:String(e.title||e.key)}),Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":l===e.key,"vm-table__sort-icon_desc":"desc"===c&&l===e.key}),children:Ct(Hn,{})})]})},String(e.key));var t})),i&&Ct("th",{className:"vm-table-cell vm-table-cell_header"})]})}),Ct("tbody",{className:"vm-table-body",children:m.map(((e,t)=>Ct("tr",{className:"vm-table__row",children:[n.map((t=>Ct("td",{className:xr()({"vm-table-cell":!0,["".concat(t.className)]:t.className}),children:e[t.key]||"-"},String(t.key)))),i&&Ct("td",{className:"vm-table-cell vm-table-cell_right",children:e[i]&&Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:d===t?"Copied":"Copy row",children:Ct(aa,{variant:"text",color:d===t?"success":"gray",size:"small",startIcon:Ct(d===t?er:ar,{}),onClick:p(e[i],t),ariaLabel:"copy row"})})})})]},t)))})]})},Zm=()=>{const{isMobile:e}=Wr(),{timezone:t}=vn(),{data:n,lastUpdated:a,isLoading:o,error:l,fetchData:s}=(()=>{const{serverUrl:e}=Nt(),[t,n]=(0,r.useState)([]),[a,o]=(0,r.useState)(i()().format(Ot)),[l,s]=(0,r.useState)(!1),[c,u]=(0,r.useState)(),d=(0,r.useMemo)((()=>"".concat(e,"/api/v1/status/active_queries")),[e]),h=async()=>{s(!0);try{const e=await fetch(d),t=await e.json();n(t.data),o(i()().format("HH:mm:ss:SSS")),e.ok?u(void 0):u("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(ap){ap instanceof Error&&u("".concat(ap.name,": ").concat(ap.message))}s(!1)};return(0,r.useEffect)((()=>{h().catch(console.error)}),[d]),{data:t,lastUpdated:a,isLoading:l,error:c,fetchData:h}})(),c=(0,r.useMemo)((()=>n.map((e=>{const t=i()(e.start).tz().format(Lt),n=i()(e.end).tz().format(Lt);return{duration:e.duration,remote_addr:e.remote_addr,query:e.query,args:"".concat(t," to ").concat(n,", step=").concat(qt(e.step)),data:JSON.stringify(e,null,2)}}))),[n,t]),u=(0,r.useMemo)((()=>{if(null===c||void 0===c||!c.length)return[];const e=Object.keys(c[0]),t={remote_addr:"client address"},n=["data"];return e.filter((e=>!n.includes(e))).map((e=>({key:e,title:t[e]||e})))}),[c]);return Ct("div",{className:"vm-active-queries",children:[o&&Ct(Dd,{}),Ct("div",{className:"vm-active-queries-header",children:[!c.length&&!l&&Ct(Qr,{variant:"info",children:"There are currently no active queries running"}),l&&Ct(Qr,{variant:"error",children:l}),Ct("div",{className:"vm-active-queries-header-controls",children:[Ct(aa,{variant:"contained",onClick:async()=>{s().catch(console.error)},startIcon:Ct(jn,{}),children:"Update"}),Ct("div",{className:"vm-active-queries-header__update-msg",children:["Last updated: ",a]})]})]}),!!c.length&&Ct("div",{className:xr()({"vm-block":!0,"vm-block_mobile":e}),children:Ct(Qm,{rows:c,columns:u,defaultOrderBy:"duration",copyToClipboard:"data",paginationOffset:{startIndex:0,endIndex:1/0}})})]})},Gm=e=>{let{onClose:t,onUpload:n}=e;const{isMobile:a}=Wr(),[i,o]=(0,r.useState)(""),[l,s]=(0,r.useState)(""),c=(0,r.useMemo)((()=>{try{return JSON.parse(i),""}catch(ap){return ap instanceof Error?ap.message:"Unknown error"}}),[i]),u=()=>{s(c),c||(n(i),t())};return Ct("div",{className:xr()({"vm-json-form vm-json-form_one-field":!0,"vm-json-form_mobile vm-json-form_one-field_mobile":a}),children:[Ct($a,{value:i,label:"JSON",type:"textarea",error:l,autofocus:!0,onChange:e=>{s(""),o(e)},onEnter:u}),Ct("div",{className:"vm-json-form-footer",children:Ct("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Ct(aa,{variant:"outlined",color:"error",onClick:t,children:"Cancel"}),Ct(aa,{variant:"contained",onClick:u,children:"apply"})]})})]})},Jm=e=>{let{data:t,period:n}=e;const{isMobile:a}=Wr(),{tableCompact:i}=Fr(),o=jr(),[l,s]=(0,r.useState)([]),[c,u]=(0,r.useState)(),[d,h]=(0,r.useState)(),[m,p]=(0,r.useState)(!1),[f,v]=(0,r.useState)([]),[g,y]=(0,r.useState)(),_=(0,r.useMemo)((()=>bh(d||[]).map((e=>e.key))),[d]),b=(0,r.useMemo)((()=>{const e=t.some((e=>"matrix"===e.data.resultType));return t.some((e=>"vector"===e.data.resultType))&&e?Tr:e?Tr.filter((e=>"chart"===e.value)):Tr.filter((e=>"chart"!==e.value))}),[t]),[w,k]=(0,r.useState)(b[0].value),{yaxis:x,spanGaps:S}=Ur(),C=Br(),E=e=>{C({type:"SET_YAXIS_LIMITS",payload:e})};return(0,r.useEffect)((()=>{const e="chart"===w?"matrix":"vector",n=t.filter((t=>t.data.resultType===e&&t.trace)).map((e=>{var t,n;return e.trace?new Pd(e.trace,(null===e||void 0===e||null===(t=e.vmui)||void 0===t||null===(n=t.params)||void 0===n?void 0:n.query)||"Query"):null}));s(n.filter(Boolean))}),[t,w]),(0,r.useEffect)((()=>{const e=[],n=[],r=[];t.forEach(((t,a)=>{const i=t.data.result.map((e=>{var n,r,i;return{...e,group:Number(null!==(n=null===(r=t.vmui)||void 0===r||null===(i=r.params)||void 0===i?void 0:i.id)&&void 0!==n?n:a)+1}}));var o,l;"matrix"===t.data.resultType?(n.push(...i),e.push((null===(o=t.vmui)||void 0===o||null===(l=o.params)||void 0===l?void 0:l.query)||"Query")):r.push(...i)})),v(e),u(n),h(r)}),[t]),(0,r.useEffect)((()=>{p(!!c&&Gc(c))}),[c]),Ct("div",{className:xr()({"vm-query-analyzer-view":!0,"vm-query-analyzer-view_mobile":a}),children:[!!l.length&&Ct(Yd,{traces:l,onDeleteClick:e=>{s((t=>t.filter((t=>t.idValue!==e.idValue))))}}),Ct("div",{className:xr()({"vm-block":!0,"vm-block_mobile":a}),children:[Ct("div",{className:"vm-custom-panel-body-header",children:[Ct("div",{className:"vm-custom-panel-body-header__tabs",children:Ct(Mr,{activeItem:w,items:b,onChange:e=>{k(e)}})}),Ct("div",{className:"vm-custom-panel-body-header__graph-controls",children:["chart"===w&&Ct(ba,{}),"chart"===w&&Ct(gh,{yaxis:x,setYaxisLimits:E,toggleEnableLimits:()=>{C({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})},spanGaps:{value:S,onChange:e=>{C({type:"SET_SPAN_GAPS",payload:e})}}}),"table"===w&&Ct(Ch,{columns:_,defaultColumns:g,onChangeColumns:y,tableCompact:i,toggleTableCompact:()=>{o({type:"TOGGLE_TABLE_COMPACT"})}})]})]}),c&&n&&"chart"===w&&Ct(mh,{data:c,period:n,customStep:n.step||"1s",query:f,yaxis:x,setYaxisLimits:E,setPeriod:()=>null,height:a?.5*window.innerHeight:500,isHistogram:m,spanGaps:S}),d&&"code"===w&&Ct(_h,{data:d}),d&&"table"===w&&Ct(kh,{data:d,displayColumns:g})]})]})},Xm=e=>{var t,n;let{data:a,period:o}=e;const l=(0,r.useMemo)((()=>a.filter((e=>e.stats&&"matrix"===e.data.resultType))),[a]),s=(0,r.useMemo)((()=>{var e,t;return null===(e=a.find((e=>{var t;return null===e||void 0===e||null===(t=e.vmui)||void 0===t?void 0:t.comment})))||void 0===e||null===(t=e.vmui)||void 0===t?void 0:t.comment}),[a]),c=(0,r.useMemo)((()=>{if(!o)return"";const e=i()(1e3*o.start).tz().format(Lt),t=i()(1e3*o.end).tz().format(Lt);return"".concat(e," - ").concat(t)}),[o]),{value:u,setTrue:d,setFalse:h}=oa(!1);return Ct(xt.FK,{children:[Ct("div",{className:"vm-query-analyzer-info-header",children:[Ct(aa,{startIcon:Ct(Rn,{}),variant:"outlined",color:"warning",onClick:d,children:"Show report info"}),o&&Ct(xt.FK,{children:[Ct("div",{className:"vm-query-analyzer-info-header__period",children:[Ct(or,{})," step: ",o.step]}),Ct("div",{className:"vm-query-analyzer-info-header__period",children:[Ct(Vn,{})," ",c]})]})]}),u&&Ct(pa,{title:"Report info",onClose:h,children:Ct("div",{className:"vm-query-analyzer-info",children:[s&&Ct("div",{className:"vm-query-analyzer-info-item vm-query-analyzer-info-item_comment",children:[Ct("div",{className:"vm-query-analyzer-info-item__title",children:"Comment:"}),Ct("div",{className:"vm-query-analyzer-info-item__text",children:s})]}),l.map(((e,t)=>{var n;return Ct("div",{className:"vm-query-analyzer-info-item",children:[Ct("div",{className:"vm-query-analyzer-info-item__title",children:l.length>1?"Query ".concat(t+1,":"):"Stats:"}),Ct("div",{className:"vm-query-analyzer-info-item__text",children:[Object.entries(e.stats||{}).map((e=>{let[t,n]=e;return Ct("div",{children:[t,": ",null!==n&&void 0!==n?n:"-"]},t)})),"isPartial: ",String(null!==(n=e.isPartial)&&void 0!==n?n:"-")]})]},t)})),Ct("div",{className:"vm-query-analyzer-info-type",children:null!==(t=l[0])&&void 0!==t&&null!==(n=t.vmui)&&void 0!==n&&n.params?"The report was created using vmui":"The report was created manually"})]})})]})},ep=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)(""),i=(0,r.useMemo)((()=>!!e.length),[e]),{value:o,setTrue:l,setFalse:s}=oa(!1),c=(0,r.useMemo)((()=>{var t,n;if(!e)return;const r=null===(t=e[0])||void 0===t||null===(n=t.vmui)||void 0===n?void 0:n.params,a={start:+((null===r||void 0===r?void 0:r.start)||0),end:+((null===r||void 0===r?void 0:r.end)||0),step:null===r||void 0===r?void 0:r.step,date:""};if(!r){const t=e.filter((e=>"matrix"===e.data.resultType)).map((e=>e.data.result)).flat().map((e=>{var t;return e.values?null===(t=e.values)||void 0===t?void 0:t.map((e=>e[0])):[0]})).flat(),n=Array.from(new Set(t.filter(Boolean))).sort(((e,t)=>e-t));a.start=n[0],a.end=n[n.length-1],a.step=Bt((e=>{const t=e.slice(1).map(((t,n)=>t-e[n])),n={};t.forEach((e=>{const t=e.toString();n[t]=(n[t]||0)+1}));let r=0,a=0;for(const i in n)n[i]>a&&(a=n[i],r=Number(i));return r})(n))}return a.date=Zt(Xt(a.end)),a}),[e]),u=e=>{try{const n=JSON.parse(e),r=Array.isArray(n)?n:[n];(e=>e.every((e=>{if("object"===typeof e&&null!==e){const t=e.data;if("object"===typeof t&&null!==t){const e=t.result,n=t.resultType;return Array.isArray(e)&&"string"===typeof n}}return!1})))(r)?t(r):a("Invalid structure - JSON does not match the expected format")}catch(ap){ap instanceof Error&&a("".concat(ap.name,": ").concat(ap.message))}},d=e=>{e.map((e=>{const t=new FileReader;t.onload=e=>{var t;const n=String(null===(t=e.target)||void 0===t?void 0:t.result);u(n)},t.readAsText(e)}))},h=e=>{a("");const t=Array.from(e.target.files||[]);d(t),e.target.value=""},{files:m,dragging:p}=bm();return(0,r.useEffect)((()=>{d(m)}),[m]),Ct("div",{className:"vm-trace-page",children:[i&&Ct("div",{className:"vm-trace-page-header",children:[Ct("div",{className:"vm-trace-page-header-errors",children:Ct(Xm,{data:e,period:c})}),Ct("div",{children:Ct(wm,{onOpenModal:l,onChange:h})})]}),n&&Ct("div",{className:"vm-trace-page-header-errors-item vm-trace-page-header-errors-item_margin-bottom",children:[Ct(Qr,{variant:"error",children:n}),Ct(aa,{className:"vm-trace-page-header-errors-item__close",startIcon:Ct(Pn,{}),variant:"text",color:"error",onClick:()=>{a("")}})]}),i&&Ct(Jm,{data:e,period:c}),!i&&Ct("div",{className:"vm-trace-page-preview",children:[Ct("p",{className:"vm-trace-page-preview__text",children:["Please, upload file with JSON response content.","\n","The file must contain query information in JSON format.","\n","Graph will be displayed after file upload.","\n","Attach files by dragging & dropping, selecting or pasting them."]}),Ct(wm,{onOpenModal:l,onChange:h})]}),o&&Ct(pa,{title:"Paste JSON",onClose:s,children:Ct(Gm,{onClose:s,onUpload:u})}),p&&Ct("div",{className:"vm-trace-page__dropzone"})]})},tp=()=>{const[e,t]=(0,r.useState)(!1);return Ct(xt.FK,{children:Ct(Le,{children:Ct(ta,{children:Ct(xt.FK,{children:[Ct(_m,{onLoaded:t}),e&&Ct(ke,{children:Ct(be,{path:"/",element:Ct(Ai,{}),children:[Ct(be,{path:Ye.home,element:Ct(Lh,{})}),Ct(be,{path:Ye.metrics,element:Ct(zm,{})}),Ct(be,{path:Ye.cardinality,element:Ct(sm,{})}),Ct(be,{path:Ye.topQueries,element:Ct(pm,{})}),Ct(be,{path:Ye.trace,element:Ct(km,{})}),Ct(be,{path:Ye.queryAnalyzer,element:Ct(ep,{})}),Ct(be,{path:Ye.dashboards,element:Ct(Ih,{})}),Ct(be,{path:Ye.withTemplate,element:Ct(qm,{})}),Ct(be,{path:Ye.relabel,element:Ct(Km,{})}),Ct(be,{path:Ye.activeQueries,element:Ct(Zm,{})}),Ct(be,{path:Ye.icons,element:Ct(Fm,{})})]})})]})})})})},np=e=>{e&&n.e(685).then(n.bind(n,685)).then((t=>{let{getCLS:n,getFID:r,getFCP:a,getLCP:i,getTTFB:o}=t;n(e),r(e),a(e),i(e),o(e)}))},rp=document.getElementById("root");rp&&(0,r.render)(Ct(tp,{}),rp),np()})()})(); \ No newline at end of file diff --git a/app/vmselect/vmui/static/js/main.59c17910.js b/app/vmselect/vmui/static/js/main.59c17910.js new file mode 100644 index 000000000..49c700607 --- /dev/null +++ b/app/vmselect/vmui/static/js/main.59c17910.js @@ -0,0 +1,2 @@ +/*! For license information please see main.59c17910.js.LICENSE.txt */ +(()=>{var e={61:(e,t,n)=>{"use strict";var r=n(375),a=n(629),i=a(r("String.prototype.indexOf"));e.exports=function(e,t){var n=r(e,!!t);return"function"===typeof n&&i(e,".prototype.")>-1?a(n):n}},629:(e,t,n)=>{"use strict";var r=n(989),a=n(375),i=n(259),o=n(277),l=a("%Function.prototype.apply%"),s=a("%Function.prototype.call%"),c=a("%Reflect.apply%",!0)||r.call(s,l),u=n(709),d=a("%Math.max%");e.exports=function(e){if("function"!==typeof e)throw new o("a function is required");var t=c(r,s,arguments);return i(t,1+d(0,e.length-(arguments.length-1)),!0)};var h=function(){return c(r,l,arguments)};u?u(e.exports,"apply",{value:h}):e.exports.apply=h},159:function(e){e.exports=function(){"use strict";var e=1e3,t=6e4,n=36e5,r="millisecond",a="second",i="minute",o="hour",l="day",s="week",c="month",u="quarter",d="year",h="date",m="Invalid Date",p=/^(\d{4})[-/]?(\d{1,2})?[-/]?(\d{0,2})[Tt\s]*(\d{1,2})?:?(\d{1,2})?:?(\d{1,2})?[.:]?(\d+)?$/,f=/\[([^\]]+)]|Y{1,4}|M{1,4}|D{1,2}|d{1,4}|H{1,2}|h{1,2}|a|A|m{1,2}|s{1,2}|Z{1,2}|SSS/g,v={name:"en",weekdays:"Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),months:"January_February_March_April_May_June_July_August_September_October_November_December".split("_"),ordinal:function(e){var t=["th","st","nd","rd"],n=e%100;return"["+e+(t[(n-20)%10]||t[n]||t[0])+"]"}},g=function(e,t,n){var r=String(e);return!r||r.length>=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),a=n%60;return(t<=0?"+":"-")+g(r,2,"0")+":"+g(a,2,"0")},m:function e(t,n){if(t.date()1)return e(o[0])}else{var l=t.name;b[l]=t,a=l}return!r&&a&&(_=a),a||!r&&_},S=function(e,t){if(k(e))return e.clone();var n="object"==typeof t?t:{};return n.date=e,n.args=arguments,new E(n)},C=y;C.l=x,C.i=k,C.w=function(e,t){return S(e,{locale:t.$L,utc:t.$u,x:t.$x,$offset:t.$offset})};var E=function(){function v(e){this.$L=x(e.locale,null,!0),this.parse(e),this.$x=this.$x||e.x||{},this[w]=!0}var g=v.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(C.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 a=r[2]-1||0,i=(r[7]||"0").substring(0,3);return n?new Date(Date.UTC(r[1],a,r[3]||1,r[4]||0,r[5]||0,r[6]||0,i)):new Date(r[1],a,r[3]||1,r[4]||0,r[5]||0,r[6]||0,i)}}return new Date(t)}(e),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 C},g.isValid=function(){return!(this.$d.toString()===m)},g.isSame=function(e,t){var n=S(e);return this.startOf(t)<=n&&n<=this.endOf(t)},g.isAfter=function(e,t){return S(e)=0&&(i[d]=parseInt(u,10))}var h=i[3],m=24===h?0:h,p=i[0]+"-"+i[1]+"-"+i[2]+" "+m+":"+i[4]+":"+i[5]+":000",f=+t;return(a.utc(p).valueOf()-(f-=f%1e3))/6e4},s=r.prototype;s.tz=function(e,t){void 0===e&&(e=i);var n=this.utcOffset(),r=this.toDate(),o=r.toLocaleString("en-US",{timeZone:e}),l=Math.round((r-new Date(o))/1e3/60),s=a(o,{locale:this.$L}).$set("millisecond",this.$ms).utcOffset(15*-Math.round(r.getTimezoneOffset()/15)-l,!0);if(t){var c=s.utcOffset();s=s.add(n-c,"minute")}return s.$x.$timezone=e,s},s.offsetName=function(e){var t=this.$x.$timezone||a.tz.guess(),n=o(this.valueOf(),t,{timeZoneName:e}).find((function(e){return"timezonename"===e.type.toLowerCase()}));return n&&n.value};var c=s.startOf;s.startOf=function(e,t){if(!this.$x||!this.$x.$timezone)return c.call(this,e,t);var n=a(this.format("YYYY-MM-DD HH:mm:ss:SSS"),{locale:this.$L});return c.call(n,e,t).tz(this.$x.$timezone,!0)},a.tz=function(e,t,n){var r=n&&t,o=n||t||i,s=l(+a(),o);if("string"!=typeof e)return a(e).tz(o);var c=function(e,t,n){var r=e-60*t*1e3,a=l(r,n);if(t===a)return[r,t];var i=l(r-=60*(a-t)*1e3,n);return a===i?[r,a]:[e-60*Math.min(a,i)*1e3,Math.max(a,i)]}(a.utc(e,r).valueOf(),s,o),u=c[0],d=c[1],h=a(u).utcOffset(d);return h.$x.$timezone=o,h},a.tz.guess=function(){return Intl.DateTimeFormat().resolvedOptions().timeZone},a.tz.setDefault=function(e){i=e}}}()},220:function(e){e.exports=function(){"use strict";var e="minute",t=/[+-]\d\d(?::?\d\d)?/g,n=/([+-]|\d\d)/g;return function(r,a,i){var o=a.prototype;i.utc=function(e){return new a({date:e,utc:!0,args:arguments})},o.utc=function(t){var n=i(this.toDate(),{locale:this.$L,utc:!0});return t?n.add(this.utcOffset(),e):n},o.local=function(){return i(this.toDate(),{locale:this.$L,utc:!1})};var l=o.parse;o.parse=function(e){e.utc&&(this.$u=!0),this.$utils().u(e.$offset)||(this.$offset=e.$offset),l.call(this,e)};var s=o.init;o.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 s.call(this)};var c=o.utcOffset;o.utcOffset=function(r,a){var i=this.$utils().u;if(i(r))return this.$u?0:i(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 a=(""+r[0]).match(n)||["-",0,0],i=a[0],o=60*+a[1]+ +a[2];return 0===o?0:"+"===i?o:-o}(r),null===r))return this;var o=Math.abs(r)<=16?60*r:r,l=this;if(a)return l.$offset=o,l.$u=0===r,l;if(0!==r){var s=this.$u?this.toDate().getTimezoneOffset():-1*this.utcOffset();(l=this.local().add(o+s,e)).$offset=o,l.$x.$localOffset=s}else l=this.utc();return l};var u=o.format;o.format=function(e){var t=e||(this.$u?"YYYY-MM-DDTHH:mm:ss[Z]":"");return u.call(this,t)},o.valueOf=function(){var e=this.$utils().u(this.$offset)?0:this.$offset+(this.$x.$localOffset||this.$d.getTimezoneOffset());return this.$d.valueOf()-6e4*e},o.isUTC=function(){return!!this.$u},o.toISOString=function(){return this.toDate().toISOString()},o.toString=function(){return this.toDate().toUTCString()};var d=o.toDate;o.toDate=function(e){return"s"===e&&this.$offset?i(this.format("YYYY-MM-DD HH:mm:ss:SSS")).toDate():d.call(this)};var h=o.diff;o.diff=function(e,t,n){if(e&&this.$u===e.$u)return h.call(this,e,t,n);var r=this.local(),a=i(e).local();return h.call(r,a,t,n)}}}()},411:(e,t,n)=>{"use strict";var r=n(709),a=n(430),i=n(277),o=n(553);e.exports=function(e,t,n){if(!e||"object"!==typeof e&&"function"!==typeof e)throw new i("`obj` must be an object or a function`");if("string"!==typeof t&&"symbol"!==typeof t)throw new i("`property` must be a string or a symbol`");if(arguments.length>3&&"boolean"!==typeof arguments[3]&&null!==arguments[3])throw new i("`nonEnumerable`, if provided, must be a boolean or null");if(arguments.length>4&&"boolean"!==typeof arguments[4]&&null!==arguments[4])throw new i("`nonWritable`, if provided, must be a boolean or null");if(arguments.length>5&&"boolean"!==typeof arguments[5]&&null!==arguments[5])throw new i("`nonConfigurable`, if provided, must be a boolean or null");if(arguments.length>6&&"boolean"!==typeof arguments[6])throw new i("`loose`, if provided, must be a boolean");var l=arguments.length>3?arguments[3]:null,s=arguments.length>4?arguments[4]:null,c=arguments.length>5?arguments[5]:null,u=arguments.length>6&&arguments[6],d=!!o&&o(e,t);if(r)r(e,t,{configurable:null===c&&d?d.configurable:!c,enumerable:null===l&&d?d.enumerable:!l,value:n,writable:null===s&&d?d.writable:!s});else{if(!u&&(l||s||c))throw new a("This environment does not support defining a property as non-configurable, non-writable, or non-enumerable.");e[t]=n}}},709:(e,t,n)=>{"use strict";var r=n(375)("%Object.defineProperty%",!0)||!1;if(r)try{r({},"a",{value:1})}catch(a){r=!1}e.exports=r},123:e=>{"use strict";e.exports=EvalError},953:e=>{"use strict";e.exports=Error},780:e=>{"use strict";e.exports=RangeError},768:e=>{"use strict";e.exports=ReferenceError},430:e=>{"use strict";e.exports=SyntaxError},277:e=>{"use strict";e.exports=TypeError},619:e=>{"use strict";e.exports=URIError},307:e=>{"use strict";var t=Object.prototype.toString,n=Math.max,r=function(e,t){for(var n=[],r=0;r{"use strict";var r=n(307);e.exports=Function.prototype.bind||r},375:(e,t,n)=>{"use strict";var r,a=n(953),i=n(123),o=n(780),l=n(768),s=n(430),c=n(277),u=n(619),d=Function,h=function(e){try{return d('"use strict"; return ('+e+").constructor;")()}catch(t){}},m=Object.getOwnPropertyDescriptor;if(m)try{m({},"")}catch(R){m=null}var p=function(){throw new c},f=m?function(){try{return p}catch(e){try{return m(arguments,"callee").get}catch(t){return p}}}():p,v=n(757)(),g=n(442)(),y=Object.getPrototypeOf||(g?function(e){return e.__proto__}:null),_={},b="undefined"!==typeof Uint8Array&&y?y(Uint8Array):r,w={__proto__:null,"%AggregateError%":"undefined"===typeof AggregateError?r:AggregateError,"%Array%":Array,"%ArrayBuffer%":"undefined"===typeof ArrayBuffer?r:ArrayBuffer,"%ArrayIteratorPrototype%":v&&y?y([][Symbol.iterator]()):r,"%AsyncFromSyncIteratorPrototype%":r,"%AsyncFunction%":_,"%AsyncGenerator%":_,"%AsyncGeneratorFunction%":_,"%AsyncIteratorPrototype%":_,"%Atomics%":"undefined"===typeof Atomics?r:Atomics,"%BigInt%":"undefined"===typeof BigInt?r:BigInt,"%BigInt64Array%":"undefined"===typeof BigInt64Array?r:BigInt64Array,"%BigUint64Array%":"undefined"===typeof BigUint64Array?r:BigUint64Array,"%Boolean%":Boolean,"%DataView%":"undefined"===typeof DataView?r:DataView,"%Date%":Date,"%decodeURI%":decodeURI,"%decodeURIComponent%":decodeURIComponent,"%encodeURI%":encodeURI,"%encodeURIComponent%":encodeURIComponent,"%Error%":a,"%eval%":eval,"%EvalError%":i,"%Float32Array%":"undefined"===typeof Float32Array?r:Float32Array,"%Float64Array%":"undefined"===typeof Float64Array?r:Float64Array,"%FinalizationRegistry%":"undefined"===typeof FinalizationRegistry?r:FinalizationRegistry,"%Function%":d,"%GeneratorFunction%":_,"%Int8Array%":"undefined"===typeof Int8Array?r:Int8Array,"%Int16Array%":"undefined"===typeof Int16Array?r:Int16Array,"%Int32Array%":"undefined"===typeof Int32Array?r:Int32Array,"%isFinite%":isFinite,"%isNaN%":isNaN,"%IteratorPrototype%":v&&y?y(y([][Symbol.iterator]())):r,"%JSON%":"object"===typeof JSON?JSON:r,"%Map%":"undefined"===typeof Map?r:Map,"%MapIteratorPrototype%":"undefined"!==typeof Map&&v&&y?y((new Map)[Symbol.iterator]()):r,"%Math%":Math,"%Number%":Number,"%Object%":Object,"%parseFloat%":parseFloat,"%parseInt%":parseInt,"%Promise%":"undefined"===typeof Promise?r:Promise,"%Proxy%":"undefined"===typeof Proxy?r:Proxy,"%RangeError%":o,"%ReferenceError%":l,"%Reflect%":"undefined"===typeof Reflect?r:Reflect,"%RegExp%":RegExp,"%Set%":"undefined"===typeof Set?r:Set,"%SetIteratorPrototype%":"undefined"!==typeof Set&&v&&y?y((new Set)[Symbol.iterator]()):r,"%SharedArrayBuffer%":"undefined"===typeof SharedArrayBuffer?r:SharedArrayBuffer,"%String%":String,"%StringIteratorPrototype%":v&&y?y(""[Symbol.iterator]()):r,"%Symbol%":v?Symbol:r,"%SyntaxError%":s,"%ThrowTypeError%":f,"%TypedArray%":b,"%TypeError%":c,"%Uint8Array%":"undefined"===typeof Uint8Array?r:Uint8Array,"%Uint8ClampedArray%":"undefined"===typeof Uint8ClampedArray?r:Uint8ClampedArray,"%Uint16Array%":"undefined"===typeof Uint16Array?r:Uint16Array,"%Uint32Array%":"undefined"===typeof Uint32Array?r:Uint32Array,"%URIError%":u,"%WeakMap%":"undefined"===typeof WeakMap?r:WeakMap,"%WeakRef%":"undefined"===typeof WeakRef?r:WeakRef,"%WeakSet%":"undefined"===typeof WeakSet?r:WeakSet};if(y)try{null.error}catch(R){var k=y(y(R));w["%Error.prototype%"]=k}var x=function e(t){var n;if("%AsyncFunction%"===t)n=h("async function () {}");else if("%GeneratorFunction%"===t)n=h("function* () {}");else if("%AsyncGeneratorFunction%"===t)n=h("async function* () {}");else if("%AsyncGenerator%"===t){var r=e("%AsyncGeneratorFunction%");r&&(n=r.prototype)}else if("%AsyncIteratorPrototype%"===t){var a=e("%AsyncGenerator%");a&&y&&(n=y(a.prototype))}return w[t]=n,n},S={__proto__:null,"%ArrayBufferPrototype%":["ArrayBuffer","prototype"],"%ArrayPrototype%":["Array","prototype"],"%ArrayProto_entries%":["Array","prototype","entries"],"%ArrayProto_forEach%":["Array","prototype","forEach"],"%ArrayProto_keys%":["Array","prototype","keys"],"%ArrayProto_values%":["Array","prototype","values"],"%AsyncFunctionPrototype%":["AsyncFunction","prototype"],"%AsyncGenerator%":["AsyncGeneratorFunction","prototype"],"%AsyncGeneratorPrototype%":["AsyncGeneratorFunction","prototype","prototype"],"%BooleanPrototype%":["Boolean","prototype"],"%DataViewPrototype%":["DataView","prototype"],"%DatePrototype%":["Date","prototype"],"%ErrorPrototype%":["Error","prototype"],"%EvalErrorPrototype%":["EvalError","prototype"],"%Float32ArrayPrototype%":["Float32Array","prototype"],"%Float64ArrayPrototype%":["Float64Array","prototype"],"%FunctionPrototype%":["Function","prototype"],"%Generator%":["GeneratorFunction","prototype"],"%GeneratorPrototype%":["GeneratorFunction","prototype","prototype"],"%Int8ArrayPrototype%":["Int8Array","prototype"],"%Int16ArrayPrototype%":["Int16Array","prototype"],"%Int32ArrayPrototype%":["Int32Array","prototype"],"%JSONParse%":["JSON","parse"],"%JSONStringify%":["JSON","stringify"],"%MapPrototype%":["Map","prototype"],"%NumberPrototype%":["Number","prototype"],"%ObjectPrototype%":["Object","prototype"],"%ObjProto_toString%":["Object","prototype","toString"],"%ObjProto_valueOf%":["Object","prototype","valueOf"],"%PromisePrototype%":["Promise","prototype"],"%PromiseProto_then%":["Promise","prototype","then"],"%Promise_all%":["Promise","all"],"%Promise_reject%":["Promise","reject"],"%Promise_resolve%":["Promise","resolve"],"%RangeErrorPrototype%":["RangeError","prototype"],"%ReferenceErrorPrototype%":["ReferenceError","prototype"],"%RegExpPrototype%":["RegExp","prototype"],"%SetPrototype%":["Set","prototype"],"%SharedArrayBufferPrototype%":["SharedArrayBuffer","prototype"],"%StringPrototype%":["String","prototype"],"%SymbolPrototype%":["Symbol","prototype"],"%SyntaxErrorPrototype%":["SyntaxError","prototype"],"%TypedArrayPrototype%":["TypedArray","prototype"],"%TypeErrorPrototype%":["TypeError","prototype"],"%Uint8ArrayPrototype%":["Uint8Array","prototype"],"%Uint8ClampedArrayPrototype%":["Uint8ClampedArray","prototype"],"%Uint16ArrayPrototype%":["Uint16Array","prototype"],"%Uint32ArrayPrototype%":["Uint32Array","prototype"],"%URIErrorPrototype%":["URIError","prototype"],"%WeakMapPrototype%":["WeakMap","prototype"],"%WeakSetPrototype%":["WeakSet","prototype"]},C=n(989),E=n(155),N=C.call(Function.call,Array.prototype.concat),A=C.call(Function.apply,Array.prototype.splice),M=C.call(Function.call,String.prototype.replace),T=C.call(Function.call,String.prototype.slice),L=C.call(Function.call,RegExp.prototype.exec),P=/[^%.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|%$))/g,O=/\\(\\)?/g,I=function(e,t){var n,r=e;if(E(S,r)&&(r="%"+(n=S[r])[0]+"%"),E(w,r)){var a=w[r];if(a===_&&(a=x(r)),"undefined"===typeof a&&!t)throw new c("intrinsic "+e+" exists, but is not available. Please file an issue!");return{alias:n,name:r,value:a}}throw new s("intrinsic "+e+" does not exist!")};e.exports=function(e,t){if("string"!==typeof e||0===e.length)throw new c("intrinsic name must be a non-empty string");if(arguments.length>1&&"boolean"!==typeof t)throw new c('"allowMissing" argument must be a boolean');if(null===L(/^%?[^%]*%?$/,e))throw new s("`%` may not be present anywhere but at the beginning and end of the intrinsic name");var n=function(e){var t=T(e,0,1),n=T(e,-1);if("%"===t&&"%"!==n)throw new s("invalid intrinsic syntax, expected closing `%`");if("%"===n&&"%"!==t)throw new s("invalid intrinsic syntax, expected opening `%`");var r=[];return M(e,P,(function(e,t,n,a){r[r.length]=n?M(a,O,"$1"):t||e})),r}(e),r=n.length>0?n[0]:"",a=I("%"+r+"%",t),i=a.name,o=a.value,l=!1,u=a.alias;u&&(r=u[0],A(n,N([0,1],u)));for(var d=1,h=!0;d=n.length){var g=m(o,p);o=(h=!!g)&&"get"in g&&!("originalValue"in g.get)?g.get:o[p]}else h=E(o,p),o=o[p];h&&!l&&(w[i]=o)}}return o}},553:(e,t,n)=>{"use strict";var r=n(375)("%Object.getOwnPropertyDescriptor%",!0);if(r)try{r([],"length")}catch(a){r=null}e.exports=r},734:(e,t,n)=>{"use strict";var r=n(709),a=function(){return!!r};a.hasArrayLengthDefineBug=function(){if(!r)return null;try{return 1!==r([],"length",{value:1}).length}catch(e){return!0}},e.exports=a},442:e=>{"use strict";var t={foo:{}},n=Object;e.exports=function(){return{__proto__:t}.foo===t.foo&&!({__proto__:null}instanceof n)}},757:(e,t,n)=>{"use strict";var r="undefined"!==typeof Symbol&&Symbol,a=n(175);e.exports=function(){return"function"===typeof r&&("function"===typeof Symbol&&("symbol"===typeof r("foo")&&("symbol"===typeof Symbol("bar")&&a())))}},175: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 a=Object.getOwnPropertyDescriptor(e,t);if(42!==a.value||!0!==a.enumerable)return!1}return!0}},155:(e,t,n)=>{"use strict";var r=Function.prototype.call,a=Object.prototype.hasOwnProperty,i=n(989);e.exports=i.call(r,a)},267:(e,t,n)=>{var r=NaN,a="[object Symbol]",i=/^\s+|\s+$/g,o=/^[-+]0x[0-9a-f]+$/i,l=/^0b[01]+$/i,s=/^0o[0-7]+$/i,c=parseInt,u="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,d="object"==typeof self&&self&&self.Object===Object&&self,h=u||d||Function("return this")(),m=Object.prototype.toString,p=Math.max,f=Math.min,v=function(){return h.Date.now()};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)&&m.call(e)==a}(e))return r;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=l.test(e);return n||s.test(e)?c(e.slice(2),n?2:8):o.test(e)?r:+e}e.exports=function(e,t,n){var r,a,i,o,l,s,c=0,u=!1,d=!1,h=!0;if("function"!=typeof e)throw new TypeError("Expected a function");function m(t){var n=r,i=a;return r=a=void 0,c=t,o=e.apply(i,n)}function _(e){var n=e-s;return void 0===s||n>=t||n<0||d&&e-c>=i}function b(){var e=v();if(_(e))return w(e);l=setTimeout(b,function(e){var n=t-(e-s);return d?f(n,i-(e-c)):n}(e))}function w(e){return l=void 0,h&&r?m(e):(r=a=void 0,o)}function k(){var e=v(),n=_(e);if(r=arguments,a=this,s=e,n){if(void 0===l)return function(e){return c=e,l=setTimeout(b,t),u?m(e):o}(s);if(d)return l=setTimeout(b,t),m(s)}return void 0===l&&(l=setTimeout(b,t)),o}return t=y(t)||0,g(n)&&(u=!!n.leading,i=(d="maxWait"in n)?p(y(n.maxWait)||0,t):i,h="trailing"in n?!!n.trailing:h),k.cancel=function(){void 0!==l&&clearTimeout(l),c=0,r=s=a=l=void 0},k.flush=function(){return void 0===l?o:w(v())},k}},424:(e,t,n)=>{var r="__lodash_hash_undefined__",a=1/0,i="[object Function]",o="[object GeneratorFunction]",l="[object Symbol]",s=/\.|\[(?:[^[\]]*|(["'])(?:(?!\1)[^\\]|\\.)*?\1)\]/,c=/^\w*$/,u=/^\./,d=/[^.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\\]|\\.)*?)\2)\]|(?=(?:\.|\[\])(?:\.|\[\]|$))/g,h=/\\(\\)?/g,m=/^\[object .+?Constructor\]$/,p="object"==typeof n.g&&n.g&&n.g.Object===Object&&n.g,f="object"==typeof self&&self&&self.Object===Object&&self,v=p||f||Function("return this")();var g=Array.prototype,y=Function.prototype,_=Object.prototype,b=v["__core-js_shared__"],w=function(){var e=/[^.]+$/.exec(b&&b.keys&&b.keys.IE_PROTO||"");return e?"Symbol(src)_1."+e:""}(),k=y.toString,x=_.hasOwnProperty,S=_.toString,C=RegExp("^"+k.call(x).replace(/[\\^$.*+?()[\]{}|]/g,"\\$&").replace(/hasOwnProperty|(function).*?(?=\\\()| for .+?(?=\\\])/g,"$1.*?")+"$"),E=v.Symbol,N=g.splice,A=j(v,"Map"),M=j(Object,"create"),T=E?E.prototype:void 0,L=T?T.toString:void 0;function P(e){var t=-1,n=e?e.length:0;for(this.clear();++t-1},O.prototype.set=function(e,t){var n=this.__data__,r=R(n,e);return r<0?n.push([e,t]):n[r][1]=t,this},I.prototype.clear=function(){this.__data__={hash:new P,map:new(A||O),string:new P}},I.prototype.delete=function(e){return F(this,e).delete(e)},I.prototype.get=function(e){return F(this,e).get(e)},I.prototype.has=function(e){return F(this,e).has(e)},I.prototype.set=function(e,t){return F(this,e).set(e,t),this};var H=$((function(e){var t;e=null==(t=e)?"":function(e){if("string"==typeof e)return e;if(q(e))return L?L.call(e):"";var t=e+"";return"0"==t&&1/e==-a?"-0":t}(t);var n=[];return u.test(e)&&n.push(""),e.replace(d,(function(e,t,r,a){n.push(r?a.replace(h,"$1"):t||e)})),n}));function V(e){if("string"==typeof e||q(e))return e;var t=e+"";return"0"==t&&1/e==-a?"-0":t}function $(e,t){if("function"!=typeof e||t&&"function"!=typeof t)throw new TypeError("Expected a function");var n=function(){var r=arguments,a=t?t.apply(this,r):r[0],i=n.cache;if(i.has(a))return i.get(a);var o=e.apply(this,r);return n.cache=i.set(a,o),o};return n.cache=new($.Cache||I),n}$.Cache=I;var U=Array.isArray;function B(e){var t=typeof e;return!!e&&("object"==t||"function"==t)}function q(e){return"symbol"==typeof e||function(e){return!!e&&"object"==typeof e}(e)&&S.call(e)==l}e.exports=function(e,t,n){var r=null==e?void 0:D(e,t);return void 0===r?n:r}},141:(e,t,n)=>{var r="function"===typeof Map&&Map.prototype,a=Object.getOwnPropertyDescriptor&&r?Object.getOwnPropertyDescriptor(Map.prototype,"size"):null,i=r&&a&&"function"===typeof a.get?a.get:null,o=r&&Map.prototype.forEach,l="function"===typeof Set&&Set.prototype,s=Object.getOwnPropertyDescriptor&&l?Object.getOwnPropertyDescriptor(Set.prototype,"size"):null,c=l&&s&&"function"===typeof s.get?s.get:null,u=l&&Set.prototype.forEach,d="function"===typeof WeakMap&&WeakMap.prototype?WeakMap.prototype.has:null,h="function"===typeof WeakSet&&WeakSet.prototype?WeakSet.prototype.has:null,m="function"===typeof WeakRef&&WeakRef.prototype?WeakRef.prototype.deref:null,p=Boolean.prototype.valueOf,f=Object.prototype.toString,v=Function.prototype.toString,g=String.prototype.match,y=String.prototype.slice,_=String.prototype.replace,b=String.prototype.toUpperCase,w=String.prototype.toLowerCase,k=RegExp.prototype.test,x=Array.prototype.concat,S=Array.prototype.join,C=Array.prototype.slice,E=Math.floor,N="function"===typeof BigInt?BigInt.prototype.valueOf:null,A=Object.getOwnPropertySymbols,M="function"===typeof Symbol&&"symbol"===typeof Symbol.iterator?Symbol.prototype.toString:null,T="function"===typeof Symbol&&"object"===typeof Symbol.iterator,L="function"===typeof Symbol&&Symbol.toStringTag&&(typeof Symbol.toStringTag===T||"symbol")?Symbol.toStringTag:null,P=Object.prototype.propertyIsEnumerable,O=("function"===typeof Reflect?Reflect.getPrototypeOf:Object.getPrototypeOf)||([].__proto__===Array.prototype?function(e){return e.__proto__}:null);function I(e,t){if(e===1/0||e===-1/0||e!==e||e&&e>-1e3&&e<1e3||k.call(/e/,t))return t;var n=/[0-9](?=(?:[0-9]{3})+(?![0-9]))/g;if("number"===typeof e){var r=e<0?-E(-e):E(e);if(r!==e){var a=String(r),i=y.call(t,a.length+1);return _.call(a,n,"$&_")+"."+_.call(_.call(i,/([0-9]{3})/g,"$&_"),/_$/,"")}}return _.call(t,n,"$&_")}var R=n(634),D=R.custom,z=$(D)?D:null;function F(e,t,n){var r="double"===(n.quoteStyle||t)?'"':"'";return r+e+r}function j(e){return _.call(String(e),/"/g,""")}function H(e){return"[object Array]"===q(e)&&(!L||!("object"===typeof e&&L in e))}function V(e){return"[object RegExp]"===q(e)&&(!L||!("object"===typeof e&&L in e))}function $(e){if(T)return e&&"object"===typeof e&&e instanceof Symbol;if("symbol"===typeof e)return!0;if(!e||"object"!==typeof e||!M)return!1;try{return M.call(e),!0}catch(t){}return!1}e.exports=function e(t,r,a,l){var s=r||{};if(B(s,"quoteStyle")&&"single"!==s.quoteStyle&&"double"!==s.quoteStyle)throw new TypeError('option "quoteStyle" must be "single" or "double"');if(B(s,"maxStringLength")&&("number"===typeof s.maxStringLength?s.maxStringLength<0&&s.maxStringLength!==1/0:null!==s.maxStringLength))throw new TypeError('option "maxStringLength", if provided, must be a positive integer, Infinity, or `null`');var f=!B(s,"customInspect")||s.customInspect;if("boolean"!==typeof f&&"symbol"!==f)throw new TypeError("option \"customInspect\", if provided, must be `true`, `false`, or `'symbol'`");if(B(s,"indent")&&null!==s.indent&&"\t"!==s.indent&&!(parseInt(s.indent,10)===s.indent&&s.indent>0))throw new TypeError('option "indent" must be "\\t", an integer > 0, or `null`');if(B(s,"numericSeparator")&&"boolean"!==typeof s.numericSeparator)throw new TypeError('option "numericSeparator", if provided, must be `true` or `false`');var b=s.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,s);if("number"===typeof t){if(0===t)return 1/0/t>0?"0":"-0";var k=String(t);return b?I(t,k):k}if("bigint"===typeof t){var E=String(t)+"n";return b?I(t,E):E}var A="undefined"===typeof s.depth?5:s.depth;if("undefined"===typeof a&&(a=0),a>=A&&A>0&&"object"===typeof t)return H(t)?"[Array]":"[Object]";var D=function(e,t){var n;if("\t"===e.indent)n="\t";else{if(!("number"===typeof e.indent&&e.indent>0))return null;n=S.call(Array(e.indent+1)," ")}return{base:n,prev:S.call(Array(t+1),n)}}(s,a);if("undefined"===typeof l)l=[];else if(Y(l,t)>=0)return"[Circular]";function U(t,n,r){if(n&&(l=C.call(l)).push(n),r){var i={depth:s.depth};return B(s,"quoteStyle")&&(i.quoteStyle=s.quoteStyle),e(t,i,a+1,l)}return e(t,s,a+1,l)}if("function"===typeof t&&!V(t)){var K=function(e){if(e.name)return e.name;var t=g.call(v.call(e),/^function\s*([\w$]+)/);if(t)return t[1];return null}(t),ee=X(t,U);return"[Function"+(K?": "+K:" (anonymous)")+"]"+(ee.length>0?" { "+S.call(ee,", ")+" }":"")}if($(t)){var te=T?_.call(String(t),/^(Symbol\(.*\))_[^)]*$/,"$1"):M.call(t);return"object"!==typeof t||T?te:Q(te)}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 ne="<"+w.call(String(t.nodeName)),re=t.attributes||[],ae=0;ae"}if(H(t)){if(0===t.length)return"[]";var ie=X(t,U);return D&&!function(e){for(var t=0;t=0)return!1;return!0}(ie)?"["+J(ie,D)+"]":"[ "+S.call(ie,", ")+" ]"}if(function(e){return"[object Error]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t)){var oe=X(t,U);return"cause"in Error.prototype||!("cause"in t)||P.call(t,"cause")?0===oe.length?"["+String(t)+"]":"{ ["+String(t)+"] "+S.call(oe,", ")+" }":"{ ["+String(t)+"] "+S.call(x.call("[cause]: "+U(t.cause),oe),", ")+" }"}if("object"===typeof t&&f){if(z&&"function"===typeof t[z]&&R)return R(t,{depth:A-a});if("symbol"!==f&&"function"===typeof t.inspect)return t.inspect()}if(function(e){if(!i||!e||"object"!==typeof e)return!1;try{i.call(e);try{c.call(e)}catch(ne){return!0}return e instanceof Map}catch(t){}return!1}(t)){var le=[];return o&&o.call(t,(function(e,n){le.push(U(n,t,!0)+" => "+U(e,t))})),G("Map",i.call(t),le,D)}if(function(e){if(!c||!e||"object"!==typeof e)return!1;try{c.call(e);try{i.call(e)}catch(t){return!0}return e instanceof Set}catch(n){}return!1}(t)){var se=[];return u&&u.call(t,(function(e){se.push(U(e,t))})),G("Set",c.call(t),se,D)}if(function(e){if(!d||!e||"object"!==typeof e)return!1;try{d.call(e,d);try{h.call(e,h)}catch(ne){return!0}return e instanceof WeakMap}catch(t){}return!1}(t))return Z("WeakMap");if(function(e){if(!h||!e||"object"!==typeof e)return!1;try{h.call(e,h);try{d.call(e,d)}catch(ne){return!0}return e instanceof WeakSet}catch(t){}return!1}(t))return Z("WeakSet");if(function(e){if(!m||!e||"object"!==typeof e)return!1;try{return m.call(e),!0}catch(t){}return!1}(t))return Z("WeakRef");if(function(e){return"[object Number]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(U(Number(t)));if(function(e){if(!e||"object"!==typeof e||!N)return!1;try{return N.call(e),!0}catch(t){}return!1}(t))return Q(U(N.call(t)));if(function(e){return"[object Boolean]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(p.call(t));if(function(e){return"[object String]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t))return Q(U(String(t)));if("undefined"!==typeof window&&t===window)return"{ [object Window] }";if(t===n.g)return"{ [object globalThis] }";if(!function(e){return"[object Date]"===q(e)&&(!L||!("object"===typeof e&&L in e))}(t)&&!V(t)){var ce=X(t,U),ue=O?O(t)===Object.prototype:t instanceof Object||t.constructor===Object,de=t instanceof Object?"":"null prototype",he=!ue&&L&&Object(t)===t&&L in t?y.call(q(t),8,-1):de?"Object":"",me=(ue||"function"!==typeof t.constructor?"":t.constructor.name?t.constructor.name+" ":"")+(he||de?"["+S.call(x.call([],he||[],de||[]),": ")+"] ":"");return 0===ce.length?me+"{}":D?me+"{"+J(ce,D)+"}":me+"{ "+S.call(ce,", ")+" }"}return String(t)};var U=Object.prototype.hasOwnProperty||function(e){return e in this};function B(e,t){return U.call(e,t)}function q(e){return f.call(e)}function Y(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 F(_.call(_.call(e,/(['\\])/g,"\\$1"),/[\x00-\x1f]/g,K),"single",t)}function K(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 Q(e){return"Object("+e+")"}function Z(e){return e+" { ? }"}function G(e,t,n,r){return e+" ("+t+") {"+(r?J(n,r):S.call(n,", "))+"}"}function J(e,t){if(0===e.length)return"";var n="\n"+t.prev+t.base;return n+S.call(e,","+n)+"\n"+t.prev}function X(e,t){var n=H(e),r=[];if(n){r.length=e.length;for(var a=0;a{"use strict";n.r(t),n.d(t,{Children:()=>Y,Component:()=>l.uA,Fragment:()=>l.FK,PureComponent:()=>H,StrictMode:()=>Pe,Suspense:()=>G,SuspenseList:()=>ee,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:()=>ke,cloneElement:()=>Ne,createContext:()=>l.q6,createElement:()=>l.n,createFactory:()=>Se,createPortal:()=>ae,createRef:()=>l._3,default:()=>He,findDOMNode:()=>Me,flushSync:()=>Le,forwardRef:()=>B,hydrate:()=>he,isElement:()=>ze,isFragment:()=>Ee,isValidElement:()=>Ce,lazy:()=>X,memo:()=>V,render:()=>de,startTransition:()=>Oe,unmountComponentAtNode:()=>Ae,unstable_batchedUpdates:()=>Te,useCallback:()=>E,useContext:()=>N,useDebugValue:()=>A,useDeferredValue:()=>Ie,useEffect:()=>w,useErrorBoundary:()=>M,useId:()=>T,useImperativeHandle:()=>S,useInsertionEffect:()=>De,useLayoutEffect:()=>k,useMemo:()=>C,useReducer:()=>b,useRef:()=>x,useState:()=>_,useSyncExternalStore:()=>Fe,useTransition:()=>Re,version:()=>xe});var r,a,i,o,l=n(746),s=0,c=[],u=[],d=l.fF,h=d.__b,m=d.__r,p=d.diffed,f=d.__c,v=d.unmount,g=d.__;function y(e,t){d.__h&&d.__h(a,e,s||t),s=0;var n=a.__H||(a.__H={__:[],__h:[]});return e>=n.__.length&&n.__.push({__V:u}),n.__[e]}function _(e){return s=1,b(z,e)}function b(e,t,n){var i=y(r++,2);if(i.t=e,!i.__c&&(i.__=[n?n(t):z(void 0,t),function(e){var t=i.__N?i.__N[0]:i.__[0],n=i.t(t,e);t!==n&&(i.__N=[n,i.__[1]],i.__c.setState({}))}],i.__c=a,!a.u)){var o=function(e,t,n){if(!i.__c.__H)return!0;var r=i.__c.__H.__.filter((function(e){return!!e.__c}));if(r.every((function(e){return!e.__N})))return!l||l.call(this,e,t,n);var a=!1;return r.forEach((function(e){if(e.__N){var t=e.__[0];e.__=e.__N,e.__N=void 0,t!==e.__[0]&&(a=!0)}})),!(!a&&i.__c.props===e)&&(!l||l.call(this,e,t,n))};a.u=!0;var l=a.shouldComponentUpdate,s=a.componentWillUpdate;a.componentWillUpdate=function(e,t,n){if(this.__e){var r=l;l=void 0,o(e,t,n),l=r}s&&s.call(this,e,t,n)},a.shouldComponentUpdate=o}return i.__N||i.__}function w(e,t){var n=y(r++,3);!d.__s&&D(n.__H,t)&&(n.__=e,n.i=t,a.__H.__h.push(n))}function k(e,t){var n=y(r++,4);!d.__s&&D(n.__H,t)&&(n.__=e,n.i=t,a.__h.push(n))}function x(e){return s=5,C((function(){return{current:e}}),[])}function S(e,t,n){s=6,k((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 C(e,t){var n=y(r++,7);return D(n.__H,t)?(n.__V=e(),n.i=t,n.__h=e,n.__V):n.__}function E(e,t){return s=8,C((function(){return e}),t)}function N(e){var t=a.context[e.__c],n=y(r++,9);return n.c=e,t?(null==n.__&&(n.__=!0,t.sub(a)),t.props.value):e.__}function A(e,t){d.useDebugValue&&d.useDebugValue(t?t(e):e)}function M(e){var t=y(r++,10),n=_();return t.__=e,a.componentDidCatch||(a.componentDidCatch=function(e,r){t.__&&t.__(e,r),n[1](e)}),[n[0],function(){n[1](void 0)}]}function T(){var e=y(r++,11);if(!e.__){for(var t=a.__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 L(){for(var e;e=c.shift();)if(e.__P&&e.__H)try{e.__H.__h.forEach(I),e.__H.__h.forEach(R),e.__H.__h=[]}catch(r){e.__H.__h=[],d.__e(r,e.__v)}}d.__b=function(e){a=null,h&&h(e)},d.__=function(e,t){e&&t.__k&&t.__k.__m&&(e.__m=t.__k.__m),g&&g(e,t)},d.__r=function(e){m&&m(e),r=0;var t=(a=e.__c).__H;t&&(i===a?(t.__h=[],a.__h=[],t.__.forEach((function(e){e.__N&&(e.__=e.__N),e.__V=u,e.__N=e.i=void 0}))):(t.__h.forEach(I),t.__h.forEach(R),t.__h=[],r=0)),i=a},d.diffed=function(e){p&&p(e);var t=e.__c;t&&t.__H&&(t.__H.__h.length&&(1!==c.push(t)&&o===d.requestAnimationFrame||((o=d.requestAnimationFrame)||O)(L)),t.__H.__.forEach((function(e){e.i&&(e.__H=e.i),e.__V!==u&&(e.__=e.__V),e.i=void 0,e.__V=u}))),i=a=null},d.__c=function(e,t){t.some((function(e){try{e.__h.forEach(I),e.__h=e.__h.filter((function(e){return!e.__||R(e)}))}catch(a){t.some((function(e){e.__h&&(e.__h=[])})),t=[],d.__e(a,e.__v)}})),f&&f(e,t)},d.unmount=function(e){v&&v(e);var t,n=e.__c;n&&n.__H&&(n.__H.__.forEach((function(e){try{I(e)}catch(e){t=e}})),n.__H=void 0,t&&d.__e(t,n.__v))};var P="function"==typeof requestAnimationFrame;function O(e){var t,n=function(){clearTimeout(r),P&&cancelAnimationFrame(t),setTimeout(e)},r=setTimeout(n,100);P&&(t=requestAnimationFrame(n))}function I(e){var t=a,n=e.__c;"function"==typeof n&&(e.__c=void 0,n()),a=t}function R(e){var t=a;e.__c=e.__(),a=t}function D(e,t){return!e||e.length!==t.length||t.some((function(t,n){return t!==e[n]}))}function z(e,t){return"function"==typeof t?t(e):t}function F(e,t){for(var n in t)e[n]=t[n];return e}function j(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 H(e,t){this.props=e,this.context=t}function V(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:j(this.props,e)}function r(t){return this.shouldComponentUpdate=n,(0,l.n)(e,t)}return r.displayName="Memo("+(e.displayName||e.name)+")",r.prototype.isReactComponent=!0,r.__f=!0,r}(H.prototype=new l.uA).isPureReactComponent=!0,H.prototype.shouldComponentUpdate=function(e,t){return j(this.props,e)||j(this.state,t)};var $=l.fF.__b;l.fF.__b=function(e){e.type&&e.type.__f&&e.ref&&(e.props.ref=e.ref,e.ref=null),$&&$(e)};var U="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.forward_ref")||3911;function B(e){function t(t){var n=F({},t);return delete n.ref,e(n,t.ref||null)}return t.$$typeof=U,t.render=t,t.prototype.isReactComponent=t.__f=!0,t.displayName="ForwardRef("+(e.displayName||e.name)+")",t}var q=function(e,t){return null==e?null:(0,l.v2)((0,l.v2)(e).map(t))},Y={map:q,forEach:q,count:function(e){return e?(0,l.v2)(e).length:0},only:function(e){var t=(0,l.v2)(e);if(1!==t.length)throw"Children.only";return t[0]},toArray:l.v2},W=l.fF.__e;l.fF.__e=function(e,t,n,r){if(e.then)for(var a,i=t;i=i.__;)if((a=i.__c)&&a.__c)return null==t.__e&&(t.__e=n.__e,t.__k=n.__k),a.__c(e,t);W(e,t,n,r)};var K=l.fF.unmount;function Q(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=F({},e)).__c&&(e.__c.__P===n&&(e.__c.__P=t),e.__c=null),e.__k=e.__k&&e.__k.map((function(e){return Q(e,t,n)}))),e}function Z(e,t,n){return e&&n&&(e.__v=null,e.__k=e.__k&&e.__k.map((function(e){return Z(e,t,n)})),e.__c&&e.__c.__P===t&&(e.__e&&n.appendChild(e.__e),e.__c.__e=!0,e.__c.__P=n)),e}function G(){this.__u=0,this.t=null,this.__b=null}function J(e){var t=e.__.__c;return t&&t.__a&&t.__a(e)}function X(e){var t,n,r;function a(a){if(t||(t=e()).then((function(e){n=e.default||e}),(function(e){r=e})),r)throw r;if(!n)throw t;return(0,l.n)(n,a)}return a.displayName="Lazy",a.__f=!0,a}function ee(){this.u=null,this.o=null}l.fF.unmount=function(e){var t=e.__c;t&&t.__R&&t.__R(),t&&32&e.__u&&(e.type=null),K&&K(e)},(G.prototype=new l.uA).__c=function(e,t){var n=t.__c,r=this;null==r.t&&(r.t=[]),r.t.push(n);var a=J(r.__v),i=!1,o=function(){i||(i=!0,n.__R=null,a?a(l):l())};n.__R=o;var l=function(){if(! --r.__u){if(r.state.__a){var e=r.state.__a;r.__v.__k[0]=Z(e,e.__c.__P,e.__c.__O)}var t;for(r.setState({__a:r.__b=null});t=r.t.pop();)t.forceUpdate()}};r.__u++||32&t.__u||r.setState({__a:r.__b=r.__v.__k[0]}),e.then(o,o)},G.prototype.componentWillUnmount=function(){this.t=[]},G.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]=Q(this.__b,n,r.__O=r.__P)}this.__b=null}var a=t.__a&&(0,l.n)(l.FK,null,e.fallback);return a&&(a.__u&=-33),[(0,l.n)(l.FK,null,t.__a?null:e.children),a]};var te=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)}}),(0,l.XX)((0,l.n)(ne,{context:t.context},e.__v),t.l)}function ae(e,t){var n=(0,l.n)(re,{__v:e,i:t});return n.containerInfo=t,n}(ee.prototype=new l.uA).__a=function(e){var t=this,n=J(t.__v),r=t.o.get(e);return r[0]++,function(a){var i=function(){t.props.revealOrder?(r.push(a),te(t,e,r)):a()};n?n(i):i()}},ee.prototype.render=function(e){this.u=null,this.o=new Map;var t=(0,l.v2)(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},ee.prototype.componentDidUpdate=ee.prototype.componentDidMount=function(){var e=this;this.o.forEach((function(t,n){te(e,n,t)}))};var ie="undefined"!=typeof Symbol&&Symbol.for&&Symbol.for("react.element")||60103,oe=/^(?:accent|alignment|arabic|baseline|cap|clip(?!PathU)|color|dominant|fill|flood|font|glyph(?!R)|horiz|image(!S)|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]/,le=/^on(Ani|Tra|Tou|BeforeInp|Compo)/,se=/[A-Z0-9]/g,ce="undefined"!=typeof document,ue=function(e){return("undefined"!=typeof Symbol&&"symbol"==typeof Symbol()?/fil|che|rad/:/fil|che|ra/).test(e)};function de(e,t,n){return null==t.__k&&(t.textContent=""),(0,l.XX)(e,t),"function"==typeof n&&n(),e?e.__c:null}function he(e,t,n){return(0,l.Qv)(e,t),"function"==typeof n&&n(),e?e.__c:null}l.uA.prototype.isReactComponent={},["componentWillMount","componentWillReceiveProps","componentWillUpdate"].forEach((function(e){Object.defineProperty(l.uA.prototype,e,{configurable:!0,get:function(){return this["UNSAFE_"+e]},set:function(t){Object.defineProperty(this,e,{configurable:!0,writable:!0,value:t})}})}));var me=l.fF.event;function pe(){}function fe(){return this.cancelBubble}function ve(){return this.defaultPrevented}l.fF.event=function(e){return me&&(e=me(e)),e.persist=pe,e.isPropagationStopped=fe,e.isDefaultPrevented=ve,e.nativeEvent=e};var ge,ye={enumerable:!1,configurable:!0,get:function(){return this.class}},_e=l.fF.vnode;l.fF.vnode=function(e){"string"==typeof e.type&&function(e){var t=e.props,n=e.type,r={};for(var a in t){var i=t[a];if(!("value"===a&&"defaultValue"in t&&null==i||ce&&"children"===a&&"noscript"===n||"class"===a||"className"===a)){var o=a.toLowerCase();"defaultValue"===a&&"value"in t&&null==t.value?a="value":"download"===a&&!0===i?i="":"translate"===o&&"no"===i?i=!1:"ondoubleclick"===o?a="ondblclick":"onchange"!==o||"input"!==n&&"textarea"!==n||ue(t.type)?"onfocus"===o?a="onfocusin":"onblur"===o?a="onfocusout":le.test(a)?a=o:-1===n.indexOf("-")&&oe.test(a)?a=a.replace(se,"-$&").toLowerCase():null===i&&(i=void 0):o=a="oninput","oninput"===o&&r[a=o]&&(a="oninputCapture"),r[a]=i}}"select"==n&&r.multiple&&Array.isArray(r.value)&&(r.value=(0,l.v2)(t.children).forEach((function(e){e.props.selected=-1!=r.value.indexOf(e.props.value)}))),"select"==n&&null!=r.defaultValue&&(r.value=(0,l.v2)(t.children).forEach((function(e){e.props.selected=r.multiple?-1!=r.defaultValue.indexOf(e.props.value):r.defaultValue==e.props.value}))),t.class&&!t.className?(r.class=t.class,Object.defineProperty(r,"className",ye)):(t.className&&!t.class||t.class&&t.className)&&(r.class=r.className=t.className),e.props=r}(e),e.$$typeof=ie,_e&&_e(e)};var be=l.fF.__r;l.fF.__r=function(e){be&&be(e),ge=e.__c};var we=l.fF.diffed;l.fF.diffed=function(e){we&&we(e);var t=e.props,n=e.__e;null!=n&&"textarea"===e.type&&"value"in t&&t.value!==n.value&&(n.value=null==t.value?"":t.value),ge=null};var ke={ReactCurrentDispatcher:{current:{readContext:function(e){return ge.__n[e.__c].props.value}}}},xe="17.0.2";function Se(e){return l.n.bind(null,e)}function Ce(e){return!!e&&e.$$typeof===ie}function Ee(e){return Ce(e)&&e.type===l.FK}function Ne(e){return Ce(e)?l.Ob.apply(null,arguments):e}function Ae(e){return!!e.__k&&((0,l.XX)(null,e),!0)}function Me(e){return e&&(e.base||1===e.nodeType&&e)||null}var Te=function(e,t){return e(t)},Le=function(e,t){return e(t)},Pe=l.FK;function Oe(e){e()}function Ie(e){return e}function Re(){return[!1,Oe]}var De=k,ze=Ce;function Fe(e,t){var n=t(),r=_({h:{__:n,v:t}}),a=r[0].h,i=r[1];return k((function(){a.__=n,a.v=t,je(a)&&i({h:a})}),[e,n,t]),w((function(){return je(a)&&i({h:a}),e((function(){je(a)&&i({h:a})}))}),[e]),n}function je(e){var t,n,r=e.v,a=e.__;try{var i=r();return!((t=a)===(n=i)&&(0!==t||1/t==1/n)||t!=t&&n!=n)}catch(e){return!0}}var He={useState:_,useId:T,useReducer:b,useEffect:w,useLayoutEffect:k,useInsertionEffect:De,useTransition:Re,useDeferredValue:Ie,useSyncExternalStore:Fe,startTransition:Oe,useRef:x,useImperativeHandle:S,useMemo:C,useCallback:E,useContext:N,useDebugValue:A,version:"17.0.2",Children:Y,render:de,hydrate:he,unmountComponentAtNode:Ae,createPortal:ae,createElement:l.n,createContext:l.q6,createFactory:Se,cloneElement:Ne,createRef:l._3,Fragment:l.FK,isValidElement:Ce,isElement:ze,isFragment:Ee,findDOMNode:Me,Component:l.uA,PureComponent:H,memo:V,forwardRef:B,flushSync:Le,unstable_batchedUpdates:Te,StrictMode:Pe,Suspense:G,SuspenseList:ee,lazy:X,__SECRET_INTERNALS_DO_NOT_USE_OR_YOU_WILL_BE_FIRED:ke}},746:(e,t,n)=>{"use strict";n.d(t,{FK:()=>b,Ob:()=>B,Qv:()=>U,XX:()=>$,_3:()=>_,fF:()=>a,n:()=>g,q6:()=>q,uA:()=>w,v2:()=>T});var r,a,i,o,l,s,c,u,d={},h=[],m=/acit|ex(?:s|g|n|p|$)|rph|grid|ows|mnc|ntw|ine[ch]|zoo|^ord|itera/i,p=Array.isArray;function f(e,t){for(var n in t)e[n]=t[n];return e}function v(e){var t=e.parentNode;t&&t.removeChild(e)}function g(e,t,n){var a,i,o,l={};for(o in t)"key"==o?a=t[o]:"ref"==o?i=t[o]:l[o]=t[o];if(arguments.length>2&&(l.children=arguments.length>3?r.call(arguments,2):n),"function"==typeof e&&null!=e.defaultProps)for(o in e.defaultProps)void 0===l[o]&&(l[o]=e.defaultProps[o]);return y(e,l,a,i,null)}function y(e,t,n,r,o){var l={type:e,props:t,key:n,ref:r,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,constructor:void 0,__v:null==o?++i:o,__i:-1,__u:0};return null==o&&null!=a.vnode&&a.vnode(l),l}function _(){return{current:null}}function b(e){return e.children}function w(e,t){this.props=e,this.context=t}function k(e,t){if(null==t)return e.__?k(e.__,e.__i+1):null;for(var n;tn?(z(r,t,i),i.length=r.length=0,t=void 0,o.sort(c)):t&&a.__c&&a.__c(t,h));t&&z(r,t,i),E.__r=0}function N(e,t,n,r,a,i,o,l,s,c,u){var m,p,f,v,g,y=r&&r.__k||h,_=t.length;for(n.__d=s,A(n,t,y),s=n.__d,m=0;m<_;m++)null!=(f=n.__k[m])&&"boolean"!=typeof f&&"function"!=typeof f&&(p=-1===f.__i?d:y[f.__i]||d,f.__i=m,D(e,f,p,a,i,o,l,s,c,u),v=f.__e,f.ref&&p.ref!=f.ref&&(p.ref&&j(p.ref,null,f),u.push(f.ref,f.__c||v,f)),null==g&&null!=v&&(g=v),65536&f.__u||p.__k===f.__k?s=M(f,s,e):"function"==typeof f.type&&void 0!==f.__d?s=f.__d:v&&(s=v.nextSibling),f.__d=void 0,f.__u&=-196609);n.__d=s,n.__e=g}function A(e,t,n){var r,a,i,o,l,s=t.length,c=n.length,u=c,d=0;for(e.__k=[],r=0;r0?y(a.type,a.props,a.key,a.ref?a.ref:null,a.__v):a)?(a.__=e,a.__b=e.__b+1,l=L(a,n,o=r+d,u),a.__i=l,i=null,-1!==l&&(u--,(i=n[l])&&(i.__u|=131072)),null==i||null===i.__v?(-1==l&&d--,"function"!=typeof a.type&&(a.__u|=65536)):l!==o&&(l===o+1?d++:l>o?u>s-o?d+=l-o:d--:d=l(null!=s&&0==(131072&s.__u)?1:0))for(;o>=0||l=0){if((s=t[o])&&0==(131072&s.__u)&&a==s.key&&i===s.type)return o;o--}if(l2&&(s.children=arguments.length>3?r.call(arguments,2):n),y(e.type,s,a||e.key,i||e.ref,null)}function q(e,t){var n={__c:t="__cC"+u++,__: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((function(e){e.__e=!0,C(e)}))},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=h.slice,a={__e:function(e,t,n,r){for(var a,i,o;t=t.__;)if((a=t.__c)&&!a.__)try{if((i=a.constructor)&&null!=i.getDerivedStateFromError&&(a.setState(i.getDerivedStateFromError(e)),o=a.__d),null!=a.componentDidCatch&&(a.componentDidCatch(e,r||{}),o=a.__d),o)return a.__E=a}catch(t){e=t}throw e}},i=0,w.prototype.setState=function(e,t){var n;n=null!=this.__s&&this.__s!==this.state?this.__s:this.__s=f({},this.state),"function"==typeof e&&(e=e(f({},n),this.props)),e&&f(n,e),null!=e&&this.__v&&(t&&this._sb.push(t),C(this))},w.prototype.forceUpdate=function(e){this.__v&&(this.__e=!0,e&&this.__h.push(e),C(this))},w.prototype.render=b,o=[],s="function"==typeof Promise?Promise.prototype.then.bind(Promise.resolve()):setTimeout,c=function(e,t){return e.__v.__b-t.__v.__b},E.__r=0,u=0},640:e=>{"use strict";var t=String.prototype.replace,n=/%20/g,r="RFC1738",a="RFC3986";e.exports={default:a,formatters:{RFC1738:function(e){return t.call(e,n,"+")},RFC3986:function(e){return String(e)}},RFC1738:r,RFC3986:a}},215:(e,t,n)=>{"use strict";var r=n(518),a=n(968),i=n(640);e.exports={formats:i,parse:a,stringify:r}},968:(e,t,n)=>{"use strict";var r=n(570),a=Object.prototype.hasOwnProperty,i=Array.isArray,o={allowDots:!1,allowPrototypes:!1,allowSparse:!1,arrayLimit:20,charset:"utf-8",charsetSentinel:!1,comma:!1,decoder:r.decode,delimiter:"&",depth:5,ignoreQueryPrefix:!1,interpretNumericEntities:!1,parameterLimit:1e3,parseArrays:!0,plainObjects:!1,strictNullHandling:!1},l=function(e){return e.replace(/&#(\d+);/g,(function(e,t){return String.fromCharCode(parseInt(t,10))}))},s=function(e,t){return e&&"string"===typeof e&&t.comma&&e.indexOf(",")>-1?e.split(","):e},c=function(e,t,n,r){if(e){var i=n.allowDots?e.replace(/\.([^.[]+)/g,"[$1]"):e,o=/(\[[^[\]]*])/g,l=n.depth>0&&/(\[[^[\]]*])/.exec(i),c=l?i.slice(0,l.index):i,u=[];if(c){if(!n.plainObjects&&a.call(Object.prototype,c)&&!n.allowPrototypes)return;u.push(c)}for(var d=0;n.depth>0&&null!==(l=o.exec(i))&&d=0;--i){var o,l=e[i];if("[]"===l&&n.parseArrays)o=[].concat(a);else{o=n.plainObjects?Object.create(null):{};var c="["===l.charAt(0)&&"]"===l.charAt(l.length-1)?l.slice(1,-1):l,u=parseInt(c,10);n.parseArrays||""!==c?!isNaN(u)&&l!==c&&String(u)===c&&u>=0&&n.parseArrays&&u<=n.arrayLimit?(o=[])[u]=a:"__proto__"!==c&&(o[c]=a):o={0:a}}a=o}return a}(u,t,n,r)}};e.exports=function(e,t){var n=function(e){if(!e)return o;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?o.charset:e.charset;return{allowDots:"undefined"===typeof e.allowDots?o.allowDots:!!e.allowDots,allowPrototypes:"boolean"===typeof e.allowPrototypes?e.allowPrototypes:o.allowPrototypes,allowSparse:"boolean"===typeof e.allowSparse?e.allowSparse:o.allowSparse,arrayLimit:"number"===typeof e.arrayLimit?e.arrayLimit:o.arrayLimit,charset:t,charsetSentinel:"boolean"===typeof e.charsetSentinel?e.charsetSentinel:o.charsetSentinel,comma:"boolean"===typeof e.comma?e.comma:o.comma,decoder:"function"===typeof e.decoder?e.decoder:o.decoder,delimiter:"string"===typeof e.delimiter||r.isRegExp(e.delimiter)?e.delimiter:o.delimiter,depth:"number"===typeof e.depth||!1===e.depth?+e.depth:o.depth,ignoreQueryPrefix:!0===e.ignoreQueryPrefix,interpretNumericEntities:"boolean"===typeof e.interpretNumericEntities?e.interpretNumericEntities:o.interpretNumericEntities,parameterLimit:"number"===typeof e.parameterLimit?e.parameterLimit:o.parameterLimit,parseArrays:!1!==e.parseArrays,plainObjects:"boolean"===typeof e.plainObjects?e.plainObjects:o.plainObjects,strictNullHandling:"boolean"===typeof e.strictNullHandling?e.strictNullHandling:o.strictNullHandling}}(t);if(""===e||null===e||"undefined"===typeof e)return n.plainObjects?Object.create(null):{};for(var u="string"===typeof e?function(e,t){var n,c={__proto__:null},u=t.ignoreQueryPrefix?e.replace(/^\?/,""):e,d=t.parameterLimit===1/0?void 0:t.parameterLimit,h=u.split(t.delimiter,d),m=-1,p=t.charset;if(t.charsetSentinel)for(n=0;n-1&&(v=i(v)?[v]:v),a.call(c,f)?c[f]=r.combine(c[f],v):c[f]=v}return c}(e,n):e,d=n.plainObjects?Object.create(null):{},h=Object.keys(u),m=0;m{"use strict";var r=n(670),a=n(570),i=n(640),o=Object.prototype.hasOwnProperty,l={brackets:function(e){return e+"[]"},comma:"comma",indices:function(e,t){return e+"["+t+"]"},repeat:function(e){return e}},s=Array.isArray,c=Array.prototype.push,u=function(e,t){c.apply(e,s(t)?t:[t])},d=Date.prototype.toISOString,h=i.default,m={addQueryPrefix:!1,allowDots:!1,charset:"utf-8",charsetSentinel:!1,delimiter:"&",encode:!0,encoder:a.encode,encodeValuesOnly:!1,format:h,formatter:i.formatters[h],indices:!1,serializeDate:function(e){return d.call(e)},skipNulls:!1,strictNullHandling:!1},p={},f=function e(t,n,i,o,l,c,d,h,f,v,g,y,_,b,w,k){for(var x,S=t,C=k,E=0,N=!1;void 0!==(C=C.get(p))&&!N;){var A=C.get(t);if(E+=1,"undefined"!==typeof A){if(A===E)throw new RangeError("Cyclic object value");N=!0}"undefined"===typeof C.get(p)&&(E=0)}if("function"===typeof h?S=h(n,S):S instanceof Date?S=g(S):"comma"===i&&s(S)&&(S=a.maybeMap(S,(function(e){return e instanceof Date?g(e):e}))),null===S){if(l)return d&&!b?d(n,m.encoder,w,"key",y):n;S=""}if("string"===typeof(x=S)||"number"===typeof x||"boolean"===typeof x||"symbol"===typeof x||"bigint"===typeof x||a.isBuffer(S))return d?[_(b?n:d(n,m.encoder,w,"key",y))+"="+_(d(S,m.encoder,w,"value",y))]:[_(n)+"="+_(String(S))];var M,T=[];if("undefined"===typeof S)return T;if("comma"===i&&s(S))b&&d&&(S=a.maybeMap(S,d)),M=[{value:S.length>0?S.join(",")||null:void 0}];else if(s(h))M=h;else{var L=Object.keys(S);M=f?L.sort(f):L}for(var P=o&&s(S)&&1===S.length?n+"[]":n,O=0;O0?w+b:""}},570:(e,t,n)=>{"use strict";var r=n(640),a=Object.prototype.hasOwnProperty,i=Array.isArray,o=function(){for(var e=[],t=0;t<256;++t)e.push("%"+((t<16?"0":"")+t.toString(16)).toUpperCase());return e}(),l=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(i(n)){for(var r=[],a=0;a=48&&u<=57||u>=65&&u<=90||u>=97&&u<=122||i===r.RFC1738&&(40===u||41===u)?s+=l.charAt(c):u<128?s+=o[u]:u<2048?s+=o[192|u>>6]+o[128|63&u]:u<55296||u>=57344?s+=o[224|u>>12]+o[128|u>>6&63]+o[128|63&u]:(c+=1,u=65536+((1023&u)<<10|1023&l.charCodeAt(c)),s+=o[240|u>>18]+o[128|u>>12&63]+o[128|u>>6&63]+o[128|63&u])}return s},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(i(e)){for(var n=[],r=0;r{e.exports=n(204)},204:(e,t,n)=>{"use strict";var r=function(e){return e&&"object"==typeof e&&"default"in e?e.default:e}(n(609)),a=n(609);function i(){return(i=Object.assign||function(e){for(var t=1;tr.length&&h(e,t.length-1);)t=t.slice(0,t.length-1);return t.length}for(var a=r.length,i=t.length;i>=r.length;i--){var o=t[i];if(!h(e,i)&&m(e,i,o)){a=i+1;break}}return a}function v(e,t){return f(e,t)===e.mask.length}function g(e,t){var n=e.maskChar,r=e.mask,a=e.prefix;if(!n){for((t=y(e,"",t,0)).lengtht.length&&(t+=a.slice(t.length,r)),l.every((function(n){for(;u=n,h(e,c=r)&&u!==a[c];){if(r>=t.length&&(t+=a[r]),l=n,i&&h(e,r)&&l===i)return!0;if(++r>=a.length)return!1}var l,c,u;return!m(e,r,n)&&n!==i||(ra.start?d=(u=function(e,t,n,r){var a=e.mask,i=e.maskChar,o=n.split(""),l=r;return o.every((function(t){for(;o=t,h(e,n=r)&&o!==a[n];)if(++r>=a.length)return!1;var n,o;return(m(e,r,t)||t===i)&&r++,r=i.length?p=i.length:p=o.length&&p{"use strict";var r=n(375),a=n(411),i=n(734)(),o=n(553),l=n(277),s=r("%Math.floor%");e.exports=function(e,t){if("function"!==typeof e)throw new l("`fn` is not a function");if("number"!==typeof t||t<0||t>4294967295||s(t)!==t)throw new l("`length` must be a positive 32-bit integer");var n=arguments.length>2&&!!arguments[2],r=!0,c=!0;if("length"in e&&o){var u=o(e,"length");u&&!u.configurable&&(r=!1),u&&!u.writable&&(c=!1)}return(r||c||!n)&&(i?a(e,"length",t,!0,!0):a(e,"length",t)),e}},670:(e,t,n)=>{"use strict";var r=n(375),a=n(61),i=n(141),o=n(277),l=r("%WeakMap%",!0),s=r("%Map%",!0),c=a("WeakMap.prototype.get",!0),u=a("WeakMap.prototype.set",!0),d=a("WeakMap.prototype.has",!0),h=a("Map.prototype.get",!0),m=a("Map.prototype.set",!0),p=a("Map.prototype.has",!0),f=function(e,t){for(var n,r=e;null!==(n=r.next);r=n)if(n.key===t)return r.next=n.next,n.next=e.next,e.next=n,n};e.exports=function(){var e,t,n,r={assert:function(e){if(!r.has(e))throw new o("Side channel does not contain "+i(e))},get:function(r){if(l&&r&&("object"===typeof r||"function"===typeof r)){if(e)return c(e,r)}else if(s){if(t)return h(t,r)}else if(n)return function(e,t){var n=f(e,t);return n&&n.value}(n,r)},has:function(r){if(l&&r&&("object"===typeof r||"function"===typeof r)){if(e)return d(e,r)}else if(s){if(t)return p(t,r)}else if(n)return function(e,t){return!!f(e,t)}(n,r);return!1},set:function(r,a){l&&r&&("object"===typeof r||"function"===typeof r)?(e||(e=new l),u(e,r,a)):s?(t||(t=new s),m(t,r,a)):(n||(n={key:{},next:null}),function(e,t,n){var r=f(e,t);r?r.value=n:e.next={key:t,next:e.next,value:n}}(n,r,a))}};return r}},634:()=>{},738:(e,t)=>{var n;!function(){"use strict";var r={}.hasOwnProperty;function a(){for(var e="",t=0;t{var t=e&&e.__esModule?()=>e.default:()=>e;return n.d(t,{a:t}),t},n.d=(e,t)=>{for(var r in t)n.o(t,r)&&!n.o(e,r)&&Object.defineProperty(e,r,{enumerable:!0,get:t[r]})},n.f={},n.e=e=>Promise.all(Object.keys(n.f).reduce(((t,r)=>(n.f[r](e,t),t)),[])),n.u=e=>"static/js/"+e+".bebe1265.chunk.js",n.miniCssF=e=>{},n.g=function(){if("object"===typeof globalThis)return globalThis;try{return this||new Function("return this")()}catch(e){if("object"===typeof window)return window}}(),n.o=(e,t)=>Object.prototype.hasOwnProperty.call(e,t),(()=>{var e={},t="vmui:";n.l=(r,a,i,o)=>{if(e[r])e[r].push(a);else{var l,s;if(void 0!==i)for(var c=document.getElementsByTagName("script"),u=0;u{l.onerror=l.onload=null,clearTimeout(m);var a=e[r];if(delete e[r],l.parentNode&&l.parentNode.removeChild(l),a&&a.forEach((e=>e(n))),t)return t(n)},m=setTimeout(h.bind(null,void 0,{type:"timeout",target:l}),12e4);l.onerror=h.bind(null,l.onerror),l.onload=h.bind(null,l.onload),s&&document.head.appendChild(l)}}})(),n.r=e=>{"undefined"!==typeof Symbol&&Symbol.toStringTag&&Object.defineProperty(e,Symbol.toStringTag,{value:"Module"}),Object.defineProperty(e,"__esModule",{value:!0})},n.p="./",(()=>{var e={792:0};n.f.j=(t,r)=>{var a=n.o(e,t)?e[t]:void 0;if(0!==a)if(a)r.push(a[2]);else{var i=new Promise(((n,r)=>a=e[t]=[n,r]));r.push(a[2]=i);var o=n.p+n.u(t),l=new Error;n.l(o,(r=>{if(n.o(e,t)&&(0!==(a=e[t])&&(e[t]=void 0),a)){var i=r&&("load"===r.type?"missing":r.type),o=r&&r.target&&r.target.src;l.message="Loading chunk "+t+" failed.\n("+i+": "+o+")",l.name="ChunkLoadError",l.type=i,l.request=o,a[1](l)}}),"chunk-"+t,t)}};var t=(t,r)=>{var a,i,o=r[0],l=r[1],s=r[2],c=0;if(o.some((t=>0!==e[t]))){for(a in l)n.o(l,a)&&(n.m[a]=l[a]);if(s)s(n)}for(t&&t(r);c{"use strict";var e={};n.r(e),n.d(e,{AlarmIcon:()=>Bn,ArrowDownIcon:()=>Hn,ArrowDropDownIcon:()=>Vn,CalendarIcon:()=>Un,ChartIcon:()=>Kn,ClockIcon:()=>$n,CloseIcon:()=>On,CodeIcon:()=>Zn,CopyIcon:()=>ar,DeleteIcon:()=>Gn,DoneIcon:()=>er,DownloadIcon:()=>wr,DragIcon:()=>ir,ErrorIcon:()=>zn,FunctionIcon:()=>yr,InfoIcon:()=>Rn,IssueIcon:()=>sr,KeyboardIcon:()=>qn,LabelIcon:()=>_r,ListIcon:()=>pr,LogoAnomalyIcon:()=>Tn,LogoIcon:()=>An,LogoLogsIcon:()=>Mn,LogoShortIcon:()=>Ln,MetricIcon:()=>gr,MinusIcon:()=>Xn,MoreIcon:()=>dr,PlayCircleOutlineIcon:()=>Wn,PlayIcon:()=>Yn,PlusIcon:()=>Jn,Prettify:()=>rr,QuestionIcon:()=>cr,RefreshIcon:()=>jn,RestartIcon:()=>In,SettingsIcon:()=>Pn,StarBorderIcon:()=>fr,StarIcon:()=>vr,StorageIcon:()=>ur,SuccessIcon:()=>Fn,TableIcon:()=>Qn,TimelineIcon:()=>or,TipIcon:()=>mr,TuneIcon:()=>hr,ValueIcon:()=>br,VisibilityIcon:()=>tr,VisibilityOffIcon:()=>nr,WarningIcon:()=>Dn,WikiIcon:()=>lr});var t,r=n(609),a=n(159),i=n.n(a),o=n(7),l=n.n(o),s=n(648),c=n.n(s),u=n(220),d=n.n(u);function h(){return h=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0&&(t.hash=e.substr(n),e=e.substr(0,n));let r=e.indexOf("?");r>=0&&(t.search=e.substr(r),e=e.substr(0,r)),e&&(t.pathname=e)}return t}function b(e,n,r,a){void 0===a&&(a={});let{window:i=document.defaultView,v5Compat:o=!1}=a,l=i.history,s=t.Pop,c=null,u=d();function d(){return(l.state||{idx:null}).idx}function f(){s=t.Pop;let e=d(),n=null==e?null:e-u;u=e,c&&c({action:s,location:b.location,delta:n})}function _(e){let t="null"!==i.location.origin?i.location.origin:i.location.href,n="string"===typeof e?e:y(e);return n=n.replace(/ $/,"%20"),p(t,"No window.location.(origin|href) available to create URL for href: "+n),new URL(n,t)}null==u&&(u=0,l.replaceState(h({},l.state,{idx:u}),""));let b={get action(){return s},get location(){return e(i,l)},listen(e){if(c)throw new Error("A history only accepts one active listener");return i.addEventListener(m,f),c=e,()=>{i.removeEventListener(m,f),c=null}},createHref:e=>n(i,e),createURL:_,encodeLocation(e){let t=_(e);return{pathname:t.pathname,search:t.search,hash:t.hash}},push:function(e,n){s=t.Push;let a=g(b.location,e,n);r&&r(a,e),u=d()+1;let h=v(a,u),m=b.createHref(a);try{l.pushState(h,"",m)}catch(p){if(p instanceof DOMException&&"DataCloneError"===p.name)throw p;i.location.assign(m)}o&&c&&c({action:s,location:b.location,delta:1})},replace:function(e,n){s=t.Replace;let a=g(b.location,e,n);r&&r(a,e),u=d();let i=v(a,u),h=b.createHref(a);l.replaceState(i,"",h),o&&c&&c({action:s,location:b.location,delta:0})},go:e=>l.go(e)};return b}var w;!function(e){e.data="data",e.deferred="deferred",e.redirect="redirect",e.error="error"}(w||(w={}));new Set(["lazy","caseSensitive","path","id","index","children"]);function k(e,t,n){void 0===n&&(n="/");let r=D(("string"===typeof t?_(t):t).pathname||"/",n);if(null==r)return null;let a=x(e);!function(e){e.sort(((e,t)=>e.score!==t.score?t.score-e.score:function(e,t){let n=e.length===t.length&&e.slice(0,-1).every(((e,n)=>e===t[n]));return n?e[e.length-1]-t[t.length-1]:0}(e.routesMeta.map((e=>e.childrenIndex)),t.routesMeta.map((e=>e.childrenIndex)))))}(a);let i=null;for(let o=0;null==i&&o{let o={relativePath:void 0===i?e.path||"":i,caseSensitive:!0===e.caseSensitive,childrenIndex:a,route:e};o.relativePath.startsWith("/")&&(p(o.relativePath.startsWith(r),'Absolute route path "'+o.relativePath+'" nested under path "'+r+'" is not valid. An absolute child route path must start with the combined path of all its parent routes.'),o.relativePath=o.relativePath.slice(r.length));let l=V([r,o.relativePath]),s=n.concat(o);e.children&&e.children.length>0&&(p(!0!==e.index,'Index routes must not have child routes. Please remove all child routes from route path "'+l+'".'),x(e.children,t,s,l)),(null!=e.path||e.index)&&t.push({path:l,score:P(l,e.index),routesMeta:s})};return e.forEach(((e,t)=>{var n;if(""!==e.path&&null!=(n=e.path)&&n.includes("?"))for(let r of S(e.path))a(e,t,r);else a(e,t)})),t}function S(e){let t=e.split("/");if(0===t.length)return[];let[n,...r]=t,a=n.endsWith("?"),i=n.replace(/\?$/,"");if(0===r.length)return a?[i,""]:[i];let o=S(r.join("/")),l=[];return l.push(...o.map((e=>""===e?i:[i,e].join("/")))),a&&l.push(...o),l.map((t=>e.startsWith("/")&&""===t?"/":t))}const C=/^:[\w-]+$/,E=3,N=2,A=1,M=10,T=-2,L=e=>"*"===e;function P(e,t){let n=e.split("/"),r=n.length;return n.some(L)&&(r+=T),t&&(r+=N),n.filter((e=>!L(e))).reduce(((e,t)=>e+(C.test(t)?E:""===t?A:M)),r)}function O(e,t){let{routesMeta:n}=e,r={},a="/",i=[];for(let o=0;o(r.push({paramName:t,isOptional:null!=n}),n?"/?([^\\/]+)?":"/([^\\/]+)")));e.endsWith("*")?(r.push({paramName:"*"}),a+="*"===e||"/*"===e?"(.*)$":"(?:\\/(.+)|\\/*)$"):n?a+="\\/*$":""!==e&&"/"!==e&&(a+="(?:(?=\\/|$))");let i=new RegExp(a,t?void 0:"i");return[i,r]}(e.path,e.caseSensitive,e.end),a=t.match(n);if(!a)return null;let i=a[0],o=i.replace(/(.)\/+$/,"$1"),l=a.slice(1);return{params:r.reduce(((e,t,n)=>{let{paramName:r,isOptional:a}=t;if("*"===r){let e=l[n]||"";o=i.slice(0,i.length-e.length).replace(/(.)\/+$/,"$1")}const s=l[n];return e[r]=a&&!s?void 0:(s||"").replace(/%2F/g,"/"),e}),{}),pathname:i,pathnameBase:o,pattern:e}}function R(e){try{return e.split("/").map((e=>decodeURIComponent(e).replace(/\//g,"%2F"))).join("/")}catch(t){return f(!1,'The URL path "'+e+'" could not be decoded because it is is a malformed URL segment. This is probably due to a bad percent encoding ('+t+")."),e}}function D(e,t){if("/"===t)return e;if(!e.toLowerCase().startsWith(t.toLowerCase()))return null;let n=t.endsWith("/")?t.length-1:t.length,r=e.charAt(n);return r&&"/"!==r?null:e.slice(n)||"/"}function z(e,t,n,r){return"Cannot include a '"+e+"' character in a manually specified `to."+t+"` field ["+JSON.stringify(r)+"]. Please separate it out to the `to."+n+'` field. Alternatively you may provide the full path as a string in and the router will parse it for you.'}function F(e){return e.filter(((e,t)=>0===t||e.route.path&&e.route.path.length>0))}function j(e,t){let n=F(e);return t?n.map(((t,n)=>n===e.length-1?t.pathname:t.pathnameBase)):n.map((e=>e.pathnameBase))}function H(e,t,n,r){let a;void 0===r&&(r=!1),"string"===typeof e?a=_(e):(a=h({},e),p(!a.pathname||!a.pathname.includes("?"),z("?","pathname","search",a)),p(!a.pathname||!a.pathname.includes("#"),z("#","pathname","hash",a)),p(!a.search||!a.search.includes("#"),z("#","search","hash",a)));let i,o=""===e||""===a.pathname,l=o?"/":a.pathname;if(null==l)i=n;else{let e=t.length-1;if(!r&&l.startsWith("..")){let t=l.split("/");for(;".."===t[0];)t.shift(),e-=1;a.pathname=t.join("/")}i=e>=0?t[e]:"/"}let s=function(e,t){void 0===t&&(t="/");let{pathname:n,search:r="",hash:a=""}="string"===typeof e?_(e):e,i=n?n.startsWith("/")?n:function(e,t){let n=t.replace(/\/+$/,"").split("/");return e.split("/").forEach((e=>{".."===e?n.length>1&&n.pop():"."!==e&&n.push(e)})),n.length>1?n.join("/"):"/"}(n,t):t;return{pathname:i,search:U(r),hash:B(a)}}(a,i),c=l&&"/"!==l&&l.endsWith("/"),u=(o||"."===l)&&n.endsWith("/");return s.pathname.endsWith("/")||!c&&!u||(s.pathname+="/"),s}const V=e=>e.join("/").replace(/\/\/+/g,"/"),$=e=>e.replace(/\/+$/,"").replace(/^\/*/,"/"),U=e=>e&&"?"!==e?e.startsWith("?")?e:"?"+e:"",B=e=>e&&"#"!==e?e.startsWith("#")?e:"#"+e:"";Error;function q(e){return null!=e&&"number"===typeof e.status&&"string"===typeof e.statusText&&"boolean"===typeof e.internal&&"data"in e}const Y=["post","put","patch","delete"],W=(new Set(Y),["get",...Y]);new Set(W),new Set([301,302,303,307,308]),new Set([307,308]);Symbol("deferred");function K(){return K=Object.assign?Object.assign.bind():function(e){for(var t=1;t{n.current=!0}));let a=r.useCallback((function(r,a){void 0===a&&(a={}),n.current&&("number"===typeof r?e.navigate(r):e.navigate(r,K({fromRouteId:t},a)))}),[e,t]);return a}():function(){te()||p(!1);let e=r.useContext(Q),{basename:t,future:n,navigator:a}=r.useContext(G),{matches:i}=r.useContext(X),{pathname:o}=ne(),l=JSON.stringify(j(i,n.v7_relativeSplatPath)),s=r.useRef(!1);re((()=>{s.current=!0}));let c=r.useCallback((function(n,r){if(void 0===r&&(r={}),!s.current)return;if("number"===typeof n)return void a.go(n);let i=H(n,JSON.parse(l),o,"path"===r.relative);null==e&&"/"!==t&&(i.pathname="/"===i.pathname?t:V([t,i.pathname])),(r.replace?a.replace:a.push)(i,r.state,r)}),[t,a,l,o,e]);return c}()}const ie=r.createContext(null);function oe(e,t){let{relative:n}=void 0===t?{}:t,{future:a}=r.useContext(G),{matches:i}=r.useContext(X),{pathname:o}=ne(),l=JSON.stringify(j(i,a.v7_relativeSplatPath));return r.useMemo((()=>H(e,JSON.parse(l),o,"path"===n)),[e,l,o,n])}function le(e,n,a,i){te()||p(!1);let{navigator:o}=r.useContext(G),{matches:l}=r.useContext(X),s=l[l.length-1],c=s?s.params:{},u=(s&&s.pathname,s?s.pathnameBase:"/");s&&s.route;let d,h=ne();if(n){var m;let e="string"===typeof n?_(n):n;"/"===u||(null==(m=e.pathname)?void 0:m.startsWith(u))||p(!1),d=e}else d=h;let f=d.pathname||"/",v=f;if("/"!==u){let e=u.replace(/^\//,"").split("/");v="/"+f.replace(/^\//,"").split("/").slice(e.length).join("/")}let g=k(e,{pathname:v});let y=he(g&&g.map((e=>Object.assign({},e,{params:Object.assign({},c,e.params),pathname:V([u,o.encodeLocation?o.encodeLocation(e.pathname).pathname:e.pathname]),pathnameBase:"/"===e.pathnameBase?u:V([u,o.encodeLocation?o.encodeLocation(e.pathnameBase).pathname:e.pathnameBase])}))),l,a,i);return n&&y?r.createElement(J.Provider,{value:{location:K({pathname:"/",search:"",hash:"",state:null,key:"default"},d),navigationType:t.Pop}},y):y}function se(){let e=function(){var e;let t=r.useContext(ee),n=ve(pe.UseRouteError),a=ge(pe.UseRouteError);if(void 0!==t)return t;return null==(e=n.errors)?void 0:e[a]}(),t=q(e)?e.status+" "+e.statusText:e instanceof Error?e.message:JSON.stringify(e),n=e instanceof Error?e.stack:null,a="rgba(200,200,200, 0.5)",i={padding:"0.5rem",backgroundColor:a};return r.createElement(r.Fragment,null,r.createElement("h2",null,"Unexpected Application Error!"),r.createElement("h3",{style:{fontStyle:"italic"}},t),n?r.createElement("pre",{style:i},n):null,null)}const ce=r.createElement(se,null);class ue extends r.Component{constructor(e){super(e),this.state={location:e.location,revalidation:e.revalidation,error:e.error}}static getDerivedStateFromError(e){return{error:e}}static getDerivedStateFromProps(e,t){return t.location!==e.location||"idle"!==t.revalidation&&"idle"===e.revalidation?{error:e.error,location:e.location,revalidation:e.revalidation}:{error:void 0!==e.error?e.error:t.error,location:t.location,revalidation:e.revalidation||t.revalidation}}componentDidCatch(e,t){console.error("React Router caught the following error during render",e,t)}render(){return void 0!==this.state.error?r.createElement(X.Provider,{value:this.props.routeContext},r.createElement(ee.Provider,{value:this.state.error,children:this.props.component})):this.props.children}}function de(e){let{routeContext:t,match:n,children:a}=e,i=r.useContext(Q);return i&&i.static&&i.staticContext&&(n.route.errorElement||n.route.ErrorBoundary)&&(i.staticContext._deepestRenderedBoundaryId=n.route.id),r.createElement(X.Provider,{value:t},a)}function he(e,t,n,a){var i;if(void 0===t&&(t=[]),void 0===n&&(n=null),void 0===a&&(a=null),null==e){var o;if(null==(o=n)||!o.errors)return null;e=n.matches}let l=e,s=null==(i=n)?void 0:i.errors;if(null!=s){let e=l.findIndex((e=>e.route.id&&(null==s?void 0:s[e.route.id])));e>=0||p(!1),l=l.slice(0,Math.min(l.length,e+1))}let c=!1,u=-1;if(n&&a&&a.v7_partialHydration)for(let r=0;r=0?l.slice(0,u+1):[l[0]];break}}}return l.reduceRight(((e,a,i)=>{let o,d=!1,h=null,m=null;var p;n&&(o=s&&a.route.id?s[a.route.id]:void 0,h=a.route.errorElement||ce,c&&(u<0&&0===i?(p="route-fallback",!1||ye[p]||(ye[p]=!0),d=!0,m=null):u===i&&(d=!0,m=a.route.hydrateFallbackElement||null)));let f=t.concat(l.slice(0,i+1)),v=()=>{let t;return t=o?h:d?m:a.route.Component?r.createElement(a.route.Component,null):a.route.element?a.route.element:e,r.createElement(de,{match:a,routeContext:{outlet:e,matches:f,isDataRoute:null!=n},children:t})};return n&&(a.route.ErrorBoundary||a.route.errorElement||0===i)?r.createElement(ue,{location:n.location,revalidation:n.revalidation,component:h,error:o,children:v(),routeContext:{outlet:null,matches:f,isDataRoute:!0}}):v()}),null)}var me=function(e){return e.UseBlocker="useBlocker",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e}(me||{}),pe=function(e){return e.UseBlocker="useBlocker",e.UseLoaderData="useLoaderData",e.UseActionData="useActionData",e.UseRouteError="useRouteError",e.UseNavigation="useNavigation",e.UseRouteLoaderData="useRouteLoaderData",e.UseMatches="useMatches",e.UseRevalidator="useRevalidator",e.UseNavigateStable="useNavigate",e.UseRouteId="useRouteId",e}(pe||{});function fe(e){let t=r.useContext(Q);return t||p(!1),t}function ve(e){let t=r.useContext(Z);return t||p(!1),t}function ge(e){let t=function(e){let t=r.useContext(X);return t||p(!1),t}(),n=t.matches[t.matches.length-1];return n.route.id||p(!1),n.route.id}const ye={};r.startTransition;function _e(e){return function(e){let t=r.useContext(X).outlet;return t?r.createElement(ie.Provider,{value:e},t):t}(e.context)}function be(e){p(!1)}function we(e){let{basename:n="/",children:a=null,location:i,navigationType:o=t.Pop,navigator:l,static:s=!1,future:c}=e;te()&&p(!1);let u=n.replace(/^\/*/,"/"),d=r.useMemo((()=>({basename:u,navigator:l,static:s,future:K({v7_relativeSplatPath:!1},c)})),[u,c,l,s]);"string"===typeof i&&(i=_(i));let{pathname:h="/",search:m="",hash:f="",state:v=null,key:g="default"}=i,y=r.useMemo((()=>{let e=D(h,u);return null==e?null:{location:{pathname:e,search:m,hash:f,state:v,key:g},navigationType:o}}),[u,h,m,f,v,g,o]);return null==y?null:r.createElement(G.Provider,{value:d},r.createElement(J.Provider,{children:a,value:y}))}function ke(e){let{children:t,location:n}=e;return le(xe(t),n)}new Promise((()=>{}));r.Component;function xe(e,t){void 0===t&&(t=[]);let n=[];return r.Children.forEach(e,((e,a)=>{if(!r.isValidElement(e))return;let i=[...t,a];if(e.type===r.Fragment)return void n.push.apply(n,xe(e.props.children,i));e.type!==be&&p(!1),e.props.index&&e.props.children&&p(!1);let o={id:e.props.id||i.join("-"),caseSensitive:e.props.caseSensitive,element:e.props.element,Component:e.props.Component,index:e.props.index,path:e.props.path,loader:e.props.loader,action:e.props.action,errorElement:e.props.errorElement,ErrorBoundary:e.props.ErrorBoundary,hasErrorBoundary:null!=e.props.ErrorBoundary||null!=e.props.errorElement,shouldRevalidate:e.props.shouldRevalidate,handle:e.props.handle,lazy:e.props.lazy};e.props.children&&(o.children=xe(e.props.children,i)),n.push(o)})),n}function Se(){return Se=Object.assign?Object.assign.bind():function(e){for(var t=1;t=0||(a[n]=e[n]);return a}function Ee(e){return void 0===e&&(e=""),new URLSearchParams("string"===typeof e||Array.isArray(e)||e instanceof URLSearchParams?e:Object.keys(e).reduce(((t,n)=>{let r=e[n];return t.concat(Array.isArray(r)?r.map((e=>[n,e])):[[n,r]])}),[]))}new Set(["application/x-www-form-urlencoded","multipart/form-data","text/plain"]);const Ne=["onClick","relative","reloadDocument","replace","state","target","to","preventScrollReset","unstable_viewTransition"],Ae=["aria-current","caseSensitive","className","end","style","to","unstable_viewTransition","children"];try{window.__reactRouterVersion="6"}catch(rp){}const Me=r.createContext({isTransitioning:!1});new Map;const Te=r.startTransition;r.flushSync,r.useId;function Le(e){let{basename:t,children:n,future:a,window:i}=e,o=r.useRef();null==o.current&&(o.current=function(e){return void 0===e&&(e={}),b((function(e,t){let{pathname:n="/",search:r="",hash:a=""}=_(e.location.hash.substr(1));return n.startsWith("/")||n.startsWith(".")||(n="/"+n),g("",{pathname:n,search:r,hash:a},t.state&&t.state.usr||null,t.state&&t.state.key||"default")}),(function(e,t){let n=e.document.querySelector("base"),r="";if(n&&n.getAttribute("href")){let t=e.location.href,n=t.indexOf("#");r=-1===n?t:t.slice(0,n)}return r+"#"+("string"===typeof t?t:y(t))}),(function(e,t){f("/"===e.pathname.charAt(0),"relative pathnames are not supported in hash history.push("+JSON.stringify(t)+")")}),e)}({window:i,v5Compat:!0}));let l=o.current,[s,c]=r.useState({action:l.action,location:l.location}),{v7_startTransition:u}=a||{},d=r.useCallback((e=>{u&&Te?Te((()=>c(e))):c(e)}),[c,u]);return r.useLayoutEffect((()=>l.listen(d)),[l,d]),r.createElement(we,{basename:t,children:n,location:s.location,navigationType:s.action,navigator:l,future:a})}const Pe="undefined"!==typeof window&&"undefined"!==typeof window.document&&"undefined"!==typeof window.document.createElement,Oe=/^(?:[a-z][a-z0-9+.-]*:|\/\/)/i,Ie=r.forwardRef((function(e,t){let n,{onClick:a,relative:i,reloadDocument:o,replace:l,state:s,target:c,to:u,preventScrollReset:d,unstable_viewTransition:h}=e,m=Ce(e,Ne),{basename:f}=r.useContext(G),v=!1;if("string"===typeof u&&Oe.test(u)&&(n=u,Pe))try{let e=new URL(window.location.href),t=u.startsWith("//")?new URL(e.protocol+u):new URL(u),n=D(t.pathname,f);t.origin===e.origin&&null!=n?u=n+t.search+t.hash:v=!0}catch(rp){}let g=function(e,t){let{relative:n}=void 0===t?{}:t;te()||p(!1);let{basename:a,navigator:i}=r.useContext(G),{hash:o,pathname:l,search:s}=oe(e,{relative:n}),c=l;return"/"!==a&&(c="/"===l?a:V([a,l])),i.createHref({pathname:c,search:s,hash:o})}(u,{relative:i}),_=function(e,t){let{target:n,replace:a,state:i,preventScrollReset:o,relative:l,unstable_viewTransition:s}=void 0===t?{}:t,c=ae(),u=ne(),d=oe(e,{relative:l});return r.useCallback((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,n)){t.preventDefault();let n=void 0!==a?a:y(u)===y(d);c(e,{replace:n,state:i,preventScrollReset:o,relative:l,unstable_viewTransition:s})}}),[u,c,d,a,i,n,e,o,l,s])}(u,{replace:l,state:s,target:c,preventScrollReset:d,relative:i,unstable_viewTransition:h});return r.createElement("a",Se({},m,{href:n||g,onClick:v||o?a:function(e){a&&a(e),e.defaultPrevented||_(e)},ref:t,target:c}))}));const Re=r.forwardRef((function(e,t){let{"aria-current":n="page",caseSensitive:a=!1,className:i="",end:o=!1,style:l,to:s,unstable_viewTransition:c,children:u}=e,d=Ce(e,Ae),h=oe(s,{relative:d.relative}),m=ne(),f=r.useContext(Z),{navigator:v,basename:g}=r.useContext(G),y=null!=f&&function(e,t){void 0===t&&(t={});let n=r.useContext(Me);null==n&&p(!1);let{basename:a}=Fe(De.useViewTransitionState),i=oe(e,{relative:t.relative});if(!n.isTransitioning)return!1;let o=D(n.currentLocation.pathname,a)||n.currentLocation.pathname,l=D(n.nextLocation.pathname,a)||n.nextLocation.pathname;return null!=I(i.pathname,l)||null!=I(i.pathname,o)}(h)&&!0===c,_=v.encodeLocation?v.encodeLocation(h).pathname:h.pathname,b=m.pathname,w=f&&f.navigation&&f.navigation.location?f.navigation.location.pathname:null;a||(b=b.toLowerCase(),w=w?w.toLowerCase():null,_=_.toLowerCase()),w&&g&&(w=D(w,g)||w);const k="/"!==_&&_.endsWith("/")?_.length-1:_.length;let x,S=b===_||!o&&b.startsWith(_)&&"/"===b.charAt(k),C=null!=w&&(w===_||!o&&w.startsWith(_)&&"/"===w.charAt(_.length)),E={isActive:S,isPending:C,isTransitioning:y},N=S?n:void 0;x="function"===typeof i?i(E):[i,S?"active":null,C?"pending":null,y?"transitioning":null].filter(Boolean).join(" ");let A="function"===typeof l?l(E):l;return r.createElement(Ie,Se({},d,{"aria-current":N,className:x,ref:t,style:A,to:s,unstable_viewTransition:c}),"function"===typeof u?u(E):u)}));var De,ze;function Fe(e){let t=r.useContext(Q);return t||p(!1),t}function je(e){let t=r.useRef(Ee(e)),n=r.useRef(!1),a=ne(),i=r.useMemo((()=>function(e,t){let n=Ee(e);return t&&t.forEach(((e,r)=>{n.has(r)||t.getAll(r).forEach((e=>{n.append(r,e)}))})),n}(a.search,n.current?null:t.current)),[a.search]),o=ae(),l=r.useCallback(((e,t)=>{const r=Ee("function"===typeof e?e(i):e);n.current=!0,o("?"+r,t)}),[o,i]);return[i,l]}(function(e){e.UseScrollRestoration="useScrollRestoration",e.UseSubmit="useSubmit",e.UseSubmitFetcher="useSubmitFetcher",e.UseFetcher="useFetcher",e.useViewTransitionState="useViewTransitionState"})(De||(De={})),function(e){e.UseFetcher="useFetcher",e.UseFetchers="useFetchers",e.UseScrollRestoration="useScrollRestoration"}(ze||(ze={}));let He=function(e){return e.logs="logs",e.anomaly="anomaly",e}({});const Ve={home:"/",metrics:"/metrics",dashboards:"/dashboards",cardinality:"/cardinality",topQueries:"/top-queries",trace:"/trace",withTemplate:"/expand-with-exprs",relabel:"/relabeling",logs:"/logs",activeQueries:"/active-queries",queryAnalyzer:"/query-analyzer",icons:"/icons",anomaly:"/anomaly",query:"/query"},{REACT_APP_TYPE:$e}={},Ue=$e===He.logs,Be={header:{tenant:!0,stepControl:!Ue,timeSelector:!Ue,executionControls:!Ue}},qe={[Ve.home]:{title:"Query",...Be},[Ve.metrics]:{title:"Explore Prometheus metrics",header:{tenant:!0,stepControl:!0,timeSelector:!0}},[Ve.cardinality]:{title:"Explore cardinality",header:{tenant:!0,cardinalityDatePicker:!0}},[Ve.topQueries]:{title:"Top queries",header:{tenant:!0}},[Ve.trace]:{title:"Trace analyzer",header:{}},[Ve.queryAnalyzer]:{title:"Query analyzer",header:{}},[Ve.dashboards]:{title:"Dashboards",...Be},[Ve.withTemplate]:{title:"WITH templates",header:{}},[Ve.relabel]:{title:"Metric relabel debug",header:{}},[Ve.logs]:{title:"Logs Explorer",header:{}},[Ve.activeQueries]:{title:"Active Queries",header:{}},[Ve.icons]:{title:"Icons",header:{}},[Ve.anomaly]:{title:"Anomaly exploration",...Be},[Ve.query]:{title:"Query",...Be}},Ye=Ve,We=()=>{var e;const t=(null===(e=document.getElementById("root"))||void 0===e?void 0:e.dataset.params)||"{}";try{return JSON.parse(t)}catch(rp){return console.error(rp),{}}},Ke=()=>!!Object.keys(We()).length,Qe=/(\/select\/)(\d+|\d.+)(\/)(.+)/,Ze=(e,t)=>e.replace(Qe,"$1".concat(t,"/$4")),Ge=e=>{var t;return(null===(t=e.match(Qe))||void 0===t?void 0:t[2])||""},Je=(e,t)=>{t?window.localStorage.setItem(e,JSON.stringify({value:t})):et([e]),window.dispatchEvent(new Event("storage"))},Xe=e=>{const t=window.localStorage.getItem(e);if(null!==t)try{var n;return null===(n=JSON.parse(t))||void 0===n?void 0:n.value}catch(rp){return t}},et=e=>e.forEach((e=>window.localStorage.removeItem(e))),{REACT_APP_TYPE:tt}={};var nt=n(215),rt=n.n(nt),at=n(424),it=n.n(at);const ot={table:100,chart:20,code:1e3},lt=[{id:"small",isDefault:!0,height:()=>.2*window.innerHeight},{id:"medium",height:()=>.4*window.innerHeight},{id:"large",height:()=>.8*window.innerHeight}],st=["min","median","max"],ct=(e,t)=>{const n=window.location.hash.split("?")[1],r=rt().parse(n,{ignoreQueryPrefix:!0});return it()(r,e,t||"")};let ut=function(e){return e.yhat="yhat",e.yhatUpper="yhat_upper",e.yhatLower="yhat_lower",e.anomaly="vmui_anomalies_points",e.training="vmui_training_data",e.actual="actual",e.anomalyScore="anomaly_score",e}({}),dt=function(e){return e.table="table",e.chart="chart",e.code="code",e}({}),ht=function(e){return e.emptyServer="Please enter Server URL",e.validServer="Please provide a valid Server URL",e.validQuery="Please enter a valid Query and execute it",e.traceNotFound="Not found the tracing information",e.emptyTitle="Please enter title",e.positiveNumber="Please enter positive number",e.validStep="Please enter a valid step",e.unknownType="Unknown server response format: must have 'errorType'",e}({}),mt=function(e){return e.system="system",e.light="light",e.dark="dark",e}({}),pt=function(e){return e.empty="empty",e.metricsql="metricsql",e.label="label",e.labelValue="labelValue",e}({});const ft=e=>getComputedStyle(document.documentElement).getPropertyValue("--".concat(e)),vt=(e,t)=>{document.documentElement.style.setProperty("--".concat(e),t)},gt=()=>window.matchMedia("(prefers-color-scheme: dark)").matches,yt=e=>{let t;try{t=new URL(e)}catch(n){return!1}return"http:"===t.protocol||"https:"===t.protocol},_t=e=>e.replace(/\/$/,""),bt=ct("g0.tenantID",""),wt={serverUrl:_t((e=>{const{serverURL:t}=We(),n=Xe("SERVER_URL"),r=window.location.href.replace(/\/(select\/)?(vmui)\/.*/,""),a=window.location.href.replace(/(?:graph|vmui)\/.*/,""),i=window.location.href.replace(/\/(?:prometheus\/)?(?:graph|vmui)\/.*/,"/prometheus"),o=t||n||i;switch(tt){case He.logs:return r;case He.anomaly:return t||n||a;default:return e?Ze(o,e):o}})(bt)),tenantId:bt,theme:Xe("THEME")||mt.system,isDarkTheme:null};function kt(e,t){switch(t.type){case"SET_SERVER":return{...e,serverUrl:_t(t.payload)};case"SET_TENANT_ID":return{...e,tenantId:t.payload};case"SET_THEME":return Je("THEME",t.payload),{...e,theme:t.payload};case"SET_DARK_THEME":return{...e,isDarkTheme:(n=e.theme,n===mt.system&>()||n===mt.dark)};default:throw new Error}var n}var xt=n(746);var St=0;Array.isArray;function Ct(e,t,n,r,a,i){var o,l,s={};for(l in t)"ref"==l?o=t[l]:s[l]=t[l];var c={type:e,props:s,key:n,ref:o,__k:null,__:null,__b:0,__e:null,__d:void 0,__c:null,constructor:void 0,__v:--St,__i:-1,__u:0,__source:a,__self:i};if("function"==typeof e&&(o=e.defaultProps))for(l in o)void 0===s[l]&&(s[l]=o[l]);return xt.fF.vnode&&xt.fF.vnode(c),c}const Et=(0,r.createContext)({}),Nt=()=>(0,r.useContext)(Et).state,At=()=>(0,r.useContext)(Et).dispatch,Mt=Object.entries(wt).reduce(((e,t)=>{let[n,r]=t;return{...e,[n]:ct(n)||r}}),{}),Tt="YYYY-MM-DD",Lt="YYYY-MM-DD HH:mm:ss",Pt="YYYY-MM-DD HH:mm:ss:SSS (Z)",Ot="YYYY-MM-DD[T]HH:mm:ss",It="YYYY-MM-DD_HHmmss",Rt=window.innerWidth/4,Dt=window.innerWidth/40,zt=1,Ft=1578e8,jt=Intl.supportedValuesOf,Ht=jt?jt("timeZone"):["Africa/Abidjan","Africa/Accra","Africa/Addis_Ababa","Africa/Algiers","Africa/Asmera","Africa/Bamako","Africa/Bangui","Africa/Banjul","Africa/Bissau","Africa/Blantyre","Africa/Brazzaville","Africa/Bujumbura","Africa/Cairo","Africa/Casablanca","Africa/Ceuta","Africa/Conakry","Africa/Dakar","Africa/Dar_es_Salaam","Africa/Djibouti","Africa/Douala","Africa/El_Aaiun","Africa/Freetown","Africa/Gaborone","Africa/Harare","Africa/Johannesburg","Africa/Juba","Africa/Kampala","Africa/Khartoum","Africa/Kigali","Africa/Kinshasa","Africa/Lagos","Africa/Libreville","Africa/Lome","Africa/Luanda","Africa/Lubumbashi","Africa/Lusaka","Africa/Malabo","Africa/Maputo","Africa/Maseru","Africa/Mbabane","Africa/Mogadishu","Africa/Monrovia","Africa/Nairobi","Africa/Ndjamena","Africa/Niamey","Africa/Nouakchott","Africa/Ouagadougou","Africa/Porto-Novo","Africa/Sao_Tome","Africa/Tripoli","Africa/Tunis","Africa/Windhoek","America/Adak","America/Anchorage","America/Anguilla","America/Antigua","America/Araguaina","America/Argentina/La_Rioja","America/Argentina/Rio_Gallegos","America/Argentina/Salta","America/Argentina/San_Juan","America/Argentina/San_Luis","America/Argentina/Tucuman","America/Argentina/Ushuaia","America/Aruba","America/Asuncion","America/Bahia","America/Bahia_Banderas","America/Barbados","America/Belem","America/Belize","America/Blanc-Sablon","America/Boa_Vista","America/Bogota","America/Boise","America/Buenos_Aires","America/Cambridge_Bay","America/Campo_Grande","America/Cancun","America/Caracas","America/Catamarca","America/Cayenne","America/Cayman","America/Chicago","America/Chihuahua","America/Coral_Harbour","America/Cordoba","America/Costa_Rica","America/Creston","America/Cuiaba","America/Curacao","America/Danmarkshavn","America/Dawson","America/Dawson_Creek","America/Denver","America/Detroit","America/Dominica","America/Edmonton","America/Eirunepe","America/El_Salvador","America/Fort_Nelson","America/Fortaleza","America/Glace_Bay","America/Godthab","America/Goose_Bay","America/Grand_Turk","America/Grenada","America/Guadeloupe","America/Guatemala","America/Guayaquil","America/Guyana","America/Halifax","America/Havana","America/Hermosillo","America/Indiana/Knox","America/Indiana/Marengo","America/Indiana/Petersburg","America/Indiana/Tell_City","America/Indiana/Vevay","America/Indiana/Vincennes","America/Indiana/Winamac","America/Indianapolis","America/Inuvik","America/Iqaluit","America/Jamaica","America/Jujuy","America/Juneau","America/Kentucky/Monticello","America/Kralendijk","America/La_Paz","America/Lima","America/Los_Angeles","America/Louisville","America/Lower_Princes","America/Maceio","America/Managua","America/Manaus","America/Marigot","America/Martinique","America/Matamoros","America/Mazatlan","America/Mendoza","America/Menominee","America/Merida","America/Metlakatla","America/Mexico_City","America/Miquelon","America/Moncton","America/Monterrey","America/Montevideo","America/Montreal","America/Montserrat","America/Nassau","America/New_York","America/Nipigon","America/Nome","America/Noronha","America/North_Dakota/Beulah","America/North_Dakota/Center","America/North_Dakota/New_Salem","America/Ojinaga","America/Panama","America/Pangnirtung","America/Paramaribo","America/Phoenix","America/Port-au-Prince","America/Port_of_Spain","America/Porto_Velho","America/Puerto_Rico","America/Punta_Arenas","America/Rainy_River","America/Rankin_Inlet","America/Recife","America/Regina","America/Resolute","America/Rio_Branco","America/Santa_Isabel","America/Santarem","America/Santiago","America/Santo_Domingo","America/Sao_Paulo","America/Scoresbysund","America/Sitka","America/St_Barthelemy","America/St_Johns","America/St_Kitts","America/St_Lucia","America/St_Thomas","America/St_Vincent","America/Swift_Current","America/Tegucigalpa","America/Thule","America/Thunder_Bay","America/Tijuana","America/Toronto","America/Tortola","America/Vancouver","America/Whitehorse","America/Winnipeg","America/Yakutat","America/Yellowknife","Antarctica/Casey","Antarctica/Davis","Antarctica/DumontDUrville","Antarctica/Macquarie","Antarctica/Mawson","Antarctica/McMurdo","Antarctica/Palmer","Antarctica/Rothera","Antarctica/Syowa","Antarctica/Troll","Antarctica/Vostok","Arctic/Longyearbyen","Asia/Aden","Asia/Almaty","Asia/Amman","Asia/Anadyr","Asia/Aqtau","Asia/Aqtobe","Asia/Ashgabat","Asia/Atyrau","Asia/Baghdad","Asia/Bahrain","Asia/Baku","Asia/Bangkok","Asia/Barnaul","Asia/Beirut","Asia/Bishkek","Asia/Brunei","Asia/Calcutta","Asia/Chita","Asia/Choibalsan","Asia/Colombo","Asia/Damascus","Asia/Dhaka","Asia/Dili","Asia/Dubai","Asia/Dushanbe","Asia/Famagusta","Asia/Gaza","Asia/Hebron","Asia/Hong_Kong","Asia/Hovd","Asia/Irkutsk","Asia/Jakarta","Asia/Jayapura","Asia/Jerusalem","Asia/Kabul","Asia/Kamchatka","Asia/Karachi","Asia/Katmandu","Asia/Khandyga","Asia/Krasnoyarsk","Asia/Kuala_Lumpur","Asia/Kuching","Asia/Kuwait","Asia/Macau","Asia/Magadan","Asia/Makassar","Asia/Manila","Asia/Muscat","Asia/Nicosia","Asia/Novokuznetsk","Asia/Novosibirsk","Asia/Omsk","Asia/Oral","Asia/Phnom_Penh","Asia/Pontianak","Asia/Pyongyang","Asia/Qatar","Asia/Qostanay","Asia/Qyzylorda","Asia/Rangoon","Asia/Riyadh","Asia/Saigon","Asia/Sakhalin","Asia/Samarkand","Asia/Seoul","Asia/Shanghai","Asia/Singapore","Asia/Srednekolymsk","Asia/Taipei","Asia/Tashkent","Asia/Tbilisi","Asia/Tehran","Asia/Thimphu","Asia/Tokyo","Asia/Tomsk","Asia/Ulaanbaatar","Asia/Urumqi","Asia/Ust-Nera","Asia/Vientiane","Asia/Vladivostok","Asia/Yakutsk","Asia/Yekaterinburg","Asia/Yerevan","Atlantic/Azores","Atlantic/Bermuda","Atlantic/Canary","Atlantic/Cape_Verde","Atlantic/Faeroe","Atlantic/Madeira","Atlantic/Reykjavik","Atlantic/South_Georgia","Atlantic/St_Helena","Atlantic/Stanley","Australia/Adelaide","Australia/Brisbane","Australia/Broken_Hill","Australia/Currie","Australia/Darwin","Australia/Eucla","Australia/Hobart","Australia/Lindeman","Australia/Lord_Howe","Australia/Melbourne","Australia/Perth","Australia/Sydney","Europe/Amsterdam","Europe/Andorra","Europe/Astrakhan","Europe/Athens","Europe/Belgrade","Europe/Berlin","Europe/Bratislava","Europe/Brussels","Europe/Bucharest","Europe/Budapest","Europe/Busingen","Europe/Chisinau","Europe/Copenhagen","Europe/Dublin","Europe/Gibraltar","Europe/Guernsey","Europe/Helsinki","Europe/Isle_of_Man","Europe/Istanbul","Europe/Jersey","Europe/Kaliningrad","Europe/Kiev","Europe/Kirov","Europe/Lisbon","Europe/Ljubljana","Europe/London","Europe/Luxembourg","Europe/Madrid","Europe/Malta","Europe/Mariehamn","Europe/Minsk","Europe/Monaco","Europe/Moscow","Europe/Oslo","Europe/Paris","Europe/Podgorica","Europe/Prague","Europe/Riga","Europe/Rome","Europe/Samara","Europe/San_Marino","Europe/Sarajevo","Europe/Saratov","Europe/Simferopol","Europe/Skopje","Europe/Sofia","Europe/Stockholm","Europe/Tallinn","Europe/Tirane","Europe/Ulyanovsk","Europe/Uzhgorod","Europe/Vaduz","Europe/Vatican","Europe/Vienna","Europe/Vilnius","Europe/Volgograd","Europe/Warsaw","Europe/Zagreb","Europe/Zaporozhye","Europe/Zurich","Indian/Antananarivo","Indian/Chagos","Indian/Christmas","Indian/Cocos","Indian/Comoro","Indian/Kerguelen","Indian/Mahe","Indian/Maldives","Indian/Mauritius","Indian/Mayotte","Indian/Reunion","Pacific/Apia","Pacific/Auckland","Pacific/Bougainville","Pacific/Chatham","Pacific/Easter","Pacific/Efate","Pacific/Enderbury","Pacific/Fakaofo","Pacific/Fiji","Pacific/Funafuti","Pacific/Galapagos","Pacific/Gambier","Pacific/Guadalcanal","Pacific/Guam","Pacific/Honolulu","Pacific/Johnston","Pacific/Kiritimati","Pacific/Kosrae","Pacific/Kwajalein","Pacific/Majuro","Pacific/Marquesas","Pacific/Midway","Pacific/Nauru","Pacific/Niue","Pacific/Norfolk","Pacific/Noumea","Pacific/Pago_Pago","Pacific/Palau","Pacific/Pitcairn","Pacific/Ponape","Pacific/Port_Moresby","Pacific/Rarotonga","Pacific/Saipan","Pacific/Tahiti","Pacific/Tarawa","Pacific/Tongatapu","Pacific/Truk","Pacific/Wake","Pacific/Wallis"],Vt=[{long:"years",short:"y",possible:"year"},{long:"weeks",short:"w",possible:"week"},{long:"days",short:"d",possible:"day"},{long:"hours",short:"h",possible:"hour"},{long:"minutes",short:"m",possible:"min"},{long:"seconds",short:"s",possible:"sec"},{long:"milliseconds",short:"ms",possible:"millisecond"}],$t=Vt.map((e=>e.short)),Ut=e=>Math.round(1e3*e)/1e3,Bt=e=>Jt(i().duration(e,"seconds").asMilliseconds()),qt=e=>{let t=Ut(e);const n=Math.round(e);e>=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);return Bt(t||.001).replace(/\s/g,"")},Yt=e=>{const t=e.match(/\d+/g),n=e.match(/[a-zA-Z]+/g);if(n&&t&&$t.includes(n[0]))return{[n[0]]:t[0]}},Wt=e=>{const t=Vt.map((e=>e.short)).join("|"),n=new RegExp("\\d+(\\.\\d+)?[".concat(t,"]+"),"g"),r=(e.match(n)||[]).reduce(((e,t)=>{const n=Yt(t);return n?{...e,...n}:{...e}}),{});return i().duration(r).asSeconds()},Kt=(e,t)=>qt(e/(t?Dt:Rt)),Qt=(e,t)=>{const n=(t||i()().toDate()).valueOf()/1e3,r=Wt(e);return{start:n-r,end:n,step:Kt(r),date:Zt(t||i()().toDate())}},Zt=e=>i().tz(e).utc().format(Ot),Gt=e=>i().tz(e).format(Ot),Jt=e=>{const t=Math.floor(e%1e3),n=Math.floor(e/1e3%60),r=Math.floor(e/1e3/60%60),a=Math.floor(e/1e3/3600%24),i=Math.floor(e/864e5),o=["d","h","m","s","ms"],l=[i,a,r,n,t].map(((e,t)=>e?"".concat(e).concat(o[t]):""));return l.filter((e=>e)).join("")},Xt=e=>{const t=i()(1e3*e);return t.isValid()?t.toDate():new Date},en=[{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:()=>i()().tz().subtract(1,"day").endOf("day").toDate()},{title:"Today",duration:"1d",until:()=>i()().tz().endOf("day").toDate()}].map((e=>({id:e.title.replace(/\s/g,"_").toLocaleLowerCase(),until:e.until?e.until:()=>i()().tz().toDate(),...e}))),tn=e=>{var t;let{relativeTimeId:n,defaultDuration:r,defaultEndInput:a}=e;const i=null===(t=en.find((e=>e.isDefault)))||void 0===t?void 0:t.id,o=n||ct("g0.relative_time",i),l=en.find((e=>e.id===o));return{relativeTimeId:l?o:"none",duration:l?l.duration:r,endInput:l?l.until():a}},nn=e=>{const t=i()().tz(e);return"UTC".concat(t.format("Z"))},rn=function(){let e=arguments.length>0&&void 0!==arguments[0]?arguments[0]:"";const t=new RegExp(e,"i");return Ht.reduce(((n,r)=>{const a=(r.match(/^(.*?)\//)||[])[1]||"unknown",i=nn(r),o=i.replace(/UTC|0/,""),l=r.replace(/[/_]/g," "),s={region:r,utc:i,search:"".concat(r," ").concat(i," ").concat(l," ").concat(o)},c=!e||e&&t.test(s.search);return c&&n[a]?n[a].push(s):c&&(n[a]=[s]),n}),{})},an=e=>{i().tz.setDefault(e)},on=()=>{const e=i().tz.guess(),t=(e=>{try{return i()().tz(e),!0}catch(rp){return!1}})(e);return{isValid:t,title:t?"Browser Time (".concat(e,")"):"Browser timezone (UTC)",region:t?e:"UTC"}},ln=Xe("TIMEZONE")||on().region;an(ln);const sn=ct("g0.range_input"),{duration:cn,endInput:un,relativeTimeId:dn}=tn({defaultDuration:sn||"1h",defaultEndInput:(hn=ct("g0.end_input",i()().utc().format(Ot)),i()(hn).utcOffset(0,!0).toDate()),relativeTimeId:sn?ct("g0.relative_time","none"):void 0});var hn;const mn={duration:cn,period:Qt(cn,un),relativeTime:dn,timezone:ln};function pn(e,t){switch(t.type){case"SET_DURATION":return{...e,duration:t.payload,period:Qt(t.payload,Xt(e.period.end)),relativeTime:"none"};case"SET_RELATIVE_TIME":return{...e,duration:t.payload.duration,period:Qt(t.payload.duration,t.payload.until),relativeTime:t.payload.id};case"SET_PERIOD":const n=(e=>{const t=e.to.valueOf()-e.from.valueOf();return Jt(t)})(t.payload);return{...e,duration:n,period:Qt(n,t.payload.to),relativeTime:"none"};case"RUN_QUERY":const{duration:r,endInput:a}=tn({relativeTimeId:e.relativeTime,defaultDuration:e.duration,defaultEndInput:Xt(e.period.end)});return{...e,period:Qt(r,a)};case"RUN_QUERY_TO_NOW":return{...e,period:Qt(e.duration)};case"SET_TIMEZONE":return an(t.payload),Je("TIMEZONE",t.payload),e.defaultTimezone&&Je("DISABLED_DEFAULT_TIMEZONE",t.payload!==e.defaultTimezone),{...e,timezone:t.payload};case"SET_DEFAULT_TIMEZONE":return{...e,defaultTimezone:t.payload};default:throw new Error}}const fn=(0,r.createContext)({}),vn=()=>(0,r.useContext)(fn).state,gn=()=>(0,r.useContext)(fn).dispatch,yn=e=>{const t=Xe(e);return t?JSON.parse(t):[]},_n=50,bn=1e3,wn=1e3;const kn=(()=>{var e;const 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(((e,t)=>ct("g".concat(t,".expr"),"")))})(),xn={query:kn,queryHistory:kn.map((e=>({index:0,values:[e]}))),autocomplete:Xe("AUTOCOMPLETE")||!1,autocompleteQuick:!1,autocompleteCache:new class{constructor(){this.maxSize=void 0,this.map=void 0,this.maxSize=wn,this.map=new Map}get(e){for(const[t,n]of this.map){const r=JSON.parse(t),a=r.start===e.start&&r.end===e.end,i=r.type===e.type,o=e.value&&r.value&&e.value.includes(r.value),l=r.match===e.match||o,s=n.length=this.maxSize){const e=this.map.keys().next().value;this.map.delete(e)}this.map.set(JSON.stringify(e),t)}},metricsQLFunctions:[]};function Sn(e,t){switch(t.type){case"SET_QUERY":return{...e,query:t.payload.map((e=>e))};case"SET_QUERY_HISTORY":return(e=>{const t=e.map((e=>e.values[e.index])),n=yn("QUERY_HISTORY");n[0]||(n[0]=[]);const r=n[0];t.forEach((e=>{!r.includes(e)&&e&&r.unshift(e),r.length>100&&r.shift()})),Je("QUERY_HISTORY",JSON.stringify(n))})(t.payload),{...e,queryHistory:t.payload};case"SET_QUERY_HISTORY_BY_INDEX":return e.queryHistory.splice(t.payload.queryNumber,1,t.payload.value),{...e,queryHistory:e.queryHistory};case"TOGGLE_AUTOCOMPLETE":return Je("AUTOCOMPLETE",!e.autocomplete),{...e,autocomplete:!e.autocomplete};case"SET_AUTOCOMPLETE_QUICK":return{...e,autocompleteQuick:t.payload};case"SET_AUTOCOMPLETE_CACHE":return e.autocompleteCache.put(t.payload.key,t.payload.value),{...e};case"SET_METRICSQL_FUNCTIONS":return{...e,metricsQLFunctions:t.payload};default:throw new Error}}const Cn=(0,r.createContext)({}),En=()=>(0,r.useContext)(Cn).state,Nn=()=>(0,r.useContext)(Cn).dispatch,An=()=>Ct("svg",{viewBox:"0 0 74 24",fill:"currentColor",children:Ct("path",{d:"M6.12 10.48c.36.28.8.43 1.26.43h.05c.48 0 .96-.19 1.25-.44 1.5-1.28 5.88-5.29 5.88-5.29C15.73 4.1 12.46 3.01 7.43 3h-.06C2.33 3-.93 4.1.24 5.18c0 0 4.37 4 5.88 5.3Zm2.56 2.16c-.36.28-.8.44-1.26.45h-.04c-.46 0-.9-.17-1.26-.45-1.04-.88-4.74-4.22-6.12-5.5v1.94c0 .21.08.5.22.63l.07.06c1.05.96 4.55 4.16 5.83 5.25.36.28.8.43 1.26.44h.04c.49-.02.96-.2 1.26-.44 1.3-1.11 4.94-4.45 5.88-5.31.15-.14.23-.42.23-.63V7.15a454.94 454.94 0 0 1-6.11 5.5Zm-1.26 4.99c.46 0 .9-.16 1.26-.44a454.4 454.4 0 0 0 6.1-5.5v1.94c0 .2-.07.48-.22.62-.94.87-4.57 4.2-5.88 5.3-.3.26-.77.44-1.26.45h-.04c-.46 0-.9-.16-1.26-.44-1.2-1.02-4.38-3.92-5.62-5.06l-.28-.25c-.14-.14-.22-.42-.22-.62v-1.94c1.38 1.26 5.08 4.6 6.12 5.5.36.28.8.43 1.26.44h.04ZM35 5l-5.84 14.46h-2.43L20.89 5h2.16a.9.9 0 0 1 .9.61l3.41 8.82a18.8 18.8 0 0 1 .62 2.02 19.44 19.44 0 0 1 .57-2.02l3.39-8.82c.05-.15.16-.3.31-.42a.9.9 0 0 1 .58-.19H35Zm17.18 0v14.46H49.8v-9.34c0-.37.02-.78.06-1.21l-4.37 8.21c-.21.4-.53.59-.95.59h-.38c-.43 0-.75-.2-.95-.59L38.8 8.88a22.96 22.96 0 0 1 .07 1.24v9.34H36.5V5h2.03l.3.01c.1 0 .17.02.24.05.07.03.13.07.19.13a1 1 0 0 1 .17.24l4.33 8.03a16.97 16.97 0 0 1 .6 1.36 14.34 14.34 0 0 1 .6-1.38l4.28-8.01c.05-.1.1-.18.17-.24.06-.06.12-.1.19-.13a.9.9 0 0 1 .24-.05l.3-.01h2.04Zm8.88 13.73a4.5 4.5 0 0 0 1.82-.35 3.96 3.96 0 0 0 2.22-2.47c.2-.57.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.12 1.51-.37 2.19a4.88 4.88 0 0 1-2.76 2.95c-.66.29-1.4.43-2.23.43-.82 0-1.57-.14-2.24-.43a5.01 5.01 0 0 1-2.75-2.95 6.37 6.37 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.1 1.28.3 1.85a3.98 3.98 0 0 0 2.21 2.47c.53.24 1.14.36 1.82.36Zm10.38.73h-1.03V5.31h1.03v14.15Z"})}),Mn=()=>Ct("svg",{viewBox:"0 0 85 38",fill:"currentColor",children:[Ct("path",{d:"M11.12 10.48c.36.28.8.43 1.26.43h.05c.48 0 .96-.19 1.25-.44 1.5-1.28 5.88-5.29 5.88-5.29 1.17-1.09-2.1-2.17-7.13-2.18h-.06c-5.04 0-8.3 1.1-7.13 2.18 0 0 4.37 4 5.88 5.3Zm2.56 2.16c-.36.28-.8.44-1.26.45h-.04c-.46 0-.9-.17-1.26-.45-1.04-.88-4.74-4.22-6.12-5.5v1.94c0 .21.08.5.22.63l.07.06c1.05.96 4.55 4.16 5.83 5.25.36.28.8.43 1.26.44h.04c.49-.02.96-.2 1.26-.44 1.3-1.11 4.94-4.45 5.88-5.31.15-.14.23-.42.23-.63V7.15a455.13 455.13 0 0 1-6.11 5.5Zm-1.26 4.99c.46 0 .9-.16 1.26-.44 2.05-1.82 4.09-3.65 6.1-5.5v1.94c0 .2-.07.48-.22.62-.94.87-4.57 4.2-5.88 5.3-.3.26-.77.44-1.26.45h-.04c-.46 0-.9-.16-1.26-.44-1.2-1.02-4.38-3.92-5.62-5.06l-.28-.25c-.14-.14-.22-.42-.22-.62v-1.94c1.38 1.26 5.08 4.6 6.12 5.5.36.28.8.43 1.26.44h.04ZM40 5l-5.84 14.46h-2.43L25.89 5h2.16a.9.9 0 0 1 .9.61l3.41 8.82a18.8 18.8 0 0 1 .62 2.02 19.44 19.44 0 0 1 .57-2.02l3.39-8.82c.05-.15.16-.3.31-.42a.9.9 0 0 1 .58-.19H40Zm17.18 0v14.46H54.8v-9.34c0-.37.02-.78.06-1.21l-4.37 8.21c-.21.4-.53.59-.95.59h-.38c-.43 0-.75-.2-.95-.59L43.8 8.88a22.96 22.96 0 0 1 .07 1.24v9.34H41.5V5h2.03l.3.01c.1 0 .17.02.24.05.07.03.13.07.19.13a1 1 0 0 1 .17.24l4.33 8.03a16.97 16.97 0 0 1 .6 1.36 14.34 14.34 0 0 1 .6-1.38l4.28-8.01c.05-.1.1-.18.17-.24.06-.06.12-.1.19-.13a.9.9 0 0 1 .24-.05l.3-.01h2.04Zm8.88 13.73a4.5 4.5 0 0 0 1.82-.35 3.96 3.96 0 0 0 2.22-2.47c.2-.57.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.12 1.51-.37 2.19a4.88 4.88 0 0 1-2.76 2.95c-.66.29-1.4.43-2.23.43-.82 0-1.57-.14-2.24-.43a5.01 5.01 0 0 1-2.75-2.95 6.37 6.37 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.1 1.28.3 1.85a3.98 3.98 0 0 0 2.21 2.47c.53.24 1.14.36 1.82.36Zm10.38.73h-1.03V5.31h1.03v14.15ZM1.73 36v-5.17l-.67-.07a.6.6 0 0 1-.21-.1.23.23 0 0 1-.08-.18v-.44h.96v-.59c0-.34.05-.65.14-.92a1.79 1.79 0 0 1 1.08-1.11 2.45 2.45 0 0 1 1.62-.02l-.03.53c0 .1-.06.15-.16.16H4c-.18 0-.35.03-.5.08a.95.95 0 0 0-.39.23c-.1.11-.19.25-.25.43-.05.18-.08.4-.08.65v.56h1.75v.78H2.8V36H1.73Zm6.17-6.17c.45 0 .85.07 1.2.22a2.57 2.57 0 0 1 1.5 1.62c.13.38.2.81.2 1.29s-.07.91-.2 1.3a2.57 2.57 0 0 1-1.49 1.61c-.36.14-.76.21-1.2.21-.45 0-.86-.07-1.22-.21a2.57 2.57 0 0 1-1.5-1.62c-.12-.38-.19-.81-.19-1.3 0-.47.07-.9.2-1.28a2.57 2.57 0 0 1 1.5-1.62c.35-.15.76-.22 1.2-.22Zm0 5.42c.6 0 1.05-.2 1.35-.6.3-.4.44-.97.44-1.69s-.15-1.28-.44-1.69c-.3-.4-.75-.6-1.35-.6-.3 0-.57.05-.8.15-.22.1-.4.26-.56.45-.15.2-.26.44-.33.73-.08.28-.11.6-.11.96 0 .72.15 1.29.44 1.69.3.4.76.6 1.36.6Zm5.26-4.11c.2-.42.43-.74.71-.97.28-.24.62-.36 1.03-.36.13 0 .25.02.36.05.12.02.23.07.32.13l-.08.8c-.02.1-.08.15-.18.15l-.24-.04a1.7 1.7 0 0 0-.88.05c-.15.05-.29.14-.4.25-.12.1-.23.24-.32.4-.1.17-.18.35-.26.56V36h-1.07v-6.08h.61c.12 0 .2.02.24.07.05.04.08.12.1.23l.06.92Zm13.73-3.82L23.39 36h-1.46l-3.5-8.68h1.29a.54.54 0 0 1 .54.37l2.04 5.3a11.31 11.31 0 0 1 .37 1.21 11.65 11.65 0 0 1 .35-1.22l2.03-5.29c.03-.1.1-.18.19-.25.1-.08.21-.12.35-.12h1.3Zm2.2 2.52V36H27.6v-6.16h1.49Zm.2-1.79c0 .13-.02.25-.08.36a1 1 0 0 1-.51.5.96.96 0 0 1-.73 0 1.02 1.02 0 0 1-.5-.5.96.96 0 0 1 0-.73.93.93 0 0 1 .86-.58.9.9 0 0 1 .37.08c.12.05.22.11.3.2a.94.94 0 0 1 .3.67Zm5.72 3.1a.68.68 0 0 1-.13.13c-.04.03-.1.05-.18.05a.42.42 0 0 1-.22-.07 3.95 3.95 0 0 0-.62-.31c-.14-.05-.3-.07-.51-.07-.26 0-.5.04-.69.14-.2.1-.36.23-.49.4-.13.18-.22.4-.29.64-.06.25-.1.53-.1.85 0 .33.04.62.1.88.08.25.18.47.32.64.13.18.29.3.48.4.18.09.4.13.63.13a1.6 1.6 0 0 0 .94-.27l.26-.2a.4.4 0 0 1 .25-.09.3.3 0 0 1 .27.14l.43.54a2.76 2.76 0 0 1-1.77.96c-.22.03-.43.05-.65.05a2.57 2.57 0 0 1-1.96-.83c-.25-.28-.45-.6-.6-1-.14-.4-.21-.85-.21-1.35 0-.45.06-.87.2-1.25a2.61 2.61 0 0 1 1.51-1.67c.37-.16.8-.24 1.28-.24.46 0 .86.07 1.2.22.35.15.66.36.94.64l-.4.54Zm3.43 4.95c-.54 0-.95-.15-1.24-.45-.28-.3-.42-.73-.42-1.26v-3.44h-.63a.29.29 0 0 1-.2-.07c-.06-.06-.09-.13-.09-.24v-.59l.99-.16.31-1.68a.33.33 0 0 1 .12-.18.34.34 0 0 1 .21-.07h.77v1.94h1.64v1.05h-1.64v3.34c0 .2.05.34.14.45.1.1.22.16.39.16a.73.73 0 0 0 .39-.1l.12-.07a.2.2 0 0 1 .11-.03c.05 0 .08.01.11.03l.09.1.44.72c-.21.18-.46.32-.74.4-.28.1-.57.15-.87.15Zm5.09-6.35c.46 0 .87.07 1.24.22a2.7 2.7 0 0 1 1.58 1.63c.14.39.22.83.22 1.31 0 .49-.08.93-.22 1.32-.14.4-.35.73-.62 1-.26.28-.58.49-.96.64-.37.15-.78.22-1.24.22a3.4 3.4 0 0 1-1.25-.22 2.71 2.71 0 0 1-1.59-1.64 3.8 3.8 0 0 1-.21-1.32c0-.48.07-.92.21-1.31a2.75 2.75 0 0 1 1.58-1.63c.38-.15.8-.22 1.26-.22Zm0 5.2c.51 0 .89-.17 1.13-.52.25-.34.38-.84.38-1.5a2.6 2.6 0 0 0-.38-1.53c-.24-.34-.62-.52-1.13-.52-.52 0-.9.18-1.16.53-.25.35-.37.85-.37 1.51s.12 1.17.37 1.51c.25.35.64.52 1.16.52Zm5.56-4.04c.2-.37.42-.65.69-.86.26-.21.57-.32.94-.32.28 0 .5.06.68.19l-.1 1.1a.3.3 0 0 1-.09.16.24.24 0 0 1-.15.04 1.8 1.8 0 0 1-.27-.03 2.01 2.01 0 0 0-.34-.03c-.16 0-.3.03-.44.08a1.1 1.1 0 0 0-.34.2c-.1.1-.2.2-.27.33-.08.13-.15.27-.22.44V36H47.7v-6.16h.87c.15 0 .26.03.31.09.06.05.1.15.13.29l.09.7Zm4.62-1.07V36h-1.49v-6.16h1.49Zm.2-1.79c0 .13-.02.25-.07.36a1 1 0 0 1-.51.5.96.96 0 0 1-.74 0 1.02 1.02 0 0 1-.5-.5.96.96 0 0 1 0-.73.93.93 0 0 1 .86-.58.9.9 0 0 1 .38.08c.11.05.21.11.3.2a.94.94 0 0 1 .28.67Zm4.56 5.32a7.8 7.8 0 0 0-1.08.12c-.29.05-.52.12-.7.2a.92.92 0 0 0-.38.3.64.64 0 0 0-.11.36c0 .26.07.45.23.56.15.11.35.17.6.17.3 0 .57-.06.79-.17.22-.1.44-.28.65-.5v-1.04Zm-3.4-2.67c.71-.65 1.57-.97 2.56-.97.36 0 .68.06.97.18a1.99 1.99 0 0 1 1.16 1.24c.1.3.16.61.16.96V36h-.67a.7.7 0 0 1-.33-.06c-.07-.04-.13-.13-.18-.26l-.13-.44c-.16.14-.3.26-.46.37a2.8 2.8 0 0 1-.97.43 2.77 2.77 0 0 1-1.32-.05 1.62 1.62 0 0 1-.57-.31 1.41 1.41 0 0 1-.38-.53 1.85 1.85 0 0 1-.05-1.18c.05-.16.14-.3.25-.45.12-.14.28-.27.46-.4a3 3 0 0 1 .7-.32 9.19 9.19 0 0 1 2.2-.33v-.36c0-.41-.09-.71-.26-.91-.18-.2-.43-.3-.76-.3a1.84 1.84 0 0 0-1.02.28l-.33.18c-.1.06-.2.09-.32.09-.1 0-.2-.03-.27-.08a.72.72 0 0 1-.17-.2l-.26-.47Zm11.49 4.32V36h-4.88v-8.6h1.16v7.62h3.72Zm3.16-5.2c.44 0 .84.08 1.2.23a2.57 2.57 0 0 1 1.49 1.62c.13.38.2.81.2 1.29s-.07.91-.2 1.3a2.57 2.57 0 0 1-1.49 1.61c-.36.14-.76.21-1.2.21-.45 0-.85-.07-1.21-.21a2.57 2.57 0 0 1-1.5-1.62c-.13-.38-.2-.81-.2-1.3 0-.47.07-.9.2-1.28.14-.39.33-.72.59-1 .25-.26.55-.47.9-.62.37-.15.77-.22 1.22-.22Zm0 5.43c.6 0 1.05-.2 1.34-.6.3-.4.45-.97.45-1.69s-.15-1.28-.45-1.69c-.3-.4-.74-.6-1.34-.6-.3 0-.57.05-.8.15-.22.1-.4.26-.56.45-.15.2-.26.44-.34.73-.07.28-.1.6-.1.96 0 .72.14 1.29.44 1.69.3.4.75.6 1.36.6Zm6.33-2.22c.22 0 .4-.03.57-.09.16-.06.3-.14.41-.25.12-.11.2-.24.26-.39.05-.15.08-.31.08-.5 0-.37-.11-.66-.34-.88-.23-.22-.55-.33-.98-.33-.43 0-.76.1-.99.33-.22.22-.34.51-.34.89 0 .18.03.34.09.5a1.1 1.1 0 0 0 .67.63c.16.06.35.09.57.09Zm1.93 3.3a.51.51 0 0 0-.13-.36.84.84 0 0 0-.34-.22 8.57 8.57 0 0 0-1.73-.2 7.5 7.5 0 0 1-.62-.05c-.23.1-.41.23-.56.4a.8.8 0 0 0-.1.92c.07.12.18.22.32.3.14.1.32.16.54.21a3.5 3.5 0 0 0 1.55 0c.23-.05.42-.12.57-.22.16-.1.29-.21.37-.34a.8.8 0 0 0 .13-.44Zm1.08-6.17v.4c0 .13-.08.21-.25.25l-.69.09c.14.26.2.56.2.88a1.86 1.86 0 0 1-1.36 1.82 3.07 3.07 0 0 1-1.72.04c-.12.08-.22.16-.29.25a.44.44 0 0 0-.1.27c0 .15.06.26.17.33.12.08.28.13.47.16a5 5 0 0 0 .66.06 16.56 16.56 0 0 1 1.5.13c.26.05.48.12.67.22.19.1.34.24.46.41.12.18.18.4.18.69 0 .26-.07.5-.2.75s-.31.46-.56.65c-.24.2-.54.34-.9.46a4.57 4.57 0 0 1-2.36.04c-.33-.09-.6-.2-.82-.36a1.56 1.56 0 0 1-.5-.51c-.1-.2-.16-.4-.16-.6 0-.3.1-.56.28-.77.19-.2.45-.37.77-.5a1.15 1.15 0 0 1-.43-.32.88.88 0 0 1-.15-.54c0-.09.01-.18.04-.27.04-.1.08-.2.15-.28a1.55 1.55 0 0 1 .58-.5c-.3-.16-.53-.39-.7-.66-.17-.28-.25-.6-.25-.97 0-.3.05-.57.16-.8.12-.25.28-.46.48-.63.2-.17.45-.3.73-.4a3 3 0 0 1 2.3.21h1.64Zm4.65.76a.24.24 0 0 1-.23.14.42.42 0 0 1-.2-.07 3.59 3.59 0 0 0-.67-.3 1.8 1.8 0 0 0-1.03 0c-.14.05-.27.11-.37.2a.87.87 0 0 0-.23.27.75.75 0 0 0-.08.35c0 .15.04.28.13.39.1.1.21.19.36.27.15.07.32.14.5.2a13.63 13.63 0 0 1 1.16.4c.2.08.36.18.5.3a1.33 1.33 0 0 1 .5 1.07 2 2 0 0 1-.15.78c-.1.24-.25.44-.45.62-.2.17-.43.3-.72.4a3.1 3.1 0 0 1-2.14-.05 2.97 2.97 0 0 1-.87-.53l.25-.41c.04-.05.07-.1.12-.12a.3.3 0 0 1 .17-.04.4.4 0 0 1 .22.08l.3.19a1.91 1.91 0 0 0 1.03.27c.2 0 .38-.03.54-.08.16-.06.29-.13.4-.22a.96.96 0 0 0 .3-.7c0-.17-.05-.31-.14-.42-.09-.11-.2-.2-.36-.28a2.6 2.6 0 0 0-.5-.2l-.59-.19c-.2-.06-.39-.14-.58-.22a2.14 2.14 0 0 1-.5-.3 1.45 1.45 0 0 1-.36-.46c-.1-.19-.14-.41-.14-.67a1.6 1.6 0 0 1 .57-1.23c.18-.16.4-.3.68-.39.26-.1.57-.14.91-.14a2.84 2.84 0 0 1 1.9.7l-.23.4Z"}),Ct("defs",{children:Ct("path",{d:"M0 0h85v38H0z"})})]}),Tn=()=>Ct("svg",{viewBox:"0 0 85 38",fill:"currentColor",children:Ct("path",{d:"M11.118 10.476c.36.28.801.433 1.257.436h.052c.48-.007.961-.192 1.25-.444 1.509-1.279 5.88-5.287 5.88-5.287 1.168-1.087-2.093-2.174-7.13-2.181h-.06c-5.036.007-8.298 1.094-7.13 2.181 0 0 4.372 4.008 5.88 5.295zm2.559 2.166c-.359.283-.801.439-1.258.444h-.044a2.071 2.071 0 0 1-1.257-.444C10.082 11.755 6.384 8.42 5 7.148v1.93c0 .215.081.496.222.629l.07.064c1.045.955 4.546 4.154 5.825 5.245.358.283.8.438 1.257.444h.044c.489-.015.962-.2 1.258-.444 1.309-1.11 4.948-4.444 5.887-5.31.148-.132.222-.413.222-.628v-1.93a455.127 455.127 0 0 1-6.11 5.494zm-1.258 4.984a2.071 2.071 0 0 0 1.258-.436c2.053-1.815 4.09-3.65 6.11-5.502v1.938c0 .207-.075.488-.223.621-.94.873-4.578 4.2-5.887 5.31-.296.25-.77.436-1.258.443h-.044a2.071 2.071 0 0 1-1.257-.436c-1.204-1.027-4.376-3.928-5.616-5.062l-.28-.255c-.14-.133-.221-.414-.221-.621v-1.938c1.383 1.265 5.081 4.607 6.117 5.495.358.282.8.438 1.257.443h.044zM40 5l-5.84 14.46h-2.43L25.89 5h2.16c.233 0 .423.057.57.17.146.113.256.26.33.44l3.41 8.82c.113.287.22.603.32.95.106.34.206.697.3 1.07.08-.373.166-.73.26-1.07a8.84 8.84 0 0 1 .31-.95l3.39-8.82a.959.959 0 0 1 .31-.42.906.906 0 0 1 .58-.19H40zm17.176 0v14.46h-2.37v-9.34c0-.373.02-.777.06-1.21l-4.37 8.21c-.206.393-.523.59-.95.59h-.38c-.426 0-.743-.197-.95-.59l-4.42-8.24c.02.22.037.437.05.65.014.213.02.41.02.59v9.34h-2.37V5h2.03c.12 0 .224.003.31.01a.778.778 0 0 1 .23.05c.074.027.137.07.19.13.06.06.117.14.17.24l4.33 8.03c.114.213.217.433.31.66.1.227.197.46.29.7.094-.247.19-.483.29-.71.1-.233.207-.457.32-.67l4.27-8.01c.054-.1.11-.18.17-.24a.57.57 0 0 1 .19-.13.903.903 0 0 1 .24-.05c.087-.007.19-.01.31-.01h2.03zm8.887 13.73c.68 0 1.286-.117 1.82-.35.54-.24.996-.57 1.37-.99a4.28 4.28 0 0 0 .85-1.48c.2-.573.3-1.19.3-1.85V5.31h1.02v8.75c0 .78-.124 1.51-.37 2.19a5.248 5.248 0 0 1-1.07 1.77c-.46.5-1.024.893-1.69 1.18-.66.287-1.404.43-2.23.43-.827 0-1.574-.143-2.24-.43a5.012 5.012 0 0 1-1.69-1.18 5.33 5.33 0 0 1-1.06-1.77 6.373 6.373 0 0 1-.37-2.19V5.31h1.03v8.74c0 .66.096 1.277.29 1.85.2.567.483 1.06.85 1.48.373.42.826.75 1.36.99.54.24 1.15.36 1.83.36zm10.38.73h-1.03V5.31h1.03v14.15zM4.242 35v-5.166l-.672-.078a.595.595 0 0 1-.21-.09.23.23 0 0 1-.078-.186v-.438h.96v-.588c0-.348.048-.656.144-.924.1-.272.24-.5.42-.684a1.79 1.79 0 0 1 .66-.426c.256-.096.544-.144.864-.144.272 0 .522.04.75.12l-.024.534c-.008.096-.062.148-.162.156a4.947 4.947 0 0 1-.39.012c-.184 0-.352.024-.504.072a.949.949 0 0 0-.384.234c-.108.108-.192.25-.252.426a2.184 2.184 0 0 0-.084.654v.558h1.752v.774H5.316V35H4.242zM10.416 28.826a3.1 3.1 0 0 1 1.2.222c.356.148.66.358.912.63s.444.602.576.99c.136.384.204.814.204 1.29 0 .48-.068.912-.204 1.296a2.735 2.735 0 0 1-.576.984 2.572 2.572 0 0 1-.912.63 3.175 3.175 0 0 1-1.2.216c-.448 0-.852-.072-1.212-.216a2.572 2.572 0 0 1-.912-.63 2.805 2.805 0 0 1-.582-.984 3.972 3.972 0 0 1-.198-1.296c0-.476.066-.906.198-1.29.136-.388.33-.718.582-.99.252-.272.556-.482.912-.63.36-.148.764-.222 1.212-.222zm0 5.424c.6 0 1.048-.2 1.344-.6.296-.404.444-.966.444-1.686 0-.724-.148-1.288-.444-1.692-.296-.404-.744-.606-1.344-.606-.304 0-.57.052-.798.156a1.507 1.507 0 0 0-.564.45c-.148.196-.26.438-.336.726a3.941 3.941 0 0 0-.108.966c0 .72.148 1.282.444 1.686.3.4.754.6 1.362.6zM15.677 30.14c.192-.416.428-.74.708-.972.28-.236.622-.354 1.026-.354.128 0 .25.014.366.042.12.028.226.072.318.132l-.078.798c-.024.1-.084.15-.18.15-.056 0-.138-.012-.246-.036a1.694 1.694 0 0 0-.366-.036c-.192 0-.364.028-.516.084-.148.056-.282.14-.402.252a1.782 1.782 0 0 0-.318.408c-.092.16-.176.344-.252.552V35h-1.074v-6.078h.612c.116 0 .196.022.24.066.044.044.074.12.09.228l.072.924zM26.761 28.922 24.283 35h-.96l-2.478-6.078h.87a.33.33 0 0 1 .33.222l1.542 3.912c.048.148.09.292.126.432.036.14.07.28.102.42.032-.14.066-.28.102-.42.036-.14.08-.284.132-.432l1.56-3.912a.33.33 0 0 1 .12-.156.311.311 0 0 1 .198-.066h.834zM27.74 35v-6.078h.643c.152 0 .246.074.282.222l.078.624c.224-.276.476-.502.756-.678.28-.176.604-.264.972-.264.408 0 .738.114.99.342.256.228.44.536.552.924.088-.22.2-.41.336-.57a1.987 1.987 0 0 1 1.014-.624c.196-.048.394-.072.594-.072.32 0 .604.052.852.156.252.1.464.248.636.444.176.196.31.438.402.726.092.284.138.61.138.978V35H34.91v-3.87c0-.476-.104-.836-.312-1.08-.208-.248-.508-.372-.9-.372-.176 0-.344.032-.504.096-.156.06-.294.15-.414.27-.12.12-.216.272-.288.456-.068.18-.102.39-.102.63V35h-1.074v-3.87c0-.488-.098-.852-.294-1.092-.196-.24-.482-.36-.858-.36-.264 0-.508.072-.732.216a2.38 2.38 0 0 0-.618.576V35H27.74zM40.746 32.372c-.428.02-.788.058-1.08.114-.292.052-.526.12-.702.204a.923.923 0 0 0-.378.294.639.639 0 0 0-.114.366c0 .26.076.446.228.558.156.112.358.168.606.168.304 0 .566-.054.786-.162.224-.112.442-.28.654-.504v-1.038zm-3.396-2.67c.708-.648 1.56-.972 2.556-.972.36 0 .682.06.966.18.284.116.524.28.72.492.196.208.344.458.444.75.104.292.156.612.156.96V35h-.672a.708.708 0 0 1-.324-.06c-.076-.044-.136-.13-.18-.258l-.132-.444c-.156.14-.308.264-.456.372a2.804 2.804 0 0 1-.462.264c-.16.072-.332.126-.516.162-.18.04-.38.06-.6.06-.26 0-.5-.034-.72-.102a1.618 1.618 0 0 1-.57-.318 1.414 1.414 0 0 1-.372-.522 1.852 1.852 0 0 1-.132-.726 1.419 1.419 0 0 1 .33-.906c.12-.14.274-.272.462-.396s.418-.232.69-.324c.276-.092.596-.166.96-.222.364-.06.78-.096 1.248-.108v-.36c0-.412-.088-.716-.264-.912-.176-.2-.43-.3-.762-.3-.24 0-.44.028-.6.084-.156.056-.294.12-.414.192l-.33.186a.631.631 0 0 1-.324.084.439.439 0 0 1-.264-.078.716.716 0 0 1-.174-.192l-.264-.474zM44.974 29.6c.124-.124.254-.238.39-.342a2.395 2.395 0 0 1 .936-.444c.176-.044.368-.066.576-.066.336 0 .634.058.894.174.26.112.476.272.648.48.176.204.308.45.396.738.092.284.138.598.138.942V35H47.47v-3.918c0-.376-.086-.666-.258-.87-.172-.208-.434-.312-.786-.312-.256 0-.496.058-.72.174a2.58 2.58 0 0 0-.636.474V35h-1.482v-6.156h.906c.192 0 .318.09.378.27l.102.486zM53.085 28.748c.456 0 .87.074 1.242.222a2.692 2.692 0 0 1 1.578 1.626c.144.392.216.83.216 1.314 0 .488-.072.928-.216 1.32-.144.392-.35.726-.618 1.002a2.653 2.653 0 0 1-.96.636 3.333 3.333 0 0 1-1.242.222c-.46 0-.878-.074-1.254-.222a2.712 2.712 0 0 1-.966-.636 2.922 2.922 0 0 1-.618-1.002 3.807 3.807 0 0 1-.216-1.32c0-.484.072-.922.216-1.314.148-.392.354-.724.618-.996.268-.272.59-.482.966-.63a3.397 3.397 0 0 1 1.254-.222zm0 5.202c.512 0 .89-.172 1.134-.516.248-.344.372-.848.372-1.512s-.124-1.17-.372-1.518c-.244-.348-.622-.522-1.134-.522-.52 0-.906.176-1.158.528-.248.348-.372.852-.372 1.512s.124 1.164.372 1.512c.252.344.638.516 1.158.516zM57.252 35v-6.156h.906c.192 0 .318.09.378.27l.096.456c.108-.12.22-.23.336-.33a2.017 2.017 0 0 1 1.32-.492c.388 0 .706.106.954.318.252.208.44.486.564.834a1.93 1.93 0 0 1 .834-.882c.172-.092.354-.16.546-.204.196-.044.392-.066.588-.066.34 0 .642.052.906.156.264.104.486.256.666.456.18.2.316.444.408.732.096.288.144.618.144.99V35h-1.482v-3.918c0-.392-.086-.686-.258-.882-.172-.2-.424-.3-.756-.3-.152 0-.294.026-.426.078a1.026 1.026 0 0 0-.342.228 1.019 1.019 0 0 0-.228.366 1.435 1.435 0 0 0-.084.51V35h-1.488v-3.918c0-.412-.084-.712-.252-.9-.164-.188-.406-.282-.726-.282-.216 0-.418.054-.606.162a1.979 1.979 0 0 0-.516.432V35h-1.482zM70.558 32.372c-.428.02-.788.058-1.08.114-.292.052-.526.12-.702.204a.923.923 0 0 0-.378.294.639.639 0 0 0-.114.366c0 .26.076.446.228.558.156.112.358.168.606.168.304 0 .566-.054.786-.162.224-.112.442-.28.654-.504v-1.038zm-3.396-2.67c.708-.648 1.56-.972 2.556-.972.36 0 .682.06.966.18.284.116.524.28.72.492.196.208.344.458.444.75.104.292.156.612.156.96V35h-.672a.708.708 0 0 1-.324-.06c-.076-.044-.136-.13-.18-.258l-.132-.444c-.156.14-.308.264-.456.372a2.804 2.804 0 0 1-.462.264c-.16.072-.332.126-.516.162-.18.04-.38.06-.6.06-.26 0-.5-.034-.72-.102a1.618 1.618 0 0 1-.57-.318 1.414 1.414 0 0 1-.372-.522 1.852 1.852 0 0 1-.132-.726 1.419 1.419 0 0 1 .33-.906c.12-.14.274-.272.462-.396s.418-.232.69-.324c.276-.092.596-.166.96-.222.364-.06.78-.096 1.248-.108v-.36c0-.412-.088-.716-.264-.912-.176-.2-.43-.3-.762-.3-.24 0-.44.028-.6.084-.156.056-.294.12-.414.192l-.33.186a.631.631 0 0 1-.324.084.439.439 0 0 1-.264-.078.716.716 0 0 1-.174-.192l-.264-.474zM74.9 26.084V35h-1.482v-8.916H74.9zM81.969 28.844l-3.354 7.848a.538.538 0 0 1-.174.234c-.068.056-.174.084-.318.084h-1.104l1.152-2.472-2.49-5.694h1.302c.116 0 .206.028.27.084.068.056.118.12.15.192l1.308 3.192c.044.108.08.216.108.324.032.108.062.218.09.33a32.3 32.3 0 0 1 .108-.33c.036-.112.076-.222.12-.33l1.236-3.186a.437.437 0 0 1 .408-.276h1.188z"})}),Ln=()=>Ct("svg",{viewBox:"0 0 15 17",fill:"currentColor",children:Ct("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"})}),Pn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),On=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),In=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Rn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Dn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M1 21h22L12 2 1 21zm12-3h-2v-2h2v2zm0-4h-2v-4h2v4z"})}),zn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Fn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),jn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Hn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M7.41 8.59 12 13.17l4.59-4.58L18 10l-6 6-6-6 1.41-1.41z"})}),Vn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"m7 10 5 5 5-5z"})}),$n=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Ct("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"}),Ct("path",{d:"M12.5 7H11v6l5.25 3.15.75-1.23-4.5-2.67z"})]}),Un=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Bn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),qn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Yn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M8 5v14l11-7z"})}),Wn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Kn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Qn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Zn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),Gn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M6 19c0 1.1.9 2 2 2h8c1.1 0 2-.9 2-2V7H6v12zM19 4h-3.5l-1-1h-5l-1 1H5v2h14V4z"})}),Jn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 13h-6v6h-2v-6H5v-2h6V5h2v6h6v2z"})}),Xn=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 13H5v-2h14v2z"})}),er=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),tr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),nr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),rr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 9l1.25-2.75L23 5l-2.75-1.25L19 1l-1.25 2.75L15 5l2.75 1.25L19 9zm-7.5.5L9 4 6.5 9.5 1 12l5.5 2.5L9 20l2.5-5.5L17 12l-5.5-2.5zM19 15l-1.25 2.75L15 19l2.75 1.25L19 23l1.25-2.75L23 19l-2.75-1.25L19 15z"})}),ar=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),ir=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M20 9H4v2h16V9zM4 15h16v-2H4v2z"})}),or=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),lr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:[Ct("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"}),Ct("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"}),Ct("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"})]}),sr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),cr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),ur=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),dr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("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"})}),hr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M3 17v2h6v-2H3zM3 5v2h10V5H3zm10 16v-2h8v-2h-8v-2h-2v6h2zM7 9v2H3v2h4v2h2V9H7zm14 4v-2H11v2h10zm-6-4h2V7h4V5h-4V3h-2v6z"})}),mr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M7 20h4c0 1.1-.9 2-2 2s-2-.9-2-2zm-2-1h8v-2H5v2zm11.5-9.5c0 3.82-2.66 5.86-3.77 6.5H5.27c-1.11-.64-3.77-2.68-3.77-6.5C1.5 5.36 4.86 2 9 2s7.5 3.36 7.5 7.5zm4.87-2.13L20 8l1.37.63L22 10l.63-1.37L24 8l-1.37-.63L22 6l-.63 1.37zM19 6l.94-2.06L22 3l-2.06-.94L19 0l-.94 2.06L16 3l2.06.94L19 6z"})}),pr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M3 14h4v-4H3v4zm0 5h4v-4H3v4zM3 9h4V5H3v4zm5 5h13v-4H8v4zm0 5h13v-4H8v4zM8 5v4h13V5H8z"})}),fr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"m22 9.24-7.19-.62L12 2 9.19 8.63 2 9.24l5.46 4.73L5.82 21 12 17.27 18.18 21l-1.63-7.03L22 9.24zM12 15.4l-3.76 2.27 1-4.28-3.32-2.88 4.38-.38L12 6.1l1.71 4.04 4.38.38-3.32 2.88 1 4.28L12 15.4z"})}),vr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M12 17.27 18.18 21l-1.64-7.03L22 9.24l-7.19-.61L12 2 9.19 8.63 2 9.24l5.46 4.73L5.82 21z"})}),gr=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-error"),children:Ct("path",{d:"M13.5095 4L8.50952 1H7.50952L2.50952 4L2.01953 4.85999V10.86L2.50952 11.71L7.50952 14.71H8.50952L13.5095 11.71L13.9995 10.86V4.85999L13.5095 4ZM7.50952 13.5601L3.00952 10.86V5.69995L7.50952 8.15002V13.5601ZM3.26953 4.69995L8.00952 1.85999L12.7495 4.69995L8.00952 7.29004L3.26953 4.69995ZM13.0095 10.86L8.50952 13.5601V8.15002L13.0095 5.69995V10.86Z"})}),yr=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-primary"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M2 5H4V4H1.5L1 4.5V12.5L1.5 13H4V12H2V5ZM14.5 4H12V5H14V12H12V13H14.5L15 12.5V4.5L14.5 4ZM11.76 6.56995L12 7V9.51001L11.7 9.95996L7.19995 11.96H6.73999L4.23999 10.46L4 10.03V7.53003L4.30005 7.06995L8.80005 5.06995H9.26001L11.76 6.56995ZM5 9.70996L6.5 10.61V9.28003L5 8.38V9.70996ZM5.57996 7.56006L7.03003 8.43005L10.42 6.93005L8.96997 6.06006L5.57996 7.56006ZM7.53003 10.73L11.03 9.17004V7.77002L7.53003 9.31995V10.73Z"})}),_r=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-warning"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M14 2H8L7 3V6H8V3H14V8H10V9H14L15 8V3L14 2ZM9 6H13V7H9.41L9 6.59V6ZM7 7H2L1 8V13L2 14H8L9 13V8L8 7H7ZM8 13H2V8H8V9V13ZM3 9H7V10H3V9ZM3 11H7V12H3V11ZM9 4H13V5H9V4Z"})}),br=()=>Ct("svg",{viewBox:"0 0 16 16",fill:ft("color-primary"),children:Ct("path",{fillRule:"evenodd",clipRule:"evenodd",d:"M7 3L8 2H14L15 3V8L14 9H10V8H14V3H8V6H7V3ZM9 9V8L8 7H7H2L1 8V13L2 14H8L9 13V9ZM8 8V9V13H2V8H7H8ZM9.41421 7L9 6.58579V6H13V7H9.41421ZM9 4H13V5H9V4ZM7 10H3V11H7V10Z"})}),wr=()=>Ct("svg",{viewBox:"0 0 24 24",fill:"currentColor",children:Ct("path",{d:"M19 9h-4V3H9v6H5l7 7 7-7zM5 18v2h14v-2H5z"})});var kr=n(738),xr=n.n(kr);const Sr=e=>{let{to:t,isNavLink:n,children:r,...a}=e;return n?Ct(Re,{to:t,...a,children:r}):Ct("div",{...a,children:r})},Cr=e=>{let{activeItem:t,item:n,color:r=ft("color-primary"),activeNavRef:a,onChange:i,isNavLink:o}=e;return Ct(Sr,{className:xr()({"vm-tabs-item":!0,"vm-tabs-item_active":t===n.value,[n.className||""]:n.className}),isNavLink:o,to:n.value,style:{color:r},onClick:(l=n.value,()=>{i&&i(l)}),ref:t===n.value?a:void 0,children:[n.icon&&Ct("div",{className:xr()({"vm-tabs-item__icon":!0,"vm-tabs-item__icon_single":!n.label}),children:n.icon}),n.label]});var l},Er="undefined"!==typeof window?r.useLayoutEffect:r.useEffect;const Nr=function(e,t,n,a){const i=(0,r.useRef)(t);Er((()=>{i.current=t}),[t]),(0,r.useEffect)((()=>{var t;const r=null!==(t=null===n||void 0===n?void 0:n.current)&&void 0!==t?t:window;if(!r||!r.addEventListener)return;const o=e=>i.current(e);return r.addEventListener(e,o,a),()=>{r.removeEventListener(e,o,a)}}),[e,n,a])},Ar=()=>{const[e,t]=(0,r.useState)({width:0,height:0}),n=()=>{t({width:window.innerWidth,height:window.innerHeight})};return Nr("resize",n),Er(n,[]),e},Mr=e=>{let{activeItem:t,items:n,color:a=ft("color-primary"),onChange:i,indicatorPlacement:o="bottom",isNavLink:l}=e;const s=Ar(),c=(0,r.useRef)(null),[u,d]=(0,r.useState)({left:0,width:0,bottom:0});return(0,r.useEffect)((()=>{var e;if((null===(e=c.current)||void 0===e?void 0:e.base)instanceof HTMLElement){const{offsetLeft:e,offsetWidth:t,offsetHeight:n}=c.current.base;d({left:e,width:t,bottom:"top"===o?n-2:0})}}),[s,t,c,n]),Ct("div",{className:"vm-tabs",children:[n.map((e=>Ct(Cr,{activeItem:t,item:e,onChange:i,color:a,activeNavRef:c,isNavLink:l},e.value))),Ct("div",{className:"vm-tabs__indicator",style:{...u,borderColor:a}})]})},Tr=[{value:dt.chart,icon:Ct(Kn,{}),label:"Graph",prometheusCode:0},{value:dt.code,icon:Ct(Zn,{}),label:"JSON",prometheusCode:3},{value:dt.table,icon:Ct(Qn,{}),label:"Table",prometheusCode:1}],Lr=()=>{const{displayType:e}=Fr(),t=jr();return Ct(Mr,{activeItem:e,items:Tr,onChange:n=>{var r;t({type:"SET_DISPLAY_TYPE",payload:null!==(r=n)&&void 0!==r?r:e})}})},Pr=ct("g0.tab",0),Or=Tr.find((e=>e.prometheusCode===+Pr||e.value===Pr)),Ir=Xe("SERIES_LIMITS"),Rr={displayType:(null===Or||void 0===Or?void 0:Or.value)||dt.chart,nocache:!1,isTracingEnabled:!1,seriesLimits:Ir?JSON.parse(Ir):ot,tableCompact:Xe("TABLE_COMPACT")||!1};function Dr(e,t){switch(t.type){case"SET_DISPLAY_TYPE":return{...e,displayType:t.payload};case"SET_SERIES_LIMITS":return Je("SERIES_LIMITS",JSON.stringify(t.payload)),{...e,seriesLimits:t.payload};case"TOGGLE_QUERY_TRACING":return{...e,isTracingEnabled:!e.isTracingEnabled};case"TOGGLE_NO_CACHE":return{...e,nocache:!e.nocache};case"TOGGLE_TABLE_COMPACT":return Je("TABLE_COMPACT",!e.tableCompact),{...e,tableCompact:!e.tableCompact};default:throw new Error}}const zr=(0,r.createContext)({}),Fr=()=>(0,r.useContext)(zr).state,jr=()=>(0,r.useContext)(zr).dispatch,Hr={customStep:ct("g0.step_input",""),yaxis:{limits:{enable:!1,range:{1:[0,0]}}},isHistogram:!1,spanGaps:!1};function Vr(e,t){switch(t.type){case"TOGGLE_ENABLE_YAXIS_LIMITS":return{...e,yaxis:{...e.yaxis,limits:{...e.yaxis.limits,enable:!e.yaxis.limits.enable}}};case"SET_CUSTOM_STEP":return{...e,customStep:t.payload};case"SET_YAXIS_LIMITS":return{...e,yaxis:{...e.yaxis,limits:{...e.yaxis.limits,range:t.payload}}};case"SET_IS_HISTOGRAM":return{...e,isHistogram:t.payload};case"SET_SPAN_GAPS":return{...e,spanGaps:t.payload};default:throw new Error}}const $r=(0,r.createContext)({}),Ur=()=>(0,r.useContext)($r).state,Br=()=>(0,r.useContext)($r).dispatch,qr={windows:"Windows",mac:"Mac OS",linux:"Linux"},Yr=()=>(Object.values(qr).find((e=>navigator.userAgent.indexOf(e)>=0))||"unknown")===qr.mac;function Wr(){const e=Ar(),t=()=>{const e=["Android","webOS","iPhone","iPad","iPod","BlackBerry","Windows Phone"].map((e=>navigator.userAgent.match(new RegExp(e,"i")))).some((e=>e)),t=window.innerWidth<500;return e||t},[n,a]=(0,r.useState)(t());return(0,r.useEffect)((()=>{a(t())}),[e]),{isMobile:n}}const Kr={success:Ct(Fn,{}),error:Ct(zn,{}),warning:Ct(Dn,{}),info:Ct(Rn,{})},Qr=e=>{let{variant:t,children:n}=e;const{isDarkTheme:r}=Nt(),{isMobile:a}=Wr();return Ct("div",{className:xr()({"vm-alert":!0,["vm-alert_".concat(t)]:t,"vm-alert_dark":r,"vm-alert_mobile":a}),children:[Ct("div",{className:"vm-alert__icon",children:Kr[t||"info"]}),Ct("div",{className:"vm-alert__content",children:n})]})},Zr=(0,r.createContext)({showInfoMessage:()=>{}}),Gr={dashboardsSettings:[],dashboardsLoading:!1,dashboardsError:""};function Jr(e,t){switch(t.type){case"SET_DASHBOARDS_SETTINGS":return{...e,dashboardsSettings:t.payload};case"SET_DASHBOARDS_LOADING":return{...e,dashboardsLoading:t.payload};case"SET_DASHBOARDS_ERROR":return{...e,dashboardsError:t.payload};default:throw new Error}}const Xr=(0,r.createContext)({}),ea=()=>(0,r.useContext)(Xr).state,ta=function(){for(var e=arguments.length,t=new Array(e),n=0;nn=>{let{children:r}=n;return Ct(e,{children:Ct(t,{children:r})})}),(e=>{let{children:t}=e;return Ct(xt.FK,{children:t})}))}(...[e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(kt,Mt),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Et.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(pn,mn),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(fn.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Sn,xn),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Cn.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Dr,Rr),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(zr.Provider,{value:i,children:t})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Vr,Hr),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct($r.Provider,{value:i,children:t})},e=>{let{children:t}=e;const{isMobile:n}=Wr(),[a,i]=(0,r.useState)({}),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(void 0);(0,r.useEffect)((()=>{if(!s)return;i({message:s.text,variant:s.type,key:Date.now()}),l(!0);const e=setTimeout(u,4e3);return()=>clearTimeout(e)}),[s]);const u=()=>{c(void 0),l(!1)};return Ct(Zr.Provider,{value:{showInfoMessage:c},children:[o&&Ct("div",{className:xr()({"vm-snackbar":!0,"vm-snackbar_mobile":n}),children:Ct(Qr,{variant:a.variant,children:Ct("div",{className:"vm-snackbar-content",children:[Ct("span",{children:a.message}),Ct("div",{className:"vm-snackbar-content__close",onClick:u,children:Ct(On,{})})]})})}),t]})},e=>{let{children:t}=e;const[n,a]=(0,r.useReducer)(Jr,Gr),i=(0,r.useMemo)((()=>({state:n,dispatch:a})),[n,a]);return Ct(Xr.Provider,{value:i,children:t})}]),na=e=>{let{activeMenu:t,label:n,value:r,color:a}=e;return Ct(Re,{className:xr()({"vm-header-nav-item":!0,"vm-header-nav-item_active":t===r}),style:{color:a},to:r,children:n})},ra=(e,t,n)=>{const a=(0,r.useCallback)((r=>{const a=null===e||void 0===e?void 0:e.current,i=r.target,o=(null===n||void 0===n?void 0:n.current)&&n.current.contains(i);!a||a.contains((null===r||void 0===r?void 0:r.target)||null)||o||t(r)}),[e,t]);Nr("mousedown",a),Nr("touchstart",a)},aa=e=>{let{variant:t="contained",color:n="primary",size:r="medium",ariaLabel:a,children:i,endIcon:o,startIcon:l,fullWidth:s=!1,className:c,disabled:u,onClick:d,onMouseDown:h}=e;return Ct("button",{className:xr()({"vm-button":!0,["vm-button_".concat(t,"_").concat(n)]:!0,["vm-button_".concat(r)]:r,"vm-button_icon":(l||o)&&!i,"vm-button_full-width":s,"vm-button_with-icon":l||o,"vm-button_disabled":u,[c||""]:c}),disabled:u,"aria-label":a,onClick:d,onMouseDown:h,children:Ct(xt.FK,{children:[l&&Ct("span",{className:"vm-button__start-icon",children:l}),i&&Ct("span",{children:i}),o&&Ct("span",{className:"vm-button__end-icon",children:o})]})})},ia=e=>{let{children:t,buttonRef:n,placement:a="bottom-left",open:i=!1,onClose:o,offset:l={top:6,left:0},clickOutside:s=!0,fullWidth:c,title:u,disabledFullScreen:d,variant:h}=e;const{isMobile:m}=Wr(),p=ae(),f=ne(),[v,g]=(0,r.useState)({width:0,height:0}),[y,_]=(0,r.useState)(!1),b=(0,r.useRef)(null);(0,r.useEffect)((()=>(_(i),!i&&o&&o(),i&&m&&!d&&(document.body.style.overflow="hidden"),()=>{document.body.style.overflow="auto"})),[i]),(0,r.useEffect)((()=>{var e,t;g({width:(null===b||void 0===b||null===(e=b.current)||void 0===e?void 0:e.clientWidth)||0,height:(null===b||void 0===b||null===(t=b.current)||void 0===t?void 0:t.clientHeight)||0}),_(!1)}),[b]);const w=(0,r.useMemo)((()=>{const e=n.current;if(!e||!y)return{};const t=e.getBoundingClientRect(),r={top:0,left:0,width:"auto"},i="bottom-right"===a||"top-right"===a,o=null===a||void 0===a?void 0:a.includes("top"),s=(null===l||void 0===l?void 0:l.top)||0,u=(null===l||void 0===l?void 0:l.left)||0;r.left=r.left=t.left+u,r.top=t.height+t.top+s,i&&(r.left=t.right-v.width),o&&(r.top=t.top-v.height-s);const{innerWidth:d,innerHeight:h}=window,m=r.top+v.height+20>h,p=r.top-20<0,f=r.left+v.width+20>d,g=r.left-20<0;return m&&(r.top=t.top-v.height-s),p&&(r.top=t.height+t.top+s),f&&(r.left=t.right-v.width-u),g&&(r.left=t.left+u),c&&(r.width="".concat(t.width,"px")),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[n,a,y,t,c]),k=()=>{_(!1),o()};(0,r.useEffect)((()=>{if(!b.current||!y||m&&!d)return;const{right:e,width:t}=b.current.getBoundingClientRect();if(e>window.innerWidth){const e=window.innerWidth-20-t;b.current.style.left=e{y&&m&&!d&&(p(f,{replace:!0}),o())}),[y,m,d,f,o]);return Nr("scroll",k),Nr("popstate",x),ra(b,(()=>{s&&k()}),n),Ct(xt.FK,{children:(y||!v.width)&&r.default.createPortal(Ct("div",{className:xr()({"vm-popper":!0,["vm-popper_".concat(h)]:h,"vm-popper_mobile":m&&!d,"vm-popper_open":(m||Object.keys(w).length)&&y}),ref:b,style:m&&!d?{}:w,children:[(u||m&&!d)&&Ct("div",{className:"vm-popper-header",children:[Ct("p",{className:"vm-popper-header__title",children:u}),Ct(aa,{variant:"text",color:"dark"===h?"white":"primary",size:"small",onClick:e=>{e.stopPropagation(),o()},ariaLabel:"close",children:Ct(On,{})})]}),t]}),document.body)})},oa=e=>{const[t,n]=(0,r.useState)(!!e),a=(0,r.useCallback)((()=>n(!0)),[]),i=(0,r.useCallback)((()=>n(!1)),[]),o=(0,r.useCallback)((()=>n((e=>!e))),[]);return{value:t,setValue:n,setTrue:a,setFalse:i,toggle:o}},la=e=>{let{activeMenu:t,label:n,color:a,background:i,submenu:o,direction:l}=e;const{pathname:s}=ne(),[c,u]=(0,r.useState)(null),d=(0,r.useRef)(null),{value:h,setFalse:m,setTrue:p}=oa(!1),f=()=>{c&&clearTimeout(c);const e=setTimeout(m,300);u(e)};return(0,r.useEffect)((()=>{m()}),[s]),"column"===l?Ct(xt.FK,{children:o.map((e=>Ct(na,{activeMenu:t,value:e.value||"",label:e.label||""},e.value)))}):Ct("div",{className:xr()({"vm-header-nav-item":!0,"vm-header-nav-item_sub":!0,"vm-header-nav-item_open":h,"vm-header-nav-item_active":o.find((e=>e.value===t))}),style:{color:a},onMouseEnter:()=>{p(),c&&clearTimeout(c)},onMouseLeave:f,ref:d,children:[n,Ct(Vn,{}),Ct(ia,{open:h,placement:"bottom-left",offset:{top:12,left:0},onClose:m,buttonRef:d,children:Ct("div",{className:"vm-header-nav-item-submenu",style:{background:i},onMouseLeave:f,onMouseEnter:()=>{c&&clearTimeout(c)},children:o.map((e=>Ct(na,{activeMenu:t,value:e.value||"",label:e.label||"",color:a},e.value)))})})]})},sa={label:"Explore",submenu:[{label:qe[Ye.metrics].title,value:Ye.metrics},{label:qe[Ye.cardinality].title,value:Ye.cardinality},{label:qe[Ye.topQueries].title,value:Ye.topQueries},{label:qe[Ye.activeQueries].title,value:Ye.activeQueries}]},ca={label:"Tools",submenu:[{label:qe[Ye.trace].title,value:Ye.trace},{label:qe[Ye.queryAnalyzer].title,value:Ye.queryAnalyzer},{label:qe[Ye.withTemplate].title,value:Ye.withTemplate},{label:qe[Ye.relabel].title,value:Ye.relabel}]},ua=[{label:qe[Ye.logs].title,value:Ye.home}],da=[{label:qe[Ye.anomaly].title,value:Ye.home}],ha=[{label:qe[Ye.home].title,value:Ye.home},sa,ca],ma=e=>{let{color:t,background:n,direction:a}=e;const i=Ke(),{dashboardsSettings:o}=ea(),{pathname:l}=ne(),[s,c]=(0,r.useState)(l),u=(0,r.useMemo)((()=>{switch({NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){case He.logs:return ua;case He.anomaly:return da;default:return[...ha,{label:qe[Ye.dashboards].title,value:Ye.dashboards,hide:i||!o.length}].filter((e=>!e.hide))}}),[i,o]);return(0,r.useEffect)((()=>{c(l)}),[l]),Ct("nav",{className:xr()({"vm-header-nav":!0,["vm-header-nav_".concat(a)]:a}),children:u.map((e=>e.submenu?Ct(la,{activeMenu:s,label:e.label||"",submenu:e.submenu,color:t,background:n,direction:a},e.label):Ct(na,{activeMenu:s,value:e.value||"",label:e.label||"",color:t},e.value)))})},pa=e=>{let{title:t,children:n,onClose:a,className:i,isOpen:o=!0}=e;const{isMobile:l}=Wr(),s=ae(),c=ne(),u=(0,r.useCallback)((e=>{o&&"Escape"===e.key&&a()}),[o]),d=e=>{e.stopPropagation()},h=(0,r.useCallback)((()=>{o&&(s(c,{replace:!0}),a())}),[o,c,a]);return(0,r.useEffect)((()=>{if(o)return document.body.style.overflow="hidden",()=>{document.body.style.overflow="auto"}}),[o]),Nr("popstate",h),Nr("keyup",u),r.default.createPortal(Ct("div",{className:xr()({"vm-modal":!0,"vm-modal_mobile":l,["".concat(i)]:i}),onMouseDown:a,children:Ct("div",{className:"vm-modal-content",children:[Ct("div",{className:"vm-modal-content-header",onMouseDown:d,children:[t&&Ct("div",{className:"vm-modal-content-header__title",children:t}),Ct("div",{className:"vm-modal-header__close",children:Ct(aa,{variant:"text",size:"small",onClick:a,ariaLabel:"close",children:Ct(On,{})})})]}),Ct("div",{className:"vm-modal-content-body",onMouseDown:d,children:n})]})}),document.body)},fa=e=>{let{children:t,title:n,open:a,placement:i="bottom-center",offset:o={top:6,left:0}}=e;const{isMobile:l}=Wr(),[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)({width:0,height:0}),h=(0,r.useRef)(null),m=(0,r.useRef)(null),p=()=>c(!1);(0,r.useEffect)((()=>{if(m.current&&s)return d({width:m.current.clientWidth,height:m.current.clientHeight}),window.addEventListener("scroll",p),()=>{window.removeEventListener("scroll",p)}}),[s,n]);const f=(0,r.useMemo)((()=>{var e;const t=null===h||void 0===h||null===(e=h.current)||void 0===e?void 0:e.base;if(!t||!s)return{};const n=t.getBoundingClientRect(),r={top:0,left:0},a="bottom-right"===i||"top-right"===i,l="bottom-left"===i||"top-left"===i,c=null===i||void 0===i?void 0:i.includes("top"),d=(null===o||void 0===o?void 0:o.top)||0,m=(null===o||void 0===o?void 0:o.left)||0;r.left=n.left-(u.width-n.width)/2+m,r.top=n.height+n.top+d,a&&(r.left=n.right-u.width),l&&(r.left=n.left+m),c&&(r.top=n.top-u.height-d);const{innerWidth:p,innerHeight:f}=window,v=r.top+u.height+20>f,g=r.top-20<0,y=r.left+u.width+20>p,_=r.left-20<0;return v&&(r.top=n.top-u.height-d),g&&(r.top=n.height+n.top+d),y&&(r.left=n.right-u.width-m),_&&(r.left=n.left+m),r.top<0&&(r.top=20),r.left<0&&(r.left=20),r}),[h,i,s,u]),v=()=>{"boolean"!==typeof a&&c(!0)},g=()=>{c(!1)};return(0,r.useEffect)((()=>{"boolean"===typeof a&&c(a)}),[a]),(0,r.useEffect)((()=>{var e;const t=null===h||void 0===h||null===(e=h.current)||void 0===e?void 0:e.base;if(t)return t.addEventListener("mouseenter",v),t.addEventListener("mouseleave",g),()=>{t.removeEventListener("mouseenter",v),t.removeEventListener("mouseleave",g)}}),[h]),Ct(xt.FK,{children:[Ct(r.Fragment,{ref:h,children:t}),!l&&s&&r.default.createPortal(Ct("div",{className:"vm-tooltip",ref:m,style:f,children:n}),document.body)]})},va=Ct("code",{children:Yr()?"Cmd":"Ctrl"}),ga=[{title:"Zoom in",description:Ct(xt.FK,{children:["To zoom in, hold down the ",va," + ",Ct("code",{children:"scroll up"}),", or press the ",Ct("code",{children:"+"}),". Also, you can zoom in on a range on the graph by holding down your mouse button and selecting the range."]})},{title:"Zoom out",description:Ct(xt.FK,{children:["To zoom out, hold down the ",va," + ",Ct("code",{children:"scroll down"}),", or press the ",Ct("code",{children:"-"}),"."]})},{title:"Move horizontal axis",description:Ct(xt.FK,{children:["To move the graph, hold down the ",va," + ",Ct("code",{children:"drag"})," the graph to the right or left."]})},{title:"Fixing a tooltip",description:Ct(xt.FK,{children:["To fix the tooltip, ",Ct("code",{children:"click"})," mouse when it's open. Then, you can drag the fixed tooltip by ",Ct("code",{children:"clicking"})," and ",Ct("code",{children:"dragging"})," on the ",Ct(ir,{})," icon."]})},{title:"Set a custom range for the vertical axis",description:Ct(xt.FK,{children:["To set a custom range for the vertical axis, click on the ",Ct(Pn,{})," icon located in the upper right corner of the graph, activate the toggle, and set the values."]})}],ya=[{title:"Show/hide a legend item",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on a legend item to isolate it on the graph.",va," + ",Ct("code",{children:"click"})," on a legend item to remove it from the graph. To revert to the previous state, click again."]})},{title:"Copy label key-value pairs",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on a label key-value pair to save it to the clipboard."]})},{title:"Collapse/Expand the legend group",description:Ct(xt.FK,{children:[Ct("code",{children:"click"})," on the group name (e.g. ",Ct("b",{children:'Query 1: {__name__!=""}'}),") to collapse or expand the legend."]})}],_a=ga.concat(ya),ba=()=>{const{value:e,setFalse:t,setTrue:n}=oa(!1);return Ct(xt.FK,{children:[Ct(fa,{title:"Show tips on working with the graph",children:Ct(aa,{variant:"text",color:"gray",startIcon:Ct(mr,{}),onClick:n,ariaLabel:"open the tips"})}),e&&Ct(pa,{title:"Tips on working with the graph and the legend",onClose:t,children:Ct("div",{className:"fc-graph-tips",children:_a.map((e=>{let{title:t,description:n}=e;return Ct("div",{className:"fc-graph-tips-item",children:[Ct("h4",{className:"fc-graph-tips-item__action",children:t}),Ct("p",{className:"fc-graph-tips-item__description",children:n})]},t)}))})})]})},wa=Ct("code",{children:Yr()?"Cmd":"Ctrl"}),ka=Ct(xt.FK,{children:[Ct("code",{children:Yr()?"Option":"Ctrl"})," + ",Ct("code",{children:"Space"})]}),xa=[{title:"Query",list:[{keys:Ct("code",{children:"Enter"}),description:"Run"},{keys:Ct(xt.FK,{children:[Ct("code",{children:"Shift"})," + ",Ct("code",{children:"Enter"})]}),description:"Multi-line queries"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"Arrow Up"})]}),description:"Previous command from the Query history"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"Arrow Down"})]}),description:"Next command from the Query history"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"click"})," by ",Ct(tr,{})]}),description:"Toggle multiple queries"},{keys:ka,description:"Show quick autocomplete tips"}]},{title:"Graph",readMore:Ct(ba,{}),list:[{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"scroll Up"})," or ",Ct("code",{children:"+"})]}),description:"Zoom in"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"scroll Down"})," or ",Ct("code",{children:"-"})]}),description:"Zoom out"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"drag"})]}),description:"Move the graph left/right"},{keys:Ct(xt.FK,{children:Ct("code",{children:"click"})}),description:"Select the series in the legend"},{keys:Ct(xt.FK,{children:[wa," + ",Ct("code",{children:"click"})]}),description:"Toggle multiple series in the legend"}]}],Sa="Shortcut keys",Ca=Yr(),Ea=Ca?"Cmd + /":"F1",Na=e=>{let{showTitle:t}=e;const n=Ke(),{value:a,setTrue:i,setFalse:o}=oa(!1),l=(0,r.useCallback)((e=>{const t=Ca&&"/"===e.key&&e.metaKey,n=!Ca&&"F1"===e.key&&!e.metaKey;(t||n)&&i()}),[i]);return Nr("keydown",l),Ct(xt.FK,{children:[Ct(fa,{open:!0!==t&&void 0,title:"".concat(Sa," (").concat(Ea,")"),placement:"bottom-center",children:Ct(aa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(qn,{}),onClick:i,ariaLabel:Sa,children:t&&Sa})}),a&&Ct(pa,{title:"Shortcut keys",onClose:o,children:Ct("div",{className:"vm-shortcuts",children:xa.map((e=>Ct("div",{className:"vm-shortcuts-section",children:[e.readMore&&Ct("div",{className:"vm-shortcuts-section__read-more",children:e.readMore}),Ct("h3",{className:"vm-shortcuts-section__title",children:e.title}),Ct("div",{className:"vm-shortcuts-section-list",children:e.list.map(((t,n)=>Ct("div",{className:"vm-shortcuts-section-list-item",children:[Ct("div",{className:"vm-shortcuts-section-list-item__key",children:t.keys}),Ct("p",{className:"vm-shortcuts-section-list-item__description",children:t.description})]},"".concat(e.title,"_").concat(n))))})]},e.title)))})})]})},Aa=e=>{let{open:t}=e;return Ct("button",{className:xr()({"vm-menu-burger":!0,"vm-menu-burger_opened":t}),"aria-label":"menu",children:Ct("span",{})})},{REACT_APP_TYPE:Ma}={},Ta=Ma===He.logs,La=e=>{let{background:t,color:n}=e;const{pathname:a}=ne(),{isMobile:i}=Wr(),o=(0,r.useRef)(null),{value:l,toggle:s,setFalse:c}=oa(!1);return(0,r.useEffect)(c,[a]),ra(o,c),Ct("div",{className:"vm-header-sidebar",ref:o,children:[Ct("div",{className:xr()({"vm-header-sidebar-button":!0,"vm-header-sidebar-button_open":l}),onClick:s,children:Ct(Aa,{open:l})}),Ct("div",{className:xr()({"vm-header-sidebar-menu":!0,"vm-header-sidebar-menu_open":l}),children:[Ct("div",{children:Ct(ma,{color:n,background:t,direction:"column"})}),Ct("div",{className:"vm-header-sidebar-menu-settings",children:!i&&!Ta&&Ct(Na,{showTitle:!0})})]})]})},Pa=e=>{let{controlsComponent:t,isMobile:n,...a}=e;const i=Ke(),{pathname:o}=ne(),{accountIds:l}=(()=>{const{useTenantID:e}=We(),t=Ke(),{serverUrl:n}=Nt(),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(),[s,c]=(0,r.useState)([]),u=(0,r.useMemo)((()=>"".concat(n.replace(/^(.+)(\/select.+)/,"$1"),"/admin/tenants")),[n]),d=(0,r.useMemo)((()=>!!Ge(n)),[n]),h=t?!e:!d;return(0,r.useEffect)((()=>{h||(async()=>{i(!0);try{const e=await fetch(u),t=await e.json(),n=t.data||[];c(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?l(void 0):l("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(rp){rp instanceof Error&&l("".concat(rp.name,": ").concat(rp.message))}i(!1)})().catch(console.error)}),[u]),{accountIds:s,isLoading:a,error:o}})(),{value:s,toggle:c,setFalse:u}=oa(!1),d=Ct(t,{...a,isMobile:n,accountIds:l,headerSetup:(0,r.useMemo)((()=>(qe[o]||{}).header||{}),[o])});return n?Ct(xt.FK,{children:[Ct("div",{children:Ct(aa,{className:xr()({"vm-header-button":!i}),startIcon:Ct(dr,{}),onClick:c,ariaLabel:"controls"})}),Ct(pa,{title:"Controls",onClose:u,isOpen:s,className:xr()({"vm-header-controls-modal":!0,"vm-header-controls-modal_open":s}),children:d})]}):d},{REACT_APP_TYPE:Oa}={},Ia=Oa===He.logs||Oa===He.anomaly,Ra=()=>{switch(Oa){case He.logs:return Ct(Mn,{});case He.anomaly:return Ct(Tn,{});default:return Ct(An,{})}},Da=e=>{let{controlsComponent:t}=e;const{isMobile:n}=Wr(),a=Ar(),i=(0,r.useMemo)((()=>window.innerWidth<1e3),[a]),{isDarkTheme:o}=Nt(),l=Ke(),s=(0,r.useMemo)((()=>ft(o?"color-background-block":"color-primary")),[o]),{background:c,color:u}=(0,r.useMemo)((()=>{const{headerStyles:{background:e=(l?"#FFF":s),color:t=(l?s:"#FFF")}={}}=We();return{background:e,color:t}}),[s]),d=ae(),h=()=>{d({pathname:Ye.home}),window.location.reload()};return Ct("header",{className:xr()({"vm-header":!0,"vm-header_app":l,"vm-header_dark":o,"vm-header_sidebar":i,"vm-header_mobile":n}),style:{background:c,color:u},children:[i?Ct(La,{background:c,color:u}):Ct(xt.FK,{children:[!l&&Ct("div",{className:xr()({"vm-header-logo":!0,"vm-header-logo_logs":Ia}),onClick:h,style:{color:u},children:Ct(Ra,{})}),Ct(ma,{color:u,background:c})]}),i&&Ct("div",{className:xr()({"vm-header-logo":!0,"vm-header-logo_mobile":!0,"vm-header-logo_logs":Ia}),onClick:h,style:{color:u},children:Ct(Ra,{})}),Ct(Pa,{controlsComponent:t,displaySidebar:i,isMobile:n})]})},za=(0,r.memo)((()=>{const e="2019-".concat((new Date).getFullYear());return Ct("footer",{className:"vm-footer",children:[Ct("a",{className:"vm-link vm-footer__website",target:"_blank",href:"https://victoriametrics.com/",rel:"me noreferrer",children:[Ct(Ln,{}),"victoriametrics.com"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/MetricsQL.html",rel:"help noreferrer",children:[Ct(Zn,{}),"MetricsQL"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://docs.victoriametrics.com/#vmui",rel:"help noreferrer",children:[Ct(lr,{}),"Documentation"]}),Ct("a",{className:"vm-link vm-footer__link",target:"_blank",href:"https://github.com/VictoriaMetrics/VictoriaMetrics/issues/new/choose",rel:"noreferrer",children:[Ct(sr,{}),"Create an issue"]}),Ct("div",{className:"vm-footer__copyright",children:["\xa9 ",e," VictoriaMetrics"]})]})})),Fa=()=>{const e=Ke(),{serverUrl:t}=Nt(),n=(0,r.useContext)(Xr).dispatch,[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(""),[s,c]=(0,r.useState)([]),u=async()=>{try{const e=window.__VMUI_PREDEFINED_DASHBOARDS__;if(null===e||void 0===e||!e.length)return[];const t=await Promise.all(e.map((async e=>(async e=>{const t=await fetch("./dashboards/".concat(e));return await t.json()})(e))));c((e=>[...t,...e]))}catch(rp){rp instanceof Error&&l("".concat(rp.name,": ").concat(rp.message))}};return(0,r.useEffect)((()=>{e||(c([]),(async()=>{if(t&&!{NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){l(""),i(!0);try{const e=await fetch("".concat(t,"/vmui/custom-dashboards")),n=await e.json();if(e.ok){const{dashboardsSettings:e}=n;e&&e.length>0?c((t=>[...t,...e])):await u(),i(!1)}else await u(),l(n.error),i(!1)}catch(rp){i(!1),rp instanceof Error&&l("".concat(rp.name,": ").concat(rp.message)),await u()}}})())}),[t]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_SETTINGS",payload:s})}),[s]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_LOADING",payload:a})}),[a]),(0,r.useEffect)((()=>{n({type:"SET_DASHBOARDS_ERROR",payload:o})}),[o]),{dashboardsSettings:s,isLoading:a,error:o}},ja=e=>{let{error:t,warning:n,info:a}=e;const i=(0,r.useRef)(null),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(!1),u=(0,r.useMemo)((()=>t?"ERROR: ":n?"WARNING: ":""),[t,n]),d="".concat(u).concat(t||n||a),h=()=>{const e=i.current;if(e){const{offsetWidth:t,scrollWidth:n,offsetHeight:r,scrollHeight:a}=e;l(t+1{c(!1),h()}),[i,d]),Nr("resize",h),t||n||a?Ct("span",{className:xr()({"vm-text-field__error":!0,"vm-text-field__warning":n&&!t,"vm-text-field__helper-text":!n&&!t,"vm-text-field__error_overflowed":o,"vm-text-field__error_full":s}),"data-show":!!d,ref:i,onClick:()=>{o&&(c(!0),l(!1))},children:d}):null},Ha=e=>{let{label:t,value:n,type:a="text",error:i="",warning:o="",helperText:l="",placeholder:s,endIcon:c,startIcon:u,disabled:d=!1,autofocus:h=!1,inputmode:m="text",caretPosition:p,onChange:f,onEnter:v,onKeyDown:g,onFocus:y,onBlur:_,onChangeCaret:b}=e;const{isDarkTheme:w}=Nt(),{isMobile:k}=Wr(),x=(0,r.useRef)(null),S=(0,r.useRef)(null),C=(0,r.useMemo)((()=>"textarea"===a?S:x),[a]),[E,N]=(0,r.useState)([0,0]),A=xr()({"vm-text-field__input":!0,"vm-text-field__input_error":i,"vm-text-field__input_warning":!i&&o,"vm-text-field__input_icon-start":u,"vm-text-field__input_disabled":d,"vm-text-field__input_textarea":"textarea"===a}),M=e=>{const{selectionStart:t,selectionEnd:n}=e;N([t||0,n||0])},T=e=>{M(e.currentTarget)},L=e=>{g&&g(e);const{key:t,ctrlKey:n,metaKey:r}=e,i="Enter"===t;("textarea"!==a?i:i&&(r||n))&&v&&(e.preventDefault(),v())},P=e=>{M(e.currentTarget)},O=e=>{d||(f&&f(e.currentTarget.value),M(e.currentTarget))},I=()=>{y&&y()},R=()=>{_&&_()},D=e=>{try{C.current&&C.current.setSelectionRange(e[0],e[1])}catch(rp){return rp}};return(0,r.useEffect)((()=>{var e;h&&!k&&(null===C||void 0===C||null===(e=C.current)||void 0===e?void 0:e.focus)&&C.current.focus()}),[C,h]),(0,r.useEffect)((()=>{b&&b(E)}),[E]),(0,r.useEffect)((()=>{D(E)}),[n]),(0,r.useEffect)((()=>{p&&D(p)}),[p]),Ct("label",{className:xr()({"vm-text-field":!0,"vm-text-field_textarea":"textarea"===a,"vm-text-field_dark":w}),"data-replicated-value":n,children:[u&&Ct("div",{className:"vm-text-field__icon-start",children:u}),c&&Ct("div",{className:"vm-text-field__icon-end",children:c}),"textarea"===a?Ct("textarea",{className:A,disabled:d,ref:S,value:n,rows:1,inputMode:m,placeholder:s,autoCapitalize:"none",onInput:O,onKeyDown:L,onKeyUp:P,onFocus:I,onBlur:R,onMouseUp:T}):Ct("input",{className:A,disabled:d,ref:x,value:n,type:a,placeholder:s,inputMode:m,autoCapitalize:"none",onInput:O,onKeyDown:L,onKeyUp:P,onFocus:I,onBlur:R,onMouseUp:T}),t&&Ct("span",{className:"vm-text-field__label",children:t}),Ct(ja,{error:i,warning:o,info:l})]})},Va=e=>{let{accountIds:t}=e;const n=Ke(),{isMobile:a}=Wr(),{tenantId:i,serverUrl:o}=Nt(),l=At(),s=gn(),[c,u]=(0,r.useState)(""),d=(0,r.useRef)(null),{value:h,toggle:m,setFalse:p}=oa(!1),f=(0,r.useMemo)((()=>{if(!c)return t;try{const e=new RegExp(c,"i");return t.filter((t=>e.test(t))).sort(((t,n)=>{var r,a;return((null===(r=t.match(e))||void 0===r?void 0:r.index)||0)-((null===(a=n.match(e))||void 0===a?void 0:a.index)||0)}))}catch(rp){return[]}}),[c,t]),v=(0,r.useMemo)((()=>t.length>1),[t]),g=e=>()=>{const t=e;if(l({type:"SET_TENANT_ID",payload:t}),o){const e=Ze(o,t);if(e===o)return;l({type:"SET_SERVER",payload:e}),s({type:"RUN_QUERY"})}p()};return(0,r.useEffect)((()=>{const e=Ge(o);i&&i!==e?g(i)():g(e)()}),[o]),v?Ct("div",{className:"vm-tenant-input",children:[Ct(fa,{title:"Define Tenant ID if you need request to another storage",children:Ct("div",{ref:d,children:a?Ct("div",{className:"vm-mobile-option",onClick:m,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(ur,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Tenant ID"}),Ct("span",{className:"vm-mobile-option-text__value",children:i})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(aa,{className:n?"":"vm-header-button",variant:"contained",color:"primary",fullWidth:!0,startIcon:Ct(ur,{}),endIcon:Ct("div",{className:xr()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":h}),children:Ct(Hn,{})}),onClick:m,children:i})})}),Ct(ia,{open:h,placement:"bottom-right",onClose:p,buttonRef:d,title:a?"Define Tenant ID":void 0,children:Ct("div",{className:xr()({"vm-list vm-tenant-input-list":!0,"vm-list vm-tenant-input-list_mobile":a}),children:[Ct("div",{className:"vm-tenant-input-list__search",children:Ct(Ha,{autofocus:!0,label:"Search",value:c,onChange:u,type:"search"})}),f.map((e=>Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":a,"vm-list-item_active":e===i}),onClick:g(e),children:e},e)))]})})]}):null};const $a=function(e){const t=(0,r.useRef)();return(0,r.useEffect)((()=>{t.current=e}),[e]),t.current},Ua=()=>{const e=Ke(),{isMobile:t}=Wr(),{customStep:n,isHistogram:a}=Ur(),{period:{step:i,end:o,start:l}}=vn(),s=Br(),c=$a(o-l),u=(0,r.useMemo)((()=>Kt(o-l,a)),[i,a]),[d,h]=(0,r.useState)(n||u),[m,p]=(0,r.useState)(""),{value:f,toggle:v,setFalse:g}=oa(!1),y=(0,r.useRef)(null),_=e=>{const t=e||d||u||"1s",n=(t.match(/[a-zA-Z]+/g)||[]).length?t:"".concat(t,"s");s({type:"SET_CUSTOM_STEP",payload:n}),h(n),p("")},b=()=>{_(),g()},w=e=>{const t=e.match(/[-+]?([0-9]*\.[0-9]+|[0-9]+)/g)||[],n=e.match(/[a-zA-Z]+/g)||[],r=t.length&&t.every((e=>parseFloat(e)>0)),a=n.every((e=>Vt.find((t=>t.short===e)))),i=r&&a;h(e),p(i?"":ht.validStep)};return(0,r.useEffect)((()=>{n&&_(n)}),[n]),(0,r.useEffect)((()=>{!n&&u&&_(u)}),[u]),(0,r.useEffect)((()=>{o-l!==c&&c&&u&&_(u)}),[o,l,c,u]),(0,r.useEffect)((()=>{i!==n&&i!==u||_(u)}),[a]),Ct("div",{className:"vm-step-control",ref:y,children:[t?Ct("div",{className:"vm-mobile-option",onClick:v,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(or,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Step"}),Ct("span",{className:"vm-mobile-option-text__value",children:d})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(fa,{title:"Query resolution step width",children:Ct(aa,{className:e?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(or,{}),onClick:v,children:Ct("p",{children:["STEP",Ct("p",{className:"vm-step-control__value",children:d})]})})}),Ct(ia,{open:f,placement:"bottom-right",onClose:b,buttonRef:y,title:t?"Query resolution step width":void 0,children:Ct("div",{className:xr()({"vm-step-control-popper":!0,"vm-step-control-popper_mobile":t}),children:[Ct(Ha,{autofocus:!0,label:"Step value",value:d,error:m,onChange:w,onEnter:()=>{_(),b()},onFocus:()=>{document.activeElement instanceof HTMLInputElement&&document.activeElement.select()},onBlur:_,endIcon:Ct(fa,{title:"Set default step value: ".concat(u),children:Ct(aa,{size:"small",variant:"text",color:"primary",startIcon:Ct(In,{}),onClick:()=>{const e=u||"1s";w(e),_(e)},ariaLabel:"reset step"})})}),Ct("div",{className:"vm-step-control-popper-info",children:[Ct("code",{children:"step"})," - the ",Ct("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 ",Ct("code",{children:"query"})," is executed at",Ct("code",{children:"start"}),", ",Ct("code",{children:"start+step"}),", ",Ct("code",{children:"start+2*step"}),", \u2026, ",Ct("code",{children:"end"})," timestamps.",Ct("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"})]})]})})]})},Ba=e=>{let{relativeTime:t,setDuration:n}=e;const{isMobile:r}=Wr();return Ct("div",{className:xr()({"vm-time-duration":!0,"vm-time-duration_mobile":r}),children:en.map((e=>{let{id:a,duration:i,until:o,title:l}=e;return Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":r,"vm-list-item_active":a===t}),onClick:(s={duration:i,until:o(),id:a},()=>{n(s)}),children:l||i},a);var s}))})},qa=e=>{let{viewDate:t,showArrowNav:n,onChangeViewDate:r,toggleDisplayYears:a}=e;return Ct("div",{className:"vm-calendar-header",children:[Ct("div",{className:"vm-calendar-header-left",onClick:a,children:[Ct("span",{className:"vm-calendar-header-left__date",children:t.format("MMMM YYYY")}),Ct("div",{className:"vm-calendar-header-left__select-year",children:Ct(Vn,{})})]}),n&&Ct("div",{className:"vm-calendar-header-right",children:[Ct("div",{className:"vm-calendar-header-right__prev",onClick:()=>{r(t.subtract(1,"month"))},children:Ct(Hn,{})}),Ct("div",{className:"vm-calendar-header-right__next",onClick:()=>{r(t.add(1,"month"))},children:Ct(Hn,{})})]})]})},Ya=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],Wa=e=>{let{viewDate:t,selectDate:n,onChangeSelectDate:a}=e;const o="YYYY-MM-DD",l=i()().tz().startOf("day"),s=(0,r.useMemo)((()=>{const e=new Array(42).fill(null),n=t.startOf("month"),r=t.endOf("month").diff(n,"day")+1,a=new Array(r).fill(n).map(((e,t)=>e.add(t,"day"))),i=n.day();return e.splice(i,r,...a),e}),[t]),c=e=>()=>{e&&a(e)};return Ct("div",{className:"vm-calendar-body",children:[Ya.map((e=>Ct("div",{className:"vm-calendar-body-cell vm-calendar-body-cell_weekday",children:e[0]},e))),s.map(((e,t)=>Ct("div",{className:xr()({"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.format(o))===n.format(o),"vm-calendar-body-cell_day_today":(e&&e.format(o))===l.format(o)}),onClick:c(e),children:e&&e.format("D")},e?e.format(o):t)))]})},Ka=e=>{let{viewDate:t,onChangeViewDate:n}=e;const a=i()().format("YYYY"),o=(0,r.useMemo)((()=>t.format("YYYY")),[t]),l=(0,r.useMemo)((()=>{const e=i()().subtract(9,"year");return new Array(18).fill(e).map(((e,t)=>e.add(t,"year")))}),[t]);(0,r.useEffect)((()=>{const e=document.getElementById("vm-calendar-year-".concat(o));e&&e.scrollIntoView({block:"center"})}),[]);return Ct("div",{className:"vm-calendar-years",children:l.map((e=>{return Ct("div",{className:xr()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("YYYY")===o,"vm-calendar-years__year_today":e.format("YYYY")===a}),id:"vm-calendar-year-".concat(e.format("YYYY")),onClick:(t=e,()=>{n(t)}),children:e.format("YYYY")},e.format("YYYY"));var t}))})},Qa=e=>{let{viewDate:t,selectDate:n,onChangeViewDate:a}=e;const o=i()().format("MM"),l=(0,r.useMemo)((()=>n.format("MM")),[n]),s=(0,r.useMemo)((()=>new Array(12).fill("").map(((e,n)=>i()(t).month(n)))),[t]);(0,r.useEffect)((()=>{const e=document.getElementById("vm-calendar-year-".concat(l));e&&e.scrollIntoView({block:"center"})}),[]);const c=e=>()=>{a(e)};return Ct("div",{className:"vm-calendar-years",children:s.map((e=>Ct("div",{className:xr()({"vm-calendar-years__year":!0,"vm-calendar-years__year_selected":e.format("MM")===l,"vm-calendar-years__year_today":e.format("MM")===o}),id:"vm-calendar-year-".concat(e.format("MM")),onClick:c(e),children:e.format("MMMM")},e.format("MM"))))})};var Za=function(e){return e[e.days=0]="days",e[e.months=1]="months",e[e.years=2]="years",e}(Za||{});const Ga=e=>{let{date:t,format:n=Lt,onChange:a}=e;const[o,l]=(0,r.useState)(Za.days),[s,c]=(0,r.useState)(i().tz(t)),[u,d]=(0,r.useState)(i().tz(t)),h=i()().startOf("day").tz(),m=h.format()===s.format(),{isMobile:p}=Wr(),f=e=>{c(e),l((e=>e===Za.years?Za.months:Za.days))};return(0,r.useEffect)((()=>{u.format()!==i().tz(t).format()&&a(u.format(n))}),[u]),(0,r.useEffect)((()=>{const e=i().tz(t);c(e),d(e)}),[t]),Ct("div",{className:xr()({"vm-calendar":!0,"vm-calendar_mobile":p}),children:[Ct(qa,{viewDate:s,onChangeViewDate:f,toggleDisplayYears:()=>{l((e=>e===Za.years?Za.days:Za.years))},showArrowNav:o===Za.days}),o===Za.days&&Ct(Wa,{viewDate:s,selectDate:u,onChangeSelectDate:e=>{d(e)}}),o===Za.years&&Ct(Ka,{viewDate:s,onChangeViewDate:f}),o===Za.months&&Ct(Qa,{selectDate:u,viewDate:s,onChangeViewDate:f}),!m&&o===Za.days&&Ct("div",{className:"vm-calendar-footer",children:Ct(aa,{variant:"text",size:"small",onClick:()=>{c(h)},children:"show today"})})]})},Ja=(0,r.forwardRef)(((e,t)=>{let{date:n,targetRef:a,format:o=Lt,onChange:l,label:s}=e;const c=(0,r.useMemo)((()=>i()(n).isValid()?i().tz(n):i()().tz()),[n]),{isMobile:u}=Wr(),{value:d,toggle:h,setFalse:m}=oa(!1);return Nr("click",h,a),Nr("keyup",(e=>{"Escape"!==e.key&&"Enter"!==e.key||m()})),Ct(xt.FK,{children:Ct(ia,{open:d,buttonRef:a,placement:"bottom-right",onClose:m,title:u?s:void 0,children:Ct("div",{ref:t,children:Ct(Ga,{date:c,format:o,onChange:e=>{l(e),m()}})})})})})),Xa=Ja;var ei=n(494),ti=n.n(ei);const ni=e=>i()(e).isValid()?i().tz(e).format(Lt):e,ri=e=>{let{value:t="",label:n,pickerLabel:a,pickerRef:o,onChange:l,onEnter:s}=e;const c=(0,r.useRef)(null),[u,d]=(0,r.useState)(null),[h,m]=(0,r.useState)(ni(t)),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)(!1),y=i()(h).isValid()?"":"Invalid date format";return(0,r.useEffect)((()=>{const e=ni(t);e!==h&&m(e),v&&(s(),g(!1))}),[t]),(0,r.useEffect)((()=>{p&&u&&(u.focus(),u.setSelectionRange(11,11),f(!1))}),[p]),Ct("div",{className:xr()({"vm-date-time-input":!0,"vm-date-time-input_error":y}),children:[Ct("label",{children:n}),Ct(ti(),{tabIndex:1,inputRef:d,mask:"9999-99-99 99:99:99",placeholder:"YYYY-MM-DD HH:mm:ss",value:h,autoCapitalize:"none",inputMode:"numeric",maskChar:null,onChange:e=>{m(e.currentTarget.value)},onBlur:()=>{l(h)},onKeyUp:e=>{"Enter"===e.key&&(l(h),g(!0))}}),y&&Ct("span",{className:"vm-date-time-input__error-text",children:y}),Ct("div",{className:"vm-date-time-input__icon",ref:c,children:Ct(aa,{variant:"text",color:"gray",size:"small",startIcon:Ct(Un,{}),ariaLabel:"calendar"})}),Ct(Xa,{label:a,ref:o,date:h,onChange:e=>{m(e),f(!0)},targetRef:c})]})},ai=()=>{const{isMobile:e}=Wr(),{isDarkTheme:t}=Nt(),n=(0,r.useRef)(null),a=Ar(),o=(0,r.useMemo)((()=>a.width>1120),[a]),[l,s]=(0,r.useState)(),[c,u]=(0,r.useState)(),{period:{end:d,start:h},relativeTime:m,timezone:p,duration:f}=vn(),v=gn(),g=Ke(),y=$a(p),{value:_,toggle:b,setFalse:w}=oa(!1),k=(0,r.useMemo)((()=>({region:p,utc:nn(p)})),[p]);(0,r.useEffect)((()=>{s(Gt(Xt(d)))}),[p,d]),(0,r.useEffect)((()=>{u(Gt(Xt(h)))}),[p,h]);const x=e=>{let{duration:t,until:n,id:r}=e;v({type:"SET_RELATIVE_TIME",payload:{duration:t,until:n,id:r}}),w()},S=(0,r.useMemo)((()=>({start:i().tz(Xt(h)).format(Lt),end:i().tz(Xt(d)).format(Lt)})),[h,d,p]),C=(0,r.useMemo)((()=>m&&"none"!==m?m.replace(/_/g," "):"".concat(S.start," - ").concat(S.end)),[m,S]),E=(0,r.useRef)(null),N=(0,r.useRef)(null),A=(0,r.useRef)(null),M=()=>{c&&l&&v({type:"SET_PERIOD",payload:{from:i().tz(c).toDate(),to:i().tz(l).toDate()}}),w()};return(0,r.useEffect)((()=>{const e=tn({relativeTimeId:m,defaultDuration:f,defaultEndInput:Xt(d)});y&&p!==y&&x({id:e.relativeTimeId,duration:e.duration,until:e.endInput})}),[p,y]),ra(n,(t=>{var n,r;if(e)return;const a=t.target,i=(null===E||void 0===E?void 0:E.current)&&(null===E||void 0===E||null===(n=E.current)||void 0===n?void 0:n.contains(a)),o=(null===N||void 0===N?void 0:N.current)&&(null===N||void 0===N||null===(r=N.current)||void 0===r?void 0:r.contains(a));i||o||w()})),Ct(xt.FK,{children:[Ct("div",{ref:A,children:e?Ct("div",{className:"vm-mobile-option",onClick:b,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct($n,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Time range"}),Ct("span",{className:"vm-mobile-option-text__value",children:C})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(fa,{title:o?"Time range controls":C,children:Ct(aa,{className:g?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct($n,{}),onClick:b,ariaLabel:"time range controls",children:o&&Ct("span",{children:C})})})}),Ct(ia,{open:_,buttonRef:A,placement:"bottom-right",onClose:w,clickOutside:!1,title:e?"Time range controls":"",children:Ct("div",{className:xr()({"vm-time-selector":!0,"vm-time-selector_mobile":e}),ref:n,children:[Ct("div",{className:"vm-time-selector-left",children:[Ct("div",{className:xr()({"vm-time-selector-left-inputs":!0,"vm-time-selector-left-inputs_dark":t}),children:[Ct(ri,{value:c,label:"From:",pickerLabel:"Date From",pickerRef:E,onChange:u,onEnter:M}),Ct(ri,{value:l,label:"To:",pickerLabel:"Date To",pickerRef:N,onChange:s,onEnter:M})]}),Ct("div",{className:"vm-time-selector-left-timezone",children:[Ct("div",{className:"vm-time-selector-left-timezone__title",children:k.region}),Ct("div",{className:"vm-time-selector-left-timezone__utc",children:k.utc})]}),Ct(aa,{variant:"text",startIcon:Ct(Bn,{}),onClick:()=>v({type:"RUN_QUERY_TO_NOW"}),children:"switch to now"}),Ct("div",{className:"vm-time-selector-left__controls",children:[Ct(aa,{color:"error",variant:"outlined",onClick:()=>{s(Gt(Xt(d))),u(Gt(Xt(h))),w()},children:"Cancel"}),Ct(aa,{color:"primary",onClick:M,children:"Apply"})]})]}),Ct(Ba,{relativeTime:m||"",setDuration:x})]})})]})},ii=()=>{const e=ae(),[t,n]=je();return{setSearchParamsFromKeys:(0,r.useCallback)((r=>{const a=!!Array.from(t.values()).length;let i=!1;Object.entries(r).forEach((e=>{let[n,r]=e;t.get(n)!=="".concat(r)&&(t.set(n,"".concat(r)),i=!0)})),i&&(a?n(t):e("?".concat(t.toString()),{replace:!0}))}),[t,e])}},oi=()=>{const{isMobile:e}=Wr(),t=Ke(),n=(0,r.useRef)(null),[a]=je(),{setSearchParamsFromKeys:o}=ii(),l=a.get("date")||i()().tz().format(Tt),s=(0,r.useMemo)((()=>i().tz(l).format(Tt)),[l]),c=e=>{o({date:e})};return(0,r.useEffect)((()=>{c(l)}),[]),Ct("div",{children:[Ct("div",{ref:n,children:e?Ct("div",{className:"vm-mobile-option",children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(Un,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Date control"}),Ct("span",{className:"vm-mobile-option-text__value",children:s})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(fa,{title:"Date control",children:Ct(aa,{className:t?"":"vm-header-button",variant:"contained",color:"primary",startIcon:Ct(Un,{}),children:s})})}),Ct(Xa,{label:"Date control",date:l||"",format:Tt,onChange:c,targetRef:n})]})},li=[{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"}],si=()=>{const{isMobile:e}=Wr(),t=gn(),n=Ke(),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)(li[0]),{value:s,toggle:c,setFalse:u}=oa(!1),d=(0,r.useRef)(null);(0,r.useEffect)((()=>{const e=o.seconds;let n;return a?n=setInterval((()=>{t({type:"RUN_QUERY"})}),1e3*e):l(li[0]),()=>{n&&clearInterval(n)}}),[o,a]);const h=e=>()=>{(e=>{(a&&!e.seconds||!a&&e.seconds)&&i((e=>!e)),l(e),u()})(e)};return Ct(xt.FK,{children:[Ct("div",{className:"vm-execution-controls",children:Ct("div",{className:xr()({"vm-execution-controls-buttons":!0,"vm-execution-controls-buttons_mobile":e,"vm-header-button":!n}),children:[!e&&Ct(fa,{title:"Refresh dashboard",children:Ct(aa,{variant:"contained",color:"primary",onClick:()=>{t({type:"RUN_QUERY"})},startIcon:Ct(jn,{}),ariaLabel:"refresh dashboard"})}),e?Ct("div",{className:"vm-mobile-option",onClick:c,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(In,{})}),Ct("div",{className:"vm-mobile-option-text",children:[Ct("span",{className:"vm-mobile-option-text__label",children:"Auto-refresh"}),Ct("span",{className:"vm-mobile-option-text__value",children:o.title})]}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(fa,{title:"Auto-refresh control",children:Ct("div",{ref:d,children:Ct(aa,{variant:"contained",color:"primary",fullWidth:!0,endIcon:Ct("div",{className:xr()({"vm-execution-controls-buttons__arrow":!0,"vm-execution-controls-buttons__arrow_open":s}),children:Ct(Hn,{})}),onClick:c,children:o.title})})})]})}),Ct(ia,{open:s,placement:"bottom-right",onClose:u,buttonRef:d,title:e?"Auto-refresh duration":void 0,children:Ct("div",{className:xr()({"vm-execution-controls-list":!0,"vm-execution-controls-list_mobile":e}),children:li.map((t=>Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":e,"vm-list-item_active":t.seconds===o.seconds}),onClick:h(t),children:t.title},t.seconds)))})})]})},ci="Enable to save the modified server URL to local storage, preventing reset upon page refresh.",ui="Disable to stop saving the server URL to local storage, reverting to the default URL on page refresh.",di=e=>{let{serverUrl:t,stateServerUrl:n,onChange:a,onEnter:i}=e;const{value:o,toggle:l}=oa(!!Xe("SERVER_URL")),[s,c]=(0,r.useState)("");return(0,r.useEffect)((()=>{n||c(ht.emptyServer),yt(n)||c(ht.validServer)}),[n]),(0,r.useEffect)((()=>{o?Je("SERVER_URL",t):et(["SERVER_URL"])}),[o]),(0,r.useEffect)((()=>{o&&Je("SERVER_URL",t)}),[t]),Ct("div",{children:[Ct("div",{className:"vm-server-configurator__title",children:"Server URL"}),Ct("div",{className:"vm-server-configurator-url",children:[Ct(Ha,{autofocus:!0,value:t,error:s,onChange:e=>{a(e||""),c("")},onEnter:i,inputmode:"url"}),Ct(fa,{title:o?ui:ci,children:Ct(aa,{className:"vm-server-configurator-url__button",variant:"text",color:o?"primary":"gray",onClick:l,startIcon:Ct(ur,{})})})]})]})},hi=[{label:"Graph",type:dt.chart},{label:"JSON",type:dt.code},{label:"Table",type:dt.table}],mi=e=>{let{limits:t,onChange:n,onEnter:a}=e;const{isMobile:i}=Wr(),[o,l]=(0,r.useState)({table:"",chart:"",code:""}),s=e=>r=>{((e,r)=>{const a=e||"";l((e=>({...e,[r]:+a<0?ht.positiveNumber:""}))),n({...t,[r]:a||1/0})})(r,e)};return Ct("div",{className:"vm-limits-configurator",children:[Ct("div",{className:"vm-server-configurator__title",children:["Series limits by tabs",Ct(fa,{title:"Set to 0 to disable the limit",children:Ct(aa,{variant:"text",color:"primary",size:"small",startIcon:Ct(Rn,{})})}),Ct("div",{className:"vm-limits-configurator-title__reset",children:Ct(aa,{variant:"text",color:"primary",size:"small",startIcon:Ct(In,{}),onClick:()=>{n(ot)},children:"Reset limits"})})]}),Ct("div",{className:xr()({"vm-limits-configurator__inputs":!0,"vm-limits-configurator__inputs_mobile":i}),children:hi.map((e=>Ct("div",{children:Ct(Ha,{label:e.label,value:t[e.type],error:o[e.type],onChange:s(e.type),onEnter:a,type:"number"})},e.type)))})]})},pi=e=>{let{defaultExpanded:t=!1,onChange:n,title:a,children:i}=e;const[o,l]=(0,r.useState)(t);return(0,r.useEffect)((()=>{n&&n(o)}),[o]),Ct(xt.FK,{children:[Ct("header",{className:"vm-accordion-header ".concat(o&&"vm-accordion-header_open"),onClick:()=>{l((e=>!e))},children:[a,Ct("div",{className:"vm-accordion-header__arrow ".concat(o&&"vm-accordion-header__arrow_open"),children:Ct(Hn,{})})]}),o&&Ct("section",{className:"vm-accordion-section",children:i},"content")]})},fi=()=>Ct(fa,{title:"Browser timezone is not recognized, supported, or could not be determined.",children:Ct(Dn,{})}),vi=on(),gi=e=>{let{timezoneState:t,defaultTimezone:n,onChange:a}=e;const{isMobile:i}=Wr(),o=rn(),[l,s]=(0,r.useState)(""),c=(0,r.useRef)(null),{value:u,toggle:d,setFalse:h}=oa(!1),m=(0,r.useMemo)((()=>[{title:"Default time (".concat(n,")"),region:n,utc:n?nn(n):"UTC"},{title:vi.title,region:vi.region,utc:nn(vi.region),isInvalid:!vi.isValid},{title:"UTC (Coordinated Universal Time)",region:"UTC",utc:"UTC"}].filter((e=>e.region))),[n]),p=(0,r.useMemo)((()=>{if(!l)return o;try{return rn(l)}catch(rp){return{}}}),[l,o]),f=(0,r.useMemo)((()=>Object.keys(p)),[p]),v=(0,r.useMemo)((()=>({region:t,utc:nn(t)})),[t]),g=e=>()=>{(e=>{a(e.region),s(""),h()})(e)};return Ct("div",{className:"vm-timezones",children:[Ct("div",{className:"vm-server-configurator__title",children:"Time zone"}),Ct("div",{className:"vm-timezones-item vm-timezones-item_selected",onClick:d,ref:c,children:[Ct("div",{className:"vm-timezones-item__title",children:v.region}),Ct("div",{className:"vm-timezones-item__utc",children:v.utc}),Ct("div",{className:xr()({"vm-timezones-item__icon":!0,"vm-timezones-item__icon_open":u}),children:Ct(Vn,{})})]}),Ct(ia,{open:u,buttonRef:c,placement:"bottom-left",onClose:h,fullWidth:!0,title:i?"Time zone":void 0,children:Ct("div",{className:xr()({"vm-timezones-list":!0,"vm-timezones-list_mobile":i}),children:[Ct("div",{className:"vm-timezones-list-header",children:[Ct("div",{className:"vm-timezones-list-header__search",children:Ct(Ha,{autofocus:!0,label:"Search",value:l,onChange:e=>{s(e)}})}),m.map(((e,t)=>e&&Ct("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:g(e),children:[Ct("div",{className:"vm-timezones-item__title",children:[e.title,e.isInvalid&&Ct(fi,{})]}),Ct("div",{className:"vm-timezones-item__utc",children:e.utc})]},"".concat(t,"_").concat(e.region))))]}),f.map((e=>Ct("div",{className:"vm-timezones-list-group",children:Ct(pi,{defaultExpanded:!0,title:Ct("div",{className:"vm-timezones-list-group__title",children:e}),children:Ct("div",{className:"vm-timezones-list-group-options",children:p[e]&&p[e].map((e=>Ct("div",{className:"vm-timezones-item vm-timezones-list-group-options__item",onClick:g(e),children:[Ct("div",{className:"vm-timezones-item__title",children:e.region}),Ct("div",{className:"vm-timezones-item__utc",children:e.utc})]},e.search)))})})},e)))]})})]})},yi=e=>{let{options:t,value:n,label:a,onChange:i}=e;const o=(0,r.useRef)(null),[l,s]=(0,r.useState)({width:"0px",left:"0px",borderRadius:"0px"}),c=e=>()=>{i(e)};return(0,r.useEffect)((()=>{if(!o.current)return void s({width:"0px",left:"0px",borderRadius:"0px"});const e=t.findIndex((e=>e.value===n)),{width:r}=o.current.getBoundingClientRect();let a=r,i=e*a,l="0";0===e&&(l="16px 0 0 16px"),e===t.length-1&&(l="10px",i-=1,l="0 16px 16px 0"),0!==e&&e!==t.length-1&&(a+=1,i-=1),s({width:"".concat(a,"px"),left:"".concat(i,"px"),borderRadius:l})}),[o,n,t]),Ct("div",{className:"vm-toggles",children:[a&&Ct("label",{className:"vm-toggles__label",children:a}),Ct("div",{className:"vm-toggles-group",style:{gridTemplateColumns:"repeat(".concat(t.length,", 1fr)")},children:[l.borderRadius&&Ct("div",{className:"vm-toggles-group__highlight",style:l}),t.map(((e,t)=>Ct("div",{className:xr()({"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)))]})]})},_i=Object.values(mt).map((e=>({title:e,value:e}))),bi=e=>{let{theme:t,onChange:n}=e;const{isMobile:r}=Wr();return Ct("div",{className:xr()({"vm-theme-control":!0,"vm-theme-control_mobile":r}),children:[Ct("div",{className:"vm-server-configurator__title",children:"Theme preferences"}),Ct("div",{className:"vm-theme-control__toggle",children:Ct(yi,{options:_i,value:t,onChange:e=>{n(e)}})},"".concat(r))]})},wi="Settings",{REACT_APP_TYPE:ki}={},xi=ki===He.logs,Si=()=>{const{isMobile:e}=Wr(),t=Ke(),{serverUrl:n,theme:a}=Nt(),{timezone:i,defaultTimezone:o}=vn(),{seriesLimits:l}=Fr(),s=At(),c=gn(),u=jr(),[d,h]=(0,r.useState)(n),[m,p]=(0,r.useState)(l),[f,v]=(0,r.useState)(i),{value:g,setTrue:y,setFalse:_}=oa(!1),b=()=>{_(),h(n),p(l),v(i)},w=()=>{const e=Ge(d);""!==e&&s({type:"SET_TENANT_ID",payload:e}),s({type:"SET_SERVER",payload:d}),c({type:"SET_TIMEZONE",payload:f}),u({type:"SET_SERIES_LIMITS",payload:m}),_()};(0,r.useEffect)((()=>{n!==d&&h(n)}),[n]),(0,r.useEffect)((()=>{v(i)}),[i]);const k=[{show:!t&&!xi,component:Ct(di,{stateServerUrl:n,serverUrl:d,onChange:h,onEnter:w})},{show:!xi,component:Ct(mi,{limits:m,onChange:p,onEnter:w})},{show:!0,component:Ct(gi,{timezoneState:f,defaultTimezone:o,onChange:v})},{show:!t,component:Ct(bi,{theme:a,onChange:e=>{s({type:"SET_THEME",payload:e})}})}].filter((e=>e.show));return Ct(xt.FK,{children:[e?Ct("div",{className:"vm-mobile-option",onClick:y,children:[Ct("span",{className:"vm-mobile-option__icon",children:Ct(Pn,{})}),Ct("div",{className:"vm-mobile-option-text",children:Ct("span",{className:"vm-mobile-option-text__label",children:wi})}),Ct("span",{className:"vm-mobile-option__arrow",children:Ct(Hn,{})})]}):Ct(fa,{title:wi,children:Ct(aa,{className:xr()({"vm-header-button":!t}),variant:"contained",color:"primary",startIcon:Ct(Pn,{}),onClick:y,ariaLabel:"settings"})}),g&&Ct(pa,{title:wi,onClose:b,children:Ct("div",{className:xr()({"vm-server-configurator":!0,"vm-server-configurator_mobile":e}),children:[k.map(((e,t)=>Ct("div",{className:"vm-server-configurator__input",children:e.component},t))),Ct("div",{className:"vm-server-configurator-footer",children:[Ct(aa,{color:"error",variant:"outlined",onClick:b,children:"Cancel"}),Ct(aa,{color:"primary",variant:"contained",onClick:w,children:"Apply"})]})]})})]})},Ci=e=>{let{displaySidebar:t,isMobile:n,headerSetup:r,accountIds:a}=e;return Ct("div",{className:xr()({"vm-header-controls":!0,"vm-header-controls_mobile":n}),children:[(null===r||void 0===r?void 0:r.tenant)&&Ct(Va,{accountIds:a||[]}),(null===r||void 0===r?void 0:r.stepControl)&&Ct(Ua,{}),(null===r||void 0===r?void 0:r.timeSelector)&&Ct(ai,{}),(null===r||void 0===r?void 0:r.cardinalityDatePicker)&&Ct(oi,{}),(null===r||void 0===r?void 0:r.executionControls)&&Ct(si,{}),Ct(Si,{}),!t&&Ct(Na,{})]})},Ei=Boolean(Xe("DISABLED_DEFAULT_TIMEZONE")),Ni=()=>{const{serverUrl:e}=Nt(),t=gn(),[n,a]=(0,r.useState)(!1),[o,l]=(0,r.useState)(""),s=async()=>{if(e&&!{NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE){l(""),a(!0);try{const n=await fetch("".concat(e,"/vmui/timezone")),r=await n.json();n.ok?((e=>{const n="local"===e.toLowerCase()?on().region:e;try{if(i()().tz(n).isValid(),t({type:"SET_DEFAULT_TIMEZONE",payload:n}),Ei)return;t({type:"SET_TIMEZONE",payload:n})}catch(rp){rp instanceof Error&&l("".concat(rp.name,": ").concat(rp.message))}})(r.timezone),a(!1)):(l(r.error),a(!1))}catch(rp){a(!1),rp instanceof Error&&l("".concat(rp.name,": ").concat(rp.message))}}};return(0,r.useEffect)((()=>{s()}),[e]),{isLoading:n,error:o}},Ai=()=>{const e=Ke(),{isMobile:t}=Wr(),{pathname:n}=ne(),[a,i]=je();Fa(),Ni();return(0,r.useEffect)((()=>{var e;const t="vmui",r=null===(e=qe[n])||void 0===e?void 0:e.title;document.title=r?"".concat(r," - ").concat(t):t}),[n]),(0,r.useEffect)((()=>{const{search:e,href:t}=window.location;if(e){const t=rt().parse(e,{ignoreQueryPrefix:!0});Object.entries(t).forEach((e=>{let[t,n]=e;return a.set(t,n)})),i(a),window.location.search=""}const n=t.replace(/\/\?#\//,"/#/");n!==t&&window.location.replace(n)}),[]),Ct("section",{className:"vm-container",children:[Ct(Da,{controlsComponent:Ci}),Ct("div",{className:xr()({"vm-container-body":!0,"vm-container-body_mobile":t,"vm-container-body_app":e}),children:Ct(_e,{})}),!e&&Ct(za,{})]})};var Mi=function(e){return e[e.mouse=0]="mouse",e[e.keyboard=1]="keyboard",e}(Mi||{});const Ti=e=>{var t;let{value:n,options:a,anchor:i,disabled:o,minLength:l=2,fullWidth:s,selected:c,noOptionsText:u,label:d,disabledFullScreen:h,offset:m,maxDisplayResults:p,loading:f,onSelect:v,onOpenAutocomplete:g,onFoundOptions:y,onChangeWrapperRef:_}=e;const{isMobile:b}=Wr(),w=(0,r.useRef)(null),[k,x]=(0,r.useState)({index:-1}),[S,C]=(0,r.useState)(""),[E,N]=(0,r.useState)(0),{value:A,setValue:M,setFalse:T}=oa(!1),L=(0,r.useMemo)((()=>{if(!A)return[];try{const e=new RegExp(String(n.trim()),"i"),t=a.filter((t=>e.test(t.value))).sort(((t,r)=>{var a,i;return t.value.toLowerCase()===n.trim().toLowerCase()?-1:r.value.toLowerCase()===n.trim().toLowerCase()?1:((null===(a=t.value.match(e))||void 0===a?void 0:a.index)||0)-((null===(i=r.value.match(e))||void 0===i?void 0:i.index)||0)}));return N(t.length),C(t.length>Number(null===p||void 0===p?void 0:p.limit)&&(null===p||void 0===p?void 0:p.message)||""),null!==p&&void 0!==p&&p.limit?t.slice(0,p.limit):t}catch(rp){return[]}}),[A,a,n]),P=(0,r.useMemo)((()=>{var e;return 1===L.length&&(null===(e=L[0])||void 0===e?void 0:e.value)===n}),[L]),O=(0,r.useMemo)((()=>u&&!L.length),[u,L]),I=()=>{x({index:-1})},R=(0,r.useCallback)((e=>{const{key:t,ctrlKey:n,metaKey:r,shiftKey:a}=e,i=n||r||a,o=L.length&&!P;if("ArrowUp"===t&&!i&&o&&(e.preventDefault(),x((e=>{let{index:t}=e;return{index:t<=0?0:t-1,type:Mi.keyboard}}))),"ArrowDown"===t&&!i&&o){e.preventDefault();const t=L.length-1;x((e=>{let{index:n}=e;return{index:n>=t?t:n+1,type:Mi.keyboard}}))}if("Enter"===t){const e=L[k.index];e&&v(e.value),c||T()}"Escape"===t&&T()}),[k,L,P,T,v,c]);return(0,r.useEffect)((()=>{M(n.length>=l)}),[n,a]),Nr("keydown",R),(0,r.useEffect)((()=>{if(!w.current||k.type===Mi.mouse)return;const e=w.current.childNodes[k.index];null!==e&&void 0!==e&&e.scrollIntoView&&e.scrollIntoView({block:"center"})}),[k,L]),(0,r.useEffect)((()=>{x({index:-1})}),[L]),(0,r.useEffect)((()=>{g&&g(A)}),[A]),(0,r.useEffect)((()=>{y&&y(P?[]:L)}),[L,P]),(0,r.useEffect)((()=>{_&&_(w)}),[w]),Ct(ia,{open:A,buttonRef:i,placement:"bottom-left",onClose:T,fullWidth:s,title:b?d:void 0,disabledFullScreen:h,offset:m,children:[Ct("div",{className:xr()({"vm-autocomplete":!0,"vm-autocomplete_mobile":b&&!h}),ref:w,children:[f&&Ct("div",{className:"vm-autocomplete__loader",children:[Ct(jn,{}),Ct("span",{children:"Loading..."})]}),O&&Ct("div",{className:"vm-autocomplete__no-options",children:u}),!P&&L.map(((e,t)=>{return Ct("div",{className:xr()({"vm-list-item":!0,"vm-list-item_mobile":b,"vm-list-item_active":t===k.index,"vm-list-item_multiselect":c,"vm-list-item_multiselect_selected":null===c||void 0===c?void 0:c.includes(e.value),"vm-list-item_with-icon":e.icon}),id:"$autocomplete$".concat(e.value),onClick:(r=e.value,()=>{o||(v(r),c||T())}),onMouseEnter:(n=t,()=>{x({index:n,type:Mi.mouse})}),onMouseLeave:I,children:[(null===c||void 0===c?void 0:c.includes(e.value))&&Ct(er,{}),Ct(xt.FK,{children:e.icon}),Ct("span",{children:e.value})]},"".concat(t).concat(e.value));var n,r}))]}),S&&Ct("div",{className:"vm-autocomplete-message",children:["Shown ",null===p||void 0===p?void 0:p.limit," results out of ",E,". ",S]}),(null===(t=L[k.index])||void 0===t?void 0:t.description)&&Ct("div",{className:"vm-autocomplete-info",children:[Ct("div",{className:"vm-autocomplete-info__type",children:L[k.index].type}),Ct("div",{className:"vm-autocomplete-info__description",dangerouslySetInnerHTML:{__html:L[k.index].description||""}})]})]})};var Li=n(267),Pi=n.n(Li);const Oi=e=>e.replace(/[/\-\\^$*+?.()|[\]{}]/g,"\\$&"),Ii=e=>JSON.stringify(e).slice(1,-1);var Ri=function(e){return e.metric="metric",e.label="label",e.labelValue="labelValue",e}(Ri||{});const Di={[Ri.metric]:Ct(gr,{}),[Ri.label]:Ct(_r,{}),[Ri.labelValue]:Ct(br,{})};function zi(e,t){!function(e,t){if(t.has(e))throw new TypeError("Cannot initialize the same private elements twice on an object")}(e,t),t.add(e)}function Fi(e,t,n){if(!t.has(e))throw new TypeError("attempted to get private field on non-instance");return n}function ji(e){return ji="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},ji(e)}function Hi(e){var t=function(e,t){if("object"!=ji(e)||!e)return e;var n=e[Symbol.toPrimitive];if(void 0!==n){var r=n.call(e,t||"default");if("object"!=ji(r))return r;throw new TypeError("@@toPrimitive must return a primitive value.")}return("string"===t?String:Number)(e)}(e,"string");return"symbol"==ji(t)?t:String(t)}function Vi(e,t,n){return(t=Hi(t))in e?Object.defineProperty(e,t,{value:n,enumerable:!0,configurable:!0,writable:!0}):e[t]=n,e}function $i(){return{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}}let Ui={async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1};function Bi(e){Ui=e}const qi=/[&<>"']/,Yi=new RegExp(qi.source,"g"),Wi=/[<>"']|&(?!(#\d{1,7}|#[Xx][a-fA-F0-9]{1,6}|\w+);)/,Ki=new RegExp(Wi.source,"g"),Qi={"&":"&","<":"<",">":">",'"':""","'":"'"},Zi=e=>Qi[e];function Gi(e,t){if(t){if(qi.test(e))return e.replace(Yi,Zi)}else if(Wi.test(e))return e.replace(Ki,Zi);return e}const Ji=/&(#(?:\d+)|(?:#x[0-9A-Fa-f]+)|(?:\w+));?/gi;function Xi(e){return e.replace(Ji,((e,t)=>"colon"===(t=t.toLowerCase())?":":"#"===t.charAt(0)?"x"===t.charAt(1)?String.fromCharCode(parseInt(t.substring(2),16)):String.fromCharCode(+t.substring(1)):""))}const eo=/(^|[^\[])\^/g;function to(e,t){e="string"===typeof e?e:e.source,t=t||"";const n={replace:(t,r)=>(r=(r=r.source||r).replace(eo,"$1"),e=e.replace(t,r),n),getRegex:()=>new RegExp(e,t)};return n}const no=/[^\w:]/g,ro=/^$|^[a-z][a-z0-9+.-]*:|^[?#]/i;function ao(e,t,n){if(e){let e;try{e=decodeURIComponent(Xi(n)).replace(no,"").toLowerCase()}catch(rp){return null}if(0===e.indexOf("javascript:")||0===e.indexOf("vbscript:")||0===e.indexOf("data:"))return null}t&&!ro.test(n)&&(n=function(e,t){io[" "+e]||(oo.test(e)?io[" "+e]=e+"/":io[" "+e]=ho(e,"/",!0));e=io[" "+e];const n=-1===e.indexOf(":");return"//"===t.substring(0,2)?n?t:e.replace(lo,"$1")+t:"/"===t.charAt(0)?n?t:e.replace(so,"$1")+t:e+t}(t,n));try{n=encodeURI(n).replace(/%25/g,"%")}catch(rp){return null}return n}const io={},oo=/^[^:]+:\/*[^/]*$/,lo=/^([^:]+:)[\s\S]*$/,so=/^([^:]+:\/*[^/]*)[\s\S]*$/;const co={exec:function(){}};function uo(e,t){const n=e.replace(/\|/g,((e,t,n)=>{let r=!1,a=t;for(;--a>=0&&"\\"===n[a];)r=!r;return r?"|":" |"})).split(/ \|/);let 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.length0)return{type:"space",raw:t[0]}}code(e){const t=this.rules.block.code.exec(e);if(t){const e=t[0].replace(/^ {1,4}/gm,"");return{type:"code",raw:t[0],codeBlockStyle:"indented",text:this.options.pedantic?e:ho(e,"\n")}}}fences(e){const t=this.rules.block.fences.exec(e);if(t){const e=t[0],n=function(e,t){const n=e.match(/^(\s+)(?:```)/);if(null===n)return t;const r=n[1];return t.split("\n").map((e=>{const t=e.match(/^\s+/);if(null===t)return e;const[n]=t;return n.length>=r.length?e.slice(r.length):e})).join("\n")}(e,t[3]||"");return{type:"code",raw:e,lang:t[2]?t[2].trim().replace(this.rules.inline._escapes,"$1"):t[2],text:n}}}heading(e){const t=this.rules.block.heading.exec(e);if(t){let e=t[2].trim();if(/#$/.test(e)){const t=ho(e,"#");this.options.pedantic?e=t.trim():t&&!/ $/.test(t)||(e=t.trim())}return{type:"heading",raw:t[0],depth:t[1].length,text:e,tokens:this.lexer.inline(e)}}}hr(e){const t=this.rules.block.hr.exec(e);if(t)return{type:"hr",raw:t[0]}}blockquote(e){const t=this.rules.block.blockquote.exec(e);if(t){const e=t[0].replace(/^ *>[ \t]?/gm,""),n=this.lexer.state.top;this.lexer.state.top=!0;const r=this.lexer.blockTokens(e);return this.lexer.state.top=n,{type:"blockquote",raw:t[0],tokens:r,text:e}}}list(e){let t=this.rules.block.list.exec(e);if(t){let n,r,a,i,o,l,s,c,u,d,h,m,p=t[1].trim();const f=p.length>1,v={type:"list",raw:"",ordered:f,start:f?+p.slice(0,-1):"",loose:!1,items:[]};p=f?"\\d{1,9}\\".concat(p.slice(-1)):"\\".concat(p),this.options.pedantic&&(p=f?p:"[*+-]");const g=new RegExp("^( {0,3}".concat(p,")((?:[\t ][^\\n]*)?(?:\\n|$))"));for(;e&&(m=!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+/,(e=>" ".repeat(3*e.length))),u=e.split("\n",1)[0],this.options.pedantic?(i=2,h=c.trimLeft()):(i=t[2].search(/[^ ]/),i=i>4?1:i,h=c.slice(i),i+=t[1].length),l=!1,!c&&/^ *$/.test(u)&&(n+=u+"\n",e=e.substring(u.length+1),m=!0),!m){const t=new RegExp("^ {0,".concat(Math.min(3,i-1),"}(?:[*+-]|\\d{1,9}[.)])((?:[ \t][^\\n]*)?(?:\\n|$))")),r=new RegExp("^ {0,".concat(Math.min(3,i-1),"}((?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$)")),a=new RegExp("^ {0,".concat(Math.min(3,i-1),"}(?:```|~~~)")),o=new RegExp("^ {0,".concat(Math.min(3,i-1),"}#"));for(;e&&(d=e.split("\n",1)[0],u=d,this.options.pedantic&&(u=u.replace(/^ {1,4}(?=( {4})*[^ ])/g," ")),!a.test(u))&&!o.test(u)&&!t.test(u)&&!r.test(e);){if(u.search(/[^ ]/)>=i||!u.trim())h+="\n"+u.slice(i);else{if(l)break;if(c.search(/[^ ]/)>=4)break;if(a.test(c))break;if(o.test(c))break;if(r.test(c))break;h+="\n"+u}l||u.trim()||(l=!0),n+=d+"\n",e=e.substring(d.length+1),c=u.slice(i)}}v.loose||(s?v.loose=!0:/\n *\n *$/.test(n)&&(s=!0)),this.options.gfm&&(r=/^\[[ xX]\] /.exec(h),r&&(a="[ ] "!==r[0],h=h.replace(/^\[[ xX]\] +/,""))),v.items.push({type:"list_item",raw:n,task:!!r,checked:a,loose:!1,text:h}),v.raw+=n}v.items[v.items.length-1].raw=n.trimRight(),v.items[v.items.length-1].text=h.trimRight(),v.raw=v.raw.trimRight();const y=v.items.length;for(o=0;o"space"===e.type)),t=e.length>0&&e.some((e=>/\n.*\n/.test(e.raw)));v.loose=t}if(v.loose)for(o=0;o$/,"$1").replace(this.rules.inline._escapes,"$1"):"",r=t[3]?t[3].substring(1,t[3].length-1).replace(this.rules.inline._escapes,"$1"):t[3];return{type:"def",tag:e,raw:t[0],href:n,title:r}}}table(e){const t=this.rules.block.table.exec(e);if(t){const e={type:"table",header:uo(t[1]).map((e=>({text:e}))),align:t[2].replace(/^ *|\| *$/g,"").split(/ *\| */),rows:t[3]&&t[3].trim()?t[3].replace(/\n[ \t]*$/,"").split("\n"):[]};if(e.header.length===e.align.length){e.raw=t[0];let n,r,a,i,o=e.align.length;for(n=0;n({text:e})));for(o=e.header.length,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,block:!1,text:this.options.sanitize?this.options.sanitizer?this.options.sanitizer(t[0]):Gi(t[0]):t[0]}}link(e){const t=this.rules.inline.link.exec(e);if(t){const e=t[2].trim();if(!this.options.pedantic&&/^$/.test(e))return;const t=ho(e.slice(0,-1),"\\");if((e.length-t.length)%2===0)return}else{const e=function(e,t){if(-1===e.indexOf(t[1]))return-1;const n=e.length;let r=0,a=0;for(;a-1){const n=(0===t[0].indexOf("!")?5:4)+t[1].length+e;t[2]=t[2].substring(0,e),t[0]=t[0].substring(0,n).trim(),t[3]=""}}let n=t[2],r="";if(this.options.pedantic){const e=/^([^'"]*[^\s])\s+(['"])(.*)\2/.exec(n);e&&(n=e[1],r=e[3])}else r=t[3]?t[3].slice(1,-1):"";return n=n.trim(),/^$/.test(e)?n.slice(1):n.slice(1,-1)),mo(t,{href:n?n.replace(this.rules.inline._escapes,"$1"):n,title:r?r.replace(this.rules.inline._escapes,"$1"):r},t[0],this.lexer)}}reflink(e,t){let n;if((n=this.rules.inline.reflink.exec(e))||(n=this.rules.inline.nolink.exec(e))){let e=(n[2]||n[1]).replace(/\s+/g," ");if(e=t[e.toLowerCase()],!e){const e=n[0].charAt(0);return{type:"text",raw:e,text:e}}return mo(n,e,n[0],this.lexer)}}emStrong(e,t){let n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"",r=this.rules.inline.emStrong.lDelim.exec(e);if(!r)return;if(r[3]&&n.match(/[\p{L}\p{N}]/u))return;if(!(r[1]||r[2]||"")||!n||this.rules.inline.punctuation.exec(n)){const n=r[0].length-1;let a,i,o=n,l=0;const s="*"===r[0][0]?this.rules.inline.emStrong.rDelimAst:this.rules.inline.emStrong.rDelimUnd;for(s.lastIndex=0,t=t.slice(-1*e.length+n);null!=(r=s.exec(t));){if(a=r[1]||r[2]||r[3]||r[4]||r[5]||r[6],!a)continue;if(i=a.length,r[3]||r[4]){o+=i;continue}if((r[5]||r[6])&&n%3&&!((n+i)%3)){l+=i;continue}if(o-=i,o>0)continue;i=Math.min(i,i+o+l);const t=e.slice(0,n+r.index+i+1);if(Math.min(n,i)%2){const e=t.slice(1,-1);return{type:"em",raw:t,text:e,tokens:this.lexer.inlineTokens(e)}}const s=t.slice(2,-2);return{type:"strong",raw:t,text:s,tokens:this.lexer.inlineTokens(s)}}}}codespan(e){const t=this.rules.inline.code.exec(e);if(t){let e=t[2].replace(/\n/g," ");const n=/[^ ]/.test(e),r=/^ /.test(e)&&/ $/.test(e);return n&&r&&(e=e.substring(1,e.length-1)),e=Gi(e,!0),{type:"codespan",raw:t[0],text:e}}}br(e){const t=this.rules.inline.br.exec(e);if(t)return{type:"br",raw:t[0]}}del(e){const t=this.rules.inline.del.exec(e);if(t)return{type:"del",raw:t[0],text:t[2],tokens:this.lexer.inlineTokens(t[2])}}autolink(e,t){const n=this.rules.inline.autolink.exec(e);if(n){let e,r;return"@"===n[2]?(e=Gi(this.options.mangle?t(n[1]):n[1]),r="mailto:"+e):(e=Gi(n[1]),r=e),{type:"link",raw:n[0],text:e,href:r,tokens:[{type:"text",raw:e,text:e}]}}}url(e,t){let n;if(n=this.rules.inline.url.exec(e)){let e,r;if("@"===n[2])e=Gi(this.options.mangle?t(n[0]):n[0]),r="mailto:"+e;else{let t;do{t=n[0],n[0]=this.rules.inline._backpedal.exec(n[0])[0]}while(t!==n[0]);e=Gi(n[0]),r="www."===n[1]?"http://"+n[0]:n[0]}return{type:"link",raw:n[0],text:e,href:r,tokens:[{type:"text",raw:e,text:e}]}}}inlineText(e,t){const n=this.rules.inline.text.exec(e);if(n){let e;return e=this.lexer.state.inRawBlock?this.options.sanitize?this.options.sanitizer?this.options.sanitizer(n[0]):Gi(n[0]):n[0]:Gi(this.options.smartypants?t(n[0]):n[0]),{type:"text",raw:n[0],text:e}}}}const fo={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]*?(?:[^\\n]*\\n+|$)|comment[^\\n]*(\\n+|$)|<\\?[\\s\\S]*?(?:\\?>\\n*|$)|\\n*|$)|\\n*|$)|)[\\s\\S]*?(?:(?:\\n *)+\\n|$)|<(?!script|pre|style|textarea)([a-z][\\w-]*)(?:attribute)*? */?>(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$)|(?=[ \\t]*(?:\\n|$))[\\s\\S]*?(?:(?:\\n *)+\\n|$))",def:/^ {0,3}\[(label)\]: *(?:\n *)?([^<\s][^\s]*|<.*?>)(?:(?: +(?:\n *)?| *\n *)(title))? *(?:\n+|$)/,table:co,lheading:/^((?:(?!^bull ).|\n(?!\n|bull ))+?)\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?'|\([^()]*\))/};fo.def=to(fo.def).replace("label",fo._label).replace("title",fo._title).getRegex(),fo.bullet=/(?:[*+-]|\d{1,9}[.)])/,fo.listItemStart=to(/^( *)(bull) */).replace("bull",fo.bullet).getRegex(),fo.list=to(fo.list).replace(/bull/g,fo.bullet).replace("hr","\\n+(?=\\1?(?:(?:- *){3,}|(?:_ *){3,}|(?:\\* *){3,})(?:\\n+|$))").replace("def","\\n+(?="+fo.def.source+")").getRegex(),fo._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",fo._comment=/|$)/,fo.html=to(fo.html,"i").replace("comment",fo._comment).replace("tag",fo._tag).replace("attribute",/ +[a-zA-Z:_][\w.:-]*(?: *= *"[^"\n]*"| *= *'[^'\n]*'| *= *[^\s"'=<>`]+)?/).getRegex(),fo.lheading=to(fo.lheading).replace(/bull/g,fo.bullet).getRegex(),fo.paragraph=to(fo._paragraph).replace("hr",fo.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",")|<(?:script|pre|style|textarea|!--)").replace("tag",fo._tag).getRegex(),fo.blockquote=to(fo.blockquote).replace("paragraph",fo.paragraph).getRegex(),fo.normal={...fo},fo.gfm={...fo.normal,table:"^ *([^\\n ].*\\|.*)\\n {0,3}(?:\\| *)?(:?-+:? *(?:\\| *:?-+:? *)*)(?:\\| *)?(?:\\n((?:(?! *\\n|hr|heading|blockquote|code|fences|list|html).*(?:\\n|$))*)\\n*|$)"},fo.gfm.table=to(fo.gfm.table).replace("hr",fo.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",")|<(?:script|pre|style|textarea|!--)").replace("tag",fo._tag).getRegex(),fo.gfm.paragraph=to(fo._paragraph).replace("hr",fo.hr).replace("heading"," {0,3}#{1,6} ").replace("|lheading","").replace("table",fo.gfm.table).replace("blockquote"," {0,3}>").replace("fences"," {0,3}(?:`{3,}(?=[^`\\n]*\\n)|~{3,})[^\\n]*\\n").replace("list"," {0,3}(?:[*+-]|1[.)]) ").replace("html",")|<(?:script|pre|style|textarea|!--)").replace("tag",fo._tag).getRegex(),fo.pedantic={...fo.normal,html:to("^ *(?:comment *(?:\\n|\\s*$)|<(tag)[\\s\\S]+? *(?:\\n{2,}|\\s*$)|\\s]*)*?/?> *(?:\\n{2,}|\\s*$))").replace("comment",fo._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:/^ *\[([^\]]+)\]: *]+)>?(?: +(["(][^\n]+[")]))? *(?:\n+|$)/,heading:/^(#{1,6})(.*)(?:\n+|$)/,fences:co,lheading:/^(.+?)\n {0,3}(=+|-+) *(?:\n+|$)/,paragraph:to(fo.normal._paragraph).replace("hr",fo.hr).replace("heading"," *#{1,6} *[^\n]").replace("lheading",fo.lheading).replace("blockquote"," {0,3}>").replace("|fences","").replace("|list","").replace("|html","").getRegex()};const vo={escape:/^\\([!"#$%&'()*+,\-./:;<=>?@\[\]\\^_`{|}~])/,autolink:/^<(scheme:[^\s\x00-\x1f<>]*|email)>/,url:co,tag:"^comment|^|^<[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:co,text:/^(`+|[^`])(?:(?= {2,}\n)|[\s\S]*?(?:(?=[\\.5&&(n="x"+n.toString(16)),r+="&#"+n+";";return r}vo._punctuation="\\p{P}$+<=>`^|~",vo.punctuation=to(vo.punctuation,"u").replace(/punctuation/g,vo._punctuation).getRegex(),vo.blockSkip=/\[[^[\]]*?\]\([^\(\)]*?\)|`[^`]*?`|<[^<>]*?>/g,vo.anyPunctuation=/\\[punct]/g,vo._escapes=/\\([punct])/g,vo._comment=to(fo._comment).replace("(?:--\x3e|$)","--\x3e").getRegex(),vo.emStrong.lDelim=to(vo.emStrong.lDelim,"u").replace(/punct/g,vo._punctuation).getRegex(),vo.emStrong.rDelimAst=to(vo.emStrong.rDelimAst,"gu").replace(/punct/g,vo._punctuation).getRegex(),vo.emStrong.rDelimUnd=to(vo.emStrong.rDelimUnd,"gu").replace(/punct/g,vo._punctuation).getRegex(),vo.anyPunctuation=to(vo.anyPunctuation,"gu").replace(/punct/g,vo._punctuation).getRegex(),vo._escapes=to(vo._escapes,"gu").replace(/punct/g,vo._punctuation).getRegex(),vo._scheme=/[a-zA-Z][a-zA-Z0-9+.-]{1,31}/,vo._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])?)+(?![-_])/,vo.autolink=to(vo.autolink).replace("scheme",vo._scheme).replace("email",vo._email).getRegex(),vo._attribute=/\s+[a-zA-Z:_][\w.:-]*(?:\s*=\s*"[^"]*"|\s*=\s*'[^']*'|\s*=\s*[^\s"'=<>`]+)?/,vo.tag=to(vo.tag).replace("comment",vo._comment).replace("attribute",vo._attribute).getRegex(),vo._label=/(?:\[(?:\\.|[^\[\]\\])*\]|\\.|`[^`]*`|[^\[\]\\`])*?/,vo._href=/<(?:\\.|[^\n<>\\])+>|[^\s\x00-\x1f]*/,vo._title=/"(?:\\"?|[^"\\])*"|'(?:\\'?|[^'\\])*'|\((?:\\\)?|[^)\\])*\)/,vo.link=to(vo.link).replace("label",vo._label).replace("href",vo._href).replace("title",vo._title).getRegex(),vo.reflink=to(vo.reflink).replace("label",vo._label).replace("ref",fo._label).getRegex(),vo.nolink=to(vo.nolink).replace("ref",fo._label).getRegex(),vo.reflinkSearch=to(vo.reflinkSearch,"g").replace("reflink",vo.reflink).replace("nolink",vo.nolink).getRegex(),vo.normal={...vo},vo.pedantic={...vo.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:to(/^!?\[(label)\]\((.*?)\)/).replace("label",vo._label).getRegex(),reflink:to(/^!?\[(label)\]\s*\[([^\]]*)\]/).replace("label",vo._label).getRegex()},vo.gfm={...vo.normal,escape:to(vo.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]:[];for(e=this.options.pedantic?e.replace(/\t/g," ").replace(/^ +$/gm,""):e.replace(/^( *)(\t+)/gm,((e,t,n)=>t+" ".repeat(n.length)));e;)if(!(this.options.extensions&&this.options.extensions.block&&this.options.extensions.block.some((n=>!!(t=n.call({lexer:this},e,i))&&(e=e.substring(t.raw.length),i.push(t),!0)))))if(t=this.tokenizer.space(e))e=e.substring(t.raw.length),1===t.raw.length&&i.length>0?i[i.length-1].raw+="\n":i.push(t);else if(t=this.tokenizer.code(e))e=e.substring(t.raw.length),n=i[i.length-1],!n||"paragraph"!==n.type&&"text"!==n.type?i.push(t):(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue[this.inlineQueue.length-1].src=n.text);else if(t=this.tokenizer.fences(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.heading(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.hr(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.blockquote(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.list(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.html(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.def(e))e=e.substring(t.raw.length),n=i[i.length-1],!n||"paragraph"!==n.type&&"text"!==n.type?this.tokens.links[t.tag]||(this.tokens.links[t.tag]={href:t.href,title:t.title}):(n.raw+="\n"+t.raw,n.text+="\n"+t.raw,this.inlineQueue[this.inlineQueue.length-1].src=n.text);else if(t=this.tokenizer.table(e))e=e.substring(t.raw.length),i.push(t);else if(t=this.tokenizer.lheading(e))e=e.substring(t.raw.length),i.push(t);else{if(r=e,this.options.extensions&&this.options.extensions.startBlock){let t=1/0;const n=e.slice(1);let a;this.options.extensions.startBlock.forEach((function(e){a=e.call({lexer:this},n),"number"===typeof a&&a>=0&&(t=Math.min(t,a))})),t<1/0&&t>=0&&(r=e.substring(0,t+1))}if(this.state.top&&(t=this.tokenizer.paragraph(r)))n=i[i.length-1],a&&"paragraph"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue.pop(),this.inlineQueue[this.inlineQueue.length-1].src=n.text):i.push(t),a=r.length!==e.length,e=e.substring(t.raw.length);else if(t=this.tokenizer.text(e))e=e.substring(t.raw.length),n=i[i.length-1],n&&"text"===n.type?(n.raw+="\n"+t.raw,n.text+="\n"+t.text,this.inlineQueue.pop(),this.inlineQueue[this.inlineQueue.length-1].src=n.text):i.push(t);else if(e){const t="Infinite loop on byte: "+e.charCodeAt(0);if(this.options.silent){console.error(t);break}throw new Error(t)}}return this.state.top=!0,i}inline(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[];return this.inlineQueue.push({src:e,tokens:t}),t}inlineTokens(e){let t,n,r,a,i,o,l=arguments.length>1&&void 0!==arguments[1]?arguments[1]:[],s=e;if(this.tokens.links){const e=Object.keys(this.tokens.links);if(e.length>0)for(;null!=(a=this.tokenizer.rules.inline.reflinkSearch.exec(s));)e.includes(a[0].slice(a[0].lastIndexOf("[")+1,-1))&&(s=s.slice(0,a.index)+"["+"a".repeat(a[0].length-2)+"]"+s.slice(this.tokenizer.rules.inline.reflinkSearch.lastIndex))}for(;null!=(a=this.tokenizer.rules.inline.blockSkip.exec(s));)s=s.slice(0,a.index)+"["+"a".repeat(a[0].length-2)+"]"+s.slice(this.tokenizer.rules.inline.blockSkip.lastIndex);for(;null!=(a=this.tokenizer.rules.inline.anyPunctuation.exec(s));)s=s.slice(0,a.index)+"++"+s.slice(this.tokenizer.rules.inline.anyPunctuation.lastIndex);for(;e;)if(i||(o=""),i=!1,!(this.options.extensions&&this.options.extensions.inline&&this.options.extensions.inline.some((n=>!!(t=n.call({lexer:this},e,l))&&(e=e.substring(t.raw.length),l.push(t),!0)))))if(t=this.tokenizer.escape(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.tag(e))e=e.substring(t.raw.length),n=l[l.length-1],n&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(t=this.tokenizer.link(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.reflink(e,this.tokens.links))e=e.substring(t.raw.length),n=l[l.length-1],n&&"text"===t.type&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(t=this.tokenizer.emStrong(e,s,o))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.codespan(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.br(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.del(e))e=e.substring(t.raw.length),l.push(t);else if(t=this.tokenizer.autolink(e,yo))e=e.substring(t.raw.length),l.push(t);else if(this.state.inLink||!(t=this.tokenizer.url(e,yo))){if(r=e,this.options.extensions&&this.options.extensions.startInline){let t=1/0;const n=e.slice(1);let a;this.options.extensions.startInline.forEach((function(e){a=e.call({lexer:this},n),"number"===typeof a&&a>=0&&(t=Math.min(t,a))})),t<1/0&&t>=0&&(r=e.substring(0,t+1))}if(t=this.tokenizer.inlineText(r,go))e=e.substring(t.raw.length),"_"!==t.raw.slice(-1)&&(o=t.raw.slice(-1)),i=!0,n=l[l.length-1],n&&"text"===n.type?(n.raw+=t.raw,n.text+=t.text):l.push(t);else if(e){const t="Infinite loop on byte: "+e.charCodeAt(0);if(this.options.silent){console.error(t);break}throw new Error(t)}}else e=e.substring(t.raw.length),l.push(t);return l}}class bo{constructor(e){this.options=e||Ui}code(e,t,n){const r=(t||"").match(/\S*/)[0];if(this.options.highlight){const t=this.options.highlight(e,r);null!=t&&t!==e&&(n=!0,e=t)}return e=e.replace(/\n$/,"")+"\n",r?'
    '+(n?e:Gi(e,!0))+"
    \n":"
    "+(n?e:Gi(e,!0))+"
    \n"}blockquote(e){return"
    \n".concat(e,"
    \n")}html(e,t){return e}heading(e,t,n,r){if(this.options.headerIds){const a=this.options.headerPrefix+r.slug(n);return"').concat(e,"\n")}return"").concat(e,"\n")}hr(){return this.options.xhtml?"
    \n":"
    \n"}list(e,t,n){const r=t?"ol":"ul";return"<"+r+(t&&1!==n?' start="'+n+'"':"")+">\n"+e+"\n"}listitem(e){return"
  • ".concat(e,"
  • \n")}checkbox(e){return" "}paragraph(e){return"

    ".concat(e,"

    \n")}table(e,t){return t&&(t="".concat(t,"")),"\n\n"+e+"\n"+t+"
    \n"}tablerow(e){return"\n".concat(e,"\n")}tablecell(e,t){const n=t.header?"th":"td";return(t.align?"<".concat(n,' align="').concat(t.align,'">'):"<".concat(n,">"))+e+"\n")}strong(e){return"".concat(e,"")}em(e){return"".concat(e,"")}codespan(e){return"".concat(e,"")}br(){return this.options.xhtml?"
    ":"
    "}del(e){return"".concat(e,"")}link(e,t,n){if(null===(e=ao(this.options.sanitize,this.options.baseUrl,e)))return n;let r='
    ",r}image(e,t,n){if(null===(e=ao(this.options.sanitize,this.options.baseUrl,e)))return n;let r='').concat(n,'":">",r}text(e){return e}}class wo{strong(e){return e}em(e){return e}codespan(e){return e}del(e){return e}html(e){return e}text(e){return e}link(e,t,n){return""+n}image(e,t,n){return""+n}br(){return""}}class ko{constructor(){this.seen={}}serialize(e){return e.toLowerCase().trim().replace(/<[!\/a-z].*?>/gi,"").replace(/[\u2000-\u206F\u2E00-\u2E7F\\'!"#$%&()*+,./:;<=>?@[\]^`{|}~]/g,"").replace(/\s/g,"-")}getNextSafeSlug(e,t){let n=e,r=0;if(this.seen.hasOwnProperty(n)){r=this.seen[e];do{r++,n=e+"-"+r}while(this.seen.hasOwnProperty(n))}return t||(this.seen[e]=r,this.seen[n]=0),n}slug(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:{};const n=this.serialize(e);return this.getNextSafeSlug(n,t.dryrun)}}class xo{constructor(e){this.options=e||Ui,this.options.renderer=this.options.renderer||new bo,this.renderer=this.options.renderer,this.renderer.options=this.options,this.textRenderer=new wo,this.slugger=new ko}static parse(e,t){return new xo(t).parse(e)}static parseInline(e,t){return new xo(t).parseInline(e)}parse(e){let t,n,r,a,i,o,l,s,c,u,d,h,m,p,f,v,g,y,_,b=!(arguments.length>1&&void 0!==arguments[1])||arguments[1],w="";const k=e.length;for(t=0;t0&&"paragraph"===f.tokens[0].type?(f.tokens[0].text=y+" "+f.tokens[0].text,f.tokens[0].tokens&&f.tokens[0].tokens.length>0&&"text"===f.tokens[0].tokens[0].type&&(f.tokens[0].tokens[0].text=y+" "+f.tokens[0].tokens[0].text)):f.tokens.unshift({type:"text",text:y}):p+=y),p+=this.parse(f.tokens,m),c+=this.renderer.listitem(p,g,v);w+=this.renderer.list(c,d,h);continue;case"html":w+=this.renderer.html(u.text,u.block);continue;case"paragraph":w+=this.renderer.paragraph(this.parseInline(u.tokens));continue;case"text":for(c=u.tokens?this.parseInline(u.tokens):u.text;t+1{"function"===typeof r&&(a=r,r=null);const i={...r};r={...this.defaults,...i};const o=Fi(this,Eo,Ao).call(this,r.silent,r.async,a);if("undefined"===typeof n||null===n)return o(new Error("marked(): input parameter is undefined or null"));if("string"!==typeof n)return o(new Error("marked(): input parameter is of type "+Object.prototype.toString.call(n)+", string expected"));if(function(e,t){e&&!e.silent&&(t&&console.warn("marked(): callback is deprecated since version 5.0.0, should not be used and will be removed in the future. Read more here: https://marked.js.org/using_pro#async"),(e.sanitize||e.sanitizer)&&console.warn("marked(): sanitize and sanitizer parameters are deprecated since version 0.7.0, should not be used and will be removed in the future. Read more here: https://marked.js.org/#/USING_ADVANCED.md#options"),(e.highlight||"language-"!==e.langPrefix)&&console.warn("marked(): highlight and langPrefix parameters are deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-highlight."),e.mangle&&console.warn("marked(): mangle parameter is enabled by default, but is deprecated since version 5.0.0, and will be removed in the future. To clear this warning, install https://www.npmjs.com/package/marked-mangle, or disable by setting `{mangle: false}`."),e.baseUrl&&console.warn("marked(): baseUrl parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-base-url."),e.smartypants&&console.warn("marked(): smartypants parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-smartypants."),e.xhtml&&console.warn("marked(): xhtml parameter is deprecated since version 5.0.0, should not be used and will be removed in the future. Instead use https://www.npmjs.com/package/marked-xhtml."),(e.headerIds||e.headerPrefix)&&console.warn("marked(): headerIds and headerPrefix parameters enabled by default, but are deprecated since version 5.0.0, and will be removed in the future. To clear this warning, install https://www.npmjs.com/package/marked-gfm-heading-id, or disable by setting `{headerIds: false}`."))}(r,a),r.hooks&&(r.hooks.options=r),a){const i=r.highlight;let l;try{r.hooks&&(n=r.hooks.preprocess(n)),l=e(n,r)}catch(rp){return o(rp)}const s=e=>{let n;if(!e)try{r.walkTokens&&this.walkTokens(l,r.walkTokens),n=t(l,r),r.hooks&&(n=r.hooks.postprocess(n))}catch(rp){e=rp}return r.highlight=i,e?o(e):a(null,n)};if(!i||i.length<3)return s();if(delete r.highlight,!l.length)return s();let c=0;return this.walkTokens(l,(e=>{"code"===e.type&&(c++,setTimeout((()=>{i(e.text,e.lang,((t,n)=>{if(t)return s(t);null!=n&&n!==e.text&&(e.text=n,e.escaped=!0),c--,0===c&&s()}))}),0))})),void(0===c&&s())}if(r.async)return Promise.resolve(r.hooks?r.hooks.preprocess(n):n).then((t=>e(t,r))).then((e=>r.walkTokens?Promise.all(this.walkTokens(e,r.walkTokens)).then((()=>e)):e)).then((e=>t(e,r))).then((e=>r.hooks?r.hooks.postprocess(e):e)).catch(o);try{r.hooks&&(n=r.hooks.preprocess(n));const a=e(n,r);r.walkTokens&&this.walkTokens(a,r.walkTokens);let i=t(a,r);return r.hooks&&(i=r.hooks.postprocess(i)),i}catch(rp){return o(rp)}}}function Ao(e,t,n){return r=>{if(r.message+="\nPlease report this to https://github.com/markedjs/marked.",e){const e="

    An error occurred:

    "+Gi(r.message+"",!0)+"
    ";return t?Promise.resolve(e):n?void n(null,e):e}if(t)return Promise.reject(r);if(!n)throw r;n(r)}}const Mo=new class{constructor(){zi(this,Eo),zi(this,Co),Vi(this,"defaults",{async:!1,baseUrl:null,breaks:!1,extensions:null,gfm:!0,headerIds:!0,headerPrefix:"",highlight:null,hooks:null,langPrefix:"language-",mangle:!0,pedantic:!1,renderer:null,sanitize:!1,sanitizer:null,silent:!1,smartypants:!1,tokenizer:null,walkTokens:null,xhtml:!1}),Vi(this,"options",this.setOptions),Vi(this,"parse",Fi(this,Co,No).call(this,_o.lex,xo.parse)),Vi(this,"parseInline",Fi(this,Co,No).call(this,_o.lexInline,xo.parseInline)),Vi(this,"Parser",xo),Vi(this,"parser",xo.parse),Vi(this,"Renderer",bo),Vi(this,"TextRenderer",wo),Vi(this,"Lexer",_o),Vi(this,"lexer",_o.lex),Vi(this,"Tokenizer",po),Vi(this,"Slugger",ko),Vi(this,"Hooks",So),this.use(...arguments)}walkTokens(e,t){let n=[];for(const r of e)switch(n=n.concat(t.call(this,r)),r.type){case"table":for(const e of r.header)n=n.concat(this.walkTokens(e.tokens,t));for(const e of r.rows)for(const r of e)n=n.concat(this.walkTokens(r.tokens,t));break;case"list":n=n.concat(this.walkTokens(r.items,t));break;default:this.defaults.extensions&&this.defaults.extensions.childTokens&&this.defaults.extensions.childTokens[r.type]?this.defaults.extensions.childTokens[r.type].forEach((e=>{n=n.concat(this.walkTokens(r[e],t))})):r.tokens&&(n=n.concat(this.walkTokens(r.tokens,t)))}return n}use(){const e=this.defaults.extensions||{renderers:{},childTokens:{}};for(var t=arguments.length,n=new Array(t),r=0;r{const n={...t};if(n.async=this.defaults.async||n.async||!1,t.extensions&&(t.extensions.forEach((t=>{if(!t.name)throw new Error("extension name required");if(t.renderer){const n=e.renderers[t.name];e.renderers[t.name]=n?function(){for(var e=arguments.length,r=new Array(e),a=0;a{if(this.defaults.async)return Promise.resolve(t.hooks[n].call(e,a)).then((t=>r.call(e,t)));const i=t.hooks[n].call(e,a);return r.call(e,i)}:e[n]=function(){for(var a=arguments.length,i=new Array(a),o=0;o{const t='$1 target="_blank" class="'.concat("vm-link vm-link_colored",'" $2').concat("https://docs.victoriametrics.com/MetricsQL.html","#");return e.replace(/({var n;const r="h3"===e.tagName.toLowerCase();return t=r?null!==(n=e.textContent)&&void 0!==n?n:"":t,r?null:((e,t)=>{var n;const r=null!==(n=t.textContent)&&void 0!==n?n:"",a=(e=>{const t=[];let n=e.nextElementSibling;for(;n&&"p"===n.tagName.toLowerCase();)n&&t.push(n),n=n.nextElementSibling;return t})(t).map((e=>{var t;return null!==(t=e.outerHTML)&&void 0!==t?t:""})).join("\n");return{type:e,value:r,description:Po(a),icon:Ct(yr,{})}})(t,e)})).filter(Boolean)},Io=()=>{const{metricsQLFunctions:e}=En(),t=Nn();return(0,r.useEffect)((()=>{e.length||(async()=>{try{const e=await fetch(Lo),n=(e=>{const t=document.createElement("div");t.innerHTML=To(e);const n=t.querySelectorAll("".concat("h3",", ").concat("h4"));return Oo(n)})(await e.text());t({type:"SET_METRICSQL_FUNCTIONS",payload:n})}catch(rp){console.error("Error fetching or processing the MetricsQL.md file:",rp)}})()}),[]),e},Ro=e=>{let{value:t,anchorEl:n,caretPosition:a,hasHelperText:o,onSelect:l,onFoundOptions:s}=e;const[c,u]=(0,r.useState)({top:0,left:0}),d=Io(),h=(0,r.useMemo)((()=>{if(a[0]!==a[1])return{beforeCursor:t,afterCursor:""};return{beforeCursor:t.substring(0,a[0]),afterCursor:t.substring(a[1])}}),[t,a]),m=(0,r.useMemo)((()=>{const e=h.beforeCursor.split("}");return e[e.length-1]}),[h]),p=(0,r.useMemo)((()=>{const e=m.match(/\b[^{}(),\s]+(?={|$)/g);return e?e[0]:""}),[m]),f=(0,r.useMemo)((()=>{const e=m.match(/[a-z_:-][\w\-.:/]*\b(?=\s*(=|!=|=~|!~))/g);return e?e[e.length-1]:""}),[m]),v=(0,r.useMemo)((()=>{if(!h.beforeCursor||h.beforeCursor.endsWith("}")||(e=>{const t=e.split(/\s+/),n=t.length,r=t[n-1],a=t[n-2],i=!r&&(e=>{const t=e.match(/"/g);return!!t&&t.length%2!==0})(e),o=(!r||t.length>1)&&!/([{(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/.test(a);return i||o})(h.beforeCursor))return pt.empty;const e=/\{[^}]*$/;switch(!0){case new RegExp("(".concat(Oi(p),")?{?.+").concat(Oi(f),'(=|!=|=~|!~)"?([^"]*)$'),"g").test(h.beforeCursor):return pt.labelValue;case e.test(h.beforeCursor):return pt.label;default:return pt.metricsql}}),[h,p,f]),g=(0,r.useMemo)((()=>{const e=h.beforeCursor.match(/([\w_\-.:/]+(?![},]))$/);return e?e[0]:""}),[h.beforeCursor]),{metrics:y,labels:_,labelValues:b,loading:w}=(e=>{let{valueByContext:t,metric:n,label:a,context:o}=e;const{serverUrl:l}=Nt(),{period:{start:s,end:c}}=vn(),{autocompleteCache:u}=En(),d=Nn(),[h,m]=(0,r.useState)(!1),[p,f]=(0,r.useState)(t),v=Pi()(f,500);(0,r.useEffect)((()=>(v(t),v.cancel)),[t,v]);const[g,y]=(0,r.useState)([]),[_,b]=(0,r.useState)([]),[w,k]=(0,r.useState)([]),x=(0,r.useRef)(new AbortController),S=(0,r.useCallback)((e=>{const t=i()(1e3*s).startOf("day").valueOf()/1e3,n=i()(1e3*c).endOf("day").valueOf()/1e3;return new URLSearchParams({...e||{},limit:"".concat(bn),start:"".concat(t),end:"".concat(n)})}),[s,c]),C=(e,t)=>e.map((e=>({value:e,type:"".concat(t),icon:Di[t]}))),E=async e=>{let{value:t,urlSuffix:n,setter:r,type:a,params:i}=e;if(!t&&a===Ri.metric)return;x.current.abort(),x.current=new AbortController;const{signal:o}=x.current,s={type:a,value:t,start:(null===i||void 0===i?void 0:i.get("start"))||"",end:(null===i||void 0===i?void 0:i.get("end"))||"",match:(null===i||void 0===i?void 0:i.get("match[]"))||""};m(!0);try{const e=u.get(s);if(e)return r(C(e,a)),void m(!1);const t=await fetch("".concat(l,"/api/v1/").concat(n,"?").concat(i),{signal:o});if(t.ok){const{data:e}=await t.json();r(C(e,a)),d({type:"SET_AUTOCOMPLETE_CACHE",payload:{key:s,value:e}})}m(!1)}catch(rp){rp instanceof Error&&"AbortError"!==rp.name&&(d({type:"SET_AUTOCOMPLETE_CACHE",payload:{key:s,value:[]}}),m(!1),console.error(rp))}};return(0,r.useEffect)((()=>{const e=o!==pt.metricsql&&o!==pt.empty;if(!l||!n||e)return;y([]);const t=Ii(Oi(n));return E({value:p,urlSuffix:"label/__name__/values",setter:y,type:Ri.metric,params:S({"match[]":'{__name__=~".*'.concat(t,'.*"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n]),(0,r.useEffect)((()=>{if(!l||!n||o!==pt.label)return;b([]);const e=Ii(n);return E({value:p,urlSuffix:"labels",setter:b,type:Ri.label,params:S({"match[]":'{__name__="'.concat(e,'"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n]),(0,r.useEffect)((()=>{if(!l||!n||!a||o!==pt.labelValue)return;k([]);const e=Ii(n),t=Ii(Oi(p));return E({value:p,urlSuffix:"label/".concat(a,"/values"),setter:k,type:Ri.labelValue,params:S({"match[]":'{__name__="'.concat(e,'", ').concat(a,'=~".*').concat(t,'.*"}')})}),()=>{var e;return null===(e=x.current)||void 0===e?void 0:e.abort()}}),[l,p,o,n,a]),{metrics:g,labels:_,labelValues:w,loading:h}})({valueByContext:g,metric:p,label:f,context:v}),k=(0,r.useMemo)((()=>{switch(v){case pt.metricsql:return[...y,...d];case pt.label:return _;case pt.labelValue:return b;default:return[]}}),[v,y,_,b]),x=(0,r.useCallback)((e=>{const t=h.beforeCursor;let n=h.afterCursor;const r=t.lastIndexOf(g,a[0]),i=r+g.length,o=t.substring(0,r),s=t.substring(i);if(v===pt.labelValue){const t='"',r=/(?:=|!=|=~|!~)$/.test(o);n=n.replace(/^[^\s"|},]*/,""),e="".concat(r?t:"").concat(e)}v===pt.label&&(n=n.replace(/^[^\s=!,{}()"|+\-/*^]*/,"")),v===pt.metricsql&&(n=n.replace(/^[^\s[\]{}()"|+\-/*^]*/,""));const c="".concat(o).concat(e).concat(s).concat(n);l(c,o.length+e.length)}),[h]);return(0,r.useEffect)((()=>{if(!n.current)return void u({top:0,left:0});const e=n.current.querySelector("textarea")||n.current,t=window.getComputedStyle(e),r="".concat(t.getPropertyValue("font-size")),a="".concat(t.getPropertyValue("font-family")),i=parseInt("".concat(t.getPropertyValue("line-height"))),l=document.createElement("div");l.style.font="".concat(r," ").concat(a),l.style.padding=t.getPropertyValue("padding"),l.style.lineHeight="".concat(i,"px"),l.style.width="".concat(e.offsetWidth,"px"),l.style.maxWidth="".concat(e.offsetWidth,"px"),l.style.whiteSpace=t.getPropertyValue("white-space"),l.style.overflowWrap=t.getPropertyValue("overflow-wrap");const s=document.createElement("span");l.appendChild(document.createTextNode(h.beforeCursor)),l.appendChild(s),l.appendChild(document.createTextNode(h.afterCursor)),document.body.appendChild(l);const c=l.getBoundingClientRect(),d=s.getBoundingClientRect(),m=d.left-c.left,p=d.bottom-c.bottom-(o?i:0);u({top:p,left:m}),l.remove(),s.remove()}),[n,a,o]),Ct(xt.FK,{children:Ct(Ti,{loading:w,disabledFullScreen:!0,value:g,options:k,anchor:n,minLength:0,offset:c,onSelect:x,onFoundOptions:s,maxDisplayResults:{limit:_n,message:"Please, specify the query more precisely."}})})},Do="No match! \nThis query hasn't selected any time series from database.\nEither the requested metrics are missing in the database,\nor there is a typo in series selector.",zo="The shown results are marked as PARTIAL.\nThe result is marked as partial if one or more vmstorage nodes failed to respond to the query.",Fo=e=>{let{value:t,onChange:n,onEnter:a,onArrowUp:i,onArrowDown:o,autocomplete:l,error:s,stats:c,label:u,disabled:d=!1}=e;const{autocompleteQuick:h}=En(),{isMobile:m}=Wr(),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)([0,0]),y=(0,r.useRef)(null),[_,b]=(0,r.useState)(l),w=(0,r.useRef)(Pi()(b,500)).current,k=[{show:"0"===(null===c||void 0===c?void 0:c.seriesFetched)&&!c.resultLength,text:Do},{show:null===c||void 0===c?void 0:c.isPartial,text:zo}].filter((e=>e.show)).map((e=>e.text)).join("");c&&(u="".concat(u," (").concat(c.executionTimeMsec||0,"ms)"));return(0,r.useEffect)((()=>{f(l)}),[h]),(0,r.useEffect)((()=>{b(!1),w(!0)}),[v]),Ct("div",{className:"vm-query-editor",ref:y,children:[Ct(Ha,{value:t,label:u,type:"textarea",autofocus:!m,error:s,warning:k,onKeyDown:e=>{const{key:t,ctrlKey:n,metaKey:r,shiftKey:l}=e,s=(e.target.value||"").split("\n").length>1,c=n||r,u="ArrowDown"===t,d="Enter"===t;"ArrowUp"===t&&c&&(e.preventDefault(),i()),u&&c&&(e.preventDefault(),o()),d&&p&&e.preventDefault(),!d||l||s&&!c||p||(e.preventDefault(),a())},onChange:n,onChangeCaret:e=>{g((t=>t[0]===e[0]&&t[1]===e[1]?t:e))},disabled:d,inputmode:"search",caretPosition:v}),_&&l&&Ct(Ro,{value:t,anchorEl:y,caretPosition:v,hasHelperText:Boolean(k||s),onSelect:(e,t)=>{n(e),g([t,t])},onFoundOptions:e=>{f(!!e.length)}})]})},jo=e=>{let{value:t=!1,disabled:n=!1,label:r,color:a="secondary",fullWidth:i,onChange:o}=e;return Ct("div",{className:xr()({"vm-switch":!0,"vm-switch_full-width":i,"vm-switch_disabled":n,"vm-switch_active":t,["vm-switch_".concat(a,"_active")]:t,["vm-switch_".concat(a)]:a}),onClick:()=>{n||o(!t)},children:[Ct("div",{className:"vm-switch-track",children:Ct("div",{className:"vm-switch-track__thumb"})}),r&&Ct("span",{className:"vm-switch__label",children:r})]})},Ho=e=>{let{isMobile:t,hideButtons:n}=e;const{autocomplete:r}=En(),a=Nn(),{nocache:i,isTracingEnabled:o}=Fr(),l=jr();return Nr("keydown",(e=>{const{code:t,ctrlKey:n,altKey:r}=e;"Space"===t&&(n||r)&&(e.preventDefault(),a({type:"SET_AUTOCOMPLETE_QUICK",payload:!0}))})),Ct("div",{className:xr()({"vm-additional-settings":!0,"vm-additional-settings_mobile":t}),children:[!(null!==n&&void 0!==n&&n.autocomplete)&&Ct(fa,{title:Ct(xt.FK,{children:["Quick tip: ",ka]}),children:Ct(jo,{label:"Autocomplete",value:r,onChange:()=>{a({type:"TOGGLE_AUTOCOMPLETE"})},fullWidth:t})}),Ct(jo,{label:"Disable cache",value:i,onChange:()=>{l({type:"TOGGLE_NO_CACHE"})},fullWidth:t}),!(null!==n&&void 0!==n&&n.traceQuery)&&Ct(jo,{label:"Trace query",value:o,onChange:()=>{l({type:"TOGGLE_QUERY_TRACING"})},fullWidth:t})]})},Vo=e=>{const{isMobile:t}=Wr(),n=(0,r.useRef)(null),{value:a,toggle:i,setFalse:o}=oa(!1);return t?Ct(xt.FK,{children:[Ct("div",{ref:n,children:Ct(aa,{variant:"outlined",startIcon:Ct(hr,{}),onClick:i,ariaLabel:"additional the query settings"})}),Ct(ia,{open:a,buttonRef:n,placement:"bottom-left",onClose:o,title:"Query settings",children:Ct(Ho,{isMobile:t,...e})})]}):Ct(Ho,{...e})},$o=(e,t)=>e.length===t.length&&e.every(((e,n)=>e===t[n]));const Uo=()=>{const{showInfoMessage:e}=(0,r.useContext)(Zr);return async(t,n)=>{var r;if(null===(r=navigator)||void 0===r||!r.clipboard)return e({text:"Clipboard not supported",type:"error"}),console.warn("Clipboard not supported"),!1;try{return await navigator.clipboard.writeText(t),n&&e({text:n,type:"success"}),!0}catch(a){return a instanceof Error&&e({text:"".concat(a.name,": ").concat(a.message),type:"error"}),console.warn("Copy failed",a),!1}}},Bo=e=>{let{query:t,favorites:n,onRun:a,onToggleFavorite:i}=e;const o=Uo(),l=(0,r.useMemo)((()=>n.includes(t)),[t,n]);return Ct("div",{className:"vm-query-history-item",children:[Ct("span",{className:"vm-query-history-item__value",children:t}),Ct("div",{className:"vm-query-history-item__buttons",children:[Ct(fa,{title:"Execute query",children:Ct(aa,{size:"small",variant:"text",onClick:()=>{a(t)},startIcon:Ct(Wn,{})})}),Ct(fa,{title:"Copy query",children:Ct(aa,{size:"small",variant:"text",onClick:async()=>{await o(t,"Query has been copied")},startIcon:Ct(ar,{})})}),Ct(fa,{title:l?"Remove Favorite":"Add to Favorites",children:Ct(aa,{size:"small",variant:"text",color:l?"warning":"primary",onClick:()=>{i(t,l)},startIcon:Ct(l?vr:fr,{})})})]})]})},qo="saved",Yo="favorite",Wo=[{label:"Session history",value:"session"},{label:"Saved history",value:qo},{label:"Favorite queries",value:Yo}],Ko=e=>{let{handleSelectQuery:t}=e;const{queryHistory:n}=En(),{isMobile:a}=Wr(),{value:i,setTrue:o,setFalse:l}=oa(!1),[s,c]=(0,r.useState)(Wo[0].value),[u,d]=(0,r.useState)(yn("QUERY_HISTORY")),[h,m]=(0,r.useState)(yn("QUERY_FAVORITES")),p=(0,r.useMemo)((()=>n.map((e=>e.values.filter((e=>e)).reverse()))),[n]),f=(0,r.useMemo)((()=>{switch(s){case Yo:return h;case qo:return u;default:return p}}),[s,h,u,p]),v=null===f||void 0===f?void 0:f.every((e=>!e.length)),g=(0,r.useMemo)((()=>s===Yo?"Favorites queries are empty.\nTo see your favorites, mark a query as a favorite.":"Query history is empty.\nTo see the history, please make a query."),[s]),y=e=>n=>{t(n,e),l()},_=(e,t)=>{m((n=>{const r=n[0]||[];return t?[r.filter((t=>t!==e))]:t||r.includes(e)?n:[[...r,e]]}))};return(0,r.useEffect)((()=>{const e=h[0]||[],t=yn("QUERY_FAVORITES")[0]||[];$o(e,t)||Je("QUERY_FAVORITES",JSON.stringify(h))}),[h]),Nr("storage",(()=>{d(yn("QUERY_HISTORY")),m(yn("QUERY_FAVORITES"))})),Ct(xt.FK,{children:[Ct(fa,{title:"Show history",children:Ct(aa,{color:"primary",variant:"text",onClick:o,startIcon:Ct($n,{}),ariaLabel:"Show history"})}),i&&Ct(pa,{title:"Query history",onClose:l,children:Ct("div",{className:xr()({"vm-query-history":!0,"vm-query-history_mobile":a}),children:[Ct("div",{className:xr()({"vm-query-history__tabs":!0,"vm-section-header__tabs":!0,"vm-query-history__tabs_mobile":a}),children:Ct(Mr,{activeItem:s,items:Wo,onChange:c})}),Ct("div",{className:"vm-query-history-list",children:[v&&Ct("div",{className:"vm-query-history-list__no-data",children:g}),f.map(((e,t)=>Ct("div",{children:[f.length>1&&Ct("div",{className:xr()({"vm-query-history-list__group-title":!0,"vm-query-history-list__group-title_first":0===t}),children:["Query ",t+1]}),e.map(((e,n)=>Ct(Bo,{query:e,favorites:h.flat(),onRun:y(t),onToggleFavorite:_},n)))]},t))),s===qo&&!v&&Ct("div",{className:"vm-query-history-footer",children:Ct(aa,{color:"error",variant:"outlined",size:"small",startIcon:Ct(Gn,{}),onClick:()=>{Je("QUERY_HISTORY","")},children:"clear history"})})]})]})})]})},Qo=e=>{let{queryErrors:t,setQueryErrors:n,setHideError:a,stats:i,onHideQuery:o,onRunQuery:l,hideButtons:s}=e;const{isMobile:c}=Wr(),{query:u,queryHistory:d,autocomplete:h,autocompleteQuick:m}=En(),p=Nn(),f=gn(),[v,g]=(0,r.useState)(u||[]),[y,_]=(0,r.useState)([]),[b,w]=(0,r.useState)(!1),k=$a(v),x=(()=>{const{serverUrl:e}=Nt();return async t=>{try{const n=encodeURIComponent(t),r="".concat(e,"/prettify-query?query=").concat(n),a=await fetch(r);if(200!=a.status)return{query:t,error:"Error requesting /prettify-query, status: "+a.status};const i=await a.json();return"success"!=i.status?{query:t,error:String(i.msg)}:{query:String(i.query),error:""}}catch(rp){return console.error(rp),rp instanceof Error&&"AbortError"!==rp.name?{query:t,error:"".concat(rp.name,": ").concat(rp.message)}:{query:t,error:String(rp)}}}})(),S=()=>{p({type:"SET_QUERY_HISTORY",payload:v.map(((e,t)=>{const n=d[t]||{values:[]},r=e===n.values[n.values.length-1],a=!r&&e?[...n.values,e]:n.values;return a.length>25&&a.shift(),{index:n.values.length-Number(r),values:a}}))}),p({type:"SET_QUERY",payload:v}),f({type:"RUN_QUERY"}),l()},C=(e,t)=>{g((n=>n.map(((n,r)=>r===t?e:n))))},E=(e,t)=>()=>{((e,t)=>{const{index:n,values:r}=d[t],a=n+e;a<0||a>=r.length||(C(r[a]||"",t),p({type:"SET_QUERY_HISTORY_BY_INDEX",payload:{value:{values:r,index:a},queryNumber:t}}))})(e,t)},N=e=>t=>{C(t,e),p({type:"SET_AUTOCOMPLETE_QUICK",payload:!1})},A=e=>()=>{var t;t=e,g((e=>e.filter(((e,n)=>n!==t)))),_((t=>t.includes(e)?t.filter((t=>t!==e)):t.map((t=>t>e?t-1:t))))},M=e=>t=>{((e,t)=>{const{ctrlKey:n,metaKey:r}=e;if(n||r){const e=v.map(((e,t)=>t)).filter((e=>e!==t));_((t=>$o(e,t)?[]:e))}else _((e=>e.includes(t)?e.filter((e=>e!==t)):[...e,t]))})(t,e)};return(0,r.useEffect)((()=>{k&&v.length{o&&o(y)}),[y]),(0,r.useEffect)((()=>{b&&(S(),w(!1))}),[v,b]),Ct("div",{className:xr()({"vm-query-configurator":!0,"vm-block":!0,"vm-block_mobile":c}),children:[Ct("div",{className:"vm-query-configurator-list",children:v.map(((e,r)=>Ct("div",{className:xr()({"vm-query-configurator-list-row":!0,"vm-query-configurator-list-row_disabled":y.includes(r),"vm-query-configurator-list-row_mobile":c}),children:[Ct(Fo,{value:v[r],autocomplete:!(null!==s&&void 0!==s&&s.autocomplete)&&(h||m),error:t[r],stats:i[r],onArrowUp:E(-1,r),onArrowDown:E(1,r),onEnter:S,onChange:N(r),label:"Query ".concat(v.length>1?r+1:""),disabled:y.includes(r)}),o&&Ct(fa,{title:y.includes(r)?"Enable query":"Disable query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"gray",startIcon:y.includes(r)?Ct(nr,{}):Ct(tr,{}),onClick:M(r),ariaLabel:"visibility query"})})}),!(null!==s&&void 0!==s&&s.prettify)&&Ct(fa,{title:"Prettify query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"gray",startIcon:Ct(rr,{}),onClick:async()=>await(async e=>{const t=await x(v[e]);a(!1),C(t.query,e),n((n=>(n[e]=t.error,[...n])))})(r),className:"prettify",ariaLabel:"prettify the query"})})}),v.length>1&&Ct(fa,{title:"Remove Query",children:Ct("div",{className:"vm-query-configurator-list-row__button",children:Ct(aa,{variant:"text",color:"error",startIcon:Ct(Gn,{}),onClick:A(r),ariaLabel:"remove query"})})})]},r)))}),Ct("div",{className:"vm-query-configurator-settings",children:[Ct(Vo,{hideButtons:s}),Ct("div",{className:"vm-query-configurator-settings__buttons",children:[Ct(Ko,{handleSelectQuery:(e,t)=>{C(e,t),w(!0)}}),!(null!==s&&void 0!==s&&s.addQuery)&&v.length<4&&Ct(aa,{variant:"outlined",onClick:()=>{g((e=>[...e,""]))},startIcon:Ct(Jn,{}),children:"Add Query"}),Ct(aa,{variant:"contained",onClick:S,startIcon:Ct(Yn,{}),children:c?"Execute":"Execute Query"})]})]})]})};let Zo=0;class Go{constructor(e,t){this.tracing=void 0,this.query=void 0,this.tracingChildren=void 0,this.originalTracing=void 0,this.id=void 0,this.tracing=e,this.originalTracing=JSON.parse(JSON.stringify(e)),this.query=t,this.id=Zo++;const n=e.children||[];this.tracingChildren=n.map((e=>new Go(e,t)))}get queryValue(){return this.query}get idValue(){return this.id}get children(){return this.tracingChildren}get message(){return this.tracing.message}get duration(){return this.tracing.duration_msec}get JSON(){return JSON.stringify(this.tracing,null,2)}get originalJSON(){return JSON.stringify(this.originalTracing,null,2)}setTracing(e){this.tracing=e;const t=e.children||[];this.tracingChildren=t.map((e=>new Go(e,this.query)))}setQuery(e){this.query=e}resetTracing(){this.tracing=this.originalTracing}}const Jo=function(e,t){let n=!(arguments.length>2&&void 0!==arguments[2])||arguments[2];const{__name__:r,...a}=e.metric,i=t||"".concat(n?"[Query ".concat(e.group,"] "):"").concat(r||"");return 0==Object.keys(a).length?i||"value":"".concat(i,"{").concat(Object.entries(a).map((e=>"".concat(e[0],"=").concat(JSON.stringify(e[1])))).join(", "),"}")},Xo=e=>{switch(e){case"NaN":return NaN;case"Inf":case"+Inf":return 1/0;case"-Inf":return-1/0;default:return parseFloat(e)}},el=e=>{if(e.length<2)return!1;const t=["le","vmrange"],n=Object.keys(e[0].metric).filter((e=>!t.includes(e))),r=e.every((r=>{const a=Object.keys(r.metric).filter((e=>!t.includes(e)));return n.length===a.length&&a.every((t=>r.metric[t]===e[0].metric[t]))}));return r&&e.every((e=>t.some((t=>t in e.metric))))},tl=He.anomaly==={NODE_ENV:"production",PUBLIC_URL:".",WDS_SOCKET_HOST:void 0,WDS_SOCKET_PATH:void 0,WDS_SOCKET_PORT:void 0,FAST_REFRESH:!1}.REACT_APP_TYPE,nl=e=>{let{predefinedQuery:t,visible:n,display:a,customStep:i,hideQuery:o,showAllSeries:l}=e;const{query:s}=En(),{period:c}=vn(),{displayType:u,nocache:d,isTracingEnabled:h,seriesLimits:m}=Fr(),{serverUrl:p}=Nt(),{isHistogram:f}=Ur(),[v,g]=(0,r.useState)(!1),[y,_]=(0,r.useState)(),[b,w]=(0,r.useState)(),[k,x]=(0,r.useState)(),[S,C]=(0,r.useState)(),[E,N]=(0,r.useState)([]),[A,M]=(0,r.useState)([]),[T,L]=(0,r.useState)(),[P,O]=(0,r.useState)([]),[I,R]=(0,r.useState)(!1),D=(0,r.useMemo)((()=>{const{end:e,start:t}=c;return Kt(e-t,f)}),[c,f]),z=(0,r.useCallback)(Pi()((async e=>{let{fetchUrl:t,fetchQueue:n,displayType:r,query:a,stateSeriesLimits:i,showAllSeries:o,hideQuery:l}=e;const s=new AbortController;O([...n,s]);try{const e=r===dt.chart,n=o?1/0:+i[r]||1/0;let c=n;const u=[],d=[];let h=1,m=0,p=!1;for await(const r of t){if(null===l||void 0===l?void 0:l.includes(h-1)){N((e=>[...e,""])),M((e=>[...e,{}])),h++;continue}const t=await fetch(r,{signal:s.signal}),i=await t.json();if(t.ok){if(M((e=>[...e,{...null===i||void 0===i?void 0:i.stats,isPartial:null===i||void 0===i?void 0:i.isPartial,resultLength:i.data.result.length}])),N((e=>[...e,""])),i.trace){const e=new Go(i.trace,a[h-1]);d.push(e)}p=!tl&&e&&el(i.data.result),c=p?1/0:n;const t=c-u.length;i.data.result.slice(0,t).forEach((e=>{e.group=h,u.push(e)})),m+=i.data.result.length}else{u.push({metric:{},values:[],group:h});const e=i.errorType||ht.unknownType,t=[e,(null===i||void 0===i?void 0:i.error)||(null===i||void 0===i?void 0:i.message)||"see console for more details"].join(",\r\n");N((e=>[...e,"".concat(t)])),console.error("Fetch query error: ".concat(e),i)}h++}const f="Showing ".concat(u.length," series out of ").concat(m," series due to performance reasons. Please narrow down the query, so it returns less series");L(m>c?f:""),e?_(u):w(u),x(d),R((e=>m?p:e))}catch(rp){const t=rp;if("AbortError"===t.name)return;const n="Please check your serverURL settings and confirm server availability.";let r="Error executing query: ".concat(t.message,". ").concat(n);"Unexpected end of JSON input"===t.message&&(r+="\nAdditionally, this error can occur if the server response is too large to process. Apply more specific filters to reduce the data volume."),C(r)}g(!1)}),300),[]),F=(0,r.useMemo)((()=>{C(""),N([]),M([]);const e=null!==t&&void 0!==t?t:s,n=(a||u)===dt.chart;if(c)if(p)if(e.every((e=>!e.trim())))N(e.map((()=>ht.validQuery)));else{if(yt(p)){const t={...c};return t.step=tl?"".concat(1e3*Wt(i),"ms"):i,e.map((e=>n?((e,t,n,r,a)=>"".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(a?"&trace=1":""))(p,e,t,d,h):((e,t,n,r,a)=>"".concat(e,"/api/v1/query?query=").concat(encodeURIComponent(t),"&time=").concat(n.end,"&step=").concat(n.step).concat(r?"&nocache=1":"").concat(a?"&trace=1":""))(p,e,t,d,h)))}C(ht.validServer)}else C(ht.emptyServer)}),[p,c,u,i,o]),[j,H]=(0,r.useState)([]);return(0,r.useEffect)((()=>{const e=F===j&&!!t;if(!n||null===F||void 0===F||!F.length||e)return;g(!0);z({fetchUrl:F,fetchQueue:P,displayType:a||u,query:null!==t&&void 0!==t?t:s,stateSeriesLimits:m,showAllSeries:l,hideQuery:o}),H(F)}),[F,n,m,l]),(0,r.useEffect)((()=>{const e=P.slice(0,-1);e.length&&(e.map((e=>e.abort())),O(P.filter((e=>!e.signal.aborted))))}),[P]),(0,r.useEffect)((()=>{D===i&&_([])}),[I]),{fetchUrl:F,isLoading:v,graphData:y,liveData:b,error:S,queryErrors:E,setQueryErrors:N,queryStats:A,warning:T,traces:k,isHistogram:I}},rl=e=>{let{containerStyles:t={},message:n}=e;const{isDarkTheme:r}=Nt();return Ct("div",{className:xr()({"vm-spinner":!0,"vm-spinner_dark":r}),style:t&&{},children:[Ct("div",{className:"half-circle-spinner",children:[Ct("div",{className:"circle circle-1"}),Ct("div",{className:"circle circle-2"})]}),n&&Ct("div",{className:"vm-spinner__message",children:n})]})};function al(e){return function(e,t){return Object.fromEntries(Object.entries(e).filter(t))}(e,(e=>!!e[1]||"number"===typeof e[1]))}const il=e=>{let{text:t,href:n,children:r,colored:a=!0,underlined:i=!1,withIcon:o=!1}=e;return Ct("a",{href:n,className:xr()({"vm-link":!0,"vm-link_colored":a,"vm-link_underlined":i,"vm-link_with-icon":o}),target:"_blank",rel:"noreferrer",children:t||r})},ol=Ct(il,{text:"last_over_time",href:"https://docs.victoriametrics.com/MetricsQL.html#last_over_time",underlined:!0}),ll=Ct(il,{text:"instant query",href:"https://docs.victoriametrics.com/keyConcepts.html#instant-query",underlined:!0}),sl=()=>Ct("div",{children:[Ct("p",{children:["This tab shows ",ll," results for the last 5 minutes ending at the selected time range."]}),Ct("p",{children:["Please wrap the query into ",ol," if you need results over arbitrary lookbehind interval."]})]}),cl=e=>{let{value:t}=e;return Ct("div",{className:"vm-line-progress",children:[Ct("div",{className:"vm-line-progress-track",children:Ct("div",{className:"vm-line-progress-track__thumb",style:{width:"".concat(t,"%")}})}),Ct("span",{children:[t.toFixed(2),"%"]})]})},ul=e=>{let{isRoot:t,trace:n,totalMsec:a,isExpandedAll:i}=e;const{isDarkTheme:o}=Nt(),{isMobile:l}=Wr(),[s,c]=(0,r.useState)({}),u=(0,r.useRef)(null),[d,h]=(0,r.useState)(!1),[m,p]=(0,r.useState)(!1),f=Bt(n.duration/1e3)||"".concat(n.duration,"ms");(0,r.useEffect)((()=>{if(!u.current)return;const e=u.current,t=u.current.children[0],{height:n}=t.getBoundingClientRect();h(n>e.clientHeight)}),[n]);const v=n.children&&!!n.children.length,g=n.duration/a*100,y=e=>{var t;const n=[e.idValue];return null===e||void 0===e||null===(t=e.children)||void 0===t||t.forEach((e=>{n.push(...y(e))})),n};return(0,r.useEffect)((()=>{if(!i)return void c([]);const e=y(n),t={};e.forEach((e=>{t[e]=!0})),c(t)}),[i]),Ct("div",{className:xr()({"vm-nested-nav":!0,"vm-nested-nav_root":t,"vm-nested-nav_dark":o,"vm-nested-nav_mobile":l}),children:[Ct("div",{className:xr()({"vm-nested-nav-header":!0,"vm-nested-nav-header_open":s[n.idValue]}),onClick:(_=n.idValue,()=>{v&&c((e=>({...e,[_]:!e[_]})))}),children:[v&&Ct("div",{className:xr()({"vm-nested-nav-header__icon":!0,"vm-nested-nav-header__icon_open":s[n.idValue]}),children:Ct(Hn,{})}),Ct("div",{className:"vm-nested-nav-header__progress",children:Ct(cl,{value:g})}),Ct("div",{className:xr()({"vm-nested-nav-header__message":!0,"vm-nested-nav-header__message_show-full":m}),ref:u,children:[Ct("span",{className:"vm-nested-nav-header__message_duration",children:f}),":\xa0",Ct("span",{children:n.message})]}),Ct("div",{className:"vm-nested-nav-header-bottom",children:(d||m)&&Ct(aa,{variant:"text",size:"small",onClick:e=>{e.stopPropagation(),p((e=>!e))},children:m?"Hide":"Show full query"})})]}),s[n.idValue]&&Ct("div",{className:"vm-nested-nav__childrens",children:v&&n.children.map((e=>Ct(ul,{trace:e,totalMsec:a,isExpandedAll:i},e.duration)))})]});var _},dl=ul,hl=e=>{let{editable:t=!1,defaultTile:n="JSON",displayTitle:a=!0,defaultJson:i="",resetValue:o="",onClose:l,onUpload:s}=e;const c=Uo(),{isMobile:u}=Wr(),[d,h]=(0,r.useState)(i),[m,p]=(0,r.useState)(n),[f,v]=(0,r.useState)(""),[g,y]=(0,r.useState)(""),_=(0,r.useMemo)((()=>{try{const e=JSON.parse(d),t=e.trace||e;return t.duration_msec?(new Go(t,""),""):ht.traceNotFound}catch(rp){return rp instanceof Error?rp.message:"Unknown error"}}),[d]),b=()=>{y(_);m.trim()||v(ht.emptyTitle),_||f||(s(d,m),l())};return Ct("div",{className:xr()({"vm-json-form":!0,"vm-json-form_one-field":!a,"vm-json-form_one-field_mobile":!a&&u,"vm-json-form_mobile":u}),children:[a&&Ct(Ha,{value:m,label:"Title",error:f,onEnter:b,onChange:e=>{p(e)}}),Ct(Ha,{value:d,label:"JSON",type:"textarea",error:g,autofocus:!0,onChange:e=>{y(""),h(e)},onEnter:b,disabled:!t}),Ct("div",{className:"vm-json-form-footer",children:[Ct("div",{className:"vm-json-form-footer__controls",children:[Ct(aa,{variant:"outlined",startIcon:Ct(ar,{}),onClick:async()=>{await c(d,"Formatted JSON has been copied")},children:"Copy JSON"}),o&&Ct(aa,{variant:"text",startIcon:Ct(In,{}),onClick:()=>{h(o)},children:"Reset JSON"})]}),Ct("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Ct(aa,{variant:"outlined",color:"error",onClick:l,children:"Cancel"}),Ct(aa,{variant:"contained",onClick:b,children:"apply"})]})]})]})},ml=e=>{let{traces:t,jsonEditor:n=!1,onDeleteClick:a}=e;const{isMobile:i}=Wr(),[o,l]=(0,r.useState)(null),[s,c]=(0,r.useState)([]),u=()=>{l(null)};if(!t.length)return Ct(Qr,{variant:"info",children:"Please re-run the query to see results of the tracing"});const d=e=>()=>{a(e)},h=e=>()=>{l(e)},m=e=>()=>{const t=new Blob([e.originalJSON],{type:"application/json"}),n=URL.createObjectURL(t),r=document.createElement("a");r.href=n,r.download="vmui_trace_".concat(e.queryValue,".json"),document.body.appendChild(r),r.click(),document.body.removeChild(r),URL.revokeObjectURL(n)};return Ct(xt.FK,{children:[Ct("div",{className:"vm-tracings-view",children:t.map((e=>{return Ct("div",{className:"vm-tracings-view-trace vm-block vm-block_empty-padding",children:[Ct("div",{className:"vm-tracings-view-trace-header",children:[Ct("h3",{className:"vm-tracings-view-trace-header-title",children:["Trace for ",Ct("b",{className:"vm-tracings-view-trace-header-title__query",children:e.queryValue})]}),Ct(fa,{title:s.includes(e.idValue)?"Collapse All":"Expand All",children:Ct(aa,{variant:"text",startIcon:Ct("div",{className:xr()({"vm-tracings-view-trace-header__expand-icon":!0,"vm-tracings-view-trace-header__expand-icon_open":s.includes(e.idValue)}),children:Ct(Hn,{})}),onClick:(t=e,()=>{c((e=>e.includes(t.idValue)?e.filter((e=>e!==t.idValue)):[...e,t.idValue]))}),ariaLabel:s.includes(e.idValue)?"Collapse All":"Expand All"})}),Ct(fa,{title:"Save Trace to JSON",children:Ct(aa,{variant:"text",startIcon:Ct(wr,{}),onClick:m(e),ariaLabel:"Save trace to JSON"})}),Ct(fa,{title:"Open JSON",children:Ct(aa,{variant:"text",startIcon:Ct(Zn,{}),onClick:h(e),ariaLabel:"open JSON"})}),Ct(fa,{title:"Remove trace",children:Ct(aa,{variant:"text",color:"error",startIcon:Ct(Gn,{}),onClick:d(e),ariaLabel:"remove trace"})})]}),Ct("nav",{className:xr()({"vm-tracings-view-trace__nav":!0,"vm-tracings-view-trace__nav_mobile":i}),children:Ct(dl,{isRoot:!0,trace:e,totalMsec:e.duration,isExpandedAll:s.includes(e.idValue)})})]},e.idValue);var t}))}),o&&Ct(pa,{title:o.queryValue,onClose:u,children:Ct(hl,{editable:n,displayTitle:n,defaultTile:o.queryValue,defaultJson:o.JSON,resetValue:o.originalJSON,onClose:u,onUpload:(e,t)=>{if(n&&o)try{o.setTracing(JSON.parse(e)),o.setQuery(t),l(null)}catch(rp){console.error(rp)}}})})]})},pl=e=>{let{traces:t,displayType:n}=e;const{isTracingEnabled:a}=Fr(),[i,o]=(0,r.useState)([]);return(0,r.useEffect)((()=>{t&&o([...i,...t])}),[t]),(0,r.useEffect)((()=>{o([])}),[n]),Ct(xt.FK,{children:a&&Ct("div",{className:"vm-custom-panel__trace",children:Ct(ml,{traces:i,onDeleteClick:e=>{const t=i.filter((t=>t.idValue!==e.idValue));o([...t])}})})})},fl=e=>{let{warning:t,query:n,onChange:a}=e;const{isMobile:i}=Wr(),{value:o,setTrue:l,setFalse:s}=oa(!1);return(0,r.useEffect)(s,[n]),(0,r.useEffect)((()=>{a(o)}),[o]),Ct(Qr,{variant:"warning",children:Ct("div",{className:xr()({"vm-custom-panel__warning":!0,"vm-custom-panel__warning_mobile":i}),children:[Ct("p",{children:t}),Ct(aa,{color:"warning",variant:"outlined",onClick:l,children:"Show all"})]})})},vl="u-off",gl="u-label",yl="width",_l="height",bl="top",wl="bottom",kl="left",xl="right",Sl="#000",Cl=Sl+"0",El="mousemove",Nl="mousedown",Al="mouseup",Ml="mouseenter",Tl="mouseleave",Ll="dblclick",Pl="change",Ol="dppxchange",Il="--",Rl="undefined"!=typeof window,Dl=Rl?document:null,zl=Rl?window:null,Fl=Rl?navigator:null;let jl,Hl;function Vl(e,t){if(null!=t){let n=e.classList;!n.contains(t)&&n.add(t)}}function $l(e,t){let n=e.classList;n.contains(t)&&n.remove(t)}function Ul(e,t,n){e.style[t]=n+"px"}function Bl(e,t,n,r){let a=Dl.createElement(e);return null!=t&&Vl(a,t),null!=n&&n.insertBefore(a,r),a}function ql(e,t){return Bl("div",e,t)}const Yl=new WeakMap;function Wl(e,t,n,r,a){let i="translate("+t+"px,"+n+"px)";i!=Yl.get(e)&&(e.style.transform=i,Yl.set(e,i),t<0||n<0||t>r||n>a?Vl(e,vl):$l(e,vl))}const Kl=new WeakMap;function Ql(e,t,n){let r=t+n;r!=Kl.get(e)&&(Kl.set(e,r),e.style.background=t,e.style.borderColor=n)}const Zl=new WeakMap;function Gl(e,t,n,r){let a=t+""+n;a!=Zl.get(e)&&(Zl.set(e,a),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)}const Jl={passive:!0},Xl={...Jl,capture:!0};function es(e,t,n,r){t.addEventListener(e,n,r?Xl:Jl)}function ts(e,t,n,r){t.removeEventListener(e,n,r?Xl:Jl)}function ns(e,t,n,r){let a;n=n||0;let i=(r=r||t.length-1)<=2147483647;for(;r-n>1;)a=i?n+r>>1:ys((n+r)/2),t[a]=t&&a<=n;a+=r)if(null!=e[a])return a;return-1}function as(e,t,n,r){let a=Ss(e),i=Ss(t);e==t&&(-1==a?(e*=n,t/=n):(e/=n,t*=n));let o=10==n?Cs:Es,l=1==i?bs:ys,s=(1==a?ys:bs)(o(gs(e))),c=l(o(gs(t))),u=xs(n,s),d=xs(n,c);return 10==n&&(s<0&&(u=Vs(u,-s)),c<0&&(d=Vs(d,-c))),r||2==n?(e=u*a,t=d*i):(e=Hs(e,u),t=js(t,d)),[e,t]}function is(e,t,n,r){let a=as(e,t,n,r);return 0==e&&(a[0]=0),0==t&&(a[1]=0),a}Rl&&function e(){let t=devicePixelRatio;jl!=t&&(jl=t,Hl&&ts(Pl,Hl,e),Hl=matchMedia("(min-resolution: ".concat(jl-.001,"dppx) and (max-resolution: ").concat(jl+.001,"dppx)")),es(Pl,Hl,e),zl.dispatchEvent(new CustomEvent(Ol)))}();const os=.1,ls={mode:3,pad:os},ss={pad:0,soft:null,mode:0},cs={min:ss,max:ss};function us(e,t,n,r){return Gs(n)?hs(e,t,n):(ss.pad=n,ss.soft=r?0:null,ss.mode=r?3:0,hs(e,t,cs))}function ds(e,t){return null==e?t:e}function hs(e,t,n){let r=n.min,a=n.max,i=ds(r.pad,0),o=ds(a.pad,0),l=ds(r.hard,-As),s=ds(a.hard,As),c=ds(r.soft,As),u=ds(a.soft,-As),d=ds(r.mode,0),h=ds(a.mode,0),m=t-e,p=Cs(m),f=ks(gs(e),gs(t)),v=Cs(f),g=gs(v-p);(m<1e-9||g>10)&&(m=0,0!=e&&0!=t||(m=1e-9,2==d&&c!=As&&(i=0),2==h&&u!=-As&&(o=0)));let y=m||f||1e3,_=Cs(y),b=xs(10,ys(_)),w=Vs(Hs(e-y*(0==m?0==e?.1:1:i),b/10),9),k=e>=c&&(1==d||3==d&&w<=c||2==d&&w>=c)?c:As,x=ks(l,w=k?k:ws(k,w)),S=Vs(js(t+y*(0==m?0==t?.1:1:o),b/10),9),C=t<=u&&(1==h||3==h&&S>=u||2==h&&S<=u)?u:-As,E=ws(s,S>C&&t<=C?C:ks(C,S));return x==E&&0==x&&(E=100),[x,E]}const ms=new Intl.NumberFormat(Rl?Fl.language:"en-US"),ps=e=>ms.format(e),fs=Math,vs=fs.PI,gs=fs.abs,ys=fs.floor,_s=fs.round,bs=fs.ceil,ws=fs.min,ks=fs.max,xs=fs.pow,Ss=fs.sign,Cs=fs.log10,Es=fs.log2,Ns=function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return fs.asinh(e/t)},As=1/0;function Ms(e){return 1+(0|Cs((e^e>>31)-(e>>31)))}function Ts(e,t,n){return ws(ks(e,t),n)}function Ls(e){return"function"==typeof e?e:()=>e}const Ps=e=>e,Os=(e,t)=>t,Is=e=>null,Rs=e=>!0,Ds=(e,t)=>e==t,zs=e=>Vs(e,14);function Fs(e,t){return zs(Vs(zs(e/t))*t)}function js(e,t){return zs(bs(zs(e/t))*t)}function Hs(e,t){return zs(ys(zs(e/t))*t)}function Vs(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:0;if(Qs(e))return e;let n=10**t,r=e*n*(1+Number.EPSILON);return _s(r)/n}const $s=new Map;function Us(e){return((""+e).split(".")[1]||"").length}function Bs(e,t,n,r){let a=[],i=r.map(Us);for(let o=t;o=0&&o>=0?0:t)+(o>=i[e]?0:i[e]),c=Vs(l,s);a.push(c),$s.set(c,s)}}return a}const qs={},Ys=[],Ws=[null,null],Ks=Array.isArray,Qs=Number.isInteger;function Zs(e){return"string"==typeof e}function Gs(e){let t=!1;if(null!=e){let n=e.constructor;t=null==n||n==Object}return t}function Js(e){return null!=e&&"object"==typeof e}const Xs=Object.getPrototypeOf(Uint8Array);function ec(e){let t,n=arguments.length>1&&void 0!==arguments[1]?arguments[1]:Gs;if(Ks(e)){let r=e.find((e=>null!=e));if(Ks(r)||n(r)){t=Array(e.length);for(let r=0;ri){for(r=o-1;r>=0&&null==e[r];)e[r--]=null;for(r=o+1;rPromise.resolve().then(e):queueMicrotask;const ac=["January","February","March","April","May","June","July","August","September","October","November","December"],ic=["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"];function oc(e){return e.slice(0,3)}const lc=ic.map(oc),sc=ac.map(oc),cc={MMMM:ac,MMM:sc,WWWW:ic,WWW:lc};function uc(e){return(e<10?"0":"")+e}const dc={YYYY:e=>e.getFullYear(),YY:e=>(e.getFullYear()+"").slice(2),MMMM:(e,t)=>t.MMMM[e.getMonth()],MMM:(e,t)=>t.MMM[e.getMonth()],MM:e=>uc(e.getMonth()+1),M:e=>e.getMonth()+1,DD:e=>uc(e.getDate()),D:e=>e.getDate(),WWWW:(e,t)=>t.WWWW[e.getDay()],WWW:(e,t)=>t.WWW[e.getDay()],HH:e=>uc(e.getHours()),H:e=>e.getHours(),h:e=>{let t=e.getHours();return 0==t?12:t>12?t-12:t},AA:e=>e.getHours()>=12?"PM":"AM",aa:e=>e.getHours()>=12?"pm":"am",a:e=>e.getHours()>=12?"p":"a",mm:e=>uc(e.getMinutes()),m:e=>e.getMinutes(),ss:e=>uc(e.getSeconds()),s:e=>e.getSeconds(),fff:e=>{return((t=e.getMilliseconds())<10?"00":t<100?"0":"")+t;var t}};function hc(e,t){t=t||cc;let n,r=[],a=/\{([a-z]+)\}|[^{]+/gi;for(;n=a.exec(e);)r.push("{"==n[0][0]?dc[n[1]]:n[0]);return e=>{let n="";for(let a=0;ae%1==0,fc=[1,2,2.5,5],vc=Bs(10,-16,0,fc),gc=Bs(10,0,16,fc),yc=gc.filter(pc),_c=vc.concat(gc),bc="{YYYY}",wc="\n"+bc,kc="{M}/{D}",xc="\n"+kc,Sc=xc+"/{YY}",Cc="{aa}",Ec="{h}:{mm}"+Cc,Nc="\n"+Ec,Ac=":{ss}",Mc=null;function Tc(e){let t=1e3*e,n=60*t,r=60*n,a=24*r,i=30*a,o=365*a;return[(1==e?Bs(10,0,3,fc).filter(pc):Bs(10,-3,0,fc)).concat([t,5*t,10*t,15*t,30*t,n,5*n,10*n,15*n,30*n,r,2*r,3*r,4*r,6*r,8*r,12*r,a,2*a,3*a,4*a,5*a,6*a,7*a,8*a,9*a,10*a,15*a,i,2*i,3*i,4*i,6*i,o,2*o,5*o,10*o,25*o,50*o,100*o]),[[o,bc,Mc,Mc,Mc,Mc,Mc,Mc,1],[28*a,"{MMM}",wc,Mc,Mc,Mc,Mc,Mc,1],[a,kc,wc,Mc,Mc,Mc,Mc,Mc,1],[r,"{h}"+Cc,Sc,Mc,xc,Mc,Mc,Mc,1],[n,Ec,Sc,Mc,xc,Mc,Mc,Mc,1],[t,Ac,Sc+" "+Ec,Mc,xc+" "+Ec,Mc,Nc,Mc,1],[e,Ac+".{fff}",Sc+" "+Ec,Mc,xc+" "+Ec,Mc,Nc,Mc,1]],function(t){return(l,s,c,u,d,h)=>{let m=[],p=d>=o,f=d>=i&&d=a?a:d,o=_+(ys(c)-ys(g))+js(g-_,i);m.push(o);let p=t(o),f=p.getHours()+p.getMinutes()/n+p.getSeconds()/r,v=d/r,y=h/l.axes[s]._space;for(;o=Vs(o+d,1==e?0:3),!(o>u);)if(v>1){let e=ys(Vs(f+v,6))%24,n=t(o).getHours()-e;n>1&&(n=-1),o-=n*r,f=(f+v)%24,Vs((o-m[m.length-1])/d,3)*y>=.7&&m.push(o)}else m.push(o)}return m}}]}const[Lc,Pc,Oc]=Tc(1),[Ic,Rc,Dc]=Tc(.001);function zc(e,t){return e.map((e=>e.map(((n,r)=>0==r||8==r||null==n?n:t(1==r||0==e[8]?n:e[1]+n)))))}function Fc(e,t){return(n,r,a,i,o)=>{let l,s,c,u,d,h,m=t.find((e=>o>=e[0]))||t[t.length-1];return r.map((t=>{let n=e(t),r=n.getFullYear(),a=n.getMonth(),i=n.getDate(),o=n.getHours(),p=n.getMinutes(),f=n.getSeconds(),v=r!=l&&m[2]||a!=s&&m[3]||i!=c&&m[4]||o!=u&&m[5]||p!=d&&m[6]||f!=h&&m[7]||m[1];return l=r,s=a,c=i,u=o,d=p,h=f,v(n)}))}}function jc(e,t,n){return new Date(e,t,n)}function Hc(e,t){return t(e)}Bs(2,-53,53,[1]);function Vc(e,t){return(n,r,a,i)=>null==i?Il:t(e(r))}const $c={show:!0,live:!0,isolate:!1,mount:()=>{},markers:{show:!0,width:2,stroke:function(e,t){let 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:[]};const Uc=[0,0];function Bc(e,t,n){let r=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];return e=>{0==e.button&&(!r||e.target==t)&&n(e)}}function qc(e,t,n){let r=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];return e=>{(!r||e.target==t)&&n(e)}}const Yc={show:!0,x:!0,y:!0,lock:!1,move:function(e,t,n){return Uc[0]=t,Uc[1]=n,Uc},points:{show:function(e,t){let n=e.cursor.points,r=ql(),a=n.size(e,t);Ul(r,yl,a),Ul(r,_l,a);let i=a/-2;Ul(r,"marginLeft",i),Ul(r,"marginTop",i);let o=n.width(e,t,a);return o&&Ul(r,"borderWidth",o),r},size:function(e,t){return e.series[t].points.size},width:0,stroke:function(e,t){let n=e.series[t].points;return n._stroke||n._fill},fill:function(e,t){let n=e.series[t].points;return n._fill||n._stroke}},bind:{mousedown:Bc,mouseup:Bc,click:Bc,dblclick:Bc,mousemove:qc,mouseleave:qc,mouseenter:qc},drag:{setScale:!0,x:!0,y:!1,dist:0,uni:null,click:(e,t)=>{t.stopPropagation(),t.stopImmediatePropagation()},_x:!1,_y:!1},focus:{dist:(e,t,n,r,a)=>r-a,prox:-1,bias:0},hover:{skip:[void 0],prox:null,bias:0},left:-10,top:-10,idx:null,dataIdx:null,idxs:null,event:null},Wc={show:!0,stroke:"rgba(0,0,0,0.07)",width:2},Kc=tc({},Wc,{filter:Os}),Qc=tc({},Kc,{size:10}),Zc=tc({},Wc,{show:!1}),Gc='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"',Jc="bold "+Gc,Xc={show:!0,scale:"x",stroke:Sl,space:50,gap:5,size:50,labelGap:0,labelSize:30,labelFont:Jc,side:2,grid:Kc,ticks:Qc,border:Zc,font:Gc,lineGap:1.5,rotate:0},eu={show:!0,scale:"x",auto:!1,sorted:1,min:As,max:-As,idxs:[]};function tu(e,t,n,r,a){return t.map((e=>null==e?"":ps(e)))}function nu(e,t,n,r,a,i,o){let l=[],s=$s.get(a)||0;for(let c=n=o?n:Vs(js(n,a),s);c<=r;c=Vs(c+a,s))l.push(Object.is(c,-0)?0:c);return l}function ru(e,t,n,r,a,i,o){const l=[],s=e.scales[e.axes[t].scale].log,c=ys((10==s?Cs:Es)(n));a=xs(s,c),10==s&&c<0&&(a=Vs(a,-c));let u=n;do{l.push(u),u+=a,10==s&&(u=Vs(u,$s.get(a))),u>=a*s&&(a=u)}while(u<=r);return l}function au(e,t,n,r,a,i,o){let l=e.scales[e.axes[t].scale].asinh,s=r>l?ru(e,t,ks(l,n),r,a):[l],c=r>=0&&n<=0?[0]:[];return(n<-l?ru(e,t,ks(l,-r),-n,a):[l]).reverse().map((e=>-e)).concat(c,s)}const iu=/./,ou=/[12357]/,lu=/[125]/,su=/1/,cu=(e,t,n,r)=>e.map(((e,a)=>4==t&&0==e||a%r==0&&n.test(e.toExponential()[e<0?1:0])?e:null));function uu(e,t,n,r,a){let i=e.axes[n],o=i.scale,l=e.scales[o],s=e.valToPos,c=i._space,u=s(10,o),d=s(9,o)-u>=c?iu:s(7,o)-u>=c?ou:s(5,o)-u>=c?lu:su;if(d==su){let e=gs(s(1,o)-u);if(ea,vu={show:!0,auto:!0,sorted:0,gaps:fu,alpha:1,facets:[tc({},pu,{scale:"x"}),tc({},pu,{scale:"y"})]},gu={scale:"y",auto:!0,sorted:0,show:!0,spanGaps:!1,gaps:fu,alpha:1,points:{show:function(e,t){let{scale:n,idxs:r}=e.series[0],a=e._data[0],i=e.valToPos(a[r[0]],n,!0),o=e.valToPos(a[r[1]],n,!0),l=gs(o-i)/(e.series[t].points.space*jl);return r[1]-r[0]<=l},filter:null},values:null,min:As,max:-As,idxs:[],path:null,clip:null};function yu(e,t,n,r,a){return n/10}const _u={time:!0,auto:!0,distr:1,log:10,asinh:1,min:null,max:null,dir:1,ori:0},bu=tc({},_u,{time:!1,ori:1}),wu={};function ku(e,t){let n=wu[e];return n||(n={key:e,plots:[],sub(e){n.plots.push(e)},unsub(e){n.plots=n.plots.filter((t=>t!=e))},pub(e,t,r,a,i,o,l){for(let s=0;s{let f=e.pxRound;const v=l.dir*(0==l.ori?1:-1),g=0==l.ori?Iu:Ru;let y,_;1==v?(y=n,_=r):(y=r,_=n);let b=f(c(t[y],l,m,d)),w=f(u(o[y],s,p,h)),k=f(c(t[_],l,m,d)),x=f(u(1==i?s.max:s.min,s,p,h)),S=new Path2D(a);return g(S,k,x),g(S,b,x),g(S,b,w),S}))}function Au(e,t,n,r,a,i){let o=null;if(e.length>0){o=new Path2D;const l=0==t?Du:zu;let s=n;for(let t=0;tn[0]){let e=n[0]-s;e>0&&l(o,s,r,e,r+i),s=n[1]}}let c=n+a-s,u=10;c>0&&l(o,s,r-u/2,c,r+i+u)}return o}function Mu(e,t,n,r,a,i,o){let l=[],s=e.length;for(let c=1==a?n:r;c>=n&&c<=r;c+=a){if(null===t[c]){let u=c,d=c;if(1==a)for(;++c<=r&&null===t[c];)d=c;else for(;--c>=n&&null===t[c];)d=c;let h=i(e[u]),m=d==u?h:i(e[d]),p=u-a;h=o<=0&&p>=0&&p=0&&f>=0&&f=h&&l.push([h,m])}}return l}function Tu(e){return 0==e?Ps:1==e?_s:t=>Fs(t,e)}function Lu(e){let t=0==e?Pu:Ou,n=0==e?(e,t,n,r,a,i)=>{e.arcTo(t,n,r,a,i)}:(e,t,n,r,a,i)=>{e.arcTo(n,t,a,r,i)},r=0==e?(e,t,n,r,a)=>{e.rect(t,n,r,a)}:(e,t,n,r,a)=>{e.rect(n,t,a,r)};return function(e,a,i,o,l){let s=arguments.length>5&&void 0!==arguments[5]?arguments[5]:0,c=arguments.length>6&&void 0!==arguments[6]?arguments[6]:0;0==s&&0==c?r(e,a,i,o,l):(s=ws(s,o/2,l/2),c=ws(c,o/2,l/2),t(e,a+s,i),n(e,a+o,i,a+o,i+l,s),n(e,a+o,i+l,a,i+l,c),n(e,a,i+l,a,i,c),n(e,a,i,a+o,i,s),e.closePath())}}const Pu=(e,t,n)=>{e.moveTo(t,n)},Ou=(e,t,n)=>{e.moveTo(n,t)},Iu=(e,t,n)=>{e.lineTo(t,n)},Ru=(e,t,n)=>{e.lineTo(n,t)},Du=Lu(0),zu=Lu(1),Fu=(e,t,n,r,a,i)=>{e.arc(t,n,r,a,i)},ju=(e,t,n,r,a,i)=>{e.arc(n,t,r,a,i)},Hu=(e,t,n,r,a,i,o)=>{e.bezierCurveTo(t,n,r,a,i,o)},Vu=(e,t,n,r,a,i,o)=>{e.bezierCurveTo(n,t,a,r,o,i)};function $u(e){return(e,t,n,r,a)=>Su(e,t,((t,i,o,l,s,c,u,d,h,m,p)=>{let f,v,{pxRound:g,points:y}=t;0==l.ori?(f=Pu,v=Fu):(f=Ou,v=ju);const _=Vs(y.width*jl,3);let b=(y.size-y.width)/2*jl,w=Vs(2*b,3),k=new Path2D,x=new Path2D,{left:S,top:C,width:E,height:N}=e.bbox;Du(x,S-w,C-w,E+2*w,N+2*w);const A=e=>{if(null!=o[e]){let t=g(c(i[e],l,m,d)),n=g(u(o[e],s,p,h));f(k,t+b,n),v(k,t,n,b,0,2*vs)}};if(a)a.forEach(A);else for(let e=n;e<=r;e++)A(e);return{stroke:_>0?k:null,fill:k,clip:x,flags:2|xu}}))}function Uu(e){return(t,n,r,a,i,o)=>{r!=a&&(i!=r&&o!=r&&e(t,n,r),i!=a&&o!=a&&e(t,n,a),e(t,n,o))}}const Bu=Uu(Iu),qu=Uu(Ru);function Yu(e){const t=ds(null===e||void 0===e?void 0:e.alignGaps,0);return(e,n,r,a)=>Su(e,n,((i,o,l,s,c,u,d,h,m,p,f)=>{let v,g,y=i.pxRound,_=e=>y(u(e,s,p,h)),b=e=>y(d(e,c,f,m));0==s.ori?(v=Iu,g=Bu):(v=Ru,g=qu);const w=s.dir*(0==s.ori?1:-1),k={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:xu},x=k.stroke;let S,C,E,N=As,A=-As,M=_(o[1==w?r:a]),T=rs(l,r,a,1*w),L=rs(l,r,a,-1*w),P=_(o[T]),O=_(o[L]),I=!1;for(let e=1==w?r:a;e>=r&&e<=a;e+=w){let t=_(o[e]),n=l[e];t==M?null!=n?(C=b(n),N==As&&(v(x,t,C),S=C),N=ws(C,N),A=ks(C,A)):null===n&&(I=!0):(N!=As&&(g(x,M,N,A,S,C),E=M),null!=n?(C=b(n),v(x,t,C),N=A=S=C):(N=As,A=-As,null===n&&(I=!0)),M=t)}N!=As&&N!=A&&E!=M&&g(x,M,N,A,S,C);let[R,D]=Cu(e,n);if(null!=i.fill||0!=R){let t=k.fill=new Path2D(x),r=b(i.fillTo(e,n,i.min,i.max,R));v(t,O,r),v(t,P,r)}if(!i.spanGaps){let c=[];I&&c.push(...Mu(o,l,r,a,w,_,t)),k.gaps=c=i.gaps(e,n,r,a,c),k.clip=Au(c,s.ori,h,m,p,f)}return 0!=D&&(k.band=2==D?[Nu(e,n,r,a,x,-1),Nu(e,n,r,a,x,1)]:Nu(e,n,r,a,x,D)),k}))}function Wu(e,t,n,r,a,i){let o=arguments.length>6&&void 0!==arguments[6]?arguments[6]:As;if(e.length>1){let l=null;for(let s=0,c=1/0;s0!==r[e]>0?n[e]=0:(n[e]=3*(s[e-1]+s[e])/((2*s[e]+s[e-1])/r[e-1]+(s[e]+2*s[e-1])/r[e]),isFinite(n[e])||(n[e]=0));n[o-1]=r[o-2];for(let c=0;c{dd.pxRatio=jl})));const Gu=Yu(),Ju=$u();function Xu(e,t,n,r){return(r?[e[0],e[1]].concat(e.slice(2)):[e[0]].concat(e.slice(1))).map(((e,r)=>ed(e,r,t,n)))}function ed(e,t,n,r){return tc({},0==t?n:r,e)}function td(e,t,n){return null==t?Ws:[t,n]}const nd=td;function rd(e,t,n){return null==t?Ws:us(t,n,os,!0)}function ad(e,t,n,r){return null==t?Ws:as(t,n,e.scales[r].log,!1)}const id=ad;function od(e,t,n,r){return null==t?Ws:is(t,n,e.scales[r].log,!1)}const ld=od;function sd(e,t,n,r,a){let i=ks(Ms(e),Ms(t)),o=t-e,l=ns(a/r*o,n);do{let e=n[l],t=r*e/o;if(t>=a&&i+(e<5?$s.get(e):0)<=17)return[e,t]}while(++l(t=_s((n=+r)*jl))+"px")),t,n]}function ud(e){e.show&&[e.font,e.labelFont].forEach((e=>{let t=Vs(e[2]*jl,1);e[0]=e[0].replace(/[0-9.]+px/,t+"px"),e[1]=t}))}function dd(e,t,n){const r={mode:ds(e.mode,1)},a=r.mode;function i(e,t){return((3==t.distr?Cs(e>0?e:t.clamp(r,e,t.min,t.max,t.key)):4==t.distr?Ns(e,t.asinh):e)-t._min)/(t._max-t._min)}function o(e,t,n,r){let a=i(e,t);return r+n*(-1==t.dir?1-a:a)}function l(e,t,n,r){let a=i(e,t);return r+n*(-1==t.dir?a:1-a)}function s(e,t,n,r){return 0==t.ori?o(e,t,n,r):l(e,t,n,r)}r.valToPosH=o,r.valToPosV=l;let c=!1;r.status=0;const u=r.root=ql("uplot");if(null!=e.id&&(u.id=e.id),Vl(u,e.class),e.title){ql("u-title",u).textContent=e.title}const d=Bl("canvas"),h=r.ctx=d.getContext("2d"),m=ql("u-wrap",u);es("click",m,(e=>{if(e.target===f){(Mt!=Ct||Tt!=Et)&&jt.click(r,e)}}),!0);const p=r.under=ql("u-under",m);m.appendChild(d);const f=r.over=ql("u-over",m),v=+ds((e=ec(e)).pxAlign,1),g=Tu(v);(e.plugins||[]).forEach((t=>{t.opts&&(e=t.opts(r,e)||e)}));const y=e.ms||.001,_=r.series=1==a?Xu(e.series||[],eu,gu,!1):(b=e.series||[null],w=vu,b.map(((e,t)=>0==t?null:tc({},w,e))));var b,w;const k=r.axes=Xu(e.axes||[],Xc,mu,!0),x=r.scales={},S=r.bands=e.bands||[];S.forEach((e=>{e.fill=Ls(e.fill||null),e.dir=ds(e.dir,-1)}));const C=2==a?_[1].facets[0].scale:_[0].scale,E={axes:function(){for(let e=0;ett[e])):y,b=2==m.distr?tt[y[1]]-tt[y[0]]:u,w=t.ticks,S=t.border,C=w.show?_s(w.size*jl):0,E=t._rotate*-vs/180,N=g(t._pos*jl),A=N+(C+v)*c;a=0==o?A:0,n=1==o?A:0,ot(t.font[0],l,1==t.align?kl:2==t.align?xl:E>0?kl:E<0?xl:0==o?"center":3==i?xl:kl,E||1==o?"middle":2==i?bl:wl);let M=t.font[1]*t.lineGap,T=y.map((e=>g(s(e,m,p,f)))),L=t._values;for(let e=0;e0&&(_.forEach(((e,n)=>{if(n>0&&e.show&&(ct(n,!1),ct(n,!0),null==e._paths)){et!=e.alpha&&(h.globalAlpha=et=e.alpha);let i=2==a?[0,t[n][0].length-1]:function(e){let t=Ts($e-1,0,Ve-1),n=Ts(Ue+1,0,Ve-1);for(;null==e[t]&&t>0;)t--;for(;null==e[n]&&n{if(t>0&&e.show){et!=e.alpha&&(h.globalAlpha=et=e.alpha),null!=e._paths&&ut(t,!1);{let n=null!=e._paths?e._paths.gaps:null,a=e.points.show(r,t,$e,Ue,n),i=e.points.filter(r,t,a,n);(a||i)&&(e.points._paths=e.points.paths(r,t,$e,Ue,i),ut(t,!0))}1!=et&&(h.globalAlpha=et=1),kn("drawSeries",t)}})))}},N=(e.drawOrder||["axes","series"]).map((e=>E[e]));function A(t){let n=x[t];if(null==n){let r=(e.scales||qs)[t]||qs;if(null!=r.from)A(r.from),x[t]=tc({},x[r.from],r,{key:t});else{n=x[t]=tc({},t==C?_u:bu,r),n.key=t;let e=n.time,i=n.range,o=Ks(i);if((t!=C||2==a&&!e)&&(!o||null!=i[0]&&null!=i[1]||(i={min:null==i[0]?ls:{mode:1,hard:i[0],soft:i[0]},max:null==i[1]?ls:{mode:1,hard:i[1],soft:i[1]}},o=!1),!o&&Gs(i))){let e=i;i=(t,n,r)=>null==n?Ws:us(n,r,e)}n.range=Ls(i||(e?nd:t==C?3==n.distr?id:4==n.distr?ld:td:3==n.distr?ad:4==n.distr?od:rd)),n.auto=Ls(!o&&n.auto),n.clamp=Ls(n.clamp||yu),n._min=n._max=null}}}A("x"),A("y"),1==a&&_.forEach((e=>{A(e.scale)})),k.forEach((e=>{A(e.scale)}));for(let Mn in e.scales)A(Mn);const M=x[C],T=M.distr;let L,P;0==M.ori?(Vl(u,"u-hz"),L=o,P=l):(Vl(u,"u-vt"),L=l,P=o);const O={};for(let Mn in x){let e=x[Mn];null==e.min&&null==e.max||(O[Mn]={min:e.min,max:e.max},e.min=e.max=null)}const I=e.tzDate||(e=>new Date(_s(e/y))),R=e.fmtDate||hc,D=1==y?Oc(I):Dc(I),z=Fc(I,zc(1==y?Pc:Rc,R)),F=Vc(I,Hc("{YYYY}-{MM}-{DD} {h}:{mm}{aa}",R)),j=[],H=r.legend=tc({},$c,e.legend),V=H.show,$=H.markers;let U,B,q;H.idxs=j,$.width=Ls($.width),$.dash=Ls($.dash),$.stroke=Ls($.stroke),$.fill=Ls($.fill);let Y,W=[],K=[],Q=!1,Z={};if(H.live){const e=_[1]?_[1].values:null;Q=null!=e,Y=Q?e(r,1,0):{_:0};for(let t in Y)Z[t]=Il}if(V)if(U=Bl("table","u-legend",u),q=Bl("tbody",null,U),H.mount(r,U),Q){B=Bl("thead",null,U,q);let e=Bl("tr",null,B);for(var G in Bl("th",null,e),Y)Bl("th",gl,e).textContent=G}else Vl(U,"u-inline"),H.live&&Vl(U,"u-live");const J={show:!0},X={show:!1};const ee=new Map;function te(e,t,n){let a=!(arguments.length>3&&void 0!==arguments[3])||arguments[3];const i=ee.get(t)||{},o=Ee.bind[e](r,t,n,a);o&&(es(e,t,i[e]=o),ee.set(t,i))}function ne(e,t,n){const r=ee.get(t)||{};for(let a in r)null!=e&&a!=e||(ts(a,t,r[a]),delete r[a]);null==e&&ee.delete(t)}let re=0,ae=0,ie=0,oe=0,le=0,se=0,ce=le,ue=se,de=ie,he=oe,me=0,pe=0,fe=0,ve=0;r.bbox={};let ge=!1,ye=!1,_e=!1,be=!1,we=!1,ke=!1;function xe(e,t,n){(n||e!=r.width||t!=r.height)&&Se(e,t),yt(!1),_e=!0,ye=!0,Rt()}function Se(e,t){r.width=re=ie=e,r.height=ae=oe=t,le=se=0,function(){let e=!1,t=!1,n=!1,r=!1;k.forEach(((a,i)=>{if(a.show&&a._show){let{side:i,_size:o}=a,l=i%2,s=o+(null!=a.label?a.labelSize:0);s>0&&(l?(ie-=s,3==i?(le+=s,r=!0):n=!0):(oe-=s,0==i?(se+=s,e=!0):t=!0))}})),ze[0]=e,ze[1]=n,ze[2]=t,ze[3]=r,ie-=He[1]+He[3],le+=He[3],oe-=He[2]+He[0],se+=He[0]}(),function(){let e=le+ie,t=se+oe,n=le,r=se;function a(a,i){switch(a){case 1:return e+=i,e-i;case 2:return t+=i,t-i;case 3:return n-=i,n+i;case 0:return r-=i,r+i}}k.forEach(((e,t)=>{if(e.show&&e._show){let t=e.side;e._pos=a(t,e._size),null!=e.label&&(e._lpos=a(t,e.labelSize))}}))}();let n=r.bbox;me=n.left=Fs(le*jl,.5),pe=n.top=Fs(se*jl,.5),fe=n.width=Fs(ie*jl,.5),ve=n.height=Fs(oe*jl,.5)}const Ce=3;r.setSize=function(e){let{width:t,height:n}=e;xe(t,n)};const Ee=r.cursor=tc({},Yc,{drag:{y:2==a}},e.cursor);if(null==Ee.dataIdx){var Ne,Ae;let e=Ee.hover,n=e.skip=new Set(null!==(Ne=e.skip)&&void 0!==Ne?Ne:[]);n.add(void 0);let r=e.prox=Ls(e.prox),a=null!==(Ae=e.bias)&&void 0!==Ae?Ae:e.bias=0;Ee.dataIdx=(e,i,o,l)=>{var s;if(0==i)return o;let c=o,u=null!==(s=r(e,i,o,l))&&void 0!==s?s:As,d=u>=0&&u0;)n.has(f[e])||(t=e);if(0==a||1==a)for(e=o;null==r&&e++u&&(c=null)}return c}}const Me=e=>{Ee.event=e};Ee.idxs=j,Ee._lock=!1;let Te=Ee.points;Te.show=Ls(Te.show),Te.size=Ls(Te.size),Te.stroke=Ls(Te.stroke),Te.width=Ls(Te.width),Te.fill=Ls(Te.fill);const Le=r.focus=tc({},e.focus||{alpha:.3},Ee.focus),Pe=Le.prox>=0;let Oe=[null],Ie=[null],Re=[null];function De(e,t){if(1==a||t>0){let t=1==a&&x[e.scale].time,n=e.value;e.value=t?Zs(n)?Vc(I,Hc(n,R)):n||F:n||hu,e.label=e.label||(t?"Time":"Value")}if(t>0){e.width=null==e.width?1:e.width,e.paths=e.paths||Gu||Is,e.fillTo=Ls(e.fillTo||Eu),e.pxAlign=+ds(e.pxAlign,v),e.pxRound=Tu(e.pxAlign),e.stroke=Ls(e.stroke||null),e.fill=Ls(e.fill||null),e._stroke=e._fill=e._paths=e._focus=null;let t=Vs((3+2*(ks(1,e.width)||1))*1,3),n=e.points=tc({},{size:t,width:ks(1,.2*t),stroke:e.stroke,space:2*t,paths:Ju,_stroke:null,_fill:null},e.points);n.show=Ls(n.show),n.filter=Ls(n.filter),n.fill=Ls(n.fill),n.stroke=Ls(n.stroke),n.paths=Ls(n.paths),n.pxAlign=e.pxAlign}if(V){let n=function(e,t){if(0==t&&(Q||!H.live||2==a))return Ws;let n=[],i=Bl("tr","u-series",q,q.childNodes[t]);Vl(i,e.class),e.show||Vl(i,vl);let o=Bl("th",null,i);if($.show){let e=ql("u-marker",o);if(t>0){let n=$.width(r,t);n&&(e.style.border=n+"px "+$.dash(r,t)+" "+$.stroke(r,t)),e.style.background=$.fill(r,t)}}let l=ql(gl,o);for(var s in l.textContent=e.label,t>0&&($.show||(l.style.color=e.width>0?$.stroke(r,t):$.fill(r,t)),te("click",o,(t=>{if(Ee._lock)return;Me(t);let n=_.indexOf(e);if((t.ctrlKey||t.metaKey)!=H.isolate){let e=_.some(((e,t)=>t>0&&t!=n&&e.show));_.forEach(((t,r)=>{r>0&&Yt(r,e?r==n?J:X:J,!0,Sn.setSeries)}))}else Yt(n,{show:!e.show},!0,Sn.setSeries)}),!1),Pe&&te(Ml,o,(t=>{Ee._lock||(Me(t),Yt(_.indexOf(e),Zt,!0,Sn.setSeries))}),!1)),Y){let e=Bl("td","u-value",i);e.textContent="--",n.push(e)}return[i,n]}(e,t);W.splice(t,0,n[0]),K.splice(t,0,n[1]),H.values.push(null)}if(Ee.show){j.splice(t,0,null);let n=function(e,t){if(t>0){let n=Ee.points.show(r,t);if(n)return Vl(n,"u-cursor-pt"),Vl(n,e.class),Wl(n,-10,-10,ie,oe),f.insertBefore(n,Oe[t]),n}}(e,t);null!=n&&(Oe.splice(t,0,n),Ie.splice(t,0,0),Re.splice(t,0,0))}kn("addSeries",t)}r.addSeries=function(e,t){t=null==t?_.length:t,e=1==a?ed(e,t,eu,gu):ed(e,t,null,vu),_.splice(t,0,e),De(_[t],t)},r.delSeries=function(e){if(_.splice(e,1),V){H.values.splice(e,1),K.splice(e,1);let t=W.splice(e,1)[0];ne(null,t.firstChild),t.remove()}Ee.show&&(j.splice(e,1),Oe.length>1&&(Oe.splice(e,1)[0].remove(),Ie.splice(e,1),Re.splice(e,1))),kn("delSeries",e)};const ze=[!1,!1,!1,!1];function Fe(e,t,n,r){let[a,i,o,l]=n,s=t%2,c=0;return 0==s&&(l||i)&&(c=0==t&&!a||2==t&&!o?_s(Xc.size/3):0),1==s&&(a||o)&&(c=1==t&&!i||3==t&&!l?_s(mu.size/2):0),c}const je=r.padding=(e.padding||[Fe,Fe,Fe,Fe]).map((e=>Ls(ds(e,Fe)))),He=r._padding=je.map(((e,t)=>e(r,t,ze,0)));let Ve,$e=null,Ue=null;const Be=1==a?_[0].idxs:null;let qe,Ye,We,Ke,Qe,Ze,Ge,Je,Xe,et,tt=null,nt=!1;function rt(e,n){if(t=null==e?[]:e,r.data=r._data=t,2==a){Ve=0;for(let e=1;e<_.length;e++)Ve+=t[e][0].length}else{0==t.length&&(r.data=r._data=t=[[]]),tt=t[0],Ve=tt.length;let e=t;if(2==T){e=t.slice();let n=e[0]=Array(Ve);for(let e=0;e=0,ke=!0,Rt()}}function at(){let e,n;nt=!0,1==a&&(Ve>0?($e=Be[0]=0,Ue=Be[1]=Ve-1,e=t[0][$e],n=t[0][Ue],2==T?(e=$e,n=Ue):e==n&&(3==T?[e,n]=as(e,e,M.log,!1):4==T?[e,n]=is(e,e,M.log,!1):M.time?n=e+_s(86400/y):[e,n]=us(e,n,os,!0))):($e=Be[0]=e=null,Ue=Be[1]=n=null)),qt(C,e,n)}function it(e,t,n,r,a,i){var o,l,s,c,u;null!==(o=e)&&void 0!==o||(e=Cl),null!==(l=n)&&void 0!==l||(n=Ys),null!==(s=r)&&void 0!==s||(r="butt"),null!==(c=a)&&void 0!==c||(a=Cl),null!==(u=i)&&void 0!==u||(i="round"),e!=qe&&(h.strokeStyle=qe=e),a!=Ye&&(h.fillStyle=Ye=a),t!=We&&(h.lineWidth=We=t),i!=Qe&&(h.lineJoin=Qe=i),r!=Ze&&(h.lineCap=Ze=r),n!=Ke&&h.setLineDash(Ke=n)}function ot(e,t,n,r){t!=Ye&&(h.fillStyle=Ye=t),e!=Ge&&(h.font=Ge=e),n!=Je&&(h.textAlign=Je=n),r!=Xe&&(h.textBaseline=Xe=r)}function lt(e,t,n,a){let i=arguments.length>4&&void 0!==arguments[4]?arguments[4]:0;if(a.length>0&&e.auto(r,nt)&&(null==t||null==t.min)){let t=ds($e,0),r=ds(Ue,a.length-1),o=null==n.min?3==e.distr?function(e,t,n){let r=As,a=-As;for(let i=t;i<=n;i++){let t=e[i];null!=t&&t>0&&(ta&&(a=t))}return[r,a]}(a,t,r):function(e,t,n,r){let a=As,i=-As;if(1==r)a=e[t],i=e[n];else if(-1==r)a=e[n],i=e[t];else for(let o=t;o<=n;o++){let t=e[o];null!=t&&(ti&&(i=t))}return[a,i]}(a,t,r,i):[n.min,n.max];e.min=ws(e.min,n.min=o[0]),e.max=ks(e.max,n.max=o[1])}}r.setData=rt;const st={min:null,max:null};function ct(e,t){let n=t?_[e].points:_[e];n._stroke=n.stroke(r,e),n._fill=n.fill(r,e)}function ut(e,n){let a=n?_[e].points:_[e],{stroke:i,fill:o,clip:l,flags:s,_stroke:c=a._stroke,_fill:u=a._fill,_width:d=a.width}=a._paths;d=Vs(d*jl,3);let m=null,p=d%2/2;n&&null==u&&(u=d>0?"#fff":c);let f=1==a.pxAlign&&p>0;if(f&&h.translate(p,p),!n){let e=me-d/2,t=pe-d/2,n=fe+d,r=ve+d;m=new Path2D,m.rect(e,t,n,r)}n?ht(c,d,a.dash,a.cap,u,i,o,s,l):function(e,n,a,i,o,l,s,c,u,d,h){let m=!1;0!=u&&S.forEach(((p,f)=>{if(p.series[0]==e){let e,v=_[p.series[1]],g=t[p.series[1]],y=(v._paths||qs).band;Ks(y)&&(y=1==p.dir?y[0]:y[1]);let b=null;v.show&&y&&function(e,t,n){for(t=ds(t,0),n=ds(n,e.length-1);t<=n;){if(null!=e[t])return!0;t++}return!1}(g,$e,Ue)?(b=p.fill(r,f)||l,e=v._paths.clip):y=null,ht(n,a,i,o,b,s,c,u,d,h,e,y),m=!0}})),m||ht(n,a,i,o,l,s,c,u,d,h)}(e,c,d,a.dash,a.cap,u,i,o,s,m,l),f&&h.translate(-p,-p)}const dt=2|xu;function ht(e,t,n,r,a,i,o,l,s,c,u,d){it(e,t,n,r,a),(s||c||d)&&(h.save(),s&&h.clip(s),c&&h.clip(c)),d?(l&dt)==dt?(h.clip(d),u&&h.clip(u),pt(a,o),mt(e,i,t)):2&l?(pt(a,o),h.clip(d),mt(e,i,t)):l&xu&&(h.save(),h.clip(d),u&&h.clip(u),pt(a,o),h.restore(),mt(e,i,t)):(pt(a,o),mt(e,i,t)),(s||c||d)&&h.restore()}function mt(e,t,n){n>0&&(t instanceof Map?t.forEach(((e,t)=>{h.strokeStyle=qe=t,h.stroke(e)})):null!=t&&e&&h.stroke(t))}function pt(e,t){t instanceof Map?t.forEach(((e,t)=>{h.fillStyle=Ye=t,h.fill(e)})):null!=t&&e&&h.fill(t)}function ft(e,t,n,r,a,i,o,l,s,c){let u=o%2/2;1==v&&h.translate(u,u),it(l,o,s,c,l),h.beginPath();let d,m,p,f,g=a+(0==r||3==r?-i:i);0==n?(m=a,f=g):(d=a,p=g);for(let v=0;v{if(!n.show)return;let i=x[n.scale];if(null==i.min)return void(n._show&&(t=!1,n._show=!1,yt(!1)));n._show||(t=!1,n._show=!0,yt(!1));let o=n.side,l=o%2,{min:s,max:c}=i,[u,d]=function(e,t,n,a){let i,o=k[e];if(a<=0)i=[0,0];else{let l=o._space=o.space(r,e,t,n,a);i=sd(t,n,o._incrs=o.incrs(r,e,t,n,a,l),a,l)}return o._found=i}(a,s,c,0==l?ie:oe);if(0==d)return;let h=2==i.distr,m=n._splits=n.splits(r,a,s,c,u,d,h),p=2==i.distr?m.map((e=>tt[e])):m,f=2==i.distr?tt[m[1]]-tt[m[0]]:u,v=n._values=n.values(r,n.filter(r,p,a,d,f),a,d,f);n._rotate=2==o?n.rotate(r,v,a,d):0;let g=n._size;n._size=bs(n.size(r,v,a,e)),null!=g&&n._size!=g&&(t=!1)})),t}function gt(e){let t=!0;return je.forEach(((n,a)=>{let i=n(r,a,ze,e);i!=He[a]&&(t=!1),He[a]=i})),t}function yt(e){_.forEach(((t,n)=>{n>0&&(t._paths=null,e&&(1==a?(t.min=null,t.max=null):t.facets.forEach((e=>{e.min=null,e.max=null}))))}))}let _t,bt,wt,kt,xt,St,Ct,Et,Nt,At,Mt,Tt,Lt=!1,Pt=!1,Ot=[];function It(){Pt=!1;for(let e=0;e0){_.forEach(((n,i)=>{if(1==a){let a=n.scale,o=O[a];if(null==o)return;let l=e[a];if(0==i){let e=l.range(r,l.min,l.max,a);l.min=e[0],l.max=e[1],$e=ns(l.min,t[0]),Ue=ns(l.max,t[0]),Ue-$e>1&&(t[0][$e]l.max&&Ue--),n.min=tt[$e],n.max=tt[Ue]}else n.show&&n.auto&<(l,o,n,t[i],n.sorted);n.idxs[0]=$e,n.idxs[1]=Ue}else if(i>0&&n.show&&n.auto){let[r,a]=n.facets,o=r.scale,l=a.scale,[s,c]=t[i],u=e[o],d=e[l];null!=u&<(u,O[o],r,s,r.sorted),null!=d&<(d,O[l],a,c,a.sorted),n.min=a.min,n.max=a.max}}));for(let t in e){let n=e[t],a=O[t];if(null==n.from&&(null==a||null==a.min)){let e=n.range(r,n.min==As?null:n.min,n.max==-As?null:n.max,t);n.min=e[0],n.max=e[1]}}}for(let t in e){let n=e[t];if(null!=n.from){let a=e[n.from];if(null==a.min)n.min=n.max=null;else{let e=n.range(r,a.min,a.max,t);n.min=e[0],n.max=e[1]}}}let n={},i=!1;for(let t in e){let r=e[t],a=x[t];if(a.min!=r.min||a.max!=r.max){a.min=r.min,a.max=r.max;let e=a.distr;a._min=3==e?Cs(a.min):4==e?Ns(a.min,a.asinh):a.min,a._max=3==e?Cs(a.max):4==e?Ns(a.max,a.asinh):a.max,n[t]=i=!0}}if(i){_.forEach(((e,t)=>{2==a?t>0&&n.y&&(e._paths=null):n[e.scale]&&(e._paths=null)}));for(let e in n)_e=!0,kn("setScale",e);Ee.show&&Ee.left>=0&&(be=ke=!0)}for(let t in O)O[t]=null}(),ge=!1),_e&&(!function(){let e=!1,t=0;for(;!e;){t++;let n=vt(t),a=gt(t);e=t==Ce||n&&a,e||(Se(r.width,r.height),ye=!0)}}(),_e=!1),ye){if(Ul(p,kl,le),Ul(p,bl,se),Ul(p,yl,ie),Ul(p,_l,oe),Ul(f,kl,le),Ul(f,bl,se),Ul(f,yl,ie),Ul(f,_l,oe),Ul(m,yl,re),Ul(m,_l,ae),d.width=_s(re*jl),d.height=_s(ae*jl),k.forEach((e=>{let{_el:t,_show:n,_size:r,_pos:a,side:i}=e;if(null!=t)if(n){let e=i%2==1;Ul(t,e?"left":"top",a-(3===i||0===i?r:0)),Ul(t,e?"width":"height",r),Ul(t,e?"top":"left",e?se:le),Ul(t,e?"height":"width",e?oe:ie),$l(t,vl)}else Vl(t,vl)})),qe=Ye=We=Qe=Ze=Ge=Je=Xe=Ke=null,et=1,ln(!0),le!=ce||se!=ue||ie!=de||oe!=he){yt(!1);let e=ie/de,t=oe/he;if(Ee.show&&!be&&Ee.left>=0){Ee.left*=e,Ee.top*=t,wt&&Wl(wt,_s(Ee.left),0,ie,oe),kt&&Wl(kt,0,_s(Ee.top),ie,oe);for(let n=1;n=0&&$t.width>0){$t.left*=e,$t.width*=e,$t.top*=t,$t.height*=t;for(let e in un)Ul(Ut,e,$t[e])}ce=le,ue=se,de=ie,he=oe}kn("setSize"),ye=!1}re>0&&ae>0&&(h.clearRect(0,0,d.width,d.height),kn("drawClear"),N.forEach((e=>e())),kn("draw")),$t.show&&we&&(Bt($t),we=!1),Ee.show&&be&&(an(null,!0,!1),be=!1),H.show&&H.live&&ke&&(nn(),ke=!1),c||(c=!0,r.status=1,kn("ready")),nt=!1,Lt=!1}function zt(e,n){let a=x[e];if(null==a.from){if(0==Ve){let t=a.range(r,n.min,n.max,e);n.min=t[0],n.max=t[1]}if(n.min>n.max){let e=n.min;n.min=n.max,n.max=e}if(Ve>1&&null!=n.min&&null!=n.max&&n.max-n.min<1e-16)return;e==C&&2==a.distr&&Ve>0&&(n.min=ns(n.min,t[0]),n.max=ns(n.max,t[0]),n.min==n.max&&n.max++),O[e]=n,ge=!0,Rt()}}r.batch=function(e){let t=arguments.length>1&&void 0!==arguments[1]&&arguments[1];Lt=!0,Pt=t,e(r),Dt(),t&&Ot.length>0&&queueMicrotask(It)},r.redraw=(e,t)=>{_e=t||!1,!1!==e?qt(C,M.min,M.max):Rt()},r.setScale=zt;let Ft=!1;const jt=Ee.drag;let Ht=jt.x,Vt=jt.y;Ee.show&&(Ee.x&&(_t=ql("u-cursor-x",f)),Ee.y&&(bt=ql("u-cursor-y",f)),0==M.ori?(wt=_t,kt=bt):(wt=bt,kt=_t),Mt=Ee.left,Tt=Ee.top);const $t=r.select=tc({show:!0,over:!0,left:0,width:0,top:0,height:0},e.select),Ut=$t.show?ql("u-select",$t.over?f:p):null;function Bt(e,t){if($t.show){for(let t in e)$t[t]=e[t],t in un&&Ul(Ut,t,e[t]);!1!==t&&kn("setSelect")}}function qt(e,t,n){zt(e,{min:t,max:n})}function Yt(e,t,n,i){null!=t.focus&&function(e){if(e!=Qt){let t=null==e,n=1!=Le.alpha;_.forEach(((r,i)=>{if(1==a||i>0){let a=t||0==i||i==e;r._focus=t?null:a,n&&function(e,t){_[e].alpha=t,Ee.show&&Oe[e]&&(Oe[e].style.opacity=t);V&&W[e]&&(W[e].style.opacity=t)}(i,a?1:Le.alpha)}})),Qt=e,n&&Rt()}}(e),null!=t.show&&_.forEach(((n,r)=>{r>0&&(e==r||null==e)&&(n.show=t.show,function(e,t){let n=_[e],r=V?W[e]:null;n.show?r&&$l(r,vl):(r&&Vl(r,vl),Oe.length>1&&Wl(Oe[e],-10,-10,ie,oe))}(r,t.show),2==a?(qt(n.facets[0].scale,null,null),qt(n.facets[1].scale,null,null)):qt(n.scale,null,null),Rt())})),!1!==n&&kn("setSeries",e,t),i&&Nn("setSeries",r,e,t)}let Wt,Kt,Qt;r.setSelect=Bt,r.setSeries=Yt,r.addBand=function(e,t){e.fill=Ls(e.fill||null),e.dir=ds(e.dir,-1),t=null==t?S.length:t,S.splice(t,0,e)},r.setBand=function(e,t){tc(S[e],t)},r.delBand=function(e){null==e?S.length=0:S.splice(e,1)};const Zt={focus:!0};function Gt(e,t,n){let r=x[t];n&&(e=e/jl-(1==r.ori?se:le));let a=ie;1==r.ori&&(a=oe,e=a-e),-1==r.dir&&(e=a-e);let i=r._min,o=i+(r._max-i)*(e/a),l=r.distr;return 3==l?xs(10,o):4==l?function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:1;return fs.sinh(e)*t}(o,r.asinh):o}function Jt(e,t){Ul(Ut,kl,$t.left=e),Ul(Ut,yl,$t.width=t)}function Xt(e,t){Ul(Ut,bl,$t.top=e),Ul(Ut,_l,$t.height=t)}V&&Pe&&te(Tl,U,(e=>{Ee._lock||(Me(e),null!=Qt&&Yt(null,Zt,!0,Sn.setSeries))})),r.valToIdx=e=>ns(e,t[0]),r.posToIdx=function(e,n){return ns(Gt(e,C,n),t[0],$e,Ue)},r.posToVal=Gt,r.valToPos=(e,t,n)=>0==x[t].ori?o(e,x[t],n?fe:ie,n?me:0):l(e,x[t],n?ve:oe,n?pe:0),r.setCursor=(e,t,n)=>{Mt=e.left,Tt=e.top,an(null,t,n)};let en=0==M.ori?Jt:Xt,tn=1==M.ori?Jt:Xt;function nn(e,t){null!=e&&(e.idxs?e.idxs.forEach(((e,t)=>{j[t]=e})):void 0!==e.idx&&j.fill(e.idx),H.idx=j[0]);for(let n=0;n<_.length;n++)(n>0||1==a&&!Q)&&rn(n,j[n]);V&&H.live&&function(){if(V&&H.live)for(let e=2==a?1:0;e<_.length;e++){if(0==e&&Q)continue;let t=H.values[e],n=0;for(let r in t)K[e][n++].firstChild.nodeValue=t[r]}}(),ke=!1,!1!==t&&kn("setLegend")}function rn(e,n){var a;let i,o=_[e],l=0==e&&2==T?tt:t[e];Q?i=null!==(a=o.values(r,e,n))&&void 0!==a?a:Z:(i=o.value(r,null==n?null:l[n],e,n),i=null==i?Z:{_:i}),H.values[e]=i}function an(e,n,i){let o;Nt=Mt,At=Tt,[Mt,Tt]=Ee.move(r,Mt,Tt),Ee.left=Mt,Ee.top=Tt,Ee.show&&(wt&&Wl(wt,_s(Mt),0,ie,oe),kt&&Wl(kt,0,_s(Tt),ie,oe));let l=$e>Ue;Wt=As;let s=0==M.ori?ie:oe,c=1==M.ori?ie:oe;if(Mt<0||0==Ve||l){o=Ee.idx=null;for(let e=0;e<_.length;e++)e>0&&Oe.length>1&&Wl(Oe[e],-10,-10,ie,oe);Pe&&Yt(null,Zt,!0,null==e&&Sn.setSeries),H.live&&(j.fill(o),ke=!0)}else{let e,n,i;1==a&&(e=0==M.ori?Mt:Tt,n=Gt(e,C),o=Ee.idx=ns(n,t[0],$e,Ue),i=L(t[0][o],M,s,0));for(let l=2==a?1:0;l<_.length;l++){let e=_[l],u=j[l],d=null==u?null:1==a?t[l][u]:t[l][1][u],h=Ee.dataIdx(r,l,o,n),m=null==h?null:1==a?t[l][h]:t[l][1][h];ke=ke||m!=d||h!=u,j[l]=h;let p=h==o?i:L(1==a?t[0][h]:t[l][0][h],M,s,0);if(l>0&&e.show){let t,n,i=null==m?-10:P(m,1==a?x[e.scale]:x[e.facets[1].scale],c,0);if(Pe&&null!=m){let t=1==M.ori?Mt:Tt,n=gs(Le.dist(r,l,h,i,t));if(n=0?1:-1;i==(m>=0?1:-1)&&(1==i?1==r?m>=a:m<=a:1==r?m<=a:m>=a)&&(Wt=n,Kt=l)}else Wt=n,Kt=l}}if(0==M.ori?(t=p,n=i):(t=i,n=p),ke&&Oe.length>1){Ql(Oe[l],Ee.points.fill(r,l),Ee.points.stroke(r,l));let e,a,i,o,s=!0,c=Ee.points.bbox;if(null!=c){s=!1;let t=c(r,l);i=t.left,o=t.top,e=t.width,a=t.height}else i=t,o=n,e=a=Ee.points.size(r,l);Gl(Oe[l],e,a,s),Ie[l]=i,Re[l]=o,Wl(Oe[l],js(i,1),js(o,1),ie,oe)}}}}if($t.show&&Ft)if(null!=e){let[t,n]=Sn.scales,[r,a]=Sn.match,[i,o]=e.cursor.sync.scales,l=e.cursor.drag;if(Ht=l._x,Vt=l._y,Ht||Vt){let l,u,d,h,m,{left:p,top:f,width:v,height:g}=e.select,y=e.scales[t].ori,_=e.posToVal,b=null!=t&&r(t,i),w=null!=n&&a(n,o);b&&Ht?(0==y?(l=p,u=v):(l=f,u=g),d=x[t],h=L(_(l,i),d,s,0),m=L(_(l+u,i),d,s,0),en(ws(h,m),gs(m-h))):en(0,s),w&&Vt?(1==y?(l=p,u=v):(l=f,u=g),d=x[n],h=P(_(l,o),d,c,0),m=P(_(l+u,o),d,c,0),tn(ws(h,m),gs(m-h))):tn(0,c)}else dn()}else{let e=gs(Nt-xt),t=gs(At-St);if(1==M.ori){let n=e;e=t,t=n}Ht=jt.x&&e>=jt.dist,Vt=jt.y&&t>=jt.dist;let n,r,a=jt.uni;null!=a?Ht&&Vt&&(Ht=e>=a,Vt=t>=a,Ht||Vt||(t>e?Vt=!0:Ht=!0)):jt.x&&jt.y&&(Ht||Vt)&&(Ht=Vt=!0),Ht&&(0==M.ori?(n=Ct,r=Mt):(n=Et,r=Tt),en(ws(n,r),gs(r-n)),Vt||tn(0,c)),Vt&&(1==M.ori?(n=Ct,r=Mt):(n=Et,r=Tt),tn(ws(n,r),gs(r-n)),Ht||en(0,s)),Ht||Vt||(en(0,0),tn(0,0))}if(jt._x=Ht,jt._y=Vt,null==e){if(i){if(null!=Cn){let[e,t]=Sn.scales;Sn.values[0]=null!=e?Gt(0==M.ori?Mt:Tt,e):null,Sn.values[1]=null!=t?Gt(1==M.ori?Mt:Tt,t):null}Nn(El,r,Mt,Tt,ie,oe,o)}if(Pe){let e=i&&Sn.setSeries,t=Le.prox;null==Qt?Wt<=t&&Yt(Kt,Zt,!0,e):Wt>t?Yt(null,Zt,!0,e):Kt!=Qt&&Yt(Kt,Zt,!0,e)}}ke&&(H.idx=o,nn()),!1!==n&&kn("setCursor")}r.setLegend=nn;let on=null;function ln(){arguments.length>0&&void 0!==arguments[0]&&arguments[0]?on=null:(on=f.getBoundingClientRect(),kn("syncRect",on))}function sn(e,t,n,r,a,i,o){Ee._lock||Ft&&null!=e&&0==e.movementX&&0==e.movementY||(cn(e,t,n,r,a,i,o,!1,null!=e),null!=e?an(null,!0,!0):an(t,!0,!1))}function cn(e,t,n,a,i,o,l,c,u){if(null==on&&ln(!1),Me(e),null!=e)n=e.clientX-on.left,a=e.clientY-on.top;else{if(n<0||a<0)return Mt=-10,void(Tt=-10);let[e,r]=Sn.scales,l=t.cursor.sync,[c,u]=l.values,[d,h]=l.scales,[m,p]=Sn.match,f=t.axes[0].side%2==1,v=0==M.ori?ie:oe,g=1==M.ori?ie:oe,y=f?o:i,_=f?i:o,b=f?a:n,w=f?n:a;if(n=null!=d?m(e,d)?s(c,x[e],v,0):-10:v*(b/y),a=null!=h?p(r,h)?s(u,x[r],g,0):-10:g*(w/_),1==M.ori){let e=n;n=a,a=e}}u&&((n<=1||n>=ie-1)&&(n=Fs(n,ie)),(a<=1||a>=oe-1)&&(a=Fs(a,oe))),c?(xt=n,St=a,[Ct,Et]=Ee.move(r,n,a)):(Mt=n,Tt=a)}Object.defineProperty(r,"rect",{get:()=>(null==on&&ln(!1),on)});const un={width:0,height:0,left:0,top:0};function dn(){Bt(un,!1)}let hn,mn,pn,fn;function vn(e,t,n,a,i,o,l){Ft=!0,Ht=Vt=jt._x=jt._y=!1,cn(e,t,n,a,i,o,0,!0,!1),null!=e&&(te(Al,Dl,gn,!1),Nn(Nl,r,Ct,Et,ie,oe,null));let{left:s,top:c,width:u,height:d}=$t;hn=s,mn=c,pn=u,fn=d,dn()}function gn(e,t,n,a,i,o,l){Ft=jt._x=jt._y=!1,cn(e,t,n,a,i,o,0,!1,!0);let{left:s,top:c,width:u,height:d}=$t,h=u>0||d>0,m=hn!=s||mn!=c||pn!=u||fn!=d;if(h&&m&&Bt($t),jt.setScale&&h&&m){let e=s,t=u,n=c,r=d;if(1==M.ori&&(e=c,t=d,n=s,r=u),Ht&&qt(C,Gt(e,C),Gt(e+t,C)),Vt)for(let a in x){let e=x[a];a!=C&&null==e.from&&e.min!=As&&qt(a,Gt(n+r,a),Gt(n,a))}dn()}else Ee.lock&&(Ee._lock=!Ee._lock,Ee._lock||an(null,!0,!1));null!=e&&(ne(Al,Dl),Nn(Al,r,Mt,Tt,ie,oe,null))}function yn(e,t,n,a,i,o,l){Ee._lock||(Me(e),at(),dn(),null!=e&&Nn(Ll,r,Mt,Tt,ie,oe,null))}function _n(){k.forEach(ud),xe(r.width,r.height,!0)}es(Ol,zl,_n);const bn={};bn.mousedown=vn,bn.mousemove=sn,bn.mouseup=gn,bn.dblclick=yn,bn.setSeries=(e,t,n,a)=>{-1!=(n=(0,Sn.match[2])(r,t,n))&&Yt(n,a,!0,!1)},Ee.show&&(te(Nl,f,vn),te(El,f,sn),te(Ml,f,(e=>{Me(e),ln(!1)})),te(Tl,f,(function(e,t,n,r,a,i,o){if(Ee._lock)return;Me(e);let l=Ft;if(Ft){let e,t,n=!0,r=!0,a=10;0==M.ori?(e=Ht,t=Vt):(e=Vt,t=Ht),e&&t&&(n=Mt<=a||Mt>=ie-a,r=Tt<=a||Tt>=oe-a),e&&n&&(Mt=Mt{e.call(null,r,t,n)}))}(e.plugins||[]).forEach((e=>{for(let t in e.hooks)wn[t]=(wn[t]||[]).concat(e.hooks[t])}));const xn=(e,t,n)=>n,Sn=tc({key:null,setSeries:!1,filters:{pub:Rs,sub:Rs},scales:[C,_[1]?_[1].scale:null],match:[Ds,Ds,xn],values:[null,null]},Ee.sync);2==Sn.match.length&&Sn.match.push(xn),Ee.sync=Sn;const Cn=Sn.key,En=ku(Cn);function Nn(e,t,n,r,a,i,o){Sn.filters.pub(e,t,n,r,a,i,o)&&En.pub(e,t,n,r,a,i,o)}function An(){kn("init",e,t),rt(t||e.data,!1),O[C]?zt(C,O[C]):at(),we=$t.show&&($t.width>0||$t.height>0),be=ke=!0,xe(e.width,e.height)}return En.sub(r),r.pub=function(e,t,n,r,a,i,o){Sn.filters.sub(e,t,n,r,a,i,o)&&bn[e](null,t,n,r,a,i,o)},r.destroy=function(){var e;En.unsub(r),Qu.delete(r),ee.clear(),ts(Ol,zl,_n),u.remove(),null===(e=U)||void 0===e||e.remove(),kn("destroy")},_.forEach(De),k.forEach((function(e,t){if(e._show=e.show,e.show){let n=e.side%2,a=x[e.scale];null==a&&(e.scale=n?_[1].scale:C,a=x[e.scale]);let i=a.time;e.size=Ls(e.size),e.space=Ls(e.space),e.rotate=Ls(e.rotate),Ks(e.incrs)&&e.incrs.forEach((e=>{!$s.has(e)&&$s.set(e,Us(e))})),e.incrs=Ls(e.incrs||(2==a.distr?yc:i?1==y?Lc:Ic:_c)),e.splits=Ls(e.splits||(i&&1==a.distr?D:3==a.distr?ru:4==a.distr?au:nu)),e.stroke=Ls(e.stroke),e.grid.stroke=Ls(e.grid.stroke),e.ticks.stroke=Ls(e.ticks.stroke),e.border.stroke=Ls(e.border.stroke);let o=e.values;e.values=Ks(o)&&!Ks(o[0])?Ls(o):i?Ks(o)?Fc(I,zc(o,R)):Zs(o)?function(e,t){let n=hc(t);return(t,r,a,i,o)=>r.map((t=>n(e(t))))}(I,o):o||z:o||tu,e.filter=Ls(e.filter||(a.distr>=3&&10==a.log?uu:3==a.distr&&2==a.log?du:Os)),e.font=cd(e.font),e.labelFont=cd(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&&(ze[t]=!0,e._el=ql("u-axis",m))}})),n?n instanceof HTMLElement?(n.appendChild(u),An()):n(r,An):An(),r}dd.assign=tc,dd.fmtNum=ps,dd.rangeNum=us,dd.rangeLog=as,dd.rangeAsinh=is,dd.orient=Su,dd.pxRatio=jl,dd.join=function(e,t){if(function(e){let t=e[0][0],n=t.length;for(let r=1;r1&&void 0!==arguments[1]?arguments[1]:100;const n=e.length;if(n<=1)return!0;let r=0,a=n-1;for(;r<=a&&null==e[r];)r++;for(;a>=r&&null==e[a];)a--;if(a<=r)return!0;const i=ks(1,ys((a-r+1)/t));for(let o=e[r],l=r+i;l<=a;l+=i){const t=e[l];if(null!=t){if(t<=o)return!1;o=t}}return!0}(t[0])||(t=function(e){let t=e[0],n=t.length,r=Array(n);for(let i=0;it[e]-t[n]));let a=[];for(let i=0;ie-t))],a=r[0].length,i=new Map;for(let o=0;oSu(e,i,((s,c,u,d,h,m,p,f,v,g,y)=>{let _=s.pxRound,{left:b,width:w}=e.bbox,k=e=>_(m(e,d,g,f)),x=e=>_(p(e,h,y,v)),S=0==d.ori?Iu:Ru;const C={stroke:new Path2D,fill:null,clip:null,band:null,gaps:null,flags:xu},E=C.stroke,N=d.dir*(0==d.ori?1:-1);o=rs(u,o,l,1),l=rs(u,o,l,-1);let A=x(u[1==N?o:l]),M=k(c[1==N?o:l]),T=M,L=M;a&&-1==t&&(L=b,S(E,L,A)),S(E,M,A);for(let e=1==N?o:l;e>=o&&e<=l;e+=N){let n=u[e];if(null==n)continue;let r=k(c[e]),a=x(n);1==t?S(E,r,A):S(E,T,a),S(E,r,a),A=a,T=r}let P=T;a&&1==t&&(P=b+w,S(E,P,A));let[O,I]=Cu(e,i);if(null!=s.fill||0!=O){let t=C.fill=new Path2D(E),n=x(s.fillTo(e,i,s.min,s.max,O));S(t,P,n),S(t,L,n)}if(!s.spanGaps){let a=[];a.push(...Mu(c,u,o,l,N,k,r));let h=s.width*jl/2,m=n||1==t?h:-h,p=n||-1==t?-h:h;a.forEach((e=>{e[0]+=m,e[1]+=p})),C.gaps=a=s.gaps(e,i,o,l,a),C.clip=Au(a,d.ori,f,v,g,y)}return 0!=I&&(C.band=2==I?[Nu(e,i,o,l,E,-1),Nu(e,i,o,l,E,1)]:Nu(e,i,o,l,E,I)),C}))},e.bars=function(e){const t=ds((e=e||qs).size,[.6,As,1]),n=e.align||0,r=e.gap||0;let a=e.radius;a=null==a?[0,0]:"number"==typeof a?[a,0]:a;const i=Ls(a),o=1-t[0],l=ds(t[1],As),s=ds(t[2],1),c=ds(e.disp,qs),u=ds(e.each,(e=>{})),{fill:d,stroke:h}=c;return(e,t,a,m)=>Su(e,t,((p,f,v,g,y,_,b,w,k,x,S)=>{let C,E,N=p.pxRound,A=n,M=r*jl,T=l*jl,L=s*jl;0==g.ori?[C,E]=i(e,t):[E,C]=i(e,t);const P=g.dir*(0==g.ori?1:-1);let O,I,R,D=0==g.ori?Du:zu,z=0==g.ori?u:(e,t,n,r,a,i,o)=>{u(e,t,n,a,r,o,i)},F=ds(e.bands,Ys).find((e=>e.series[0]==t)),j=null!=F?F.dir:0,H=p.fillTo(e,t,p.min,p.max,j),V=N(b(H,y,S,k)),$=x,U=N(p.width*jl),B=!1,q=null,Y=null,W=null,K=null;null==d||0!=U&&null==h||(B=!0,q=d.values(e,t,a,m),Y=new Map,new Set(q).forEach((e=>{null!=e&&Y.set(e,new Path2D)})),U>0&&(W=h.values(e,t,a,m),K=new Map,new Set(W).forEach((e=>{null!=e&&K.set(e,new Path2D)}))));let{x0:Q,size:Z}=c;if(null!=Q&&null!=Z){A=1,f=Q.values(e,t,a,m),2==Q.unit&&(f=f.map((t=>e.posToVal(w+t*x,g.key,!0))));let n=Z.values(e,t,a,m);I=2==Z.unit?n[0]*x:_(n[0],g,x,w)-_(0,g,x,w),$=Wu(f,v,_,g,x,w,$),R=$-I+M}else $=Wu(f,v,_,g,x,w,$),R=$*o+M,I=$-R;R<1&&(R=0),U>=I/2&&(U=0),R<5&&(N=Ps);let G=R>0;I=N(Ts($-R-(G?U:0),L,T)),O=(0==A?I/2:A==P?0:I)-A*P*((0==A?M/2:0)+(G?U/2:0));const J={stroke:null,fill:null,clip:null,band:null,gaps:null,flags:0},X=B?null:new Path2D;let ee=null;if(null!=F)ee=e.data[F.series[1]];else{let{y0:n,y1:r}=c;null!=n&&null!=r&&(v=r.values(e,t,a,m),ee=n.values(e,t,a,m))}let te=C*I,ne=E*I;for(let n=1==P?a:m;n>=a&&n<=m;n+=P){let r=v[n];if(null==r)continue;if(null!=ee){var re;let e=null!==(re=ee[n])&&void 0!==re?re:0;if(r-e==0)continue;V=b(e,y,S,k)}let a=_(2!=g.distr||null!=c?f[n]:n,g,x,w),i=b(ds(r,H),y,S,k),o=N(a-O),l=N(ks(i,V)),s=N(ws(i,V)),u=l-s;if(null!=r){let a=r<0?ne:te,i=r<0?te:ne;B?(U>0&&null!=W[n]&&D(K.get(W[n]),o,s+ys(U/2),I,ks(0,u-U),a,i),null!=q[n]&&D(Y.get(q[n]),o,s+ys(U/2),I,ks(0,u-U),a,i)):D(X,o,s+ys(U/2),I,ks(0,u-U),a,i),z(e,t,n,o-U/2,s,I+U,u)}}if(U>0)J.stroke=B?K:X;else if(!B){var ae;J._fill=0==p.width?p._fill:null!==(ae=p._stroke)&&void 0!==ae?ae:p._fill,J.width=0}return J.fill=B?Y:X,J}))},e.spline=function(e){return function(e,t){const n=ds(null===t||void 0===t?void 0:t.alignGaps,0);return(t,r,a,i)=>Su(t,r,((o,l,s,c,u,d,h,m,p,f,v)=>{let g,y,_,b=o.pxRound,w=e=>b(d(e,c,f,m)),k=e=>b(h(e,u,v,p));0==c.ori?(g=Pu,_=Iu,y=Hu):(g=Ou,_=Ru,y=Vu);const x=c.dir*(0==c.ori?1:-1);a=rs(s,a,i,1),i=rs(s,a,i,-1);let S=w(l[1==x?a:i]),C=S,E=[],N=[];for(let e=1==x?a:i;e>=a&&e<=i;e+=x)if(null!=s[e]){let t=w(l[e]);E.push(C=t),N.push(k(s[e]))}const A={stroke:e(E,N,g,_,y,b),fill:null,clip:null,band:null,gaps:null,flags:xu},M=A.stroke;let[T,L]=Cu(t,r);if(null!=o.fill||0!=T){let e=A.fill=new Path2D(M),n=k(o.fillTo(t,r,o.min,o.max,T));_(e,C,n),_(e,S,n)}if(!o.spanGaps){let e=[];e.push(...Mu(l,s,a,i,x,w,n)),A.gaps=e=o.gaps(t,r,a,i,e),A.clip=Au(e,c.ori,m,p,f,v)}return 0!=L&&(A.band=2==L?[Nu(t,r,a,i,M,-1),Nu(t,r,a,i,M,1)]:Nu(t,r,a,i,M,L)),A}))}(Ku,e)}}const hd=e=>{let t=e.length,n=-1/0;for(;t--;){const r=e[t];Number.isFinite(r)&&r>n&&(n=r)}return Number.isFinite(n)?n:null},md=e=>{let t=e.length,n=1/0;for(;t--;){const r=e[t];Number.isFinite(r)&&r{let t=e.length;const n=[];for(;t--;){const r=e[t];Number.isFinite(r)&&n.push(r)}return n.sort(),n[n.length>>1]},fd=e=>{let t=e.length;for(;t--;){const n=e[t];if(Number.isFinite(n))return n}},vd=(e,t,n)=>{if(void 0===e||null===e)return"";n=n||0,t=t||0;const r=Math.abs(n-t);if(isNaN(r)||0==r)return Math.abs(e)>=1e3?e.toLocaleString("en-US"):e.toString();let a=3+Math.floor(1+Math.log10(Math.max(Math.abs(t),Math.abs(n)))-Math.log10(r));return(isNaN(a)||a>20)&&(a=20),e.toLocaleString("en-US",{minimumSignificantDigits:1,maximumSignificantDigits:a})},gd=e=>{const t=(null===e||void 0===e?void 0:e.metric)||{},n=Object.keys(t).filter((e=>"__name__"!=e)).map((e=>"".concat(e,"=").concat(JSON.stringify(t[e]))));let r=t.__name__||"";return n.length>0&&(r+="{"+n.join(",")+"}"),r},yd=[[31536e3,"{YYYY}",null,null,null,null,null,null,1],[2419200,"{MMM}","\n{YYYY}",null,null,null,null,null,1],[86400,"{MM}-{DD}","\n{YYYY}",null,null,null,null,null,1],[3600,"{HH}:{mm}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD}",null,null,null,1],[60,"{HH}:{mm}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD}",null,null,null,1],[1,"{HH}:{mm}:{ss}","\n{YYYY}-{MM}-{DD}",null,"\n{MM}-{DD} {HH}:{mm}",null,null,null,1],[.001,":{ss}.{fff}","\n{YYYY}-{MM}-{DD} {HH}:{mm}",null,"\n{MM}-{DD} {HH}:{mm}",null,"\n{HH}:{mm}",null,1]],_d=(e,t)=>Array.from(new Set(e.map((e=>e.scale)))).map((e=>{const n="10px Arial",r=ft("color-text"),a={scale:e,show:!0,size:wd,stroke:r,font:n,values:(e,n)=>function(e,t){let n=arguments.length>2&&void 0!==arguments[2]?arguments[2]:"";const r=t[0],a=t[t.length-1];return n?t.map((e=>"".concat(vd(e,r,a)," ").concat(n))):t.map((e=>vd(e,r,a)))}(e,n,t)};return e?Number(e)%2?a:{...a,side:1}:{space:80,values:yd,stroke:r,font:n}})),bd=(e,t)=>{if(null==e||null==t)return[-1,1];const n=.02*(Math.abs(t-e)||Math.abs(e)||1);return[e-n,t+n]},wd=(e,t,n,r)=>{var a;const i=e.axes[n];if(r>1)return i._size||60;let o=6+((null===i||void 0===i||null===(a=i.ticks)||void 0===a?void 0:a.size)||0)+(i.gap||0);const l=(null!==t&&void 0!==t?t:[]).reduce(((e,t)=>(null===t||void 0===t?void 0:t.length)>e.length?t:e),"");return""!=l&&(o+=((e,t)=>{const 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);const r=n.offsetWidth;return n.remove(),r})(l,"10px Arial")),Math.ceil(o)},kd=["#e54040","#32a9dc","#2ee329","#7126a1","#e38f0f","#3d811a","#ffea00","#2d2d2d","#da42a6","#a44e0c"],xd=e=>{if(7!=e.length)return"0, 0, 0";const t=parseInt(e.slice(1,3),16),n=parseInt(e.slice(3,5),16),r=parseInt(e.slice(5,7),16);return"".concat(t,", ").concat(n,", ").concat(r)},Sd={[ut.yhatUpper]:"#7126a1",[ut.yhatLower]:"#7126a1",[ut.yhat]:"#da42a6",[ut.anomaly]:"#da4242",[ut.anomalyScore]:"#7126a1",[ut.actual]:"#203ea9",[ut.training]:"rgba(".concat(xd("#203ea9"),", 0.2)")},Cd=e=>{const t=16777215;let n=1,r=0,a=1;if(e.length>0)for(let o=0;or&&(r=e[o].charCodeAt(0)),a=parseInt(String(t/r)),n=(n+e[o].charCodeAt(0)*a*49979693)%t;let i=(n*e.length%t).toString(16);return i=i.padEnd(6,i),"#".concat(i)},Ed=((e,t,n)=>{const r=[];for(let a=0;aMath.round(e))).join(", "))}return r.map((e=>"rgb(".concat(e,")")))})([246,226,219],[127,39,4],16),Nd=()=>(e,t)=>{const n=Math.round(devicePixelRatio);dd.orient(e,t,((r,a,i,o,l,s,c,u,d,h,m,p,f,v)=>{const[g,y,_]=e.data[t],b=g.length,w=((e,t)=>{const n=e.data[t][2],r=Ed;let a=1/0,i=-1/0;for(let c=0;c0&&(a=Math.min(a,n[c]),i=Math.max(i,n[c]));const o=i-a,l=r.length,s=Array(n.length);for(let c=0;cnew Path2D)),S=b-y.lastIndexOf(y[0]),C=b/S,E=y[1]-y[0],N=g[S]-g[0],A=s(N,o,h,u)-s(0,o,h,u)-n,M=c(E,l,m,d)-c(0,l,m,d)+n,T=y.slice(0,S).map((e=>Math.round(c(e,l,m,d)-M/2))),L=Array.from({length:C},((e,t)=>Math.round(s(g[t*S],o,h,u)-A)));for(let e=0;e0&&g[e]>=(o.min||-1/0)&&g[e]<=(o.max||1/0)&&y[e]>=(l.min||-1/0)&&y[e]<=(l.max||1/0)){const t=L[~~(e/S)],n=T[e%S];v(x[w[e]],t,n,A,M)}e.ctx.save(),e.ctx.rect(e.bbox.left,e.bbox.top,e.bbox.width,e.bbox.height),e.ctx.clip(),x.forEach(((t,n)=>{e.ctx.fillStyle=k[n],e.ctx.fill(t)})),e.ctx.restore()}))},Ad=e=>{const t=(e.metric.vmrange||e.metric.le||"").split("...");return Xo(t[t.length-1])},Md=(e,t)=>Ad(e)-Ad(t),Td=(e,t)=>{if(!t)return e;const n=(e=>{var t;if(!e.every((e=>e.metric.le)))return e;const n=e.sort(((e,t)=>parseFloat(e.metric.le)-parseFloat(t.metric.le))),r=(null===(t=e[0])||void 0===t?void 0:t.group)||1;let a={metric:{le:""},values:[],group:r};const i=[];for(const l of n){const e=[a.metric.le,l.metric.le].filter((e=>e)).join("..."),t=[];for(const[n,r]of l.values){var o;const e=+r-+((null===(o=a.values.find((e=>e[0]===n)))||void 0===o?void 0:o[1])||0);t.push([n,"".concat(e)])}i.push({metric:{vmrange:e},values:t,group:r}),a=l}return i})(e.sort(Md)),r={};n.forEach((e=>e.values.forEach((e=>{let[t,n]=e;r[t]=(r[t]||0)+ +n}))));return n.map((e=>{const t=e.values.map((e=>{let[t,n]=e;const a=r[t];return[t,"".concat(Math.round(+n/a*100))]}));return{...e,values:t}})).filter((e=>!e.values.every((e=>"0"===e[1]))))},Ld=e=>{const t=["__name__","for"];return Object.entries(e).filter((e=>{let[n]=e;return!t.includes(n)})).map((e=>{let[t,n]=e;return"".concat(t,": ").concat(n)})).join(",")},Pd=(e,t,n,r)=>{const a={},i=r?0:Math.min(e.length,kd.length);for(let o=0;o{const l=r?(e=>{const t=(null===e||void 0===e?void 0:e.__name__)||"",n=new RegExp("(".concat(Object.values(ut).join("|"),")$")),r=t.match(n),a=r&&r[0];return{value:/(?:^|[^a-zA-Z0-9_])y(?:$|[^a-zA-Z0-9_])/.test(t)?ut.actual:a,group:Ld(e)}})(e[o].metric):null,s=r?(null===l||void 0===l?void 0:l.group)||"":Jo(i,n[i.group-1]);return{label:s,dash:zd(l),width:Fd(l),stroke:Hd({metricInfo:l,label:s,isAnomalyUI:r,colorState:a}),points:jd(l),spanGaps:!1,forecast:null===l||void 0===l?void 0:l.value,forecastGroup:null===l||void 0===l?void 0:l.group,freeFormFields:i.metric,show:!Rd(s,t),scale:"1",...Od(i)}}},Od=e=>{const t=e.values.map((e=>Xo(e[1]))),{min:n,max:r,median:a,last:i}={min:md(t),max:hd(t),median:pd(t),last:fd(t)};return{median:a,statsFormatted:{min:vd(n,n,r),max:vd(r,n,r),median:vd(a,n,r),last:vd(i,n,r)}}},Id=(e,t)=>({group:t,label:e.label||"",color:e.stroke,checked:e.show||!1,freeFormFields:e.freeFormFields,statsFormatted:e.statsFormatted,median:e.median}),Rd=(e,t)=>t.includes("".concat(e)),Dd=e=>{for(let t=e.series.length-1;t>=0;t--)e.delSeries(t)},zd=e=>{const t=(null===e||void 0===e?void 0:e.value)===ut.yhatLower,n=(null===e||void 0===e?void 0:e.value)===ut.yhatUpper,r=(null===e||void 0===e?void 0:e.value)===ut.yhat;return t||n?[10,5]:r?[10,2]:[]},Fd=e=>{const t=(null===e||void 0===e?void 0:e.value)===ut.yhatLower,n=(null===e||void 0===e?void 0:e.value)===ut.yhatUpper,r=(null===e||void 0===e?void 0:e.value)===ut.yhat,a=(null===e||void 0===e?void 0:e.value)===ut.anomaly;return n||t?.7:r?1:a?0:1.4},jd=e=>(null===e||void 0===e?void 0:e.value)===ut.anomaly?{size:8,width:4,space:0}:{size:4.2,width:1.4},Hd=e=>{let{metricInfo:t,label:n,isAnomalyUI:r,colorState:a}=e;const i=a[n]||Cd(n),o=(null===t||void 0===t?void 0:t.value)===ut.anomaly;return r&&o?Sd[ut.anomaly]:!r||o||null!==t&&void 0!==t&&t.value?null!==t&&void 0!==t&&t.value?null!==t&&void 0!==t&&t.value?Sd[null===t||void 0===t?void 0:t.value]:i:a[n]||Cd(n):Sd[ut.actual]},Vd=e=>{let{width:t=400,height:n=500}=e;return{width:t,height:n,series:[],tzDate:e=>i()(Gt(Xt(e))).local().toDate(),legend:{show:!1},cursor:{drag:{x:!0,y:!1},focus:{prox:30},points:{size:5.6,width:1.4},bind:{click:()=>null,dblclick:()=>null}}}},$d=e=>{Dd(e),(e=>{Object.keys(e.hooks).forEach((t=>{e.hooks[t]=[]}))})(e),e.setData([])},Ud=e=>{let{min:t,max:n}=e;return[t,n]},Bd=function(e){let 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,a=arguments.length>4?arguments[4]:void 0;return a.limits.enable?a.limits.range[r]:bd(t,n)},qd=(e,t)=>{const n={x:{range:()=>Ud(t)}},r=Object.keys(e.limits.range);return(r.length?r:["1"]).forEach((t=>{n[t]={range:function(n){return Bd(n,arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,t,e)}}})),n},Yd=e=>t=>{const n=t.posToVal(t.select.left,"x"),r=t.posToVal(t.select.left+t.select.width,"x");e({min:n,max:r})};function Wd(e){const t=xd(Sd[e]);return"rgba(".concat(t,", 0.05)")}const Kd=e=>(e=>e instanceof MouseEvent)(e)?e.clientX:e.touches[0].clientX,Qd=e=>{let{dragSpeed:t=.85,setPanning:n,setPlotScale:a}=e;const i=(0,r.useRef)({leftStart:0,xUnitsPerPx:0,scXMin:0,scXMax:0}),o=e=>{e.preventDefault();const n=Kd(e),{leftStart:r,xUnitsPerPx:o,scXMin:l,scXMax:s}=i.current,c=o*((n-r)*t);a({min:l-c,max:s-c})},l=()=>{n(!1),document.removeEventListener("mousemove",o),document.removeEventListener("mouseup",l),document.removeEventListener("touchmove",o),document.removeEventListener("touchend",l)};return e=>{let{e:t,u:r}=e;t.preventDefault(),n(!0),i.current={leftStart:Kd(t),xUnitsPerPx:r.posToVal(1,"x")-r.posToVal(0,"x"),scXMin:r.scales.x.min||0,scXMax:r.scales.x.max||0},document.addEventListener("mousemove",o),document.addEventListener("mouseup",l),document.addEventListener("touchmove",o),document.addEventListener("touchend",l)}},Zd=e=>{const[t,n]=(0,r.useState)(!1),a=Qd({dragSpeed:.9,setPanning:n,setPlotScale:e});return{onReadyChart:t=>{const n=e=>{const n=e instanceof MouseEvent&&(e=>{const{ctrlKey:t,metaKey:n,button:r}=e;return 0===r&&(t||n)})(e),r=window.TouchEvent&&e instanceof TouchEvent&&e.touches.length>1;(n||r)&&a({u:t,e:e})};t.over.addEventListener("mousedown",n),t.over.addEventListener("touchstart",n),t.over.addEventListener("wheel",(n=>{if(!n.ctrlKey&&!n.metaKey)return;n.preventDefault();const{width:r}=t.over.getBoundingClientRect(),a=t.cursor.left&&t.cursor.left>0?t.cursor.left:0,i=t.posToVal(a,"x"),o=(t.scales.x.max||0)-(t.scales.x.min||0),l=n.deltaY<0?.9*o:o/.9,s=i-a/r*l,c=s+l;t.batch((()=>e({min:s,max:c})))}))},isPanning:t}},Gd=e=>{const t=e[0].clientX-e[1].clientX,n=e[0].clientY-e[1].clientY;return Math.sqrt(t*t+n*n)},Jd=e=>{let{uPlotInst:t,xRange:n,setPlotScale:a}=e;const[i,o]=(0,r.useState)(0),l=(0,r.useCallback)((e=>{const{target:r,ctrlKey:i,metaKey:o,key:l}=e,s=r instanceof HTMLInputElement||r instanceof HTMLTextAreaElement;if(!t||s)return;const c="+"===l||"="===l;if(("-"===l||c)&&!(i||o)){e.preventDefault();const t=(n.max-n.min)/10*(c?1:-1);a({min:n.min+t,max:n.max-t})}}),[t,n]),s=(0,r.useCallback)((e=>{if(!t||2!==e.touches.length)return;e.preventDefault();const r=Gd(e.touches),o=i-r,l=t.scales.x.max||n.max,s=t.scales.x.min||n.min,c=(l-s)/50*(o>0?-1:1);t.batch((()=>a({min:s+c,max:l-c})))}),[t,i,n]);return Nr("keydown",l),Nr("touchmove",s),Nr("touchstart",(e=>{2===e.touches.length&&(e.preventDefault(),o(Gd(e.touches)))})),null},Xd=e=>{let{period:t,setPeriod:n}=e;const[a,o]=(0,r.useState)({min:t.start,max:t.end});return(0,r.useEffect)((()=>{o({min:t.start,max:t.end})}),[t]),{xRange:a,setPlotScale:e=>{let{min:t,max:r}=e;const a=1e3*(r-t);aFt||n({from:i()(1e3*t).toDate(),to:i()(1e3*r).toDate()})}}},eh=e=>{let{u:t,metrics:n,series:a,unit:o,isAnomalyView:l}=e;const[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)({seriesIdx:-1,dataIdx:-1}),[h,m]=(0,r.useState)([]),p=(0,r.useCallback)((()=>{const{seriesIdx:e,dataIdx:r}=u,s=n[e-1],c=a[e],d=new Set(n.map((e=>e.group))),h=(null===s||void 0===s?void 0:s.group)||0,m=it()(t,["data",e,r],0),p=it()(t,["scales","1","min"],0),f=it()(t,["scales","1","max"],1),v=it()(t,["data",0,r],0),g={top:t?t.valToPos(m||0,(null===c||void 0===c?void 0:c.scale)||"1"):0,left:t?t.valToPos(v,"x"):0};return{unit:o,point:g,u:t,id:"".concat(e,"_").concat(r),title:d.size>1&&!l?"Query ".concat(h):"",dates:[v?i()(1e3*v).tz().format(Pt):"-"],value:vd(m,p,f),info:gd(s),statsFormatted:null===c||void 0===c?void 0:c.statsFormatted,marker:"".concat(null===c||void 0===c?void 0:c.stroke)}}),[t,u,n,a,o,l]),f=(0,r.useCallback)((()=>{if(!s)return;const e=p();h.find((t=>t.id===e.id))||m((t=>[...t,e]))}),[p,h,s]);return(0,r.useEffect)((()=>{c(-1!==u.dataIdx&&-1!==u.seriesIdx)}),[u]),Nr("click",f),{showTooltip:s,stickyTooltips:h,handleUnStick:e=>{m((t=>t.filter((t=>t.id!==e))))},getTooltipProps:p,seriesFocus:(e,t)=>{const n=null!==t&&void 0!==t?t:-1;d((e=>({...e,seriesIdx:n})))},setCursor:e=>{var t;const n=null!==(t=e.cursor.idx)&&void 0!==t?t:-1;d((e=>({...e,dataIdx:n})))},resetTooltips:()=>{m([]),d({seriesIdx:-1,dataIdx:-1})}}},th=e=>{let{u:t,id:n,title:a,dates:i,value:o,point:l,unit:s="",info:c,statsFormatted:u,isSticky:d,marker:h,onClose:m}=e;const p=(0,r.useRef)(null),[f,v]=(0,r.useState)({top:-999,left:-999}),[g,y]=(0,r.useState)(!1),[_,b]=(0,r.useState)(!1),w=(0,r.useCallback)((e=>{if(!g)return;const{clientX:t,clientY:n}=e;v({top:n,left:t})}),[g]);return(0,r.useEffect)((()=>{if(!p.current||!t)return;const{top:e,left:n}=l,r=parseFloat(t.over.style.left),a=parseFloat(t.over.style.top),{width:i,height:o}=t.over.getBoundingClientRect(),{width:s,height:c}=p.current.getBoundingClientRect(),u={top:e+a+10-(e+c>=o?c+20:0),left:n+r+10-(n+s>=i?s+20:0)};u.left<0&&(u.left=20),u.top<0&&(u.top=20),v(u)}),[t,o,l,p]),Nr("mousemove",w),Nr("mouseup",(()=>{y(!1)})),t?r.default.createPortal(Ct("div",{className:xr()({"vm-chart-tooltip":!0,"vm-chart-tooltip_sticky":d,"vm-chart-tooltip_moved":_}),ref:p,style:f,children:[Ct("div",{className:"vm-chart-tooltip-header",children:[a&&Ct("div",{className:"vm-chart-tooltip-header__title",children:a}),Ct("div",{className:"vm-chart-tooltip-header__date",children:i.map(((e,t)=>Ct("span",{children:e},t)))}),d&&Ct(xt.FK,{children:[Ct(aa,{className:"vm-chart-tooltip-header__drag",variant:"text",size:"small",startIcon:Ct(ir,{}),onMouseDown:e=>{b(!0),y(!0);const{clientX:t,clientY:n}=e;v({top:n,left:t})},ariaLabel:"drag the tooltip"}),Ct(aa,{className:"vm-chart-tooltip-header__close",variant:"text",size:"small",startIcon:Ct(On,{}),onClick:()=>{m&&m(n)},ariaLabel:"close the tooltip"})]})]}),Ct("div",{className:"vm-chart-tooltip-data",children:[h&&Ct("span",{className:"vm-chart-tooltip-data__marker",style:{background:h}}),Ct("p",{className:"vm-chart-tooltip-data__value",children:[Ct("b",{children:o}),s]})]}),u&&Ct("table",{className:"vm-chart-tooltip-stats",children:st.map(((e,t)=>Ct("div",{className:"vm-chart-tooltip-stats-row",children:[Ct("span",{className:"vm-chart-tooltip-stats-row__key",children:[e,":"]}),Ct("span",{className:"vm-chart-tooltip-stats-row__value",children:u[e]})]},t)))}),c&&Ct("p",{className:"vm-chart-tooltip__info",children:c})]}),t.root):null},nh=e=>{let{showTooltip:t,tooltipProps:n,stickyTooltips:a,handleUnStick:i}=e;return Ct(xt.FK,{children:[t&&n&&Ct(th,{...n}),a.map((e=>(0,r.createElement)(th,{...e,isSticky:!0,key:e.id,onClose:i})))]})},rh=e=>{let{data:t,series:n,metrics:a=[],period:i,yaxis:o,unit:l,setPeriod:s,layoutSize:c,height:u,isAnomalyView:d,spanGaps:h=!1}=e;const{isDarkTheme:m}=Nt(),p=(0,r.useRef)(null),[f,v]=(0,r.useState)(),{xRange:g,setPlotScale:y}=Xd({period:i,setPeriod:s}),{onReadyChart:_,isPanning:b}=Zd(y);Jd({uPlotInst:f,xRange:g,setPlotScale:y});const{showTooltip:w,stickyTooltips:k,handleUnStick:x,getTooltipProps:S,seriesFocus:C,setCursor:E,resetTooltips:N}=eh({u:f,metrics:a,series:n,unit:l,isAnomalyView:d}),A={...Vd({width:c.width,height:u}),series:n,axes:_d([{},{scale:"1"}],l),scales:qd(o,g),hooks:{ready:[_],setSeries:[C],setCursor:[E],setSelect:[Yd(y)],destroy:[$d]},bands:[]};return(0,r.useEffect)((()=>{if(N(),!p.current)return;f&&f.destroy();const e=new dd(A,t,p.current);return v(e),e.destroy}),[p,m]),(0,r.useEffect)((()=>{f&&(f.setData(t),f.redraw())}),[t]),(0,r.useEffect)((()=>{f&&(Dd(f),function(e,t){let n=arguments.length>2&&void 0!==arguments[2]&&arguments[2];t.forEach((t=>{t.label&&(t.spanGaps=n),e.addSeries(t)}))}(f,n,h),((e,t)=>{if(e.delBand(),t.length<2)return;const n=t.map(((e,t)=>({...e,index:t}))),r=n.filter((e=>e.forecast===ut.yhatUpper)),a=n.filter((e=>e.forecast===ut.yhatLower)),i=r.map((e=>{const t=a.find((t=>t.forecastGroup===e.forecastGroup));return t?{series:[e.index,t.index],fill:Wd(ut.yhatUpper)}:null})).filter((e=>null!==e));i.length&&i.forEach((t=>{e.addBand(t)}))})(f,n),f.redraw())}),[n,h]),(0,r.useEffect)((()=>{f&&(Object.keys(o.limits.range).forEach((e=>{f.scales[e]&&(f.scales[e].range=function(t){return Bd(t,arguments.length>1&&void 0!==arguments[1]?arguments[1]:0,arguments.length>2&&void 0!==arguments[2]?arguments[2]:1,e,o)})})),f.redraw())}),[o]),(0,r.useEffect)((()=>{f&&(f.scales.x.range=()=>Ud(g),f.redraw())}),[g]),(0,r.useEffect)((()=>{f&&(f.setSize({width:c.width||400,height:u||500}),f.redraw())}),[u,c]),Ct("div",{className:xr()({"vm-line-chart":!0,"vm-line-chart_panning":b}),style:{minWidth:"".concat(c.width||400,"px"),minHeight:"".concat(u||500,"px")},children:[Ct("div",{className:"vm-line-chart__u-plot",ref:p}),Ct(nh,{showTooltip:w,tooltipProps:S(),stickyTooltips:k,handleUnStick:x})]})},ah=e=>{let{legend:t,onChange:n,isHeatmap:a,isAnomalyView:i}=e;const o=Uo(),l=(0,r.useMemo)((()=>{const e=(e=>{const t=Object.keys(e.freeFormFields).filter((e=>"__name__"!==e));return t.map((t=>{const n="".concat(t,"=").concat(JSON.stringify(e.freeFormFields[t]));return{id:"".concat(e.label,".").concat(n),freeField:n,key:t}}))})(t);return a?e.filter((e=>"vmrange"!==e.key)):e}),[t,a]),s=t.statsFormatted,c=Object.values(s).some((e=>e)),u=e=>t=>{t.stopPropagation(),(async e=>{await o(e,"".concat(e," has been copied"))})(e)};return Ct("div",{className:xr()({"vm-legend-item":!0,"vm-legend-row":!0,"vm-legend-item_hide":!t.checked&&!a,"vm-legend-item_static":a}),onClick:(e=>t=>{n&&n(e,t.ctrlKey||t.metaKey)})(t),children:[!i&&!a&&Ct("div",{className:"vm-legend-item__marker",style:{backgroundColor:t.color}}),Ct("div",{className:"vm-legend-item-info",children:Ct("span",{className:"vm-legend-item-info__label",children:[t.freeFormFields.__name__,!!l.length&&Ct(xt.FK,{children:"{"}),l.map(((e,t)=>Ct("span",{className:"vm-legend-item-info__free-fields",onClick:u(e.freeField),title:"copy to clipboard",children:[e.freeField,t+1Ct("div",{className:"vm-legend-item-stats-row",children:[Ct("span",{className:"vm-legend-item-stats-row__key",children:[e,":"]}),Ct("span",{className:"vm-legend-item-stats-row__value",children:s[e]})]},t)))})]})},ih=e=>{let{labels:t,query:n,isAnomalyView:a,onChange:i}=e;const o=(0,r.useMemo)((()=>Array.from(new Set(t.map((e=>e.group))))),[t]),l=o.length>1;return Ct(xt.FK,{children:Ct("div",{className:"vm-legend",children:o.map((e=>Ct("div",{className:"vm-legend-group",children:Ct(pi,{defaultExpanded:!0,title:Ct("div",{className:"vm-legend-group-title",children:[l&&Ct("span",{className:"vm-legend-group-title__count",children:["Query ",e,": "]}),Ct("span",{className:"vm-legend-group-title__query",children:n[e-1]})]}),children:Ct("div",{children:t.filter((t=>t.group===e)).sort(((e,t)=>(t.median||0)-(e.median||0))).map((e=>Ct(ah,{legend:e,isAnomalyView:a,onChange:i},e.label)))})})},e)))})})},oh=e=>{var t;let{min:n,max:a,legendValue:i,series:o}=e;const[l,s]=(0,r.useState)(0),[c,u]=(0,r.useState)(""),[d,h]=(0,r.useState)(""),[m,p]=(0,r.useState)(""),f=(0,r.useMemo)((()=>parseFloat(String((null===i||void 0===i?void 0:i.value)||0).replace("%",""))),[i]);return(0,r.useEffect)((()=>{s(f?(f-n)/(a-n)*100:0),u(f?"".concat(f,"%"):""),h("".concat(n,"%")),p("".concat(a,"%"))}),[f,n,a]),Ct("div",{className:"vm-legend-heatmap__wrapper",children:[Ct("div",{className:"vm-legend-heatmap",children:[Ct("div",{className:"vm-legend-heatmap-gradient",style:{background:"linear-gradient(to right, ".concat(Ed.join(", "),")")},children:!!f&&Ct("div",{className:"vm-legend-heatmap-gradient__value",style:{left:"".concat(l,"%")},children:Ct("span",{children:c})})}),Ct("div",{className:"vm-legend-heatmap__value",children:d}),Ct("div",{className:"vm-legend-heatmap__value",children:m})]}),o[1]&&Ct(ah,{legend:o[1],isHeatmap:!0},null===(t=o[1])||void 0===t?void 0:t.label)]})},lh=e=>{let{u:t,metrics:n,unit:a}=e;const[o,l]=(0,r.useState)({left:0,top:0}),[s,c]=(0,r.useState)([]),u=(0,r.useCallback)((()=>{var e;const{left:r,top:l}=o,s=it()(t,["data",1,0],[])||[],c=t?t.posToVal(r,"x"):0,u=t?t.posToVal(l,"y"):0,d=s.findIndex(((e,t)=>c>=e&&ce[0]===h))||[],v=s[d],g=i()(1e3*v).tz().format(Pt),y=i()(1e3*p).tz().format(Pt),_=(null===m||void 0===m||null===(e=m.metric)||void 0===e?void 0:e.vmrange)||"";return{unit:a,point:o,u:t,id:"".concat(_,"_").concat(g),dates:[g,y],value:"".concat(f,"%"),info:_,show:+f>0}}),[t,o,n,a]),d=(0,r.useCallback)((()=>{const e=u();e.show&&(s.find((t=>t.id===e.id))||c((t=>[...t,e])))}),[u,s]);return Nr("click",d),{stickyTooltips:s,handleUnStick:e=>{c((t=>t.filter((t=>t.id!==e))))},getTooltipProps:u,setCursor:e=>{const t=e.cursor.left||0,n=e.cursor.top||0;l({left:t,top:n})},resetTooltips:()=>{c([]),l({left:0,top:0})}}},sh=e=>{let{data:t,metrics:n=[],period:a,unit:i,setPeriod:o,layoutSize:l,height:s,onChangeLegend:c}=e;const{isDarkTheme:u}=Nt(),d=(0,r.useRef)(null),[h,m]=(0,r.useState)(),{xRange:p,setPlotScale:f}=Xd({period:a,setPeriod:o}),{onReadyChart:v,isPanning:g}=Zd(f);Jd({uPlotInst:h,xRange:p,setPlotScale:f});const{stickyTooltips:y,handleUnStick:_,getTooltipProps:b,setCursor:w,resetTooltips:k}=lh({u:h,metrics:n,unit:i}),x=(0,r.useMemo)((()=>b()),[b]),S={...Vd({width:l.width,height:s}),mode:2,series:[{},{paths:Nd(),facets:[{scale:"x",auto:!0,sorted:1},{scale:"y",auto:!0}]}],axes:(()=>{const e=_d([{}],i);return[...e,{scale:"y",stroke:e[0].stroke,font:e[0].font,size:wd,splits:n.map(((e,t)=>t)),values:n.map((e=>e.metric.vmrange))}]})(),scales:{x:{time:!0},y:{log:2,time:!1,range:(e,t,n)=>[t-1,n+1]}},hooks:{ready:[v],setCursor:[w],setSelect:[Yd(f)],destroy:[$d]}};return(0,r.useEffect)((()=>{k();const e=null===t[0]&&Array.isArray(t[1]);if(!d.current||!e)return;const n=new dd(S,t,d.current);return m(n),n.destroy}),[d,t,u]),(0,r.useEffect)((()=>{h&&(h.setSize({width:l.width||400,height:s||500}),h.redraw())}),[s,l]),(0,r.useEffect)((()=>{c(x)}),[x]),Ct("div",{className:xr()({"vm-line-chart":!0,"vm-line-chart_panning":g}),style:{minWidth:"".concat(l.width||400,"px"),minHeight:"".concat(s||500,"px")},children:[Ct("div",{className:"vm-line-chart__u-plot",ref:d}),Ct(nh,{showTooltip:!!x.show,tooltipProps:x,stickyTooltips:y,handleUnStick:_})]})},ch=()=>{const[e,t]=(0,r.useState)(null),[n,a]=(0,r.useState)({width:0,height:0}),i=(0,r.useCallback)((()=>{a({width:(null===e||void 0===e?void 0:e.offsetWidth)||0,height:(null===e||void 0===e?void 0:e.offsetHeight)||0})}),[null===e||void 0===e?void 0:e.offsetHeight,null===e||void 0===e?void 0:e.offsetWidth]);return Nr("resize",i),Er(i,[null===e||void 0===e?void 0:e.offsetHeight,null===e||void 0===e?void 0:e.offsetWidth]),[t,n]},uh={[ut.yhat]:"yhat",[ut.yhatLower]:"yhat_upper - yhat_lower",[ut.yhatUpper]:"yhat_upper - yhat_lower",[ut.anomaly]:"anomalies",[ut.training]:"training data",[ut.actual]:"y"},dh=e=>{let{series:t}=e;const n=(0,r.useMemo)((()=>{const e=t.reduce(((e,t)=>{const n=Object.prototype.hasOwnProperty.call(t,"forecast"),r=t.forecast!==ut.yhatUpper,a=!e.find((e=>e.forecast===t.forecast));return n&&a&&r&&e.push(t),e}),[]),n={...e[0],forecast:ut.training,color:Sd[ut.training]};return e.splice(1,0,n),e.map((e=>({...e,color:"string"===typeof e.stroke?e.stroke:Sd[e.forecast||ut.actual]})))}),[t]);return Ct(xt.FK,{children:Ct("div",{className:"vm-legend-anomaly",children:n.filter((e=>e.forecast!==ut.training)).map(((e,t)=>{var n;return Ct("div",{className:"vm-legend-anomaly-item",children:[Ct("svg",{children:e.forecast===ut.anomaly?Ct("circle",{cx:"15",cy:"7",r:"4",fill:e.color,stroke:e.color,strokeWidth:"1.4"}):Ct("line",{x1:"0",y1:"7",x2:"30",y2:"7",stroke:e.color,strokeWidth:e.width||1,strokeDasharray:null===(n=e.dash)||void 0===n?void 0:n.join(",")})}),Ct("div",{className:"vm-legend-anomaly-item__title",children:uh[e.forecast||ut.actual]})]},"".concat(t,"_").concat(e.forecast))}))})})},hh=e=>{let{data:t=[],period:n,customStep:a,query:i,yaxis:o,unit:l,showLegend:s=!0,setYaxisLimits:c,setPeriod:u,alias:d=[],fullWidth:h=!0,height:m,isHistogram:p,isAnomalyView:f,spanGaps:v}=e;const{isMobile:g}=Wr(),{timezone:y}=vn(),_=(0,r.useMemo)((()=>a||n.step||"1s"),[n.step,a]),b=(0,r.useMemo)((()=>Td(t,p)),[p,t]),[w,k]=(0,r.useState)([[]]),[x,S]=(0,r.useState)([]),[C,E]=(0,r.useState)([]),[N,A]=(0,r.useState)([]),[M,T]=(0,r.useState)(null),L=(0,r.useMemo)((()=>Pd(b,N,d,f)),[b,N,d,f]),P=e=>{const t=((e,t)=>{const n={},r=Object.values(e).flat(),a=md(r)||0,i=hd(r)||1;return n[1]=t?bd(a,i):[a,i],n})(e,!p);c(t)},O=e=>{if(!f)return e;const t=function(e,t){const n=e.reduce(((e,n)=>{const r=t.map((e=>"".concat(e,": ").concat(n[e]||"-"))).join("|");return(e[r]=e[r]||[]).push(n),e}),{});return Object.entries(n).map((e=>{let[t,n]=e;return{keys:t.split("|"),values:n}}))}(e,["group","label"]);return t.map((e=>{const t=e.values[0];return{...t,freeFormFields:{...t.freeFormFields,__name__:""}}}))};(0,r.useEffect)((()=>{const e=[],t={},r=[],a=[{}];null===b||void 0===b||b.forEach(((n,i)=>{const o=L(n,i);a.push(o),r.push(Id(o,n.group));const l=t[n.group]||[];for(const t of n.values)e.push(t[0]),l.push(Xo(t[1]));t[n.group]=l}));const i=((e,t,n)=>{const r=Wt(t)||1,a=Array.from(new Set(e)).sort(((e,t)=>e-t));let i=n.start;const o=Ut(n.end+r);let l=0;const s=[];for(;i<=o;){for(;l=a.length||a[l]>i)&&s.push(i)}for(;s.length<2;)s.push(i),i=Ut(i+r);return s})(e,_,n),o=b.map((e=>{const t=[],n=e.values,r=n.length;let a=0;for(const u of i){for(;anull!==e)),l=Math.abs((e=>{let t=e[0],n=1;for(let r=1;r1e10*c&&!f?t.map((()=>l)):t}));o.unshift(i),P(t);const l=p?(e=>{const t=e.slice(1,e.length),n=[],r=[];t.forEach(((e,n)=>{e.forEach(((e,a)=>{const i=a*t.length+n;r[i]=e}))})),e[0].forEach((e=>{const r=new Array(t.length).fill(e);n.push(...r)}));const a=new Array(n.length).fill(0).map(((e,n)=>n%t.length));return[null,[n,a,r]]})(o):o;k(l),S(a);const s=O(r);E(s),f&&A(s.map((e=>e.label||"")).slice(1))}),[b,y,p]),(0,r.useEffect)((()=>{const e=[],t=[{}];null===b||void 0===b||b.forEach(((n,r)=>{const a=L(n,r);t.push(a),e.push(Id(a,n.group))})),S(t),E(O(e))}),[N]);const[I,R]=ch();return Ct("div",{className:xr()({"vm-graph-view":!0,"vm-graph-view_full-width":h,"vm-graph-view_full-width_mobile":h&&g}),ref:I,children:[!p&&Ct(rh,{data:w,series:x,metrics:b,period:n,yaxis:o,unit:l,setPeriod:u,layoutSize:R,height:m,isAnomalyView:f,spanGaps:v}),p&&Ct(sh,{data:w,metrics:b,period:n,unit:l,setPeriod:u,layoutSize:R,height:m,onChangeLegend:T}),f&&s&&Ct(dh,{series:x}),!p&&s&&Ct(ih,{labels:C,query:i,isAnomalyView:f,onChange:(e,t)=>{A((e=>{let{hideSeries:t,legend:n,metaKey:r,series:a,isAnomalyView:i}=e;const{label:o}=n,l=Rd(o,t),s=a.map((e=>e.label||""));return i?s.filter((e=>e!==o)):r?l?t.filter((e=>e!==o)):[...t,o]:t.length?l?[...s.filter((e=>e!==o))]:[]:[...s.filter((e=>e!==o))]})({hideSeries:N,legend:e,metaKey:t,series:x,isAnomalyView:f}))}}),p&&s&&Ct(oh,{series:x,min:o.limits.range[1][0]||0,max:o.limits.range[1][1]||0,legendValue:M})]})},mh=e=>{let{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a}=e;const{isMobile:i}=Wr(),o=(0,r.useMemo)((()=>Object.keys(t.limits.range)),[t.limits.range]),l=(0,r.useCallback)(Pi()(((e,r,a)=>{const i=t.limits.range;i[r][a]=+e,i[r][0]===i[r][1]||i[r][0]>i[r][1]||n(i)}),500),[t.limits.range]),s=(e,t)=>n=>{l(n,e,t)};return Ct("div",{className:xr()({"vm-axes-limits":!0,"vm-axes-limits_mobile":i}),children:[Ct(jo,{value:t.limits.enable,onChange:a,label:"Fix the limits for y-axis",fullWidth:i}),Ct("div",{className:"vm-axes-limits-list",children:o.map((e=>Ct("div",{className:"vm-axes-limits-list__inputs",children:[Ct(Ha,{label:"Min ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][0],onChange:s(e,0)}),Ct(Ha,{label:"Max ".concat(e),type:"number",disabled:!t.limits.enable,value:t.limits.range[e][1],onChange:s(e,1)})]},e)))})]})},ph=e=>{let{spanGaps:t,onChange:n}=e;const{isMobile:r}=Wr();return Ct("div",{children:Ct(jo,{value:t,onChange:n,label:"Connect null values",fullWidth:r})})},fh="Graph settings",vh=e=>{let{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a,spanGaps:i}=e;const o=(0,r.useRef)(null),l=(0,r.useRef)(null),{value:s,toggle:c,setFalse:u}=oa(!1);return Ct("div",{className:"vm-graph-settings",children:[Ct(fa,{title:fh,children:Ct("div",{ref:l,children:Ct(aa,{variant:"text",startIcon:Ct(Pn,{}),onClick:c,ariaLabel:"settings"})})}),Ct(ia,{open:s,buttonRef:l,placement:"bottom-right",onClose:u,title:fh,children:Ct("div",{className:"vm-graph-settings-popper",ref:o,children:Ct("div",{className:"vm-graph-settings-popper__body",children:[Ct(mh,{yaxis:t,setYaxisLimits:n,toggleEnableLimits:a}),Ct(ph,{spanGaps:i.value,onChange:i.onChange})]})})})]})},gh=e=>{let{isHistogram:t,graphData:n,controlsRef:a,isAnomalyView:i}=e;const{isMobile:o}=Wr(),{customStep:l,yaxis:s,spanGaps:c}=Ur(),{period:u}=vn(),{query:d}=En(),h=gn(),m=Br(),p=e=>{m({type:"SET_YAXIS_LIMITS",payload:e})},f=Ct("div",{className:"vm-custom-panel-body-header__graph-controls",children:[Ct(ba,{}),Ct(vh,{yaxis:s,setYaxisLimits:p,toggleEnableLimits:()=>{m({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})},spanGaps:{value:c,onChange:e=>{m({type:"SET_SPAN_GAPS",payload:e})}}})]});return Ct(xt.FK,{children:[a.current&&(0,r.createPortal)(f,a.current),Ct(hh,{data:n,period:u,customStep:l,query:d,yaxis:s,setYaxisLimits:p,setPeriod:e=>{let{from:t,to:n}=e;h({type:"SET_PERIOD",payload:{from:t,to:n}})},height:o?.5*window.innerHeight:500,isHistogram:t,isAnomalyView:i,spanGaps:c})]})},yh=e=>{let{data:t}=e;const n=Uo(),a=(0,r.useMemo)((()=>JSON.stringify(t,null,2)),[t]);return Ct("div",{className:"vm-json-view",children:[Ct("div",{className:"vm-json-view__copy",children:Ct(aa,{variant:"outlined",onClick:async()=>{await n(a,"Formatted JSON has been copied")},children:"Copy JSON"})}),Ct("pre",{className:"vm-json-view__code",children:Ct("code",{children:a})})]})},_h=e=>{const t={};return e.forEach((e=>Object.entries(e.metric).forEach((e=>t[e[0]]?t[e[0]].options.add(e[1]):t[e[0]]={options:new Set([e[1]])})))),Object.entries(t).map((e=>({key:e[0],variations:e[1].options.size}))).sort(((e,t)=>e.variations-t.variations))},bh=(e,t)=>(0,r.useMemo)((()=>{const n=_h(e);return t?n.filter((e=>t.includes(e.key))):n}),[e,t]),wh=e=>{let{data:t,displayColumns:n}=e;const a=Uo(),{isMobile:i}=Wr(),{tableCompact:o}=Fr(),l=(0,r.useRef)(null),[s,c]=(0,r.useState)(""),[u,d]=(0,r.useState)("asc"),h=o?bh([{group:0,metric:{Data:"Data"}}],["Data"]):bh(t,n),m=e=>{const{__name__:t,...n}=e;return t||Object.keys(n).length?"".concat(t," ").concat(JSON.stringify(n)):""},p=new Set(null===t||void 0===t?void 0:t.map((e=>e.group))).size>1,f=(0,r.useMemo)((()=>{const e=null===t||void 0===t?void 0:t.map((e=>({metadata:h.map((t=>o?Jo(e,"",p):e.metric[t.key]||"-")),value:e.value?e.value[1]:"-",values:e.values?e.values.map((e=>{let[t,n]=e;return"".concat(n," @").concat(t)})):[],copyValue:m(e.metric)}))),n="Value"===s,r=h.findIndex((e=>e.key===s));return n||-1!==r?e.sort(((e,t)=>{const a=n?Number(e.value):e.metadata[r],i=n?Number(t.value):t.metadata[r];return("asc"===u?ai)?-1:1})):e}),[h,t,s,u,o]),v=(0,r.useMemo)((()=>f.some((e=>e.copyValue))),[f]),g=e=>()=>{(e=>{d((t=>"asc"===t&&s===e?"desc":"asc")),c(e)})(e)};return f.length?Ct("div",{className:xr()({"vm-table-view":!0,"vm-table-view_mobile":i}),children:Ct("table",{className:"vm-table",ref:l,children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[h.map(((e,t)=>Ct("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:g(e.key),children:Ct("div",{className:"vm-table-cell__content",children:[e.key,Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":s===e.key,"vm-table__sort-icon_desc":"desc"===u&&s===e.key}),children:Ct(Vn,{})})]})},t))),Ct("td",{className:"vm-table-cell vm-table-cell_header vm-table-cell_right vm-table-cell_sort",onClick:g("Value"),children:Ct("div",{className:"vm-table-cell__content",children:[Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":"Value"===s,"vm-table__sort-icon_desc":"desc"===u}),children:Ct(Vn,{})}),"Value"]})}),v&&Ct("td",{className:"vm-table-cell vm-table-cell_header"})]})}),Ct("tbody",{className:"vm-table-body",children:f.map(((e,t)=>{return Ct("tr",{className:"vm-table__row",children:[e.metadata.map(((e,n)=>Ct("td",{className:xr()({"vm-table-cell vm-table-cell_no-wrap":!0,"vm-table-cell_gray":f[t-1]&&f[t-1].metadata[n]===e}),children:e},n))),Ct("td",{className:"vm-table-cell vm-table-cell_right vm-table-cell_no-wrap",children:e.values.length?e.values.map((e=>Ct("p",{children:e},e))):e.value}),v&&Ct("td",{className:"vm-table-cell vm-table-cell_right",children:e.copyValue&&Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:"Copy row",children:Ct(aa,{variant:"text",color:"gray",size:"small",startIcon:Ct(ar,{}),onClick:(n=e.copyValue,async()=>{await a(n,"Row has been copied")}),ariaLabel:"copy row"})})})})]},t);var n}))})]})}):Ct(Qr,{variant:"warning",children:"No data to show"})},kh=e=>{let{checked:t=!1,disabled:n=!1,label:r,color:a="secondary",onChange:i}=e;return Ct("div",{className:xr()({"vm-checkbox":!0,"vm-checkbox_disabled":n,"vm-checkbox_active":t,["vm-checkbox_".concat(a,"_active")]:t,["vm-checkbox_".concat(a)]:a}),onClick:()=>{n||i(!t)},children:[Ct("div",{className:"vm-checkbox-track",children:Ct("div",{className:"vm-checkbox-track__thumb",children:Ct(er,{})})}),r&&Ct("span",{className:"vm-checkbox__label",children:r})]})},xh="Table settings",Sh=e=>{let{columns:t,defaultColumns:n=[],tableCompact:a,onChangeColumns:i,toggleTableCompact:o}=e;const{isMobile:l}=Wr(),s=(0,r.useRef)(null),{value:c,toggle:u,setFalse:d}=oa(!1),h=(0,r.useMemo)((()=>!t.length),[t]),m=e=>()=>{(e=>{i(n.includes(e)?n.filter((t=>t!==e)):[...n,e])})(e)};return(0,r.useEffect)((()=>{$o(t,n)||i(t)}),[t]),Ct("div",{className:"vm-table-settings",children:[Ct(fa,{title:xh,children:Ct("div",{ref:s,children:Ct(aa,{variant:"text",startIcon:Ct(Pn,{}),onClick:u,disabled:h,ariaLabel:"table settings"})})}),Ct(ia,{open:c,onClose:d,placement:"bottom-right",buttonRef:s,title:xh,children:Ct("div",{className:xr()({"vm-table-settings-popper":!0,"vm-table-settings-popper_mobile":l}),children:[Ct("div",{className:"vm-table-settings-popper-list vm-table-settings-popper-list_first",children:Ct(jo,{label:"Compact view",value:a,onChange:o})}),Ct("div",{className:"vm-table-settings-popper-list",children:[Ct("div",{className:"vm-table-settings-popper-list-header",children:[Ct("h3",{className:"vm-table-settings-popper-list-header__title",children:"Display columns"}),Ct(fa,{title:"Reset to default",children:Ct(aa,{color:"primary",variant:"text",size:"small",onClick:()=>{d(),i(t)},startIcon:Ct(In,{}),ariaLabel:"reset columns"})})]}),t.map((e=>Ct("div",{className:"vm-table-settings-popper-list__item",children:Ct(kh,{checked:n.includes(e),onChange:m(e),label:e,disabled:a})},e)))]})]})})]})},Ch=e=>{let{liveData:t,controlsRef:n}=e;const{tableCompact:a}=Fr(),i=jr(),[o,l]=(0,r.useState)(),s=(0,r.useMemo)((()=>_h(t||[]).map((e=>e.key))),[t]),c=Ct(Sh,{columns:s,defaultColumns:o,onChangeColumns:l,tableCompact:a,toggleTableCompact:()=>{i({type:"TOGGLE_TABLE_COMPACT"})}});return Ct(xt.FK,{children:[n.current&&(0,r.createPortal)(c,n.current),Ct(wh,{data:t,displayColumns:o})]})},Eh=e=>{let{graphData:t,liveData:n,isHistogram:r,displayType:a,controlsRef:i}=e;return a===dt.code&&n?Ct(yh,{data:n}):a===dt.table&&n?Ct(Ch,{liveData:n,controlsRef:i}):a===dt.chart&&t?Ct(gh,{graphData:t,isHistogram:r,controlsRef:i}):null},Nh=[Ct(xt.FK,{children:[Ct("p",{children:"Filename - specify the name for your report file."}),Ct("p",{children:["Default format: ",Ct("code",{children:["vmui_report_$",It,".json"]}),"."]}),Ct("p",{children:"This name will be used when saving your report on your device."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Comment (optional) - add a comment to your report."}),Ct("p",{children:"This can be any additional information that will be useful when reviewing the report later."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Query trace - enable this option to include a query trace in your report."}),Ct("p",{children:"This will assist in analyzing and diagnosing the query processing."})]}),Ct(xt.FK,{children:[Ct("p",{children:"Generate Report - click this button to generate and save your report. "}),Ct("p",{children:["After creation, the report can be downloaded and examined on the ",Ct(Ie,{to:Ye.queryAnalyzer,target:"_blank",rel:"noreferrer",className:"vm-link vm-link_underlined",children:qe[Ye.queryAnalyzer].title})," page."]})]})],Ah=()=>"vmui_report_".concat(i()().utc().format(It)),Mh=e=>{let{fetchUrl:t}=e;const{query:n}=En(),[a,i]=(0,r.useState)(Ah()),[o,l]=(0,r.useState)(""),[s,c]=(0,r.useState)(!0),[u,d]=(0,r.useState)(),[h,m]=(0,r.useState)(!1),p=(0,r.useRef)(null),f=(0,r.useRef)(null),v=(0,r.useRef)(null),g=(0,r.useRef)(null),y=[p,f,v,g],[_,b]=(0,r.useState)(0),{value:w,toggle:k,setFalse:x}=oa(!1),{value:S,toggle:C,setFalse:E}=oa(!1),N=(0,r.useMemo)((()=>{if(t)return t.map(((e,t)=>{const n=new URL(e);return s?n.searchParams.set("trace","1"):n.searchParams.delete("trace"),{id:t,url:n}}))}),[t,s]),A=(0,r.useCallback)((e=>{const t=JSON.stringify(e,null,2),n=new Blob([t],{type:"application/json"}),r=URL.createObjectURL(n),i=document.createElement("a");i.href=r,i.download="".concat(a||Ah(),".json"),document.body.appendChild(i),i.click(),document.body.removeChild(i),URL.revokeObjectURL(r),x()}),[a]),M=(0,r.useCallback)((async()=>{if(N){d(""),m(!0);try{const e=[];for await(const{url:t,id:n}of N){const r=await fetch(t),a=await r.json();if(r.ok)a.vmui={id:n,comment:o,params:rt().parse(new URL(t).search.replace(/^\?/,""))},e.push(a);else{const e=a.errorType?"".concat(a.errorType,"\r\n"):"";d("".concat(e).concat((null===a||void 0===a?void 0:a.error)||(null===a||void 0===a?void 0:a.message)||"unknown error"))}}e.length&&A(e)}catch(rp){rp instanceof Error&&"AbortError"!==rp.name&&d("".concat(rp.name,": ").concat(rp.message))}finally{m(!1)}}else d(ht.validQuery)}),[N,o,A,n]),T=e=>()=>{b((t=>t+e))};return(0,r.useEffect)((()=>{d(""),i(Ah()),l("")}),[w]),(0,r.useEffect)((()=>{b(0)}),[S]),Ct(xt.FK,{children:[Ct(fa,{title:"Export query",children:Ct(aa,{variant:"text",startIcon:Ct(wr,{}),onClick:k,ariaLabel:"export query"})}),w&&Ct(pa,{title:"Export query",onClose:x,isOpen:w,children:Ct("div",{className:"vm-download-report",children:[Ct("div",{className:"vm-download-report-settings",children:[Ct("div",{ref:p,children:Ct(Ha,{label:"Filename",value:a,onChange:i})}),Ct("div",{ref:f,children:Ct(Ha,{type:"textarea",label:"Comment",value:o,onChange:l})}),Ct("div",{ref:v,children:Ct(kh,{checked:s,onChange:c,label:"Include query trace"})})]}),u&&Ct(Qr,{variant:"error",children:u}),Ct("div",{className:"vm-download-report__buttons",children:[Ct(aa,{variant:"text",onClick:C,children:"Help"}),Ct("div",{ref:g,children:Ct(aa,{onClick:M,disabled:h,children:h?"Loading data...":"Generate Report"})})]}),Ct(ia,{open:S,buttonRef:y[_],placement:"top-left",variant:"dark",onClose:E,children:Ct("div",{className:"vm-download-report-helper",children:[Ct("div",{className:"vm-download-report-helper__description",children:Nh[_]}),Ct("div",{className:"vm-download-report-helper__buttons",children:[0!==_&&Ct(aa,{onClick:T(-1),size:"small",color:"white",children:"Prev"}),Ct(aa,{onClick:_===y.length-1?E:T(1),size:"small",color:"white",variant:"text",children:_===y.length-1?"Close":"Next"})]})]})})]})})]})},Th=()=>{(()=>{const{tenantId:e}=Nt(),{displayType:t}=Fr(),{query:n}=En(),{duration:a,relativeTime:i,period:{date:o,step:l}}=vn(),{customStep:s}=Ur(),[,c]=je(),u=()=>{const r={};n.forEach(((n,c)=>{var u;const d="g".concat(c);r["".concat(d,".expr")]=n,r["".concat(d,".range_input")]=a,r["".concat(d,".end_input")]=o,r["".concat(d,".tab")]=(null===(u=Tr.find((e=>e.value===t)))||void 0===u?void 0:u.prometheusCode)||0,r["".concat(d,".relative_time")]=i,r["".concat(d,".tenantID")]=e,l!==s&&s&&(r["".concat(d,".step_input")]=s)})),c(al(r))};(0,r.useEffect)(u,[e,t,n,a,i,o,l,s]),(0,r.useEffect)(u,[])})();const{isMobile:e}=Wr(),{displayType:t}=Fr(),{query:n}=En(),{customStep:a}=Ur(),i=Br(),[o,l]=(0,r.useState)([]),[s,c]=(0,r.useState)(!n[0]),[u,d]=(0,r.useState)(!1),h=(0,r.useRef)(null),{fetchUrl:m,isLoading:p,liveData:f,graphData:v,error:g,queryErrors:y,setQueryErrors:_,queryStats:b,warning:w,traces:k,isHistogram:x}=nl({visible:!0,customStep:a,hideQuery:o,showAllSeries:u}),S=!(null!==f&&void 0!==f&&f.length)&&t!==dt.chart,C=!s&&g;return Nr("popstate",(()=>window.location.reload())),(0,r.useEffect)((()=>{i({type:"SET_IS_HISTOGRAM",payload:x})}),[v]),Ct("div",{className:xr()({"vm-custom-panel":!0,"vm-custom-panel_mobile":e}),children:[Ct(Qo,{queryErrors:s?[]:y,setQueryErrors:_,setHideError:c,stats:b,onHideQuery:e=>{l(e)},onRunQuery:()=>{c(!1)}}),Ct(pl,{traces:k,displayType:t}),p&&Ct(rl,{}),C&&Ct(Qr,{variant:"error",children:g}),S&&Ct(Qr,{variant:"info",children:Ct(sl,{})}),w&&Ct(fl,{warning:w,query:n,onChange:d}),Ct("div",{className:xr()({"vm-custom-panel-body":!0,"vm-custom-panel-body_mobile":e,"vm-block":!0,"vm-block_mobile":e}),children:[Ct("div",{className:"vm-custom-panel-body-header",ref:h,children:[Ct("div",{className:"vm-custom-panel-body-header__tabs",children:Ct(Lr,{})}),(v||f)&&Ct(Mh,{fetchUrl:m})]}),Ct(Eh,{graphData:v,liveData:f,isHistogram:x,displayType:t,controlsRef:h})]})]})},Lh=e=>{let{title:t,description:n,unit:a,expr:i,showLegend:o,filename:l,alias:s}=e;const{isMobile:c}=Wr(),{period:u}=vn(),{customStep:d}=Ur(),h=gn(),m=(0,r.useRef)(null),[p,f]=(0,r.useState)(!1),[v,g]=(0,r.useState)(!1),[y,_]=(0,r.useState)({limits:{enable:!1,range:{1:[0,0]}}}),b=(0,r.useMemo)((()=>Array.isArray(i)&&i.every((e=>e))),[i]),{isLoading:w,graphData:k,error:x,warning:S}=nl({predefinedQuery:b?i:[],display:dt.chart,visible:p,customStep:d}),C=e=>{const t={...y};t.limits.range=e,_(t)};if((0,r.useEffect)((()=>{const e=new IntersectionObserver((e=>{e.forEach((e=>f(e.isIntersecting)))}),{threshold:.1});return m.current&&e.observe(m.current),()=>{m.current&&e.unobserve(m.current)}}),[m]),!b)return Ct(Qr,{variant:"error",children:[Ct("code",{children:'"expr"'})," not found. Check the configuration file ",Ct("b",{children:l}),"."]});const E=()=>Ct("div",{className:"vm-predefined-panel-header__description vm-default-styles",children:[n&&Ct(xt.FK,{children:[Ct("div",{children:[Ct("span",{children:"Description:"}),Ct("div",{dangerouslySetInnerHTML:{__html:To.parse(n)}})]}),Ct("hr",{})]}),Ct("div",{children:[Ct("span",{children:"Queries:"}),Ct("div",{children:i.map(((e,t)=>Ct("div",{children:e},"".concat(t,"_").concat(e))))})]})]});return Ct("div",{className:"vm-predefined-panel",ref:m,children:[Ct("div",{className:"vm-predefined-panel-header",children:[Ct(fa,{title:Ct(E,{}),children:Ct("div",{className:"vm-predefined-panel-header__info",children:Ct(Rn,{})})}),Ct("h3",{className:"vm-predefined-panel-header__title",children:t||""}),Ct(vh,{yaxis:y,setYaxisLimits:C,toggleEnableLimits:()=>{const e={...y};e.limits.enable=!e.limits.enable,_(e)},spanGaps:{value:v,onChange:g}})]}),Ct("div",{className:"vm-predefined-panel-body",children:[w&&Ct(rl,{}),x&&Ct(Qr,{variant:"error",children:x}),S&&Ct(Qr,{variant:"warning",children:S}),k&&Ct(hh,{data:k,period:u,customStep:d,query:i,yaxis:y,unit:a,alias:s,showLegend:o,setYaxisLimits:C,setPeriod:e=>{let{from:t,to:n}=e;h({type:"SET_PERIOD",payload:{from:t,to:n}})},fullWidth:!1,height:c?.5*window.innerHeight:500,spanGaps:v})]})]})},Ph=e=>{let{index:t,title:n,panels:a,filename:i}=e;const o=Ar(),l=(0,r.useMemo)((()=>o.width/12),[o]),[s,c]=(0,r.useState)(!t),[u,d]=(0,r.useState)([]);(0,r.useEffect)((()=>{d(a&&a.map((e=>e.width||12)))}),[a]);const[h,m]=(0,r.useState)({start:0,target:0,enable:!1}),p=(0,r.useCallback)((e=>{if(!h.enable)return;const{start:t}=h,n=Math.ceil((t-e.clientX)/l);if(Math.abs(n)>=12)return;const r=u.map(((e,t)=>e-(t===h.target?n:0)));d(r)}),[h,l]),f=(0,r.useCallback)((()=>{m({...h,enable:!1})}),[h]),v=e=>t=>{((e,t)=>{m({start:e.clientX,target:t,enable:!0})})(t,e)};Nr("mousemove",p),Nr("mouseup",f);return Ct("div",{className:"vm-predefined-dashboard",children:Ct(pi,{defaultExpanded:s,onChange:e=>c(e),title:Ct((()=>Ct("div",{className:xr()({"vm-predefined-dashboard-header":!0,"vm-predefined-dashboard-header_open":s}),children:[(n||i)&&Ct("span",{className:"vm-predefined-dashboard-header__title",children:n||"".concat(t+1,". ").concat(i)}),a&&Ct("span",{className:"vm-predefined-dashboard-header__count",children:["(",a.length," panels)"]})]})),{}),children:Ct("div",{className:"vm-predefined-dashboard-panels",children:Array.isArray(a)&&a.length?a.map(((e,t)=>Ct("div",{className:"vm-predefined-dashboard-panels-panel vm-block vm-block_empty-padding",style:{gridColumn:"span ".concat(u[t])},children:[Ct(Lh,{title:e.title,description:e.description,unit:e.unit,expr:e.expr,alias:e.alias,filename:i,showLegend:e.showLegend}),Ct("button",{className:"vm-predefined-dashboard-panels-panel__resizer",onMouseDown:v(t),"aria-label":"resize the panel"})]},t))):Ct("div",{className:"vm-predefined-dashboard-panels-panel__alert",children:Ct(Qr,{variant:"error",children:[Ct("code",{children:'"panels"'})," not found. Check the configuration file ",Ct("b",{children:i}),"."]})})})})})},Oh=()=>{(()=>{const{duration:e,relativeTime:t,period:{date:n}}=vn(),{customStep:a}=Ur(),{setSearchParamsFromKeys:i}=ii(),o=()=>{const r=al({"g0.range_input":e,"g0.end_input":n,"g0.step_input":a,"g0.relative_time":t});i(r)};(0,r.useEffect)(o,[e,t,n,a]),(0,r.useEffect)(o,[])})();const{isMobile:e}=Wr(),{dashboardsSettings:t,dashboardsLoading:n,dashboardsError:a}=ea(),[i,o]=(0,r.useState)(0),l=(0,r.useMemo)((()=>t.map(((e,t)=>({label:e.title||"",value:t})))),[t]),s=(0,r.useMemo)((()=>t[i]||{}),[t,i]),c=(0,r.useMemo)((()=>null===s||void 0===s?void 0:s.rows),[s]),u=(0,r.useMemo)((()=>s.title||s.filename||""),[s]),d=(0,r.useMemo)((()=>Array.isArray(c)&&!!c.length),[c]),h=e=>()=>{(e=>{o(e)})(e)};return Ct("div",{className:"vm-predefined-panels",children:[n&&Ct(rl,{}),!t.length&&a&&Ct(Qr,{variant:"error",children:a}),!t.length&&Ct(Qr,{variant:"info",children:"Dashboards not found"}),l.length>1&&Ct("div",{className:xr()({"vm-predefined-panels-tabs":!0,"vm-predefined-panels-tabs_mobile":e}),children:l.map((e=>Ct("div",{className:xr()({"vm-predefined-panels-tabs__tab":!0,"vm-predefined-panels-tabs__tab_active":e.value==i}),onClick:h(e.value),children:e.label},e.value)))}),Ct("div",{className:"vm-predefined-panels__dashboards",children:[d&&c.map(((e,t)=>Ct(Ph,{index:t,filename:u,title:e.title,panels:e.panels},"".concat(i,"_").concat(t)))),!!t.length&&!d&&Ct(Qr,{variant:"error",children:[Ct("code",{children:'"rows"'})," not found. Check the configuration file ",Ct("b",{children:u}),"."]})]})]})},Ih=(e,t)=>{const 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)};class Rh{constructor(){this.tsdbStatus=void 0,this.tabsNames=void 0,this.isPrometheus=void 0,this.tsdbStatus=this.defaultTSDBStatus,this.tabsNames=["table","graph"],this.isPrometheus=!1,this.getDefaultState=this.getDefaultState.bind(this)}set tsdbStatusData(e){this.isPrometheus=!(null===e||void 0===e||!e.headStats),this.tsdbStatus=e}get tsdbStatusData(){return this.tsdbStatus}get defaultTSDBStatus(){return{totalSeries:0,totalSeriesPrev:0,totalSeriesByAll:0,totalLabelValuePairs:0,seriesCountByMetricName:[],seriesCountByLabelName:[],seriesCountByFocusLabelValue:[],seriesCountByLabelValuePair:[],labelValueCountByLabelName:[]}}get isPrometheusData(){return this.isPrometheus}keys(e,t){const n=e&&/__name__=".+"/.test(e),r=e&&/{.+=".+"}/g.test(e),a=e&&/__name__=".+", .+!=""/g.test(e);let i=[];return i=t||a?i.concat("seriesCountByFocusLabelValue"):n?i.concat("labelValueCountByLabelName"):r?i.concat("seriesCountByMetricName","seriesCountByLabelName"):i.concat("seriesCountByMetricName","seriesCountByLabelName","seriesCountByLabelValuePair","labelValueCountByLabelName"),i}getDefaultState(e,t){return this.keys(e,t).reduce(((e,t)=>({...e,tabs:{...e.tabs,[t]:this.tabsNames},containerRefs:{...e.containerRefs,[t]:(0,r.useRef)(null)}})),{tabs:{},containerRefs:{}})}sectionsTitles(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"}}get sectionsTips(){return{seriesCountByMetricName:"\n

    \n This table returns a list of metrics with the highest cardinality.\n The cardinality of a metric is the number of time series associated with that metric,\n where each time series is defined as a unique combination of key-value label pairs.\n

    \n

    \n When looking to reduce the number of active series in your data source,\n you can start by inspecting individual metrics with high cardinality\n (i.e. that have lots of active time series associated with them),\n since that single metric contributes a large fraction of the series that make up your total series count.\n

    ",seriesCountByLabelName:"\n

    \n This table returns a list of the labels with the highest number of series.\n

    \n

    \n Use this table to identify labels that are storing dimensions with high cardinality\n (many different label values).\n

    \n

    \n It is recommended to choose labels such that they have a finite set of values,\n since every unique combination of key-value label pairs creates a new time series\n and therefore can dramatically increase the number of time series in your system.\n

    ",seriesCountByFocusLabelValue:"\n

    \n This table returns a list of unique label values per selected label.\n

    \n

    \n Use this table to identify label values that are storing per each selected series.\n

    ",labelValueCountByLabelName:"\n

    \n This table returns a list of labels with the highest number of the unique values.\n

    \n ",seriesCountByLabelValuePair:"\n

    \n This table returns a list of the label values pairs with the highest number of series.\n

    \n

    \n Use this table to identify unique label values pairs. This helps to identify same labels \n is applied to count timeseries in your system, since every unique combination of key-value label pairs \n creates a new time series and therefore can dramatically increase the number of time series in your system\n

    "}}get tablesHeaders(){return{seriesCountByMetricName:Dh,seriesCountByLabelName:zh,seriesCountByFocusLabelValue:Fh,seriesCountByLabelValuePair:jh,labelValueCountByLabelName:Hh}}totalSeries(e){return"labelValueCountByLabelName"===e?-1:arguments.length>1&&void 0!==arguments[1]&&arguments[1]?this.tsdbStatus.totalSeriesPrev:this.tsdbStatus.totalSeries}}const Dh=[{id:"name",label:"Metric name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of a metric to the total number of series"},{id:"action",label:""}],zh=[{id:"name",label:"Label name"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of the label to the total number of series"},{id:"action",label:""}],Fh=[{id:"name",label:"Label value"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total"},{disablePadding:!1,id:"action",label:"",numeric:!1}],jh=[{id:"name",label:"Label=value pair"},{id:"value",label:"Number of series"},{id:"percentage",label:"Share in total",info:"Shows the share of the label value pair to the total number of series"},{id:"action",label:""}],Hh=[{id:"name",label:"Label name"},{id:"value",label:"Number of unique values"},{id:"action",label:""}],Vh=()=>{const e=new Rh,[t]=je(),n=t.get("match"),a=t.get("focusLabel"),o=+(t.get("topN")||10),l=t.get("date")||i()().tz().format(Tt),s=$a(l),{serverUrl:c}=Nt(),[u,d]=(0,r.useState)(!1),[h,m]=(0,r.useState)(),[p,f]=(0,r.useState)(e.defaultTSDBStatus),[v,g]=(0,r.useState)(!1),y=async e=>{const t=await fetch(e);if(t.ok)return await t.json();throw new Error("Request failed with status ".concat(t.status))},_=async t=>{if(!c)return;m(""),d(!0),f(e.defaultTSDBStatus);const r={...t,date:t.date,topN:0,match:"",focusLabel:""},a={...t,date:i()(t.date).subtract(1,"day").tz().format(Tt)},o=[Ih(c,t),Ih(c,a)];s!==l&&o.push(Ih(c,r));try{var u,h,v,g,_,b,w,k,x,S;const[e,t,r={}]=await Promise.all(o.map(y)),a={...t.data},{data:i}=r,l={...e.data,totalSeries:(null===(u=e.data)||void 0===u?void 0:u.totalSeries)||(null===(h=e.data)||void 0===h||null===(v=h.headStats)||void 0===v?void 0:v.numSeries)||0,totalLabelValuePairs:(null===(g=e.data)||void 0===g?void 0:g.totalLabelValuePairs)||(null===(_=e.data)||void 0===_||null===(b=_.headStats)||void 0===b?void 0:b.numLabelValuePairs)||0,seriesCountByLabelName:(null===(w=e.data)||void 0===w?void 0:w.seriesCountByLabelName)||[],seriesCountByFocusLabelValue:(null===(k=e.data)||void 0===k?void 0:k.seriesCountByFocusLabelValue)||[],totalSeriesByAll:(null===i||void 0===i?void 0:i.totalSeries)||(null===i||void 0===i||null===(x=i.headStats)||void 0===x?void 0:x.numSeries)||p.totalSeriesByAll||0,totalSeriesPrev:(null===a||void 0===a?void 0:a.totalSeries)||(null===a||void 0===a||null===(S=a.headStats)||void 0===S?void 0:S.numSeries)||0},s=null===n||void 0===n?void 0:n.replace(/[{}"]/g,"");l.seriesCountByLabelValuePair=l.seriesCountByLabelValuePair.filter((e=>e.name!==s)),((e,t)=>{Object.keys(e).forEach((n=>{const r=n,a=e[r],i=t[r];Array.isArray(a)&&Array.isArray(i)&&a.forEach((e=>{var t;const n=null===(t=i.find((t=>t.name===e.name)))||void 0===t?void 0:t.value;e.diff=n?e.value-n:0,e.valuePrev=n||0}))}))})(l,a),f(l),d(!1)}catch(rp){d(!1),rp instanceof Error&&m("".concat(rp.name,": ").concat(rp.message))}};return(0,r.useEffect)((()=>{_({topN:o,match:n,date:l,focusLabel:a})}),[c,n,a,o,l]),(0,r.useEffect)((()=>{h&&(f(e.defaultTSDBStatus),d(!1))}),[h]),(0,r.useEffect)((()=>{const e=Ge(c);g(!!e)}),[c]),e.tsdbStatusData=p,{isLoading:u,appConfigurator:e,error:h,isCluster:v}},$h={seriesCountByMetricName:e=>{let{query:t}=e;return Uh("__name__",t)},seriesCountByLabelName:e=>{let{query:t}=e;return"{".concat(t,'!=""}')},seriesCountByFocusLabelValue:e=>{let{query:t,focusLabel:n}=e;return Uh(n,t)},seriesCountByLabelValuePair:e=>{let{query:t}=e;const n=t.split("="),r=n[0],a=n.slice(1).join("=");return Uh(r,a)},labelValueCountByLabelName:e=>{let{query:t,match:n}=e;return""===n?"{".concat(t,'!=""}'):"".concat(n.replace("}",""),", ").concat(t,'!=""}')}},Uh=(e,t)=>e?"{"+e+"="+JSON.stringify(t)+"}":"",Bh=e=>{var t;let{totalSeries:n=0,totalSeriesPrev:r=0,totalSeriesAll:a=0,seriesCountByMetricName:i=[],isPrometheus:o}=e;const{isMobile:l}=Wr(),[s]=je(),c=s.get("match"),u=s.get("focusLabel"),d=/__name__/.test(c||""),h=(null===(t=i[0])||void 0===t?void 0:t.value)/a*100,m=n-r,p=Math.abs(m)/r*100,f=[{title:"Total series",value:n.toLocaleString("en-US"),dynamic:n&&r&&!o?"".concat(p.toFixed(2),"%"):"",display:!u,info:'The total number of active time series. \n A time series is uniquely identified by its name plus a set of its labels. \n For example, temperature{city="NY",country="US"} and temperature{city="SF",country="US"} \n are two distinct series, since they differ by the city label.'},{title:"Percentage from total",value:isNaN(h)?"-":"".concat(h.toFixed(2),"%"),display:d,info:"The share of these series in the total number of time series."}].filter((e=>e.display));return f.length?Ct("div",{className:xr()({"vm-cardinality-totals":!0,"vm-cardinality-totals_mobile":l}),children:f.map((e=>{let{title:t,value:n,info:a,dynamic:i}=e;return Ct("div",{className:"vm-cardinality-totals-card",children:[Ct("h4",{className:"vm-cardinality-totals-card__title",children:[t,a&&Ct(fa,{title:Ct("p",{className:"vm-cardinality-totals-card__tooltip",children:a}),children:Ct("div",{className:"vm-cardinality-totals-card__info-icon",children:Ct(Rn,{})})})]}),Ct("span",{className:"vm-cardinality-totals-card__value",children:n}),!!i&&Ct(fa,{title:"in relation to the previous day: ".concat(r.toLocaleString("en-US")),children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive vm-dynamic-number_down":m<0,"vm-dynamic-number_negative vm-dynamic-number_up":m>0}),children:i})})]},t)}))}):null},qh=(e,t)=>{const[n]=je(),a=n.get(t)?n.get(t):e,[i,o]=(0,r.useState)(a);return(0,r.useEffect)((()=>{a!==i&&o(a)}),[a]),[i,o]},Yh=e=>{let{isPrometheus:t,isCluster:n,...a}=e;const{isMobile:i}=Wr(),[o]=je(),{setSearchParamsFromKeys:l}=ii(),s=o.get("tips")||"",[c,u]=qh("","match"),[d,h]=qh("","focusLabel"),[m,p]=qh(10,"topN"),f=(0,r.useMemo)((()=>m<0?"Number must be bigger than zero":""),[m]),v=()=>{l({match:c,topN:m,focusLabel:d})};return(0,r.useEffect)((()=>{const e=o.get("match"),t=+(o.get("topN")||10),n=o.get("focusLabel");e!==c&&u(e||""),t!==m&&p(t),n!==d&&h(n||"")}),[o]),Ct("div",{className:xr()({"vm-cardinality-configurator":!0,"vm-cardinality-configurator_mobile":i,"vm-block":!0,"vm-block_mobile":i}),children:[Ct("div",{className:"vm-cardinality-configurator-controls",children:[Ct("div",{className:"vm-cardinality-configurator-controls__query",children:Ct(Ha,{label:"Time series selector",type:"string",value:c,onChange:u,onEnter:v})}),Ct("div",{className:"vm-cardinality-configurator-controls__item",children:Ct(Ha,{label:"Focus label",type:"text",value:d||"",onChange:h,onEnter:v,endIcon:Ct(fa,{title:Ct("div",{children:Ct("p",{children:"To identify values with the highest number of series for the selected label."})}),children:Ct(cr,{})})})}),Ct("div",{className:"vm-cardinality-configurator-controls__item vm-cardinality-configurator-controls__item_limit",children:Ct(Ha,{label:"Limit entries",type:"number",value:t?10:m,error:f,disabled:t,helperText:t?"not available for Prometheus":"",onChange:e=>{const t=+e;p(isNaN(t)?0:t)},onEnter:v})})]}),Ct("div",{className:"vm-cardinality-configurator-bottom",children:[Ct(Bh,{isPrometheus:t,isCluster:n,...a}),n&&Ct("div",{className:"vm-cardinality-configurator-bottom-helpful",children:Ct(il,{href:"https://docs.victoriametrics.com/Single-server-VictoriaMetrics.html#cardinality-explorer-statistic-inaccuracy",withIcon:!0,children:[Ct(lr,{}),"Statistic inaccuracy explanation"]})}),Ct("div",{className:"vm-cardinality-configurator-bottom-helpful",children:Ct(il,{href:"https://docs.victoriametrics.com/#cardinality-explorer",withIcon:!0,children:[Ct(lr,{}),"Documentation"]})}),Ct("div",{className:"vm-cardinality-configurator-bottom__execute",children:[Ct(fa,{title:s?"Hide tips":"Show tips",children:Ct(aa,{variant:"text",color:s?"warning":"gray",startIcon:Ct(mr,{}),onClick:()=>{const e=o.get("tips")||"";l({tips:e?"":"true"})},ariaLabel:"visibility tips"})}),Ct(aa,{variant:"text",startIcon:Ct(In,{}),onClick:()=>{l({match:"",focusLabel:""})},children:"Reset"}),Ct(aa,{startIcon:Ct(Yn,{}),onClick:v,children:"Execute Query"})]})]})]})};function Wh(e){const{order:t,orderBy:n,onRequestSort:r,headerCells:a}=e;return Ct("thead",{className:"vm-table-header vm-cardinality-panel-table__header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:a.map((e=>{return Ct("th",{className:xr()({"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:(a=e.id,e=>{r(e,a)}),children:Ct("div",{className:"vm-table-cell__content",children:[e.info?Ct(fa,{title:e.info,children:[Ct("div",{className:"vm-metrics-content-header__tip-icon",children:Ct(Rn,{})}),e.label]}):Ct(xt.FK,{children:e.label}),"action"!==e.id&&"percentage"!==e.id&&Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":n===e.id,"vm-table__sort-icon_desc":"desc"===t&&n===e.id}),children:Ct(Vn,{})})]})},e.id);var a}))})})}const Kh=["date","timestamp","time"];function Qh(e,t,n){const r=e[n],a=t[n],o=Kh.includes("".concat(n))?i()("".concat(r)).unix():r,l=Kh.includes("".concat(n))?i()("".concat(a)).unix():a;return lo?1:0}function Zh(e,t){return"desc"===e?(e,n)=>Qh(e,n,t):(e,n)=>-Qh(e,n,t)}function Gh(e,t){const n=e.map(((e,t)=>[e,t]));return n.sort(((e,n)=>{const r=t(e[0],n[0]);return 0!==r?r:e[1]-n[1]})),n.map((e=>e[0]))}const Jh=e=>{let{rows:t,headerCells:n,defaultSortColumn:a,tableCells:i}=e;const[o,l]=(0,r.useState)("desc"),[s,c]=(0,r.useState)(a),u=Gh(t,Zh(o,s));return Ct("table",{className:"vm-table vm-cardinality-panel-table",children:[Ct(Wh,{order:o,orderBy:s,onRequestSort:(e,t)=>{l(s===t&&"asc"===o?"desc":"asc"),c(t)},rowCount:t.length,headerCells:n}),Ct("tbody",{className:"vm-table-header",children:u.map((e=>Ct("tr",{className:"vm-table__row",children:i(e)},e.name)))})]})},Xh=e=>{let{row:t,totalSeries:n,totalSeriesPrev:r,onActionClick:a}=e;const i=n>0?t.value/n*100:-1,o=r>0?t.valuePrev/r*100:-1,l=[i,o].some((e=>-1===e)),s=i-o,c=l?"":"".concat(s.toFixed(2),"%"),u=()=>{a(t.name)};return Ct(xt.FK,{children:[Ct("td",{className:"vm-table-cell",children:Ct("span",{className:"vm-link vm-link_colored",onClick:u,children:t.name})},t.name),Ct("td",{className:"vm-table-cell",children:[t.value,!!t.diff&&Ct(fa,{title:"in relation to the previous day: ".concat(t.valuePrev),children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive":t.diff<0,"vm-dynamic-number_negative":t.diff>0}),children:["\xa0",t.diff>0?"+":"",t.diff]})})]},t.value),i>0&&Ct("td",{className:"vm-table-cell",children:Ct("div",{className:"vm-cardinality-panel-table__progress",children:[Ct(cl,{value:i}),c&&Ct(fa,{title:"in relation to the previous day",children:Ct("span",{className:xr()({"vm-dynamic-number":!0,"vm-dynamic-number_positive vm-dynamic-number_down":s<0,"vm-dynamic-number_negative vm-dynamic-number_up":s>0}),children:c})})]})},t.progressValue),Ct("td",{className:"vm-table-cell vm-table-cell_right",children:Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:"Filter by ".concat(t.name),children:Ct(aa,{variant:"text",size:"small",onClick:u,children:Ct(Wn,{})})})})},"action")]})},em=e=>{let{data:t}=e;const[n,a]=(0,r.useState)([]),[i,o]=(0,r.useState)([0,0]);return(0,r.useEffect)((()=>{const e=t.sort(((e,t)=>t.value-e.value)),n=(e=>{const t=e.map((e=>e.value)),n=Math.ceil(t[0]||1),r=n/9;return new Array(11).fill(n+r).map(((e,t)=>Math.round(e-r*t)))})(e);o(n),a(e.map((e=>({...e,percentage:e.value/n[0]*100}))))}),[t]),Ct("div",{className:"vm-simple-bar-chart",children:[Ct("div",{className:"vm-simple-bar-chart-y-axis",children:i.map((e=>Ct("div",{className:"vm-simple-bar-chart-y-axis__tick",children:e},e)))}),Ct("div",{className:"vm-simple-bar-chart-data",children:n.map((e=>{let{name:t,value:n,percentage:r}=e;return Ct(fa,{title:"".concat(t,": ").concat(n),placement:"top-center",children:Ct("div",{className:"vm-simple-bar-chart-data-item",style:{maxHeight:"".concat(r||0,"%")}})},"".concat(t,"_").concat(n))}))})]})},tm=e=>{let{rows:t,tabs:n=[],chartContainer:a,totalSeries:i,totalSeriesPrev:o,onActionClick:l,sectionTitle:s,tip:c,tableHeaderCells:u,isPrometheus:d}=e;const{isMobile:h}=Wr(),[m,p]=(0,r.useState)("table"),f=d&&!t.length,v=(0,r.useMemo)((()=>n.map(((e,t)=>({value:e,label:e,icon:Ct(0===t?Qn:Kn,{})})))),[n]);return Ct("div",{className:xr()({"vm-metrics-content":!0,"vm-metrics-content_mobile":h,"vm-block":!0,"vm-block_mobile":h}),children:[Ct("div",{className:"vm-metrics-content-header vm-section-header",children:[Ct("h5",{className:xr()({"vm-metrics-content-header__title":!0,"vm-section-header__title":!0,"vm-section-header__title_mobile":h}),children:[!h&&c&&Ct(fa,{title:Ct("p",{dangerouslySetInnerHTML:{__html:c},className:"vm-metrics-content-header__tip"}),children:Ct("div",{className:"vm-metrics-content-header__tip-icon",children:Ct(Rn,{})})}),s]}),Ct("div",{className:"vm-section-header__tabs",children:Ct(Mr,{activeItem:m,items:v,onChange:p})})]}),f&&Ct("div",{className:"vm-metrics-content-prom-data",children:[Ct("div",{className:"vm-metrics-content-prom-data__icon",children:Ct(Rn,{})}),Ct("h3",{className:"vm-metrics-content-prom-data__title",children:"Prometheus Data Limitation"}),Ct("p",{className:"vm-metrics-content-prom-data__text",children:["Due to missing data from your Prometheus source, some tables may appear empty.",Ct("br",{}),"This does not indicate an issue with your system or our tool."]})]}),!f&&"table"===m&&Ct("div",{ref:a,className:xr()({"vm-metrics-content__table":!0,"vm-metrics-content__table_mobile":h}),children:Ct(Jh,{rows:t,headerCells:u,defaultSortColumn:"value",tableCells:e=>Ct(Xh,{row:e,totalSeries:i,totalSeriesPrev:o,onActionClick:l})})}),!f&&"graph"===m&&Ct("div",{className:"vm-metrics-content__chart",children:Ct(em,{data:t.map((e=>{let{name:t,value:n}=e;return{name:t,value:n}}))})})]})},nm=e=>{let{title:t,children:n}=e;return Ct("div",{className:"vm-cardinality-tip",children:[Ct("div",{className:"vm-cardinality-tip-header",children:[Ct("div",{className:"vm-cardinality-tip-header__tip-icon",children:Ct(mr,{})}),Ct("h4",{className:"vm-cardinality-tip-header__title",children:t||"Tips"})]}),Ct("p",{className:"vm-cardinality-tip__description",children:n})]})},rm=()=>Ct(nm,{title:"Metrics with a high number of series",children:Ct("ul",{children:[Ct("li",{children:["Identify and eliminate labels with frequently changed values to reduce their\xa0",Ct(il,{href:"https://docs.victoriametrics.com/FAQ.html#what-is-high-cardinality",children:"cardinality"}),"\xa0and\xa0",Ct(il,{href:"https://docs.victoriametrics.com/FAQ.html#what-is-high-churn-rate",children:"high churn rate"})]}),Ct("li",{children:["Find unused time series and\xa0",Ct(il,{href:"https://docs.victoriametrics.com/relabeling.html",children:"drop entire metrics"})]}),Ct("li",{children:["Aggregate time series before they got ingested into the database via\xa0",Ct(il,{href:"https://docs.victoriametrics.com/stream-aggregation.html",children:"streaming aggregation"})]})]})}),am=()=>Ct(nm,{title:"Labels with a high number of unique values",children:Ct("ul",{children:[Ct("li",{children:"Decrease the number of unique label values to reduce cardinality"}),Ct("li",{children:["Drop the label entirely via\xa0",Ct(il,{href:"https://docs.victoriametrics.com/relabeling.html",children:"relabeling"})]}),Ct("li",{children:"For volatile label values (such as URL path, user session, etc.) consider printing them to the log file instead of adding to time series"})]})}),im=()=>Ct(nm,{title:"Dashboard of a single metric",children:[Ct("p",{children:"This dashboard helps to understand the cardinality of a single metric."}),Ct("p",{children:"Each time series is a unique combination of key-value label pairs. Therefore a label key with many values can create a lot of time series for a particular metric. If you\u2019re trying to decrease the cardinality of a metric, start by looking at the labels with the highest number of values."}),Ct("p",{children:"Use the series selector at the top of the page to apply additional filters."})]}),om=()=>Ct(nm,{title:"Dashboard of a label",children:[Ct("p",{children:"This dashboard helps you understand the count of time series per label."}),Ct("p",{children:"Use the selector at the top of the page to pick a label name you\u2019d like to inspect. For the selected label name, you\u2019ll see the label values that have the highest number of series associated with them. So if you\u2019ve chosen `instance` as your label name, you may see that `657` time series have value \u201chost-1\u201d attached to them and `580` time series have value `host-2` attached to them."}),Ct("p",{children:"This can be helpful in allowing you to determine where the bulk of your time series are coming from. If the label \u201cinstance=host-1\u201d was applied to 657 series and the label \u201cinstance=host-2\u201d was only applied to 580 series, you\u2019d know, for example, that host-01 was responsible for sending the majority of the time series."})]}),lm=()=>{const{isMobile:e}=Wr(),[t]=je(),{setSearchParamsFromKeys:n}=ii(),r=t.get("tips")||"",a=t.get("match")||"",i=t.get("focusLabel")||"",{isLoading:o,appConfigurator:l,error:s,isCluster:c}=Vh(),{tsdbStatusData:u,getDefaultState:d,tablesHeaders:h,sectionsTips:m}=l,p=d(a,i);return Ct("div",{className:xr()({"vm-cardinality-panel":!0,"vm-cardinality-panel_mobile":e}),children:[o&&Ct(rl,{message:"Please wait while cardinality stats is calculated. \n This may take some time if the db contains big number of time series."}),Ct(Yh,{isPrometheus:l.isPrometheusData,totalSeries:u.totalSeries,totalSeriesPrev:u.totalSeriesPrev,totalSeriesAll:u.totalSeriesByAll,totalLabelValuePairs:u.totalLabelValuePairs,seriesCountByMetricName:u.seriesCountByMetricName,isCluster:c}),r&&Ct("div",{className:"vm-cardinality-panel-tips",children:[!a&&!i&&Ct(rm,{}),a&&!i&&Ct(im,{}),!a&&!i&&Ct(am,{}),i&&Ct(om,{})]}),s&&Ct(Qr,{variant:"error",children:s}),l.keys(a,i).map((e=>{return Ct(tm,{sectionTitle:l.sectionsTitles(i)[e],tip:m[e],rows:u[e],onActionClick:(t=e,e=>{const r={match:$h[t]({query:e,focusLabel:i,match:a})};"labelValueCountByLabelName"!==t&&"seriesCountByLabelName"!=t||(r.focusLabel=e),"seriesCountByFocusLabelValue"==t&&(r.focusLabel=""),n(r)}),tabs:p.tabs[e],chartContainer:p.containerRefs[e],totalSeriesPrev:l.totalSeries(e,!0),totalSeries:l.totalSeries(e),tableHeaderCells:h[e],isPrometheus:l.isPrometheusData},e);var t}))]})},sm=e=>(["topByAvgDuration","topByCount","topBySumDuration"].forEach((t=>{const n=e[t];Array.isArray(n)&&n.forEach((e=>{const t=Jt(1e3*e.timeRangeSeconds);e.url=((e,t)=>{var n;const{query:r,timeRangeSeconds:a}=e,i=["g0.expr=".concat(encodeURIComponent(r))],o=null===(n=en.find((e=>e.duration===t)))||void 0===n?void 0:n.id;return o&&i.push("g0.relative_time=".concat(o)),a&&i.push("g0.range_input=".concat(t)),"".concat(Ye.home,"?").concat(i.join("&"))})(e,t),e.timeRange=t}))})),e),cm=e=>{let{topN:t,maxLifetime:n}=e;const{serverUrl:a}=Nt(),{setSearchParamsFromKeys:i}=ii(),[o,l]=(0,r.useState)(null),[s,c]=(0,r.useState)(!1),[u,d]=(0,r.useState)(),h=(0,r.useMemo)((()=>((e,t,n)=>"".concat(e,"/api/v1/status/top_queries?topN=").concat(t||"","&maxLifetime=").concat(n||""))(a,t,n)),[a,t,n]);return{data:o,error:u,loading:s,fetch:async()=>{c(!0),i({topN:t,maxLifetime:n});try{const e=await fetch(h),t=await e.json();l(e.ok?sm(t):null),d(String(t.error||""))}catch(rp){rp instanceof Error&&"AbortError"!==rp.name&&d("".concat(rp.name,": ").concat(rp.message))}c(!1)}}},um=e=>{let{rows:t,columns:n,defaultOrderBy:a}=e;const i=Uo(),[o,l]=(0,r.useState)(a||"count"),[s,c]=(0,r.useState)("desc"),u=(0,r.useMemo)((()=>Gh(t,Zh(s,o))),[t,o,s]),d=e=>()=>{var t;t=e,c((e=>"asc"===e&&o===t?"desc":"asc")),l(t)},h=e=>{let{query:t}=e;return async()=>{await i(t,"Query has been copied")}};return Ct("table",{className:"vm-table",children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[n.map((e=>Ct("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:d(e.sortBy||e.key),children:Ct("div",{className:"vm-table-cell__content",children:[e.title||e.key,Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":o===e.key,"vm-table__sort-icon_desc":"desc"===s&&o===e.key}),children:Ct(Vn,{})})]})},e.key))),Ct("th",{className:"vm-table-cell vm-table-cell_header"})," "]})}),Ct("tbody",{className:"vm-table-body",children:u.map(((e,t)=>Ct("tr",{className:"vm-table__row",children:[n.map((t=>Ct("td",{className:"vm-table-cell",children:e[t.key]||"-"},t.key))),Ct("td",{className:"vm-table-cell vm-table-cell_no-padding",children:Ct("div",{className:"vm-top-queries-panels__table-actions",children:[e.url&&Ct(fa,{title:"Execute query",children:Ct(Ie,{to:e.url,target:"_blank",rel:"noreferrer","aria-disabled":!0,children:Ct(aa,{variant:"text",size:"small",startIcon:Ct(Wn,{}),ariaLabel:"execute query"})})}),Ct(fa,{title:"Copy query",children:Ct(aa,{variant:"text",size:"small",startIcon:Ct(ar,{}),onClick:h(e),ariaLabel:"copy query"})})]})})]},t)))})]})},dm=["table","JSON"].map(((e,t)=>({value:String(t),label:e,icon:Ct(0===t?Qn:Zn,{})}))),hm=e=>{let{rows:t,title:n,columns:a,defaultOrderBy:i}=e;const{isMobile:o}=Wr(),[l,s]=(0,r.useState)(0);return Ct("div",{className:xr()({"vm-top-queries-panel":!0,"vm-block":!0,"vm-block_mobile":o}),children:[Ct("div",{className:xr()({"vm-top-queries-panel-header":!0,"vm-section-header":!0,"vm-top-queries-panel-header_mobile":o}),children:[Ct("h5",{className:xr()({"vm-section-header__title":!0,"vm-section-header__title_mobile":o}),children:n}),Ct("div",{className:"vm-section-header__tabs",children:Ct(Mr,{activeItem:String(l),items:dm,onChange:e=>{s(+e)}})})]}),Ct("div",{className:xr()({"vm-top-queries-panel__table":!0,"vm-top-queries-panel__table_mobile":o}),children:[0===l&&Ct(um,{rows:t,columns:a,defaultOrderBy:i}),1===l&&Ct(yh,{data:t})]})]})},mm=()=>{const{isMobile:e}=Wr(),[t,n]=qh(10,"topN"),[a,o]=qh("10m","maxLifetime"),{data:l,error:s,loading:c,fetch:u}=cm({topN:t,maxLifetime:a}),d=(0,r.useMemo)((()=>{const e=a.trim().split(" ").reduce(((e,t)=>{const n=Yt(t);return n?{...e,...n}:{...e}}),{});return!!i().duration(e).asMilliseconds()}),[a]),h=(0,r.useMemo)((()=>!!t&&t<1),[t]),m=(0,r.useMemo)((()=>h?"Number must be bigger than zero":""),[h]),p=(0,r.useMemo)((()=>d?"":"Invalid duration value"),[d]),f=e=>{if(!l)return e;const t=l[e];return"number"===typeof t?vd(t,t,t):t||e},v=e=>{"Enter"===e.key&&u()};return(0,r.useEffect)((()=>{l&&(t||n(+l.topN),a||o(l.maxLifetime))}),[l]),(0,r.useEffect)((()=>(u(),window.addEventListener("popstate",u),()=>{window.removeEventListener("popstate",u)})),[]),Ct("div",{className:xr()({"vm-top-queries":!0,"vm-top-queries_mobile":e}),children:[c&&Ct(rl,{containerStyles:{height:"500px"}}),Ct("div",{className:xr()({"vm-top-queries-controls":!0,"vm-block":!0,"vm-block_mobile":e}),children:[Ct("div",{className:"vm-top-queries-controls-fields",children:[Ct("div",{className:"vm-top-queries-controls-fields__item",children:Ct(Ha,{label:"Max lifetime",value:a,error:p,helperText:"For example ".concat("30ms, 15s, 3d4h, 1y2w"),onChange:e=>{o(e)},onKeyDown:v})}),Ct("div",{className:"vm-top-queries-controls-fields__item",children:Ct(Ha,{label:"Number of returned queries",type:"number",value:t||"",error:m,onChange:e=>{n(+e)},onKeyDown:v})})]}),Ct("div",{className:xr()({"vm-top-queries-controls-bottom":!0,"vm-top-queries-controls-bottom_mobile":e}),children:[Ct("div",{className:"vm-top-queries-controls-bottom__info",children:["VictoriaMetrics tracks the last\xa0",Ct(fa,{title:"search.queryStats.lastQueriesCount",children:Ct("b",{children:f("search.queryStats.lastQueriesCount")})}),"\xa0queries with durations at least\xa0",Ct(fa,{title:"search.queryStats.minQueryDuration",children:Ct("b",{children:f("search.queryStats.minQueryDuration")})})]}),Ct("div",{className:"vm-top-queries-controls-bottom__button",children:Ct(aa,{startIcon:Ct(Yn,{}),onClick:u,children:"Execute"})})]})]}),s&&Ct(Qr,{variant:"error",children:s}),l&&Ct(xt.FK,{children:Ct("div",{className:"vm-top-queries-panels",children:[Ct(hm,{rows:l.topBySumDuration,title:"Queries with most summary time to execute",columns:[{key:"query"},{key:"sumDurationSeconds",title:"sum duration, sec"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}],defaultOrderBy:"sumDurationSeconds"}),Ct(hm,{rows:l.topByAvgDuration,title:"Most heavy queries",columns:[{key:"query"},{key:"avgDurationSeconds",title:"avg duration, sec"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}],defaultOrderBy:"avgDurationSeconds"}),Ct(hm,{rows:l.topByCount,title:"Most frequently executed queries",columns:[{key:"query"},{key:"timeRange",sortBy:"timeRangeSeconds",title:"query time interval"},{key:"count"}]})]})})]})},pm={"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)"},fm={"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(80,80,80,0.9)","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)"},vm=()=>{const[e,t]=(0,r.useState)(gt()),n=e=>{t(e.matches)};return(0,r.useEffect)((()=>{const e=window.matchMedia("(prefers-color-scheme: dark)");return e.addEventListener("change",n),()=>e.removeEventListener("change",n)}),[]),e},gm=["primary","secondary","error","warning","info","success"],ym=e=>{let{onLoaded:t}=e;const n=Ke(),{palette:a={}}=We(),{theme:i}=Nt(),o=vm(),l=At(),s=Ar(),[c,u]=(0,r.useState)({[mt.dark]:pm,[mt.light]:fm,[mt.system]:gt()?pm:fm}),d=()=>{const{innerWidth:e,innerHeight:t}=window,{clientWidth:n,clientHeight:r}=document.documentElement;vt("scrollbar-width","".concat(e-n,"px")),vt("scrollbar-height","".concat(t-r,"px")),vt("vh","".concat(.01*t,"px"))},h=()=>{gm.forEach(((e,n)=>{const r=(e=>{let 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"})(ft("color-".concat(e)));vt("".concat(e,"-text"),r),n===gm.length-1&&(l({type:"SET_DARK_THEME"}),t(!0))}))},m=()=>{const e=Xe("THEME")||mt.system,t=c[e];Object.entries(t).forEach((e=>{let[t,n]=e;vt(t,n)})),h(),n&&(gm.forEach((e=>{const t=a[e];t&&vt("color-".concat(e),t)})),h())};return(0,r.useEffect)((()=>{d(),m()}),[c]),(0,r.useEffect)(d,[s]),(0,r.useEffect)((()=>{const e=gt()?pm:fm;c[mt.system]!==e?u((t=>({...t,[mt.system]:e}))):m()}),[i,o]),(0,r.useEffect)((()=>{n&&l({type:"SET_THEME",payload:mt.light})}),[]),null},_m=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)(!1),i=(0,r.useRef)(document.body),o=e=>{e.preventDefault(),e.stopPropagation(),"dragenter"===e.type||"dragover"===e.type?a(!0):"dragleave"===e.type&&a(!1)};return Nr("dragenter",o,i),Nr("dragleave",o,i),Nr("dragover",o,i),Nr("drop",(e=>{var n;e.preventDefault(),e.stopPropagation(),a(!1),null!==e&&void 0!==e&&null!==(n=e.dataTransfer)&&void 0!==n&&n.files&&e.dataTransfer.files[0]&&(e=>{const n=Array.from(e||[]);t(n)})(e.dataTransfer.files)}),i),Nr("paste",(e=>{var n;const r=null===(n=e.clipboardData)||void 0===n?void 0:n.items;if(!r)return;const a=Array.from(r).filter((e=>"application/json"===e.type)).map((e=>e.getAsFile())).filter((e=>null!==e));t(a)}),i),{files:e,dragging:n}},bm=e=>{let{onOpenModal:t,onChange:n}=e;return Ct("div",{className:"vm-upload-json-buttons",children:[Ct(aa,{variant:"outlined",onClick:t,children:"Paste JSON"}),Ct(aa,{children:["Upload Files",Ct("input",{id:"json",type:"file",accept:"application/json",multiple:!0,title:" ",onChange:n})]})]})},wm=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)([]),i=(0,r.useMemo)((()=>!!e.length),[e]),{value:o,setTrue:l,setFalse:s}=oa(!1),c=function(e){let t=arguments.length>1&&void 0!==arguments[1]?arguments[1]:"";a((n=>[{filename:t,text:": ".concat(e.message)},...n]))},u=(e,n)=>{try{const r=JSON.parse(e),a=r.trace||r;if(!a.duration_msec)return void c(new Error(ht.traceNotFound),n);const i=new Go(a,n);t((e=>[i,...e]))}catch(rp){rp instanceof Error&&c(rp,n)}},d=e=>{e.map((e=>{const t=new FileReader,n=(null===e||void 0===e?void 0:e.name)||"";t.onload=e=>{var t;const r=String(null===(t=e.target)||void 0===t?void 0:t.result);u(r,n)},t.readAsText(e)}))},h=e=>{a([]);const t=Array.from(e.target.files||[]);d(t),e.target.value=""},m=e=>()=>{(e=>{a((t=>t.filter(((t,n)=>n!==e))))})(e)},{files:p,dragging:f}=_m();return(0,r.useEffect)((()=>{d(p)}),[p]),Ct("div",{className:"vm-trace-page",children:[Ct("div",{className:"vm-trace-page-header",children:[Ct("div",{className:"vm-trace-page-header-errors",children:n.map(((e,t)=>Ct("div",{className:"vm-trace-page-header-errors-item",children:[Ct(Qr,{variant:"error",children:[Ct("b",{className:"vm-trace-page-header-errors-item__filename",children:e.filename}),Ct("span",{children:e.text})]}),Ct(aa,{className:"vm-trace-page-header-errors-item__close",startIcon:Ct(On,{}),variant:"text",color:"error",onClick:m(t)})]},"".concat(e,"_").concat(t))))}),Ct("div",{children:i&&Ct(bm,{onOpenModal:l,onChange:h})})]}),i&&Ct("div",{children:Ct(ml,{jsonEditor:!0,traces:e,onDeleteClick:n=>{const r=e.filter((e=>e.idValue!==n.idValue));t([...r])}})}),!i&&Ct("div",{className:"vm-trace-page-preview",children:[Ct("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",Ct("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.","\n","Attach files by dragging & dropping, selecting or pasting them."]}),Ct(bm,{onOpenModal:l,onChange:h})]}),o&&Ct(pa,{title:"Paste JSON",onClose:s,children:Ct(hl,{editable:!0,displayTitle:!0,defaultTile:"JSON ".concat(e.length+1),onClose:s,onUpload:u})}),f&&Ct("div",{className:"vm-trace-page__dropzone"})]})},km=e=>{const{serverUrl:t}=Nt(),{period:n}=vn(),[a,i]=(0,r.useState)([]),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)(),u=(0,r.useMemo)((()=>((e,t,n)=>{const 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(0,r.useEffect)((()=>{if(!e)return;(async()=>{l(!0);try{const e=await fetch(u),t=await e.json(),n=t.data||[];i(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?c(void 0):c("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(rp){rp instanceof Error&&c("".concat(rp.name,": ").concat(rp.message))}l(!1)})().catch(console.error)}),[u]),{instances:a,isLoading:o,error:s}},xm=(e,t)=>{const{serverUrl:n}=Nt(),{period:a}=vn(),[i,o]=(0,r.useState)([]),[l,s]=(0,r.useState)(!1),[c,u]=(0,r.useState)(),d=(0,r.useMemo)((()=>((e,t,n,r)=>{const a=Object.entries({job:n,instance:r}).filter((e=>e[1])).map((e=>{let[t,n]=e;return"".concat(t,"=").concat(JSON.stringify(n))})).join(","),i="{".concat(a,"}");return"".concat(e,"/api/v1/label/__name__/values?match[]=").concat(encodeURIComponent(i),"&start=").concat(t.start,"&end=").concat(t.end)})(n,a,e,t)),[n,a,e,t]);return(0,r.useEffect)((()=>{if(!e)return;(async()=>{s(!0);try{const e=await fetch(d),t=await e.json(),n=t.data||[];o(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?u(void 0):u("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(rp){rp instanceof Error&&u("".concat(rp.name,": ").concat(rp.message))}s(!1)})().catch(console.error)}),[d]),{names:i,isLoading:l,error:c}},Sm=e=>{let{name:t,job:n,instance:a,rateEnabled:i,isBucket:o,height:l}=e;const{isMobile:s}=Wr(),{customStep:c,yaxis:u}=Ur(),{period:d}=vn(),h=Br(),m=gn(),p=Kt(d.end-d.start),f=Wt(c),v=Jt(10*f*1e3),[g,y]=(0,r.useState)(!1),[_,b]=(0,r.useState)(!1),w=g&&c===p?v:c,k=(0,r.useMemo)((()=>{const e=Object.entries({job:n,instance:a}).filter((e=>e[1])).map((e=>{let[t,n]=e;return"".concat(t,"=").concat(JSON.stringify(n))}));e.push("__name__=".concat(JSON.stringify(t))),"node_cpu_seconds_total"==t&&e.push('mode!="idle"');const r="{".concat(e.join(","),"}");if(o)return"sum(rate(".concat(r,")) by (vmrange, le)");const l=i?"rollup_rate(".concat(r,")"):"rollup(".concat(r,")");return"\nwith (q = ".concat(l,') (\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,a,i,o]),{isLoading:x,graphData:S,error:C,queryErrors:E,warning:N,isHistogram:A}=nl({predefinedQuery:[k],visible:!0,customStep:w,showAllSeries:_});return(0,r.useEffect)((()=>{y(A)}),[A]),Ct("div",{className:xr()({"vm-explore-metrics-graph":!0,"vm-explore-metrics-graph_mobile":s}),children:[x&&Ct(rl,{}),C&&Ct(Qr,{variant:"error",children:C}),E[0]&&Ct(Qr,{variant:"error",children:E[0]}),N&&Ct(fl,{warning:N,query:[k],onChange:b}),S&&d&&Ct(hh,{data:S,period:d,customStep:w,query:[k],yaxis:u,setYaxisLimits:e=>{h({type:"SET_YAXIS_LIMITS",payload:e})},setPeriod:e=>{let{from:t,to:n}=e;m({type:"SET_PERIOD",payload:{from:t,to:n}})},showLegend:!1,height:l,isHistogram:A})]})},Cm=e=>{let{name:t,index:n,length:r,isBucket:a,rateEnabled:i,onChangeRate:o,onRemoveItem:l,onChangeOrder:s}=e;const{isMobile:c}=Wr(),{value:u,setTrue:d,setFalse:h}=oa(!1),m=()=>{l(t)},p=()=>{s(t,n,n+1)},f=()=>{s(t,n,n-1)};return Ct("div",c?{className:"vm-explore-metrics-item-header vm-explore-metrics-item-header_mobile",children:[Ct("div",{className:"vm-explore-metrics-item-header__name",children:t}),Ct(aa,{variant:"text",size:"small",startIcon:Ct(dr,{}),onClick:d,ariaLabel:"open panel settings"}),u&&Ct(pa,{title:t,onClose:h,children:Ct("div",{className:"vm-explore-metrics-item-header-modal",children:[Ct("div",{className:"vm-explore-metrics-item-header-modal-order",children:[Ct(aa,{startIcon:Ct(Xn,{}),variant:"outlined",onClick:f,disabled:0===n,ariaLabel:"move graph up"}),Ct("p",{children:["position:",Ct("span",{className:"vm-explore-metrics-item-header-modal-order__index",children:["#",n+1]})]}),Ct(aa,{endIcon:Ct(Jn,{}),variant:"outlined",onClick:p,disabled:n===r-1,ariaLabel:"move graph down"})]}),!a&&Ct("div",{className:"vm-explore-metrics-item-header-modal__rate",children:[Ct(jo,{label:Ct("span",{children:["enable ",Ct("code",{children:"rate()"})]}),value:i,onChange:o,fullWidth:!0}),Ct("p",{children:"calculates the average per-second speed of metrics change"})]}),Ct(aa,{startIcon:Ct(On,{}),color:"error",variant:"outlined",onClick:m,fullWidth:!0,children:"Remove graph"})]})})]}:{className:"vm-explore-metrics-item-header",children:[Ct("div",{className:"vm-explore-metrics-item-header-order",children:[Ct(fa,{title:"move graph up",children:Ct(aa,{className:"vm-explore-metrics-item-header-order__up",startIcon:Ct(Hn,{}),variant:"text",color:"gray",size:"small",onClick:f,ariaLabel:"move graph up"})}),Ct("div",{className:"vm-explore-metrics-item-header__index",children:["#",n+1]}),Ct(fa,{title:"move graph down",children:Ct(aa,{className:"vm-explore-metrics-item-header-order__down",startIcon:Ct(Hn,{}),variant:"text",color:"gray",size:"small",onClick:p,ariaLabel:"move graph down"})})]}),Ct("div",{className:"vm-explore-metrics-item-header__name",children:t}),!a&&Ct("div",{className:"vm-explore-metrics-item-header__rate",children:Ct(fa,{title:"calculates the average per-second speed of metric's change",children:Ct(jo,{label:Ct("span",{children:["enable ",Ct("code",{children:"rate()"})]}),value:i,onChange:o})})}),Ct("div",{className:"vm-explore-metrics-item-header__close",children:Ct(fa,{title:"close graph",children:Ct(aa,{startIcon:Ct(On,{}),variant:"text",color:"gray",size:"small",onClick:m,ariaLabel:"close graph"})})})]})},Em=e=>{let{name:t,job:n,instance:a,index:i,length:o,size:l,onRemoveItem:s,onChangeOrder:c}=e;const u=(0,r.useMemo)((()=>/_sum?|_total?|_count?/.test(t)),[t]),d=(0,r.useMemo)((()=>/_bucket?/.test(t)),[t]),[h,m]=(0,r.useState)(u),p=Ar(),f=(0,r.useMemo)(l.height,[l,p]);return(0,r.useEffect)((()=>{m(u)}),[n]),Ct("div",{className:"vm-explore-metrics-item vm-block vm-block_empty-padding",children:[Ct(Cm,{name:t,index:i,length:o,isBucket:d,rateEnabled:h,size:l.id,onChangeRate:m,onRemoveItem:s,onChangeOrder:c}),Ct(Sm,{name:t,job:n,instance:a,rateEnabled:h,isBucket:d,height:f},"".concat(t,"_").concat(n,"_").concat(a,"_").concat(h))]})},Nm=e=>{let{values:t,onRemoveItem:n}=e;const{isMobile:r}=Wr();return r?Ct("span",{className:"vm-select-input-content__counter",children:["selected ",t.length]}):Ct(xt.FK,{children:t.map((e=>{return Ct("div",{className:"vm-select-input-content__selected",children:[Ct("span",{children:e}),Ct("div",{onClick:(t=e,e=>{n(t),e.stopPropagation()}),children:Ct(On,{})})]},e);var t}))})},Am=e=>{let{value:t,list:n,label:a,placeholder:i,noOptionsText:o,clearable:l=!1,searchable:s=!1,autofocus:c,disabled:u,onChange:d}=e;const{isDarkTheme:h}=Nt(),{isMobile:m}=Wr(),[p,f]=(0,r.useState)(""),v=(0,r.useRef)(null),[g,y]=(0,r.useState)(null),[_,b]=(0,r.useState)(!1),w=(0,r.useRef)(null),k=Array.isArray(t),x=Array.isArray(t)?t:void 0,S=m&&k&&!(null===x||void 0===x||!x.length),C=(0,r.useMemo)((()=>_?p:Array.isArray(t)?"":t),[t,p,_,k]),E=(0,r.useMemo)((()=>_?p||"(.+)":""),[p,_]),N=()=>{w.current&&w.current.blur()},A=()=>{b(!1),N()},M=e=>{f(""),d(e),k||A(),k&&w.current&&w.current.focus()};return(0,r.useEffect)((()=>{f(""),_&&w.current&&w.current.focus(),_||N()}),[_,w]),(0,r.useEffect)((()=>{c&&w.current&&!m&&w.current.focus()}),[c,w]),Nr("keyup",(e=>{w.current!==e.target&&b(!1)})),ra(v,A,g),Ct("div",{className:xr()({"vm-select":!0,"vm-select_dark":h,"vm-select_disabled":u}),children:[Ct("div",{className:"vm-select-input",onClick:e=>{e.target instanceof HTMLInputElement||u||b((e=>!e))},ref:v,children:[Ct("div",{className:"vm-select-input-content",children:[!(null===x||void 0===x||!x.length)&&Ct(Nm,{values:x,onRemoveItem:M}),!S&&Ct("input",{value:C,type:"text",placeholder:i,onInput:e=>{f(e.target.value)},onFocus:()=>{u||b(!0)},onBlur:()=>{n.includes(p)&&d(p)},ref:w,readOnly:m||!s})]}),a&&Ct("span",{className:"vm-text-field__label",children:a}),l&&t&&Ct("div",{className:"vm-select-input__icon",onClick:(e=>t=>{M(e),t.stopPropagation()})(""),children:Ct(On,{})}),Ct("div",{className:xr()({"vm-select-input__icon":!0,"vm-select-input__icon_open":_}),children:Ct(Vn,{})})]}),Ct(Ti,{label:a,value:E,options:n.map((e=>({value:e}))),anchor:v,selected:x,minLength:1,fullWidth:!0,noOptionsText:o,onSelect:M,onOpenAutocomplete:b,onChangeWrapperRef:y})]})},Mm=lt.map((e=>e.id)),Tm=e=>{let{jobs:t,instances:n,names:a,job:i,instance:o,size:l,selectedMetrics:s,onChangeJob:c,onChangeInstance:u,onToggleMetric:d,onChangeSize:h}=e;const m=(0,r.useMemo)((()=>i?"":"No instances. Please select job"),[i]),p=(0,r.useMemo)((()=>i?"":"No metric names. Please select job"),[i]),{isMobile:f}=Wr(),{value:v,toggle:g,setFalse:y}=oa("false"!==Xe("EXPLORE_METRICS_TIPS"));return(0,r.useEffect)((()=>{Je("EXPLORE_METRICS_TIPS","".concat(v))}),[v]),Ct(xt.FK,{children:[Ct("div",{className:xr()({"vm-explore-metrics-header":!0,"vm-explore-metrics-header_mobile":f,"vm-block":!0,"vm-block_mobile":f}),children:[Ct("div",{className:"vm-explore-metrics-header__job",children:Ct(Am,{value:i,list:t,label:"Job",placeholder:"Please select job",onChange:c,autofocus:!i&&!!t.length&&!f,searchable:!0})}),Ct("div",{className:"vm-explore-metrics-header__instance",children:Ct(Am,{value:o,list:n,label:"Instance",placeholder:"Please select instance",onChange:u,noOptionsText:m,clearable:!0,searchable:!0})}),Ct("div",{className:"vm-explore-metrics-header__size",children:[Ct(Am,{label:"Size graphs",value:l,list:Mm,onChange:h}),Ct(fa,{title:"".concat(v?"Hide":"Show"," tip"),children:Ct(aa,{variant:"text",color:v?"warning":"gray",startIcon:Ct(mr,{}),onClick:g,ariaLabel:"visibility tips"})})]}),Ct("div",{className:"vm-explore-metrics-header-metrics",children:Ct(Am,{label:"Metrics",value:s,list:a,placeholder:"Search metric name",onChange:d,noOptionsText:p,clearable:!0,searchable:!0})})]}),v&&Ct(Qr,{variant:"warning",children:Ct("div",{className:"vm-explore-metrics-header-description",children:[Ct("p",{children:["Please note: this page is solely designed for exploring Prometheus metrics. Prometheus metrics always contain ",Ct("code",{children:"job"})," and ",Ct("code",{children:"instance"})," labels (see ",Ct("a",{className:"vm-link vm-link_colored",href:"https://prometheus.io/docs/concepts/jobs_instances/",children:"these docs"}),"), and this page relies on them as filters. ",Ct("br",{}),"Please use this page for Prometheus metrics only, in accordance with their naming conventions."]}),Ct(aa,{variant:"text",size:"small",startIcon:Ct(On,{}),onClick:y,ariaLabel:"close tips"})]})})]})},Lm=ct("job",""),Pm=ct("instance",""),Om=ct("metrics",""),Im=ct("size",""),Rm=lt.find((e=>Im?e.id===Im:e.isDefault))||lt[0],Dm=()=>{const[e,t]=(0,r.useState)(Lm),[n,a]=(0,r.useState)(Pm),[i,o]=(0,r.useState)(Om?Om.split("&"):[]),[l,s]=(0,r.useState)(Rm);(e=>{let{job:t,instance:n,metrics:a,size:i}=e;const{duration:o,relativeTime:l,period:{date:s}}=vn(),{customStep:c}=Ur(),{setSearchParamsFromKeys:u}=ii(),d=()=>{const e=al({"g0.range_input":o,"g0.end_input":s,"g0.step_input":c,"g0.relative_time":l,size:i,job:t,instance:n,metrics:a});u(e)};(0,r.useEffect)(d,[o,l,s,c,t,n,a,i]),(0,r.useEffect)(d,[])})({job:e,instance:n,metrics:i.join("&"),size:l.id});const{jobs:c,isLoading:u,error:d}=(()=>{const{serverUrl:e}=Nt(),{period:t}=vn(),[n,a]=(0,r.useState)([]),[i,o]=(0,r.useState)(!1),[l,s]=(0,r.useState)(),c=(0,r.useMemo)((()=>((e,t)=>"".concat(e,"/api/v1/label/job/values?start=").concat(t.start,"&end=").concat(t.end))(e,t)),[e,t]);return(0,r.useEffect)((()=>{(async()=>{o(!0);try{const e=await fetch(c),t=await e.json(),n=t.data||[];a(n.sort(((e,t)=>e.localeCompare(t)))),e.ok?s(void 0):s("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(rp){rp instanceof Error&&s("".concat(rp.name,": ").concat(rp.message))}o(!1)})().catch(console.error)}),[c]),{jobs:n,isLoading:i,error:l}})(),{instances:h,isLoading:m,error:p}=km(e),{names:f,isLoading:v,error:g}=xm(e,n),y=(0,r.useMemo)((()=>u||m||v),[u,m,v]),_=(0,r.useMemo)((()=>d||p||g),[d,p,g]),b=e=>{o(e?t=>t.includes(e)?t.filter((t=>t!==e)):[...t,e]:[])},w=(e,t,n)=>{const r=n>i.length-1;n<0||r||o((e=>{const r=[...e],[a]=r.splice(t,1);return r.splice(n,0,a),r}))};return(0,r.useEffect)((()=>{n&&h.length&&!h.includes(n)&&a("")}),[h,n]),Ct("div",{className:"vm-explore-metrics",children:[Ct(Tm,{jobs:c,instances:h,names:f,job:e,size:l.id,instance:n,selectedMetrics:i,onChangeJob:t,onChangeSize:e=>{const t=lt.find((t=>t.id===e));t&&s(t)},onChangeInstance:a,onToggleMetric:b}),y&&Ct(rl,{}),_&&Ct(Qr,{variant:"error",children:_}),!e&&Ct(Qr,{variant:"info",children:"Please select job to see list of metric names."}),e&&!i.length&&Ct(Qr,{variant:"info",children:"Please select metric names to see the graphs."}),Ct("div",{className:"vm-explore-metrics-body",children:i.map(((t,r)=>Ct(Em,{name:t,job:e,instance:n,index:r,length:i.length,size:l,onRemoveItem:b,onChangeOrder:w},t)))})]})},zm=()=>{const t=Uo();return Ct("div",{className:"vm-preview-icons",children:Object.entries(e).map((e=>{let[n,r]=e;return Ct("div",{className:"vm-preview-icons-item",onClick:(a=n,async()=>{await t("<".concat(a,"/>"),"<".concat(a,"/> has been copied"))}),children:[Ct("div",{className:"vm-preview-icons-item__svg",children:r()}),Ct("div",{className:"vm-preview-icons-item__name",children:"<".concat(n,"/>")})]},n);var a}))})};var Fm=function(e){return e.copy="Copy",e.copied="Copied",e}(Fm||{});const jm=e=>{let{code:t}=e;const[n,a]=(0,r.useState)(Fm.copy);return(0,r.useEffect)((()=>{let e=null;return n===Fm.copied&&(e=setTimeout((()=>a(Fm.copy)),1e3)),()=>{e&&clearTimeout(e)}}),[n]),Ct("code",{className:"vm-code-example",children:[t,Ct("div",{className:"vm-code-example__copy",children:Ct(fa,{title:n,children:Ct(aa,{size:"small",variant:"text",onClick:()=>{navigator.clipboard.writeText(t),a(Fm.copied)},startIcon:Ct(ar,{}),ariaLabel:"close"})})})]})},Hm=()=>Ct("a",{className:"vm-link vm-link_colored",href:"https://docs.victoriametrics.com/MetricsQL.html",target:"_blank",rel:"help noreferrer",children:"MetricsQL"}),Vm=()=>Ct("a",{className:"vm-link vm-link_colored",href:"https://grafana.com/grafana/dashboards/1860-node-exporter-full/",target:"_blank",rel:"help noreferrer",children:"Node Exporter Full"}),$m=()=>Ct("section",{className:"vm-with-template-tutorial",children:[Ct("h2",{className:"vm-with-template-tutorial__title",children:["Tutorial for WITH expressions in ",Ct(Hm,{})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Let's look at the following real query from ",Ct(Vm,{})," dashboard:"]}),Ct(jm,{code:'(\n (\n node_memory_MemTotal_bytes{instance=~"$node:$port", job=~"$job"}\n -\n node_memory_MemFree_bytes{instance=~"$node:$port", job=~"$job"}\n )\n /\n node_memory_MemTotal_bytes{instance=~"$node:$port", job=~"$job"}\n) * 100'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"It is clear the query calculates the percentage of used memory for the given $node, $port and $job. Isn't it? :)"})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"What's wrong with this query? Copy-pasted label filters for distinct timeseries which makes it easy to mistype these filters during modification. Let's simplify the query with WITH expressions:"}),Ct(jm,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"}\n)\n(\n node_memory_MemTotal_bytes{commonFilters}\n -\n node_memory_MemFree_bytes{commonFilters}\n)\n /\nnode_memory_MemTotal_bytes{commonFilters} * 100'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Now label filters are located in a single place instead of three distinct places. The query mentions node_memory_MemTotal_bytes metric twice and ","{commonFilters}"," three times. WITH expressions may improve this:"]}),Ct(jm,{code:'WITH (\n my_resource_utilization(free, limit, filters) = (limit{filters} - free{filters}) / limit{filters} * 100\n)\nmy_resource_utilization(\n node_memory_MemFree_bytes,\n node_memory_MemTotal_bytes,\n {instance=~"$node:$port",job=~"$job"},\n)'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Now the template function my_resource_utilization() may be used for monitoring arbitrary resources - memory, CPU, network, storage, you name it."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["Let's take another nice query from ",Ct(Vm,{})," dashboard:"]}),Ct(jm,{code:'(\n (\n (\n count(\n count(node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}) by (cpu)\n )\n )\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',instance=~"$node:$port",job=~"$job"}[5m]))\n )\n )\n *\n 100\n)\n /\ncount(\n count(node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"}) by (cpu)\n)'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Do you understand what does this mess do? Is it manageable? :) WITH expressions are happy to help in a few iterations."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"1. Extract common filters used in multiple places into a commonFilters variable:"}),Ct(jm,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"}\n)\n(\n (\n (\n count(\n count(node_cpu_seconds_total{commonFilters}) by (cpu)\n )\n )\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n )\n )\n *\n 100\n)\n /\ncount(\n count(node_cpu_seconds_total{commonFilters}) by (cpu)\n)'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:'2. Extract "count(count(...) by (cpu))" into cpuCount variable:'}),Ct(jm,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(node_cpu_seconds_total{commonFilters}) by (cpu))\n)\n(\n (\n cpuCount\n -\n avg(\n sum by (mode) (rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n )\n )\n *\n 100\n) / cpuCount'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"3. Extract rate(...) part into cpuIdle variable, since it is clear now that this part calculates the number of idle CPUs:"}),Ct(jm,{code:'WITH (\n commonFilters = {instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(node_cpu_seconds_total{commonFilters}) by (cpu)),\n cpuIdle = sum(rate(node_cpu_seconds_total{mode=\'idle\',commonFilters}[5m]))\n)\n((cpuCount - cpuIdle) * 100) / cpuCount'})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:["4. Put node_cpu_seconds_total","{commonFilters}"," into its own varialbe with the name cpuSeconds:"]}),Ct(jm,{code:'WITH (\n cpuSeconds = node_cpu_seconds_total{instance=~"$node:$port",job=~"$job"},\n cpuCount = count(count(cpuSeconds) by (cpu)),\n cpuIdle = sum(rate(cpuSeconds{mode=\'idle\'}[5m]))\n)\n((cpuCount - cpuIdle) * 100) / cpuCount'}),Ct("p",{className:"vm-with-template-tutorial-section__text",children:"Now the query became more clear comparing to the initial query."})]}),Ct("div",{className:"vm-with-template-tutorial-section",children:[Ct("p",{className:"vm-with-template-tutorial-section__text",children:"WITH expressions may be nested and may be put anywhere. Try expanding the following query:"}),Ct(jm,{code:"WITH (\n f(a, b) = WITH (\n f1(x) = b-x,\n f2(x) = x+x\n ) f1(a)*f2(b)\n) f(foo, with(x=bar) x)"})]})]}),Um=()=>{const{serverUrl:e}=Nt(),[t,n]=je(),[a,i]=(0,r.useState)(""),[o,l]=(0,r.useState)(!1),[s,c]=(0,r.useState)();return{data:a,error:s,loading:o,expand:async r=>{t.set("expr",r),n(t);const a=((e,t)=>"".concat(e,"/expand-with-exprs?query=").concat(encodeURIComponent(t),"&format=json"))(e,r);l(!0);try{const e=await fetch(a),t=await e.json();i((null===t||void 0===t?void 0:t.expr)||""),c(String(t.error||""))}catch(rp){rp instanceof Error&&"AbortError"!==rp.name&&c("".concat(rp.name,": ").concat(rp.message))}l(!1)}}},Bm=()=>{const[e]=je(),{data:t,loading:n,error:a,expand:i}=Um(),[o,l]=(0,r.useState)(e.get("expr")||""),s=()=>{i(o)};return(0,r.useEffect)((()=>{o&&i(o)}),[]),Ct("section",{className:"vm-with-template",children:[n&&Ct(rl,{}),Ct("div",{className:"vm-with-template-body vm-block",children:[Ct("div",{className:"vm-with-template-body__expr",children:Ct(Ha,{type:"textarea",label:"MetricsQL query with optional WITH expressions",value:o,error:a,autofocus:!0,onEnter:s,onChange:e=>{l(e)}})}),Ct("div",{className:"vm-with-template-body__result",children:Ct(Ha,{type:"textarea",label:"MetricsQL query after expanding WITH expressions and applying other optimizations",value:t,disabled:!0})}),Ct("div",{className:"vm-with-template-body-top",children:Ct(aa,{variant:"contained",onClick:s,startIcon:Ct(Yn,{}),children:"Expand"})})]}),Ct("div",{className:"vm-block",children:Ct($m,{})})]})},qm=()=>{const{serverUrl:e}=Nt(),[t,n]=(0,r.useState)(null),[a,i]=(0,r.useState)(!1),[o,l]=(0,r.useState)();return{data:t,error:o,loading:a,fetchData:async(t,r)=>{const a=((e,t,n)=>{const r=["format=json","relabel_configs=".concat(encodeURIComponent(t)),"metric=".concat(encodeURIComponent(n))];return"".concat(e,"/metric-relabel-debug?").concat(r.join("&"))})(e,t,r);i(!0);try{const e=await fetch(a),t=await e.json();n(t.error?null:t),l(String(t.error||""))}catch(rp){rp instanceof Error&&"AbortError"!==rp.name&&l("".concat(rp.name,": ").concat(rp.message))}i(!1)}}},Ym={config:'- if: \'{bar_label=~"b.*"}\'\n source_labels: [foo_label, bar_label]\n separator: "_"\n target_label: foobar\n- action: labeldrop\n regex: "foo_.*"\n- target_label: job\n replacement: "my-application-2"',labels:'{__name__="my_metric", bar_label="bar", foo_label="foo", job="my-application", instance="192.168.0.1"}'},Wm=()=>{const[e,t]=je(),{data:n,loading:a,error:i,fetchData:o}=qm(),[l,s]=qh("","config"),[c,u]=qh("","labels"),d=(0,r.useCallback)((()=>{o(l,c),e.set("config",l),e.set("labels",c),t(e)}),[l,c]);return(0,r.useEffect)((()=>{const t=e.get("config")||"",n=e.get("labels")||"";(n||t)&&(o(t,n),s(t),u(n))}),[]),Ct("section",{className:"vm-relabeling",children:[a&&Ct(rl,{}),Ct("div",{className:"vm-relabeling-header vm-block",children:[Ct("div",{className:"vm-relabeling-header-configs",children:Ct(Ha,{type:"textarea",label:"Relabel configs",value:l,autofocus:!0,onChange:e=>{s(e||"")},onEnter:d})}),Ct("div",{className:"vm-relabeling-header__labels",children:Ct(Ha,{type:"textarea",label:"Labels",value:c,onChange:e=>{u(e||"")},onEnter:d})}),Ct("div",{className:"vm-relabeling-header-bottom",children:[Ct("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/relabeling.html",rel:"help noreferrer",children:[Ct(Rn,{}),"Relabeling cookbook"]}),Ct("a",{className:"vm-link vm-link_with-icon",target:"_blank",href:"https://docs.victoriametrics.com/vmagent.html#relabeling",rel:"help noreferrer",children:[Ct(lr,{}),"Documentation"]}),Ct(aa,{variant:"text",onClick:()=>{const{config:n,labels:r}=Ym;s(n),u(r),o(n,r),e.set("config",n),e.set("labels",r),t(e)},children:"Try example"}),Ct(aa,{variant:"contained",onClick:d,startIcon:Ct(Yn,{}),children:"Submit"})]})]}),i&&Ct(Qr,{variant:"error",children:i}),n&&Ct("div",{className:"vm-relabeling-steps vm-block",children:[n.originalLabels&&Ct("div",{className:"vm-relabeling-steps-item",children:Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Original labels:"}),Ct("code",{dangerouslySetInnerHTML:{__html:n.originalLabels}})]})}),n.steps.map(((e,t)=>Ct("div",{className:"vm-relabeling-steps-item",children:[Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Step:"}),t+1]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Relabeling Rule:"}),Ct("code",{children:Ct("pre",{children:e.rule})})]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Input Labels:"}),Ct("code",{children:Ct("pre",{dangerouslySetInnerHTML:{__html:e.inLabels}})})]}),Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Output labels:"}),Ct("code",{children:Ct("pre",{dangerouslySetInnerHTML:{__html:e.outLabels}})})]})]},t))),n.resultingLabels&&Ct("div",{className:"vm-relabeling-steps-item",children:Ct("div",{className:"vm-relabeling-steps-item__row",children:[Ct("span",{children:"Resulting labels:"}),Ct("code",{dangerouslySetInnerHTML:{__html:n.resultingLabels}})]})})]})]})},Km=e=>{let{rows:t,columns:n,defaultOrderBy:a,copyToClipboard:i,paginationOffset:o}=e;const[l,s]=(0,r.useState)(a),[c,u]=(0,r.useState)("desc"),[d,h]=(0,r.useState)(null),m=(0,r.useMemo)((()=>{const{startIndex:e,endIndex:n}=o;return Gh(t,Zh(c,l)).slice(e,n)}),[t,l,c,o]),p=(e,t)=>async()=>{if(d!==t)try{await navigator.clipboard.writeText(String(e)),h(t)}catch(rp){console.error(rp)}};return(0,r.useEffect)((()=>{if(null===d)return;const e=setTimeout((()=>h(null)),2e3);return()=>clearTimeout(e)}),[d]),Ct("table",{className:"vm-table",children:[Ct("thead",{className:"vm-table-header",children:Ct("tr",{className:"vm-table__row vm-table__row_header",children:[n.map((e=>{return Ct("th",{className:"vm-table-cell vm-table-cell_header vm-table-cell_sort",onClick:(t=e.key,()=>{u((e=>"asc"===e&&l===t?"desc":"asc")),s(t)}),children:Ct("div",{className:"vm-table-cell__content",children:[Ct("div",{children:String(e.title||e.key)}),Ct("div",{className:xr()({"vm-table__sort-icon":!0,"vm-table__sort-icon_active":l===e.key,"vm-table__sort-icon_desc":"desc"===c&&l===e.key}),children:Ct(Vn,{})})]})},String(e.key));var t})),i&&Ct("th",{className:"vm-table-cell vm-table-cell_header"})]})}),Ct("tbody",{className:"vm-table-body",children:m.map(((e,t)=>Ct("tr",{className:"vm-table__row",children:[n.map((t=>Ct("td",{className:xr()({"vm-table-cell":!0,["".concat(t.className)]:t.className}),children:e[t.key]||"-"},String(t.key)))),i&&Ct("td",{className:"vm-table-cell vm-table-cell_right",children:e[i]&&Ct("div",{className:"vm-table-cell__content",children:Ct(fa,{title:d===t?"Copied":"Copy row",children:Ct(aa,{variant:"text",color:d===t?"success":"gray",size:"small",startIcon:Ct(d===t?er:ar,{}),onClick:p(e[i],t),ariaLabel:"copy row"})})})})]},t)))})]})},Qm=()=>{const{isMobile:e}=Wr(),{timezone:t}=vn(),{data:n,lastUpdated:a,isLoading:o,error:l,fetchData:s}=(()=>{const{serverUrl:e}=Nt(),[t,n]=(0,r.useState)([]),[a,o]=(0,r.useState)(i()().format(Pt)),[l,s]=(0,r.useState)(!1),[c,u]=(0,r.useState)(),d=(0,r.useMemo)((()=>"".concat(e,"/api/v1/status/active_queries")),[e]),h=async()=>{s(!0);try{const e=await fetch(d),t=await e.json();n(t.data),o(i()().format("HH:mm:ss:SSS")),e.ok?u(void 0):u("".concat(t.errorType,"\r\n").concat(null===t||void 0===t?void 0:t.error))}catch(rp){rp instanceof Error&&u("".concat(rp.name,": ").concat(rp.message))}s(!1)};return(0,r.useEffect)((()=>{h().catch(console.error)}),[d]),{data:t,lastUpdated:a,isLoading:l,error:c,fetchData:h}})(),c=(0,r.useMemo)((()=>n.map((e=>{const t=i()(e.start).tz().format(Lt),n=i()(e.end).tz().format(Lt);return{duration:e.duration,remote_addr:e.remote_addr,query:e.query,args:"".concat(t," to ").concat(n,", step=").concat(qt(e.step)),data:JSON.stringify(e,null,2)}}))),[n,t]),u=(0,r.useMemo)((()=>{if(null===c||void 0===c||!c.length)return[];const e=Object.keys(c[0]),t={remote_addr:"client address"},n=["data"];return e.filter((e=>!n.includes(e))).map((e=>({key:e,title:t[e]||e})))}),[c]);return Ct("div",{className:"vm-active-queries",children:[o&&Ct(rl,{}),Ct("div",{className:"vm-active-queries-header",children:[!c.length&&!l&&Ct(Qr,{variant:"info",children:"There are currently no active queries running"}),l&&Ct(Qr,{variant:"error",children:l}),Ct("div",{className:"vm-active-queries-header-controls",children:[Ct(aa,{variant:"contained",onClick:async()=>{s().catch(console.error)},startIcon:Ct(jn,{}),children:"Update"}),Ct("div",{className:"vm-active-queries-header__update-msg",children:["Last updated: ",a]})]})]}),!!c.length&&Ct("div",{className:xr()({"vm-block":!0,"vm-block_mobile":e}),children:Ct(Km,{rows:c,columns:u,defaultOrderBy:"duration",copyToClipboard:"data",paginationOffset:{startIndex:0,endIndex:1/0}})})]})},Zm=e=>{let{onClose:t,onUpload:n}=e;const{isMobile:a}=Wr(),[i,o]=(0,r.useState)(""),[l,s]=(0,r.useState)(""),c=(0,r.useMemo)((()=>{try{return JSON.parse(i),""}catch(rp){return rp instanceof Error?rp.message:"Unknown error"}}),[i]),u=()=>{s(c),c||(n(i),t())};return Ct("div",{className:xr()({"vm-json-form vm-json-form_one-field":!0,"vm-json-form_mobile vm-json-form_one-field_mobile":a}),children:[Ct(Ha,{value:i,label:"JSON",type:"textarea",error:l,autofocus:!0,onChange:e=>{s(""),o(e)},onEnter:u}),Ct("div",{className:"vm-json-form-footer",children:Ct("div",{className:"vm-json-form-footer__controls vm-json-form-footer__controls_right",children:[Ct(aa,{variant:"outlined",color:"error",onClick:t,children:"Cancel"}),Ct(aa,{variant:"contained",onClick:u,children:"apply"})]})})]})},Gm=e=>{let{data:t,period:n}=e;const{isMobile:a}=Wr(),{tableCompact:i}=Fr(),o=jr(),[l,s]=(0,r.useState)([]),[c,u]=(0,r.useState)(),[d,h]=(0,r.useState)(),[m,p]=(0,r.useState)(!1),[f,v]=(0,r.useState)([]),[g,y]=(0,r.useState)(),_=(0,r.useMemo)((()=>_h(d||[]).map((e=>e.key))),[d]),b=(0,r.useMemo)((()=>{const e=t.some((e=>"matrix"===e.data.resultType));return t.some((e=>"vector"===e.data.resultType))&&e?Tr:e?Tr.filter((e=>"chart"===e.value)):Tr.filter((e=>"chart"!==e.value))}),[t]),[w,k]=(0,r.useState)(b[0].value),{yaxis:x,spanGaps:S}=Ur(),C=Br(),E=e=>{C({type:"SET_YAXIS_LIMITS",payload:e})};return(0,r.useEffect)((()=>{const e="chart"===w?"matrix":"vector",n=t.filter((t=>t.data.resultType===e&&t.trace)).map((e=>{var t,n;return e.trace?new Go(e.trace,(null===e||void 0===e||null===(t=e.vmui)||void 0===t||null===(n=t.params)||void 0===n?void 0:n.query)||"Query"):null}));s(n.filter(Boolean))}),[t,w]),(0,r.useEffect)((()=>{const e=[],n=[],r=[];t.forEach(((t,a)=>{const i=t.data.result.map((e=>{var n,r,i;return{...e,group:Number(null!==(n=null===(r=t.vmui)||void 0===r||null===(i=r.params)||void 0===i?void 0:i.id)&&void 0!==n?n:a)+1}}));var o,l;"matrix"===t.data.resultType?(n.push(...i),e.push((null===(o=t.vmui)||void 0===o||null===(l=o.params)||void 0===l?void 0:l.query)||"Query")):r.push(...i)})),v(e),u(n),h(r)}),[t]),(0,r.useEffect)((()=>{p(!!c&&el(c))}),[c]),Ct("div",{className:xr()({"vm-query-analyzer-view":!0,"vm-query-analyzer-view_mobile":a}),children:[!!l.length&&Ct(ml,{traces:l,onDeleteClick:e=>{s((t=>t.filter((t=>t.idValue!==e.idValue))))}}),Ct("div",{className:xr()({"vm-block":!0,"vm-block_mobile":a}),children:[Ct("div",{className:"vm-custom-panel-body-header",children:[Ct("div",{className:"vm-custom-panel-body-header__tabs",children:Ct(Mr,{activeItem:w,items:b,onChange:e=>{k(e)}})}),Ct("div",{className:"vm-custom-panel-body-header__graph-controls",children:["chart"===w&&Ct(ba,{}),"chart"===w&&Ct(vh,{yaxis:x,setYaxisLimits:E,toggleEnableLimits:()=>{C({type:"TOGGLE_ENABLE_YAXIS_LIMITS"})},spanGaps:{value:S,onChange:e=>{C({type:"SET_SPAN_GAPS",payload:e})}}}),"table"===w&&Ct(Sh,{columns:_,defaultColumns:g,onChangeColumns:y,tableCompact:i,toggleTableCompact:()=>{o({type:"TOGGLE_TABLE_COMPACT"})}})]})]}),c&&n&&"chart"===w&&Ct(hh,{data:c,period:n,customStep:n.step||"1s",query:f,yaxis:x,setYaxisLimits:E,setPeriod:()=>null,height:a?.5*window.innerHeight:500,isHistogram:m,spanGaps:S}),d&&"code"===w&&Ct(yh,{data:d}),d&&"table"===w&&Ct(wh,{data:d,displayColumns:g})]})]})},Jm=e=>{var t,n;let{data:a,period:o}=e;const l=(0,r.useMemo)((()=>a.filter((e=>e.stats&&"matrix"===e.data.resultType))),[a]),s=(0,r.useMemo)((()=>{var e,t;return null===(e=a.find((e=>{var t;return null===e||void 0===e||null===(t=e.vmui)||void 0===t?void 0:t.comment})))||void 0===e||null===(t=e.vmui)||void 0===t?void 0:t.comment}),[a]),c=(0,r.useMemo)((()=>{if(!o)return"";const e=i()(1e3*o.start).tz().format(Lt),t=i()(1e3*o.end).tz().format(Lt);return"".concat(e," - ").concat(t)}),[o]),{value:u,setTrue:d,setFalse:h}=oa(!1);return Ct(xt.FK,{children:[Ct("div",{className:"vm-query-analyzer-info-header",children:[Ct(aa,{startIcon:Ct(Rn,{}),variant:"outlined",color:"warning",onClick:d,children:"Show report info"}),o&&Ct(xt.FK,{children:[Ct("div",{className:"vm-query-analyzer-info-header__period",children:[Ct(or,{})," step: ",o.step]}),Ct("div",{className:"vm-query-analyzer-info-header__period",children:[Ct($n,{})," ",c]})]})]}),u&&Ct(pa,{title:"Report info",onClose:h,children:Ct("div",{className:"vm-query-analyzer-info",children:[s&&Ct("div",{className:"vm-query-analyzer-info-item vm-query-analyzer-info-item_comment",children:[Ct("div",{className:"vm-query-analyzer-info-item__title",children:"Comment:"}),Ct("div",{className:"vm-query-analyzer-info-item__text",children:s})]}),l.map(((e,t)=>{var n;return Ct("div",{className:"vm-query-analyzer-info-item",children:[Ct("div",{className:"vm-query-analyzer-info-item__title",children:l.length>1?"Query ".concat(t+1,":"):"Stats:"}),Ct("div",{className:"vm-query-analyzer-info-item__text",children:[Object.entries(e.stats||{}).map((e=>{let[t,n]=e;return Ct("div",{children:[t,": ",null!==n&&void 0!==n?n:"-"]},t)})),"isPartial: ",String(null!==(n=e.isPartial)&&void 0!==n?n:"-")]})]},t)})),Ct("div",{className:"vm-query-analyzer-info-type",children:null!==(t=l[0])&&void 0!==t&&null!==(n=t.vmui)&&void 0!==n&&n.params?"The report was created using vmui":"The report was created manually"})]})})]})},Xm=()=>{const[e,t]=(0,r.useState)([]),[n,a]=(0,r.useState)(""),i=(0,r.useMemo)((()=>!!e.length),[e]),{value:o,setTrue:l,setFalse:s}=oa(!1),c=(0,r.useMemo)((()=>{var t,n;if(!e)return;const r=null===(t=e[0])||void 0===t||null===(n=t.vmui)||void 0===n?void 0:n.params,a={start:+((null===r||void 0===r?void 0:r.start)||0),end:+((null===r||void 0===r?void 0:r.end)||0),step:null===r||void 0===r?void 0:r.step,date:""};if(!r){const t=e.filter((e=>"matrix"===e.data.resultType)).map((e=>e.data.result)).flat().map((e=>{var t;return e.values?null===(t=e.values)||void 0===t?void 0:t.map((e=>e[0])):[0]})).flat(),n=Array.from(new Set(t.filter(Boolean))).sort(((e,t)=>e-t));a.start=n[0],a.end=n[n.length-1],a.step=Bt((e=>{const t=e.slice(1).map(((t,n)=>t-e[n])),n={};t.forEach((e=>{const t=e.toString();n[t]=(n[t]||0)+1}));let r=0,a=0;for(const i in n)n[i]>a&&(a=n[i],r=Number(i));return r})(n))}return a.date=Zt(Xt(a.end)),a}),[e]),u=e=>{try{const n=JSON.parse(e),r=Array.isArray(n)?n:[n];(e=>e.every((e=>{if("object"===typeof e&&null!==e){const t=e.data;if("object"===typeof t&&null!==t){const e=t.result,n=t.resultType;return Array.isArray(e)&&"string"===typeof n}}return!1})))(r)?t(r):a("Invalid structure - JSON does not match the expected format")}catch(rp){rp instanceof Error&&a("".concat(rp.name,": ").concat(rp.message))}},d=e=>{e.map((e=>{const t=new FileReader;t.onload=e=>{var t;const n=String(null===(t=e.target)||void 0===t?void 0:t.result);u(n)},t.readAsText(e)}))},h=e=>{a("");const t=Array.from(e.target.files||[]);d(t),e.target.value=""},{files:m,dragging:p}=_m();return(0,r.useEffect)((()=>{d(m)}),[m]),Ct("div",{className:"vm-trace-page",children:[i&&Ct("div",{className:"vm-trace-page-header",children:[Ct("div",{className:"vm-trace-page-header-errors",children:Ct(Jm,{data:e,period:c})}),Ct("div",{children:Ct(bm,{onOpenModal:l,onChange:h})})]}),n&&Ct("div",{className:"vm-trace-page-header-errors-item vm-trace-page-header-errors-item_margin-bottom",children:[Ct(Qr,{variant:"error",children:n}),Ct(aa,{className:"vm-trace-page-header-errors-item__close",startIcon:Ct(On,{}),variant:"text",color:"error",onClick:()=>{a("")}})]}),i&&Ct(Gm,{data:e,period:c}),!i&&Ct("div",{className:"vm-trace-page-preview",children:[Ct("p",{className:"vm-trace-page-preview__text",children:["Please, upload file with JSON response content.","\n","The file must contain query information in JSON format.","\n","Graph will be displayed after file upload.","\n","Attach files by dragging & dropping, selecting or pasting them."]}),Ct(bm,{onOpenModal:l,onChange:h})]}),o&&Ct(pa,{title:"Paste JSON",onClose:s,children:Ct(Zm,{onClose:s,onUpload:u})}),p&&Ct("div",{className:"vm-trace-page__dropzone"})]})},ep=()=>{const[e,t]=(0,r.useState)(!1);return Ct(xt.FK,{children:Ct(Le,{children:Ct(ta,{children:Ct(xt.FK,{children:[Ct(ym,{onLoaded:t}),e&&Ct(ke,{children:Ct(be,{path:"/",element:Ct(Ai,{}),children:[Ct(be,{path:Ye.home,element:Ct(Th,{})}),Ct(be,{path:Ye.metrics,element:Ct(Dm,{})}),Ct(be,{path:Ye.cardinality,element:Ct(lm,{})}),Ct(be,{path:Ye.topQueries,element:Ct(mm,{})}),Ct(be,{path:Ye.trace,element:Ct(wm,{})}),Ct(be,{path:Ye.queryAnalyzer,element:Ct(Xm,{})}),Ct(be,{path:Ye.dashboards,element:Ct(Oh,{})}),Ct(be,{path:Ye.withTemplate,element:Ct(Bm,{})}),Ct(be,{path:Ye.relabel,element:Ct(Wm,{})}),Ct(be,{path:Ye.activeQueries,element:Ct(Qm,{})}),Ct(be,{path:Ye.icons,element:Ct(zm,{})})]})})]})})})})},tp=e=>{e&&n.e(685).then(n.bind(n,685)).then((t=>{let{getCLS:n,getFID:r,getFCP:a,getLCP:i,getTTFB:o}=t;n(e),r(e),a(e),i(e),o(e)}))},np=document.getElementById("root");np&&(0,r.render)(Ct(ep,{}),np),tp()})()})(); \ No newline at end of file diff --git a/app/vmselect/vmui/static/js/main.202937c2.js.LICENSE.txt b/app/vmselect/vmui/static/js/main.59c17910.js.LICENSE.txt similarity index 100% rename from app/vmselect/vmui/static/js/main.202937c2.js.LICENSE.txt rename to app/vmselect/vmui/static/js/main.59c17910.js.LICENSE.txt diff --git a/app/vmselect/vmui/static/media/MetricsQL.da86c2db4f0b05e286b0.md b/app/vmselect/vmui/static/media/MetricsQL.df7574389d8f8bbcf0c7.md similarity index 91% rename from app/vmselect/vmui/static/media/MetricsQL.da86c2db4f0b05e286b0.md rename to app/vmselect/vmui/static/media/MetricsQL.df7574389d8f8bbcf0c7.md index 7f3ccc498..fec10e5dc 100644 --- a/app/vmselect/vmui/static/media/MetricsQL.da86c2db4f0b05e286b0.md +++ b/app/vmselect/vmui/static/media/MetricsQL.df7574389d8f8bbcf0c7.md @@ -79,6 +79,8 @@ The list of MetricsQL features on top of PromQL: * [Series selectors](https://docs.victoriametrics.com/keyconcepts/#filtering) accept multiple `or` filters. For example, `{env="prod",job="a" or env="dev",job="b"}` selects series with `{env="prod",job="a"}` or `{env="dev",job="b"}` labels. See [these docs](https://docs.victoriametrics.com/keyconcepts/#filtering-by-multiple-or-filters) for details. +* Support for matching against multiple numeric constants via `q == (C1, ..., CN)` and `q != (C1, ..., CN)` syntax. For example, `status_code == (300, 301, 304)` + returns `status_code` metrics with one of `300`, `301` or `304` values. * Support for `group_left(*)` and `group_right(*)` for copying all the labels from time series on the `one` side of [many-to-one operations](https://prometheus.io/docs/prometheus/latest/querying/operators/#many-to-one-and-one-to-many-vector-matches). The copied label names may clash with the existing label names, so MetricsQL provides an ability to add prefix to the copied metric names @@ -152,9 +154,9 @@ MetricsQL provides the following functions: ### Rollup functions -**Rollup functions** (aka range functions or window functions) calculate rollups over **raw samples** +**Rollup functions** (aka range functions or window functions) calculate rollups over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window for the [selected time series](https://docs.victoriametrics.com/keyconcepts/#filtering). -For example, `avg_over_time(temperature[24h])` calculates the average temperature over raw samples for the last 24 hours. +For example, `avg_over_time(temperature[24h])` calculates the average temperature over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) for the last 24 hours. Additional details: @@ -184,7 +186,7 @@ The list of supported rollup functions: #### absent_over_time `absent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 -if the given lookbehind window `d` doesn't contain raw samples. Otherwise, it returns an empty result. +if the given lookbehind window `d` doesn't contain [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). Otherwise, it returns an empty result. This function is supported by PromQL. @@ -193,7 +195,7 @@ See also [present_over_time](#present_over_time). #### aggr_over_time `aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` is a [rollup function](#rollup-functions), -which calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`. +which calculates all the listed `rollup_func*` for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -203,7 +205,7 @@ would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time) #### ascent_over_time `ascent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates -ascent of raw sample values on the given lookbehind window `d`. The calculations are performed individually +ascent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for tracking height gains in GPS tracking. Metric names are stripped from the resulting rollups. @@ -215,7 +217,7 @@ See also [descent_over_time](#descent_over_time). #### avg_over_time `avg_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average value -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -225,7 +227,7 @@ See also [median_over_time](#median_over_time). #### changes `changes(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times -the raw samples changed on the given lookbehind window `d` per each time series returned +the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Unlike `changes()` in Prometheus it takes into account the change from the last sample before the given lookbehind window `d`. @@ -240,7 +242,7 @@ See also [changes_prometheus](#changes_prometheus). #### changes_prometheus `changes_prometheus(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times -the raw samples changed on the given lookbehind window `d` per each time series returned +the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It doesn't take into account the change from the last sample before the given lookbehind window `d` in the same way as Prometheus does. @@ -254,7 +256,7 @@ See also [changes](#changes). #### count_eq_over_time -`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -264,7 +266,7 @@ See also [count_over_time](#count_over_time), [share_eq_over_time](#share_eq_ove #### count_gt_over_time -`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -274,7 +276,7 @@ See also [count_over_time](#count_over_time) and [share_gt_over_time](#share_gt_ #### count_le_over_time -`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which don't exceed `le`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -284,7 +286,7 @@ See also [count_over_time](#count_over_time) and [share_le_over_time](#share_le_ #### count_ne_over_time -`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which aren't equal to `ne`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -294,7 +296,7 @@ See also [count_over_time](#count_over_time). #### count_over_time -`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -305,7 +307,7 @@ See also [count_le_over_time](#count_le_over_time), [count_gt_over_time](#count_ #### count_values_over_time -`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of raw samples +`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the same value over the given lookbehind window and stores the counts in a time series with an additional `label`, which contains each initial value. The results are calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -315,8 +317,8 @@ See also [count_eq_over_time](#count_eq_over_time), [count_values](#count_values #### decreases_over_time -`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value decreases -over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value decreases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -324,8 +326,8 @@ See also [increases_over_time](#increases_over_time). #### default_rollup -`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). If the lookbehind window is skipped in square brackets, then it is automatically calculated as `max(step, scrape_interval)`, where `step` is the query arg value passed to [/api/v1/query_range](https://docs.victoriametrics.com/keyconcepts/#range-query) or [/api/v1/query](https://docs.victoriametrics.com/keyconcepts/#instant-query), @@ -375,7 +377,7 @@ See also [deriv_fast](#deriv_fast) and [ideriv](#ideriv). #### deriv_fast `deriv_fast(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivative -using the first and the last raw samples on the given lookbehind window `d` per each time series returned +using the first and the last [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -384,8 +386,8 @@ See also [deriv](#deriv) and [ideriv](#ideriv). #### descent_over_time -`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of raw sample values -on the given lookbehind window `d`. The calculations are performed individually per each time series returned +`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for tracking height loss in GPS tracking. @@ -396,8 +398,8 @@ See also [ascent_over_time](#ascent_over_time). #### distinct_over_time -`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of distinct raw sample values -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of unique [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -416,15 +418,15 @@ See also [lifetime](#lifetime) and [lag](#lag). #### first_over_time -`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first raw sample value -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). See also [last_over_time](#last_over_time) and [tfirst_over_time](#tfirst_over_time). #### geomean_over_time `geomean_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [geometric mean](https://en.wikipedia.org/wiki/Geometric_mean) -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -432,8 +434,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### histogram_over_time `histogram_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates -[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over raw samples on the given lookbehind window `d`. -It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +on the given lookbehind window `d`. It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The resulting histograms are useful to pass to [histogram_quantile](#histogram_quantile) for calculating quantiles over multiple [gauges](https://docs.victoriametrics.com/keyconcepts/#gauge). For example, the following query calculates median temperature by country over the last 24 hours: @@ -457,7 +459,7 @@ See also [hoeffding_bound_lower](#hoeffding_bound_lower). #### holt_winters `holt_winters(series_selector[d], sf, tf)` is a [rollup function](#rollup-functions), which calculates Holt-Winters value -(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for raw samples +(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) over the given lookbehind window `d` using the given smoothing factor `sf` and the given trend factor `tf`. Both `sf` and `tf` must be in the range `[0...1]`. It is expected that the [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering) returns time series of [gauge type](https://docs.victoriametrics.com/keyconcepts/#gauge). @@ -468,7 +470,7 @@ See also [range_linear_regression](#range_linear_regression). #### idelta -`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two raw samples +`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -479,7 +481,8 @@ See also [delta](#delta). #### ideriv -`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based on the last two raw samples +`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based +on the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) over the given lookbehind window `d`. The derivative is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -522,8 +525,8 @@ while [increase](#increase) ignores the first value in a series if it is too big #### increases_over_time -`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value increases -over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value increases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -531,14 +534,15 @@ See also [decreases_over_time](#decreases_over_time). #### integrate -`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over raw samples on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. #### irate -`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over the last two raw samples +`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over +the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that the `series_selector` returns time series of [counter type](https://docs.victoriametrics.com/keyconcepts/#counter). @@ -560,8 +564,8 @@ See also [lifetime](#lifetime) and [duration_over_time](#duration_over_time). #### last_over_time -`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -579,13 +583,14 @@ See also [duration_over_time](#duration_over_time) and [lag](#lag). #### mad_over_time `mad_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [median absolute deviation](https://en.wikipedia.org/wiki/Median_absolute_deviation) -over raw samples on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned +from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). See also [mad](#mad), [range_mad](#range_mad) and [outlier_iqr_over_time](#outlier_iqr_over_time). #### max_over_time -`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over raw samples +`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -594,7 +599,7 @@ See also [tmax_over_time](#tmax_over_time). #### median_over_time -`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over raw samples +`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -602,7 +607,7 @@ See also [avg_over_time](#avg_over_time). #### min_over_time -`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over raw samples +`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -612,15 +617,16 @@ See also [tmin_over_time](#tmin_over_time). #### mode_over_time `mode_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [mode](https://en.wikipedia.org/wiki/Mode_(statistics)) -for raw samples on the given lookbehind window `d`. It is calculated individually per each time series returned -from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that raw sample values are discrete. +for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated individually per each time series returned +from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values are discrete. #### outlier_iqr_over_time `outlier_iqr_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last sample on the given lookbehind window `d` if its value is either smaller than the `q25-1.5*iqr` or bigger than `q75+1.5*iqr` where: -- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over raw samples on the lookbehind window `d` -- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over raw samples on the lookbehind window `d`. +- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d` +- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`. The `outlier_iqr_over_time()` is useful for detecting anomalies in gauge values based on the previous history of values. For example, `outlier_iqr_over_time(memory_usage_bytes[1h])` triggers when `memory_usage_bytes` suddenly goes outside the usual value range for the last hour. @@ -630,8 +636,8 @@ See also [outliers_iqr](#outliers_iqr). #### predict_linear `predict_linear(series_selector[d], t)` is a [rollup function](#rollup-functions), which calculates the value `t` seconds in the future using -linear interpolation over raw samples on the given lookbehind window `d`. The predicted value is calculated individually per each time series -returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +linear interpolation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. +The predicted value is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -639,7 +645,7 @@ See also [range_linear_regression](#range_linear_regression). #### present_over_time -`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single raw sample +`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. Otherwise, an empty result is returned. Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -648,7 +654,7 @@ This function is supported by PromQL. #### quantile_over_time -`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over raw samples +`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The `phi` value must be in the range `[0...1]`. @@ -659,7 +665,7 @@ See also [quantiles_over_time](#quantiles_over_time). #### quantiles_over_time `quantiles_over_time("phiLabel", phi1, ..., phiN, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi*`-quantiles -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The function returns individual series per each `phi*` with `{phiLabel="phi*"}` label. `phi*` values must be in the range `[0...1]`. @@ -667,7 +673,7 @@ See also [quantile_over_time](#quantile_over_time). #### range_over_time -`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over raw samples +`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). E.g. it calculates `max_over_time(series_selector[d]) - min_over_time(series_selector[d])`. @@ -692,7 +698,7 @@ See also [irate](#irate) and [rollup_rate](#rollup_rate). #### rate_over_sum -`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of raw samples +`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -711,7 +717,7 @@ This function is supported by PromQL. #### rollup -`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for raw samples +`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. These values are calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -721,7 +727,8 @@ See also [label_match](#label_match). #### rollup_candlestick `rollup_candlestick(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `open`, `high`, `low` and `close` values (aka OHLC) -over raw samples on the given lookbehind window `d` and returns them in time series with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels. +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series +with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for financial applications. @@ -730,7 +737,7 @@ See also [label_match](#label_match). #### rollup_delta -`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent raw samples +`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -745,8 +752,8 @@ See also [rollup_increase](#rollup_increase). #### rollup_deriv `rollup_deriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivatives -for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives -and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. +for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values +for the calculated per-second derivatives and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Optional 2nd argument `"min"`, `"max"` or `"avg"` can be passed to keep only one calculation result and without adding a label. @@ -756,7 +763,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_increase -`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent raw samples +`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -768,7 +775,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_rate -`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates for adjacent raw samples +`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates +for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. @@ -785,7 +793,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_scrape_interval `rollup_scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the interval in seconds between -adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval +adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -796,7 +804,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### scrape_interval -`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds between raw samples +`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds +between [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -805,7 +814,8 @@ See also [rollup_scrape_interval](#rollup_scrape_interval). #### share_gt_over_time -`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -817,7 +827,8 @@ See also [share_le_over_time](#share_le_over_time) and [count_gt_over_time](#cou #### share_le_over_time -`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are smaller or equal to `le`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -830,7 +841,8 @@ See also [share_gt_over_time](#share_gt_over_time) and [count_le_over_time](#cou #### share_eq_over_time -`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -848,7 +860,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### stddev_over_time -`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over raw samples +`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -859,7 +871,7 @@ See also [stdvar_over_time](#stdvar_over_time). #### stdvar_over_time -`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over raw samples +`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -870,8 +882,8 @@ See also [stddev_over_time](#stddev_over_time). #### sum_eq_over_time -`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values equal to `eq` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values equal to `eq` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -879,8 +891,8 @@ See also [sum_over_time](#sum_over_time) and [count_eq_over_time](#count_eq_over #### sum_gt_over_time -`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values bigger than `gt` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values bigger than `gt` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -888,8 +900,8 @@ See also [sum_over_time](#sum_over_time) and [count_gt_over_time](#count_gt_over #### sum_le_over_time -`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values smaller or equal to `le` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values smaller or equal to `le` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -897,7 +909,7 @@ See also [sum_over_time](#sum_over_time) and [count_le_over_time](#count_le_over #### sum_over_time -`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of raw sample values +`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -906,14 +918,15 @@ This function is supported by PromQL. #### sum2_over_time -`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for raw sample values -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. #### timestamp -`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample +`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -924,7 +937,8 @@ See also [time](#time) and [now](#now). #### timestamp_with_name -`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample +`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are preserved in the resulting rollups. @@ -933,7 +947,8 @@ See also [timestamp](#timestamp) and [keep_metric_names](#keep_metric_names) mod #### tfirst_over_time -`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the first raw sample +`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -957,7 +972,8 @@ See also [tlast_change_over_time](#tlast_change_over_time). #### tmax_over_time -`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample +`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the maximum value on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -967,7 +983,8 @@ See also [max_over_time](#max_over_time). #### tmin_over_time -`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample +`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the minimum value on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -978,7 +995,7 @@ See also [min_over_time](#min_over_time). #### zscore_over_time `zscore_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns [z-score](https://en.wikipedia.org/wiki/Standard_score) -for raw samples on the given lookbehind window `d`. It is calculated independently per each time series returned +for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -2212,6 +2229,7 @@ Any [rollup function](#rollup-functions) for something other than [series select Nested rollup functions can be implicit thanks to the [implicit query conversions](#implicit-query-conversions). For example, `delta(sum(m))` is implicitly converted to `delta(sum(default_rollup(m))[1i:1i])`, so it becomes a subquery, since it contains [default_rollup](#default_rollup) nested into [delta](#delta). +This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0. VictoriaMetrics performs subqueries in the following way: @@ -2244,4 +2262,5 @@ VictoriaMetrics performs the following implicit conversions for incoming queries For example, `avg_over_time(rate(http_requests_total[5m])[1h])` is automatically converted to `avg_over_time(rate(http_requests_total[5m])[1h:1i])`. * If something other than [series selector](https://docs.victoriametrics.com/keyconcepts/#filtering) is passed to [rollup function](#rollup-functions), then a [subquery](#subqueries) with `1i` lookbehind window and `1i` step is automatically formed. - For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`. + For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`. + This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0. diff --git a/app/vmui/Dockerfile-web b/app/vmui/Dockerfile-web index f54841860..1ccad9a66 100644 --- a/app/vmui/Dockerfile-web +++ b/app/vmui/Dockerfile-web @@ -1,4 +1,4 @@ -FROM golang:1.22.2 as build-web-stage +FROM golang:1.22.3 as build-web-stage COPY build /build WORKDIR /build diff --git a/app/vmui/packages/vmui/src/assets/MetricsQL.md b/app/vmui/packages/vmui/src/assets/MetricsQL.md index 7f3ccc498..fec10e5dc 100644 --- a/app/vmui/packages/vmui/src/assets/MetricsQL.md +++ b/app/vmui/packages/vmui/src/assets/MetricsQL.md @@ -79,6 +79,8 @@ The list of MetricsQL features on top of PromQL: * [Series selectors](https://docs.victoriametrics.com/keyconcepts/#filtering) accept multiple `or` filters. For example, `{env="prod",job="a" or env="dev",job="b"}` selects series with `{env="prod",job="a"}` or `{env="dev",job="b"}` labels. See [these docs](https://docs.victoriametrics.com/keyconcepts/#filtering-by-multiple-or-filters) for details. +* Support for matching against multiple numeric constants via `q == (C1, ..., CN)` and `q != (C1, ..., CN)` syntax. For example, `status_code == (300, 301, 304)` + returns `status_code` metrics with one of `300`, `301` or `304` values. * Support for `group_left(*)` and `group_right(*)` for copying all the labels from time series on the `one` side of [many-to-one operations](https://prometheus.io/docs/prometheus/latest/querying/operators/#many-to-one-and-one-to-many-vector-matches). The copied label names may clash with the existing label names, so MetricsQL provides an ability to add prefix to the copied metric names @@ -152,9 +154,9 @@ MetricsQL provides the following functions: ### Rollup functions -**Rollup functions** (aka range functions or window functions) calculate rollups over **raw samples** +**Rollup functions** (aka range functions or window functions) calculate rollups over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window for the [selected time series](https://docs.victoriametrics.com/keyconcepts/#filtering). -For example, `avg_over_time(temperature[24h])` calculates the average temperature over raw samples for the last 24 hours. +For example, `avg_over_time(temperature[24h])` calculates the average temperature over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) for the last 24 hours. Additional details: @@ -184,7 +186,7 @@ The list of supported rollup functions: #### absent_over_time `absent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 -if the given lookbehind window `d` doesn't contain raw samples. Otherwise, it returns an empty result. +if the given lookbehind window `d` doesn't contain [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). Otherwise, it returns an empty result. This function is supported by PromQL. @@ -193,7 +195,7 @@ See also [present_over_time](#present_over_time). #### aggr_over_time `aggr_over_time(("rollup_func1", "rollup_func2", ...), series_selector[d])` is a [rollup function](#rollup-functions), -which calculates all the listed `rollup_func*` for raw samples on the given lookbehind window `d`. +which calculates all the listed `rollup_func*` for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -203,7 +205,7 @@ would calculate [min_over_time](#min_over_time), [max_over_time](#max_over_time) #### ascent_over_time `ascent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates -ascent of raw sample values on the given lookbehind window `d`. The calculations are performed individually +ascent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for tracking height gains in GPS tracking. Metric names are stripped from the resulting rollups. @@ -215,7 +217,7 @@ See also [descent_over_time](#descent_over_time). #### avg_over_time `avg_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average value -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -225,7 +227,7 @@ See also [median_over_time](#median_over_time). #### changes `changes(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times -the raw samples changed on the given lookbehind window `d` per each time series returned +the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Unlike `changes()` in Prometheus it takes into account the change from the last sample before the given lookbehind window `d`. @@ -240,7 +242,7 @@ See also [changes_prometheus](#changes_prometheus). #### changes_prometheus `changes_prometheus(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of times -the raw samples changed on the given lookbehind window `d` per each time series returned +the [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) changed on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It doesn't take into account the change from the last sample before the given lookbehind window `d` in the same way as Prometheus does. @@ -254,7 +256,7 @@ See also [changes](#changes). #### count_eq_over_time -`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -264,7 +266,7 @@ See also [count_over_time](#count_over_time), [share_eq_over_time](#share_eq_ove #### count_gt_over_time -`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -274,7 +276,7 @@ See also [count_over_time](#count_over_time) and [share_gt_over_time](#share_gt_ #### count_le_over_time -`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which don't exceed `le`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -284,7 +286,7 @@ See also [count_over_time](#count_over_time) and [share_le_over_time](#share_le_ #### count_ne_over_time -`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_ne_over_time(series_selector[d], ne)` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which aren't equal to `ne`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -294,7 +296,7 @@ See also [count_over_time](#count_over_time). #### count_over_time -`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw samples +`count_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -305,7 +307,7 @@ See also [count_le_over_time](#count_le_over_time), [count_gt_over_time](#count_ #### count_values_over_time -`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of raw samples +`count_values_over_time("label", series_selector[d])` is a [rollup function](#rollup-functions), which counts the number of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the same value over the given lookbehind window and stores the counts in a time series with an additional `label`, which contains each initial value. The results are calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -315,8 +317,8 @@ See also [count_eq_over_time](#count_eq_over_time), [count_values](#count_values #### decreases_over_time -`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value decreases -over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`decreases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value decreases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -324,8 +326,8 @@ See also [increases_over_time](#increases_over_time). #### default_rollup -`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`default_rollup(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). If the lookbehind window is skipped in square brackets, then it is automatically calculated as `max(step, scrape_interval)`, where `step` is the query arg value passed to [/api/v1/query_range](https://docs.victoriametrics.com/keyconcepts/#range-query) or [/api/v1/query](https://docs.victoriametrics.com/keyconcepts/#instant-query), @@ -375,7 +377,7 @@ See also [deriv_fast](#deriv_fast) and [ideriv](#ideriv). #### deriv_fast `deriv_fast(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivative -using the first and the last raw samples on the given lookbehind window `d` per each time series returned +using the first and the last [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -384,8 +386,8 @@ See also [deriv](#deriv) and [ideriv](#ideriv). #### descent_over_time -`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of raw sample values -on the given lookbehind window `d`. The calculations are performed individually per each time series returned +`descent_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates descent of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for tracking height loss in GPS tracking. @@ -396,8 +398,8 @@ See also [ascent_over_time](#ascent_over_time). #### distinct_over_time -`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of distinct raw sample values -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`distinct_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the number of unique [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -416,15 +418,15 @@ See also [lifetime](#lifetime) and [lag](#lag). #### first_over_time -`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first raw sample value -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`first_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). See also [last_over_time](#last_over_time) and [tfirst_over_time](#tfirst_over_time). #### geomean_over_time `geomean_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [geometric mean](https://en.wikipedia.org/wiki/Geometric_mean) -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -432,8 +434,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### histogram_over_time `histogram_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates -[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over raw samples on the given lookbehind window `d`. -It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +[VictoriaMetrics histogram](https://godoc.org/github.com/VictoriaMetrics/metrics#Histogram) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +on the given lookbehind window `d`. It is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The resulting histograms are useful to pass to [histogram_quantile](#histogram_quantile) for calculating quantiles over multiple [gauges](https://docs.victoriametrics.com/keyconcepts/#gauge). For example, the following query calculates median temperature by country over the last 24 hours: @@ -457,7 +459,7 @@ See also [hoeffding_bound_lower](#hoeffding_bound_lower). #### holt_winters `holt_winters(series_selector[d], sf, tf)` is a [rollup function](#rollup-functions), which calculates Holt-Winters value -(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for raw samples +(aka [double exponential smoothing](https://en.wikipedia.org/wiki/Exponential_smoothing#Double_exponential_smoothing)) for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) over the given lookbehind window `d` using the given smoothing factor `sf` and the given trend factor `tf`. Both `sf` and `tf` must be in the range `[0...1]`. It is expected that the [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering) returns time series of [gauge type](https://docs.victoriametrics.com/keyconcepts/#gauge). @@ -468,7 +470,7 @@ See also [range_linear_regression](#range_linear_regression). #### idelta -`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two raw samples +`idelta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the difference between the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -479,7 +481,8 @@ See also [delta](#delta). #### ideriv -`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based on the last two raw samples +`ideriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the per-second derivative based +on the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) over the given lookbehind window `d`. The derivative is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -522,8 +525,8 @@ while [increase](#increase) ignores the first value in a series if it is too big #### increases_over_time -`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of raw sample value increases -over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`increases_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the number of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value increases over the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -531,14 +534,15 @@ See also [decreases_over_time](#decreases_over_time). #### integrate -`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over raw samples on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`integrate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the integral over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. #### irate -`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over the last two raw samples +`irate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the "instant" per-second increase rate over +the last two [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that the `series_selector` returns time series of [counter type](https://docs.victoriametrics.com/keyconcepts/#counter). @@ -560,8 +564,8 @@ See also [lifetime](#lifetime) and [duration_over_time](#duration_over_time). #### last_over_time -`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last raw sample value on the given lookbehind window `d` -per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`last_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +value on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -579,13 +583,14 @@ See also [duration_over_time](#duration_over_time) and [lag](#lag). #### mad_over_time `mad_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [median absolute deviation](https://en.wikipedia.org/wiki/Median_absolute_deviation) -over raw samples on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned +from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). See also [mad](#mad), [range_mad](#range_mad) and [outlier_iqr_over_time](#outlier_iqr_over_time). #### max_over_time -`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over raw samples +`max_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the maximum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -594,7 +599,7 @@ See also [tmax_over_time](#tmax_over_time). #### median_over_time -`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over raw samples +`median_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates median value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -602,7 +607,7 @@ See also [avg_over_time](#avg_over_time). #### min_over_time -`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over raw samples +`min_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the minimum value over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -612,15 +617,16 @@ See also [tmin_over_time](#tmin_over_time). #### mode_over_time `mode_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates [mode](https://en.wikipedia.org/wiki/Mode_(statistics)) -for raw samples on the given lookbehind window `d`. It is calculated individually per each time series returned -from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that raw sample values are discrete. +for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated individually per each time series returned +from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). It is expected that [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values are discrete. #### outlier_iqr_over_time `outlier_iqr_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the last sample on the given lookbehind window `d` if its value is either smaller than the `q25-1.5*iqr` or bigger than `q75+1.5*iqr` where: -- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over raw samples on the lookbehind window `d` -- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over raw samples on the lookbehind window `d`. +- `iqr` is an [Interquartile range](https://en.wikipedia.org/wiki/Interquartile_range) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d` +- `q25` and `q75` are 25th and 75th [percentiles](https://en.wikipedia.org/wiki/Percentile) over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the lookbehind window `d`. The `outlier_iqr_over_time()` is useful for detecting anomalies in gauge values based on the previous history of values. For example, `outlier_iqr_over_time(memory_usage_bytes[1h])` triggers when `memory_usage_bytes` suddenly goes outside the usual value range for the last hour. @@ -630,8 +636,8 @@ See also [outliers_iqr](#outliers_iqr). #### predict_linear `predict_linear(series_selector[d], t)` is a [rollup function](#rollup-functions), which calculates the value `t` seconds in the future using -linear interpolation over raw samples on the given lookbehind window `d`. The predicted value is calculated individually per each time series -returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +linear interpolation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. +The predicted value is calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is supported by PromQL. @@ -639,7 +645,7 @@ See also [range_linear_regression](#range_linear_regression). #### present_over_time -`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single raw sample +`present_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns 1 if there is at least a single [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. Otherwise, an empty result is returned. Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -648,7 +654,7 @@ This function is supported by PromQL. #### quantile_over_time -`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over raw samples +`quantile_over_time(phi, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi`-quantile over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The `phi` value must be in the range `[0...1]`. @@ -659,7 +665,7 @@ See also [quantiles_over_time](#quantiles_over_time). #### quantiles_over_time `quantiles_over_time("phiLabel", phi1, ..., phiN, series_selector[d])` is a [rollup function](#rollup-functions), which calculates `phi*`-quantiles -over raw samples on the given lookbehind window `d` per each time series returned +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). The function returns individual series per each `phi*` with `{phiLabel="phi*"}` label. `phi*` values must be in the range `[0...1]`. @@ -667,7 +673,7 @@ See also [quantile_over_time](#quantile_over_time). #### range_over_time -`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over raw samples +`range_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates value range over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). E.g. it calculates `max_over_time(series_selector[d]) - min_over_time(series_selector[d])`. @@ -692,7 +698,7 @@ See also [irate](#irate) and [rollup_rate](#rollup_rate). #### rate_over_sum -`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of raw samples +`rate_over_sum(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second rate over the sum of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -711,7 +717,7 @@ This function is supported by PromQL. #### rollup -`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for raw samples +`rollup(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `min`, `max` and `avg` values for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. These values are calculated individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -721,7 +727,8 @@ See also [label_match](#label_match). #### rollup_candlestick `rollup_candlestick(series_selector[d])` is a [rollup function](#rollup-functions), which calculates `open`, `high`, `low` and `close` values (aka OHLC) -over raw samples on the given lookbehind window `d` and returns them in time series with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels. +over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns them in time series +with `rollup="open"`, `rollup="high"`, `rollup="low"` and `rollup="close"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). This function is useful for financial applications. @@ -730,7 +737,7 @@ See also [label_match](#label_match). #### rollup_delta -`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent raw samples +`rollup_delta(series_selector[d])` is a [rollup function](#rollup-functions), which calculates differences between adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated differences and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -745,8 +752,8 @@ See also [rollup_increase](#rollup_increase). #### rollup_deriv `rollup_deriv(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second derivatives -for adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second derivatives -and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. +for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values +for the calculated per-second derivatives and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Optional 2nd argument `"min"`, `"max"` or `"avg"` can be passed to keep only one calculation result and without adding a label. @@ -756,7 +763,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_increase -`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent raw samples +`rollup_increase(series_selector[d])` is a [rollup function](#rollup-functions), which calculates increases for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated increases and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -768,7 +775,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_rate -`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates for adjacent raw samples +`rollup_rate(series_selector[d])` is a [rollup function](#rollup-functions), which calculates per-second change rates +for adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated per-second change rates and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. @@ -785,7 +793,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### rollup_scrape_interval `rollup_scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the interval in seconds between -adjacent raw samples on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval +adjacent [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` and returns `min`, `max` and `avg` values for the calculated interval and returns them in time series with `rollup="min"`, `rollup="max"` and `rollup="avg"` additional labels. The calculations are performed individually per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -796,7 +804,8 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### scrape_interval -`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds between raw samples +`scrape_interval(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the average interval in seconds +between [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -805,7 +814,8 @@ See also [rollup_scrape_interval](#rollup_scrape_interval). #### share_gt_over_time -`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are bigger than `gt`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -817,7 +827,8 @@ See also [share_le_over_time](#share_le_over_time) and [count_gt_over_time](#cou #### share_le_over_time -`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are smaller or equal to `le`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -830,7 +841,8 @@ See also [share_gt_over_time](#share_gt_over_time) and [count_le_over_time](#cou #### share_eq_over_time -`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) of raw samples +`share_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-functions), which returns share (in the range `[0...1]`) +of [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`, which are equal to `eq`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -848,7 +860,7 @@ Metric names are stripped from the resulting rollups. Add [keep_metric_names](#k #### stddev_over_time -`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over raw samples +`stddev_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard deviation over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -859,7 +871,7 @@ See also [stdvar_over_time](#stdvar_over_time). #### stdvar_over_time -`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over raw samples +`stdvar_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates standard variance over [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -870,8 +882,8 @@ See also [stddev_over_time](#stddev_over_time). #### sum_eq_over_time -`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values equal to `eq` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_eq_over_time(series_selector[d], eq)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values equal to `eq` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -879,8 +891,8 @@ See also [sum_over_time](#sum_over_time) and [count_eq_over_time](#count_eq_over #### sum_gt_over_time -`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values bigger than `gt` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_gt_over_time(series_selector[d], gt)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values bigger than `gt` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -888,8 +900,8 @@ See also [sum_over_time](#sum_over_time) and [count_gt_over_time](#count_gt_over #### sum_le_over_time -`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of raw sample values smaller or equal to `le` -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum_le_over_time(series_selector[d], le)` is a [rollup function](#rollup-function), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values smaller or equal to `le` on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -897,7 +909,7 @@ See also [sum_over_time](#sum_over_time) and [count_le_over_time](#count_le_over #### sum_over_time -`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of raw sample values +`sum_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -906,14 +918,15 @@ This function is supported by PromQL. #### sum2_over_time -`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for raw sample values -on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). +`sum2_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which calculates the sum of squares for [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) +values on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. #### timestamp -`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample +`timestamp(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -924,7 +937,8 @@ See also [time](#time) and [now](#now). #### timestamp_with_name -`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the last raw sample +`timestamp_with_name(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the last [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are preserved in the resulting rollups. @@ -933,7 +947,8 @@ See also [timestamp](#timestamp) and [keep_metric_names](#keep_metric_names) mod #### tfirst_over_time -`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the first raw sample +`tfirst_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the first [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d` per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -957,7 +972,8 @@ See also [tlast_change_over_time](#tlast_change_over_time). #### tmax_over_time -`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample +`tmax_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the maximum value on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -967,7 +983,8 @@ See also [max_over_time](#max_over_time). #### tmin_over_time -`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision for the raw sample +`tmin_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns the timestamp in seconds with millisecond precision +for the [raw sample](https://docs.victoriametrics.com/keyconcepts/#raw-samples) with the minimum value on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). @@ -978,7 +995,7 @@ See also [min_over_time](#min_over_time). #### zscore_over_time `zscore_over_time(series_selector[d])` is a [rollup function](#rollup-functions), which returns [z-score](https://en.wikipedia.org/wiki/Standard_score) -for raw samples on the given lookbehind window `d`. It is calculated independently per each time series returned +for [raw samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples) on the given lookbehind window `d`. It is calculated independently per each time series returned from the given [series_selector](https://docs.victoriametrics.com/keyconcepts/#filtering). Metric names are stripped from the resulting rollups. Add [keep_metric_names](#keep_metric_names) modifier in order to keep metric names. @@ -2212,6 +2229,7 @@ Any [rollup function](#rollup-functions) for something other than [series select Nested rollup functions can be implicit thanks to the [implicit query conversions](#implicit-query-conversions). For example, `delta(sum(m))` is implicitly converted to `delta(sum(default_rollup(m))[1i:1i])`, so it becomes a subquery, since it contains [default_rollup](#default_rollup) nested into [delta](#delta). +This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0. VictoriaMetrics performs subqueries in the following way: @@ -2244,4 +2262,5 @@ VictoriaMetrics performs the following implicit conversions for incoming queries For example, `avg_over_time(rate(http_requests_total[5m])[1h])` is automatically converted to `avg_over_time(rate(http_requests_total[5m])[1h:1i])`. * If something other than [series selector](https://docs.victoriametrics.com/keyconcepts/#filtering) is passed to [rollup function](#rollup-functions), then a [subquery](#subqueries) with `1i` lookbehind window and `1i` step is automatically formed. - For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`. + For example, `rate(sum(up))` is automatically converted to `rate((sum(default_rollup(up)))[1i:1i])`. + This behavior can be disabled or logged via cmd-line flags `-search.disableImplicitConversion` and `-search.logImplicitConversion` since v1.101.0. diff --git a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditor.tsx b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditor.tsx index a2a2c7a42..2a4290264 100644 --- a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditor.tsx +++ b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditor.tsx @@ -9,6 +9,7 @@ import { partialWarning, seriesFetchedWarning } from "./warningText"; import { AutocompleteOptions } from "../../Main/Autocomplete/Autocomplete"; import useDeviceDetect from "../../../hooks/useDeviceDetect"; import { useQueryState } from "../../../state/query/QueryStateContext"; +import debounce from "lodash.debounce"; export interface QueryEditorProps { onChange: (query: string) => void; @@ -40,9 +41,12 @@ const QueryEditor: FC = ({ const { isMobile } = useDeviceDetect(); const [openAutocomplete, setOpenAutocomplete] = useState(false); - const [caretPosition, setCaretPosition] = useState([0, 0]); + const [caretPosition, setCaretPosition] = useState<[number, number]>([0, 0]); const autocompleteAnchorEl = useRef(null); + const [showAutocomplete, setShowAutocomplete] = useState(autocomplete); + const debouncedSetShowAutocomplete = useRef(debounce(setShowAutocomplete, 500)).current; + const warning = [ { show: stats?.seriesFetched === "0" && !stats.resultLength, @@ -58,8 +62,9 @@ const QueryEditor: FC = ({ label = `${label} (${stats.executionTimeMsec || 0}ms)`; } - const handleSelect = (val: string) => { + const handleSelect = (val: string, caretPosition: number) => { onChange(val); + setCaretPosition([caretPosition, caretPosition]); }; const handleKeyDown = (e: KeyboardEvent) => { @@ -100,14 +105,19 @@ const QueryEditor: FC = ({ setOpenAutocomplete(!!val.length); }; - const handleChangeCaret = (val: number[]) => { - setCaretPosition(val); + const handleChangeCaret = (val: [number, number]) => { + setCaretPosition(prev => prev[0] === val[0] && prev[1] === val[1] ? prev : val); }; useEffect(() => { setOpenAutocomplete(autocomplete); }, [autocompleteQuick]); + useEffect(() => { + setShowAutocomplete(false); + debouncedSetShowAutocomplete(true); + }, [caretPosition]); + return (
    = ({ onChangeCaret={handleChangeCaret} disabled={disabled} inputmode={"search"} + caretPosition={caretPosition} /> - {autocomplete && ( + {showAutocomplete && autocomplete && ( diff --git a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditorAutocomplete.tsx b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditorAutocomplete.tsx index edbd85d35..c19934895 100644 --- a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditorAutocomplete.tsx +++ b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/QueryEditorAutocomplete.tsx @@ -1,7 +1,6 @@ -import React, { FC, Ref, useState, useEffect, useMemo } from "preact/compat"; +import React, { FC, Ref, useState, useEffect, useMemo, useCallback } from "preact/compat"; import Autocomplete, { AutocompleteOptions } from "../../Main/Autocomplete/Autocomplete"; import { useFetchQueryOptions } from "../../../hooks/useFetchQueryOptions"; -import { getTextWidth } from "../../../utils/uplot"; import { escapeRegexp, hasUnclosedQuotes } from "../../../utils/regexp"; import useGetMetricsQL from "../../../hooks/useGetMetricsQL"; import { QueryContextType } from "../../../types"; @@ -10,8 +9,9 @@ import { AUTOCOMPLETE_LIMITS } from "../../../constants/queryAutocomplete"; interface QueryEditorAutocompleteProps { value: string; anchorEl: Ref; - caretPosition: number[]; - onSelect: (val: string) => void; + caretPosition: [number, number]; // [start, end] + hasHelperText: boolean; + onSelect: (val: string, caretPosition: number) => void; onFoundOptions: (val: AutocompleteOptions[]) => void; } @@ -19,16 +19,24 @@ const QueryEditorAutocomplete: FC = ({ value, anchorEl, caretPosition, + hasHelperText, onSelect, onFoundOptions }) => { - const [leftOffset, setLeftOffset] = useState(0); + const [offsetPos, setOffsetPos] = useState({ top: 0, left: 0 }); const metricsqlFunctions = useGetMetricsQL(); + const values = useMemo(() => { + if (caretPosition[0] !== caretPosition[1]) return { beforeCursor: value, afterCursor: "" }; + const beforeCursor = value.substring(0, caretPosition[0]); + const afterCursor = value.substring(caretPosition[1]); + return { beforeCursor, afterCursor }; + }, [value, caretPosition]); + const exprLastPart = useMemo(() => { - const parts = value.split("}"); + const parts = values.beforeCursor.split("}"); return parts[parts.length - 1]; - }, [value]); + }, [values]); const metric = useMemo(() => { const regexp = /\b[^{}(),\s]+(?={|$)/g; @@ -43,7 +51,7 @@ const QueryEditorAutocomplete: FC = ({ }, [exprLastPart]); const shouldSuppressAutoSuggestion = (value: string) => { - const pattern = /([(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/; + const pattern = /([{(),+\-*/^]|\b(?:or|and|unless|default|ifnot|if|group_left|group_right)\b)/; const parts = value.split(/\s+/); const partsCount = parts.length; const lastPart = parts[partsCount - 1]; @@ -55,7 +63,7 @@ const QueryEditorAutocomplete: FC = ({ }; const context = useMemo(() => { - if (!value || value.endsWith("}") || shouldSuppressAutoSuggestion(value)) { + if (!values.beforeCursor || values.beforeCursor.endsWith("}") || shouldSuppressAutoSuggestion(values.beforeCursor)) { return QueryContextType.empty; } @@ -63,19 +71,19 @@ const QueryEditorAutocomplete: FC = ({ const labelValueRegexp = new RegExp(`(${escapeRegexp(metric)})?{?.+${escapeRegexp(label)}(=|!=|=~|!~)"?([^"]*)$`, "g"); switch (true) { - case labelValueRegexp.test(value): + case labelValueRegexp.test(values.beforeCursor): return QueryContextType.labelValue; - case labelRegexp.test(value): + case labelRegexp.test(values.beforeCursor): return QueryContextType.label; default: return QueryContextType.metricsql; } - }, [value, metric, label]); + }, [values, metric, label]); const valueByContext = useMemo(() => { - const wordMatch = value.match(/([\w_\-.:/]+(?![},]))$/); + const wordMatch = values.beforeCursor.match(/([\w_\-.:/]+(?![},]))$/); return wordMatch ? wordMatch[0] : ""; - }, [value]); + }, [values.beforeCursor]); const { metrics, labels, labelValues, loading } = useFetchQueryOptions({ valueByContext, @@ -97,8 +105,10 @@ const QueryEditorAutocomplete: FC = ({ } }, [context, metrics, labels, labelValues]); - const handleSelect = (insert: string) => { + const handleSelect = useCallback((insert: string) => { // Find the start and end of valueByContext in the query string + const value = values.beforeCursor; + let valueAfterCursor = values.afterCursor; const startIndexOfValueByContext = value.lastIndexOf(valueByContext, caretPosition[0]); const endIndexOfValueByContext = startIndexOfValueByContext + valueByContext.length; @@ -110,26 +120,59 @@ const QueryEditorAutocomplete: FC = ({ if (context === QueryContextType.labelValue) { const quote = "\""; const needsQuote = /(?:=|!=|=~|!~)$/.test(beforeValueByContext); + valueAfterCursor = valueAfterCursor.replace(/^[^\s"|},]*/, ""); insert = `${needsQuote ? quote : ""}${insert}`; } + if (context === QueryContextType.label) { + valueAfterCursor = valueAfterCursor.replace(/^[^\s=!,{}()"|+\-/*^]*/, ""); + } + + if (context === QueryContextType.metricsql) { + valueAfterCursor = valueAfterCursor.replace(/^[^\s[\]{}()"|+\-/*^]*/, ""); + } // Assemble the new value with the inserted text - const newVal = `${beforeValueByContext}${insert}${afterValueByContext}`; - onSelect(newVal); - }; + const newVal = `${beforeValueByContext}${insert}${afterValueByContext}${valueAfterCursor}`; + onSelect(newVal, beforeValueByContext.length + insert.length); + }, [values]); useEffect(() => { if (!anchorEl.current) { - setLeftOffset(0); + setOffsetPos({ top: 0, left: 0 }); return; } - const style = window.getComputedStyle(anchorEl.current); + const element = anchorEl.current.querySelector("textarea") || anchorEl.current; + const style = window.getComputedStyle(element); const fontSize = `${style.getPropertyValue("font-size")}`; const fontFamily = `${style.getPropertyValue("font-family")}`; - const offset = getTextWidth(value, `${fontSize} ${fontFamily}`); - setLeftOffset(offset); - }, [anchorEl, caretPosition]); + const lineHeight = parseInt(`${style.getPropertyValue("line-height")}`); + + const span = document.createElement("div"); + span.style.font = `${fontSize} ${fontFamily}`; + span.style.padding = style.getPropertyValue("padding"); + span.style.lineHeight = `${lineHeight}px`; + span.style.width = `${element.offsetWidth}px`; + span.style.maxWidth = `${element.offsetWidth}px`; + span.style.whiteSpace = style.getPropertyValue("white-space"); + span.style.overflowWrap = style.getPropertyValue("overflow-wrap"); + + const marker = document.createElement("span"); + span.appendChild(document.createTextNode(values.beforeCursor)); + span.appendChild(marker); + span.appendChild(document.createTextNode(values.afterCursor)); + document.body.appendChild(span); + + const spanRect = span.getBoundingClientRect(); + const markerRect = marker.getBoundingClientRect(); + + const leftOffset = markerRect.left - spanRect.left; + const topOffset = markerRect.bottom - spanRect.bottom - (hasHelperText ? lineHeight : 0); + setOffsetPos({ top: topOffset, left: leftOffset }); + + span.remove(); + marker.remove(); + }, [anchorEl, caretPosition, hasHelperText]); return ( <> @@ -140,7 +183,7 @@ const QueryEditorAutocomplete: FC = ({ options={options} anchor={anchorEl} minLength={0} - offset={{ top: 0, left: leftOffset }} + offset={offsetPos} onSelect={handleSelect} onFoundOptions={onFoundOptions} maxDisplayResults={{ diff --git a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/style.scss b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/style.scss index db48a9cf1..b012403ae 100644 --- a/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/style.scss +++ b/app/vmui/packages/vmui/src/components/Configurators/QueryEditor/style.scss @@ -2,4 +2,13 @@ .vm-query-editor { position: relative; + + .marker-detection { + position: absolute; + top: 0; + left: 0; + pointer-events: none; + z-index: -9999; + visibility: hidden; + } } diff --git a/app/vmui/packages/vmui/src/components/Main/Autocomplete/Autocomplete.tsx b/app/vmui/packages/vmui/src/components/Main/Autocomplete/Autocomplete.tsx index ab83c2518..bbb15c76f 100644 --- a/app/vmui/packages/vmui/src/components/Main/Autocomplete/Autocomplete.tsx +++ b/app/vmui/packages/vmui/src/components/Main/Autocomplete/Autocomplete.tsx @@ -120,7 +120,7 @@ const Autocomplete: FC = ({ const handleKeyDown = useCallback((e: KeyboardEvent) => { const { key, ctrlKey, metaKey, shiftKey } = e; const modifiers = ctrlKey || metaKey || shiftKey; - const hasOptions = foundOptions.length; + const hasOptions = foundOptions.length && !hideFoundedOptions; if (key === "ArrowUp" && !modifiers && hasOptions) { e.preventDefault(); @@ -148,7 +148,7 @@ const Autocomplete: FC = ({ if (key === "Escape") { handleCloseAutocomplete(); } - }, [focusOption, foundOptions, handleCloseAutocomplete, onSelect, selected]); + }, [focusOption, foundOptions, hideFoundedOptions, handleCloseAutocomplete, onSelect, selected]); useEffect(() => { setOpenAutocomplete(value.length >= minLength); diff --git a/app/vmui/packages/vmui/src/components/Main/TextField/TextField.tsx b/app/vmui/packages/vmui/src/components/Main/TextField/TextField.tsx index f95b4194e..a2a938885 100644 --- a/app/vmui/packages/vmui/src/components/Main/TextField/TextField.tsx +++ b/app/vmui/packages/vmui/src/components/Main/TextField/TextField.tsx @@ -1,6 +1,7 @@ import React, { FC, useEffect, + useState, useRef, useMemo, FormEvent, @@ -28,12 +29,13 @@ interface TextFieldProps { autofocus?: boolean helperText?: string inputmode?: "search" | "text" | "email" | "tel" | "url" | "none" | "numeric" | "decimal" + caretPosition?: [number, number] onChange?: (value: string) => void onEnter?: () => void onKeyDown?: (e: KeyboardEvent) => void onFocus?: () => void onBlur?: () => void - onChangeCaret?: (position: number[]) => void + onChangeCaret?: (position: [number, number]) => void } const TextField: FC = ({ @@ -49,6 +51,7 @@ const TextField: FC = ({ disabled = false, autofocus = false, inputmode = "text", + caretPosition, onChange, onEnter, onKeyDown, @@ -62,6 +65,7 @@ const TextField: FC = ({ const inputRef = useRef(null); const textareaRef = useRef(null); const fieldRef = useMemo(() => type === "textarea" ? textareaRef : inputRef, [type]); + const [selectionPos, setSelectionPos] = useState<[start: number, end: number]>([0, 0]); const inputClasses = classNames({ "vm-text-field__input": true, @@ -74,7 +78,7 @@ const TextField: FC = ({ const updateCaretPosition = (target: HTMLInputElement | HTMLTextAreaElement) => { const { selectionStart, selectionEnd } = target; - onChangeCaret && onChangeCaret([selectionStart || 0, selectionEnd || 0]); + setSelectionPos([selectionStart || 0, selectionEnd || 0]); }; const handleMouseUp = (e: MouseEvent) => { @@ -102,11 +106,6 @@ const TextField: FC = ({ updateCaretPosition(e.currentTarget); }; - useEffect(() => { - if (!autofocus || isMobile) return; - fieldRef?.current?.focus && fieldRef.current.focus(); - }, [fieldRef, autofocus]); - const handleFocus = () => { onFocus && onFocus(); }; @@ -115,6 +114,31 @@ const TextField: FC = ({ onBlur && onBlur(); }; + const setSelectionRange = (range: [number, number]) => { + try { + fieldRef.current && fieldRef.current.setSelectionRange(range[0], range[1]); + } catch (e) { + return e; + } + }; + + useEffect(() => { + if (!autofocus || isMobile) return; + fieldRef?.current?.focus && fieldRef.current.focus(); + }, [fieldRef, autofocus]); + + useEffect(() => { + onChangeCaret && onChangeCaret(selectionPos); + }, [selectionPos]); + + useEffect(() => { + setSelectionRange(selectionPos); + }, [value]); + + useEffect(() => { + caretPosition && setSelectionRange(caretPosition); + }, [caretPosition]); + return + +## [v0.43.5](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.5) - 26 Apr 2024 + +- Update VictoriaMetrics image tags to [v1.101.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.101.0). + + + +## [v0.43.4](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.4) - 25 Apr 2024 + +- [operator](./README.md): properly set status to `expanding` for `VMCluster` during initial creation. Previously, it was always `operational`. +- [operator](./README.md): adds more context to `Deployment` and `Statefulset` watch ready functions. Now, it reports state of unhealthy pod. It allows to find issue with it faster. + + + +## [v0.43.3](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.3) - 23 Apr 2024 + +- [operator](./README.md): fix conversion from `ServiceMonitor` to `VMServiceScrape`, `bearerTokenSecret` is dropped mistakenly since [v0.43.0](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.0). See [this issue](https://github.com/VictoriaMetrics/operator/issues/932). +- [operator](./README.md): fix selector match for config resources like VMUser, VMRule... , before it could be ignored when update resource labels. + + + +## [v0.43.2](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.2) - 22 Apr 2024 + +- [vmagent](./api.md#vmagent): fixes bug with `ServiceAccount` not found with `ingestOnlyMode`. +- [vmagent](./api.md#vmagent): fixes `unknown long flag '--rules-dir'` for prometheus-config-reloader. + ## [v0.43.1](https://github.com/VictoriaMetrics/operator/releases/tag/v0.43.1) - 18 Apr 2024 diff --git a/docs/operator/api.md b/docs/operator/api.md index 2a34beb26..46f9aff02 100644 --- a/docs/operator/api.md +++ b/docs/operator/api.md @@ -136,11 +136,13 @@ This Document documents the types introduced by the VictoriaMetrics to be consum * [VMUserList](#vmuserlist) * [VMUserSpec](#vmuserspec) * [EmbeddedIngress](#embeddedingress) +* [URLMapCommon](#urlmapcommon) +* [UnauthorizedAccessConfigURLMap](#unauthorizedaccessconfigurlmap) +* [UserConfigOption](#userconfigoption) * [VMAuth](#vmauth) * [VMAuthList](#vmauthlist) * [VMAuthSpec](#vmauthspec) * [VMAuthStatus](#vmauthstatus) -* [VMAuthUnauthorizedPath](#vmauthunauthorizedpath) * [TargetEndpoint](#targetendpoint) * [VMStaticScrape](#vmstaticscrape) * [VMStaticScrapeList](#vmstaticscrapelist) @@ -925,6 +927,7 @@ VMAgentSpec defines the desired state of VMAgent | useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false | | ingestOnlyMode | IngestOnlyMode switches vmagent into unmanaged mode it disables any config generation for scraping Currently it prevents vmagent from managing tls and auth options for remote write | bool | false | | license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false | +| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false | [Back to TOC](#table-of-contents) @@ -951,7 +954,7 @@ ServiceSpec defines additional service for CRD with user-defined params. by defa | Field | Description | Scheme | Required | | ----- | ----------- | ------ | -------- | -| useAsDefault | UseAsDefault applies changes from given service definition to the main object Service Chaning from headless service to clusterIP or loadbalancer may break cross-component communication | bool | false | +| useAsDefault | UseAsDefault applies changes from given service definition to the main object Service Changing from headless service to clusterIP or loadbalancer may break cross-component communication | bool | false | | metadata | EmbeddedObjectMetadata defines objectMeta for additional service. | [EmbeddedObjectMetadata](#embeddedobjectmetadata) | false | | spec | ServiceSpec describes the attributes that a user creates on a service. More info: https://kubernetes.io/docs/concepts/services-networking/service/ | v1.ServiceSpec | true | @@ -1132,11 +1135,16 @@ StreamAggrRule defines the rule in stream aggregation config | ----- | ----------- | ------ | -------- | | match | Match is a label selector (or list of label selectors) for filtering time series for the given selector.\n\nIf the match isn't set, then all the input time series are processed. | StringOrArray | false | | interval | Interval is the interval between aggregations. | string | true | -| staleness_interval | StalenessInterval defines an interval after which the series state will be reset if no samples have been sent during it. | string | false | +| no_align_flush_to_interval | NoAlighFlushToInterval disables aligning of flushes to multiples of Interval. By default flushes are aligned to Interval. | *bool | false | | flush_on_shutdown | FlushOnShutdown defines whether to flush the aggregation state on process termination or config reload. Is `false` by default. It is not recommended changing this setting, unless unfinished aggregations states are preferred to missing data points. | bool | false | +| dedup_interval | DedupInterval is an optional interval for deduplication. | string | false | +| staleness_interval | Staleness interval is interval after which the series state will be reset if no samples have been sent during it. The parameter is only relevant for outputs: total, total_prometheus, increase, increase_prometheus and histogram_bucket. | string | false | | outputs | Outputs is a list of output aggregate functions to produce.\n\nThe following names are allowed:\n\n- total - aggregates input counters - increase - counts the increase over input counters - count_series - counts the input series - count_samples - counts the input samples - sum_samples - sums the input samples - last - the last biggest sample value - min - the minimum sample value - max - the maximum sample value - avg - the average value across all the samples - stddev - standard deviation across all the samples - stdvar - standard variance across all the samples - histogram_bucket - creates VictoriaMetrics histogram for input samples - quantiles(phi1, ..., phiN) - quantiles' estimation for phi in the range [0..1]\n\nThe output time series will have the following names:\n\n input_name:aggr_<interval>_<output> | []string | true | +| keep_metric_names | KeepMetricNames instructs to leave metric names as is for the output time series without adding any suffix. | *bool | false | +| ignore_old_samples | IgnoreOldSamples instructs to ignore samples with old timestamps outside the current aggregation interval. | *bool | false | | by | By is an optional list of labels for grouping input series.\n\nSee also Without.\n\nIf neither By nor Without are set, then the Outputs are calculated individually per each input time series. | []string | false | | without | Without is an optional list of labels, which must be excluded when grouping input series.\n\nSee also By.\n\nIf neither By nor Without are set, then the Outputs are calculated individually per each input time series. | []string | false | +| drop_input_labels | DropInputLabels is an optional list with labels, which must be dropped before further processing of input samples.\n\nLabels are dropped before de-duplication and aggregation. | *[]string | false | | input_relabel_configs | InputRelabelConfigs is an optional relabeling rules, which are applied on the input before aggregation. | [][RelabelConfig](#relabelconfig) | false | | output_relabel_configs | OutputRelabelConfigs is an optional relabeling rules, which are applied on the aggregated output before being sent to remote storage. | [][RelabelConfig](#relabelconfig) | false | @@ -1297,6 +1305,7 @@ VMAlertSpec defines the desired state of VMAlert | readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false | | useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false | | license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false | +| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false | [Back to TOC](#table-of-contents) @@ -1391,6 +1400,7 @@ VMSingleSpec defines the desired state of VMSingle | readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false | | streamAggrConfig | StreamAggrConfig defines stream aggregation configuration for VMSingle | *[StreamAggrConfig](#streamaggrconfig) | false | | useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false | +| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false | [Back to TOC](#table-of-contents) @@ -1877,6 +1887,7 @@ VMClusterSpec defines the desired state of VMCluster | vmselect | | *[VMSelect](#vmselect) | false | | vminsert | | *[VMInsert](#vminsert) | false | | vmstorage | | *[VMStorage](#vmstorage) | false | +| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false | | useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false | [Back to TOC](#table-of-contents) @@ -2160,12 +2171,8 @@ TargetRef describes target for user traffic forwarding. one of target types can | static | Static - user defined url for traffic forward, for instance http://vmsingle:8429 | *[StaticRef](#staticref) | false | | paths | Paths - matched path to route. | []string | false | | hosts | | []string | false | -| target_path_suffix | QueryParams []string `json:\"queryParams,omitempty\"` TargetPathSuffix allows to add some suffix to the target path It allows to hide tenant configuration from user with crd as ref. it also may contain any url encoded params. | string | false | -| headers | Headers represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | -| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | -| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries Can be defined per target or at VMUser.spec level e.g. [429,503] | []int | false | -| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | -| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | +| URLMapCommon | | [URLMapCommon](#urlmapcommon) | false | +| target_path_suffix | TargetPathSuffix allows to add some suffix to the target path It allows to hide tenant configuration from user with crd as ref. it also may contain any url encoded params. | string | false | | targetRefBasicAuth | TargetRefBasicAuth allow an target endpoint to authenticate over basic authentication | *[TargetRefBasicAuth](#targetrefbasicauth) | false | [Back to TOC](#table-of-contents) @@ -2230,14 +2237,15 @@ VMUserSpec defines the desired state of VMUser | bearerToken | BearerToken Authorization header value for accessing protected endpoint. | *string | false | | targetRefs | TargetRefs - reference to endpoints, which user may access. | [][TargetRef](#targetref) | true | | default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false | +| tlsConfig | | *[TLSConfig](#tlsconfig) | false | | ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false | +| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false | | headers | Headers represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | | response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | | retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false | | max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false | | load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | | drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | -| tls_insecure_skip_verify | TLSInsecureSkipVerify - whether to skip TLS verification when connecting to backend over HTTPS. See https://docs.victoriametrics.com/vmauth.html#backend-tls-setup | bool | false | | metric_labels | MetricLabels - additional labels for metrics exported by vmauth for given user. | map[string]string | false | | disable_secret_creation | DisableSecretCreation skips related secret creation for vmuser | bool | false | @@ -2261,6 +2269,55 @@ EmbeddedIngress describes ingress configuration options. [Back to TOC](#table-of-contents) +## URLMapCommon + +URLMapCommon contains common fields for unauthorized user and user in vmuser + +| Field | Description | Scheme | Required | +| ----- | ----------- | ------ | -------- | +| src_query_args | SrcQueryArgs is an optional list of query args, which must match request URL query args. | []string | false | +| src_headers | SrcHeaders is an optional list of headers, which must match request headers. | []string | false | +| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false | +| headers | RequestHeaders represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | +| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | +| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries Can be defined per target or at VMUser.spec level e.g. [429,503] | []int | false | +| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | +| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | + +[Back to TOC](#table-of-contents) + +## UnauthorizedAccessConfigURLMap + + + +| Field | Description | Scheme | Required | +| ----- | ----------- | ------ | -------- | +| src_paths | SrcPaths is an optional list of regular expressions, which must match the request path. | []string | false | +| src_hosts | SrcHosts is an optional list of regular expressions, which must match the request hostname. | []string | false | +| url_prefix | UrlPrefix contains backend url prefixes for the proxied request url. | []string | false | +| URLMapCommon | | [URLMapCommon](#urlmapcommon) | false | + +[Back to TOC](#table-of-contents) + +## UserConfigOption + + + +| Field | Description | Scheme | Required | +| ----- | ----------- | ------ | -------- | +| default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false | +| tlsConfig | | *[TLSConfig](#tlsconfig) | false | +| ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false | +| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false | +| headers | Headers represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | +| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | +| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false | +| max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false | +| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | +| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | + +[Back to TOC](#table-of-contents) + ## VMAuth VMAuth is the Schema for the vmauths API @@ -2334,10 +2391,21 @@ VMAuthSpec defines the desired state of VMAuth | nodeSelector | NodeSelector Define which Nodes the Pods are scheduled on. | map[string]string | false | | terminationGracePeriodSeconds | TerminationGracePeriodSeconds period for container graceful termination | *int64 | false | | readinessGates | ReadinessGates defines pod readiness gates | []v1.PodReadinessGate | false | -| unauthorizedAccessConfig | UnauthorizedAccessConfig configures access for un authorized users | [][VMAuthUnauthorizedPath](#vmauthunauthorizedpath) | false | +| unauthorizedAccessConfig | UnauthorizedAccessConfig configures access for un authorized users | [][UnauthorizedAccessConfigURLMap](#unauthorizedaccessconfigurlmap) | false | +| default_url | DefaultURLs backend url for non-matching paths filter usually used for default backend with error message | []string | false | +| tlsConfig | | *[TLSConfig](#tlsconfig) | false | +| ip_filters | IPFilters defines per target src ip filters supported only with enterprise version of vmauth https://docs.victoriametrics.com/vmauth.html#ip-filters | [VMUserIPFilters](#vmuseripfilters) | false | +| discover_backend_ips | DiscoverBackendIPs instructs discovering URLPrefix backend IPs via DNS. | *bool | false | +| headers | Headers represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | +| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | +| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false | +| max_concurrent_requests | MaxConcurrentRequests defines max concurrent requests per user 300 is default value for vmauth | *int | false | +| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | +| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | | useStrictSecurity | UseStrictSecurity enables strict security mode for component it restricts disk writes access uses non-root user out of the box drops not needed security permissions | *bool | false | | license | License allows to configure license key to be used for enterprise features. Using license key is supported starting from VictoriaMetrics v1.94.0. See: https://docs.victoriametrics.com/enterprise.html | *[License](#license) | false | | configSecret | ConfigSecret is the name of a Kubernetes Secret in the same namespace as the VMAuth object, which contains auth configuration for vmauth, configuration must be inside secret key: config.yaml. It must be created and managed manually. If it's defined, configuration for vmauth becomes unmanaged and operator'll not create any related secrets/config-reloaders | string | false | +| paused | Paused If set to true all actions on the underlaying managed objects are not going to be performed, except for delete actions. | bool | false | [Back to TOC](#table-of-contents) @@ -2352,24 +2420,6 @@ VMAuthStatus defines the observed state of VMAuth [Back to TOC](#table-of-contents) -## VMAuthUnauthorizedPath - -VMAuthUnauthorizedPath defines url_map for unauthorized access - -| Field | Description | Scheme | Required | -| ----- | ----------- | ------ | -------- | -| src_paths | Paths src request paths | []string | false | -| url_prefix | URLs defines url_prefix for dst routing | []string | false | -| ip_filters | IPFilters defines filter for src ip address enterprise only | [VMUserIPFilters](#vmuseripfilters) | false | -| src_hosts | SrcHosts is the list of regular expressions, which match the request hostname. | []string | false | -| headers | Headers represent additional http headers, that vmauth uses in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.68.0 version of vmauth | []string | false | -| response_headers | ResponseHeaders represent additional http headers, that vmauth adds for request response in form of [\"header_key: header_value\"] multiple values for header key: [\"header_key: value1,value2\"] it's available since 1.93.0 version of vmauth | []string | false | -| retry_status_codes | RetryStatusCodes defines http status codes in numeric format for request retries e.g. [429,503] | []int | false | -| load_balancing_policy | LoadBalancingPolicy defines load balancing policy to use for backend urls. Supported policies: least_loaded, first_available. See https://docs.victoriametrics.com/vmauth.html#load-balancing for more details (default \"least_loaded\") | *string | false | -| drop_src_path_prefix_parts | DropSrcPathPrefixParts is the number of `/`-delimited request path prefix parts to drop before proxying the request to backend. See https://docs.victoriametrics.com/vmauth.html#dropping-request-path-prefix for more details. | *int | false | - -[Back to TOC](#table-of-contents) - ## TargetEndpoint TargetEndpoint defines single static target endpoint. diff --git a/docs/operator/resources/vmauth.md b/docs/operator/resources/vmauth.md index cac3df05c..6146d454e 100644 --- a/docs/operator/resources/vmauth.md +++ b/docs/operator/resources/vmauth.md @@ -33,7 +33,7 @@ The CRD specifies which `VMUser`s should be covered by the deployed `VMAuth` ins The Operator then generates a configuration based on the included `VMUser`s and updates the `Configmaps` containing the configuration. It continuously does so for all changes that are made to `VMUser`s or to the `VMAuth` resource itself. -[VMUser](./vmrule.md) objects generate part of `VMAuth` configuration. +[VMUser](./vmuser.md) objects generate part of `VMAuth` configuration. For filtering users `VMAuth` uses selectors `userNamespaceSelector` and `userSelector`. It allows configuring rules access control across namespaces and different environments. @@ -101,8 +101,8 @@ metadata: name: vmauth-unauthorized-example spec: unauthorizedAccessConfig: - - paths: ["/metrics"] - urls: + - src_paths: ["/metrics"] + url_prefix: - http://vmsingle-example.default.svc:8428 ``` @@ -245,8 +245,8 @@ spec: - 5.6.7.8 # allow read vmsingle metrics without authorization for users from internal network unauthorizedAccessConfig: - - paths: ["/metrics"] - urls: ["http://vmsingle-example.default.svc:8428"] + - src_paths: ["/metrics"] + url_prefix: ["http://vmsingle-example.default.svc:8428"] ip_filters: allow_list: - 192.168.0.0/16 diff --git a/docs/operator/vars.md b/docs/operator/vars.md index f293cefe3..b36cccee2 100644 --- a/docs/operator/vars.md +++ b/docs/operator/vars.md @@ -10,7 +10,7 @@ menu: # Auto Generated vars for package config - updated at Thu Apr 18 19:07:23 UTC 2024 + updated at Thu May 9 16:51:28 UTC 2024 | varible name | variable default value | variable required | variable description | @@ -20,7 +20,7 @@ menu: | VM_CUSTOMCONFIGRELOADERIMAGE | victoriametrics/operator:config-reloader-v0.43.0 | false | - | | VM_PSPAUTOCREATEENABLED | false | false | - | | VM_VMALERTDEFAULT_IMAGE | victoriametrics/vmalert | false | - | -| VM_VMALERTDEFAULT_VERSION | v1.100.1 | false | - | +| VM_VMALERTDEFAULT_VERSION | v1.101.0 | false | - | | VM_VMALERTDEFAULT_PORT | 8080 | false | - | | VM_VMALERTDEFAULT_USEDEFAULTRESOURCES | true | false | - | | VM_VMALERTDEFAULT_RESOURCE_LIMIT_MEM | 500Mi | false | - | @@ -31,7 +31,7 @@ menu: | VM_VMALERTDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - | | VM_VMALERTDEFAULT_CONFIGRELOADIMAGE | jimmidyson/configmap-reload:v0.3.0 | false | - | | VM_VMAGENTDEFAULT_IMAGE | victoriametrics/vmagent | false | - | -| VM_VMAGENTDEFAULT_VERSION | v1.100.1 | false | - | +| VM_VMAGENTDEFAULT_VERSION | v1.101.0 | false | - | | VM_VMAGENTDEFAULT_CONFIGRELOADIMAGE | quay.io/prometheus-operator/prometheus-config-reloader:v0.68.0 | false | - | | VM_VMAGENTDEFAULT_PORT | 8429 | false | - | | VM_VMAGENTDEFAULT_USEDEFAULTRESOURCES | true | false | - | @@ -42,7 +42,7 @@ menu: | VM_VMAGENTDEFAULT_CONFIGRELOADERCPU | 100m | false | - | | VM_VMAGENTDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - | | VM_VMSINGLEDEFAULT_IMAGE | victoriametrics/victoria-metrics | false | - | -| VM_VMSINGLEDEFAULT_VERSION | v1.100.1 | false | - | +| VM_VMSINGLEDEFAULT_VERSION | v1.101.0 | false | - | | VM_VMSINGLEDEFAULT_PORT | 8429 | false | - | | VM_VMSINGLEDEFAULT_USEDEFAULTRESOURCES | true | false | - | | VM_VMSINGLEDEFAULT_RESOURCE_LIMIT_MEM | 1500Mi | false | - | @@ -53,14 +53,14 @@ menu: | VM_VMSINGLEDEFAULT_CONFIGRELOADERMEMORY | 25Mi | false | - | | VM_VMCLUSTERDEFAULT_USEDEFAULTRESOURCES | true | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_IMAGE | victoriametrics/vmselect | false | - | -| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_VERSION | v1.100.1-cluster | false | - | +| VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_VERSION | v1.101.0-cluster | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_PORT | 8481 | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_LIMIT_MEM | 1000Mi | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_LIMIT_CPU | 500m | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_REQUEST_MEM | 500Mi | false | - | | VM_VMCLUSTERDEFAULT_VMSELECTDEFAULT_RESOURCE_REQUEST_CPU | 100m | false | - | | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_IMAGE | victoriametrics/vmstorage | false | - | -| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VERSION | v1.100.1-cluster | false | - | +| VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VERSION | v1.101.0-cluster | false | - | | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VMINSERTPORT | 8400 | false | - | | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_VMSELECTPORT | 8401 | false | - | | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_PORT | 8482 | false | - | @@ -69,7 +69,7 @@ menu: | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_RESOURCE_REQUEST_MEM | 500Mi | false | - | | VM_VMCLUSTERDEFAULT_VMSTORAGEDEFAULT_RESOURCE_REQUEST_CPU | 250m | false | - | | VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_IMAGE | victoriametrics/vminsert | false | - | -| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_VERSION | v1.100.1-cluster | false | - | +| VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_VERSION | v1.101.0-cluster | false | - | | VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_PORT | 8480 | false | - | | VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_RESOURCE_LIMIT_MEM | 500Mi | false | - | | VM_VMCLUSTERDEFAULT_VMINSERTDEFAULT_RESOURCE_LIMIT_CPU | 500m | false | - | @@ -88,7 +88,7 @@ menu: | VM_VMALERTMANAGER_RESOURCE_REQUEST_CPU | 30m | false | - | | VM_DISABLESELFSERVICESCRAPECREATION | false | false | - | | VM_VMBACKUP_IMAGE | victoriametrics/vmbackupmanager | false | - | -| VM_VMBACKUP_VERSION | v1.100.1-enterprise | false | - | +| VM_VMBACKUP_VERSION | v1.101.0-enterprise | false | - | | VM_VMBACKUP_PORT | 8300 | false | - | | VM_VMBACKUP_USEDEFAULTRESOURCES | true | false | - | | VM_VMBACKUP_RESOURCE_LIMIT_MEM | 500Mi | false | - | @@ -97,7 +97,7 @@ menu: | VM_VMBACKUP_RESOURCE_REQUEST_CPU | 150m | false | - | | VM_VMBACKUP_LOGLEVEL | INFO | false | - | | VM_VMAUTHDEFAULT_IMAGE | victoriametrics/vmauth | false | - | -| VM_VMAUTHDEFAULT_VERSION | v1.100.1 | false | - | +| VM_VMAUTHDEFAULT_VERSION | v1.101.0 | false | - | | VM_VMAUTHDEFAULT_CONFIGRELOADIMAGE | quay.io/prometheus-operator/prometheus-config-reloader:v0.68.0 | false | - | | VM_VMAUTHDEFAULT_PORT | 8427 | false | - | | VM_VMAUTHDEFAULT_USEDEFAULTRESOURCES | true | false | - | diff --git a/docs/scrape_config_examples.md b/docs/scrape_config_examples.md index 26f476b8d..d292722fb 100644 --- a/docs/scrape_config_examples.md +++ b/docs/scrape_config_examples.md @@ -250,7 +250,7 @@ scrape_configs: # If node-exporter containers have another name in your Kubernetes cluster, # then adjust the regex value accordingly. # - - soruce_labels: [__meta_kubernetes_pod_container_name] + - source_labels: [__meta_kubernetes_pod_container_name] regex: node-exporter action: keep diff --git a/docs/sd_configs.md b/docs/sd_configs.md index f2357d020..eda9a4a8c 100644 --- a/docs/sd_configs.md +++ b/docs/sd_configs.md @@ -35,6 +35,7 @@ supports the following Prometheus-compatible service discovery options for Prome * `nomad_sd_configs` is for discovering and scraping targets registered in [HashiCorp Nomad](https://www.nomadproject.io/). See [these docs](#nomad_sd_configs). * `openstack_sd_configs` is for discovering and scraping OpenStack targets. See [these docs](#openstack_sd_configs). * `static_configs` is for scraping statically defined targets. See [these docs](#static_configs). +* `vultr_sd_configs` is for discovering and scraping [Vultr](https://www.vultr.com/) targets. See [these docs](#vultr_sd_configs). * `yandexcloud_sd_configs` is for discovering and scraping [Yandex Cloud](https://cloud.yandex.com/en/) targets. See [these docs](#yandexcloud_sd_configs). Note that the `refresh_interval` option isn't supported for these scrape configs. Use the corresponding `-promscrape.*CheckInterval` @@ -1498,6 +1499,79 @@ scrape_configs: See [these examples](https://docs.victoriametrics.com/scrape_config_examples/#static-configs) on how to configure scraping for static targets. +## vultr_sd_configs +Vultr SD configuration discovers scrape targets from [Vultr](https://www.vultr.com/) Instances. + +Configuration example: + +```yaml +scrape_configs: +- job_name: vultr + vultr_sd_configs: + + # bearer_token is a Bearer token to send in every HTTP API request during service discovery (mandatory). + # See: https://my.vultr.com/settings/#settingsapi + - bearer_token: "..." + + # Vultr provides query arguments to filter instances. + # See: https://www.vultr.com/api/#tag/instances + + # label is an optional query arguments to filter instances by label. + # + # label: "..." + + # main_ip is an optional query arguments to filter instances by main ip address. + # + # main_ip: "..." + + # region is an optional query arguments to filter instances by region id. + # + # region: "..." + + # firewall_group_id is an optional query arguments to filter instances by firewall group id. + # + # firewall_group_id: "..." + + # hostname is an optional query arguments to filter instances by hostname. + # + # hostname: "..." + + # port is an optional port to scrape metrics from. + # By default, port 80 is used. + # + # port: ... + + # Additional HTTP API client options can be specified here. + # See https://docs.victoriametrics.com/sd_configs.html#http-api-client-options + + +``` + +Each discovered target has an [`__address__`](https://docs.victoriametrics.com/relabeling.html#how-to-modify-scrape-urls-in-targets) label set +to `:`, where FQDN is discovered instance address and `` is the port from the `vultr_sd_configs` (default port is `80`). + +The following meta labels are available on discovered targets during [relabeling](https://docs.victoriametrics.com/vmagent.html#relabeling): + +* `__meta_vultr_instance_id`: A unique ID for the VPS Instance. +* `__meta_vultr_instance_label`: The user-supplied label for this instance. +* `__meta_vultr_instance_os`: The [Operating System name](https://www.vultr.com/api/#operation/list-os). +* `__meta_vultr_instance_os_id`: The [Operating System id](https://www.vultr.com/api/#operation/list-os) used by this instance. +* `__meta_vultr_instance_region`: The [Region id](https://www.vultr.com/api/#operation/list-regions) where the Instance is located. +* `__meta_vultr_instance_plan`: A unique ID for the Plan. +* `__meta_vultr_instance_main_ip`: The main IPv4 address. +* `__meta_vultr_instance_internal_ip`: The internal IP used by this instance, if set. Only relevant when a VPC is attached. +* `__meta_vultr_instance_main_ipv6`: The main IPv6 network address. +* `__meta_vultr_instance_hostname`: The hostname for this instance. +* `__meta_vultr_instance_server_status`: The server health status, which could be `none`, `locked`, `installingbooting`, `ok`. +* `__meta_vultr_instance_vcpu_count`: Number of vCPUs. +* `__meta_vultr_instance_ram_mb`: The amount of RAM in MB. +* `__meta_vultr_instance_allowed_bandwidth_gb`: Monthly bandwidth quota in GB. +* `__meta_vultr_instance_disk_gb`: The size of the disk in GB. +* `__meta_vultr_instance_features`: "auto_backups", "ipv6", "ddos_protection". +* `__meta_vultr_instance_tags`: Tags to apply to the instance. + +The list of discovered Vultr targets is refreshed at the interval, which can be configured via `-promscrape.vultrSDCheckInterval` command-line flag, default: 30s. + ## yandexcloud_sd_configs [Yandex Cloud](https://cloud.yandex.com/en/) SD configurations allow retrieving scrape targets from accessible folders. diff --git a/docs/stream-aggregation.md b/docs/stream-aggregation.md index 5b7027c2f..820ca4eef 100644 --- a/docs/stream-aggregation.md +++ b/docs/stream-aggregation.md @@ -19,7 +19,7 @@ The aggregation is applied to all the metrics received via any [supported data i and/or scraped from [Prometheus-compatible targets](https://docs.victoriametrics.com/#how-to-scrape-prometheus-exporters-such-as-node-exporter) after applying all the configured [relabeling stages](https://docs.victoriametrics.com/vmagent/#relabeling). -By default stream aggregation ignores timestamps associated with the input [samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). +By default, stream aggregation ignores timestamps associated with the input [samples](https://docs.victoriametrics.com/keyconcepts/#raw-samples). It expects that the ingested samples have timestamps close to the current time. See [how to ignore old samples](#ignoring-old-samples). Stream aggregation can be configured via the following command-line flags: @@ -82,7 +82,7 @@ The online de-duplication uses the same logic as [`-dedup.minScrapeInterval` com ## Ignoring old samples -By default all the input samples are taken into account during stream aggregation. If samples with old timestamps outside the current [aggregation interval](#stream-aggregation-config) +By default, all the input samples are taken into account during stream aggregation. If samples with old timestamps outside the current [aggregation interval](#stream-aggregation-config) must be ignored, then the following options can be used: - To pass `-remoteWrite.streamAggr.ignoreOldSamples` command-line flag to [vmagent](https://docs.victoriametrics.com/vmagent/) @@ -92,9 +92,25 @@ must be ignored, then the following options can be used: - To set `ignore_old_samples: true` option at the particular [aggregation config](#stream-aggregation-config). This enables ignoring old samples for that particular aggregation config. +## Ignore aggregation intervals on start + +Stream aggregation may yield inaccurate results if it processes incomplete data. This issue can arise when data is +received from clients that maintain a queue of unsent data, such as Prometheus or vmagent. If the queue isn't fully +cleared within the aggregation `interval`, only a portion of the time series may be processed, leading to distorted +calculations. To mitigate this, consider the following options: + +- Set `-remoteWrite.streamAggr.ignoreFirstIntervals=` command-line flag to [vmagent](https://docs.victoriametrics.com/vmagent/) + or `-streamAggr.ignoreFirstIntervals=` command-line flag to [single-node VictoriaMetrics](https://docs.victoriametrics.com/) + to skip first `` [aggregation intervals](#stream-aggregation-config) + from persisting to the storage. It is expected that all incomplete or queued data will be processed during + specified `` and all subsequent aggregation intervals will produce correct data. + +- Set `ignore_first_intervals: ` option individually per [aggregation config](#stream-aggregation-config). + This enables ignoring first `` aggregation intervals for that particular aggregation config. + ## Flush time alignment -By default the time for aggregated data flush is aligned by the `interval` option specified in [aggregate config](#stream-aggregation-config). +By default, the time for aggregated data flush is aligned by the `interval` option specified in [aggregate config](#stream-aggregation-config). For example: - if `interval: 1m` is set, then the aggregated data is flushed to the storage at the end of every minute - if `interval: 1h` is set, then the aggregated data is flushed to the storage at the end of every hour @@ -117,7 +133,7 @@ Stream aggregation can be used in the following cases: ### Statsd alternative -Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) alternative in the following cases: +Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) drop-in replacement in the following cases: * [Counting input samples](#counting-input-samples) * [Summing input metrics](#summing-input-metrics) @@ -125,9 +141,6 @@ Stream aggregation can be used as [statsd](https://github.com/statsd/statsd) alt * [Histograms over input metrics](#histograms-over-input-metrics) * [Aggregating histograms](#aggregating-histograms) -Currently, streaming aggregation is available only for [supported data ingestion protocols](https://docs.victoriametrics.com/#how-to-import-time-series-data) -and not available for [Statsd metrics format](https://github.com/statsd/statsd/blob/master/docs/metric_types.md). - ### Recording rules alternative Sometimes [alerting queries](https://docs.victoriametrics.com/vmalert/#alerting-rules) may require non-trivial amounts of CPU, RAM, @@ -582,9 +595,6 @@ some_counter:5m_increase / 5m This is similar to `rate(some_counter[5m])`. -Please note, opposite to [rate](https://docs.victoriametrics.com/metricsql/#rate), `increase` aggregations can be -combined safely afterwards. This is helpful when the aggregation is calculated by more than one vmagent. - Aggregating irregular and sporadic metrics (received from [Lambdas](https://aws.amazon.com/lambda/) or [Cloud Functions](https://cloud.google.com/functions)) can be controlled via [staleness_interval](#staleness) option. @@ -875,7 +885,7 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server- # Samples are de-duplicated on a per-series basis. See https://docs.victoriametrics.com/keyconcepts/#time-series # and https://docs.victoriametrics.com/#deduplication # The deduplication is performed after input_relabel_configs relabeling is applied. - # By default the deduplication is disabled unless -remoteWrite.streamAggr.dedupInterval or -streamAggr.dedupInterval + # By default, the deduplication is disabled unless -remoteWrite.streamAggr.dedupInterval or -streamAggr.dedupInterval # command-line flags are set. # # dedup_interval: 30s @@ -892,7 +902,7 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server- # staleness_interval: 2m # no_align_flush_to_interval disables aligning of flush times for the aggregated data to multiples of interval. - # By default flush times for the aggregated data is aligned to multiples of interval. + # By default, flush times for the aggregated data is aligned to multiples of interval. # For example: # - if `interval: 1m` is set, then flushes happen at the end of every minute, # - if `interval: 1h` is set, then flushes happen at the end of every hour @@ -922,16 +932,23 @@ at [single-node VictoriaMetrics](https://docs.victoriametrics.com/single-server- # keep_metric_names instructs keeping the original metric names for the aggregated samples. # This option can be set only if outputs list contains only a single output. - # By default a special suffix is added to original metric names in the aggregated samples. + # By default, a special suffix is added to original metric names in the aggregated samples. # See https://docs.victoriametrics.com/stream-aggregation/#output-metric-names # # keep_metric_names: false # ignore_old_samples instructs ignoring input samples with old timestamps outside the current aggregation interval. + # See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples # See also -streamAggr.ignoreOldSamples command-line flag. # # ignore_old_samples: false + # ignore_first_intervals instructs ignoring first N aggregation intervals after process start. + # See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start + # See also -remoteWrite.streamAggr.ignoreFirstIntervals or -streamAggr.ignoreFirstIntervals + # + # ignore_first_intervals: false + # drop_input_labels instructs dropping the given labels from input samples. # The labels' dropping is performed before input_relabel_configs are applied. # This also means that the labels are dropped before de-duplication ( https://docs.victoriametrics.com/stream-aggregation/#deduplication ) @@ -1004,7 +1021,7 @@ These issues can be be fixed in the following ways: - By increasing the `interval` option at [stream aggregation config](#stream-aggregation-config), so it covers the expected delays in data ingestion pipelines. - By specifying the `staleness_interval` option at [stream aggregation config](#stream-aggregation-config), so it covers the expected - delays in data ingestion pipelines. By default the `staleness_interval` equals to `2 x interval`. + delays in data ingestion pipelines. By default, the `staleness_interval` equals to `2 x interval`. ### High resource usage diff --git a/docs/vmagent.md b/docs/vmagent.md index ecc2f0e07..5c77e4f35 100644 --- a/docs/vmagent.md +++ b/docs/vmagent.md @@ -107,6 +107,7 @@ additionally to pull-based Prometheus-compatible targets' scraping: * DataDog "submit metrics" API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-datadog-agent). * InfluxDB line protocol via `http://:8429/write`. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-influxdb-compatible-agents-such-as-telegraf). * Graphite plaintext protocol if `-graphiteListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-graphite-compatible-agents-such-as-statsd). +* Statsd plaintext protocol if `-statsdListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-statsd-compatible-clients). * OpenTelemetry http API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#sending-data-via-opentelemetry). * NewRelic API. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-newrelic-agent). * OpenTSDB telnet and http protocols if `-opentsdbListenAddr` command-line flag is set. See [these docs](https://docs.victoriametrics.com/single-server-victoriametrics/#how-to-send-data-from-opentsdb-compatible-agents). @@ -1186,7 +1187,7 @@ If you have suggestions for improvements or have found a bug - please open an is with `-remoteWrite.maxDiskUsagePerURL` command-line flag. If you don't want to send all the buffered data from the directory to remote storage then simply stop `vmagent` and delete the directory. -* If `vmagent` runs on a host with slow persistent storage, which cannot keep up with the volume of processed samples, then is is possible to disable +* If `vmagent` runs on a host with slow persistent storage, which cannot keep up with the volume of processed samples, then is possible to disable the persistent storage with `-remoteWrite.disableOnDiskQueue` command-line flag. See [these docs](#disabling-on-disk-persistence) for more details. * By default `vmagent` masks `-remoteWrite.url` with `secret-url` values in logs and at `/metrics` page because @@ -1707,6 +1708,10 @@ See the docs at https://docs.victoriametrics.com/vmagent/ . Whether to use proxy protocol for connections accepted at -graphiteListenAddr . See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt -graphiteTrimTimestamp duration Trim timestamps for Graphite data to this duration. Minimum practical duration is 1s. Higher duration (i.e. 1m) may be used for reducing disk space usage for timestamp data (default 1s) + -statsdListenAddr string + TCP and UDP address to listen for Statsd plaintext data. Usually :8125 must be set. Doesn't work if empty. See also -statsdListenAddr.useProxyProtocol + -statsdListenAddr.useProxyProtocol + Whether to use proxy protocol for connections accepted at -statsdListenAddr . See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt -http.connTimeout duration Incoming connections to -httpListenAddr are closed after the configured timeout. This may help evenly spreading load among a cluster of services behind TCP-level load balancer. Zero value disables closing of incoming connections (default 2m0s) -http.disableResponseCompression @@ -1977,6 +1982,8 @@ See the docs at https://docs.victoriametrics.com/vmagent/ . Whether to suppress scrape errors logging. The last error for each target is always available at '/targets' page even if scrape errors logging is suppressed. See also -promscrape.suppressScrapeErrorsDelay -promscrape.suppressScrapeErrorsDelay duration The delay for suppressing repeated scrape errors logging per each scrape targets. This may be used for reducing the number of log lines related to scrape errors. See also -promscrape.suppressScrapeErrors + -promscrape.vultrSDCheckInterval duration + Interval for checking for changes in Vultr. This works only if vultr_sd_configs is configured in '-promscrape.config' file. See https://docs.victoriametrics.com/sd_configs.html#vultr_sd_configs for details (default 30s) -promscrape.yandexcloudSDCheckInterval duration Interval for checking for changes in Yandex Cloud API. This works only if yandexcloud_sd_configs is configured in '-promscrape.config' file. See https://docs.victoriametrics.com/sd_configs/#yandexcloud_sd_configs for details (default 30s) -pushmetrics.disableCompression @@ -2050,10 +2057,14 @@ See the docs at https://docs.victoriametrics.com/vmagent/ . Optional path to bearer token file to use for the corresponding -remoteWrite.url. The token is re-read from the file every second Supports an array of values separated by comma or specified via multiple flags. Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces. - -remoteWrite.disableOnDiskQueue - Whether to disable storing pending data to -remoteWrite.tmpDataPath when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence .See also -remoteWrite.dropSamplesOnOverload - -remoteWrite.dropSamplesOnOverload - Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent/#disabling-on-disk-persistence + -remoteWrite.disableOnDiskQueue array + Whether to disable storing pending data to -remoteWrite.tmpDataPath when the configured remote storage systems cannot keep up with the data ingestion rate. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence .See also -remoteWrite.dropSamplesOnOverload + Supports array of values separated by comma or specified via multiple flags. + Empty values are set to false. + -remoteWrite.dropSamplesOnOverload array + Whether to drop samples when -remoteWrite.disableOnDiskQueue is set and if the samples cannot be pushed into the configured remote storage systems in a timely manner. See https://docs.victoriametrics.com/vmagent#disabling-on-disk-persistence + Supports array of values separated by comma or specified via multiple flags. + Empty values are set to false. -remoteWrite.flushInterval duration Interval for flushing the data to remote storage. This option takes effect only when less than 10K data points per second are pushed to -remoteWrite.url (default 1s) -remoteWrite.forcePromProto array @@ -2166,6 +2177,8 @@ See the docs at https://docs.victoriametrics.com/vmagent/ . Whether to drop all the input samples after the aggregation with -remoteWrite.streamAggr.config. By default, only aggregates samples are dropped, while the remaining samples are written to the corresponding -remoteWrite.url . See also -remoteWrite.streamAggr.keepInput and https://docs.victoriametrics.com/stream-aggregation/ Supports array of values separated by comma or specified via multiple flags. Empty values are set to false. + -remoteWrite.streamAggr.ignoreFirstIntervals int + Number of aggregation intervals to skip after the start. Increase this value if you observe incorrect aggregation results after vmagent restarts. It could be caused by receiving unordered delayed data from clients pushing data into the vmagent. See https://docs.victoriametrics.com/stream-aggregation/#ignore-aggregation-intervals-on-start -remoteWrite.streamAggr.ignoreOldSamples array Whether to ignore input samples with old timestamps outside the current aggregation interval for the corresponding -remoteWrite.streamAggr.config . See https://docs.victoriametrics.com/stream-aggregation/#ignoring-old-samples Supports array of values separated by comma or specified via multiple flags. diff --git a/docs/vmalert.md b/docs/vmalert.md index 1534ee835..e8c79b264 100644 --- a/docs/vmalert.md +++ b/docs/vmalert.md @@ -902,6 +902,25 @@ max(vmalert_alerting_rules_last_evaluation_series_fetched) by(group, alertname) See more details [here](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4039). This feature is available only if vmalert is using VictoriaMetrics v1.90 or higher as a datasource. +### Series with the same labelset + +vmalert can produce the following error message: +``` +result contains metrics with the same labelset during evaluation +``` + +The error means there is a collision between [time series](https://docs.victoriametrics.com/keyConcepts.html#time-series) +during evaluation. + +For example, a rule with `expr: {__name__=~"vmalert_alerts_.*"} > 0` returns two distinct time series in response: +``` +{__name__="vmalert_alerts_pending",job="vmalert",alertname="HostContextSwitching"} 12 +{__name__="vmalert_alerts_firing",job="vmalert",alertname="HostContextSwitching"} 0 +``` + +As label `__name__` will be dropped during evaluation, leads to duplicated time series. +To fix this, one could use function like [label_replace](https://docs.victoriametrics.com/metricsql/#label_replace) to preserve the distinct labelset. + ## mTLS protection By default `vmalert` accepts http requests at `8880` port (this port can be changed via `-httpListenAddr` command-line flags), @@ -1362,14 +1381,11 @@ The shortlist of configuration flags is the following: -rule="http:///path/to/rules". HTTP URL to a page with alerting rules. -rule="dir/*.yaml" -rule="/*.yaml" -rule="gcs://vmalert-rules/tenant_%{TENANT_ID}/prod". -rule="dir/**/*.yaml". Includes all the .yaml files in "dir" subfolders recursively. - Rule files may contain %{ENV_VAR} placeholders, which are substituted by the corresponding env vars. - + Rule files may contain %{ENV_VAR} placeholders, which are substituted by the corresponding env vars. Enterprise version of vmalert supports S3 and GCS paths to rules. For example: gs://bucket/path/to/rules, s3://bucket/path/to/rules S3 and GCS paths support only matching by prefix, e.g. s3://bucket/dir/rule_ matches all files with prefix rule_ in folder dir. - See https://docs.victoriametrics.com/vmalert/#reading-rules-from-object-storage - Supports an array of values separated by comma or specified via multiple flags. Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces. -rule.evalDelay time @@ -1386,8 +1402,7 @@ The shortlist of configuration flags is the following: -rule.templates="/path/to/file". Path to a single file with go templates -rule.templates="dir/*.tpl" -rule.templates="/*.tpl". Relative path to all .tpl files in "dir" folder, absolute path to all .tpl files in root. - -rule.templates="dir/**/*.tpl". Includes all the .tpl files in "dir" subfolders recursively. - + -rule.templates="dir/**/*.tpl". Includes all the .tpl files in "dir" subfolders recursively. Supports an array of values separated by comma or specified via multiple flags. Value can contain comma inside single-quoted or double-quoted string, {}, [] and () braces. -rule.updateEntriesLimit int diff --git a/lib/ingestserver/statsd/server.go b/lib/ingestserver/statsd/server.go new file mode 100644 index 000000000..533d2ed1b --- /dev/null +++ b/lib/ingestserver/statsd/server.go @@ -0,0 +1,173 @@ +package statsd + +import ( + "errors" + "io" + "net" + "strings" + "sync" + "time" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/ingestserver" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/netutil" + "github.com/VictoriaMetrics/metrics" +) + +var ( + writeRequestsTCP = metrics.NewCounter(`vm_ingestserver_requests_total{type="statsd", name="write", net="tcp"}`) + writeErrorsTCP = metrics.NewCounter(`vm_ingestserver_request_errors_total{type="statsd", name="write", net="tcp"}`) + + writeRequestsUDP = metrics.NewCounter(`vm_ingestserver_requests_total{type="statsd", name="write", net="udp"}`) + writeErrorsUDP = metrics.NewCounter(`vm_ingestserver_request_errors_total{type="statsd", name="write", net="udp"}`) +) + +// Server accepts Statsd plaintext lines over TCP and UDP. +type Server struct { + addr string + lnTCP net.Listener + lnUDP net.PacketConn + wg sync.WaitGroup + cm ingestserver.ConnsMap +} + +// MustStart starts statsd server on the given addr. +// +// The incoming connections are processed with insertHandler. +// +// If useProxyProtocol is set to true, then the incoming connections are accepted via proxy protocol. +// See https://www.haproxy.org/download/1.8/doc/proxy-protocol.txt +// +// MustStop must be called on the returned server when it is no longer needed. +func MustStart(addr string, useProxyProtocol bool, insertHandler func(r io.Reader) error) *Server { + logger.Infof("starting TCP Statsd server at %q", addr) + lnTCP, err := netutil.NewTCPListener("statsd", addr, useProxyProtocol, nil) + if err != nil { + logger.Fatalf("cannot start TCP Statsd server at %q: %s", addr, err) + } + + logger.Infof("starting UDP Statsd server at %q", addr) + lnUDP, err := net.ListenPacket(netutil.GetUDPNetwork(), addr) + if err != nil { + logger.Fatalf("cannot start UDP Statsd server at %q: %s", addr, err) + } + + s := &Server{ + addr: addr, + lnTCP: lnTCP, + lnUDP: lnUDP, + } + s.cm.Init("statsd") + s.wg.Add(1) + go func() { + defer s.wg.Done() + s.serveTCP(insertHandler) + logger.Infof("stopped TCP Statsd server at %q", addr) + }() + s.wg.Add(1) + go func() { + defer s.wg.Done() + s.serveUDP(insertHandler) + logger.Infof("stopped UDP Statsd server at %q", addr) + }() + return s +} + +// MustStop stops the server. +func (s *Server) MustStop() { + logger.Infof("stopping TCP Statsd server at %q...", s.addr) + if err := s.lnTCP.Close(); err != nil { + logger.Errorf("cannot close TCP Statsd server: %s", err) + } + logger.Infof("stopping UDP Statsd server at %q...", s.addr) + if err := s.lnUDP.Close(); err != nil { + logger.Errorf("cannot close UDP Statsd server: %s", err) + } + s.cm.CloseAll(0) + s.wg.Wait() + logger.Infof("TCP and UDP Statsd servers at %q have been stopped", s.addr) +} + +func (s *Server) serveTCP(insertHandler func(r io.Reader) error) { + var wg sync.WaitGroup + for { + c, err := s.lnTCP.Accept() + if err != nil { + var ne net.Error + if errors.As(err, &ne) { + if ne.Temporary() { + logger.Errorf("statsd: temporary error when listening for TCP addr %q: %s", s.lnTCP.Addr(), err) + time.Sleep(time.Second) + continue + } + if strings.Contains(err.Error(), "use of closed network connection") { + break + } + logger.Fatalf("unrecoverable error when accepting TCP Statsd connections: %s", err) + } + logger.Fatalf("unexpected error when accepting TCP Statsd connections: %s", err) + } + if !s.cm.Add(c) { + _ = c.Close() + break + } + wg.Add(1) + go func() { + defer func() { + s.cm.Delete(c) + _ = c.Close() + wg.Done() + }() + writeRequestsTCP.Inc() + if err := insertHandler(c); err != nil { + writeErrorsTCP.Inc() + logger.Errorf("error in TCP Statsd conn %q<->%q: %s", c.LocalAddr(), c.RemoteAddr(), err) + } + }() + } + wg.Wait() +} + +func (s *Server) serveUDP(insertHandler func(r io.Reader) error) { + gomaxprocs := cgroup.AvailableCPUs() + var wg sync.WaitGroup + for i := 0; i < gomaxprocs; i++ { + wg.Add(1) + go func() { + defer wg.Done() + var bb bytesutil.ByteBuffer + bb.B = bytesutil.ResizeNoCopyNoOverallocate(bb.B, 64*1024) + for { + bb.Reset() + bb.B = bb.B[:cap(bb.B)] + n, addr, err := s.lnUDP.ReadFrom(bb.B) + if err != nil { + writeErrorsUDP.Inc() + var ne net.Error + if errors.As(err, &ne) { + if ne.Temporary() { + logger.Errorf("statsd: temporary error when listening for UDP addr %q: %s", s.lnUDP.LocalAddr(), err) + time.Sleep(time.Second) + continue + } + if strings.Contains(err.Error(), "use of closed network connection") { + break + } + } + logger.Errorf("cannot read Statsd UDP data: %s", err) + continue + } + bb.B = bb.B[:n] + writeRequestsUDP.Inc() + if err := insertHandler(bb.NewReader()); err != nil { + writeErrorsUDP.Inc() + logger.Errorf("error in UDP Statsd conn %q<->%q: %s", s.lnUDP.LocalAddr(), addr, err) + continue + } + } + }() + } + wg.Wait() +} diff --git a/lib/mergeset/table.go b/lib/mergeset/table.go index 52cc74ea8..628634c3d 100644 --- a/lib/mergeset/table.go +++ b/lib/mergeset/table.go @@ -167,7 +167,7 @@ var rawItemsShardsPerTable = func() int { return cpus * multiplier }() -const maxBlocksPerShard = 256 +var maxBlocksPerShard = 256 func (riss *rawItemsShards) init() { riss.shards = make([]rawItemsShard, rawItemsShardsPerTable) diff --git a/lib/mergeset/table_test.go b/lib/mergeset/table_test.go index e044ab34a..810dd5a04 100644 --- a/lib/mergeset/table_test.go +++ b/lib/mergeset/table_test.go @@ -176,6 +176,66 @@ func TestTableCreateSnapshotAt(t *testing.T) { _ = os.RemoveAll(path) } +func TestTableAddItemsConcurrentStress(t *testing.T) { + const path = "TestTableAddItemsConcurrentStress" + if err := os.RemoveAll(path); err != nil { + t.Fatalf("cannot remove %q: %s", path, err) + } + defer func() { + _ = os.RemoveAll(path) + }() + + rawItemsShardsPerTableOrig := rawItemsShardsPerTable + maxBlocksPerShardOrig := maxBlocksPerShard + rawItemsShardsPerTable = 10 + maxBlocksPerShard = 3 + defer func() { + rawItemsShardsPerTable = rawItemsShardsPerTableOrig + maxBlocksPerShard = maxBlocksPerShardOrig + }() + + var flushes atomic.Uint64 + flushCallback := func() { + flushes.Add(1) + } + prepareBlock := func(data []byte, items []Item) ([]byte, []Item) { + return data, items + } + + blocksNeeded := rawItemsShardsPerTable * maxBlocksPerShard * 10 + testAddItems := func(tb *Table) { + itemsBatch := make([][]byte, 0) + + for j := 0; j < blocksNeeded; j++ { + item := bytes.Repeat([]byte{byte(j)}, maxInmemoryBlockSize-10) + itemsBatch = append(itemsBatch, item) + } + tb.AddItems(itemsBatch) + } + + var isReadOnly atomic.Bool + tb := MustOpenTable(path, flushCallback, prepareBlock, &isReadOnly) + + testAddItems(tb) + + // Verify items count after pending items flush. + tb.DebugFlush() + if flushes.Load() == 0 { + t.Fatalf("unexpected zero flushes") + } + + var m TableMetrics + tb.UpdateMetrics(&m) + if n := m.TotalItemsCount(); n != uint64(blocksNeeded) { + t.Fatalf("unexpected itemsCount; got %d; want %v", n, blocksNeeded) + } + + tb.MustClose() + + // Re-open the table and make sure itemsCount remains the same. + testReopenTable(t, path, blocksNeeded) +} + func TestTableAddItemsConcurrent(t *testing.T) { const path = "TestTableAddItemsConcurrent" if err := os.RemoveAll(path); err != nil { diff --git a/lib/persistentqueue/fastqueue.go b/lib/persistentqueue/fastqueue.go index a2d597411..3c194f7ba 100644 --- a/lib/persistentqueue/fastqueue.go +++ b/lib/persistentqueue/fastqueue.go @@ -63,7 +63,7 @@ func MustOpenFastQueue(path, name string, maxInmemoryBlocks int, maxPendingBytes return float64(n) }) pendingBytes := fq.GetPendingBytes() - logger.Infof("opened fast persistent queue at %q with maxInmemoryBlocks=%d, it contains %d pending bytes", path, maxInmemoryBlocks, pendingBytes) + logger.Infof("opened fast persistent queue at %q with maxInmemoryBlocks=%d isPQDisabled=%t, it contains %d pending bytes", path, maxInmemoryBlocks, isPQDisabled, pendingBytes) return fq } diff --git a/lib/prompb/prompb.go b/lib/prompb/prompb.go index 34a1c5716..adc8935e2 100644 --- a/lib/prompb/prompb.go +++ b/lib/prompb/prompb.go @@ -9,10 +9,11 @@ import ( // WriteRequest represents Prometheus remote write API request. type WriteRequest struct { // Timeseries is a list of time series in the given WriteRequest - Timeseries []TimeSeries - - labelsPool []Label - samplesPool []Sample + Timeseries []TimeSeries + labelsPool []Label + exemplarLabelsPool []Label + samplesPool []Sample + exemplarsPool []Exemplar } // Reset resets wr for subsequent re-use. @@ -29,11 +30,33 @@ func (wr *WriteRequest) Reset() { } wr.labelsPool = labelsPool[:0] + exemplarLabelsPool := wr.exemplarLabelsPool + for i := range exemplarLabelsPool { + exemplarLabelsPool[i] = Label{} + } + wr.labelsPool = labelsPool[:0] samplesPool := wr.samplesPool for i := range samplesPool { samplesPool[i] = Sample{} } wr.samplesPool = samplesPool[:0] + exemplarsPool := wr.exemplarsPool + for i := range exemplarsPool { + exemplarsPool[i] = Exemplar{} + } + wr.exemplarsPool = exemplarsPool[:0] +} + +// Exemplar is an exemplar +type Exemplar struct { + // Labels a list of labels that uniquely identifies exemplar + // Optional, can be empty. + Labels []Label + // Value: the value of the exemplar + Value float64 + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + Timestamp int64 } // TimeSeries is a timeseries. @@ -42,7 +65,8 @@ type TimeSeries struct { Labels []Label // Samples is a list of samples for the given TimeSeries - Samples []Sample + Samples []Sample + Exemplars []Exemplar } // Sample is a timeseries sample. @@ -74,7 +98,10 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) { // } tss := wr.Timeseries labelsPool := wr.labelsPool + exemplarLabelsPool := wr.exemplarLabelsPool samplesPool := wr.samplesPool + exemplarsPool := wr.exemplarsPool + var fc easyproto.FieldContext for len(src) > 0 { src, err = fc.NextField(src) @@ -93,7 +120,7 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) { tss = append(tss, TimeSeries{}) } ts := &tss[len(tss)-1] - labelsPool, samplesPool, err = ts.unmarshalProtobuf(data, labelsPool, samplesPool) + labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, err = ts.unmarshalProtobuf(data, labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool) if err != nil { return fmt.Errorf("cannot unmarshal timeseries: %w", err) } @@ -102,28 +129,31 @@ func (wr *WriteRequest) UnmarshalProtobuf(src []byte) (err error) { wr.Timeseries = tss wr.labelsPool = labelsPool wr.samplesPool = samplesPool + wr.exemplarsPool = exemplarsPool return nil } -func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesPool []Sample) ([]Label, []Sample, error) { +func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, exemplarLabelsPool []Label, samplesPool []Sample, exemplarsPool []Exemplar) ([]Label, []Label, []Sample, []Exemplar, error) { // message TimeSeries { // repeated Label labels = 1; // repeated Sample samples = 2; + // repeated Exemplar exemplars = 3 // } labelsPoolLen := len(labelsPool) samplesPoolLen := len(samplesPool) + exemplarsPoolLen := len(exemplarsPool) var fc easyproto.FieldContext for len(src) > 0 { var err error src, err = fc.NextField(src) if err != nil { - return labelsPool, samplesPool, fmt.Errorf("cannot read the next field: %w", err) + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the next field: %w", err) } switch fc.FieldNum { case 1: data, ok := fc.MessageData() if !ok { - return labelsPool, samplesPool, fmt.Errorf("cannot read label data") + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read label data") } if len(labelsPool) < cap(labelsPool) { labelsPool = labelsPool[:len(labelsPool)+1] @@ -132,12 +162,12 @@ func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesP } label := &labelsPool[len(labelsPool)-1] if err := label.unmarshalProtobuf(data); err != nil { - return labelsPool, samplesPool, fmt.Errorf("cannot unmarshal label: %w", err) + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal label: %w", err) } case 2: data, ok := fc.MessageData() if !ok { - return labelsPool, samplesPool, fmt.Errorf("cannot read the sample data") + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the sample data") } if len(samplesPool) < cap(samplesPool) { samplesPool = samplesPool[:len(samplesPool)+1] @@ -146,15 +176,78 @@ func (ts *TimeSeries) unmarshalProtobuf(src []byte, labelsPool []Label, samplesP } sample := &samplesPool[len(samplesPool)-1] if err := sample.unmarshalProtobuf(data); err != nil { - return labelsPool, samplesPool, fmt.Errorf("cannot unmarshal sample: %w", err) + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal sample: %w", err) + } + case 3: + data, ok := fc.MessageData() + if !ok { + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot read the exemplar data") + } + if len(exemplarsPool) < cap(exemplarsPool) { + exemplarsPool = exemplarsPool[:len(exemplarsPool)+1] + } else { + exemplarsPool = append(exemplarsPool, Exemplar{}) + } + exemplar := &exemplarsPool[len(exemplarsPool)-1] + if exemplarLabelsPool, err = exemplar.unmarshalProtobuf(data, exemplarLabelsPool); err != nil { + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, fmt.Errorf("cannot unmarshal exemplar: %w", err) } } } ts.Labels = labelsPool[labelsPoolLen:] ts.Samples = samplesPool[samplesPoolLen:] - return labelsPool, samplesPool, nil + ts.Exemplars = exemplarsPool[exemplarsPoolLen:] + return labelsPool, exemplarLabelsPool, samplesPool, exemplarsPool, nil } +func (exemplar *Exemplar) unmarshalProtobuf(src []byte, labelsPool []Label) ([]Label, error) { + // message Exemplar { + // repeated Label Labels = 1; + // float64 Value = 2; + // int64 Timestamp = 3; + // } + var fc easyproto.FieldContext + + labelsPoolLen := len(labelsPool) + + for len(src) > 0 { + var err error + src, err = fc.NextField(src) + if err != nil { + return labelsPool, fmt.Errorf("cannot read the next field: %w", err) + } + switch fc.FieldNum { + case 1: + data, ok := fc.MessageData() + if !ok { + return labelsPool, fmt.Errorf("cannot read label data") + } + if len(labelsPool) < cap(labelsPool) { + labelsPool = labelsPool[:len(labelsPool)+1] + } else { + labelsPool = append(labelsPool, Label{}) + } + label := &labelsPool[len(labelsPool)-1] + if err := label.unmarshalProtobuf(data); err != nil { + return labelsPool, fmt.Errorf("cannot unmarshal label: %w", err) + } + case 2: + value, ok := fc.Double() + if !ok { + return labelsPool, fmt.Errorf("cannot read exemplar value") + } + exemplar.Value = value + case 3: + timestamp, ok := fc.Int64() + if !ok { + return labelsPool, fmt.Errorf("cannot read exemplar timestamp") + } + exemplar.Timestamp = timestamp + } + } + exemplar.Labels = labelsPool[labelsPoolLen:] + return labelsPool, nil +} func (lbl *Label) unmarshalProtobuf(src []byte) (err error) { // message Label { // string name = 1; diff --git a/lib/prompb/prompb_test.go b/lib/prompb/prompb_test.go index 727101206..b1d6e02bc 100644 --- a/lib/prompb/prompb_test.go +++ b/lib/prompb/prompb_test.go @@ -36,9 +36,25 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) { Timestamp: sample.Timestamp, }) } + var exemplars []prompbmarshal.Exemplar + for _, exemplar := range ts.Exemplars { + exemplarLabels := make([]prompbmarshal.Label, len(exemplar.Labels)) + for i, label := range exemplar.Labels { + exemplarLabels[i] = prompbmarshal.Label{ + Name: label.Name, + Value: label.Value, + } + } + exemplars = append(exemplars, prompbmarshal.Exemplar{ + Labels: exemplarLabels, + Value: exemplar.Value, + Timestamp: exemplar.Timestamp, + }) + } wrm.Timeseries = append(wrm.Timeseries, prompbmarshal.TimeSeries{ - Labels: labels, - Samples: samples, + Labels: labels, + Samples: samples, + Exemplars: exemplars, }) } dataResult := wrm.MarshalProtobuf(nil) @@ -121,6 +137,19 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) { Timestamp: 18939432423, }, }, + Exemplars: []prompbmarshal.Exemplar{ + { + Labels: []prompbmarshal.Label{ + {Name: "trace-id", + Value: "123456", + }, + {Name: "log_id", + Value: "987664"}, + }, + Value: 12345.6, + Timestamp: 456, + }, + }, }, } data = wrm.MarshalProtobuf(data[:0]) @@ -153,6 +182,18 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) { Timestamp: 18939432423, }, }, + Exemplars: []prompbmarshal.Exemplar{ + { + Labels: []prompbmarshal.Label{ + { + Name: "trace-id", + Value: "123456", + }, + }, + Value: 12345.6, + Timestamp: 456, + }, + }, }, { Labels: []prompbmarshal.Label{ @@ -166,6 +207,22 @@ func TestWriteRequestUnmarshalProtobuf(t *testing.T) { Value: 9873, }, }, + Exemplars: []prompbmarshal.Exemplar{ + { + Labels: []prompbmarshal.Label{ + { + Name: "trace-id", + Value: "123456", + }, + { + Name: "log_id", + Value: "987654", + }, + }, + Value: 12345.6, + Timestamp: 456, + }, + }, }, } data = wrm.MarshalProtobuf(data[:0]) diff --git a/lib/prompbmarshal/prompbmarshal_test.go b/lib/prompbmarshal/prompbmarshal_test.go index 99fb28c10..42716fdb2 100644 --- a/lib/prompbmarshal/prompbmarshal_test.go +++ b/lib/prompbmarshal/prompbmarshal_test.go @@ -36,6 +36,22 @@ func TestWriteRequestMarshalProtobuf(t *testing.T) { Timestamp: 18939432423, }, }, + Exemplars: []prompbmarshal.Exemplar{ + { + Labels: []prompbmarshal.Label{ + { + Name: "trace-id", + Value: "123456", + }, + { + Name: "log_id", + Value: "987654", + }, + }, + Value: 12345.6, + Timestamp: 456, + }, + }, }, }, } @@ -64,9 +80,25 @@ func TestWriteRequestMarshalProtobuf(t *testing.T) { Timestamp: sample.Timestamp, }) } + var exemplars []prompbmarshal.Exemplar + for _, exemplar := range ts.Exemplars { + exemplarLabels := make([]prompbmarshal.Label, len(exemplar.Labels)) + for i, label := range exemplar.Labels { + exemplarLabels[i] = prompbmarshal.Label{ + Name: label.Name, + Value: label.Value, + } + } + exemplars = append(exemplars, prompbmarshal.Exemplar{ + Labels: exemplarLabels, + Value: exemplar.Value, + Timestamp: exemplar.Timestamp, + }) + } wrm.Timeseries = append(wrm.Timeseries, prompbmarshal.TimeSeries{ - Labels: labels, - Samples: samples, + Labels: labels, + Samples: samples, + Exemplars: exemplars, }) } dataResult := wrm.MarshalProtobuf(nil) diff --git a/lib/prompbmarshal/types.pb.go b/lib/prompbmarshal/types.pb.go index ca2b62f1b..d5295ddd0 100644 --- a/lib/prompbmarshal/types.pb.go +++ b/lib/prompbmarshal/types.pb.go @@ -13,10 +13,70 @@ type Sample struct { Timestamp int64 } +type Exemplar struct { + // Optional, can be empty. + Labels []Label + Value float64 + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + Timestamp int64 +} + +func (m *Exemplar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Timestamp != 0 { + i = encodeVarint(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x18 + } + if m.Value != 0 { + i -= 8 + binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i-- + dAtA[i] = 0x11 + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Labels[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} +func (m *Exemplar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sov(uint64(l)) + } + } + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sov(uint64(m.Timestamp)) + } + return n +} + // TimeSeries represents samples and labels for a single time series. type TimeSeries struct { - Labels []Label - Samples []Sample + Labels []Label + Samples []Sample + Exemplars []Exemplar } type Label struct { @@ -42,6 +102,16 @@ func (m *Sample) MarshalToSizedBuffer(dst []byte) (int, error) { func (m *TimeSeries) MarshalToSizedBuffer(dst []byte) (int, error) { i := len(dst) + for j := len(m.Exemplars) - 1; j >= 0; j-- { + size, err := m.Exemplars[j].MarshalToSizedBuffer(dst[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dst, i, uint64(size)) + i-- + dst[i] = 0x1a + } for j := len(m.Samples) - 1; j >= 0; j-- { size, err := m.Samples[j].MarshalToSizedBuffer(dst[:i]) if err != nil { @@ -109,6 +179,10 @@ func (m *TimeSeries) Size() (n int) { l := e.Size() n += 1 + l + sov(uint64(l)) } + for _, e := range m.Exemplars { + l := e.Size() + n += 1 + l + sov(uint64(l)) + } return n } diff --git a/lib/promscrape/client.go b/lib/promscrape/client.go index e5f7b4a5e..31457911a 100644 --- a/lib/promscrape/client.go +++ b/lib/promscrape/client.go @@ -30,6 +30,7 @@ var ( streamParse = flag.Bool("promscrape.streamParse", false, "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") + scrapeExemplars = flag.Bool("promscrape.scrapeExemplars", false, "Whether to enable scraping of exemplars from scrape targets.") ) type client struct { @@ -107,6 +108,12 @@ func (c *client) ReadData(dst *bytesutil.ByteBuffer) error { // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/608 for details. // Do not bloat the `Accept` header with OpenMetrics shit, since it looks like dead standard now. req.Header.Set("Accept", "text/plain;version=0.0.4;q=1,*/*;q=0.1") + // We set to support exemplars to be compatible with Prometheus Exposition format which uses + // Open Metrics Specification + // See https://github.com/prometheus/docs/blob/main/content/docs/instrumenting/exposition_formats.md#openmetrics-text-format + if *scrapeExemplars { + req.Header.Set("Accept", "application/openmetrics-text") + } // Set X-Prometheus-Scrape-Timeout-Seconds like Prometheus does, since it is used by some exporters such as PushProx. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1179#issuecomment-813117162 req.Header.Set("X-Prometheus-Scrape-Timeout-Seconds", c.scrapeTimeoutSecondsStr) diff --git a/lib/promscrape/config.go b/lib/promscrape/config.go index 5516b60f1..b0ea9603f 100644 --- a/lib/promscrape/config.go +++ b/lib/promscrape/config.go @@ -36,6 +36,7 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/kuma" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/nomad" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/openstack" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/vultr" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/yandexcloud" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" @@ -308,6 +309,7 @@ type ScrapeConfig struct { NomadSDConfigs []nomad.SDConfig `yaml:"nomad_sd_configs,omitempty"` OpenStackSDConfigs []openstack.SDConfig `yaml:"openstack_sd_configs,omitempty"` StaticConfigs []StaticConfig `yaml:"static_configs,omitempty"` + VultrConfigs []vultr.SDConfig `yaml:"vultr_configs,omitempty"` YandexCloudSDConfigs []yandexcloud.SDConfig `yaml:"yandexcloud_sd_configs,omitempty"` // These options are supported only by lib/promscrape. @@ -388,6 +390,9 @@ func (sc *ScrapeConfig) mustStop() { for i := range sc.OpenStackSDConfigs { sc.OpenStackSDConfigs[i].MustStop() } + for i := range sc.VultrConfigs { + sc.VultrConfigs[i].MustStop() + } } // FileSDConfig represents file-based service discovery config. @@ -745,6 +750,16 @@ func (cfg *Config) getOpenStackSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork { return cfg.getScrapeWorkGeneric(visitConfigs, "openstack_sd_config", prev) } +// getVultrSDScrapeWork returns `vultr_sd_configs` ScrapeWork from cfg. +func (cfg *Config) getVultrSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork { + visitConfigs := func(sc *ScrapeConfig, visitor func(sdc targetLabelsGetter)) { + for i := range sc.VultrConfigs { + visitor(&sc.VultrConfigs[i]) + } + } + return cfg.getScrapeWorkGeneric(visitConfigs, "vultr_sd_config", prev) +} + // getYandexCloudSDScrapeWork returns `yandexcloud_sd_configs` ScrapeWork from cfg. func (cfg *Config) getYandexCloudSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork { visitConfigs := func(sc *ScrapeConfig, visitor func(sdc targetLabelsGetter)) { diff --git a/lib/promscrape/discovery/vultr/api.go b/lib/promscrape/discovery/vultr/api.go new file mode 100644 index 000000000..86e82df3a --- /dev/null +++ b/lib/promscrape/discovery/vultr/api.go @@ -0,0 +1,76 @@ +package vultr + +import ( + "fmt" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" +) + +// apiConfig contains config for API server. +type apiConfig struct { + c *discoveryutils.Client + port int + + listParams +} + +// listParams is the query params of vultr ListInstance API. +type listParams struct { + // paging params are not exposed to user, they will be filled + // dynamically during request. See `getInstances`. + // perPage int + // cursor string + + // API query params for filtering. + label string + mainIP string + region string + firewallGroupID string + hostname string +} + +// getAPIConfig get or create API config from configMap. +func getAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) { + v, err := configMap.Get(sdc, func() (interface{}, error) { return newAPIConfig(sdc, baseDir) }) + if err != nil { + return nil, err + } + return v.(*apiConfig), nil +} + +// newAPIConfig create API Config. +func newAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) { + port := sdc.Port + if port == 0 { + port = 80 + } + + // See: https://www.vultr.com/api/ + apiServer := "https://api.vultr.com" + + ac, err := sdc.HTTPClientConfig.NewConfig(baseDir) + if err != nil { + return nil, fmt.Errorf("cannot parse auth config: %w", err) + } + proxyAC, err := sdc.ProxyClientConfig.NewConfig(baseDir) + if err != nil { + return nil, fmt.Errorf("cannot parse proxy auth config: %w", err) + } + + c, err := discoveryutils.NewClient(apiServer, ac, sdc.ProxyURL, proxyAC, &sdc.HTTPClientConfig) + if err != nil { + return nil, fmt.Errorf("cannot create client for %q: %w", apiServer, err) + } + cfg := &apiConfig{ + c: c, + port: port, + listParams: listParams{ + label: sdc.Label, + mainIP: sdc.MainIP, + region: sdc.Region, + firewallGroupID: sdc.FirewallGroupID, + hostname: sdc.Hostname, + }, + } + return cfg, nil +} diff --git a/lib/promscrape/discovery/vultr/api_test.go b/lib/promscrape/discovery/vultr/api_test.go new file mode 100644 index 000000000..06e491446 --- /dev/null +++ b/lib/promscrape/discovery/vultr/api_test.go @@ -0,0 +1,16 @@ +package vultr + +import ( + "testing" +) + +func TestNewAPIConfig(t *testing.T) { + + sdc := &SDConfig{} + baseDir := "." + _, err := newAPIConfig(sdc, baseDir) + if err != nil { + t.Errorf("newAPIConfig failed with, err: %v", err) + return + } +} diff --git a/lib/promscrape/discovery/vultr/instance.go b/lib/promscrape/discovery/vultr/instance.go new file mode 100644 index 000000000..e510b7353 --- /dev/null +++ b/lib/promscrape/discovery/vultr/instance.go @@ -0,0 +1,109 @@ +package vultr + +import ( + "encoding/json" + "fmt" + "net/url" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" +) + +// ListInstanceResponse is the response structure of Vultr ListInstance API. +type ListInstanceResponse struct { + Instances []Instance `json:"instances"` + Meta *Meta `json:"Meta"` +} + +// Instance represents Vultr Instance (VPS). +// See: https://github.com/vultr/govultr/blob/5125e02e715ae6eb3ce854f0e7116c7ce545a710/instance.go#L81 +type Instance struct { + ID string `json:"id"` + Os string `json:"os"` + RAM int `json:"ram"` + Disk int `json:"disk"` + MainIP string `json:"main_ip"` + VCPUCount int `json:"vcpu_count"` + Region string `json:"region"` + ServerStatus string `json:"server_status"` + AllowedBandwidth int `json:"allowed_bandwidth"` + V6MainIP string `json:"v6_main_ip"` + Hostname string `json:"hostname"` + Label string `json:"label"` + InternalIP string `json:"internal_ip"` + OsID int `json:"os_id"` + Features []string `json:"features"` + Plan string `json:"plan"` + Tags []string `json:"tags"` + + // The following fields are defined in the response but are not used during service discovery. + //DefaultPassword string `json:"default_password,omitempty"` + //DateCreated string `json:"date_created"` + //Status string `json:"status"` + //PowerStatus string `json:"power_status"` + //NetmaskV4 string `json:"netmask_v4"` + //GatewayV4 string `json:"gateway_v4"` + //V6Network string `json:"v6_network"` + //V6NetworkSize int `json:"v6_network_size"` + //// Deprecated: Tag should no longer be used. Instead, use Tags. + //Tag string `json:"tag"` + //KVM string `json:"kvm"` + //AppID int `json:"app_id"` + //ImageID string `json:"image_id"` + //FirewallGroupID string `json:"firewall_group_id"` + //UserScheme string `json:"user_scheme"` +} + +// Meta represents the available pagination information +type Meta struct { + Total int `json:"total"` + Links *Links +} + +// Links represent the next/previous cursor in your pagination calls +type Links struct { + Next string `json:"next"` + Prev string `json:"prev"` +} + +// getInstances retrieve instance from Vultr HTTP API. +func getInstances(cfg *apiConfig) ([]Instance, error) { + var instances []Instance + + // prepare GET params + params := url.Values{} + params.Set("per_page", "100") + params.Set("label", cfg.label) + params.Set("main_ip", cfg.mainIP) + params.Set("region", cfg.region) + params.Set("firewall_group_id", cfg.firewallGroupID) + params.Set("hostname", cfg.hostname) + + // send request to vultr API + for { + // See: https://www.vultr.com/api/#tag/instances/operation/list-instances + path := fmt.Sprintf("/v2/instances?%s", params.Encode()) + resp, err := cfg.c.GetAPIResponse(path) + if err != nil { + logger.Errorf("get response from vultr failed, path:%s, err: %v", path, err) + return nil, err + } + + var listInstanceResp ListInstanceResponse + if err = json.Unmarshal(resp, &listInstanceResp); err != nil { + logger.Errorf("unmarshal response from vultr failed, err: %v", err) + return nil, err + } + + instances = append(instances, listInstanceResp.Instances...) + + if listInstanceResp.Meta != nil && listInstanceResp.Meta.Links != nil && listInstanceResp.Meta.Links.Next != "" { + // if `next page` is available, set the cursor param and request again. + params.Set("cursor", listInstanceResp.Meta.Links.Next) + } else { + // otherwise exit the loop + break + } + } + + return instances, nil +} diff --git a/lib/promscrape/discovery/vultr/instance_test.go b/lib/promscrape/discovery/vultr/instance_test.go new file mode 100644 index 000000000..b888c1a13 --- /dev/null +++ b/lib/promscrape/discovery/vultr/instance_test.go @@ -0,0 +1,301 @@ +package vultr + +import ( + "errors" + "reflect" + "testing" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" +) + +// TestGetInstances runs general test cases for GetInstances +func TestGetInstances(t *testing.T) { + testCases := []struct { + name string + apiResponse string + apiError bool + expectError bool + expectResponse []Instance + }{ + { + name: "success response", + apiResponse: mockListInstanceSuccessResp, + apiError: false, + expectError: false, + expectResponse: expectSuccessInstances, + }, + { + name: "failed response", + apiResponse: mockListInstanceFailedResp, + apiError: true, + expectError: true, + expectResponse: nil, + }, + } + + for _, tt := range testCases { + t.Run(tt.name, func(t *testing.T) { + // Prepare a mock Vultr server. + mockServer := newMockVultrServer(func() ([]byte, error) { + var e error + if tt.apiError { + e = errors.New("mock error") + } + return []byte(tt.apiResponse), e + }) + + // Prepare a discovery HTTP client who calls mock server. + client, _ := discoveryutils.NewClient(mockServer.URL, nil, nil, nil, &promauth.HTTPClientConfig{}) + cfg := &apiConfig{ + c: client, + } + + // execute `getInstances` + instances, err := getInstances(cfg) + + // evaluate test result + if tt.expectError != (err != nil) { + t.Errorf("getInstances expect (error != nil): %t, got error: %v", tt.expectError, err) + } + + if !reflect.DeepEqual(tt.expectResponse, instances) { + t.Errorf("getInstances expect result: %v, got: %v", tt.expectResponse, instances) + } + }) + } +} + +// TestGetInstancesPaging run test cases for response with multiple pages. +func TestGetInstancesPaging(t *testing.T) { + // Prepare a mock Vultr server. + // requestCount control the mock response for different page request. + requestCount := 0 + + mockServer := newMockVultrServer(func() ([]byte, error) { + // for the 1st request, response with `next` cursor + if requestCount == 0 { + requestCount++ + return []byte(mockListInstanceSuccessPage0Resp), nil + } + // for the 2nd+ request, response with `prev` cursor and empty `next`. + return []byte(mockListInstanceSuccessPage1Resp), nil + }) + + // Prepare a discovery HTTP client who calls mock server. + client, _ := discoveryutils.NewClient(mockServer.URL, nil, nil, nil, &promauth.HTTPClientConfig{}) + cfg := &apiConfig{ + c: client, + } + + // execute `getInstances` + instances, err := getInstances(cfg) + + // evaluate test result + if err != nil { + t.Errorf("getInstances expect error: %v, got error: %v", nil, err) + } + + if !reflect.DeepEqual(expectSuccessPagingInstances, instances) { + t.Errorf("getInstances expect result: %v, got: %v", expectSuccessPagingInstances, instances) + } +} + +// ------------ Test dataset ------------ +var ( + // mockListInstanceSuccessResp is crawled from a real-world response of ListInstance API + // with sensitive info removed/modified. + mockListInstanceSuccessResp = `{ + "instances": [{ + "id": "fake-id-07f7-4b68-88ac-fake-id", + "os": "Ubuntu 22.04 x64", + "ram": 1024, + "disk": 25, + "main_ip": "64.176.84.27", + "vcpu_count": 1, + "region": "sgp", + "plan": "vc2-1c-1gb", + "date_created": "2024-04-05T05:41:28+00:00", + "status": "active", + "allowed_bandwidth": 1, + "netmask_v4": "255.255.254.0", + "gateway_v4": "64.176.63.2", + "power_status": "running", + "server_status": "installingbooting", + "v6_network": "2002:18f0:4100:263a::", + "v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + "v6_network_size": 64, + "label": "vultr-sd", + "internal_ip": "", + "kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string", + "hostname": "vultr-sd", + "tag": "", + "tags": [], + "os_id": 1743, + "app_id": 0, + "image_id": "", + "firewall_group_id": "", + "features": ["ipv6"], + "user_scheme": "root" + }], + "meta": { + "total": 1, + "links": { + "next": "", + "prev": "" + } + } +}` + expectSuccessInstances = []Instance{ + { + ID: "fake-id-07f7-4b68-88ac-fake-id", + Os: "Ubuntu 22.04 x64", + RAM: 1024, + Disk: 25, + MainIP: "64.176.84.27", + VCPUCount: 1, + Region: "sgp", + Plan: "vc2-1c-1gb", + AllowedBandwidth: 1, + ServerStatus: "installingbooting", + V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + Label: "vultr-sd", + InternalIP: "", + Hostname: "vultr-sd", + Tags: []string{}, + OsID: 1743, + Features: []string{"ipv6"}, + }, + } +) + +var ( + mockListInstanceFailedResp = `{"error":"Invalid API token.","status":401}` +) + +var ( + // mockListInstanceSuccessPage0Resp contains `next` cursor + mockListInstanceSuccessPage0Resp = `{ + "instances": [{ + "id": "fake-id-07f7-4b68-88ac-fake-id", + "os": "Ubuntu 22.04 x64", + "ram": 1024, + "disk": 25, + "main_ip": "64.176.84.27", + "vcpu_count": 1, + "region": "sgp", + "plan": "vc2-1c-1gb", + "date_created": "2024-04-05T05:41:28+00:00", + "status": "active", + "allowed_bandwidth": 1, + "netmask_v4": "255.255.254.0", + "gateway_v4": "64.176.63.2", + "power_status": "running", + "server_status": "installingbooting", + "v6_network": "2002:18f0:4100:263a::", + "v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + "v6_network_size": 64, + "label": "vultr-sd", + "internal_ip": "", + "kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string", + "hostname": "vultr-sd", + "tag": "", + "tags": [], + "os_id": 1743, + "app_id": 0, + "image_id": "", + "firewall_group_id": "", + "features": ["ipv6"], + "user_scheme": "root" + }], + "meta": { + "total": 2, + "links": { + "next": "fake-cursor-string", + "prev": "" + } + } +}` + // mockListInstanceSuccessPage1Resp contains `prev` cursor + mockListInstanceSuccessPage1Resp = `{ + "instances": [{ + "id": "fake-id-07f7-4b68-88ac-fake-id", + "os": "Ubuntu 22.04 x64", + "ram": 1024, + "disk": 25, + "main_ip": "64.176.84.27", + "vcpu_count": 1, + "region": "sgp", + "plan": "vc2-1c-1gb", + "date_created": "2024-04-05T05:41:28+00:00", + "status": "active", + "allowed_bandwidth": 1, + "netmask_v4": "255.255.254.0", + "gateway_v4": "64.176.63.2", + "power_status": "running", + "server_status": "installingbooting", + "v6_network": "2002:18f0:4100:263a::", + "v6_main_ip": "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + "v6_network_size": 64, + "label": "vultr-sd", + "internal_ip": "", + "kvm": "https:\/\/my.vultr.com\/subs\/vps\/novnc\/api.php?data=secret_data_string", + "hostname": "vultr-sd", + "tag": "", + "tags": [], + "os_id": 1743, + "app_id": 0, + "image_id": "", + "firewall_group_id": "", + "features": ["ipv6"], + "user_scheme": "root" + }], + "meta": { + "total": 2, + "links": { + "next": "", + "prev": "fake-cursor-string" + } + } +}` + expectSuccessPagingInstances = []Instance{ + { + ID: "fake-id-07f7-4b68-88ac-fake-id", + Os: "Ubuntu 22.04 x64", + RAM: 1024, + Disk: 25, + MainIP: "64.176.84.27", + VCPUCount: 1, + Region: "sgp", + Plan: "vc2-1c-1gb", + AllowedBandwidth: 1, + ServerStatus: "installingbooting", + V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + Label: "vultr-sd", + InternalIP: "", + Hostname: "vultr-sd", + Tags: []string{}, + OsID: 1743, + Features: []string{"ipv6"}, + }, + { + ID: "fake-id-07f7-4b68-88ac-fake-id", + Os: "Ubuntu 22.04 x64", + RAM: 1024, + Disk: 25, + MainIP: "64.176.84.27", + VCPUCount: 1, + Region: "sgp", + Plan: "vc2-1c-1gb", + AllowedBandwidth: 1, + ServerStatus: "installingbooting", + V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + Label: "vultr-sd", + InternalIP: "", + Hostname: "vultr-sd", + Tags: []string{}, + OsID: 1743, + Features: []string{"ipv6"}, + }, + } +) diff --git a/lib/promscrape/discovery/vultr/mock_server_test.go b/lib/promscrape/discovery/vultr/mock_server_test.go new file mode 100644 index 000000000..c0dd79e46 --- /dev/null +++ b/lib/promscrape/discovery/vultr/mock_server_test.go @@ -0,0 +1,40 @@ +package vultr + +import ( + "fmt" + "net/http" + "net/http/httptest" +) + +func newMockVultrServer(jsonResponse func() ([]byte, error)) *vultrServer { + rw := &vultrServer{} + rw.Server = httptest.NewServer(http.HandlerFunc(rw.handler)) + rw.jsonResponse = jsonResponse + return rw +} + +type vultrServer struct { + *httptest.Server + jsonResponse func() ([]byte, error) +} + +func (rw *vultrServer) err(w http.ResponseWriter, err error) { + w.WriteHeader(http.StatusBadRequest) + w.Write([]byte(err.Error())) +} + +func (rw *vultrServer) handler(w http.ResponseWriter, r *http.Request) { + if r.Method != http.MethodGet { + rw.err(w, fmt.Errorf("bad method %q", r.Method)) + return + } + + resp, err := rw.jsonResponse() + if err != nil { + rw.err(w, err) + return + } + + w.Write(resp) + w.WriteHeader(http.StatusOK) +} diff --git a/lib/promscrape/discovery/vultr/vultr.go b/lib/promscrape/discovery/vultr/vultr.go new file mode 100644 index 000000000..d8999b2bb --- /dev/null +++ b/lib/promscrape/discovery/vultr/vultr.go @@ -0,0 +1,105 @@ +package vultr + +import ( + "flag" + "fmt" + "strconv" + "strings" + "time" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" +) + +const ( + separator = "," +) + +// SDCheckInterval defines interval for docker targets refresh. +var SDCheckInterval = flag.Duration("promscrape.vultrSDCheckInterval", 30*time.Second, "Interval for checking for changes in Vultr. "+ + "This works only if vultr_sd_configs is configured in '-promscrape.config' file. "+ + "See https://docs.victoriametrics.com/sd_configs.html#vultr_sd_configs for details") + +// SDConfig represents service discovery config for Vultr. +// See: https://prometheus.io/docs/prometheus/latest/configuration/configuration/#vultr_sd_config +// Additional query params are supported, while Prometheus only supports `Port` and HTTP auth. +type SDConfig struct { + // API query params for filtering. All of them are optional. + // See: https://www.vultr.com/api/#tag/instances/operation/list-instances + Label string `yaml:"label,omitempty"` + MainIP string `yaml:"main_ip,omitempty"` + Region string `yaml:"region,omitempty"` + FirewallGroupID string `yaml:"firewall_group_id,omitempty"` + Hostname string `yaml:"hostname,omitempty"` + + // The port to scrape metrics from. Default 80. + Port int `yaml:"port"` + + // General HTTP / Auth configs. + HTTPClientConfig promauth.HTTPClientConfig `yaml:",inline"` + ProxyURL *proxy.URL `yaml:"proxy_url,omitempty"` + ProxyClientConfig promauth.ProxyClientConfig `yaml:",inline"` + + // refresh_interval is obtained from `-promscrape.vultrSDCheckInterval` command-line option. +} + +var configMap = discoveryutils.NewConfigMap() + +// GetLabels returns gce labels according to sdc. +func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) { + ac, err := getAPIConfig(sdc, baseDir) + if err != nil { + return nil, fmt.Errorf("cannot get API config: %w", err) + } + instances, err := getInstances(ac) + if err != nil { + return nil, err + } + return getInstanceLabels(instances, ac.port), nil +} + +// MustStop stops further usage for sdc. +func (sdc *SDConfig) MustStop() { + configMap.Delete(sdc) +} + +// getInstanceLabels returns labels for vultr instances obtained from the given cfg +func getInstanceLabels(instances []Instance, port int) []*promutils.Labels { + ms := make([]*promutils.Labels, 0, len(instances)) + + for _, instance := range instances { + m := promutils.NewLabels(18) + m.Add("__address__", discoveryutils.JoinHostPort(instance.MainIP, port)) + m.Add("__meta_vultr_instance_id", instance.ID) + m.Add("__meta_vultr_instance_label", instance.Label) + m.Add("__meta_vultr_instance_os", instance.Os) + m.Add("__meta_vultr_instance_os_id", strconv.Itoa(instance.OsID)) + m.Add("__meta_vultr_instance_region", instance.Region) + m.Add("__meta_vultr_instance_plan", instance.Plan) + m.Add("__meta_vultr_instance_main_ip", instance.MainIP) + m.Add("__meta_vultr_instance_internal_ip", instance.InternalIP) + m.Add("__meta_vultr_instance_main_ipv6", instance.V6MainIP) + m.Add("__meta_vultr_instance_hostname", instance.Hostname) + m.Add("__meta_vultr_instance_server_status", instance.ServerStatus) + m.Add("__meta_vultr_instance_vcpu_count", strconv.Itoa(instance.VCPUCount)) + m.Add("__meta_vultr_instance_ram_mb", strconv.Itoa(instance.RAM)) + m.Add("__meta_vultr_instance_allowed_bandwidth_gb", strconv.Itoa(instance.AllowedBandwidth)) + m.Add("__meta_vultr_instance_disk_gb", strconv.Itoa(instance.Disk)) + + // We surround the separated list with the separator as well. This way regular expressions + // in relabeling rules don't have to consider feature positions. + if len(instance.Features) > 0 { + features := separator + strings.Join(instance.Features, separator) + separator + m.Add("__meta_vultr_instance_features", features) + } + + if len(instance.Tags) > 0 { + tags := separator + strings.Join(instance.Tags, separator) + separator + m.Add("__meta_vultr_instance_tags", tags) + } + ms = append(ms, m) + } + return ms +} diff --git a/lib/promscrape/discovery/vultr/vultr_test.go b/lib/promscrape/discovery/vultr/vultr_test.go new file mode 100644 index 000000000..572782726 --- /dev/null +++ b/lib/promscrape/discovery/vultr/vultr_test.go @@ -0,0 +1,96 @@ +package vultr + +import ( + "testing" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" +) + +func TestGetInstanceLabels(t *testing.T) { + input := []Instance{ + { + ID: "fake-id-07f7-4b68-88ac-fake-id", + Os: "Ubuntu 22.04 x64", + RAM: 1024, + Disk: 25, + MainIP: "64.176.84.27", + VCPUCount: 1, + Region: "sgp", + Plan: "vc2-1c-1gb", + AllowedBandwidth: 1, + ServerStatus: "installingbooting", + V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + Label: "vultr-sd", + InternalIP: "", + Hostname: "vultr-sd", + Tags: []string{"mock tags"}, + OsID: 1743, + Features: []string{"ipv6"}, + }, + { + ID: "fake-id-07f7-4b68-88ac-fake-id", + Os: "Ubuntu 22.04 x64", + RAM: 1024, + Disk: 25, + MainIP: "64.176.84.27", + VCPUCount: 1, + Region: "sgp", + Plan: "vc2-1c-1gb", + AllowedBandwidth: 1, + ServerStatus: "installingbooting", + V6MainIP: "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + Label: "vultr-sd", + InternalIP: "", + Hostname: "vultr-sd", + Tags: []string{"mock tags"}, + OsID: 1743, + Features: []string{"ipv6"}, + }, + } + + expect := []*promutils.Labels{ + promutils.NewLabelsFromMap(map[string]string{ + "__address__": "64.176.84.27:8080", + "__meta_vultr_instance_id": "fake-id-07f7-4b68-88ac-fake-id", + "__meta_vultr_instance_label": "vultr-sd", + "__meta_vultr_instance_os": "Ubuntu 22.04 x64", + "__meta_vultr_instance_os_id": "1743", + "__meta_vultr_instance_region": "sgp", + "__meta_vultr_instance_plan": "vc2-1c-1gb", + "__meta_vultr_instance_main_ip": "64.176.84.27", + "__meta_vultr_instance_internal_ip": "", + "__meta_vultr_instance_main_ipv6": "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + "__meta_vultr_instance_hostname": "vultr-sd", + "__meta_vultr_instance_server_status": "installingbooting", + "__meta_vultr_instance_vcpu_count": "1", + "__meta_vultr_instance_ram_mb": "1024", + "__meta_vultr_instance_allowed_bandwidth_gb": "1", + "__meta_vultr_instance_disk_gb": "25", + "__meta_vultr_instance_features": ",ipv6,", + "__meta_vultr_instance_tags": ",mock tags,", + }), + promutils.NewLabelsFromMap(map[string]string{ + "__address__": "64.176.84.27:8080", + "__meta_vultr_instance_id": "fake-id-07f7-4b68-88ac-fake-id", + "__meta_vultr_instance_label": "vultr-sd", + "__meta_vultr_instance_os": "Ubuntu 22.04 x64", + "__meta_vultr_instance_os_id": "1743", + "__meta_vultr_instance_region": "sgp", + "__meta_vultr_instance_plan": "vc2-1c-1gb", + "__meta_vultr_instance_main_ip": "64.176.84.27", + "__meta_vultr_instance_internal_ip": "", + "__meta_vultr_instance_main_ipv6": "2002:18f0:4100:263a:5300:07ff:fdd7:691c", + "__meta_vultr_instance_hostname": "vultr-sd", + "__meta_vultr_instance_server_status": "installingbooting", + "__meta_vultr_instance_vcpu_count": "1", + "__meta_vultr_instance_ram_mb": "1024", + "__meta_vultr_instance_allowed_bandwidth_gb": "1", + "__meta_vultr_instance_disk_gb": "25", + "__meta_vultr_instance_features": ",ipv6,", + "__meta_vultr_instance_tags": ",mock tags,", + }), + } + labels := getInstanceLabels(input, 8080) + discoveryutils.TestEqualLabelss(t, labels, expect) +} diff --git a/lib/promscrape/scraper.go b/lib/promscrape/scraper.go index 797503298..d60511b8c 100644 --- a/lib/promscrape/scraper.go +++ b/lib/promscrape/scraper.go @@ -30,6 +30,7 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/kuma" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/nomad" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/openstack" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/vultr" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/yandexcloud" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" "github.com/VictoriaMetrics/metrics" @@ -140,6 +141,7 @@ func runScraper(configFile string, pushData func(at *auth.Token, wr *prompbmarsh scs.add("kuma_sd_configs", *kuma.SDCheckInterval, func(cfg *Config, swsPrev []*ScrapeWork) []*ScrapeWork { return cfg.getKumaSDScrapeWork(swsPrev) }) scs.add("nomad_sd_configs", *nomad.SDCheckInterval, func(cfg *Config, swsPrev []*ScrapeWork) []*ScrapeWork { return cfg.getNomadSDScrapeWork(swsPrev) }) scs.add("openstack_sd_configs", *openstack.SDCheckInterval, func(cfg *Config, swsPrev []*ScrapeWork) []*ScrapeWork { return cfg.getOpenStackSDScrapeWork(swsPrev) }) + scs.add("vultr_sd_configs", *vultr.SDCheckInterval, func(cfg *Config, swsPrev []*ScrapeWork) []*ScrapeWork { return cfg.getVultrSDScrapeWork(swsPrev) }) scs.add("yandexcloud_sd_configs", *yandexcloud.SDCheckInterval, func(cfg *Config, swsPrev []*ScrapeWork) []*ScrapeWork { return cfg.getYandexCloudSDScrapeWork(swsPrev) }) scs.add("static_configs", 0, func(cfg *Config, _ []*ScrapeWork) []*ScrapeWork { return cfg.getStaticScrapeWork() }) diff --git a/lib/promscrape/scrapework.go b/lib/promscrape/scrapework.go index 837db6dc5..ad478dbf7 100644 --- a/lib/promscrape/scrapework.go +++ b/lib/promscrape/scrapework.go @@ -670,6 +670,7 @@ type writeRequestCtx struct { writeRequest prompbmarshal.WriteRequest labels []prompbmarshal.Label samples []prompbmarshal.Sample + exemplars []prompbmarshal.Exemplar } func (wc *writeRequestCtx) reset() { @@ -684,6 +685,7 @@ func (wc *writeRequestCtx) resetNoRows() { wc.labels = wc.labels[:0] wc.samples = wc.samples[:0] + wc.exemplars = wc.exemplars[:0] } var writeRequestCtxPool leveledWriteRequestCtxPool @@ -902,10 +904,27 @@ func (sw *scrapeWork) addRowToTimeseries(wc *writeRequestCtx, r *parser.Row, tim Value: r.Value, Timestamp: sampleTimestamp, }) + // Add Exemplars to Timeseries + exemplarsLen := len(wc.exemplars) + exemplarTagsLen := len(r.Exemplar.Tags) + if exemplarTagsLen > 0 { + exemplarLabels := make([]prompbmarshal.Label, exemplarTagsLen) + for i, label := range r.Exemplar.Tags { + exemplarLabels[i].Name = label.Key + exemplarLabels[i].Value = label.Value + } + wc.exemplars = append(wc.exemplars, prompbmarshal.Exemplar{ + Labels: exemplarLabels, + Value: r.Exemplar.Value, + Timestamp: r.Exemplar.Timestamp, + }) + + } wr := &wc.writeRequest wr.Timeseries = append(wr.Timeseries, prompbmarshal.TimeSeries{ - Labels: wc.labels[labelsLen:], - Samples: wc.samples[len(wc.samples)-1:], + Labels: wc.labels[labelsLen:], + Samples: wc.samples[len(wc.samples)-1:], + Exemplars: wc.exemplars[exemplarsLen:], }) } diff --git a/lib/promscrape/scrapework_test.go b/lib/promscrape/scrapework_test.go index 787203d29..a4a972c62 100644 --- a/lib/promscrape/scrapework_test.go +++ b/lib/promscrape/scrapework_test.go @@ -708,6 +708,11 @@ func TestAddRowToTimeseriesNoRelabeling(t *testing.T) { HonorLabels: true, }, `metric{a="e",foo="bar"} 0 123`) + f(`metric{foo="bar"} 0 123 # {trace_id="12345"} 52 456`, + &ScrapeWork{ + HonorLabels: true, + }, + `metric{foo="bar"} 0 123 # {trace_id="12345"} 52 456`) } func TestSendStaleSeries(t *testing.T) { @@ -765,6 +770,8 @@ func parseData(data string) []prompbmarshal.TimeSeries { } rows.UnmarshalWithErrLogger(data, errLogger) var tss []prompbmarshal.TimeSeries + var exemplars []prompbmarshal.Exemplar + for _, r := range rows.Rows { labels := []prompbmarshal.Label{ { @@ -778,6 +785,21 @@ func parseData(data string) []prompbmarshal.TimeSeries { Value: tag.Value, }) } + exemplarLabels := []prompbmarshal.Label{} + if len(r.Exemplar.Tags) > 0 { + for _, tag := range r.Exemplar.Tags { + exemplarLabels = append(exemplarLabels, prompbmarshal.Label{ + Name: tag.Key, + Value: tag.Value, + }) + } + exemplars = append(exemplars, prompbmarshal.Exemplar{ + Labels: exemplarLabels, + Value: r.Exemplar.Value, + Timestamp: r.Exemplar.Timestamp, + }) + } + var ts prompbmarshal.TimeSeries ts.Labels = labels ts.Samples = []prompbmarshal.Sample{ @@ -786,6 +808,7 @@ func parseData(data string) []prompbmarshal.TimeSeries { Timestamp: r.Timestamp, }, } + ts.Exemplars = exemplars tss = append(tss, ts) } return tss @@ -850,6 +873,19 @@ func timeseriesToString(ts *prompbmarshal.TimeSeries) string { } s := ts.Samples[0] fmt.Fprintf(&sb, "%g %d", s.Value, s.Timestamp) + // Add Exemplars to the end of string + for j, exemplar := range ts.Exemplars { + for i, label := range exemplar.Labels { + fmt.Fprintf(&sb, "%s=%q", label.Name, label.Value) + if i+1 < len(ts.Labels) { + fmt.Fprintf(&sb, ",") + } + } + fmt.Fprintf(&sb, "%g %d", exemplar.Value, exemplar.Timestamp) + if j+1 < len(ts.Exemplars) { + fmt.Fprintf(&sb, ",") + } + } return sb.String() } diff --git a/lib/protoparser/prometheus/parser.go b/lib/protoparser/prometheus/parser.go index 923737f12..58ef46f1f 100644 --- a/lib/protoparser/prometheus/parser.go +++ b/lib/protoparser/prometheus/parser.go @@ -57,12 +57,33 @@ func (rs *Rows) UnmarshalWithErrLogger(s string, errLogger func(s string)) { rs.Rows, rs.tagsPool = unmarshalRows(rs.Rows[:0], s, rs.tagsPool[:0], noEscapes, errLogger) } +const tagsPrefix = '{' +const exemplarPreifx = '{' + +// Exemplar Item +type Exemplar struct { + // Tags: a list of labels that uniquely identifies exemplar + Tags []Tag + // Value: the value of the exemplar + Value float64 + // Timestamp: the time when exemplar was recorded + Timestamp int64 +} + +// Reset - resets the Exemplar object to defaults +func (e *Exemplar) Reset() { + e.Tags = nil + e.Value = 0 + e.Timestamp = 0 +} + // Row is a single Prometheus row. type Row struct { Metric string Tags []Tag Value float64 Timestamp int64 + Exemplar Exemplar } func (r *Row) reset() { @@ -70,6 +91,7 @@ func (r *Row) reset() { r.Tags = nil r.Value = 0 r.Timestamp = 0 + r.Exemplar = Exemplar{} } func skipTrailingComment(s string) string { @@ -110,69 +132,140 @@ func nextWhitespace(s string) int { return n1 } -func (r *Row) unmarshal(s string, tagsPool []Tag, noEscapes bool) ([]Tag, error) { - r.reset() +func parseStringToTags(s string, tagsPool []Tag, noEscapes bool) (string, []Tag, error) { + n := strings.IndexByte(s, tagsPrefix) + c := strings.IndexByte(s, '#') + if c != -1 && c < n { + return s, tagsPool, nil + } s = skipLeadingWhitespace(s) - n := strings.IndexByte(s, '{') if n >= 0 { // Tags found. Parse them. - r.Metric = skipTrailingWhitespace(s[:n]) s = s[n+1:] - tagsStart := len(tagsPool) var err error s, tagsPool, err = unmarshalTags(tagsPool, s, noEscapes) if err != nil { - return tagsPool, fmt.Errorf("cannot unmarshal tags: %w", err) + return s, tagsPool, fmt.Errorf("cannot unmarshal tags: %w", err) } if len(s) > 0 && s[0] == ' ' { // Fast path - skip whitespace. s = s[1:] } - tags := tagsPool[tagsStart:] - r.Tags = tags[:len(tags):len(tags)] - } else { - // Tags weren't found. Search for value after whitespace - n = nextWhitespace(s) - if n < 0 { - return tagsPool, fmt.Errorf("missing value") - } - r.Metric = s[:n] - s = s[n+1:] } - if len(r.Metric) == 0 { - return tagsPool, fmt.Errorf("metric cannot be empty") + return s, tagsPool, nil +} + +var tvtPool = sync.Pool{New: func() interface{} { + return &tagsValueTimestamp{} +}} + +func getTVT() *tagsValueTimestamp { + return tvtPool.Get().(*tagsValueTimestamp) +} +func putTVT(tvt *tagsValueTimestamp) { + tvt.reset() + tvtPool.Put(tvt) +} + +type tagsValueTimestamp struct { + Prefix string + Value float64 + Timestamp int64 + Comments string +} + +func (tvt *tagsValueTimestamp) reset() { + tvt.Prefix = "" + tvt.Value = 0 + tvt.Timestamp = 0 + tvt.Comments = "" +} +func parseTagsValueTimestamp(s string, tagsPool []Tag, noEscapes bool) ([]Tag, *tagsValueTimestamp, error) { + tvt := getTVT() + n := 0 + // Prefix is everything up to a tag start or a space + t := strings.IndexByte(s, tagsPrefix) + // If there is no tag start process rest of string + mustParseTags := false + if t != -1 { + // Check to see if there is a space before tag + n = nextWhitespace(s) + // If there is a space + if n > 0 { + if n < t { + tvt.Prefix = s[:n] + s = skipLeadingWhitespace(s[n:]) + // Cover the use case where there is whitespace between the prefix and the tag + if len(s) > 0 && s[0] == '{' { + mustParseTags = true + } + // Most likely this has an exemplar + } else { + tvt.Prefix = s[:t] + s = s[t:] + mustParseTags = true + } + } + if mustParseTags { + var err error + s, tagsPool, err = parseStringToTags(s, tagsPool, noEscapes) + if err != nil { + return tagsPool, tvt, err + } + } + } else { + // Tag doesn't exist + n = nextWhitespace(s) + if n != -1 { + tvt.Prefix = s[:n] + s = s[n:] + } else { + tvt.Prefix = s + return tagsPool, tvt, fmt.Errorf("missing value") + } } s = skipLeadingWhitespace(s) - s = skipTrailingComment(s) - if len(s) == 0 { - return tagsPool, fmt.Errorf("value cannot be empty") + // save and remove the comments + n = strings.IndexByte(s, '#') + if n >= 0 { + tvt.Comments = s[n:] + if len(tvt.Comments) > 1 { + tvt.Comments = s[n+1:] + } else { + tvt.Comments = "" + } + s = skipTrailingComment(s) + s = skipLeadingWhitespace(s) + s = skipTrailingWhitespace(s) } n = nextWhitespace(s) if n < 0 { // There is no timestamp. v, err := fastfloat.Parse(s) if err != nil { - return tagsPool, fmt.Errorf("cannot parse value %q: %w", s, err) + return tagsPool, tvt, fmt.Errorf("cannot parse value %q: %w", s, err) } - r.Value = v - return tagsPool, nil + tvt.Value = v + return tagsPool, tvt, nil } - // There is a timestamp. + // There is a timestamp + s = skipLeadingWhitespace(s) v, err := fastfloat.Parse(s[:n]) if err != nil { - return tagsPool, fmt.Errorf("cannot parse value %q: %w", s[:n], err) + return tagsPool, tvt, fmt.Errorf("cannot parse value %q: %w", s[:n], err) } - r.Value = v - s = skipLeadingWhitespace(s[n+1:]) + tvt.Value = v + s = s[n:] + // There are some whitespaces after timestamp + s = skipLeadingWhitespace(s) if len(s) == 0 { // There is no timestamp - just a whitespace after the value. - return tagsPool, nil + return tagsPool, tvt, nil } - // There are some whitespaces after timestamp s = skipTrailingWhitespace(s) ts, err := fastfloat.Parse(s) if err != nil { - return tagsPool, fmt.Errorf("cannot parse timestamp %q: %w", s, err) + return tagsPool, tvt, fmt.Errorf("cannot parse timestamp %q: %w", s, err) } if ts >= -1<<31 && ts < 1<<31 { // This looks like OpenMetrics timestamp in Unix seconds. @@ -181,7 +274,68 @@ func (r *Row) unmarshal(s string, tagsPool []Tag, noEscapes bool) ([]Tag, error) // See https://github.com/OpenObservability/OpenMetrics/blob/master/specification/OpenMetrics.md#timestamps ts *= 1000 } - r.Timestamp = int64(ts) + tvt.Timestamp = int64(ts) + return tagsPool, tvt, nil +} + +// Returns possible comments that could be exemplars +func (r *Row) unmarshalMetric(s string, tagsPool []Tag, noEscapes bool) (string, []Tag, error) { + tagsStart := len(tagsPool) + var err error + var tvt *tagsValueTimestamp + tagsPool, tvt, err = parseTagsValueTimestamp(s, tagsPool, noEscapes) + defer putTVT(tvt) + if err != nil { + return "", tagsPool, err + } + r.Metric = tvt.Prefix + tags := tagsPool[tagsStart:] + if len(tags) > 0 { + r.Tags = tags + } + r.Value = tvt.Value + r.Timestamp = tvt.Timestamp + return tvt.Comments, tagsPool, nil + +} +func (e *Exemplar) unmarshal(s string, tagsPool []Tag, noEscapes bool) ([]Tag, error) { + // We can use the Comment parsed out to further parse the Exemplar + s = skipLeadingWhitespace(s) + // If we are a comment immediately followed by whitespace or a labelset + // then we are an exemplar + if len(s) != 0 && s[0] == exemplarPreifx { + var err error + var tvt *tagsValueTimestamp + tagsStart := len(tagsPool) + tagsPool, tvt, err = parseTagsValueTimestamp(s, tagsPool, noEscapes) + defer putTVT(tvt) + if err != nil { + return tagsPool, err + } + tags := tagsPool[tagsStart:] + if len(tags) > 0 { + e.Tags = tags + } + e.Value = tvt.Value + e.Timestamp = tvt.Timestamp + return tagsPool, nil + } + return tagsPool, nil + +} +func (r *Row) unmarshal(s string, tagsPool []Tag, noEscapes bool) ([]Tag, error) { + r.reset() + // Parse for labels, the value and the timestamp + // Anything before labels is saved in Prefix we can use this + // for the metric name + comments, tagsPool, err := r.unmarshalMetric(s, tagsPool, noEscapes) + if err != nil { + return nil, err + } + tagsPool, err = r.Exemplar.unmarshal(comments, tagsPool, noEscapes) + if err != nil { + return nil, err + } return tagsPool, nil } diff --git a/lib/protoparser/prometheus/parser_test.go b/lib/protoparser/prometheus/parser_test.go index 39116c937..ff95f25b1 100644 --- a/lib/protoparser/prometheus/parser_test.go +++ b/lib/protoparser/prometheus/parser_test.go @@ -362,12 +362,87 @@ cassandra_token_ownership_ratio 78.9`, &Rows{ Value: 56, }}, }) - - // Exemplars - see https://github.com/OpenObservability/OpenMetrics/blob/master/OpenMetrics.md#exemplars-1 - f(`foo_bucket{le="10",a="#b"} 17 # {trace_id="oHg5SJ#YRHA0"} 9.8 1520879607.789 - abc 123 456 # foobar - foo 344#bar`, &Rows{ + // Support for Exemplars Open Metric Specification + // see: https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#exemplars-1 + f(`foo_bucket{le="25"} 17 # {trace_id="oHg5SJYRHA0", log_id="test_id"} 9.8 1520879607.789`, &Rows{ Rows: []Row{ + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "le", + Value: "25", + }, + }, + Value: 17, + Exemplar: Exemplar{ + Value: 9.8, + Tags: []Tag{ + { + Key: "trace_id", + Value: "oHg5SJYRHA0", + }, + { + Key: "log_id", + Value: "test_id", + }, + }, + Timestamp: 1520879607789, + }, + }, + }}) + f(`foo_bucket{le="0.01"} 0 +foo_bucket{le="0.1"} 8 # {} 0.054 +foo_bucket{le="1"} 11 # {trace_id="KOO5S4vxi0o"} 0.67 +foo_bucket{le="10"} 17 # {trace_id="oHg5SJYRHA0"} 9.8 1520879607.789 +foo_bucket{le="25"} 17 # {trace_id="oHg5SJYRHA0", log_id="test_id"} 9.8 1520879607.789 +foo_bucket{nospace="exemplar"} 17 #{trace_id="oHg5SJYRHA0"} 9.8 1520879607.789 +foo_bucket{le="+Inf"} 17 +foo_count 17 +foo_sum 324789.3 +foo_created 1520430000.123`, &Rows{ + Rows: []Row{ + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "le", + Value: "0.01", + }, + }, + }, + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "le", + Value: "0.1", + }, + }, + Value: 8, + Exemplar: Exemplar{ + Value: 0.054, + }, + }, + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "le", + Value: "1", + }, + }, + Value: 11, + Exemplar: Exemplar{ + Value: 0.67, + Tags: []Tag{ + { + Key: "trace_id", + Value: "KOO5S4vxi0o", + }, + }, + }, + }, { Metric: "foo_bucket", Tags: []Tag{ @@ -375,21 +450,84 @@ cassandra_token_ownership_ratio 78.9`, &Rows{ Key: "le", Value: "10", }, + }, + Value: 17, + Exemplar: Exemplar{ + Value: 9.8, + Tags: []Tag{ + { + Key: "trace_id", + Value: "oHg5SJYRHA0", + }, + }, + Timestamp: 1520879607789, + }, + }, + { + Metric: "foo_bucket", + Tags: []Tag{ { - Key: "a", - Value: "#b", + Key: "le", + Value: "25", + }, + }, + Value: 17, + Exemplar: Exemplar{ + Value: 9.8, + Tags: []Tag{ + { + Key: "trace_id", + Value: "oHg5SJYRHA0", + }, + { + Key: "log_id", + Value: "test_id", + }, + }, + Timestamp: 1520879607789, + }, + }, + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "nospace", + Value: "exemplar", + }, + }, + Value: 17, + Exemplar: Exemplar{ + Value: 9.8, + Tags: []Tag{ + { + Key: "trace_id", + Value: "oHg5SJYRHA0", + }, + }, + Timestamp: 1520879607789, + }, + }, + { + Metric: "foo_bucket", + Tags: []Tag{ + { + Key: "le", + Value: "+Inf", }, }, Value: 17, }, { - Metric: "abc", - Value: 123, - Timestamp: 456000, + Metric: "foo_count", + Value: 17, }, { - Metric: "foo", - Value: 344, + Metric: "foo_sum", + Value: 324789.3, + }, + { + Metric: "foo_created", + Value: 1520430000.123, }, }, }) diff --git a/lib/protoparser/prometheus/parser_timing_test.go b/lib/protoparser/prometheus/parser_timing_test.go index 8d6a2827b..acabb7460 100644 --- a/lib/protoparser/prometheus/parser_timing_test.go +++ b/lib/protoparser/prometheus/parser_timing_test.go @@ -146,10 +146,10 @@ container_ulimits_soft{container="kube-scheduler",id="/kubelet/kubepods/burstabl } func BenchmarkRowsUnmarshal(b *testing.B) { - s := `cpu_usage{mode="user"} 1.23 -cpu_usage{mode="system"} 23.344 -cpu_usage{mode="iowait"} 3.3443 -cpu_usage{mode="irq"} 0.34432 + s := `foo_bucket{le="0.01"} 0 + foo_bucket{le="1"} 11 # {trace_id="KOO5S4vxi0o"} 0.67 + foo_bucket{le="10"} 17 # {trace_id="oHg5SJYRHA0"} 9.8 1520879607.789 + foo_bucket{nospace="exemplar"} 17 #{trace_id="oHg5SJYRHA0"} 9.8 1520879607.789 ` b.SetBytes(int64(len(s))) b.ReportAllocs() diff --git a/lib/protoparser/statsd/parser.go b/lib/protoparser/statsd/parser.go new file mode 100644 index 000000000..3ae9800d6 --- /dev/null +++ b/lib/protoparser/statsd/parser.go @@ -0,0 +1,226 @@ +package statsd + +import ( + "fmt" + "strings" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" + "github.com/VictoriaMetrics/metrics" + "github.com/valyala/fastjson/fastfloat" +) + +// Statsd metric format with tags: MetricName:value|type|@sample_rate|#tag1:value,tag1... +const statsdSeparator = '|' +const statsdPairsSeparator = ':' +const statsdTagsStartSeparator = '#' +const statsdTagsSeparator = ',' + +// Rows contains parsed statsd rows. +type Rows struct { + Rows []Row + + tagsPool []Tag +} + +// Reset resets rs. +func (rs *Rows) Reset() { + // Reset items, so they can be GC'ed + + for i := range rs.Rows { + rs.Rows[i].reset() + } + rs.Rows = rs.Rows[:0] + + for i := range rs.tagsPool { + rs.tagsPool[i].reset() + } + rs.tagsPool = rs.tagsPool[:0] +} + +// Unmarshal unmarshals statsd plaintext protocol rows from s. +// +// s shouldn't be modified when rs is in use. +func (rs *Rows) Unmarshal(s string) { + rs.Rows, rs.tagsPool = unmarshalRows(rs.Rows[:0], s, rs.tagsPool[:0]) +} + +// Row is a single statsd row. +type Row struct { + Metric string + Tags []Tag + Value float64 + Timestamp int64 +} + +func (r *Row) reset() { + r.Metric = "" + r.Tags = nil + r.Value = 0 + r.Timestamp = 0 +} + +func (r *Row) unmarshal(s string, tagsPool []Tag) ([]Tag, error) { + r.reset() + originalString := s + s = stripTrailingWhitespace(s) + separatorPosition := strings.IndexByte(s, statsdSeparator) + if separatorPosition < 0 { + s = stripTrailingWhitespace(s) + } else { + s = stripTrailingWhitespace(s[:separatorPosition]) + } + + valuesSeparatorPosition := strings.LastIndexByte(s, statsdPairsSeparator) + + if valuesSeparatorPosition == 0 { + return tagsPool, fmt.Errorf("cannot find metric name for %q", s) + } + + if valuesSeparatorPosition < 0 { + return tagsPool, fmt.Errorf("cannot find separator for %q", s) + } + + r.Metric = s[:valuesSeparatorPosition] + valueStr := s[valuesSeparatorPosition+1:] + + v, err := fastfloat.Parse(valueStr) + if err != nil { + return tagsPool, fmt.Errorf("cannot unmarshal value from %q: %w; original line: %q", valueStr, err, originalString) + } + r.Value = v + + // parsing tags + tagsSeparatorPosition := strings.LastIndexByte(originalString, statsdTagsStartSeparator) + + if tagsSeparatorPosition < 0 { + // no tags + return tagsPool, nil + } + + tagsStart := len(tagsPool) + tagsPool = unmarshalTags(tagsPool, originalString[tagsSeparatorPosition+1:]) + tags := tagsPool[tagsStart:] + r.Tags = tags[:len(tags):len(tags)] + + return tagsPool, nil +} + +func unmarshalRows(dst []Row, s string, tagsPool []Tag) ([]Row, []Tag) { + for len(s) > 0 { + n := strings.IndexByte(s, '\n') + if n < 0 { + // The last line. + return unmarshalRow(dst, s, tagsPool) + } + dst, tagsPool = unmarshalRow(dst, s[:n], tagsPool) + s = s[n+1:] + } + return dst, tagsPool +} + +func unmarshalRow(dst []Row, s string, tagsPool []Tag) ([]Row, []Tag) { + if len(s) > 0 && s[len(s)-1] == '\r' { + s = s[:len(s)-1] + } + s = stripLeadingWhitespace(s) + if len(s) == 0 { + // Skip empty line + return dst, tagsPool + } + if cap(dst) > len(dst) { + dst = dst[:len(dst)+1] + } else { + dst = append(dst, Row{}) + } + r := &dst[len(dst)-1] + var err error + tagsPool, err = r.unmarshal(s, tagsPool) + if err != nil { + dst = dst[:len(dst)-1] + logger.Errorf("cannot unmarshal Statsd line %q: %s", s, err) + invalidLines.Inc() + } + return dst, tagsPool +} + +var invalidLines = metrics.NewCounter(`vm_rows_invalid_total{type="statsd"}`) + +func unmarshalTags(dst []Tag, s string) []Tag { + for { + if cap(dst) > len(dst) { + dst = dst[:len(dst)+1] + } else { + dst = append(dst, Tag{}) + } + tag := &dst[len(dst)-1] + + n := strings.IndexByte(s, statsdTagsSeparator) + + if n < 0 { + // The last tag found + tag.unmarshal(s) + if len(tag.Key) == 0 || len(tag.Value) == 0 { + // Skip empty tag + dst = dst[:len(dst)-1] + } + return dst + } + tag.unmarshal(s[:n]) + s = s[n+1:] + if len(tag.Key) == 0 || len(tag.Value) == 0 { + // Skip empty tag + dst = dst[:len(dst)-1] + } + } +} + +// Tag is a statsd tag. +type Tag struct { + Key string + Value string +} + +func (t *Tag) reset() { + t.Key = "" + t.Value = "" +} + +func (t *Tag) unmarshal(s string) { + t.reset() + n := strings.IndexByte(s, statsdPairsSeparator) + if n < 0 { + // Empty tag value. + // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1100 + t.Key = s + t.Value = s[len(s):] + } else { + t.Key = s[:n] + t.Value = s[n+1:] + } +} + +func stripTrailingWhitespace(s string) string { + n := len(s) + for { + n-- + if n < 0 { + return "" + } + ch := s[n] + + if ch != ' ' && ch != '\t' { + return s[:n+1] + } + } +} + +func stripLeadingWhitespace(s string) string { + for len(s) > 0 { + ch := s[0] + if ch != ' ' && ch != '\t' { + return s + } + s = s[1:] + } + return "" +} diff --git a/lib/protoparser/statsd/parser_test.go b/lib/protoparser/statsd/parser_test.go new file mode 100644 index 000000000..aeb498bb7 --- /dev/null +++ b/lib/protoparser/statsd/parser_test.go @@ -0,0 +1,367 @@ +package statsd + +import ( + "reflect" + "testing" +) + +func TestUnmarshalTagsSuccess(t *testing.T) { + f := func(dst []Tag, s string, tagsPoolExpected []Tag) { + t.Helper() + + tagsPool := unmarshalTags(dst, s) + if !reflect.DeepEqual(tagsPool, tagsPoolExpected) { + t.Fatalf("unexpected tags;\ngot\n%+v;\nwant\n%+v", tagsPool, tagsPoolExpected) + } + + // Try unmarshaling again + tagsPool = unmarshalTags(dst, s) + if !reflect.DeepEqual(tagsPool, tagsPoolExpected) { + t.Fatalf("unexpected tags on second unmarshal;\ngot\n%+v;\nwant\n%+v", tagsPool, tagsPoolExpected) + } + } + + f([]Tag{}, "foo:bar", []Tag{ + { + Key: "foo", + Value: "bar", + }, + }) + + f([]Tag{}, "foo:bar,qwe:123", []Tag{ + { + Key: "foo", + Value: "bar", + }, + { + Key: "qwe", + Value: "123", + }, + }) + + f([]Tag{}, "foo.qwe:bar", []Tag{ + { + Key: "foo.qwe", + Value: "bar", + }, + }) + + f([]Tag{}, "foo:10", []Tag{ + { + Key: "foo", + Value: "10", + }, + }) + + f([]Tag{}, "foo: _qwe", []Tag{ + { + Key: "foo", + Value: " _qwe", + }, + }) + + f([]Tag{}, "foo:qwe ", []Tag{ + { + Key: "foo", + Value: "qwe ", + }, + }) + + f([]Tag{}, "foo asd:qwe ", []Tag{ + { + Key: "foo asd", + Value: "qwe ", + }, + }) + + f([]Tag{}, "foo:var:123", []Tag{ + { + Key: "foo", + Value: "var:123", + }, + }) + + // invalid tags + f([]Tag{}, ":bar", []Tag{}) + f([]Tag{}, "foo:", []Tag{}) + f([]Tag{}, " ", []Tag{}) +} + +func TestRowsUnmarshalSuccess(t *testing.T) { + f := func(s string, rowsExpected *Rows) { + t.Helper() + var rows Rows + rows.Unmarshal(s) + if !reflect.DeepEqual(rows.Rows, rowsExpected.Rows) { + t.Fatalf("unexpected rows;\ngot\n%+v;\nwant\n%+v", rows.Rows, rowsExpected.Rows) + } + + // Try unmarshaling again + rows.Unmarshal(s) + if !reflect.DeepEqual(rows.Rows, rowsExpected.Rows) { + t.Fatalf("unexpected rows on second unmarshal;\ngot\n%+v;\nwant\n%+v", rows.Rows, rowsExpected.Rows) + } + + rows.Reset() + if len(rows.Rows) != 0 { + t.Fatalf("non-empty rows after reset: %+v", rows.Rows) + } + } + + // Empty line + f("", &Rows{}) + f("\r", &Rows{}) + f("\n\n", &Rows{}) + f("\n\r\n", &Rows{}) + + // Single line + f(" 123:455", &Rows{ + Rows: []Row{{ + Metric: "123", + Value: 455, + }}, + }) + f("123:455 |c", &Rows{ + Rows: []Row{{ + Metric: "123", + Value: 455, + }}, + }) + f("foobar:-123.456|c", &Rows{ + Rows: []Row{{ + Metric: "foobar", + Value: -123.456, + }}, + }) + f("foo.bar:123.456|c\n", &Rows{ + Rows: []Row{{ + Metric: "foo.bar", + Value: 123.456, + }}, + }) + + // with sample rate + f("foo.bar:1|c|@0.1", &Rows{ + Rows: []Row{{ + Metric: "foo.bar", + Value: 1, + }}, + }) + + // without specifying metric unit + f("foo.bar:123", &Rows{ + Rows: []Row{{ + Metric: "foo.bar", + Value: 123, + }}, + }) + // without specifying metric unit but with tags + f("foo.bar:123|#foo:bar", &Rows{ + Rows: []Row{{ + Metric: "foo.bar", + Value: 123, + Tags: []Tag{ + { + Key: "foo", + Value: "bar", + }, + }, + }}, + }) + + f("foo.bar:123.456|c|#foo:bar,qwe:asd", &Rows{ + Rows: []Row{{ + Metric: "foo.bar", + Value: 123.456, + Tags: []Tag{ + { + Key: "foo", + Value: "bar", + }, + { + Key: "qwe", + Value: "asd", + }, + }, + }}, + }) + + // Whitespace in metric name, tag name and tag value + f("s a:1|c|#ta g1:aaa1,tag2:bb b2", &Rows{ + Rows: []Row{{ + Metric: "s a", + Value: 1, + Tags: []Tag{ + { + Key: "ta g1", + Value: "aaa1", + }, + { + Key: "tag2", + Value: "bb b2", + }, + }, + }}, + }) + + // Tags + f("foo:1|c", &Rows{ + Rows: []Row{{ + Metric: "foo", + Value: 1, + }}, + }) + // Empty tag name + f("foo:1|#:123", &Rows{ + Rows: []Row{{ + Metric: "foo", + Tags: []Tag{}, + Value: 1, + }}, + }) + // Empty tag value + f("foo:1|#tag1:", &Rows{ + Rows: []Row{{ + Metric: "foo", + Tags: []Tag{}, + Value: 1, + }}, + }) + f("foo:1|#bar:baz,aa:,x:y,:z", &Rows{ + Rows: []Row{{ + Metric: "foo", + Tags: []Tag{ + { + Key: "bar", + Value: "baz", + }, + { + Key: "x", + Value: "y", + }, + }, + Value: 1, + }}, + }) + + // Multi lines + f("foo:0.3|c\naaa:3|g\nbar.baz:0.34|c\n", &Rows{ + Rows: []Row{ + { + Metric: "foo", + Value: 0.3, + }, + { + Metric: "aaa", + Value: 3, + }, + { + Metric: "bar.baz", + Value: 0.34, + }, + }, + }) + + f("foo:0.3|c|#tag1:1,tag2:2\naaa:3|g|#tag3:3,tag4:4", &Rows{ + Rows: []Row{ + { + Metric: "foo", + Value: 0.3, + Tags: []Tag{ + { + Key: "tag1", + Value: "1", + }, + { + Key: "tag2", + Value: "2", + }, + }, + }, + { + Metric: "aaa", + Value: 3, + Tags: []Tag{ + { + Key: "tag3", + Value: "3", + }, + { + Key: "tag4", + Value: "4", + }, + }, + }, + }, + }) + + // Multi lines with invalid line + f("foo:0.3|c\naaa\nbar.baz:0.34\n", &Rows{ + Rows: []Row{ + { + Metric: "foo", + Value: 0.3, + }, + { + Metric: "bar.baz", + Value: 0.34, + }, + }, + }) + + // Whitespace after at the end + f("foo.baz:125|c\na:1.34\t ", &Rows{ + Rows: []Row{ + { + Metric: "foo.baz", + Value: 125, + }, + { + Metric: "a", + Value: 1.34, + }, + }, + }) + + // ignores sample rate + f("foo.baz:125|c|@0.5#tag1:12", &Rows{ + Rows: []Row{ + { + Metric: "foo.baz", + Value: 125, + Tags: []Tag{ + { + Key: "tag1", + Value: "12", + }, + }, + }, + }, + }) +} + +func TestRowsUnmarshalFailure(t *testing.T) { + f := func(s string) { + t.Helper() + var rows Rows + rows.Unmarshal(s) + if len(rows.Rows) != 0 { + t.Fatalf("unexpected number of rows parsed; got %d; want 0", len(rows.Rows)) + } + + // Try again + rows.Unmarshal(s) + if len(rows.Rows) != 0 { + t.Fatalf("unexpected number of rows parsed; got %d; want 0", len(rows.Rows)) + } + } + + // random string + f("aaa") + + // empty value + f("foo:") + + // empty metric name + f(":12") +} diff --git a/lib/protoparser/statsd/parser_timing_test.go b/lib/protoparser/statsd/parser_timing_test.go new file mode 100644 index 000000000..d9e3c6479 --- /dev/null +++ b/lib/protoparser/statsd/parser_timing_test.go @@ -0,0 +1,25 @@ +package statsd + +import ( + "fmt" + "testing" +) + +func BenchmarkRowsUnmarshal(b *testing.B) { + s := `cpu.usage_user:1.23|c +cpu.usage_system:23.344|c +cpu.usage_iowait:3.3443|c +cpu.usage_irq:0.34432|c +` + b.SetBytes(int64(len(s))) + b.ReportAllocs() + b.RunParallel(func(pb *testing.PB) { + var rows Rows + for pb.Next() { + rows.Unmarshal(s) + if len(rows.Rows) != 4 { + panic(fmt.Errorf("unexpected number of rows unmarshaled: got %d; want 4", len(rows.Rows))) + } + } + }) +} diff --git a/lib/protoparser/statsd/stream/streamparser.go b/lib/protoparser/statsd/stream/streamparser.go new file mode 100644 index 000000000..27d9e4027 --- /dev/null +++ b/lib/protoparser/statsd/stream/streamparser.go @@ -0,0 +1,218 @@ +package stream + +import ( + "bufio" + "flag" + "fmt" + "io" + "sync" + "time" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/common" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/writeconcurrencylimiter" + "github.com/VictoriaMetrics/metrics" +) + +var ( + trimTimestamp = flag.Duration("statsdTrimTimestamp", time.Second, "Trim timestamps for Statsd data to this duration. "+ + "Minimum practical duration is 1s. Higher duration (i.e. 1m) may be used for reducing disk space usage for timestamp data") +) + +// Parse parses Statsd lines from r and calls callback for the parsed rows. +// +// The callback can be called concurrently multiple times for streamed data from r. +// +// callback shouldn't hold rows after returning. +func Parse(r io.Reader, isGzipped bool, callback func(rows []statsd.Row) error) error { + wcr := writeconcurrencylimiter.GetReader(r) + defer writeconcurrencylimiter.PutReader(wcr) + r = wcr + + if isGzipped { + zr, err := common.GetGzipReader(r) + if err != nil { + return fmt.Errorf("cannot read gzipped statsd data: %w", err) + } + defer common.PutGzipReader(zr) + r = zr + } + + ctx := getStreamContext(r) + defer putStreamContext(ctx) + + for ctx.Read() { + uw := getUnmarshalWork() + uw.ctx = ctx + uw.callback = callback + uw.reqBuf, ctx.reqBuf = ctx.reqBuf, uw.reqBuf + ctx.wg.Add(1) + common.ScheduleUnmarshalWork(uw) + wcr.DecConcurrency() + } + ctx.wg.Wait() + if err := ctx.Error(); err != nil { + return err + } + return ctx.callbackErr +} + +func (ctx *streamContext) Read() bool { + readCalls.Inc() + if ctx.err != nil || ctx.hasCallbackError() { + return false + } + ctx.reqBuf, ctx.tailBuf, ctx.err = common.ReadLinesBlock(ctx.br, ctx.reqBuf, ctx.tailBuf) + if ctx.err != nil { + if ctx.err != io.EOF { + readErrors.Inc() + ctx.err = fmt.Errorf("cannot read statsd plaintext protocol data: %w", ctx.err) + } + return false + } + return true +} + +type streamContext struct { + br *bufio.Reader + reqBuf []byte + tailBuf []byte + err error + + wg sync.WaitGroup + callbackErrLock sync.Mutex + callbackErr error +} + +func (ctx *streamContext) Error() error { + if ctx.err == io.EOF { + return nil + } + return ctx.err +} + +func (ctx *streamContext) hasCallbackError() bool { + ctx.callbackErrLock.Lock() + ok := ctx.callbackErr != nil + ctx.callbackErrLock.Unlock() + return ok +} + +func (ctx *streamContext) reset() { + ctx.br.Reset(nil) + ctx.reqBuf = ctx.reqBuf[:0] + ctx.tailBuf = ctx.tailBuf[:0] + ctx.err = nil + ctx.callbackErr = nil +} + +var ( + readCalls = metrics.NewCounter(`vm_protoparser_read_calls_total{type="statsd"}`) + readErrors = metrics.NewCounter(`vm_protoparser_read_errors_total{type="statsd"}`) + rowsRead = metrics.NewCounter(`vm_protoparser_rows_read_total{type="statsd"}`) +) + +func getStreamContext(r io.Reader) *streamContext { + select { + case ctx := <-streamContextPoolCh: + ctx.br.Reset(r) + return ctx + default: + if v := streamContextPool.Get(); v != nil { + ctx := v.(*streamContext) + ctx.br.Reset(r) + return ctx + } + return &streamContext{ + br: bufio.NewReaderSize(r, 64*1024), + } + } +} + +func putStreamContext(ctx *streamContext) { + ctx.reset() + select { + case streamContextPoolCh <- ctx: + default: + streamContextPool.Put(ctx) + } +} + +var streamContextPool sync.Pool +var streamContextPoolCh = make(chan *streamContext, cgroup.AvailableCPUs()) + +type unmarshalWork struct { + rows statsd.Rows + ctx *streamContext + callback func(rows []statsd.Row) error + reqBuf []byte +} + +func (uw *unmarshalWork) reset() { + uw.rows.Reset() + uw.ctx = nil + uw.callback = nil + uw.reqBuf = uw.reqBuf[:0] +} + +func (uw *unmarshalWork) runCallback(rows []statsd.Row) { + ctx := uw.ctx + if err := uw.callback(rows); err != nil { + ctx.callbackErrLock.Lock() + if ctx.callbackErr == nil { + ctx.callbackErr = fmt.Errorf("error when processing imported data: %w", err) + } + ctx.callbackErrLock.Unlock() + } + ctx.wg.Done() +} + +// Unmarshal implements common.UnmarshalWork +func (uw *unmarshalWork) Unmarshal() { + uw.rows.Unmarshal(bytesutil.ToUnsafeString(uw.reqBuf)) + rows := uw.rows.Rows + rowsRead.Add(len(rows)) + + // Fill missing timestamps with the current timestamp rounded to seconds. + currentTimestamp := int64(fasttime.UnixTimestamp()) + for i := range rows { + r := &rows[i] + if r.Timestamp == 0 || r.Timestamp == -1 { + r.Timestamp = currentTimestamp + } + } + + // Convert timestamps from seconds to milliseconds. + for i := range rows { + rows[i].Timestamp *= 1e3 + } + + // Trim timestamps if required. + if tsTrim := trimTimestamp.Milliseconds(); tsTrim > 1000 { + for i := range rows { + row := &rows[i] + row.Timestamp -= row.Timestamp % tsTrim + } + } + + uw.runCallback(rows) + putUnmarshalWork(uw) +} + +func getUnmarshalWork() *unmarshalWork { + v := unmarshalWorkPool.Get() + if v == nil { + return &unmarshalWork{} + } + return v.(*unmarshalWork) +} + +func putUnmarshalWork(uw *unmarshalWork) { + uw.reset() + unmarshalWorkPool.Put(uw) +} + +var unmarshalWorkPool sync.Pool diff --git a/lib/protoparser/statsd/stream/streamparser_test.go b/lib/protoparser/statsd/stream/streamparser_test.go new file mode 100644 index 000000000..8800dfd45 --- /dev/null +++ b/lib/protoparser/statsd/stream/streamparser_test.go @@ -0,0 +1,60 @@ +package stream + +import ( + "reflect" + "strings" + "testing" + + "github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/statsd" +) + +func Test_streamContext_Read(t *testing.T) { + f := func(s string, rowsExpected *statsd.Rows) { + t.Helper() + ctx := getStreamContext(strings.NewReader(s)) + if !ctx.Read() { + t.Fatalf("expecting successful read") + } + uw := getUnmarshalWork() + callbackCalls := 0 + uw.ctx = ctx + uw.callback = func(rows []statsd.Row) error { + callbackCalls++ + if len(rows) != len(rowsExpected.Rows) { + t.Fatalf("different len of expected rows;\ngot\n%+v;\nwant\n%+v", rows, rowsExpected.Rows) + } + if !reflect.DeepEqual(rows, rowsExpected.Rows) { + t.Fatalf("unexpected rows;\ngot\n%+v;\nwant\n%+v", rows, rowsExpected.Rows) + } + return nil + } + uw.reqBuf = append(uw.reqBuf[:0], ctx.reqBuf...) + ctx.wg.Add(1) + uw.Unmarshal() + if callbackCalls != 1 { + t.Fatalf("unexpected number of callback calls; got %d; want 1", callbackCalls) + } + } + + // Full line without tags + f("aaa:1123|c", &statsd.Rows{ + Rows: []statsd.Row{{ + Metric: "aaa", + Value: 1123, + Timestamp: int64(fasttime.UnixTimestamp()) * 1000, + }}, + }) + // Full line with tags + f("aaa:1123|c|#x:y", &statsd.Rows{ + Rows: []statsd.Row{{ + Metric: "aaa", + Tags: []statsd.Tag{{ + Key: "x", + Value: "y", + }}, + Value: 1123, + Timestamp: int64(fasttime.UnixTimestamp()) * 1000, + }}, + }) +} diff --git a/lib/streamaggr/dedup_timing_test.go b/lib/streamaggr/dedup_timing_test.go index 2b906fe39..abb1d0451 100644 --- a/lib/streamaggr/dedup_timing_test.go +++ b/lib/streamaggr/dedup_timing_test.go @@ -7,7 +7,6 @@ import ( "time" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" - "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" ) func BenchmarkDedupAggr(b *testing.B) { @@ -50,7 +49,6 @@ func benchmarkDedupAggr(b *testing.B, samplesPerPush int) { } func newBenchSamples(count int) []pushSample { - var lc promutils.LabelsCompressor labels := []prompbmarshal.Label{ { Name: "app", @@ -82,7 +80,7 @@ func newBenchSamples(count int) []pushSample { Name: "app", Value: fmt.Sprintf("instance-%d", i), }) - keyBuf = compressLabels(keyBuf[:0], &lc, labels[:labelsLen], labels[labelsLen:]) + keyBuf = compressLabels(keyBuf[:0], labels[:labelsLen], labels[labelsLen:]) sample.key = string(keyBuf) sample.value = float64(i) } diff --git a/lib/streamaggr/deduplicator.go b/lib/streamaggr/deduplicator.go index 9cc3fee22..a96d765c2 100644 --- a/lib/streamaggr/deduplicator.go +++ b/lib/streamaggr/deduplicator.go @@ -15,7 +15,6 @@ import ( // Deduplicator deduplicates samples per each time series. type Deduplicator struct { da *dedupAggr - lc promutils.LabelsCompressor dropLabels []string @@ -38,8 +37,7 @@ type Deduplicator struct { // MustStop must be called on the returned deduplicator in order to free up occupied resources. func NewDeduplicator(pushFunc PushFunc, dedupInterval time.Duration, dropLabels []string) *Deduplicator { d := &Deduplicator{ - da: newDedupAggr(), - + da: newDedupAggr(), dropLabels: dropLabels, stopCh: make(chan struct{}), @@ -54,13 +52,6 @@ func NewDeduplicator(pushFunc PushFunc, dedupInterval time.Duration, dropLabels return float64(d.da.itemsCount()) }) - _ = ms.NewGauge(`vm_streamaggr_labels_compressor_size_bytes`, func() float64 { - return float64(d.lc.SizeBytes()) - }) - _ = ms.NewGauge(`vm_streamaggr_labels_compressor_items_count`, func() float64 { - return float64(d.lc.ItemsCount()) - }) - d.dedupFlushDuration = ms.GetOrCreateHistogram(`vm_streamaggr_dedup_flush_duration_seconds`) d.dedupFlushTimeouts = ms.GetOrCreateCounter(`vm_streamaggr_dedup_flush_timeouts_total`) @@ -103,7 +94,7 @@ func (d *Deduplicator) Push(tss []prompbmarshal.TimeSeries) { } labels.Sort() - buf = d.lc.Compress(buf[:0], labels.Labels) + buf = lc.Compress(buf[:0], labels.Labels) key := bytesutil.InternBytes(buf) for _, s := range ts.Samples { pss = append(pss, pushSample{ @@ -155,7 +146,7 @@ func (d *Deduplicator) flush(pushFunc PushFunc, dedupInterval time.Duration) { samples := ctx.samples for _, ps := range pss { labelsLen := len(labels) - labels = decompressLabels(labels, &d.lc, ps.key) + labels = decompressLabels(labels, ps.key) samplesLen := len(samples) samples = append(samples, prompbmarshal.Sample{ diff --git a/lib/streamaggr/streamaggr.go b/lib/streamaggr/streamaggr.go index 06d281f69..3e5b72ecd 100644 --- a/lib/streamaggr/streamaggr.go +++ b/lib/streamaggr/streamaggr.go @@ -45,6 +45,19 @@ var supportedOutputs = []string{ "quantiles(phi1, ..., phiN)", } +var ( + // lc contains information about all compressed labels for streaming aggregation + lc promutils.LabelsCompressor + + _ = metrics.NewGauge(`vm_streamaggr_labels_compressor_size_bytes`, func() float64 { + return float64(lc.SizeBytes()) + }) + + _ = metrics.NewGauge(`vm_streamaggr_labels_compressor_items_count`, func() float64 { + return float64(lc.ItemsCount()) + }) +) + // LoadFromFile loads Aggregators from the given path and uses the given pushFunc for pushing the aggregated data. // // opts can contain additional options. If opts is nil, then default options are used. @@ -111,6 +124,13 @@ type Options struct { // // This option can be overridden individually per each aggregation via ignore_old_samples option. IgnoreOldSamples bool + + // IgnoreFirstIntervals sets amount of aggregation intervals to ignore on start. + // + // By default, no intervals will be ignored. + // + // This option can be overridden individually per each aggregation via ignore_first_intervals option. + IgnoreFirstIntervals int } // Config is a configuration for a single stream aggregation. @@ -175,6 +195,9 @@ type Config struct { // IgnoreOldSamples instructs to ignore samples with old timestamps outside the current aggregation interval. IgnoreOldSamples *bool `yaml:"ignore_old_samples,omitempty"` + // IgnoreFirstIntervals sets number of aggregation intervals to be ignored on start. + IgnoreFirstIntervals *int `yaml:"ignore_first_intervals,omitempty"` + // By is an optional list of labels for grouping input series. // // See also Without. @@ -259,21 +282,6 @@ func newAggregatorsFromData(data []byte, pushFunc PushFunc, opts *Options) (*Agg return float64(n) }) - _ = ms.NewGauge(`vm_streamaggr_labels_compressor_size_bytes`, func() float64 { - n := uint64(0) - for _, aggr := range as { - n += aggr.lc.SizeBytes() - } - return float64(n) - }) - _ = ms.NewGauge(`vm_streamaggr_labels_compressor_items_count`, func() float64 { - n := uint64(0) - for _, aggr := range as { - n += aggr.lc.ItemsCount() - } - return float64(n) - }) - metrics.RegisterSet(ms) return &Aggregators{ as: as, @@ -351,9 +359,6 @@ type aggregator struct { // aggrStates contains aggregate states for the given outputs aggrStates []aggrState - // lc is used for compressing series keys before passing them to dedupAggr and aggrState - lc promutils.LabelsCompressor - // minTimestamp is used for ignoring old samples when ignoreOldSamples is set minTimestamp atomic.Int64 @@ -479,6 +484,12 @@ func newAggregator(cfg *Config, pushFunc PushFunc, ms *metrics.Set, opts *Option ignoreOldSamples = *v } + // check cfg.IgnoreFirstIntervals + ignoreFirstIntervals := opts.IgnoreFirstIntervals + if v := cfg.IgnoreFirstIntervals; v != nil { + ignoreFirstIntervals = *v + } + // initialize outputs list if len(cfg.Outputs) == 0 { return nil, fmt.Errorf("`outputs` list must contain at least a single entry from the list %s; "+ @@ -600,14 +611,14 @@ func newAggregator(cfg *Config, pushFunc PushFunc, ms *metrics.Set, opts *Option a.wg.Add(1) go func() { - a.runFlusher(pushFunc, alignFlushToInterval, skipIncompleteFlush, interval, dedupInterval) + a.runFlusher(pushFunc, alignFlushToInterval, skipIncompleteFlush, interval, dedupInterval, ignoreFirstIntervals) a.wg.Done() }() return a, nil } -func (a *aggregator) runFlusher(pushFunc PushFunc, alignFlushToInterval, skipIncompleteFlush bool, interval, dedupInterval time.Duration) { +func (a *aggregator) runFlusher(pushFunc PushFunc, alignFlushToInterval, skipIncompleteFlush bool, interval, dedupInterval time.Duration, ignoreFirstIntervals int) { alignedSleep := func(d time.Duration) { if !alignFlushToInterval { return @@ -642,7 +653,12 @@ func (a *aggregator) runFlusher(pushFunc PushFunc, alignFlushToInterval, skipInc } for tickerWait(t) { - a.flush(pushFunc, interval, true) + pf := pushFunc + if ignoreFirstIntervals > 0 { + pf = nil + ignoreFirstIntervals-- + } + a.flush(pf, interval, true) if alignFlushToInterval { select { @@ -663,13 +679,17 @@ func (a *aggregator) runFlusher(pushFunc PushFunc, alignFlushToInterval, skipInc ct := time.Now() if ct.After(flushDeadline) { + pf := pushFunc + if ignoreFirstIntervals > 0 { + pf = nil + ignoreFirstIntervals-- + } // It is time to flush the aggregated state if alignFlushToInterval && skipIncompleteFlush && !isSkippedFirstFlush { - a.flush(nil, interval, true) + pf = nil isSkippedFirstFlush = true - } else { - a.flush(pushFunc, interval, true) } + a.flush(pf, interval, true) for ct.After(flushDeadline) { flushDeadline = flushDeadline.Add(interval) } @@ -684,7 +704,7 @@ func (a *aggregator) runFlusher(pushFunc PushFunc, alignFlushToInterval, skipInc } } - if !skipIncompleteFlush { + if !skipIncompleteFlush && ignoreFirstIntervals == 0 { a.dedupFlush(dedupInterval) a.flush(pushFunc, interval, true) } @@ -797,7 +817,7 @@ func (a *aggregator) Push(tss []prompbmarshal.TimeSeries, matchIdxs []byte) { outputLabels.Labels = append(outputLabels.Labels, labels.Labels...) } - buf = compressLabels(buf[:0], &a.lc, inputLabels.Labels, outputLabels.Labels) + buf = compressLabels(buf[:0], inputLabels.Labels, outputLabels.Labels) key := bytesutil.InternBytes(buf) for _, sample := range ts.Samples { if math.IsNaN(sample.Value) { @@ -825,7 +845,7 @@ func (a *aggregator) Push(tss []prompbmarshal.TimeSeries, matchIdxs []byte) { } } -func compressLabels(dst []byte, lc *promutils.LabelsCompressor, inputLabels, outputLabels []prompbmarshal.Label) []byte { +func compressLabels(dst []byte, inputLabels, outputLabels []prompbmarshal.Label) []byte { bb := bbPool.Get() bb.B = lc.Compress(bb.B, inputLabels) dst = encoding.MarshalVarUint64(dst, uint64(len(bb.B))) @@ -835,7 +855,7 @@ func compressLabels(dst []byte, lc *promutils.LabelsCompressor, inputLabels, out return dst } -func decompressLabels(dst []prompbmarshal.Label, lc *promutils.LabelsCompressor, key string) []prompbmarshal.Label { +func decompressLabels(dst []prompbmarshal.Label, key string) []prompbmarshal.Label { return lc.Decompress(dst, bytesutil.ToUnsafeBytes(key)) } @@ -1010,7 +1030,7 @@ func (ctx *flushCtx) flushSeries() { func (ctx *flushCtx) appendSeries(key, suffix string, timestamp int64, value float64) { labelsLen := len(ctx.labels) samplesLen := len(ctx.samples) - ctx.labels = decompressLabels(ctx.labels, &ctx.a.lc, key) + ctx.labels = decompressLabels(ctx.labels, key) if !ctx.a.keepMetricNames { ctx.labels = addMetricSuffix(ctx.labels, labelsLen, ctx.a.suffix, suffix) } @@ -1033,7 +1053,7 @@ func (ctx *flushCtx) appendSeries(key, suffix string, timestamp int64, value flo func (ctx *flushCtx) appendSeriesWithExtraLabel(key, suffix string, timestamp int64, value float64, extraName, extraValue string) { labelsLen := len(ctx.labels) samplesLen := len(ctx.samples) - ctx.labels = decompressLabels(ctx.labels, &ctx.a.lc, key) + ctx.labels = decompressLabels(ctx.labels, key) if !ctx.a.keepMetricNames { ctx.labels = addMetricSuffix(ctx.labels, labelsLen, ctx.a.suffix, suffix) } diff --git a/lib/streamaggr/streamaggr_test.go b/lib/streamaggr/streamaggr_test.go index 5a079979b..ae234bb76 100644 --- a/lib/streamaggr/streamaggr_test.go +++ b/lib/streamaggr/streamaggr_test.go @@ -199,6 +199,14 @@ func TestAggregatorsEqual(t *testing.T) { interval: 5m flush_on_shutdown: false `, false) + f(` +- outputs: [total] + interval: 5m + ignore_first_intervals: 2 +`, ` +- outputs: [total] + interval: 5m + ignore_first_intervals: 4`, false) } func TestAggregatorsSuccess(t *testing.T) { @@ -510,8 +518,8 @@ foo:1m_total 0 `, ` foo 123 bar{baz="qwe"} 4.34 -`, `bar:1m_total{baz="qwe"} 0 -foo:1m_total 0 +`, `bar:1m_total_prometheus{baz="qwe"} 0 +foo:1m_total_prometheus 0 `, "11") // total output for repeated series @@ -546,10 +554,10 @@ foo{baz="qwe"} -5 bar{baz="qwer"} 343 bar{baz="qwer"} 344 foo{baz="qwe"} 10 -`, `bar:1m_total{baz="qwe"} 5.02 -bar:1m_total{baz="qwer"} 1 -foo:1m_total 0 -foo:1m_total{baz="qwe"} 15 +`, `bar:1m_total_prometheus{baz="qwe"} 5.02 +bar:1m_total_prometheus{baz="qwer"} 1 +foo:1m_total_prometheus 0 +foo:1m_total_prometheus{baz="qwe"} 15 `, "11111111") // total output for repeated series with group by __name__ @@ -584,8 +592,8 @@ foo{baz="qwe"} -5 bar{baz="qwer"} 343 bar{baz="qwer"} 344 foo{baz="qwe"} 10 -`, `bar:1m_total 6.02 -foo:1m_total 15 +`, `bar:1m_total_prometheus 6.02 +foo:1m_total_prometheus 15 `, "11111111") // increase output for non-repeated series @@ -606,8 +614,8 @@ foo:1m_increase 0 `, ` foo 123 bar{baz="qwe"} 4.34 -`, `bar:1m_increase{baz="qwe"} 0 -foo:1m_increase 0 +`, `bar:1m_increase_prometheus{baz="qwe"} 0 +foo:1m_increase_prometheus 0 `, "11") // increase output for repeated series @@ -642,10 +650,10 @@ foo{baz="qwe"} -5 bar{baz="qwer"} 343 bar{baz="qwer"} 344 foo{baz="qwe"} 10 -`, `bar:1m_increase{baz="qwe"} 5.02 -bar:1m_increase{baz="qwer"} 1 -foo:1m_increase 0 -foo:1m_increase{baz="qwe"} 15 +`, `bar:1m_increase_prometheus{baz="qwe"} 5.02 +bar:1m_increase_prometheus{baz="qwer"} 1 +foo:1m_increase_prometheus 0 +foo:1m_increase_prometheus{baz="qwe"} 15 `, "11111111") // multiple aggregate configs diff --git a/lib/streamaggr/total.go b/lib/streamaggr/total.go index e0b28d437..a0576245e 100644 --- a/lib/streamaggr/total.go +++ b/lib/streamaggr/total.go @@ -53,6 +53,9 @@ func newTotalAggrState(stalenessInterval time.Duration, resetTotalOnFlush, keepF if resetTotalOnFlush { suffix = "increase" } + if !keepFirstSample { + suffix += "_prometheus" + } return &totalAggrState{ suffix: suffix, resetTotalOnFlush: resetTotalOnFlush,